1
0

[HUDI-1636] Support Builder Pattern To Build Table Properties For HoodieTableConfig (#2596)

This commit is contained in:
pengzhiwei
2021-03-05 14:10:27 +08:00
committed by GitHub
parent f53bca404f
commit bc883db5de
21 changed files with 341 additions and 197 deletions

View File

@@ -22,7 +22,6 @@ import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper; import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader; import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.exception.TableNotFoundException;
@@ -106,10 +105,13 @@ public class TableCommand implements CommandMarker {
throw new IllegalStateException("Table already existing in path : " + path); throw new IllegalStateException("Table already existing in path : " + path);
} }
final HoodieTableType tableType = HoodieTableType.valueOf(tableTypeStr); HoodieTableMetaClient.withPropertyBuilder()
HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, archiveFolder, .setTableType(tableTypeStr)
payloadClass, layoutVersion); .setTableName(name)
.setArchiveLogFolder(archiveFolder)
.setPayloadClassName(payloadClass)
.setTimelineLayoutVersion(layoutVersion)
.initTable(HoodieCLI.conf, path);
// Now connect to ensure loading works // Now connect to ensure loading works
return connect(path, layoutVersion, false, 0, 0, 0); return connect(path, layoutVersion, false, 0, 0, 0);
} }

View File

@@ -288,9 +288,14 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
String createInstantTime = latestInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); String createInstantTime = latestInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP);
LOG.info("Creating a new metadata table in " + metadataWriteConfig.getBasePath() + " at instant " + createInstantTime); LOG.info("Creating a new metadata table in " + metadataWriteConfig.getBasePath() + " at instant " + createInstantTime);
HoodieTableMetaClient.initTableType(hadoopConf.get(), metadataWriteConfig.getBasePath(), HoodieTableMetaClient.withPropertyBuilder()
HoodieTableType.MERGE_ON_READ, tableName, "archived", HoodieMetadataPayload.class.getName(), .setTableType(HoodieTableType.MERGE_ON_READ)
HoodieFileFormat.HFILE.toString()); .setTableName(tableName)
.setArchiveLogFolder("archived")
.setPayloadClassName(HoodieMetadataPayload.class.getName())
.setBaseFileFormat(HoodieFileFormat.HFILE.toString())
.initTable(hadoopConf.get(), metadataWriteConfig.getBasePath());
initTableMetadata(); initTableMetadata();
// List all partitions in the basePath of the containing dataset // List all partitions in the basePath of the containing dataset

View File

@@ -349,9 +349,11 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
.withProps(config.getProps()).withTimelineLayoutVersion( .withProps(config.getProps()).withTimelineLayoutVersion(
VERSION_0).build(); VERSION_0).build();
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(), HoodieTableMetaClient.withPropertyBuilder()
metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(), .fromMetaClient(metaClient)
metaClient.getTableConfig().getPayloadClass(), VERSION_0); .setTimelineLayoutVersion(VERSION_0)
.initTable(metaClient.getHadoopConf(), metaClient.getBasePath());
SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig);
// Write 1 (only inserts) // Write 1 (only inserts)
@@ -493,10 +495,11 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
HoodieWriteConfig hoodieWriteConfig = getConfigBuilder() HoodieWriteConfig hoodieWriteConfig = getConfigBuilder()
.withProps(config.getProps()).withMergeAllowDuplicateOnInserts(true).withTimelineLayoutVersion( .withProps(config.getProps()).withMergeAllowDuplicateOnInserts(true).withTimelineLayoutVersion(
VERSION_0).build(); VERSION_0).build();
HoodieTableMetaClient.withPropertyBuilder()
.fromMetaClient(metaClient)
.setTimelineLayoutVersion(VERSION_0)
.initTable(metaClient.getHadoopConf(), metaClient.getBasePath());
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(),
metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(),
metaClient.getTableConfig().getPayloadClass(), VERSION_0);
SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig);
// Write 1 (only inserts) // Write 1 (only inserts)
@@ -629,9 +632,11 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
.withBloomIndexUpdatePartitionPath(true) .withBloomIndexUpdatePartitionPath(true)
.withGlobalSimpleIndexUpdatePartitionPath(true) .withGlobalSimpleIndexUpdatePartitionPath(true)
.build()).withTimelineLayoutVersion(VERSION_0).build(); .build()).withTimelineLayoutVersion(VERSION_0).build();
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(),
metaClient.getTableType(), metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(), HoodieTableMetaClient.withPropertyBuilder()
metaClient.getTableConfig().getPayloadClass(), VERSION_0); .fromMetaClient(metaClient)
.setTimelineLayoutVersion(VERSION_0)
.initTable(metaClient.getHadoopConf(), metaClient.getBasePath());
// Set rollback to LAZY so no inflights are deleted // Set rollback to LAZY so no inflights are deleted
hoodieWriteConfig.getProps().put(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP, hoodieWriteConfig.getProps().put(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP,
HoodieFailedWritesCleaningPolicy.LAZY.name()); HoodieFailedWritesCleaningPolicy.LAZY.name());

