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