1
0

[HUDI-509] Renaming code in sync with cWiki restructuring (#1212)

- Storage Type replaced with Table Type (remaining instances)
 - View types replaced with query types;
 - ReadOptimized view referred as Snapshot Query
 - TableFileSystemView sub interfaces renamed to BaseFileOnly and Slice Views
 - HoodieDataFile renamed to HoodieBaseFile
 - Hive Sync tool will register RO tables for MOR with a `_ro` suffix
 - Datasource/Deltastreamer options renamed accordingly
 - Support fallback to old config values as well, so migration is painless
 - Config for controlling _ro suffix addition
 - Renaming DataFile to BaseFile across DTOs, HoodieFileSlice and AbstractTableFileSystemView
This commit is contained in:
vinoth chandar
2020-01-16 23:58:47 -08:00
committed by GitHub
parent 8a3a50309b
commit c2c0f6b13d
92 changed files with 907 additions and 822 deletions

View File

@@ -19,12 +19,12 @@
package org.apache.hudi.utilities;
import org.apache.hudi.common.SerializableConfiguration;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.FSUtils;
@@ -73,7 +73,7 @@ public class HoodieSnapshotCopier implements Serializable {
FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration());
final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration());
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir);
final ReadOptimizedView fsView = new HoodieTableFileSystemView(tableMetadata,
final BaseFileOnlyView fsView = new HoodieTableFileSystemView(tableMetadata,
tableMetadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants());
// Get the latest commit
Option<HoodieInstant> latestCommit =
@@ -101,7 +101,7 @@ public class HoodieSnapshotCopier implements Serializable {
// Only take latest version files <= latestCommit.
FileSystem fs1 = FSUtils.getFs(baseDir, serConf.newCopy());
List<Tuple2<String, String>> filePaths = new ArrayList<>();
Stream<HoodieDataFile> dataFiles = fsView.getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp);
Stream<HoodieBaseFile> dataFiles = fsView.getLatestBaseFilesBeforeOrOn(partition, latestCommitTimestamp);
dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
// also need to copy over partition metadata

View File

@@ -208,7 +208,7 @@ public class DeltaSync implements Serializable {
} else {
this.commitTimelineOpt = Option.empty();
HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath,
cfg.storageType, cfg.targetTableName, "archived", cfg.payloadClassName);
cfg.tableType, cfg.targetTableName, "archived", cfg.payloadClassName);
}
}
@@ -270,7 +270,7 @@ public class DeltaSync implements Serializable {
}
} else {
HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath,
cfg.storageType, cfg.targetTableName, "archived", cfg.payloadClassName);
cfg.tableType, cfg.targetTableName, "archived", cfg.payloadClassName);
}
if (!resumeCheckpointStr.isPresent() && cfg.checkpoint != null) {

View File

@@ -156,13 +156,11 @@ public class HoodieDeltaStreamer implements Serializable {
required = true)
public String targetBasePath;
// TODO: How to obtain hive configs to register?
@Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true)
public String targetTableName;
@Parameter(names = {"--storage-type"}, description = "Type of Storage. COPY_ON_WRITE (or) MERGE_ON_READ",
required = true)
public String storageType;
@Parameter(names = {"--table-type"}, description = "Type of table. COPY_ON_WRITE (or) MERGE_ON_READ", required = true)
public String tableType;
@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
+ "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
@@ -273,12 +271,12 @@ public class HoodieDeltaStreamer implements Serializable {
public boolean isAsyncCompactionEnabled() {
return continuousMode && !forceDisableCompaction
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(storageType));
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType));
}
public boolean isInlineCompactionEnabled() {
return !continuousMode && !forceDisableCompaction
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(storageType));
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType));
}
}
@@ -356,10 +354,10 @@ public class HoodieDeltaStreamer implements Serializable {
new HoodieTableMetaClient(new Configuration(fs.getConf()), cfg.targetBasePath, false);
tableType = meta.getTableType();
// This will guarantee there is no surprise with table type
Preconditions.checkArgument(tableType.equals(HoodieTableType.valueOf(cfg.storageType)),
"Hoodie table is of type " + tableType + " but passed in CLI argument is " + cfg.storageType);
Preconditions.checkArgument(tableType.equals(HoodieTableType.valueOf(cfg.tableType)),
"Hoodie table is of type " + tableType + " but passed in CLI argument is " + cfg.tableType);
} else {
tableType = HoodieTableType.valueOf(cfg.storageType);
tableType = HoodieTableType.valueOf(cfg.tableType);
}
this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
@@ -502,7 +500,6 @@ public class HoodieDeltaStreamer implements Serializable {
public AsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client) {
this.jssc = jssc;
this.compactor = new Compactor(client, jssc);
// TODO: HUDI-157 : Only allow 1 compactor to run in parallel till Incremental View on MOR is fully implemented.
this.maxConcurrentCompaction = 1;
}