View File

@@ -75,8 +75,11 @@ public class TestMultiFS extends HoodieClientTestHarness {
@Test @Test
public void readLocalWriteHDFS() throws Exception { public void readLocalWriteHDFS() throws Exception {
// Initialize table and filesystem // Initialize table and filesystem
HoodieTableMetaClient.initTableType(hadoopConf, dfsBasePath, HoodieTableType.valueOf(tableType), HoodieTableMetaClient.withPropertyBuilder()
tableName, HoodieAvroPayload.class.getName()); .setTableType(tableType)
.setTableName(tableName)
.setPayloadClass(HoodieAvroPayload.class)
.initTable(hadoopConf, dfsBasePath);
// Create write client to write some records in // Create write client to write some records in
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath); HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
@@ -100,8 +103,11 @@ public class TestMultiFS extends HoodieClientTestHarness {
assertEquals(readRecords.count(), records.size(), "Should contain 100 records"); assertEquals(readRecords.count(), records.size(), "Should contain 100 records");
// Write to local // Write to local
HoodieTableMetaClient.initTableType(hadoopConf, tablePath, HoodieTableType.valueOf(tableType), HoodieTableMetaClient.withPropertyBuilder()
tableName, HoodieAvroPayload.class.getName()); .setTableType(tableType)
.setTableName(tableName)
.setPayloadClass(HoodieAvroPayload.class)
.initTable(hadoopConf, tablePath);
String writeCommitTime = localWriteClient.startCommit(); String writeCommitTime = localWriteClient.startCommit();
LOG.info("Starting write commit " + writeCommitTime); LOG.info("Starting write commit " + writeCommitTime);

View File

@@ -150,9 +150,11 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase {
tableType = HoodieTableType.MERGE_ON_READ; tableType = HoodieTableType.MERGE_ON_READ;
// Create the table // Create the table
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), HoodieTableMetaClient.withPropertyBuilder()
HoodieTableType.MERGE_ON_READ, metaClient.getTableConfig().getTableName(), .fromMetaClient(metaClient)
metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1); .setTableType(HoodieTableType.MERGE_ON_READ)
.setTimelineLayoutVersion(VERSION_1)
.initTable(metaClient.getHadoopConf(), metaClient.getBasePath());
HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA);
SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig);
@@ -295,9 +297,10 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase {
@Test @Test
public void testCopyOnWriteTable() throws Exception { public void testCopyOnWriteTable() throws Exception {
// Create the table // Create the table
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), HoodieTableMetaClient.withPropertyBuilder()
HoodieTableType.COPY_ON_WRITE, metaClient.getTableConfig().getTableName(), .fromMetaClient(metaClient)
metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1); .setTimelineLayoutVersion(VERSION_1)
.initTable(metaClient.getHadoopConf(), metaClient.getBasePath());
HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA);
SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig);

View File

