1
0

[HUDI-3730] Improve meta sync class design and hierarchies (#5854)

* [HUDI-3730] Improve meta sync class design and hierarchies (#5754)
* Implements class design proposed in RFC-55

Co-authored-by: jian.feng <fengjian428@gmial.com>
Co-authored-by: jian.feng <jian.feng@shopee.com>
This commit is contained in:
Shiyan Xu
2022-07-03 04:17:25 -05:00
committed by GitHub
parent c00ea84985
commit c0e1587966
86 changed files with 2977 additions and 2877 deletions

View File

@@ -21,9 +21,8 @@ package org.apache.hudi.sync.datahub;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
import org.apache.hudi.sync.common.HoodieSyncClient;
import org.apache.hudi.sync.common.HoodieSyncException;
import org.apache.hudi.sync.datahub.config.DataHubSyncConfig;
@@ -51,8 +50,6 @@ import datahub.client.rest.RestEmitter;
import datahub.event.MetadataChangeProposalWrapper;
import org.apache.avro.AvroTypeException;
import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.parquet.schema.MessageType;
import java.util.Collections;
@@ -60,40 +57,15 @@ import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class DataHubSyncClient extends AbstractSyncHoodieClient {
public class DataHubSyncClient extends HoodieSyncClient {
private final HoodieTimeline activeTimeline;
private final DataHubSyncConfig syncConfig;
private final Configuration hadoopConf;
protected final DataHubSyncConfig config;
private final DatasetUrn datasetUrn;
public DataHubSyncClient(DataHubSyncConfig syncConfig, Configuration hadoopConf, FileSystem fs) {
super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata, false, fs);
this.syncConfig = syncConfig;
this.hadoopConf = hadoopConf;
this.datasetUrn = syncConfig.datasetIdentifier.getDatasetUrn();
this.activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
}
@Override
public void createTable(String tableName,
MessageType storageSchema,
String inputFormatClass,
String outputFormatClass,
String serdeClass,
Map<String, String> serdeProperties,
Map<String, String> tableProperties) {
throw new UnsupportedOperationException("Not supported: `createTable`");
}
@Override
public boolean doesTableExist(String tableName) {
return tableExists(tableName);
}
@Override
public boolean tableExists(String tableName) {
throw new UnsupportedOperationException("Not supported: `tableExists`");
public DataHubSyncClient(DataHubSyncConfig config) {
super(config);
this.config = config;
this.datasetUrn = config.datasetIdentifier.getDatasetUrn();
}
@Override
@@ -103,37 +75,7 @@ public class DataHubSyncClient extends AbstractSyncHoodieClient {
@Override
public void updateLastCommitTimeSynced(String tableName) {
updateTableProperties(tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, activeTimeline.lastInstant().get().getTimestamp()));
}
@Override
public Option<String> getLastReplicatedTime(String tableName) {
throw new UnsupportedOperationException("Not supported: `getLastReplicatedTime`");
}
@Override
public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) {
throw new UnsupportedOperationException("Not supported: `updateLastReplicatedTimeStamp`");
}
@Override
public void deleteLastReplicatedTimeStamp(String tableName) {
throw new UnsupportedOperationException("Not supported: `deleteLastReplicatedTimeStamp`");
}
@Override
public void addPartitionsToTable(String tableName, List<String> partitionsToAdd) {
throw new UnsupportedOperationException("Not supported: `addPartitionsToTable`");
}
@Override
public void updatePartitionsToTable(String tableName, List<String> changedPartitions) {
throw new UnsupportedOperationException("Not supported: `updatePartitionsToTable`");
}
@Override
public void dropPartitions(String tableName, List<String> partitionsToDrop) {
throw new UnsupportedOperationException("Not supported: `dropPartitions`");
updateTableProperties(tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, getActiveTimeline().lastInstant().get().getTimestamp()));
}
@Override
@@ -145,14 +87,15 @@ public class DataHubSyncClient extends AbstractSyncHoodieClient {
.aspect(new DatasetProperties().setCustomProperties(new StringMap(tableProperties)))
.build();
try (RestEmitter emitter = syncConfig.getRestEmitter()) {
try (RestEmitter emitter = config.getRestEmitter()) {
emitter.emit(propertiesChangeProposal, null).get();
} catch (Exception e) {
throw new HoodieDataHubSyncException("Fail to change properties for Dataset " + datasetUrn + ": " + tableProperties, e);
}
}
public void updateTableDefinition(String tableName) {
@Override
public void updateTableSchema(String tableName, MessageType schema) {
Schema avroSchema = getAvroSchemaWithoutMetadataFields(metaClient);
List<SchemaField> fields = avroSchema.getFields().stream().map(f -> new SchemaField()
.setFieldPath(f.name())
@@ -175,7 +118,7 @@ public class DataHubSyncClient extends AbstractSyncHoodieClient {
.setFields(new SchemaFieldArray(fields)))
.build();
try (RestEmitter emitter = syncConfig.getRestEmitter()) {
try (RestEmitter emitter = config.getRestEmitter()) {
emitter.emit(schemaChangeProposal, null).get();
} catch (Exception e) {
throw new HoodieDataHubSyncException("Fail to change schema for Dataset " + datasetUrn, e);
@@ -183,7 +126,7 @@ public class DataHubSyncClient extends AbstractSyncHoodieClient {
}
@Override
public Map<String, String> getTableSchema(String tableName) {
public Map<String, String> getMetastoreSchema(String tableName) {
throw new UnsupportedOperationException("Not supported: `getTableSchema`");
}

View File

@@ -19,14 +19,14 @@
package org.apache.hudi.sync.datahub;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.sync.common.AbstractSyncTool;
import org.apache.hudi.sync.common.HoodieSyncTool;
import org.apache.hudi.sync.datahub.config.DataHubSyncConfig;
import com.beust.jcommander.JCommander;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import java.util.Properties;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
/**
* To sync with DataHub via REST APIs.
@@ -34,17 +34,13 @@ import org.apache.hadoop.fs.FileSystem;
* @Experimental
* @see <a href="https://datahubproject.io/">https://datahubproject.io/</a>
*/
public class DataHubSyncTool extends AbstractSyncTool {
public class DataHubSyncTool extends HoodieSyncTool {
private final DataHubSyncConfig config;
protected final DataHubSyncConfig config;
public DataHubSyncTool(TypedProperties props, Configuration conf, FileSystem fs) {
this(new DataHubSyncConfig(props), conf, fs);
}
public DataHubSyncTool(DataHubSyncConfig config, Configuration conf, FileSystem fs) {
super(config.getProps(), conf, fs);
this.config = config;
public DataHubSyncTool(Properties props) {
super(props);
this.config = new DataHubSyncConfig(props);
}
/**
@@ -55,20 +51,20 @@ public class DataHubSyncTool extends AbstractSyncTool {
*/
@Override
public void syncHoodieTable() {
try (DataHubSyncClient syncClient = new DataHubSyncClient(config, conf, fs)) {
syncClient.updateTableDefinition(config.tableName);
syncClient.updateLastCommitTimeSynced(config.tableName);
try (DataHubSyncClient syncClient = new DataHubSyncClient(config)) {
syncClient.updateTableSchema(config.getString(META_SYNC_TABLE_NAME), null);
syncClient.updateLastCommitTimeSynced(config.getString(META_SYNC_TABLE_NAME));
}
}
public static void main(String[] args) {
final DataHubSyncConfig cfg = new DataHubSyncConfig();
JCommander cmd = new JCommander(cfg, null, args);
if (cfg.help || args.length == 0) {
final DataHubSyncConfig.DataHubSyncConfigParams params = new DataHubSyncConfig.DataHubSyncConfigParams();
JCommander cmd = JCommander.newBuilder().addObject(params).build();
cmd.parse(args);
if (params.isHelp()) {
cmd.usage();
System.exit(1);
System.exit(0);
}
FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
new DataHubSyncTool(cfg, fs.getConf(), fs).syncHoodieTable();
new DataHubSyncTool(params.toProps()).syncHoodieTable();
}
}

View File

@@ -25,8 +25,11 @@ import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.sync.common.HoodieSyncConfig;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.ParametersDelegate;
import datahub.client.rest.RestEmitter;
import java.util.Properties;
public class DataHubSyncConfig extends HoodieSyncConfig {
public static final ConfigProperty<String> META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS = ConfigProperty
@@ -49,45 +52,52 @@ public class DataHubSyncConfig extends HoodieSyncConfig {
.noDefaultValue()
.withDocumentation("Pluggable class to supply a DataHub REST emitter to connect to the DataHub instance. This overwrites other emitter configs.");
@Parameter(names = {"--identifier-class"}, description = "Pluggable class to help provide info to identify a DataHub Dataset.")
public String identifierClass;
@Parameter(names = {"--emitter-server"}, description = "Server URL of the DataHub instance.")
public String emitterServer;
@Parameter(names = {"--emitter-token"}, description = "Auth token to connect to the DataHub instance.")
public String emitterToken;
@Parameter(names = {"--emitter-supplier-class"}, description = "Pluggable class to supply a DataHub REST emitter to connect to the DataHub instance. This overwrites other emitter configs.")
public String emitterSupplierClass;
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
public final HoodieDataHubDatasetIdentifier datasetIdentifier;
public DataHubSyncConfig() {
this(new TypedProperties());
}
public DataHubSyncConfig(TypedProperties props) {
public DataHubSyncConfig(Properties props) {
super(props);
identifierClass = getStringOrDefault(META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS);
emitterServer = getStringOrDefault(META_SYNC_DATAHUB_EMITTER_SERVER, null);
emitterToken = getStringOrDefault(META_SYNC_DATAHUB_EMITTER_TOKEN, null);
emitterSupplierClass = getStringOrDefault(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS, null);
datasetIdentifier = (HoodieDataHubDatasetIdentifier) ReflectionUtils
.loadClass(identifierClass, new Class<?>[] {TypedProperties.class}, props);
String identifierClass = getStringOrDefault(META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS);
datasetIdentifier = (HoodieDataHubDatasetIdentifier) ReflectionUtils.loadClass(identifierClass, new Class<?>[] {Properties.class}, props);
}
public RestEmitter getRestEmitter() {
if (emitterSupplierClass != null) {
return ((DataHubEmitterSupplier) ReflectionUtils.loadClass(emitterSupplierClass)).get();
} else if (emitterServer != null) {
return RestEmitter.create(b -> b.server(emitterServer).token(emitterToken));
if (contains(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS)) {
return ((DataHubEmitterSupplier) ReflectionUtils.loadClass(getString(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS))).get();
} else if (contains(META_SYNC_DATAHUB_EMITTER_SERVER)) {
return RestEmitter.create(b -> b.server(getString(META_SYNC_DATAHUB_EMITTER_SERVER)).token(getStringOrDefault(META_SYNC_DATAHUB_EMITTER_TOKEN, null)));
} else {
return RestEmitter.createWithDefaults();
}
}
public static class DataHubSyncConfigParams {
@ParametersDelegate()
public final HoodieSyncConfigParams hoodieSyncConfigParams = new HoodieSyncConfigParams();
@Parameter(names = {"--identifier-class"}, description = "Pluggable class to help provide info to identify a DataHub Dataset.")
public String identifierClass;
@Parameter(names = {"--emitter-server"}, description = "Server URL of the DataHub instance.")
public String emitterServer;
@Parameter(names = {"--emitter-token"}, description = "Auth token to connect to the DataHub instance.")
public String emitterToken;
@Parameter(names = {"--emitter-supplier-class"}, description = "Pluggable class to supply a DataHub REST emitter to connect to the DataHub instance. This overwrites other emitter configs.")
public String emitterSupplierClass;
public boolean isHelp() {
return hoodieSyncConfigParams.isHelp();
}
public Properties toProps() {
final TypedProperties props = hoodieSyncConfigParams.toProps();
props.setPropertyIfNonNull(META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS.key(), identifierClass);
props.setPropertyIfNonNull(META_SYNC_DATAHUB_EMITTER_SERVER.key(), emitterServer);
props.setPropertyIfNonNull(META_SYNC_DATAHUB_EMITTER_TOKEN.key(), emitterToken);
props.setPropertyIfNonNull(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS.key(), emitterSupplierClass);
return props;
}
}
}

View File

@@ -19,12 +19,15 @@
package org.apache.hudi.sync.datahub.config;
import org.apache.hudi.common.config.TypedProperties;
import com.linkedin.common.FabricType;
import com.linkedin.common.urn.DataPlatformUrn;
import com.linkedin.common.urn.DatasetUrn;
import java.util.Properties;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
/**
* Construct and provide the default {@link DatasetUrn} to identify the Dataset on DataHub.
* <p>
@@ -34,15 +37,15 @@ public class HoodieDataHubDatasetIdentifier {
public static final String DEFAULT_HOODIE_DATAHUB_PLATFORM_NAME = "hudi";
protected final TypedProperties props;
protected final Properties props;
public HoodieDataHubDatasetIdentifier(TypedProperties props) {
public HoodieDataHubDatasetIdentifier(Properties props) {
this.props = props;
}
public DatasetUrn getDatasetUrn() {
DataPlatformUrn dataPlatformUrn = new DataPlatformUrn(DEFAULT_HOODIE_DATAHUB_PLATFORM_NAME);
DataHubSyncConfig config = new DataHubSyncConfig(props);
return new DatasetUrn(dataPlatformUrn, String.format("%s.%s", config.databaseName, config.tableName), FabricType.DEV);
return new DatasetUrn(dataPlatformUrn, String.format("%s.%s", config.getString(META_SYNC_DATABASE_NAME), config.getString(META_SYNC_TABLE_NAME)), FabricType.DEV);
}
}

View File

@@ -19,22 +19,32 @@
package org.apache.hudi.sync.datahub.config;
import org.apache.hudi.common.config.TypedProperties;
import com.linkedin.common.FabricType;
import com.linkedin.common.urn.DatasetUrn;
import datahub.client.rest.RestEmitter;
import org.junit.jupiter.api.Test;
import java.net.URISyntaxException;
import java.util.Properties;
import static org.apache.hudi.sync.datahub.config.DataHubSyncConfig.META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS;
import static org.apache.hudi.sync.datahub.config.DataHubSyncConfig.META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
class TestDataHubSyncConfig {
@Test
void testGetEmitterFromSupplier() {
Properties props = new Properties();
props.setProperty(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS.key(), DummySupplier.class.getName());
DataHubSyncConfig syncConfig = new DataHubSyncConfig(props);
assertNotNull(syncConfig.getRestEmitter());
}
@Test
void testInstantiationWithProps() {
TypedProperties props = new TypedProperties();
Properties props = new Properties();
props.setProperty(META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS.key(), DummyIdentifier.class.getName());
DataHubSyncConfig syncConfig = new DataHubSyncConfig(props);
DatasetUrn datasetUrn = syncConfig.datasetIdentifier.getDatasetUrn();
@@ -43,9 +53,17 @@ class TestDataHubSyncConfig {
assertEquals(FabricType.PROD, datasetUrn.getOriginEntity());
}
public static class DummySupplier implements DataHubEmitterSupplier {
@Override
public RestEmitter get() {
return RestEmitter.createWithDefaults();
}
}
public static class DummyIdentifier extends HoodieDataHubDatasetIdentifier {
public DummyIdentifier(TypedProperties props) {
public DummyIdentifier(Properties props) {
super(props);
}