1
0

[HUDI-508] Standardizing on "Table" instead of "Dataset" across code (#1197)

- Docs were talking about storage types before, cWiki moved to "Table"
 - Most of code already has HoodieTable, HoodieTableMetaClient - correct naming
 - Replacing renaming use of dataset across code/comments
 - Few usages in comments and use of Spark SQL DataSet remain unscathed
This commit is contained in:
vinoth chandar
2020-01-07 12:52:32 -08:00
committed by GitHub
parent 8306f749a2
commit 9706f659db
73 changed files with 298 additions and 298 deletions

View File

@@ -48,7 +48,7 @@ public class HoodieCLI {
* Enum for CLI state.
*/
public enum CLIState {
INIT, DATASET, SYNC
INIT, TABLE, SYNC
}
public static void setConsistencyGuardConfig(ConsistencyGuardConfig config) {
@@ -100,7 +100,7 @@ public class HoodieCLI {
*/
public static HoodieTableMetaClient getTableMetaClient() {
if (tableMetadata == null) {
throw new NullPointerException("There is no hudi dataset. Please use connect command to set dataset first");
throw new NullPointerException("There is no hudi table. Please use connect command to set table first");
}
return tableMetadata;
}

View File

@@ -37,7 +37,7 @@ public class HoodiePrompt extends DefaultPromptProvider {
switch (HoodieCLI.state) {
case INIT:
return "hudi->";
case DATASET:
case TABLE:
return "hudi:" + tableName + "->";
case SYNC:
return "hudi:" + tableName + " <==> " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";

View File

@@ -206,8 +206,8 @@ public class CommitsCommand implements CommandMarker {
return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
}
@CliCommand(value = "commits compare", help = "Compare commits with another Hoodie dataset")
public String compareCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path)
@CliCommand(value = "commits compare", help = "Compare commits with another Hoodie table")
public String compareCommits(@CliOption(key = {"path"}, help = "Path of the table to compare to") final String path)
throws Exception {
HoodieTableMetaClient source = HoodieCLI.getTableMetaClient();
@@ -234,8 +234,8 @@ public class CommitsCommand implements CommandMarker {
}
}
@CliCommand(value = "commits sync", help = "Compare commits with another Hoodie dataset")
public String syncCommits(@CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) {
@CliCommand(value = "commits sync", help = "Compare commits with another Hoodie table")
public String syncCommits(@CliOption(key = {"path"}, help = "Path of the table to compare to") final String path) {
HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path);
HoodieCLI.state = HoodieCLI.CLIState.SYNC;
return "Load sync state between " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " and "

View File

@@ -145,7 +145,7 @@ public class CompactionCommand implements CommandMarker {
@CliCommand(value = "compaction show", help = "Shows compaction details for a specific compaction instant")
public String compactionShow(
@CliOption(key = "instant", mandatory = true,
help = "Base path for the target hoodie dataset") final String compactionInstantTime,
help = "Base path for the target hoodie table") final String compactionInstantTime,
@CliOption(key = {"limit"}, help = "Limit commits",
unspecifiedDefaultValue = "-1") final Integer limit,
@CliOption(key = {"sortBy"}, help = "Sorting Field", unspecifiedDefaultValue = "") final String sortByField,
@@ -212,7 +212,7 @@ public class CompactionCommand implements CommandMarker {
@CliOption(key = "sparkMemory", unspecifiedDefaultValue = "4G",
help = "Spark executor memory") final String sparkMemory,
@CliOption(key = "retry", unspecifiedDefaultValue = "1", help = "Number of retries") final String retry,
@CliOption(key = "compactionInstant", help = "Base path for the target hoodie dataset") String compactionInstantTime,
@CliOption(key = "compactionInstant", help = "Base path for the target hoodie table") String compactionInstantTime,
@CliOption(key = "propsFilePath", help = "path to properties file on localfs or dfs with configurations for hoodie client for compacting",
unspecifiedDefaultValue = "") final String propsFilePath,
@CliOption(key = "hoodieConfigs", help = "Any configuration that can be set in the properties file can be passed here in the form of an array",
@@ -471,7 +471,7 @@ public class CompactionCommand implements CommandMarker {
if (result.get()) {
System.out.println("All renames successfully completed to " + operation + " done !!");
} else {
System.out.println("Some renames failed. DataSet could be in inconsistent-state. Try running compaction repair");
System.out.println("Some renames failed. table could be in inconsistent-state. Try running compaction repair");
}
List<Comparable[]> rows = new ArrayList<>();

View File

@@ -35,18 +35,18 @@ import org.springframework.stereotype.Component;
import scala.collection.JavaConverters;
/**
* CLI command for importing parquet dataset to hudi dataset.
* CLI command for importing parquet table to hudi table.
*/
@Component
public class HDFSParquetImportCommand implements CommandMarker {
@CliCommand(value = "hdfsparquetimport", help = "Imports Parquet dataset to a hoodie dataset")
@CliCommand(value = "hdfsparquetimport", help = "Imports Parquet table to a hoodie table")
public String convert(
@CliOption(key = "upsert", unspecifiedDefaultValue = "false",
help = "Uses upsert API instead of the default insert API of WriteClient") boolean useUpsert,
@CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset") final String srcPath,
@CliOption(key = "srcPath", mandatory = true, help = "Base path for the input table") final String srcPath,
@CliOption(key = "targetPath", mandatory = true,
help = "Base path for the target hoodie dataset") final String targetPath,
help = "Base path for the target hoodie table") final String targetPath,
@CliOption(key = "tableName", mandatory = true, help = "Table name") final String tableName,
@CliOption(key = "tableType", mandatory = true, help = "Table type") final String tableType,
@CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name") final String rowKeyField,
@@ -85,8 +85,8 @@ public class HDFSParquetImportCommand implements CommandMarker {
InputStreamConsumer.captureOutput(process);
int exitCode = process.waitFor();
if (exitCode != 0) {
return "Failed to import dataset to hoodie format";
return "Failed to import table to hoodie format";
}
return "Dataset imported to hoodie format";
return "Table imported to hoodie format";
}
}

View File

@@ -65,7 +65,7 @@ public class RepairsCommand implements CommandMarker {
return "Deduplication failed ";
}
@CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a dataset, if not present")
@CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a table, if not present")
public String addPartitionMeta(
@CliOption(key = {"dryrun"}, help = "Should we actually add or just print what would be done",
unspecifiedDefaultValue = "true") final boolean dryRun)

View File

@@ -24,7 +24,7 @@ import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.apache.hudi.exception.TableNotFoundException;
import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliCommand;
@@ -37,18 +37,18 @@ import java.util.HashMap;
import java.util.List;
/**
* CLI command to display hudi dataset options.
* CLI command to display hudi table options.
*/
@Component
public class DatasetsCommand implements CommandMarker {
public class TableCommand implements CommandMarker {
static {
System.out.println("DatasetsCommand getting loaded");
System.out.println("Table command getting loaded");
}
@CliCommand(value = "connect", help = "Connect to a hoodie dataset")
@CliCommand(value = "connect", help = "Connect to a hoodie table")
public String connect(
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path,
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the table") final String path,
@CliOption(key = {"layoutVersion"}, help = "Timeline Layout version") Integer layoutVersion,
@CliOption(key = {"eventuallyConsistent"}, unspecifiedDefaultValue = "false",
help = "Enable eventual consistency") final boolean eventuallyConsistent,
@@ -67,7 +67,7 @@ public class DatasetsCommand implements CommandMarker {
HoodieCLI.initConf();
HoodieCLI.connectTo(path, layoutVersion);
HoodieCLI.initFS(true);
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
HoodieCLI.state = HoodieCLI.CLIState.TABLE;
return "Metadata for table " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " loaded";
}
@@ -81,7 +81,7 @@ public class DatasetsCommand implements CommandMarker {
*/
@CliCommand(value = "create", help = "Create a hoodie table if not present")
public String createTable(
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path,
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the table") final String path,
@CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie Table Name") final String name,
@CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE",
help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr,
@@ -98,13 +98,13 @@ public class DatasetsCommand implements CommandMarker {
try {
new HoodieTableMetaClient(HoodieCLI.conf, path);
existing = true;
} catch (DatasetNotFoundException dfe) {
} catch (TableNotFoundException dfe) {
// expected
}
// Do not touch table that already exist
if (existing) {
throw new IllegalStateException("Dataset already existing in path : " + path);
throw new IllegalStateException("Table already existing in path : " + path);
}
final HoodieTableType tableType = HoodieTableType.valueOf(tableTypeStr);

View File

@@ -62,7 +62,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
/**
* TODO: We need to persist the index type into hoodie.properties and be able to access the index just with a simple
* basepath pointing to the dataset. Until, then just always assume a BloomIndex
* basepath pointing to the table. Until, then just always assume a BloomIndex
*/
private final transient HoodieIndex<T> index;
private final HoodieTimeline commitTimeline;
@@ -70,7 +70,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
private transient Option<SQLContext> sqlContextOpt;
/**
* @param basePath path to Hoodie dataset
* @param basePath path to Hoodie table
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath, Option<EmbeddedTimelineService> timelineService) {
this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
@@ -80,7 +80,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
}
/**
* @param basePath path to Hoodie dataset
* @param basePath path to Hoodie table
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
this(jsc, basePath, Option.empty());

View File

@@ -85,10 +85,10 @@ import java.util.stream.IntStream;
import scala.Tuple2;
/**
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient mutations on an HDFS
* dataset [upsert()]
* Hoodie Write Client helps you build tables on HDFS [insert()] and then perform efficient mutations on an HDFS
* table [upsert()]
* <p>
* Note that, at any given time, there can only be one Spark job performing these operations on a Hoodie dataset.
* Note that, at any given time, there can only be one Spark job performing these operations on a Hoodie table.
*/
public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHoodieWriteClient<T> {
@@ -242,7 +242,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
/**
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
* table for the very first time (e.g: converting an existing dataset to Hoodie).
* table for the very first time (e.g: converting an existing table to Hoodie).
* <p>
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
* the numbers of files with less memory compared to the {@link HoodieWriteClient#insert(JavaRDD, String)}
@@ -257,7 +257,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
/**
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
* table for the very first time (e.g: converting an existing dataset to Hoodie).
* table for the very first time (e.g: converting an existing table to Hoodie).
* <p>
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
* the numbers of files with less memory compared to the {@link HoodieWriteClient#insert(JavaRDD, String)}. Optionally
@@ -289,7 +289,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
/**
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
* table for the very first time (e.g: converting an existing dataset to Hoodie). The input records should contain no
* table for the very first time (e.g: converting an existing table to Hoodie). The input records should contain no
* duplicates if needed.
* <p>
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
@@ -393,7 +393,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
/**
* Save the workload profile in an intermediate file (here re-using commit files) This is useful when performing
* rollback for MOR datasets. Only updates are recorded in the workload profile metadata since updates to log blocks
* rollback for MOR tables. Only updates are recorded in the workload profile metadata since updates to log blocks
* are unknown across batches Inserts (which are new parquet files) are rolled back based on commit time. // TODO :
* Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata
*/
@@ -691,7 +691,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
}
/**
* NOTE : This action requires all writers (ingest and compact) to a dataset to be stopped before proceeding. Revert
* NOTE : This action requires all writers (ingest and compact) to a table to be stopped before proceeding. Revert
* the (inflight/committed) record changes for all commits after the provided @param. Three steps: (1) Atomically
* unpublish this commit (2) clean indexing data, (3) clean new generated parquet/log files and/or append rollback to
* existing log files. (4) Finally delete .commit, .inflight, .compaction.inflight or .compaction.requested file
@@ -765,7 +765,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
AvroUtils.convertRestoreMetadata(startRestoreTime, durationInMs, commitsToRollback, commitToStats);
table.getActiveTimeline().saveAsComplete(new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, startRestoreTime),
AvroUtils.serializeRestoreMetadata(restoreMetadata));
LOG.info("Commits " + commitsToRollback + " rollback is complete. Restored dataset to " + restoreToInstant);
LOG.info("Commits " + commitsToRollback + " rollback is complete. Restored table to " + restoreToInstant);
if (!table.getActiveTimeline().getCleanerTimeline().empty()) {
LOG.info("Cleaning up older restore meta files");
@@ -1108,7 +1108,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
}
/**
* Performs a compaction operation on a dataset, serially before or after an insert/upsert action.
* Performs a compaction operation on a table, serially before or after an insert/upsert action.
*/
private Option<String> forceCompact(Option<Map<String, String>> extraMetadata) throws IOException {
Option<String> compactionInstantTimeOpt = scheduleCompaction(extraMetadata);

View File

@@ -302,7 +302,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
/**
* For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be
* checked. For datasets, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
* checked. For tables, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
* to be compared gets cut down a lot from range pruning.
*
* Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on

View File

@@ -43,7 +43,7 @@ import java.util.stream.Collectors;
import scala.Tuple2;
/**
* This filter will only work with hoodie dataset since it will only load partitions with .hoodie_partition_metadata
* This filter will only work with hoodie table since it will only load partitions with .hoodie_partition_metadata
* file in it.
*/
public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends HoodieBloomIndex<T> {
@@ -71,7 +71,7 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends Hoodi
/**
* For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be
* checked. For datasets, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
* checked. For tables, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
* to be compared gets cut down a lot from range pruning.
* <p>
* Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on

View File

@@ -75,9 +75,9 @@ import static java.util.stream.Collectors.toList;
public class HoodieRealtimeTableCompactor implements HoodieCompactor {
private static final Logger LOG = LogManager.getLogger(HoodieRealtimeTableCompactor.class);
// Accumulator to keep track of total log files for a dataset
// Accumulator to keep track of total log files for a table
private AccumulatorV2<Long, Long> totalLogFiles;
// Accumulator to keep track of total log file slices for a dataset
// Accumulator to keep track of total log file slices for a table
private AccumulatorV2<Long, Long> totalFileSlices;
@Override

View File

@@ -34,7 +34,7 @@ import java.util.stream.Collectors;
/**
* This strategy ensures that the last N partitions are picked up even if there are later partitions created for the
* dataset. lastNPartitions is defined as the N partitions before the currentDate. currentDay = 2018/01/01 The dataset
* table. lastNPartitions is defined as the N partitions before the currentDate. currentDay = 2018/01/01 The table
* has partitions for 2018/02/02 and 2018/03/03 beyond the currentDay This strategy will pick up the following
* partitions for compaction : (2018/01/01, allPartitionsInRange[(2018/01/01 - lastNPartitions) to 2018/01/01),
* 2018/02/02, 2018/03/03)

View File

@@ -132,7 +132,7 @@ public class HoodieClientExample {
client.delete(deleteRecords, newCommitTime);
/**
* Schedule a compaction and also perform compaction on a MOR dataset
* Schedule a compaction and also perform compaction on a MOR table
*/
if (HoodieTableType.valueOf(tableType) == HoodieTableType.MERGE_ON_READ) {
Option<String> instant = client.scheduleCompaction(Option.empty());

View File

@@ -178,7 +178,7 @@ public class HoodieClientTestUtils {
}
/**
* Obtain all new data written into the Hoodie dataset since the given timestamp.
* Obtain all new data written into the Hoodie table since the given timestamp.
*/
public static Dataset<Row> readSince(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
String lastCommitTime) {
@@ -195,7 +195,7 @@ public class HoodieClientTestUtils {
}
/**
* Reads the paths under the a hoodie dataset out as a DataFrame.
* Reads the paths under the a hoodie table out as a DataFrame.
*/
public static Dataset<Row> read(JavaSparkContext jsc, String basePath, SQLContext sqlContext, FileSystem fs,
String... paths) {
@@ -212,7 +212,7 @@ public class HoodieClientTestUtils {
}
return sqlContext.read().parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
} catch (Exception e) {
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
throw new HoodieException("Error reading hoodie table as a dataframe", e);
}
}

View File

@@ -76,7 +76,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
}
@Test
public void testArchiveEmptyDataset() throws IOException {
public void testArchiveEmptyTable() throws IOException {
HoodieWriteConfig cfg =
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).forTable("test-trip-table").build();
@@ -87,7 +87,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
}
@Test
public void testArchiveDatasetWithArchival() throws IOException {
public void testArchiveTableWithArchival() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 4).build())
@@ -229,7 +229,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
}
@Test
public void testArchiveDatasetWithNoArchival() throws IOException {
public void testArchiveTableWithNoArchival() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())

View File

@@ -292,7 +292,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
}
@Test
public void testCOWToMORConvertedDatasetRollback() throws Exception {
public void testCOWToMORConvertedTableRollback() throws Exception {
// Set TableType to COW
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath, HoodieTableType.COPY_ON_WRITE);

View File

@@ -30,7 +30,7 @@ import java.io.IOException;
/**
* This is a payload to wrap a existing Hoodie Avro Record. Useful to create a HoodieRecord over existing GenericRecords
* in a hoodie datasets (useful in compactions)
* in a hoodie tables (useful in compactions)
*/
public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload> {

View File

@@ -28,7 +28,7 @@ import java.io.Serializable;
import java.util.Map;
/**
* Every Hoodie dataset has an implementation of the <code>HoodieRecordPayload</code> This abstracts out callbacks which
* Every Hoodie table has an implementation of the <code>HoodieRecordPayload</code> This abstracts out callbacks which
* depend on record specific logic.
*/
public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Serializable {

View File

@@ -27,7 +27,7 @@ import java.util.HashMap;
import java.util.Map;
/**
* This class holds statistics about files belonging to a dataset.
* This class holds statistics about files belonging to a table.
*/
public class HoodieRollingStatMetadata implements Serializable {

View File

@@ -41,7 +41,7 @@ import java.util.stream.Collectors;
/**
* Configurations on the Hoodie Table like type of ingestion, storage formats, hive table name etc Configurations are
* loaded from hoodie.properties, these properties are usually set during initializing a path as hoodie base path and
* never changes during the lifetime of a hoodie dataset.
* never changes during the lifetime of a hoodie table.
*
* @see HoodieTableMetaClient
* @since 0.3.0
@@ -71,7 +71,7 @@ public class HoodieTableConfig implements Serializable {
public HoodieTableConfig(FileSystem fs, String metaPath) {
Properties props = new Properties();
Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
LOG.info("Loading dataset properties from " + propertyPath);
LOG.info("Loading table properties from " + propertyPath);
try {
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
props.load(inputStream);
@@ -144,7 +144,7 @@ public class HoodieTableConfig implements Serializable {
* Read the payload class for HoodieRecords from the table properties.
*/
public String getPayloadClass() {
// There could be datasets written with payload class from com.uber.hoodie. Need to transparently
// There could be tables written with payload class from com.uber.hoodie. Need to transparently
// change to org.apache.hudi
return props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS).replace("com.uber.hoodie",
"org.apache.hudi");
@@ -182,7 +182,7 @@ public class HoodieTableConfig implements Serializable {
}
/**
* Get the relative path of archive log folder under metafolder, for this dataset.
* Get the relative path of archive log folder under metafolder, for this table.
*/
public String getArchivelogFolder() {
return props.getProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);

View File

@@ -30,7 +30,7 @@ import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.FailSafeConsistencyGuard;
import org.apache.hudi.common.util.NoOpConsistencyGuard;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.exception.HoodieException;
import com.google.common.base.Preconditions;
@@ -84,7 +84,7 @@ public class HoodieTableMetaClient implements Serializable {
private HoodieArchivedTimeline archivedTimeline;
private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build();
public HoodieTableMetaClient(Configuration conf, String basePath) throws DatasetNotFoundException {
public HoodieTableMetaClient(Configuration conf, String basePath) throws TableNotFoundException {
// Do not load any timeline by default
this(conf, basePath, false);
}
@@ -96,7 +96,7 @@ public class HoodieTableMetaClient implements Serializable {
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion)
throws DatasetNotFoundException {
throws TableNotFoundException {
LOG.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
this.consistencyGuardConfig = consistencyGuardConfig;
@@ -105,7 +105,7 @@ public class HoodieTableMetaClient implements Serializable {
this.metaPath = new Path(basePath, METAFOLDER_NAME).toString();
Path metaPathDir = new Path(this.metaPath);
this.fs = getFs();
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
TableNotFoundException.checkTableValidity(fs, basePathDir, metaPathDir);
this.tableConfig = new HoodieTableConfig(fs, metaPath);
this.tableType = tableConfig.getTableType();
this.timelineLayoutVersion = layoutVersion.orElse(tableConfig.getTimelineLayoutVersion());
@@ -281,7 +281,7 @@ public class HoodieTableMetaClient implements Serializable {
}
/**
* Helper method to initialize a dataset, with given basePath, tableType, name, archiveFolder.
* Helper method to initialize a table, with given basePath, tableType, name, archiveFolder.
*/
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType,
String tableName, String archiveLogFolder) throws IOException {
@@ -314,17 +314,17 @@ public class HoodieTableMetaClient implements Serializable {
if (null != timelineLayoutVersion) {
properties.put(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION, String.valueOf(timelineLayoutVersion));
}
return HoodieTableMetaClient.initDatasetAndGetMetaClient(hadoopConf, basePath, properties);
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties);
}
/**
* Helper method to initialize a given path as a hoodie dataset 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.
*
* @return Instance of HoodieTableMetaClient
*/
public static HoodieTableMetaClient initDatasetAndGetMetaClient(Configuration hadoopConf, String basePath,
public static HoodieTableMetaClient initTableAndGetMetaClient(Configuration hadoopConf, String basePath,
Properties props) throws IOException {
LOG.info("Initializing " + basePath + " as hoodie dataset " + basePath);
LOG.info("Initializing " + basePath + " as hoodie table " + basePath);
Path basePathDir = new Path(basePath);
final FileSystem fs = FSUtils.getFs(basePath, hadoopConf);
if (!fs.exists(basePathDir)) {
@@ -379,7 +379,7 @@ public class HoodieTableMetaClient implements Serializable {
return getActiveTimeline().getCommitTimeline();
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits
// Include commit action to be able to start doing a MOR over a COW dataset - no
// Include commit action to be able to start doing a MOR over a COW table - no
// migration required
return getActiveTimeline().getCommitsTimeline();
default:

View File

@@ -31,7 +31,7 @@ import java.util.function.Predicate;
import java.util.stream.Stream;
/**
* HoodieTimeline is a view of meta-data instants in the hoodie dataset. Instants are specific points in time
* HoodieTimeline is a view of meta-data instants in the hoodie table. Instants are specific points in time
* represented as HoodieInstant.
* <p>
* Timelines are immutable once created and operations create new instance of timelines which filter on the instants and

View File

@@ -47,7 +47,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Represents the Active Timeline for the HoodieDataset. Instants for the last 12 hours (configurable) is in the
* Represents the Active Timeline for the Hoodie table. Instants for the last 12 hours (configurable) is in the
* ActiveTimeline and the rest are Archived. ActiveTimeline is a special timeline that allows for creation of instants
* on the timeline.
* <p>

View File

@@ -38,7 +38,7 @@ import java.util.function.Function;
import java.util.stream.Collectors;
/**
* Represents the Archived Timeline for the HoodieDataset. Instants for the last 12 hours (configurable) is in the
* Represents the Archived Timeline for the Hoodie table. Instants for the last 12 hours (configurable) is in the
* ActiveTimeline and the rest are in ArchivedTimeline.
* <p>
* </p>

View File

@@ -30,7 +30,7 @@ import java.util.Map;
import java.util.Objects;
/**
* A Hoodie Instant represents a action done on a hoodie dataset. All actions start with a inflight instant and then
* A Hoodie Instant represents a action done on a hoodie table. All actions start with a inflight instant and then
* create a completed instant after done.
*
* @see HoodieTimeline

View File

@@ -60,7 +60,7 @@ import java.util.stream.Stream;
/**
* Common thread-safe implementation for multiple TableFileSystemView Implementations. Provides uniform handling of (a)
* Loading file-system views from underlying file-system (b) Pending compaction operations and changing file-system
* views based on that (c) Thread-safety in loading and managing file system views for this dataset. (d) resetting
* views based on that (c) Thread-safety in loading and managing file system views for this table. (d) resetting
* file-system views The actual mechanism of fetching file slices from different view storages is delegated to
* sub-classes.
*/

View File

@@ -30,16 +30,16 @@ import org.apache.log4j.Logger;
import java.util.concurrent.ConcurrentHashMap;
/**
* A container that can potentially hold one or more dataset's file-system views. There is one view for each dataset.
* A container that can potentially hold one or more table's file-system views. There is one view for each table.
* This is a view built against a timeline containing completed actions. In an embedded timeline-server mode, this
* typically holds only one dataset's view. In a stand-alone server mode, this can hold more than one dataset's views.
* typically holds only one table's view. In a stand-alone server mode, this can hold more than one table's views.
*
* FileSystemView can be stored "locally" using the following storage mechanisms: a. In Memory b. Spillable Map c.
* RocksDB
*
* But there can be cases where the file-system view is managed remoted. For example : Embedded Timeline Server). In
* this case, the clients will configure a remote filesystem view client (RemoteHoodieTableFileSystemView) for the
* dataset which can connect to the remote file system view and fetch views. THere are 2 modes here : REMOTE_FIRST and
* table which can connect to the remote file system view and fetch views. THere are 2 modes here : REMOTE_FIRST and
* REMOTE_ONLY REMOTE_FIRST : The file-system view implementation on client side will act as a remote proxy. In case, if
* there is problem (or exceptions) querying remote file-system view, a backup local file-system view(using either one
* of in-memory, spillable, rocksDB) is used to server file-system view queries REMOTE_ONLY : In this case, there is no
@@ -102,11 +102,11 @@ public class FileSystemViewManager {
// FACTORY METHODS FOR CREATING FILE-SYSTEM VIEWS
/**
* Create RocksDB based file System view for a dataset.
* Create RocksDB based file System view for a table.
*
* @param conf Hadoop Configuration
* @param viewConf View Storage Configuration
* @param basePath Base Path of dataset
* @param basePath Base Path of table
* @return
*/
private static RocksDbBasedFileSystemView createRocksDBBasedFileSystemView(SerializableConfiguration conf,
@@ -117,11 +117,11 @@ public class FileSystemViewManager {
}
/**
* Create a spillable Map based file System view for a dataset.
* Create a spillable Map based file System view for a table.
*
* @param conf Hadoop Configuration
* @param viewConf View Storage Configuration
* @param basePath Base Path of dataset
* @param basePath Base Path of table
* @return
*/
private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystemView(SerializableConfiguration conf,
@@ -133,11 +133,11 @@ public class FileSystemViewManager {
}
/**
* Create an in-memory file System view for a dataset.
* Create an in-memory file System view for a table.
*
* @param conf Hadoop Configuration
* @param viewConf View Storage Configuration
* @param basePath Base Path of dataset
* @param basePath Base Path of table
* @return
*/
private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf,
@@ -149,11 +149,11 @@ public class FileSystemViewManager {
}
/**
* Create a remote file System view for a dataset.
* Create a remote file System view for a table.
*
* @param conf Hadoop Configuration
* @param viewConf View Storage Configuration
* @param metaClient Hoodie Table MetaClient for the dataset.
* @param metaClient Hoodie Table MetaClient for the table.
* @return
*/
private static RemoteHoodieTableFileSystemView createRemoteFileSystemView(SerializableConfiguration conf,

View File

@@ -94,7 +94,7 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
public static final String TIMELINE = String.format("%s/%s", BASE_URL, "timeline/instants/all");
// POST Requests
public static final String REFRESH_DATASET = String.format("%s/%s", BASE_URL, "refresh/");
public static final String REFRESH_TABLE = String.format("%s/%s", BASE_URL, "refresh/");
public static final String PARTITION_PARAM = "partition";
public static final String BASEPATH_PARAM = "basepath";
@@ -381,7 +381,7 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
public boolean refresh() {
Map<String, String> paramsMap = getParams();
try {
return executeRequest(REFRESH_DATASET, paramsMap, new TypeReference<Boolean>() {}, RequestMethod.POST);
return executeRequest(REFRESH_TABLE, paramsMap, new TypeReference<Boolean>() {}, RequestMethod.POST);
} catch (IOException e) {
throw new HoodieRemoteException(e);
}

View File

@@ -46,7 +46,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* A file-system view implementation on top of embedded Rocks DB store. For each DataSet : 3 column Family is added for
* A file-system view implementation on top of embedded Rocks DB store. For each table : 3 column Family is added for
* storing (1) File-Slices and Data Files for View lookups (2) Pending compaction operations (3) Partitions tracked
*
* Fine-grained retrieval API to fetch latest file-slice and data-file which are common operations for
@@ -135,7 +135,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
@Override
protected void resetViewState() {
LOG.info("Deleting all rocksdb data associated with dataset filesystem view");
LOG.info("Deleting all rocksdb data associated with table filesystem view");
rocksDB.close();
rocksDB = new RocksDBDAO(metaClient.getBasePath(), config.getRocksdbBasePath());
}

View File

@@ -29,7 +29,7 @@ import java.util.List;
/**
* Helper class to generate Key and column names for rocksdb based view
*
* For RocksDB, 3 colFamilies are used for storing file-system view for each dataset. (a) View (b) Partitions Cached (c)
* For RocksDB, 3 colFamilies are used for storing file-system view for each table. (a) View (b) Partitions Cached (c)
* Pending Compactions
*
*

View File

@@ -22,7 +22,7 @@ import java.io.IOException;
/**
* <p>
* Exception thrown for dataset IO-related failures.
* Exception thrown for table IO-related failures.
* </p>
*/
public class HoodieIOException extends HoodieException {

View File

@@ -19,15 +19,15 @@
package org.apache.hudi.exception;
/**
* Exception thrown to indicate that a hoodie dataset is invalid.
* Exception thrown to indicate that a hoodie table is invalid.
*/
public class InvalidDatasetException extends HoodieException {
public class InvalidTableException extends HoodieException {
public InvalidDatasetException(String basePath) {
public InvalidTableException(String basePath) {
super(getErrorMessage(basePath));
}
private static String getErrorMessage(String basePath) {
return "Invalid Hoodie Dataset. " + basePath;
return "Invalid Hoodie Table. " + basePath;
}
}

View File

@@ -25,35 +25,35 @@ import java.io.IOException;
/**
* <p>
* Exception thrown to indicate that a hoodie dataset was not found on the path provided.
* Exception thrown to indicate that a hoodie table was not found on the path provided.
* <p>
*/
public class DatasetNotFoundException extends HoodieException {
public class TableNotFoundException extends HoodieException {
public DatasetNotFoundException(String basePath) {
public TableNotFoundException(String basePath) {
super(getErrorMessage(basePath));
}
private static String getErrorMessage(String basePath) {
return "Hoodie dataset not found in path " + basePath;
return "Hoodie table not found in path " + basePath;
}
public static void checkValidDataset(FileSystem fs, Path basePathDir, Path metaPathDir)
throws DatasetNotFoundException {
public static void checkTableValidity(FileSystem fs, Path basePathDir, Path metaPathDir)
throws TableNotFoundException {
// Check if the base path is found
try {
if (!fs.exists(basePathDir) || !fs.isDirectory(basePathDir)) {
throw new DatasetNotFoundException(basePathDir.toString());
throw new TableNotFoundException(basePathDir.toString());
}
// Check if the meta path is found
if (!fs.exists(metaPathDir) || !fs.isDirectory(metaPathDir)) {
throw new DatasetNotFoundException(metaPathDir.toString());
throw new TableNotFoundException(metaPathDir.toString());
}
} catch (IllegalArgumentException e) {
// if the base path is file:///, then we have a IllegalArgumentException
throw new DatasetNotFoundException(metaPathDir.toString());
throw new TableNotFoundException(metaPathDir.toString());
} catch (IOException e) {
throw new HoodieIOException("Could not check if dataset " + basePathDir + " is valid dataset", e);
throw new HoodieIOException("Could not check if " + basePathDir + " is a valid table", e);
}
}
}

View File

@@ -118,7 +118,7 @@ public class HoodieTestUtils {
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName());
return HoodieTableMetaClient.initDatasetAndGetMetaClient(hadoopConf, basePath, properties);
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties);
}
public static String makeNewCommitTime() {

View File

@@ -353,7 +353,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
* @param isDeltaCommit is Delta Commit ?
* @param instantsToFiles List of files associated with each instant
* @param rolledBackInstants List of rolled-back instants
* @param emptyRestoreInstant Restore instant at which dataset becomes empty
* @param emptyRestoreInstant Restore instant at which table becomes empty
*/
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants, boolean isDeltaCommit,
Map<String, List<String>> instantsToFiles, List<String> rolledBackInstants, String emptyRestoreInstant,

View File

@@ -25,9 +25,9 @@ import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.InvalidDatasetException;
import org.apache.hudi.exception.InvalidTableException;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
@@ -54,8 +54,8 @@ import java.util.stream.Collectors;
/**
* HoodieInputFormat which understands the Hoodie File Structure and filters files based on the Hoodie Mode. If paths
* that does not correspond to a hoodie dataset then they are passed in as is (as what FileInputFormat.listStatus()
* would do). The JobConf could have paths from multipe Hoodie/Non-Hoodie datasets
* that does not correspond to a hoodie table then they are passed in as is (as what FileInputFormat.listStatus()
* would do). The JobConf could have paths from multipe Hoodie/Non-Hoodie tables
*/
@UseFileSplitsFromInputFormat
public class HoodieParquetInputFormat extends MapredParquetInputFormat implements Configurable {
@@ -159,7 +159,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
try {
metadata = getTableMetaClient(status.getPath().getFileSystem(conf), status.getPath().getParent());
nonHoodieBasePath = null;
} catch (DatasetNotFoundException | InvalidDatasetException e) {
} catch (TableNotFoundException | InvalidTableException e) {
LOG.info("Handling a non-hoodie path " + status.getPath());
metadata = null;
nonHoodieBasePath = status.getPath().getParent().toString();

View File

@@ -22,7 +22,7 @@ import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.conf.Configuration;
@@ -39,11 +39,11 @@ import java.util.List;
import java.util.stream.Collectors;
/**
* Given a path is a part of - Hoodie dataset = accepts ONLY the latest version of each path - Non-Hoodie dataset = then
* Given a path is a part of - Hoodie table = accepts ONLY the latest version of each path - Non-Hoodie table = then
* always accept
* <p>
* We can set this filter, on a query engine's Hadoop Config and if it respects path filters, then you should be able to
* query both hoodie and non-hoodie datasets as you would normally do.
* query both hoodie and non-hoodie tables as you would normally do.
* <p>
* hadoopConf.setClass("mapreduce.input.pathFilter.class", org.apache.hudi.hadoop .HoodieROTablePathFilter.class,
* org.apache.hadoop.fs.PathFilter.class)
@@ -59,7 +59,7 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
private HashMap<String, HashSet<Path>> hoodiePathCache;
/**
* Paths that are known to be non-hoodie datasets.
* Paths that are known to be non-hoodie tables.
*/
private HashSet<String> nonHoodiePathCache;
@@ -156,7 +156,7 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
hoodiePathCache.get(folder.toString()).contains(path)));
}
return hoodiePathCache.get(folder.toString()).contains(path);
} catch (DatasetNotFoundException e) {
} catch (TableNotFoundException e) {
// Non-hoodie path, accept it.
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("(1) Caching non-hoodie path under %s \n", folder.toString()));

View File

@@ -60,7 +60,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Input Format, that provides a real-time view of data in a Hoodie dataset.
* Input Format, that provides a real-time view of data in a Hoodie table.
*/
@UseFileSplitsFromInputFormat
public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat implements Configurable {

View File

@@ -45,7 +45,7 @@ public class InputFormatTestUtil {
private static String TEST_WRITE_TOKEN = "1-0-1";
public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles, String commitNumber)
public static File prepareTable(TemporaryFolder basePath, int numberOfFiles, String commitNumber)
throws IOException {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
@@ -105,7 +105,7 @@ public class InputFormatTestUtil {
return new Schema.Parser().parse(InputFormatTestUtil.class.getResourceAsStream(location));
}
public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema, int numberOfFiles,
public static File prepareParquetTable(TemporaryFolder basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws IOException {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
@@ -114,7 +114,7 @@ public class InputFormatTestUtil {
return partitionPath;
}
public static File prepareSimpleParquetDataset(TemporaryFolder basePath, Schema schema, int numberOfFiles,
public static File prepareSimpleParquetTable(TemporaryFolder basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws Exception {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
@@ -123,7 +123,7 @@ public class InputFormatTestUtil {
return partitionPath;
}
public static File prepareNonPartitionedParquetDataset(TemporaryFolder baseDir, Schema schema, int numberOfFiles,
public static File prepareNonPartitionedParquetTable(TemporaryFolder baseDir, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws IOException {
baseDir.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), baseDir.getRoot().toString());

View File

@@ -56,7 +56,7 @@ public class TestHoodieInputFormat {
@Test
public void testInputFormatLoad() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
File partitionDir = InputFormatTestUtil.prepareTable(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
// Add the paths
@@ -72,7 +72,7 @@ public class TestHoodieInputFormat {
@Test
public void testInputFormatUpdates() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
File partitionDir = InputFormatTestUtil.prepareTable(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
// Add the paths
@@ -99,7 +99,7 @@ public class TestHoodieInputFormat {
@Test
public void testIncrementalSimple() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
File partitionDir = InputFormatTestUtil.prepareTable(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
// Add the paths
@@ -115,7 +115,7 @@ public class TestHoodieInputFormat {
@Test
public void testIncrementalWithMultipleCommits() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareDataset(basePath, 10, "100");
File partitionDir = InputFormatTestUtil.prepareTable(basePath, 10, "100");
InputFormatTestUtil.commit(basePath, "100");
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -166,7 +166,7 @@ public class TestHoodieInputFormat {
// initial commit
Schema schema = InputFormatTestUtil.readSchema("/sample1.avsc");
String commit1 = "20160628071126";
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 10, commit1);
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 10, commit1);
InputFormatTestUtil.commit(basePath, commit1);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());

View File

@@ -34,7 +34,7 @@ public class TestRecordReaderValueIterator {
@Test
public void testValueIterator() {
String[] values = new String[] {"hoodie", "efficient", "new project", "realtime", "spark", "dataset",};
String[] values = new String[] {"hoodie", "efficient", "new project", "realtime", "spark", "table",};
List<Pair<Integer, String>> entries =
IntStream.range(0, values.length).boxed().map(idx -> Pair.of(idx, values[idx])).collect(Collectors.toList());
TestRecordReader reader = new TestRecordReader(entries);

View File

@@ -185,8 +185,8 @@ public class TestHoodieRealtimeRecordReader {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir = partitioned ? InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 100, baseInstant)
: InputFormatTestUtil.prepareNonPartitionedParquetDataset(basePath, schema, 1, 100, baseInstant);
File partitionDir = partitioned ? InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant)
: InputFormatTestUtil.prepareNonPartitionedParquetTable(basePath, schema, 1, 100, baseInstant);
InputFormatTestUtil.commit(basePath, baseInstant);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -269,7 +269,7 @@ public class TestHoodieRealtimeRecordReader {
final int numRecords = 1000;
final int firstBatchLastRecordKey = numRecords - 1;
final int secondBatchLastRecordKey = 2 * numRecords - 1;
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, numRecords, commitTime);
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -343,7 +343,7 @@ public class TestHoodieRealtimeRecordReader {
String commitTime = "100";
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numberOfRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -470,7 +470,7 @@ public class TestHoodieRealtimeRecordReader {
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir =
InputFormatTestUtil.prepareSimpleParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
InputFormatTestUtil.prepareSimpleParquetTable(basePath, schema, 1, numberOfRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());

View File

@@ -44,7 +44,7 @@ public class HiveSyncConfig implements Serializable {
@Parameter(names = {"--jdbc-url"}, description = "Hive jdbc connect url", required = true)
public String jdbcUrl;
@Parameter(names = {"--base-path"}, description = "Basepath of hoodie dataset to sync", required = true)
@Parameter(names = {"--base-path"}, description = "Basepath of hoodie table to sync", required = true)
public String basePath;
@Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by")

View File

@@ -20,7 +20,7 @@ package org.apache.hudi.hive;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.InvalidDatasetException;
import org.apache.hudi.exception.InvalidTableException;
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
@@ -43,7 +43,7 @@ import java.util.Map;
import java.util.stream.Collectors;
/**
* Tool to sync a hoodie HDFS dataset with a hive metastore table. Either use it as a api
* Tool to sync a hoodie HDFS table with a hive metastore table. Either use it as a api
* HiveSyncTool.syncHoodieTable(HiveSyncConfig) or as a command line java -cp hoodie-hive.jar HiveSyncTool [args]
* <p>
* This utility will get the schema from the latest commit and will sync hive table schema Also this will sync the
@@ -80,7 +80,7 @@ public class HiveSyncTool {
break;
default:
LOG.error("Unknown table type " + hoodieHiveClient.getTableType());
throw new InvalidDatasetException(hoodieHiveClient.getBasePath());
throw new InvalidTableException(hoodieHiveClient.getBasePath());
}
} catch (RuntimeException re) {
LOG.error("Got runtime exception when hive syncing", re);
@@ -95,7 +95,7 @@ public class HiveSyncTool {
// Check if the necessary table exists
boolean tableExists = hoodieHiveClient.doesTableExist();
// Get the parquet schema for this dataset looking at the latest commit
// Get the parquet schema for this table looking at the latest commit
MessageType schema = hoodieHiveClient.getDataSchema();
// Sync schema if needed
syncSchema(tableExists, isRealTime, schema);
@@ -146,7 +146,7 @@ public class HiveSyncTool {
ParquetHiveSerDe.class.getName());
}
} else {
// Check if the dataset schema has evolved
// Check if the table schema has evolved
Map<String, String> tableSchema = hoodieHiveClient.getTableSchema();
SchemaDifference schemaDiff = SchemaUtil.getSchemaDifference(schema, tableSchema, cfg.partitionFields);
if (!schemaDiff.isEmpty()) {

View File

@@ -30,7 +30,7 @@ import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.InvalidDatasetException;
import org.apache.hudi.exception.InvalidTableException;
import org.apache.hudi.hive.util.SchemaUtil;
import com.google.common.base.Preconditions;
@@ -337,10 +337,10 @@ public class HoodieHiveClient {
}
/**
* Gets the schema for a hoodie dataset. Depending on the type of table, read from any file written in the latest
* Gets the schema for a hoodie table. Depending on the type of table, read from any file written in the latest
* commit. We will assume that the schema has not changed within a single atomic write.
*
* @return Parquet schema for this dataset
* @return Parquet schema for this table
*/
@SuppressWarnings("WeakerAccess")
public MessageType getDataSchema() {
@@ -350,12 +350,12 @@ public class HoodieHiveClient {
// If this is COW, get the last commit and read the schema from a file written in the
// last commit
HoodieInstant lastCommit =
activeTimeline.lastInstant().orElseThrow(() -> new InvalidDatasetException(syncConfig.basePath));
activeTimeline.lastInstant().orElseThrow(() -> new InvalidTableException(syncConfig.basePath));
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(activeTimeline.getInstantDetails(lastCommit).get(), HoodieCommitMetadata.class);
String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny()
.orElseThrow(() -> new IllegalArgumentException("Could not find any data file written for commit "
+ lastCommit + ", could not get schema for dataset " + metaClient.getBasePath() + ", Metadata :"
+ lastCommit + ", could not get schema for table " + metaClient.getBasePath() + ", Metadata :"
+ commitMetadata));
return readSchemaFromDataFile(new Path(filePath));
case MERGE_ON_READ:
@@ -390,7 +390,7 @@ public class HoodieHiveClient {
.filter(s -> s.contains((metaClient.getTableConfig().getROFileFormat().getFileExtension())))
.findAny().map(f -> Pair.of(f, HoodieFileFormat.PARQUET)).orElseThrow(() -> {
return new IllegalArgumentException("Could not find any data file written for commit "
+ lastDeltaInstant + ", could not get schema for dataset " + metaClient.getBasePath()
+ lastDeltaInstant + ", could not get schema for table " + metaClient.getBasePath()
+ ", CommitMetadata :" + commitMetadata);
});
});
@@ -408,10 +408,10 @@ public class HoodieHiveClient {
}
default:
LOG.error("Unknown table type " + tableType);
throw new InvalidDatasetException(syncConfig.basePath);
throw new InvalidTableException(syncConfig.basePath);
}
} catch (IOException e) {
throw new HoodieHiveSyncException("Failed to get dataset schema for " + syncConfig.tableName, e);
throw new HoodieHiveSyncException("Failed to get table schema for " + syncConfig.tableName, e);
}
}
@@ -428,7 +428,7 @@ public class HoodieHiveClient {
.fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get(), HoodieCommitMetadata.class);
String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny()
.orElseThrow(() -> new IllegalArgumentException("Could not find any data file written for compaction "
+ lastCompactionCommit + ", could not get schema for dataset " + metaClient.getBasePath()));
+ lastCompactionCommit + ", could not get schema for table " + metaClient.getBasePath()));
return readSchemaFromDataFile(new Path(filePath));
}

View File

@@ -95,7 +95,7 @@ public class SchemaUtil {
expectedType = expectedType.replaceAll("`", "");
if (!tableColumnType.equalsIgnoreCase(expectedType)) {
// check for incremental datasets, the schema type change is allowed as per evolution
// check for incremental queries, the schema type change is allowed as per evolution
// rules
if (!isSchemaTypeUpdateAllowed(tableColumnType, expectedType)) {
throw new HoodieHiveSyncException("Could not convert field Type from " + tableColumnType + " to "

View File

@@ -154,7 +154,7 @@ public class TestHiveSyncTool {
public void testBasicSync() throws Exception {
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
String commitTime = "100";
TestUtil.createCOWDataset(commitTime, 5);
TestUtil.createCOWTable(commitTime, 5);
HoodieHiveClient hiveClient =
new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially",
@@ -164,7 +164,7 @@ public class TestHiveSyncTool {
tool.syncHoodieTable();
assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + " should exist after sync completes",
hiveClient.doesTableExist());
assertEquals("Hive Schema should match the dataset schema + partition field", hiveClient.getTableSchema().size(),
assertEquals("Hive Schema should match the table schema + partition field", hiveClient.getTableSchema().size(),
hiveClient.getDataSchema().getColumns().size() + 1);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClient.scanTablePartitions().size());
@@ -176,7 +176,7 @@ public class TestHiveSyncTool {
public void testSyncIncremental() throws Exception {
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
String commitTime1 = "100";
TestUtil.createCOWDataset(commitTime1, 5);
TestUtil.createCOWTable(commitTime1, 5);
HoodieHiveClient hiveClient =
new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
// Lets do the sync
@@ -214,7 +214,7 @@ public class TestHiveSyncTool {
public void testSyncIncrementalWithSchemaEvolution() throws Exception {
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
String commitTime1 = "100";
TestUtil.createCOWDataset(commitTime1, 5);
TestUtil.createCOWTable(commitTime1, 5);
HoodieHiveClient hiveClient =
new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
// Lets do the sync
@@ -250,7 +250,7 @@ public class TestHiveSyncTool {
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
String commitTime = "100";
String deltaCommitTime = "101";
TestUtil.createMORDataset(commitTime, deltaCommitTime, 5);
TestUtil.createMORTable(commitTime, deltaCommitTime, 5);
HoodieHiveClient hiveClient =
new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially",
@@ -261,7 +261,7 @@ public class TestHiveSyncTool {
assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + " should exist after sync completes",
hiveClient.doesTableExist());
assertEquals("Hive Schema should match the dataset schema + partition field", hiveClient.getTableSchema().size(),
assertEquals("Hive Schema should match the table schema + partition field", hiveClient.getTableSchema().size(),
SchemaTestUtil.getSimpleSchema().getFields().size() + 1);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClient.scanTablePartitions().size());
@@ -280,11 +280,11 @@ public class TestHiveSyncTool {
tool.syncHoodieTable();
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
assertEquals("Hive Schema should match the evolved dataset schema + partition field",
assertEquals("Hive Schema should match the evolved table schema + partition field",
hiveClient.getTableSchema().size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + 1);
// Sync should add the one partition
assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be 103", deltaCommitTime2,
assertEquals("The last commit that was synced should be 103", deltaCommitTime2,
hiveClient.getLastCommitTimeSynced().get());
}
@@ -295,7 +295,7 @@ public class TestHiveSyncTool {
String deltaCommitTime = "101";
String roTablename = TestUtil.hiveSyncConfig.tableName;
TestUtil.hiveSyncConfig.tableName = TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE;
TestUtil.createMORDataset(commitTime, deltaCommitTime, 5);
TestUtil.createMORTable(commitTime, deltaCommitTime, 5);
HoodieHiveClient hiveClientRT =
new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
@@ -309,11 +309,11 @@ public class TestHiveSyncTool {
assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE
+ " should exist after sync completes", hiveClientRT.doesTableExist());
assertEquals("Hive Schema should match the dataset schema + partition field", hiveClientRT.getTableSchema().size(),
assertEquals("Hive Schema should match the table schema + partition field", hiveClientRT.getTableSchema().size(),
SchemaTestUtil.getSimpleSchema().getFields().size() + 1);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClientRT.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", deltaCommitTime,
assertEquals("The last commit that was synced should be updated in the TBLPROPERTIES", deltaCommitTime,
hiveClientRT.getLastCommitTimeSynced().get());
// Now lets create more parititions and these are the only ones which needs to be synced
@@ -328,7 +328,7 @@ public class TestHiveSyncTool {
tool.syncHoodieTable();
hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
assertEquals("Hive Schema should match the evolved dataset schema + partition field",
assertEquals("Hive Schema should match the evolved table schema + partition field",
hiveClientRT.getTableSchema().size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + 1);
// Sync should add the one partition
assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClientRT.scanTablePartitions().size());
@@ -341,7 +341,7 @@ public class TestHiveSyncTool {
public void testMultiPartitionKeySync() throws Exception {
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
String commitTime = "100";
TestUtil.createCOWDataset(commitTime, 5);
TestUtil.createCOWTable(commitTime, 5);
HiveSyncConfig hiveSyncConfig = HiveSyncConfig.copy(TestUtil.hiveSyncConfig);
hiveSyncConfig.partitionValueExtractorClass = MultiPartKeysValueExtractor.class.getCanonicalName();
@@ -355,7 +355,7 @@ public class TestHiveSyncTool {
HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
tool.syncHoodieTable();
assertTrue("Table " + hiveSyncConfig.tableName + " should exist after sync completes", hiveClient.doesTableExist());
assertEquals("Hive Schema should match the dataset schema + partition fields", hiveClient.getTableSchema().size(),
assertEquals("Hive Schema should match the table schema + partition fields", hiveClient.getTableSchema().size(),
hiveClient.getDataSchema().getColumns().size() + 3);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClient.scanTablePartitions().size());

View File

@@ -152,7 +152,7 @@ public class TestUtil {
}
}
static void createCOWDataset(String commitTime, int numberOfPartitions)
static void createCOWTable(String commitTime, int numberOfPartitions)
throws IOException, InitializationError, URISyntaxException {
Path path = new Path(hiveSyncConfig.basePath);
FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
@@ -166,7 +166,7 @@ public class TestUtil {
createCommitFile(commitMetadata, commitTime);
}
static void createMORDataset(String commitTime, String deltaCommitTime, int numberOfPartitions)
static void createMORTable(String commitTime, String deltaCommitTime, int numberOfPartitions)
throws IOException, InitializationError, URISyntaxException, InterruptedException {
Path path = new Path(hiveSyncConfig.basePath);
FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));

View File

@@ -28,7 +28,7 @@ import org.apache.hudi.common.util.TypedProperties;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.hive.HiveSyncConfig;
@@ -186,8 +186,8 @@ public class DataSourceUtils {
client = new HoodieReadClient<>(jssc, writeConfig, timelineService);
return client.tagLocation(incomingHoodieRecords)
.filter(r -> !((HoodieRecord<HoodieRecordPayload>) r).isCurrentLocationKnown());
} catch (DatasetNotFoundException e) {
// this will be executed when there is no hoodie dataset yet
} catch (TableNotFoundException e) {
// this will be executed when there is no hoodie table yet
// so no dups to drop
return incomingHoodieRecords;
} finally {

View File

@@ -36,7 +36,7 @@ import java.util.stream.Collectors;
public class HoodieDataSourceHelpers {
/**
* Checks if the Hoodie dataset has new data since given timestamp. This can be subsequently fed to an incremental
* Checks if the Hoodie table has new data since given timestamp. This can be subsequently fed to an incremental
* view read, to perform incremental processing.
*/
public static boolean hasNewCommits(FileSystem fs, String basePath, String commitTimestamp) {

View File

@@ -38,7 +38,7 @@ import java.io.IOException;
* - For updates against the source table, records contain full after image with `Op=U`
* - For deletes against the source table, records contain full before image with `Op=D`
*
* This payload implementation will issue matching insert, delete, updates against the hudi dataset
* This payload implementation will issue matching insert, delete, updates against the hudi table
*
*/
public class AWSDmsAvroPayload extends OverwriteWithLatestAvroPayload {

View File

@@ -26,7 +26,7 @@ import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
*/
/**
* Options supported for reading hoodie datasets.
* Options supported for reading hoodie tables.
*/
object DataSourceReadOptions {
/**
@@ -73,7 +73,7 @@ object DataSourceReadOptions {
}
/**
* Options supported for writing hoodie datasets.
* Options supported for writing hoodie tables.
*/
object DataSourceWriteOptions {
/**
@@ -100,7 +100,7 @@ object DataSourceWriteOptions {
val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL
/**
* Hive table name, to register the dataset into.
* Hive table name, to register the table into.
*
* Default: None (mandatory)
*/

View File

@@ -29,7 +29,7 @@ import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
/**
* Hoodie Spark Datasource, for reading and writing hoodie datasets
* Hoodie Spark Datasource, for reading and writing hoodie tables
*
*/
class DefaultSource extends RelationProvider

View File

@@ -105,19 +105,19 @@ private[hudi] object HoodieSparkSqlWriter {
// Handle various save modes
if (mode == SaveMode.ErrorIfExists && exists) {
throw new HoodieException(s"hoodie dataset at $basePath already exists.")
throw new HoodieException(s"hoodie table at $basePath already exists.")
}
if (mode == SaveMode.Ignore && exists) {
log.warn(s"hoodie dataset at $basePath already exists. Ignoring & not performing actual writes.")
log.warn(s"hoodie table at $basePath already exists. Ignoring & not performing actual writes.")
(true, common.util.Option.empty())
}
if (mode == SaveMode.Overwrite && exists) {
log.warn(s"hoodie dataset at $basePath already exists. Deleting existing data & overwriting with new data.")
log.warn(s"hoodie table at $basePath already exists. Deleting existing data & overwriting with new data.")
fs.delete(basePath, true)
exists = false
}
// Create the dataset if not present
// Create the table if not present
if (!exists) {
HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, storageType,
tblName.get, "archived")
@@ -164,7 +164,7 @@ private[hudi] object HoodieSparkSqlWriter {
val hoodieKeysToDelete = genericRecords.map(gr => keyGenerator.getKey(gr)).toJavaRDD()
if (!exists) {
throw new HoodieException(s"hoodie dataset at $basePath does not exist")
throw new HoodieException(s"hoodie table at $basePath does not exist")
}
// Create a HoodieWriteClient & issue the delete.

View File

@@ -48,9 +48,9 @@ class IncrementalRelation(val sqlContext: SQLContext,
val fs = new Path(basePath).getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
val metaClient = new HoodieTableMetaClient(sqlContext.sparkContext.hadoopConfiguration, basePath, true)
// MOR datasets not supported yet
// MOR tables not supported yet
if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) {
throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets")
throw new HoodieException("Incremental view not implemented yet, for merge-on-read tables")
}
// TODO : Figure out a valid HoodieWriteConfig
val hoodieTable = HoodieTable.getHoodieTable(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(),

View File

@@ -45,7 +45,7 @@ import java.util.ArrayList;
import java.util.List;
/**
* Sample program that writes & reads hoodie datasets via the Spark datasource.
* Sample program that writes & reads hoodie tables via the Spark datasource.
*/
public class HoodieJavaApp {

View File

@@ -45,7 +45,7 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
/**
* Sample program that writes & reads hoodie datasets via the Spark datasource streaming.
* Sample program that writes & reads hoodie tables via the Spark datasource streaming.
*/
public class HoodieJavaStreamingApp {

View File

@@ -77,7 +77,7 @@ public class FileSystemViewHandler {
}
/**
* Determines if local view of dataset's timeline is behind that of client's view.
* Determines if local view of table's timeline is behind that of client's view.
*/
private boolean isLocalViewBehind(Context ctx) {
String basePath = ctx.queryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM);
@@ -284,9 +284,9 @@ public class FileSystemViewHandler {
writeValueAsString(ctx, dtos);
}, true));
app.post(RemoteHoodieTableFileSystemView.REFRESH_DATASET, new ViewHandler(ctx -> {
app.post(RemoteHoodieTableFileSystemView.REFRESH_TABLE, new ViewHandler(ctx -> {
boolean success = sliceHandler
.refreshDataset(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow());
.refreshTable(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow());
writeValueAsString(ctx, success);
}, false));
}

View File

@@ -89,7 +89,7 @@ public class FileSliceHandler extends Handler {
.collect(Collectors.toList());
}
public boolean refreshDataset(String basePath) {
public boolean refreshTable(String basePath) {
viewManager.clearFileSystemView(basePath);
return true;
}

View File

@@ -134,7 +134,7 @@ public class HDFSParquetImporter implements Serializable {
Properties properties = new Properties();
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName);
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType);
HoodieTableMetaClient.initDatasetAndGetMetaClient(jsc.hadoopConfiguration(), cfg.targetPath, properties);
HoodieTableMetaClient.initTableAndGetMetaClient(jsc.hadoopConfiguration(), cfg.targetPath, properties);
HoodieWriteClient client =
UtilHelpers.createHoodieClient(jsc, cfg.targetPath, schemaStr, cfg.parallelism, Option.empty(), props);
@@ -190,7 +190,7 @@ public class HDFSParquetImporter implements Serializable {
}
/**
* Imports records to Hoodie dataset.
* Imports records to Hoodie table.
*
* @param client Hoodie Client
* @param instantTime Instant Time
@@ -243,9 +243,9 @@ public class HDFSParquetImporter implements Serializable {
@Parameter(names = {"--command", "-c"}, description = "Write command Valid values are insert(default)/upsert/bulkinsert",
required = false, validateValueWith = CommandValidator.class)
public String command = "INSERT";
@Parameter(names = {"--src-path", "-sp"}, description = "Base path for the input dataset", required = true)
@Parameter(names = {"--src-path", "-sp"}, description = "Base path for the input table", required = true)
public String srcPath = null;
@Parameter(names = {"--target-path", "-tp"}, description = "Base path for the target hoodie dataset",
@Parameter(names = {"--target-path", "-tp"}, description = "Base path for the target hoodie table",
required = true)
public String targetPath = null;
@Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true)

View File

@@ -234,7 +234,7 @@ public class HiveIncrementalPuller {
}
private String inferCommitTime(FileSystem fs) throws SQLException, IOException {
LOG.info("FromCommitTime not specified. Trying to infer it from Hoodie dataset " + config.targetDb + "."
LOG.info("FromCommitTime not specified. Trying to infer it from Hoodie table " + config.targetDb + "."
+ config.targetTable);
String targetDataLocation = getTableLocation(config.targetDb, config.targetTable);
return scanForCommitTime(fs, targetDataLocation);

View File

@@ -82,7 +82,7 @@ public class HoodieCleaner {
public static class Config implements Serializable {
@Parameter(names = {"--target-base-path"}, description = "base path for the hoodie dataset to be cleaner.",
@Parameter(names = {"--target-base-path"}, description = "base path for the hoodie table to be cleaner.",
required = true)
public String basePath;

View File

@@ -144,7 +144,7 @@ public class HoodieCompactionAdminTool {
@Parameter(names = {"--operation", "-op"}, description = "Operation", required = true)
public Operation operation = Operation.VALIDATE;
@Parameter(names = {"--base-path", "-bp"}, description = "Base path for the dataset", required = true)
@Parameter(names = {"--base-path", "-bp"}, description = "Base path for the table", required = true)
public String basePath = null;
@Parameter(names = {"--instant-time", "-in"}, description = "Compaction Instant time", required = false)
public String compactionInstantTime = null;

View File

@@ -51,7 +51,7 @@ public class HoodieCompactor {
}
public static class Config implements Serializable {
@Parameter(names = {"--base-path", "-sp"}, description = "Base path for the dataset", required = true)
@Parameter(names = {"--base-path", "-sp"}, description = "Base path for the table", required = true)
public String basePath = null;
@Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true)
public String tableName = null;

View File

@@ -224,7 +224,7 @@ public class UtilHelpers {
}
});
if (errors.value() == 0) {
LOG.info(String.format("Dataset imported into hoodie dataset with %s instant time.", instantTime));
LOG.info(String.format("Table imported into hoodie with %s instant time.", instantTime));
return 0;
}
LOG.error(String.format("Import failed with %d errors.", errors.value()));

View File

@@ -42,7 +42,7 @@ import java.util.Properties;
/**
* This is an one-time use class meant for migrating the configuration for "hoodie.compaction.payload.class" in
* .hoodie/hoodie.properties from com.uber.hoodie to org.apache.hudi . It takes in a file containing base-paths for a set
* of hudi datasets and does the migration
* of hudi tables and does the migration
*/
public class UpgradePayloadFromUberToApache implements Serializable {

View File

@@ -79,7 +79,7 @@ import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_REC
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;
/**
* Sync's one batch of data to hoodie dataset.
* Sync's one batch of data to hoodie table.
*/
public class DeltaSync implements Serializable {
@@ -103,12 +103,12 @@ public class DeltaSync implements Serializable {
private transient SchemaProvider schemaProvider;
/**
* Allows transforming source to target dataset before writing.
* Allows transforming source to target table before writing.
*/
private transient Transformer transformer;
/**
* Extract the key for the target dataset.
* Extract the key for the target table.
*/
private KeyGenerator keyGenerator;

View File

@@ -69,8 +69,8 @@ import java.util.stream.IntStream;
/**
* An Utility which can incrementally take the output from {@link HiveIncrementalPuller} and apply it to the target
* dataset. Does not maintain any state, queries at runtime to see how far behind the target dataset is from the source
* dataset. This can be overriden to force sync from a timestamp.
* table. Does not maintain any state, queries at runtime to see how far behind the target table is from the source
* table. This can be overriden to force sync from a timestamp.
*
* In continuous mode, DeltaStreamer runs in loop-mode going through the below operations (a) pull-from-source (b)
* write-to-sink (c) Schedule Compactions if needed (d) Conditionally Sync to Hive each cycle. For MOR table with
@@ -151,8 +151,8 @@ public class HoodieDeltaStreamer implements Serializable {
public static class Config implements Serializable {
@Parameter(names = {"--target-base-path"},
description = "base path for the target hoodie dataset. "
+ "(Will be created if did not exist first time around. If exists, expected to be a hoodie dataset)",
description = "base path for the target hoodie table. "
+ "(Will be created if did not exist first time around. If exists, expected to be a hoodie table)",
required = true)
public String targetBasePath;
@@ -199,7 +199,7 @@ public class HoodieDeltaStreamer implements Serializable {
@Parameter(names = {"--transformer-class"},
description = "subclass of org.apache.hudi.utilities.transform.Transformer"
+ ". Allows transforming raw source dataset to a target dataset (conforming to target schema) before "
+ ". Allows transforming raw source Dataset to a target Dataset (conforming to target schema) before "
+ "writing. Default : Not set. E:g - org.apache.hudi.utilities.transform.SqlQueryBasedTransformer (which "
+ "allows a SQL query templated to be passed as a transformation function)")
public String transformerClassName = null;

View File

@@ -24,7 +24,7 @@ import org.apache.avro.Schema;
import org.apache.spark.api.java.JavaSparkContext;
/**
* Schema provider that will force DeltaStreamer to infer target schema from the dataset. It can be used with SQL or
* Schema provider that will force DeltaStreamer to infer target schema from the table. It can be used with SQL or
* Flattening transformers to avoid having a target schema in the schema registry.
*/
public class NullTargetSchemaRegistryProvider extends SchemaRegistryProvider {

View File

@@ -42,7 +42,7 @@ public abstract class SchemaProvider implements Serializable {
public abstract Schema getSourceSchema();
public Schema getTargetSchema() {
// by default, use source schema as target for hoodie dataset as well
// by default, use source schema as target for hoodie table as well
return getSourceSchema();
}
}

View File

@@ -85,7 +85,7 @@ public class TestHDFSParquetImporter implements Serializable {
* Test successful data import with retries.
*/
@Test
public void testDatasetImportWithRetries() throws Exception {
public void testImportWithRetries() throws Exception {
JavaSparkContext jsc = null;
try {
jsc = getJavaSparkContext();

View File

@@ -29,7 +29,7 @@ import org.apache.hudi.common.util.DFSPropertiesConfiguration;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.TypedProperties;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.exception.DatasetNotFoundException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HoodieHiveClient;
@@ -219,49 +219,49 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
return cfg;
}
static void assertRecordCount(long expected, String datasetPath, SQLContext sqlContext) {
long recordCount = sqlContext.read().format("org.apache.hudi").load(datasetPath).count();
static void assertRecordCount(long expected, String tablePath, SQLContext sqlContext) {
long recordCount = sqlContext.read().format("org.apache.hudi").load(tablePath).count();
assertEquals(expected, recordCount);
}
static List<Row> countsPerCommit(String datasetPath, SQLContext sqlContext) {
return sqlContext.read().format("org.apache.hudi").load(datasetPath).groupBy("_hoodie_commit_time").count()
static List<Row> countsPerCommit(String tablePath, SQLContext sqlContext) {
return sqlContext.read().format("org.apache.hudi").load(tablePath).groupBy("_hoodie_commit_time").count()
.sort("_hoodie_commit_time").collectAsList();
}
static void assertDistanceCount(long expected, String datasetPath, SQLContext sqlContext) {
sqlContext.read().format("org.apache.hudi").load(datasetPath).registerTempTable("tmp_trips");
static void assertDistanceCount(long expected, String tablePath, SQLContext sqlContext) {
sqlContext.read().format("org.apache.hudi").load(tablePath).registerTempTable("tmp_trips");
long recordCount =
sqlContext.sparkSession().sql("select * from tmp_trips where haversine_distance is not NULL").count();
assertEquals(expected, recordCount);
}
static void assertDistanceCountWithExactValue(long expected, String datasetPath, SQLContext sqlContext) {
sqlContext.read().format("org.apache.hudi").load(datasetPath).registerTempTable("tmp_trips");
static void assertDistanceCountWithExactValue(long expected, String tablePath, SQLContext sqlContext) {
sqlContext.read().format("org.apache.hudi").load(tablePath).registerTempTable("tmp_trips");
long recordCount =
sqlContext.sparkSession().sql("select * from tmp_trips where haversine_distance = 1.0").count();
assertEquals(expected, recordCount);
}
static void assertAtleastNCompactionCommits(int minExpected, String datasetPath, FileSystem fs) {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath);
static void assertAtleastNCompactionCommits(int minExpected, String tablePath, FileSystem fs) {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), tablePath);
HoodieTimeline timeline = meta.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
int numCompactionCommits = (int) timeline.getInstants().count();
assertTrue("Got=" + numCompactionCommits + ", exp >=" + minExpected, minExpected <= numCompactionCommits);
}
static void assertAtleastNDeltaCommits(int minExpected, String datasetPath, FileSystem fs) {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath);
static void assertAtleastNDeltaCommits(int minExpected, String tablePath, FileSystem fs) {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), tablePath);
HoodieTimeline timeline = meta.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants();
LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList()));
int numDeltaCommits = (int) timeline.getInstants().count();
assertTrue("Got=" + numDeltaCommits + ", exp >=" + minExpected, minExpected <= numDeltaCommits);
}
static String assertCommitMetadata(String expected, String datasetPath, FileSystem fs, int totalCommits)
static String assertCommitMetadata(String expected, String tablePath, FileSystem fs, int totalCommits)
throws IOException {
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), datasetPath);
HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), tablePath);
HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
HoodieInstant lastInstant = timeline.lastInstant().get();
HoodieCommitMetadata commitMetadata =
@@ -302,9 +302,9 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
@Test
public void testPropsWithInvalidKeyGenerator() throws Exception {
try {
String datasetBasePath = dfsBasePath + "/test_dataset";
String tableBasePath = dfsBasePath + "/test_table";
HoodieDeltaStreamer deltaStreamer =
new HoodieDeltaStreamer(TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT,
new HoodieDeltaStreamer(TestHelpers.makeConfig(tableBasePath, Operation.BULK_INSERT,
TripsWithDistanceTransformer.class.getName(), PROPS_FILENAME_TEST_INVALID, false), jsc);
deltaStreamer.sync();
fail("Should error out when setting the key generator class property to an invalid value");
@@ -316,45 +316,45 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
}
@Test
public void testDatasetCreation() throws Exception {
public void testTableCreation() throws Exception {
try {
dfs.mkdirs(new Path(dfsBasePath + "/not_a_dataset"));
dfs.mkdirs(new Path(dfsBasePath + "/not_a_table"));
HoodieDeltaStreamer deltaStreamer =
new HoodieDeltaStreamer(TestHelpers.makeConfig(dfsBasePath + "/not_a_dataset", Operation.BULK_INSERT), jsc);
new HoodieDeltaStreamer(TestHelpers.makeConfig(dfsBasePath + "/not_a_table", Operation.BULK_INSERT), jsc);
deltaStreamer.sync();
fail("Should error out when pointed out at a dir thats not a dataset");
} catch (DatasetNotFoundException e) {
fail("Should error out when pointed out at a dir thats not a table");
} catch (TableNotFoundException e) {
// expected
LOG.error("Expected error during dataset creation", e);
LOG.error("Expected error during table creation", e);
}
}
@Test
public void testBulkInsertsAndUpserts() throws Exception {
String datasetBasePath = dfsBasePath + "/test_dataset";
String tableBasePath = dfsBasePath + "/test_table";
// Initial bulk insert
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT);
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, Operation.BULK_INSERT);
new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
TestHelpers.assertRecordCount(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", tableBasePath, dfs, 1);
// No new data => no commits.
cfg.sourceLimit = 0;
new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
TestHelpers.assertRecordCount(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", tableBasePath, dfs, 1);
// upsert() #1
cfg.sourceLimit = 2000;
cfg.operation = Operation.UPSERT;
new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(1950, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1950, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2);
List<Row> counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertRecordCount(1950, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1950, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00001", tableBasePath, dfs, 2);
List<Row> counts = TestHelpers.countsPerCommit(tableBasePath + "/*/*.parquet", sqlContext);
assertEquals(1950, counts.stream().mapToLong(entry -> entry.getLong(1)).sum());
}
@@ -369,12 +369,12 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
}
private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception {
String datasetBasePath = dfsBasePath + "/" + tempDir;
String tableBasePath = dfsBasePath + "/" + tempDir;
// Keep it higher than batch-size to test continuous mode
int totalRecords = 3000;
// Initial bulk insert
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.UPSERT);
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, Operation.UPSERT);
cfg.continuousMode = true;
cfg.storageType = tableType.name();
cfg.configs.add(String.format("%s=%d", TestSourceConfig.MAX_UNIQUE_RECORDS_PROP, totalRecords));
@@ -390,13 +390,13 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
TestHelpers.waitTillCondition((r) -> {
if (tableType.equals(HoodieTableType.MERGE_ON_READ)) {
TestHelpers.assertAtleastNDeltaCommits(5, datasetBasePath, dfs);
TestHelpers.assertAtleastNCompactionCommits(2, datasetBasePath, dfs);
TestHelpers.assertAtleastNDeltaCommits(5, tableBasePath, dfs);
TestHelpers.assertAtleastNCompactionCommits(2, tableBasePath, dfs);
} else {
TestHelpers.assertAtleastNCompactionCommits(5, datasetBasePath, dfs);
TestHelpers.assertAtleastNCompactionCommits(5, tableBasePath, dfs);
}
TestHelpers.assertRecordCount(totalRecords + 200, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(totalRecords + 200, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertRecordCount(totalRecords + 200, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(totalRecords + 200, tableBasePath + "/*/*.parquet", sqlContext);
return true;
}, 180);
ds.shutdownGracefully();
@@ -410,71 +410,71 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
*/
@Test
public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() throws Exception {
String datasetBasePath = dfsBasePath + "/test_dataset2";
String downstreamDatasetBasePath = dfsBasePath + "/test_downstream_dataset2";
String tableBasePath = dfsBasePath + "/test_table2";
String downstreamTableBasePath = dfsBasePath + "/test_downstream_table2";
HiveSyncConfig hiveSyncConfig = getHiveSyncConfig(datasetBasePath, "hive_trips");
HiveSyncConfig hiveSyncConfig = getHiveSyncConfig(tableBasePath, "hive_trips");
// Initial bulk insert to ingest to first hudi table
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT,
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, Operation.BULK_INSERT,
SqlQueryBasedTransformer.class.getName(), PROPS_FILENAME_TEST_SOURCE, true);
new HoodieDeltaStreamer(cfg, jsc, dfs, hiveServer.getHiveConf()).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, datasetBasePath + "/*/*.parquet", sqlContext);
String lastInstantForUpstreamTable = TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
TestHelpers.assertRecordCount(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, tableBasePath + "/*/*.parquet", sqlContext);
String lastInstantForUpstreamTable = TestHelpers.assertCommitMetadata("00000", tableBasePath, dfs, 1);
// Now incrementally pull from the above hudi table and ingest to second table
HoodieDeltaStreamer.Config downstreamCfg =
TestHelpers.makeConfigForHudiIncrSrc(datasetBasePath, downstreamDatasetBasePath, Operation.BULK_INSERT,
TestHelpers.makeConfigForHudiIncrSrc(tableBasePath, downstreamTableBasePath, Operation.BULK_INSERT,
true, null);
new HoodieDeltaStreamer(downstreamCfg, jsc, dfs, hiveServer.getHiveConf()).sync();
TestHelpers.assertRecordCount(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata(lastInstantForUpstreamTable, downstreamDatasetBasePath, dfs, 1);
TestHelpers.assertRecordCount(1000, downstreamTableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, downstreamTableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, downstreamTableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata(lastInstantForUpstreamTable, downstreamTableBasePath, dfs, 1);
// No new data => no commits for upstream table
cfg.sourceLimit = 0;
new HoodieDeltaStreamer(cfg, jsc, dfs, hiveServer.getHiveConf()).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
TestHelpers.assertRecordCount(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", tableBasePath, dfs, 1);
// with no change in upstream table, no change in downstream too when pulled.
HoodieDeltaStreamer.Config downstreamCfg1 =
TestHelpers.makeConfigForHudiIncrSrc(datasetBasePath, downstreamDatasetBasePath,
TestHelpers.makeConfigForHudiIncrSrc(tableBasePath, downstreamTableBasePath,
Operation.BULK_INSERT, true, DummySchemaProvider.class.getName());
new HoodieDeltaStreamer(downstreamCfg1, jsc).sync();
TestHelpers.assertRecordCount(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata(lastInstantForUpstreamTable, downstreamDatasetBasePath, dfs, 1);
TestHelpers.assertRecordCount(1000, downstreamTableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1000, downstreamTableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1000, downstreamTableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata(lastInstantForUpstreamTable, downstreamTableBasePath, dfs, 1);
// upsert() #1 on upstream hudi table
cfg.sourceLimit = 2000;
cfg.operation = Operation.UPSERT;
new HoodieDeltaStreamer(cfg, jsc, dfs, hiveServer.getHiveConf()).sync();
TestHelpers.assertRecordCount(1950, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1950, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1950, datasetBasePath + "/*/*.parquet", sqlContext);
lastInstantForUpstreamTable = TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2);
List<Row> counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertRecordCount(1950, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(1950, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(1950, tableBasePath + "/*/*.parquet", sqlContext);
lastInstantForUpstreamTable = TestHelpers.assertCommitMetadata("00001", tableBasePath, dfs, 2);
List<Row> counts = TestHelpers.countsPerCommit(tableBasePath + "/*/*.parquet", sqlContext);
assertEquals(1950, counts.stream().mapToLong(entry -> entry.getLong(1)).sum());
// Incrementally pull changes in upstream hudi table and apply to downstream table
downstreamCfg =
TestHelpers.makeConfigForHudiIncrSrc(datasetBasePath, downstreamDatasetBasePath, Operation.UPSERT,
TestHelpers.makeConfigForHudiIncrSrc(tableBasePath, downstreamTableBasePath, Operation.UPSERT,
false, null);
downstreamCfg.sourceLimit = 2000;
new HoodieDeltaStreamer(downstreamCfg, jsc).sync();
TestHelpers.assertRecordCount(2000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(2000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(2000, downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertRecordCount(2000, downstreamTableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCount(2000, downstreamTableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertDistanceCountWithExactValue(2000, downstreamTableBasePath + "/*/*.parquet", sqlContext);
String finalInstant =
TestHelpers.assertCommitMetadata(lastInstantForUpstreamTable, downstreamDatasetBasePath, dfs, 2);
counts = TestHelpers.countsPerCommit(downstreamDatasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata(lastInstantForUpstreamTable, downstreamTableBasePath, dfs, 2);
counts = TestHelpers.countsPerCommit(downstreamTableBasePath + "/*/*.parquet", sqlContext);
assertEquals(2000, counts.stream().mapToLong(entry -> entry.getLong(1)).sum());
// Test Hive integration
@@ -488,8 +488,8 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
@Test
public void testNullSchemaProvider() throws Exception {
String dataSetBasePath = dfsBasePath + "/test_dataset";
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(dataSetBasePath, Operation.BULK_INSERT,
String tableBasePath = dfsBasePath + "/test_table";
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, Operation.BULK_INSERT,
SqlQueryBasedTransformer.class.getName(), PROPS_FILENAME_TEST_SOURCE, true,
false);
try {
@@ -503,37 +503,37 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
@Test
public void testFilterDupes() throws Exception {
String datasetBasePath = dfsBasePath + "/test_dupes_dataset";
String tableBasePath = dfsBasePath + "/test_dupes_table";
// Initial bulk insert
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(datasetBasePath, Operation.BULK_INSERT);
HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, Operation.BULK_INSERT);
new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(1000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", datasetBasePath, dfs, 1);
TestHelpers.assertRecordCount(1000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00000", tableBasePath, dfs, 1);
// Generate the same 1000 records + 1000 new ones for upsert
cfg.filterDupes = true;
cfg.sourceLimit = 2000;
cfg.operation = Operation.UPSERT;
new HoodieDeltaStreamer(cfg, jsc).sync();
TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2);
TestHelpers.assertRecordCount(2000, tableBasePath + "/*/*.parquet", sqlContext);
TestHelpers.assertCommitMetadata("00001", tableBasePath, dfs, 2);
// 1000 records for commit 00000 & 1000 for commit 00001
List<Row> counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext);
List<Row> counts = TestHelpers.countsPerCommit(tableBasePath + "/*/*.parquet", sqlContext);
assertEquals(1000, counts.get(0).getLong(1));
assertEquals(1000, counts.get(1).getLong(1));
// Test with empty commits
HoodieTableMetaClient mClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), datasetBasePath, true);
HoodieTableMetaClient mClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), tableBasePath, true);
HoodieInstant lastFinished = mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
HoodieDeltaStreamer.Config cfg2 = TestHelpers.makeDropAllConfig(datasetBasePath, Operation.UPSERT);
HoodieDeltaStreamer.Config cfg2 = TestHelpers.makeDropAllConfig(tableBasePath, Operation.UPSERT);
cfg2.filterDupes = true;
cfg2.sourceLimit = 2000;
cfg2.operation = Operation.UPSERT;
cfg2.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP));
HoodieDeltaStreamer ds2 = new HoodieDeltaStreamer(cfg2, jsc);
ds2.sync();
mClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), datasetBasePath, true);
mClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), tableBasePath, true);
HoodieInstant newLastFinished = mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
Assert.assertTrue(HoodieTimeline.compareTimestamps(newLastFinished.getTimestamp(), lastFinished.getTimestamp(),
HoodieTimeline.GREATER));
@@ -599,7 +599,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
}
/**
* Return empty dataset.
* Return empty table.
*/
public static class DropAllTransformer implements Transformer {