@@ -24,7 +24,6 @@ import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.testutils.minicluster.HdfsTestService; import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieWriteConfig;
@@ -117,10 +116,13 @@ public class FunctionalTestHarness implements SparkProvider, DFSProvider, Hoodie
@Override @Override
public HoodieTableMetaClient getHoodieMetaClient(Configuration hadoopConf, String basePath, Properties props) throws IOException { public HoodieTableMetaClient getHoodieMetaClient(Configuration hadoopConf, String basePath, Properties props) throws IOException {
props.putIfAbsent(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, PARQUET.toString()); props = HoodieTableMetaClient.withPropertyBuilder()
props.putIfAbsent(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME); .setTableName(RAW_TRIPS_TEST_NAME)
props.putIfAbsent(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, COPY_ON_WRITE.name()); .setTableType(COPY_ON_WRITE)
props.putIfAbsent(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName()); .setPayloadClass(HoodieAvroPayload.class)
.setBaseFileFormat(PARQUET.toString())
.fromProperties(props)
.build();
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, props); return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, props);
} }

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.fs.FailSafeConsistencyGuard; import org.apache.hudi.common.fs.FailSafeConsistencyGuard;
import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.fs.NoOpConsistencyGuard; import org.apache.hudi.common.fs.NoOpConsistencyGuard;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
@@ -309,112 +310,6 @@ public class HoodieTableMetaClient implements Serializable {
return archivedTimeline; return archivedTimeline;
} }
/**
* Helper method to initialize a table, with given basePath, tableType, name, archiveFolder, payloadClass and
* base file format.
*/
public static HoodieTableMetaClient initTableTypeWithBootstrap(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
String baseFileFormat, String preCombineField, String bootstrapIndexClass,
String bootstrapBasePath) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName,
archiveLogFolder, payloadClassName, null,
baseFileFormat, preCombineField, bootstrapIndexClass, bootstrapBasePath);
}
public static HoodieTableMetaClient initTableTypeWithBootstrap(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
String baseFileFormat, String bootstrapIndexClass,
String bootstrapBasePath) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName,
archiveLogFolder, payloadClassName, null,
baseFileFormat, null, bootstrapIndexClass, bootstrapBasePath);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
String baseFileFormat, String preCombineField) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName,
archiveLogFolder, payloadClassName, null, baseFileFormat, preCombineField,
null, null);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
String baseFileFormat) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName,
archiveLogFolder, payloadClassName, null, baseFileFormat, null,
null, null);
}
/**
* Used primarily by tests, examples.
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String payloadClassName, String preCombineField) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName,
null, preCombineField);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String payloadClassName) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName,
null, (String) null);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
String preCombineField, Integer timelineLayoutVersion) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, archiveLogFolder, payloadClassName,
timelineLayoutVersion, null, preCombineField, null, null);
}
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
Integer timelineLayoutVersion) throws IOException {
return initTableType(hadoopConf, basePath, tableType, tableName, archiveLogFolder, payloadClassName,
timelineLayoutVersion, null, null, null, null);
}
private static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, HoodieTableType tableType,
String tableName, String archiveLogFolder, String payloadClassName,
Integer timelineLayoutVersion,
String baseFileFormat, String preCombineField,
String bootstrapIndexClass, String bootstrapBasePath) throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP_NAME, String.valueOf(HoodieTableVersion.current().versionCode()));
if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) {
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
}
if (null != archiveLogFolder) {
properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
}
if (null != timelineLayoutVersion) {
properties.put(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION, String.valueOf(timelineLayoutVersion));
}
if (null != baseFileFormat) {
properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, baseFileFormat.toUpperCase());
}
if (null != bootstrapIndexClass) {
properties.put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, bootstrapIndexClass);
}
if (null != bootstrapBasePath) {
properties.put(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH, bootstrapBasePath);
}
if (null != preCombineField) {
properties.put(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD, preCombineField);
}
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties);
}
/** /**
* Helper method to initialize a given path as a hoodie table with configs passed in as as Properties. * Helper method to initialize a given path as a hoodie table with configs passed in as as Properties.
* *
@@ -688,4 +583,173 @@ public class HoodieTableMetaClient implements Serializable {
} }
} }
public static PropertyBuilder withPropertyBuilder() {
return new PropertyBuilder();
}
public static class PropertyBuilder {
private HoodieTableType tableType;
private String tableName;
private String archiveLogFolder;
private String payloadClassName;
private Integer timelineLayoutVersion;
private String baseFileFormat;
private String preCombineField;
private String bootstrapIndexClass;
private String bootstrapBasePath;
private PropertyBuilder() {
}
public PropertyBuilder setTableType(HoodieTableType tableType) {
this.tableType = tableType;
return this;
}
public PropertyBuilder setTableType(String tableType) {
return setTableType(HoodieTableType.valueOf(tableType));
}
public PropertyBuilder setTableName(String tableName) {
this.tableName = tableName;
return this;
}
public PropertyBuilder setArchiveLogFolder(String archiveLogFolder) {
this.archiveLogFolder = archiveLogFolder;
return this;
}
public PropertyBuilder setPayloadClassName(String payloadClassName) {
this.payloadClassName = payloadClassName;
return this;
}
public PropertyBuilder setPayloadClass(Class<? extends HoodieRecordPayload> payloadClass) {
return setPayloadClassName(payloadClass.getName());
}
public PropertyBuilder setTimelineLayoutVersion(Integer timelineLayoutVersion) {
this.timelineLayoutVersion = timelineLayoutVersion;
return this;
}
public PropertyBuilder setBaseFileFormat(String baseFileFormat) {
this.baseFileFormat = baseFileFormat;
return this;
}
public PropertyBuilder setPreCombineField(String preCombineField) {
this.preCombineField = preCombineField;
return this;
}
public PropertyBuilder setBootstrapIndexClass(String bootstrapIndexClass) {
this.bootstrapIndexClass = bootstrapIndexClass;
return this;
}
public PropertyBuilder setBootstrapBasePath(String bootstrapBasePath) {
this.bootstrapBasePath = bootstrapBasePath;
return this;
}
public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) {
return setTableType(metaClient.getTableType())
.setTableName(metaClient.getTableConfig().getTableName())
.setArchiveLogFolder(metaClient.getArchivePath())
.setPayloadClassName(metaClient.getTableConfig().getPayloadClass());
}
public PropertyBuilder fromProperties(Properties properties) {
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME)) {
setTableName(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME)) {
setTableType(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) {
setArchiveLogFolder(
properties.getProperty(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
setPayloadClassName(
properties.getProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION)) {
setTimelineLayoutVersion(Integer
.parseInt(properties.getProperty(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION)));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME)) {
setBaseFileFormat(
properties.getProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME)) {
setBootstrapIndexClass(
properties.getProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH)) {
setBootstrapBasePath(properties.getProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD)) {
setPreCombineField(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD));
}
return this;
}
public Properties build() {
ValidationUtils.checkArgument(tableType != null, "tableType is null");
ValidationUtils.checkArgument(tableName != null, "tableName is null");
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP_NAME,
String.valueOf(HoodieTableVersion.current().versionCode()));
if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) {
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
}
if (null != archiveLogFolder) {
properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
}
if (null != timelineLayoutVersion) {
properties.put(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION,
String.valueOf(timelineLayoutVersion));
}
if (null != baseFileFormat) {
properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME,
baseFileFormat.toUpperCase());
}
if (null != bootstrapIndexClass) {
properties
.put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, bootstrapIndexClass);
}
if (null != bootstrapBasePath) {
properties.put(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH, bootstrapBasePath);
}
if (null != preCombineField) {
properties.put(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD, preCombineField);
}
return properties;
}
/**
* Init Table with the properties build by this builder.
*
* @param configuration The hadoop config.
* @param basePath The base path for hoodie table.
*/
public HoodieTableMetaClient initTable(Configuration configuration, String basePath)
throws IOException {
return HoodieTableMetaClient.initTableAndGetMetaClient(configuration, basePath, build());
}
}
} }

