[HUDI-960] Implementation of the HFile base and log file format. (#1804)
* [HUDI-960] Implementation of the HFile base and log file format. 1. Includes HFileWriter and HFileReader 2. Includes HFileInputFormat for both snapshot and realtime input format for Hive 3. Unit test for new code 4. IT for using HFile format and querying using Hive (Presto and SparkSQL are not supported) Advantage: HFile file format saves data as binary key-value pairs. This implementation chooses the following values: 1. Key = Hoodie Record Key (as bytes) 2. Value = Avro encoded GenericRecord (as bytes) HFile allows efficient lookup of a record by key or range of keys. Hence, this base file format is well suited to applications like RFC-15, RFC-08 which will benefit from the ability to lookup records by key or search in a range of keys without having to read the entire data/log format. Limitations: HFile storage format has certain limitations when used as a general purpose data storage format. 1. Does not have a implemented reader for Presto and SparkSQL 2. Is not a columnar file format and hence may lead to lower compression levels and greater IO on query side due to lack of column pruning Other changes: - Remove databricks/avro from pom - Fix HoodieClientTestUtils from not using scala imports/reflection based conversion etc - Breaking up limitFileSize(), per parquet and hfile base files - Added three new configs for HoodieHFileConfig - prefetchBlocksOnOpen, cacheDataInL1, dropBehindCacheCompaction - Throw UnsupportedException in HFileReader.getRecordKeys() - Updated HoodieCopyOnWriteTable to create the correct merge handle (HoodieSortedMergeHandle for HFile and HoodieMergeHandle otherwise) * Fixing checkstyle Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -1261,7 +1261,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
.insertSplitSize(insertSplitSize).build())
|
||||
.withStorageConfig(
|
||||
HoodieStorageConfig.newBuilder()
|
||||
.limitFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build())
|
||||
.hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200))
|
||||
.parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build())
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -411,7 +411,7 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase {
|
||||
private void checkReadRecords(String instantTime, int numExpectedRecords) throws IOException {
|
||||
if (tableType == HoodieTableType.COPY_ON_WRITE) {
|
||||
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitTimeline();
|
||||
assertEquals(numExpectedRecords, HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, instantTime).count());
|
||||
assertEquals(numExpectedRecords, HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, instantTime));
|
||||
} else {
|
||||
// TODO: This code fails to read records under the following conditions:
|
||||
// 1. No parquet files yet (i.e. no compaction done yet)
|
||||
|
||||
@@ -45,6 +45,9 @@ import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
@@ -89,8 +92,10 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
insertRecords
|
||||
.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
Map<String, HoodieRecord> insertRecordMap = insertRecords.stream()
|
||||
.collect(Collectors.toMap(r -> r.getRecordKey(), Function.identity()));
|
||||
HoodieCreateHandle createHandle =
|
||||
new HoodieCreateHandle(config, "100", table, rowChange1.getPartitionPath(), "f1-0", insertRecords.iterator(), supplier);
|
||||
new HoodieCreateHandle(config, "100", table, rowChange1.getPartitionPath(), "f1-0", insertRecordMap, supplier);
|
||||
createHandle.write();
|
||||
return createHandle.close();
|
||||
}).collect();
|
||||
|
||||
@@ -441,7 +441,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
.withWriteStatusClass(MetadataMergeWriteStatus.class)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
|
||||
@@ -465,7 +465,8 @@ public class TestHBaseIndex extends FunctionalTestHarness {
|
||||
.withParallelism(1, 1)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
|
||||
.withInlineCompaction(false).build())
|
||||
.withAutoCommit(false).withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.withAutoCommit(false).withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE)
|
||||
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder()
|
||||
|
||||
@@ -76,7 +76,8 @@ public class TestHBaseQPSResourceAllocator {
|
||||
.withParallelism(1, 1)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
|
||||
.withInlineCompaction(false).build())
|
||||
.withAutoCommit(false).withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.withAutoCommit(false).withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.hfileMaxFileSize(1000 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table").withIndexConfig(HoodieIndexConfig.newBuilder()
|
||||
.withIndexType(HoodieIndex.IndexType.HBASE).withHBaseIndexConfig(hoodieHBaseIndexConfig).build());
|
||||
}
|
||||
|
||||
@@ -178,7 +178,7 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness {
|
||||
.withWriteStatusClass(MetadataMergeWriteStatus.class)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
|
||||
@@ -310,7 +310,7 @@ public class TestHoodieMergeHandle extends HoodieClientTestHarness {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withBulkInsertParallelism(2).withWriteStatusClass(TestWriteStatus.class);
|
||||
|
||||
@@ -50,6 +50,11 @@ public class TestHoodieFileWriterFactory extends HoodieClientTestBase {
|
||||
parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
|
||||
assertTrue(parquetWriter instanceof HoodieParquetWriter);
|
||||
|
||||
final Path hfilePath = new Path(basePath + "/partition/path/f1_1-0-1_000.hfile");
|
||||
HoodieFileWriter<IndexedRecord> hfileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
|
||||
hfilePath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
|
||||
assertTrue(hfileWriter instanceof HoodieHFileWriter);
|
||||
|
||||
// other file format exception.
|
||||
final Path logPath = new Path(basePath + "/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1");
|
||||
final Throwable thrown = assertThrows(UnsupportedOperationException.class, () -> {
|
||||
|
||||
@@ -49,7 +49,9 @@ import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.HoodieHFileInputFormat;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieHFileRealtimeInputFormat;
|
||||
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
@@ -69,9 +71,9 @@ import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
@@ -100,15 +102,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
|
||||
@TempDir
|
||||
public java.nio.file.Path tempFolder;
|
||||
private HoodieFileFormat baseFileFormat;
|
||||
|
||||
static Stream<HoodieFileFormat> argumentsProvider() {
|
||||
return Stream.of(HoodieFileFormat.PARQUET);
|
||||
}
|
||||
private HoodieFileFormat baseFileFormat;
|
||||
|
||||
public void init(HoodieFileFormat baseFileFormat) throws IOException {
|
||||
this.baseFileFormat = baseFileFormat;
|
||||
|
||||
initDFS();
|
||||
initSparkContexts("TestHoodieMergeOnReadTable");
|
||||
hadoopConf.addResource(dfs.getConf());
|
||||
@@ -122,15 +120,65 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
rtJobConf = new JobConf(hadoopConf);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void init() throws IOException {
|
||||
init(HoodieFileFormat.PARQUET);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void clean() throws IOException {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testSimpleInsertAndUpdate(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
@Test
|
||||
public void testSimpleInsertAndUpdate() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
insertAndGetFilePaths(records, client, cfg, newCommitTime);
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
records = dataGen.generateUpdates(newCommitTime, 100);
|
||||
updateAndGetFilePaths(records, client, cfg, newCommitTime);
|
||||
|
||||
String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString();
|
||||
client.compact(compactionCommitTime);
|
||||
|
||||
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
|
||||
FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
||||
tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
Stream<HoodieBaseFile> dataFilesToRead = tableView.getLatestBaseFiles();
|
||||
assertTrue(dataFilesToRead.findAny().isPresent());
|
||||
|
||||
// verify that there is a commit
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
assertEquals(1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(),
|
||||
"Expecting a single commit.");
|
||||
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
|
||||
assertTrue(HoodieTimeline.compareTimestamps("000", HoodieTimeline.LESSER_THAN, latestCompactionCommitTime));
|
||||
|
||||
assertEquals(200, HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, "000"),
|
||||
"Must contain 200 records");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleInsertAndUpdateHFile() throws Exception {
|
||||
clean();
|
||||
init(HoodieFileFormat.HFILE);
|
||||
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
@@ -170,18 +218,15 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
|
||||
assertTrue(HoodieTimeline.compareTimestamps("000", HoodieTimeline.LESSER_THAN, latestCompactionCommitTime));
|
||||
|
||||
assertEquals(200, HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count(),
|
||||
assertEquals(200, HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, "000"),
|
||||
"Must contain 200 records");
|
||||
}
|
||||
}
|
||||
|
||||
// test incremental read does not go past compaction instant for RO views
|
||||
// For RT views, incremental read can go past compaction
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testIncrementalReadsWithCompaction(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
@Test
|
||||
public void testIncrementalReadsWithCompaction() throws Exception {
|
||||
String partitionPath = "2020/02/20"; // use only one partition for this test
|
||||
dataGen = new HoodieTestDataGenerator(new String[] { partitionPath });
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
@@ -279,11 +324,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
// Check if record level metadata is aggregated properly at the end of write.
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testMetadataAggregateFromWriteStatus(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
@Test
|
||||
public void testMetadataAggregateFromWriteStatus() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build();
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
@@ -305,11 +347,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testSimpleInsertUpdateAndDelete(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
@Test
|
||||
public void testSimpleInsertUpdateAndDelete() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
@@ -388,10 +427,9 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
private void testCOWToMORConvertedTableRollback(HoodieFileFormat baseFileFormat, Boolean rollbackUsingMarkers) throws Exception {
|
||||
init(baseFileFormat);
|
||||
private void testCOWToMORConvertedTableRollback(Boolean rollbackUsingMarkers) throws Exception {
|
||||
// Set TableType to COW
|
||||
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE, baseFileFormat);
|
||||
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
|
||||
HoodieWriteConfig cfg = getConfig(false, rollbackUsingMarkers);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
@@ -428,7 +466,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// Set TableType to MOR
|
||||
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, baseFileFormat);
|
||||
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
|
||||
|
||||
// rollback a COW commit when TableType is MOR
|
||||
client.rollback(newCommitTime);
|
||||
@@ -443,22 +481,19 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testCOWToMORConvertedTableRollbackUsingFileList(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testCOWToMORConvertedTableRollback(baseFileFormat, false);
|
||||
@Test
|
||||
public void testCOWToMORConvertedTableRollbackUsingFileList() throws Exception {
|
||||
testCOWToMORConvertedTableRollback(false);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testCOWToMORConvertedTableRollbackUsingMarkers(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testCOWToMORConvertedTableRollback(baseFileFormat, true);
|
||||
@Test
|
||||
public void testCOWToMORConvertedTableRollbackUsingMarkers() throws Exception {
|
||||
testCOWToMORConvertedTableRollback(true);
|
||||
}
|
||||
|
||||
private void testRollbackWithDeltaAndCompactionCommit(HoodieFileFormat baseFileFormat, Boolean rollbackUsingMarkers) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
private void testRollbackWithDeltaAndCompactionCommit(Boolean rollbackUsingMarkers) throws Exception {
|
||||
HoodieWriteConfig cfg = getConfig(false, rollbackUsingMarkers);
|
||||
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
// Test delta commit rollback
|
||||
@@ -604,23 +639,18 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testRollbackWithDeltaAndCompactionCommitUsingFileList(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testRollbackWithDeltaAndCompactionCommit(baseFileFormat, false);
|
||||
@Test
|
||||
public void testRollbackWithDeltaAndCompactionCommitUsingFileList() throws Exception {
|
||||
testRollbackWithDeltaAndCompactionCommit(false);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testRollbackWithDeltaAndCompactionCommitUsingMarkers(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testRollbackWithDeltaAndCompactionCommit(baseFileFormat, true);
|
||||
@Test
|
||||
public void testRollbackWithDeltaAndCompactionCommitUsingMarkers() throws Exception {
|
||||
testRollbackWithDeltaAndCompactionCommit(true);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testMultiRollbackWithDeltaAndCompactionCommit(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
@Test
|
||||
public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
try (final HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
/**
|
||||
@@ -777,15 +807,12 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024)
|
||||
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withEmbeddedTimelineServerEnabled(true)
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024).build()).forTable("test-trip-table")
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024).parquetMaxFileSize(1024).build()).forTable("test-trip-table")
|
||||
.build();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testUpsertPartitioner(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
@Test
|
||||
public void testUpsertPartitioner() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
@@ -863,11 +890,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testLogFileCountsAfterCompaction(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
@Test
|
||||
public void testLogFileCountsAfterCompaction() throws Exception {
|
||||
// insert 100 records
|
||||
HoodieWriteConfig config = getConfig(true);
|
||||
try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
@@ -939,11 +963,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testSimpleInsertsGeneratedIntoLogFiles(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
@Test
|
||||
public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception {
|
||||
// insert 100 records
|
||||
// Setting IndexType to be InMemory to simulate Global Index nature
|
||||
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
||||
@@ -979,10 +1000,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
private void testInsertsGeneratedIntoLogFilesRollback(HoodieFileFormat baseFileFormat,
|
||||
Boolean rollbackUsingMarkers) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
private void testInsertsGeneratedIntoLogFilesRollback(Boolean rollbackUsingMarkers) throws Exception {
|
||||
// insert 100 records
|
||||
// Setting IndexType to be InMemory to simulate Global Index nature
|
||||
HoodieWriteConfig config = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY).build();
|
||||
@@ -1069,22 +1087,17 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testInsertsGeneratedIntoLogFilesRollbackUsingFileList(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testInsertsGeneratedIntoLogFilesRollback(baseFileFormat, false);
|
||||
@Test
|
||||
public void testInsertsGeneratedIntoLogFilesRollbackUsingFileList() throws Exception {
|
||||
testInsertsGeneratedIntoLogFilesRollback(false);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testInsertsGeneratedIntoLogFilesRollbackUsingMarkers(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testInsertsGeneratedIntoLogFilesRollback(baseFileFormat, true);
|
||||
@Test
|
||||
public void testInsertsGeneratedIntoLogFilesRollbackUsingMarkers() throws Exception {
|
||||
testInsertsGeneratedIntoLogFilesRollback(true);
|
||||
}
|
||||
|
||||
private void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(HoodieFileFormat baseFileFormat,
|
||||
Boolean rollbackUsingMarkers) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
private void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(Boolean rollbackUsingMarkers) throws Exception {
|
||||
// insert 100 records
|
||||
// Setting IndexType to be InMemory to simulate Global Index nature
|
||||
HoodieWriteConfig config = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY).build();
|
||||
@@ -1135,23 +1148,20 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testInsertsGeneratedIntoLogFilesRollbackAfterCompactionUsingFileList(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(baseFileFormat, false);
|
||||
@Test
|
||||
public void testInsertsGeneratedIntoLogFilesRollbackAfterCompactionUsingFileList() throws Exception {
|
||||
testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(false);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testInsertsGeneratedIntoLogFilesRollbackAfterCompactionUsingMarkers(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(baseFileFormat, true);
|
||||
@Test
|
||||
public void testInsertsGeneratedIntoLogFilesRollbackAfterCompactionUsingMarkers() throws Exception {
|
||||
testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure metadata stats are correctly written to metadata file.
|
||||
*/
|
||||
public void testMetadataStatsOnCommit(HoodieFileFormat baseFileFormat, Boolean rollbackUsingMarkers) throws Exception {
|
||||
init(baseFileFormat);
|
||||
public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY)
|
||||
.withAutoCommit(false).build();
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
@@ -1231,26 +1241,21 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
/**
|
||||
* Test to ensure rolling stats are correctly written to metadata file.
|
||||
*/
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testMetadataStatsOnCommitUsingFileList(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testMetadataStatsOnCommit(baseFileFormat, false);
|
||||
@Test
|
||||
public void testMetadataStatsOnCommitUsingFileList() throws Exception {
|
||||
testMetadataStatsOnCommit(false);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testMetadataStatsOnCommitUsingMarkers(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
testMetadataStatsOnCommit(baseFileFormat, true);
|
||||
@Test
|
||||
public void testMetadataStatsOnCommitUsingMarkers() throws Exception {
|
||||
testMetadataStatsOnCommit(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure rolling stats are correctly written to the metadata file, identifies small files and corrects them.
|
||||
*/
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testMetadataStatsWithSmallFileHandling(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
@Test
|
||||
public void testRollingStatsWithSmallFileHandling() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
Map<String, Long> fileIdToInsertsMap = new HashMap<>();
|
||||
@@ -1364,11 +1369,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
/**
|
||||
* Test to validate invoking table.handleUpdate() with input records from multiple partitions will fail.
|
||||
*/
|
||||
@ParameterizedTest
|
||||
@MethodSource("argumentsProvider")
|
||||
public void testHandleUpdateWithMultiplePartitions(HoodieFileFormat baseFileFormat) throws Exception {
|
||||
init(baseFileFormat);
|
||||
|
||||
@Test
|
||||
public void testHandleUpdateWithMultiplePartitions() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
@@ -1467,7 +1469,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
|
||||
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table")
|
||||
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
|
||||
.withEnableBackupForRemoteFileSystemView(false).build())
|
||||
@@ -1606,6 +1608,12 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
} else {
|
||||
return ((HoodieParquetInputFormat)inputFormat).listStatus(jobConf);
|
||||
}
|
||||
case HFILE:
|
||||
if (realtime) {
|
||||
return ((HoodieHFileRealtimeInputFormat)inputFormat).listStatus(jobConf);
|
||||
} else {
|
||||
return ((HoodieHFileInputFormat)inputFormat).listStatus(jobConf);
|
||||
}
|
||||
default:
|
||||
throw new HoodieIOException("Hoodie InputFormat not implemented for base file format " + baseFileFormat);
|
||||
}
|
||||
|
||||
@@ -366,7 +366,8 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
@Test
|
||||
public void testFileSizeUpsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.limitFileSize(64 * 1024).parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024).build()).build();
|
||||
.parquetMaxFileSize(64 * 1024).hfileMaxFileSize(64 * 1024)
|
||||
.parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024).build()).build();
|
||||
String instantTime = HoodieTestUtils.makeNewCommitTime();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
|
||||
@@ -401,7 +402,8 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
@Test
|
||||
public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
final HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
|
||||
String instantTime = "000";
|
||||
|
||||
@@ -71,7 +71,8 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(smallFileSize)
|
||||
.insertSplitSize(100).autoTuneInsertSplits(autoSplitInserts).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1000 * 1024).parquetMaxFileSize(1000 * 1024).build())
|
||||
.build();
|
||||
|
||||
FileCreateUtils.createCommit(basePath, "001");
|
||||
FileCreateUtils.createDataFile(basePath, testPartitionPath, "001", "file1", fileSize);
|
||||
|
||||
@@ -72,7 +72,8 @@ public class CompactionTestBase extends HoodieClientTestBase {
|
||||
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
|
||||
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
@@ -194,7 +195,7 @@ public class CompactionTestBase extends HoodieClientTestBase {
|
||||
assertEquals(latestCompactionCommitTime, compactionInstantTime,
|
||||
"Expect compaction instant time to be the latest commit time");
|
||||
assertEquals(expectedNumRecs,
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count(),
|
||||
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, "000"),
|
||||
"Must contain expected records");
|
||||
|
||||
}
|
||||
|
||||
@@ -95,7 +95,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
.withParallelism(2, 2)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
|
||||
.withInlineCompaction(false).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.withMemoryConfig(HoodieMemoryConfig.newBuilder().withMaxDFSStreamBufferSize(1 * 1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
|
||||
|
||||
@@ -130,7 +130,7 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
|
||||
.withWriteStatusClass(MetadataMergeWriteStatus.class)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
@@ -459,12 +459,12 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
|
||||
|
||||
// Check that the incremental consumption from prevCommitTime
|
||||
assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, prevCommitTime).count(),
|
||||
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime),
|
||||
"Incremental consumption from " + prevCommitTime + " should give all records in latest commit");
|
||||
if (commitTimesBetweenPrevAndNew.isPresent()) {
|
||||
commitTimesBetweenPrevAndNew.get().forEach(ct -> {
|
||||
assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, ct).count(),
|
||||
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, ct),
|
||||
"Incremental consumption from " + ct + " should give all records in latest commit");
|
||||
});
|
||||
}
|
||||
@@ -527,7 +527,7 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
|
||||
|
||||
// Check that the incremental consumption from prevCommitTime
|
||||
assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, prevCommitTime).count(),
|
||||
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime),
|
||||
"Incremental consumption from " + prevCommitTime + " should give no records in latest commit,"
|
||||
+ " since it is a delete operation");
|
||||
}
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
@@ -45,6 +46,10 @@ import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
@@ -57,12 +62,15 @@ import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Utility methods to aid testing inside the HoodieClient module.
|
||||
@@ -129,7 +137,7 @@ public class HoodieClientTestUtils {
|
||||
/**
|
||||
* Obtain all new data written into the Hoodie table since the given timestamp.
|
||||
*/
|
||||
public static Dataset<Row> readSince(String basePath, SQLContext sqlContext,
|
||||
public static long countRecordsSince(JavaSparkContext jsc, String basePath, SQLContext sqlContext,
|
||||
HoodieTimeline commitTimeline, String lastCommitTime) {
|
||||
List<HoodieInstant> commitsToReturn =
|
||||
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE).getInstants().collect(Collectors.toList());
|
||||
@@ -137,12 +145,17 @@ public class HoodieClientTestUtils {
|
||||
// Go over the commit metadata, and obtain the new files that need to be read.
|
||||
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn);
|
||||
String[] paths = fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]);
|
||||
Dataset<Row> rows = null;
|
||||
if (paths[0].endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
|
||||
rows = sqlContext.read().parquet(paths);
|
||||
return sqlContext.read().parquet(paths)
|
||||
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime))
|
||||
.count();
|
||||
} else if (paths[0].endsWith(HoodieFileFormat.HFILE.getFileExtension())) {
|
||||
return readHFile(jsc, paths)
|
||||
.filter(gr -> HoodieTimeline.compareTimestamps(lastCommitTime, HoodieActiveTimeline.LESSER_THAN,
|
||||
gr.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()))
|
||||
.count();
|
||||
}
|
||||
|
||||
return rows.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
|
||||
throw new HoodieException("Unsupported base file format for file :" + paths[0]);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
|
||||
}
|
||||
@@ -170,6 +183,37 @@ public class HoodieClientTestUtils {
|
||||
}
|
||||
}
|
||||
|
||||
public static Stream<GenericRecord> readHFile(JavaSparkContext jsc, String[] paths) {
|
||||
// TODO: this should be ported to use HoodieStorageReader
|
||||
List<GenericRecord> valuesAsList = new LinkedList<>();
|
||||
|
||||
FileSystem fs = FSUtils.getFs(paths[0], jsc.hadoopConfiguration());
|
||||
CacheConfig cacheConfig = new CacheConfig(fs.getConf());
|
||||
Schema schema = null;
|
||||
for (String path : paths) {
|
||||
try {
|
||||
HFile.Reader reader = HFile.createReader(fs, new Path(path), cacheConfig, fs.getConf());
|
||||
if (schema == null) {
|
||||
schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get("schema".getBytes())));
|
||||
}
|
||||
HFileScanner scanner = reader.getScanner(false, false);
|
||||
if (!scanner.seekTo()) {
|
||||
// EOF reached
|
||||
continue;
|
||||
}
|
||||
|
||||
do {
|
||||
Cell c = scanner.getKeyValue();
|
||||
byte[] value = Arrays.copyOfRange(c.getValueArray(), c.getValueOffset(), c.getValueOffset() + c.getValueLength());
|
||||
valuesAsList.add(HoodieAvroUtils.bytesToAvro(value, schema));
|
||||
} while (scanner.next());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error reading hfile " + path + " as a dataframe", e);
|
||||
}
|
||||
}
|
||||
return valuesAsList.stream();
|
||||
}
|
||||
|
||||
/**
|
||||
* TODO Incorporate into {@link org.apache.hudi.common.testutils.HoodieTestTable}.
|
||||
*/
|
||||
|
||||
@@ -166,7 +166,7 @@ public class SparkDatasetTestUtils {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withBulkInsertParallelism(2);
|
||||
|
||||
Reference in New Issue
Block a user