View File

@@ -71,7 +71,7 @@ public class SchedulerConfGenerator {
Map<String, String> additionalSparkConfigs = new HashMap<>();
if (sparkSchedulerMode.isPresent() && "FAIR".equals(sparkSchedulerMode.get()) && cfg.continuousMode
&& cfg.storageType.equals(HoodieTableType.MERGE_ON_READ.name())) {
&& cfg.tableType.equals(HoodieTableType.MERGE_ON_READ.name())) {
String sparkSchedulingConfFile = generateAndStoreConfig(cfg.deltaSyncSchedulingWeight,
cfg.compactSchedulingWeight, cfg.deltaSyncSchedulingMinShare, cfg.compactSchedulingMinShare);
additionalSparkConfigs.put(SPARK_SCHEDULER_ALLOCATION_FILE_KEY, sparkSchedulingConfFile);

View File

@@ -115,7 +115,7 @@ public class HoodieIncrSource extends RowSource {
// Do Incr pull. Set end instant if available
DataFrameReader reader = sparkSession.read().format("org.apache.hudi")
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL())
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), instantEndpts.getLeft())
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY(), instantEndpts.getRight());

View File

@@ -185,11 +185,11 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
static HoodieDeltaStreamer.Config makeConfig(String basePath, Operation op, String transformerClassName,
String propsFilename, boolean enableHiveSync, boolean useSchemaProviderClass, boolean updatePayloadClass,
String payloadClassName, String storageType) {
String payloadClassName, String tableType) {
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
cfg.targetBasePath = basePath;
cfg.targetTableName = "hoodie_trips";
cfg.storageType = storageType == null ? "COPY_ON_WRITE" : storageType;
cfg.tableType = tableType == null ? "COPY_ON_WRITE" : tableType;
cfg.sourceClassName = TestDataSource.class.getName();
cfg.transformerClassName = transformerClassName;
cfg.operation = op;
@@ -211,7 +211,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
cfg.targetBasePath = basePath;
cfg.targetTableName = "hoodie_trips_copy";
cfg.storageType = "COPY_ON_WRITE";
cfg.tableType = "COPY_ON_WRITE";
cfg.sourceClassName = HoodieIncrSource.class.getName();
cfg.operation = op;
cfg.sourceOrderingField = "timestamp";
@@ -386,7 +386,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
// Initial bulk insert
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, Operation.UPSERT);
cfg.continuousMode = true;
cfg.storageType = tableType.name();
cfg.tableType = tableType.name();
cfg.configs.add(String.format("%s=%d", TestSourceConfig.MAX_UNIQUE_RECORDS_PROP, totalRecords));
cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP));
HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc);
@@ -489,11 +489,11 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
// Test Hive integration
HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), dfs);
assertTrue("Table " + hiveSyncConfig.tableName + " should exist", hiveClient.doesTableExist());
assertTrue("Table " + hiveSyncConfig.tableName + " should exist", hiveClient.doesTableExist(hiveSyncConfig.tableName));
assertEquals("Table partitions should match the number of partitions we wrote", 1,
hiveClient.scanTablePartitions().size());
hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", lastInstantForUpstreamTable,
hiveClient.getLastCommitTimeSynced().get());
hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get());
}
@Test

View File

@@ -43,12 +43,12 @@ public class TestSchedulerConfGenerator {
assertNull("continuousMode is false", configs.get(SchedulerConfGenerator.SPARK_SCHEDULER_ALLOCATION_FILE_KEY));
cfg.continuousMode = true;
cfg.storageType = HoodieTableType.COPY_ON_WRITE.name();
cfg.tableType = HoodieTableType.COPY_ON_WRITE.name();
configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
assertNull("storageType is not MERGE_ON_READ",
assertNull("table type is not MERGE_ON_READ",
configs.get(SchedulerConfGenerator.SPARK_SCHEDULER_ALLOCATION_FILE_KEY));
cfg.storageType = HoodieTableType.MERGE_ON_READ.name();
cfg.tableType = HoodieTableType.MERGE_ON_READ.name();
configs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
assertNotNull("all satisfies", configs.get(SchedulerConfGenerator.SPARK_SCHEDULER_ALLOCATION_FILE_KEY));
}