View File

@@ -109,9 +109,11 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
"Check the instants stream"); "Check the instants stream");
// Backwards compatibility testing for reading compaction plans // Backwards compatibility testing for reading compaction plans
metaClient = HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient = HoodieTableMetaClient.withPropertyBuilder()
metaClient.getBasePath(), metaClient.getTableType(), metaClient.getTableConfig().getTableName(), .fromMetaClient(metaClient)
metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_0); .setTimelineLayoutVersion(VERSION_0)
.initTable(metaClient.getHadoopConf(), metaClient.getBasePath());
HoodieInstant instant6 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "9"); HoodieInstant instant6 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "9");
byte[] dummy = new byte[5]; byte[] dummy = new byte[5];
HoodieActiveTimeline oldTimeline = new HoodieActiveTimeline( HoodieActiveTimeline oldTimeline = new HoodieActiveTimeline(

View File

@@ -101,9 +101,12 @@ public class HoodieTestUtils {
public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath, HoodieTableType tableType, public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath, HoodieTableType tableType,
Properties properties) Properties properties)
throws IOException { throws IOException {
properties.putIfAbsent(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME); properties = HoodieTableMetaClient.withPropertyBuilder()
properties.putIfAbsent(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name()); .setTableName(RAW_TRIPS_TEST_NAME)
properties.putIfAbsent(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName()); .setTableType(tableType)
.setPayloadClass(HoodieAvroPayload.class)
.fromProperties(properties)
.build();
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties); return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties);
} }

