[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:
@@ -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());
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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());
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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())
|
||||
|
||||
@@ -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);
|
||||
|
||||
Reference in New Issue
Block a user