View File

@@ -72,8 +72,11 @@ public class HoodieJavaWriteClientExample {
Path path = new Path(tablePath); Path path = new Path(tablePath);
FileSystem fs = FSUtils.getFs(tablePath, hadoopConf); FileSystem fs = FSUtils.getFs(tablePath, hadoopConf);
if (!fs.exists(path)) { if (!fs.exists(path)) {
HoodieTableMetaClient.initTableType(hadoopConf, tablePath, HoodieTableType.valueOf(tableType), HoodieTableMetaClient.withPropertyBuilder()
tableName, HoodieAvroPayload.class.getName()); .setTableType(tableType)
.setTableName(tableName)
.setPayloadClassName(HoodieAvroPayload.class.getName())
.initTable(hadoopConf, tablePath);
} }
// Create the write client to write some records in // Create the write client to write some records in

View File

@@ -85,8 +85,11 @@ public class HoodieWriteClientExample {
Path path = new Path(tablePath); Path path = new Path(tablePath);
FileSystem fs = FSUtils.getFs(tablePath, jsc.hadoopConfiguration()); FileSystem fs = FSUtils.getFs(tablePath, jsc.hadoopConfiguration());
if (!fs.exists(path)) { if (!fs.exists(path)) {
HoodieTableMetaClient.initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), HoodieTableMetaClient.withPropertyBuilder()
tableName, HoodieAvroPayload.class.getName()); .setTableType(tableType)
.setTableName(tableName)
.setPayloadClass(HoodieAvroPayload.class)
.initTable(jsc.hadoopConfiguration(), tablePath);
} }
// Create the write client to write some records in // Create the write client to write some records in

View File

@@ -19,7 +19,6 @@
package org.apache.hudi.util; package org.apache.hudi.util;
import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.TablePathUtils; import org.apache.hudi.common.util.TablePathUtils;
import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieException;
@@ -284,14 +283,13 @@ public class StreamerUtil {
// Hadoop FileSystem // Hadoop FileSystem
try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) { try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
if (!fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))) { if (!fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))) {
HoodieTableMetaClient.initTableType( HoodieTableMetaClient.withPropertyBuilder()
hadoopConf, .setTableType(conf.getString(FlinkOptions.TABLE_TYPE))
basePath, .setTableName(conf.getString(FlinkOptions.TABLE_NAME))
HoodieTableType.valueOf(conf.getString(FlinkOptions.TABLE_TYPE)), .setPayloadClassName(conf.getString(FlinkOptions.PAYLOAD_CLASS))
conf.getString(FlinkOptions.TABLE_NAME), .setArchiveLogFolder(DEFAULT_ARCHIVE_LOG_FOLDER)
DEFAULT_ARCHIVE_LOG_FOLDER, .setTimelineLayoutVersion(1)
conf.getString(FlinkOptions.PAYLOAD_CLASS), .initTable(hadoopConf, basePath);
1);
LOG.info("Table initialized under base path {}", basePath); LOG.info("Table initialized under base path {}", basePath);
} else { } else {
LOG.info("Table [{}/{}] already exists, no need to initialize the table", LOG.info("Table [{}/{}] already exists, no need to initialize the table",

View File

@@ -21,7 +21,6 @@ package org.apache.hudi.integ.testsuite;
import org.apache.hudi.DataSourceUtils; import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieException;
@@ -94,8 +93,11 @@ public class HoodieTestSuiteJob {
this.keyGenerator = (BuiltinKeyGenerator) DataSourceUtils.createKeyGenerator(props); this.keyGenerator = (BuiltinKeyGenerator) DataSourceUtils.createKeyGenerator(props);
if (!fs.exists(new Path(cfg.targetBasePath))) { if (!fs.exists(new Path(cfg.targetBasePath))) {
HoodieTableMetaClient.initTableType(jsc.hadoopConfiguration(), cfg.targetBasePath, HoodieTableMetaClient.withPropertyBuilder()
HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived"); .setTableType(cfg.tableType)
.setTableName(cfg.targetTableName)
.setArchiveLogFolder("archived")
.initTable(jsc.hadoopConfiguration(), cfg.targetBasePath);
} }
if (cfg.cleanInput) { if (cfg.cleanInput) {

View File

@@ -38,7 +38,7 @@ import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, B
import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.exception.HoodieException import org.apache.hudi.exception.HoodieException
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
import org.apache.hudi.internal.{DataSourceInternalWriterHelper, HoodieDataSourceInternalWriter} import org.apache.hudi.internal.DataSourceInternalWriterHelper
import org.apache.hudi.sync.common.AbstractSyncTool import org.apache.hudi.sync.common.AbstractSyncTool
import org.apache.log4j.LogManager import org.apache.log4j.LogManager
import org.apache.spark.SPARK_VERSION import org.apache.spark.SPARK_VERSION
@@ -111,9 +111,14 @@ private[hudi] object HoodieSparkSqlWriter {
if (!tableExists) { if (!tableExists) {
val archiveLogFolder = parameters.getOrElse( val archiveLogFolder = parameters.getOrElse(
HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived") HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived")
val tableMetaClient = HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get,
tableType, tblName, archiveLogFolder, parameters(PAYLOAD_CLASS_OPT_KEY), val tableMetaClient = HoodieTableMetaClient.withPropertyBuilder()
null.asInstanceOf[String], parameters.getOrDefault(PRECOMBINE_FIELD_OPT_KEY, null)) .setTableType(tableType)
.setTableName(tblName)
.setArchiveLogFolder(archiveLogFolder)
.setPayloadClassName(parameters(PAYLOAD_CLASS_OPT_KEY))
.setPreCombineField(parameters.getOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
.initTable(sparkContext.hadoopConfiguration, path.get)
tableConfig = tableMetaClient.getTableConfig tableConfig = tableMetaClient.getTableConfig
} }
@@ -261,10 +266,15 @@ private[hudi] object HoodieSparkSqlWriter {
if (!tableExists) { if (!tableExists) {
val archiveLogFolder = parameters.getOrElse( val archiveLogFolder = parameters.getOrElse(
HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived") HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived")
HoodieTableMetaClient.initTableTypeWithBootstrap(sparkContext.hadoopConfiguration, path, HoodieTableMetaClient.withPropertyBuilder()
HoodieTableType.valueOf(tableType), tableName, archiveLogFolder, parameters(PAYLOAD_CLASS_OPT_KEY), .setTableType(HoodieTableType.valueOf(tableType))
null, parameters.getOrDefault(PRECOMBINE_FIELD_OPT_KEY, null), .setTableName(tableName)
bootstrapIndexClass, bootstrapBasePath) .setArchiveLogFolder(archiveLogFolder)
.setPayloadClassName(parameters(PAYLOAD_CLASS_OPT_KEY))
.setPreCombineField(parameters.getOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
.setBootstrapIndexClass(bootstrapIndexClass)
.setBootstrapBasePath(bootstrapBasePath)
.initTable(sparkContext.hadoopConfiguration, path)
} }
val jsc = new JavaSparkContext(sqlContext.sparkContext) val jsc = new JavaSparkContext(sqlContext.sparkContext)

View File

@@ -44,8 +44,11 @@ class TestStreamingSource extends StreamTest {
test("test cow stream source") { test("test cow stream source") {
withTempDir { inputDir => withTempDir { inputDir =>
val tablePath = s"${inputDir.getCanonicalPath}/test_cow_stream" val tablePath = s"${inputDir.getCanonicalPath}/test_cow_stream"
HoodieTableMetaClient.initTableType(spark.sessionState.newHadoopConf(), tablePath, HoodieTableMetaClient.withPropertyBuilder()
COPY_ON_WRITE, getTableName(tablePath), DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL) .setTableType(COPY_ON_WRITE)
.setTableName(getTableName(tablePath))
.setPayloadClassName(DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL)
.initTable(spark.sessionState.newHadoopConf(), tablePath)
addData(tablePath, Seq(("1", "a1", "10", "000"))) addData(tablePath, Seq(("1", "a1", "10", "000")))
val df = spark.readStream val df = spark.readStream
@@ -91,8 +94,11 @@ class TestStreamingSource extends StreamTest {
test("test mor stream source") { test("test mor stream source") {
withTempDir { inputDir => withTempDir { inputDir =>
val tablePath = s"${inputDir.getCanonicalPath}/test_mor_stream" val tablePath = s"${inputDir.getCanonicalPath}/test_mor_stream"
HoodieTableMetaClient.initTableType(spark.sessionState.newHadoopConf(), tablePath, HoodieTableMetaClient.withPropertyBuilder()
MERGE_ON_READ, getTableName(tablePath), DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL) .setTableType(MERGE_ON_READ)
.setTableName(getTableName(tablePath))
.setPayloadClassName(DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL)
.initTable(spark.sessionState.newHadoopConf(), tablePath)
addData(tablePath, Seq(("1", "a1", "10", "000"))) addData(tablePath, Seq(("1", "a1", "10", "000")))
val df = spark.readStream val df = spark.readStream

View File

@@ -126,8 +126,11 @@ public class HiveTestUtil {
public static void clear() throws IOException { public static void clear() throws IOException {
fileSystem.delete(new Path(hiveSyncConfig.basePath), true); fileSystem.delete(new Path(hiveSyncConfig.basePath), true);
HoodieTableMetaClient.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE, HoodieTableMetaClient.withPropertyBuilder()
hiveSyncConfig.tableName, HoodieAvroPayload.class.getName()); .setTableType(HoodieTableType.COPY_ON_WRITE)
.setTableName(hiveSyncConfig.tableName)
.setPayloadClass(HoodieAvroPayload.class)
.initTable(configuration, hiveSyncConfig.basePath);
HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), fileSystem); HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), fileSystem);
for (String tableName : createdTablesSet) { for (String tableName : createdTablesSet) {
@@ -161,8 +164,12 @@ public class HiveTestUtil {
throws IOException, URISyntaxException { throws IOException, URISyntaxException {
Path path = new Path(hiveSyncConfig.basePath); Path path = new Path(hiveSyncConfig.basePath);
FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
HoodieTableMetaClient.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE, HoodieTableMetaClient.withPropertyBuilder()
hiveSyncConfig.tableName, HoodieAvroPayload.class.getName()); .setTableType(HoodieTableType.COPY_ON_WRITE)
.setTableName(hiveSyncConfig.tableName)
.setPayloadClass(HoodieAvroPayload.class)
.initTable(configuration, hiveSyncConfig.basePath);
boolean result = fileSystem.mkdirs(path); boolean result = fileSystem.mkdirs(path);
checkResult(result); checkResult(result);
DateTime dateTime = DateTime.now(); DateTime dateTime = DateTime.now();
@@ -177,8 +184,11 @@ public class HiveTestUtil {
throws IOException, URISyntaxException, InterruptedException { throws IOException, URISyntaxException, InterruptedException {
Path path = new Path(hiveSyncConfig.basePath); Path path = new Path(hiveSyncConfig.basePath);
FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
HoodieTableMetaClient.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.MERGE_ON_READ, HoodieTableMetaClient.withPropertyBuilder()
hiveSyncConfig.tableName, HoodieAvroPayload.class.getName()); .setTableType(HoodieTableType.MERGE_ON_READ)
.setTableName(hiveSyncConfig.tableName)
.setPayloadClass(HoodieAvroPayload.class)
.initTable(configuration, hiveSyncConfig.basePath);
boolean result = fileSystem.mkdirs(path); boolean result = fileSystem.mkdirs(path);
checkResult(result); checkResult(result);

View File

@@ -28,7 +28,6 @@ import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
@@ -135,9 +134,10 @@ public class HDFSParquetImporter implements Serializable {
if (!fs.exists(new Path(cfg.targetPath))) { if (!fs.exists(new Path(cfg.targetPath))) {
// Initialize target hoodie table. // Initialize target hoodie table.
Properties properties = new Properties(); Properties properties = HoodieTableMetaClient.withPropertyBuilder()
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName); .setTableName(cfg.tableName)
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType); .setTableType(cfg.tableType)
.build();
HoodieTableMetaClient.initTableAndGetMetaClient(jsc.hadoopConfiguration(), cfg.targetPath, properties); HoodieTableMetaClient.initTableAndGetMetaClient(jsc.hadoopConfiguration(), cfg.targetPath, properties);
} }

View File

@@ -23,7 +23,6 @@ import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
@@ -170,10 +169,15 @@ public class BootstrapExecutor implements Serializable {
throw new HoodieException("target base path already exists at " + cfg.targetBasePath throw new HoodieException("target base path already exists at " + cfg.targetBasePath
+ ". Cannot bootstrap data on top of an existing table"); + ". Cannot bootstrap data on top of an existing table");
} }
HoodieTableMetaClient.withPropertyBuilder()
HoodieTableMetaClient.initTableTypeWithBootstrap(new Configuration(jssc.hadoopConfiguration()), .setTableType(cfg.tableType)
cfg.targetBasePath, HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName, .setTableName(cfg.targetTableName)
cfg.baseFileFormat, cfg.bootstrapIndexClass, bootstrapBasePath); .setArchiveLogFolder("archived")
.setPayloadClassName(cfg.payloadClassName)
.setBaseFileFormat(cfg.baseFileFormat)
.setBootstrapIndexClass(cfg.bootstrapIndexClass)
.setBootstrapBasePath(bootstrapBasePath)
.initTable(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath);
} }
public HoodieWriteConfig getBootstrapConfig() { public HoodieWriteConfig getBootstrapConfig() {

View File

@@ -31,7 +31,6 @@ import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -236,8 +235,14 @@ public class DeltaSync implements Serializable {
} }
} else { } else {
this.commitTimelineOpt = Option.empty(); this.commitTimelineOpt = Option.empty();
HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, HoodieTableMetaClient.withPropertyBuilder()
HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat); .setTableType(cfg.tableType)
.setTableName(cfg.targetTableName)
.setArchiveLogFolder("archived")
.setPayloadClassName(cfg.payloadClassName)
.setBaseFileFormat(cfg.baseFileFormat)
.initTable(new Configuration(jssc.hadoopConfiguration()),
cfg.targetBasePath);
} }
} }
@@ -321,8 +326,13 @@ public class DeltaSync implements Serializable {
} }
} }
} else { } else {
HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, HoodieTableMetaClient.withPropertyBuilder()
HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName, cfg.baseFileFormat); .setTableType(cfg.tableType)
.setTableName(cfg.targetTableName)
.setArchiveLogFolder("archived")
.setPayloadClassName(cfg.payloadClassName)
.setBaseFileFormat(cfg.baseFileFormat)
.initTable(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath);
} }
if (!resumeCheckpointStr.isPresent() && cfg.checkpoint != null) { if (!resumeCheckpointStr.isPresent() && cfg.checkpoint != null) {

View File

@@ -79,9 +79,12 @@ public class TestHoodieSnapshotExporter extends FunctionalTestHarness {
sourcePath = dfsBasePath() + "/source/"; sourcePath = dfsBasePath() + "/source/";
targetPath = dfsBasePath() + "/target/"; targetPath = dfsBasePath() + "/target/";
dfs().mkdirs(new Path(sourcePath)); dfs().mkdirs(new Path(sourcePath));
HoodieTableMetaClient
.initTableType(jsc().hadoopConfiguration(), sourcePath, HoodieTableType.COPY_ON_WRITE, TABLE_NAME, HoodieTableMetaClient.withPropertyBuilder()
HoodieAvroPayload.class.getName()); .setTableType(HoodieTableType.COPY_ON_WRITE)
.setTableName(TABLE_NAME)
.setPayloadClass(HoodieAvroPayload.class)
.initTable(jsc().hadoopConfiguration(), sourcePath);
// Prepare data as source Hudi dataset // Prepare data as source Hudi dataset
HoodieWriteConfig cfg = getHoodieWriteConfig(sourcePath); HoodieWriteConfig cfg = getHoodieWriteConfig(sourcePath);

View File

@@ -180,8 +180,11 @@ public class UtilitiesTestBase {
// Create Dummy hive sync config // Create Dummy hive sync config
HiveSyncConfig hiveSyncConfig = getHiveSyncConfig("/dummy", "dummy"); HiveSyncConfig hiveSyncConfig = getHiveSyncConfig("/dummy", "dummy");
hiveConf.addResource(hiveServer.getHiveConf()); hiveConf.addResource(hiveServer.getHiveConf());
HoodieTableMetaClient.initTableType(dfs.getConf(), hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE, HoodieTableMetaClient.withPropertyBuilder()
hiveSyncConfig.tableName, null); .setTableType(HoodieTableType.COPY_ON_WRITE)
.setTableName(hiveSyncConfig.tableName)
.initTable(dfs.getConf(), hiveSyncConfig.basePath);
HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveConf, dfs); HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveConf, dfs);
client.updateHiveSQL("drop database if exists " + hiveSyncConfig.databaseName); client.updateHiveSQL("drop database if exists " + hiveSyncConfig.databaseName);
client.updateHiveSQL("create database " + hiveSyncConfig.databaseName); client.updateHiveSQL("create database " + hiveSyncConfig.databaseName);