1
0

[HUDI-2177][HUDI-2200] Adding virtual keys support for MOR table (#3315)

This commit is contained in:
Sivabalan Narayanan
2021-08-02 09:45:09 -04:00
committed by GitHub
parent dde57b293c
commit fe508376fa
37 changed files with 633 additions and 261 deletions

View File

@@ -239,7 +239,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
boolean isPrepped, boolean populateMetaFields) throws Exception {
// Set autoCommit false
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withAutoCommit(false);
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
try (SparkRDDWriteClient client = getHoodieWriteClient(cfgBuilder.build());) {
String prevCommitTime = "000";
@@ -328,7 +328,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
JavaRDD<HoodieRecord> recordList = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
HoodieWriteConfig.Builder configBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
.combineInput(true, true);
addAppropriatePropsForPopulateMetaFields(configBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(configBuilder, populateMetaFields);
try (SparkRDDWriteClient client = getHoodieWriteClient(configBuilder.build());) {
client.startCommitWithTime(newCommitTime);
@@ -365,7 +365,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@MethodSource("populateMetaFieldsParams")
public void testUpserts(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder();
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
testUpsertsInternal(cfgBuilder.build(), SparkRDDWriteClient::upsert, false);
}
@@ -376,7 +376,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@MethodSource("populateMetaFieldsParams")
public void testUpsertsPrepped(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder();
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
testUpsertsInternal(cfgBuilder.build(), SparkRDDWriteClient::upsertPreppedRecords, true);
}
@@ -526,7 +526,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@MethodSource("populateMetaFieldsParams")
public void testInsertsWithHoodieConcatHandle(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder();
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
testHoodieConcatHandle(cfgBuilder.build(), false);
}
@@ -537,7 +537,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@MethodSource("populateMetaFieldsParams")
public void testInsertsPreppedWithHoodieConcatHandle(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder();
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
testHoodieConcatHandle(cfgBuilder.build(), true);
}
@@ -588,7 +588,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@MethodSource("populateMetaFieldsParams")
public void testDeletes(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY);
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
SparkRDDWriteClient client = getHoodieWriteClient(cfgBuilder.build());
/**
* Write 1 (inserts and deletes) Write actual 200 insert records and ignore 100 delete records
@@ -639,7 +639,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@MethodSource("populateMetaFieldsParams")
public void testDeletesForInsertsInSameBatch(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY);
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
SparkRDDWriteClient client = getHoodieWriteClient(cfgBuilder.build());
/**
* Write 200 inserts and issue deletes to a subset(50) of inserts.
@@ -1209,7 +1209,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
// complete another commit after pending clustering
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER);
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig config = cfgBuilder.build();
SparkRDDWriteClient client = getHoodieWriteClient(config);
dataGen = new HoodieTestDataGenerator();
@@ -1582,7 +1582,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
public void testCommitWritesRelativePaths(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withAutoCommit(false);
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
try (SparkRDDWriteClient client = getHoodieWriteClient(cfgBuilder.build());) {
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build();
HoodieSparkTable table = HoodieSparkTable.create(cfgBuilder.build(), context, metaClient);
@@ -1628,7 +1628,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@MethodSource("populateMetaFieldsParams")
public void testMetadataStatsOnCommit(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withAutoCommit(false);
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
SparkRDDWriteClient client = getHoodieWriteClient(cfg);
@@ -2002,7 +2002,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
public void testMultiOperationsPerCommit(boolean populateMetaFields) throws IOException {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withAutoCommit(false)
.withAllowMultiWriteOnSameInstant(true);
addAppropriatePropsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
SparkRDDWriteClient client = getHoodieWriteClient(cfg);
String firstInstantTime = "0000";

View File

@@ -451,7 +451,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.countRecordsSince(jsc, basePath, sqlContext, timeline, instantTime));
assertEquals(numExpectedRecords, HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of(instantTime)));
} else {
// TODO: This code fails to read records under the following conditions:
// 1. No parquet files yet (i.e. no compaction done yet)

View File

@@ -43,7 +43,6 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
@@ -61,10 +60,10 @@ import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Disabled;
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.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.EnumSource;
import java.io.IOException;
import java.nio.file.Files;
@@ -72,9 +71,7 @@ import java.nio.file.Paths;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Properties;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -94,15 +91,16 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
private HoodieTableType tableType;
public void init(HoodieTableType tableType, boolean populateMetaFields) throws IOException {
public void init(HoodieTableType tableType) throws IOException {
this.tableType = tableType;
initPath();
initSparkContexts("TestHoodieMetadata");
initFileSystem();
fs.mkdirs(new Path(basePath));
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType, populateMetaFields ? new Properties() : getPropertiesForKeyGen());
initMetaClient(tableType);
initTestDataGenerator();
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
}
@AfterEach
@@ -110,25 +108,12 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
cleanupResources();
}
private static Stream<Arguments> populateMetaFieldsParams() {
return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of);
}
private static Stream<Arguments> tableTypePopulateMetaFieldsParams() {
return Stream.of(
Arguments.of(HoodieTableType.COPY_ON_WRITE, true),
Arguments.of(HoodieTableType.COPY_ON_WRITE, false),
Arguments.of(HoodieTableType.MERGE_ON_READ, true)
);
}
/**
* Metadata Table bootstrap scenarios.
*/
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testMetadataTableBootstrap(boolean populateMetaFields) throws Exception {
init(HoodieTableType.COPY_ON_WRITE, populateMetaFields);
@Test
public void testMetadataTableBootstrap() throws Exception {
init(HoodieTableType.COPY_ON_WRITE);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
// Metadata table should not exist until created for the first time
@@ -137,7 +122,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
// Metadata table is not created if disabled by config
String firstCommitTime = HoodieActiveTimeline.createNewInstantTime();
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
client.startCommitWithTime(firstCommitTime);
client.insert(jsc.parallelize(dataGen.generateInserts(firstCommitTime, 5)), firstCommitTime);
assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not be created");
@@ -146,7 +131,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
// Metadata table should not be created if any non-complete instants are present
String secondCommitTime = HoodieActiveTimeline.createNewInstantTime();
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(false, true, populateMetaFields), true)) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(false, true), true)) {
client.startCommitWithTime(secondCommitTime);
client.insert(jsc.parallelize(dataGen.generateUpdates(secondCommitTime, 2)), secondCommitTime);
// AutoCommit is false so no bootstrap
@@ -159,7 +144,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
// Metadata table created when enabled by config & sync is called
secondCommitTime = HoodieActiveTimeline.createNewInstantTime();
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields), true)) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) {
client.startCommitWithTime(secondCommitTime);
client.insert(jsc.parallelize(dataGen.generateUpdates(secondCommitTime, 2)), secondCommitTime);
client.syncTableMetadata();
@@ -182,7 +167,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
});
String thirdCommitTime = HoodieActiveTimeline.createNewInstantTime();
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields), true)) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) {
client.startCommitWithTime(thirdCommitTime);
client.insert(jsc.parallelize(dataGen.generateUpdates(thirdCommitTime, 2)), thirdCommitTime);
client.syncTableMetadata();
@@ -199,11 +184,10 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
/**
* Only valid partition directories are added to the metadata.
*/
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testOnlyValidPartitionsAdded(boolean populateMetaFields) throws Exception {
@Test
public void testOnlyValidPartitionsAdded() throws Exception {
// This test requires local file system
init(HoodieTableType.COPY_ON_WRITE, populateMetaFields);
init(HoodieTableType.COPY_ON_WRITE);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
// Create an empty directory which is not a partition directory (lacks partition metadata)
@@ -223,7 +207,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
.addCommit("002").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10, 10);
final HoodieWriteConfig writeConfig =
getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.NEVER, true, true, false, populateMetaFields)
getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.NEVER, true, true, false)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).withDirectoryFilterRegex(filterDirRegex).build()).build();
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) {
client.startCommitWithTime("005");
@@ -253,12 +237,12 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
* Test various table operations sync to Metadata Table correctly.
*/
@ParameterizedTest
@MethodSource("tableTypePopulateMetaFieldsParams")
public void testTableOperations(HoodieTableType tableType, boolean populateMetaFields) throws Exception {
init(tableType, populateMetaFields);
@EnumSource(HoodieTableType.class)
public void testTableOperations(HoodieTableType tableType) throws Exception {
init(tableType);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// Write 1 (Bulk insert)
String newCommitTime = "001";
@@ -341,12 +325,12 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
* Test rollback of various table operations sync to Metadata Table correctly.
*/
@ParameterizedTest
@MethodSource("tableTypePopulateMetaFieldsParams")
public void testRollbackOperations(HoodieTableType tableType, boolean populateMetaFields) throws Exception {
init(tableType, populateMetaFields);
@EnumSource(HoodieTableType.class)
public void testRollbackOperations(HoodieTableType tableType) throws Exception {
init(tableType);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// Write 1 (Bulk insert)
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
@@ -419,7 +403,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
// Rollback of partial commits
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
getWriteConfigBuilder(false, true, false, populateMetaFields).withRollbackUsingMarkers(false).build())) {
getWriteConfigBuilder(false, true, false).withRollbackUsingMarkers(false).build())) {
// Write updates and inserts
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
client.startCommitWithTime(newCommitTime);
@@ -433,7 +417,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
// Marker based rollback of partial commits
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
getWriteConfigBuilder(false, true, false, populateMetaFields).withRollbackUsingMarkers(true).build())) {
getWriteConfigBuilder(false, true, false).withRollbackUsingMarkers(true).build())) {
// Write updates and inserts
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
client.startCommitWithTime(newCommitTime);
@@ -451,12 +435,12 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
* Once explicit sync is called, metadata should match.
*/
@ParameterizedTest
@MethodSource("tableTypePopulateMetaFieldsParams")
public void testRollbackUnsyncedCommit(HoodieTableType tableType, boolean populateMetaFields) throws Exception {
init(tableType, populateMetaFields);
@EnumSource(HoodieTableType.class)
public void testRollbackUnsyncedCommit(HoodieTableType tableType) throws Exception {
init(tableType);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// Initialize table with metadata
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
@@ -466,7 +450,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
validateMetadata(client);
}
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
// Commit with metadata disabled
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateUpdates(newCommitTime, 10);
@@ -475,7 +459,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
client.rollback(newCommitTime);
}
try (SparkRDDWriteClient client = new SparkRDDWriteClient<>(engineContext, getWriteConfig(true, true, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient<>(engineContext, getWriteConfig(true, true))) {
assertFalse(metadata(client).isInSync());
client.syncTableMetadata();
validateMetadata(client);
@@ -486,10 +470,10 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
* Test sync of table operations.
*/
@ParameterizedTest
@MethodSource("tableTypePopulateMetaFieldsParams")
@EnumSource(HoodieTableType.class)
@Disabled
public void testSync(HoodieTableType tableType, boolean populateMetaFields) throws Exception {
init(tableType, populateMetaFields);
public void testSync(HoodieTableType tableType) throws Exception {
init(tableType);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
String newCommitTime;
@@ -497,7 +481,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
List<WriteStatus> writeStatuses;
// Initial commits without metadata table enabled
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
records = dataGen.generateInserts(newCommitTime, 5);
client.startCommitWithTime(newCommitTime);
@@ -512,7 +496,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
}
// Enable metadata table so it initialized by listing from file system
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// inserts
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
client.startCommitWithTime(newCommitTime);
@@ -528,7 +512,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
String restoreToInstant;
String inflightActionTimestamp;
String beforeInflightActionTimestamp;
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
// updates
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
client.startCommitWithTime(newCommitTime);
@@ -600,7 +584,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
Path inflightCleanPath = new Path(metaClient.getMetaPath(), HoodieTimeline.makeInflightCleanerFileName(inflightActionTimestamp));
fs.create(inflightCleanPath).close();
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// Restore cannot be done until the metadata table is in sync. See HUDI-1502 for details
client.syncTableMetadata();
@@ -629,7 +613,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
}
// Enable metadata table and ensure it is synced
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
client.restoreToInstant(restoreToInstant);
assertFalse(metadata(client).isInSync());
@@ -645,14 +629,13 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
/**
* Instants on Metadata Table should be archived as per config. Metadata Table should be automatically compacted as per config.
*/
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testCleaningArchivingAndCompaction(boolean populateMetaFields) throws Exception {
init(HoodieTableType.COPY_ON_WRITE, populateMetaFields);
@Test
public void testCleaningArchivingAndCompaction() throws Exception {
init(HoodieTableType.COPY_ON_WRITE);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
final int maxDeltaCommitsBeforeCompaction = 4;
HoodieWriteConfig config = getWriteConfigBuilder(true, true, false, populateMetaFields)
HoodieWriteConfig config = getWriteConfigBuilder(true, true, false)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true)
.archiveCommitsWith(6, 8).retainCommits(1)
.withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build())
@@ -693,15 +676,14 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
/**
* Test various error scenarios.
*/
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testErrorCases(boolean populateMetaFields) throws Exception {
init(HoodieTableType.COPY_ON_WRITE, populateMetaFields);
@Test
public void testErrorCases() throws Exception {
init(HoodieTableType.COPY_ON_WRITE);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
// TESTCASE: If commit on the metadata table succeeds but fails on the dataset, then on next init the metadata table
// should be rolled back to last valid commit.
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields), true)) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) {
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 10);
@@ -722,7 +704,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
commitInstantFileName), false));
}
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields), true)) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) {
String newCommitTime = client.startCommit();
// Next insert
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 5);
@@ -739,11 +721,11 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
*/
//@Test
public void testNonPartitioned() throws Exception {
init(HoodieTableType.COPY_ON_WRITE, true);
init(HoodieTableType.COPY_ON_WRITE);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
HoodieTestDataGenerator nonPartitionedGenerator = new HoodieTestDataGenerator(new String[] {""});
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, true))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// Write 1 (Bulk insert)
String newCommitTime = "001";
List<HoodieRecord> records = nonPartitionedGenerator.generateInserts(newCommitTime, 10);
@@ -759,13 +741,12 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
/**
* Test various metrics published by metadata table.
*/
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testMetadataMetrics(boolean populateMetaFields) throws Exception {
init(HoodieTableType.COPY_ON_WRITE, populateMetaFields);
@Test
public void testMetadataMetrics() throws Exception {
init(HoodieTableType.COPY_ON_WRITE);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(true, true, true, populateMetaFields).build())) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(true, true, true).build())) {
// Write
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
@@ -788,16 +769,15 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
/**
* Test when reading from metadata table which is out of sync with dataset that results are still consistent.
*/
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testMetadataOutOfSync(boolean populateMetaFields) throws Exception {
init(HoodieTableType.COPY_ON_WRITE, populateMetaFields);
@Test
public void testMetadataOutOfSync() throws Exception {
init(HoodieTableType.COPY_ON_WRITE);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
SparkRDDWriteClient unsyncedClient = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields));
SparkRDDWriteClient unsyncedClient = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true));
// Enable metadata so table is initialized
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// Perform Bulk Insert
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
@@ -806,7 +786,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
}
// Perform commit operations with metadata disabled
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
// Perform Insert
String newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
@@ -831,7 +811,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
validateMetadata(unsyncedClient);
// Perform clean operation with metadata disabled
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
// One more commit needed to trigger clean so upsert and compact
String newCommitTime = "005";
client.startCommitWithTime(newCommitTime);
@@ -853,7 +833,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
validateMetadata(unsyncedClient);
// Perform restore with metadata disabled
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false, populateMetaFields))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
client.restoreToInstant("004");
}
@@ -1028,20 +1008,18 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
}
}
private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileListingMetadata, boolean populateMetaFields) {
return getWriteConfigBuilder(autoCommit, useFileListingMetadata, false, populateMetaFields).build();
private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileListingMetadata) {
return getWriteConfigBuilder(autoCommit, useFileListingMetadata, false).build();
}
private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics, boolean populateMetaFields) {
return getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, autoCommit, useFileListingMetadata, enableMetrics, populateMetaFields);
private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) {
return getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, autoCommit, useFileListingMetadata, enableMetrics);
}
private HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata,
boolean enableMetrics, boolean populateMetaFields) {
private HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2)
.withAutoCommit(autoCommit)
.withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1)
.withFailedWritesCleaningPolicy(policy)
@@ -1050,7 +1028,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
.withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table")
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
.withEnableBackupForRemoteFileSystemView(false).build())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(populateMetaFields ? HoodieIndex.IndexType.BLOOM : HoodieIndex.IndexType.SIMPLE).build())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.enable(useFileListingMetadata)
.enableMetrics(enableMetrics).build())

View File

@@ -82,6 +82,9 @@ 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.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.io.IOException;
@@ -93,6 +96,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -113,7 +117,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
private HoodieFileFormat baseFileFormat;
public void init(HoodieFileFormat baseFileFormat) throws IOException {
public void init(HoodieFileFormat baseFileFormat, boolean populateMetaFields) throws IOException {
this.baseFileFormat = baseFileFormat;
initDFS();
initSparkContexts("TestHoodieMergeOnReadTable");
@@ -122,7 +126,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
context = new HoodieSparkEngineContext(jsc);
initPath();
dfs.mkdirs(new Path(basePath));
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, baseFileFormat);
Properties properties = populateMetaFields ? new Properties() : getPropertiesForKeyGen();
properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.key(), baseFileFormat.toString());
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, properties);
initTestDataGenerator();
roSnapshotJobConf = new JobConf(hadoopConf);
@@ -132,7 +140,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
@BeforeEach
public void init() throws IOException {
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue());
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), true);
}
@AfterEach
@@ -140,9 +148,19 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
cleanupResources();
}
@Test
public void testSimpleInsertAndUpdate() throws Exception {
HoodieWriteConfig cfg = getConfig(true);
private static Stream<Arguments> populateMetaFieldsParams() {
return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of);
}
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testSimpleInsertAndUpdate(boolean populateMetaFields) throws Exception {
clean();
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
/**
@@ -179,17 +197,20 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
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");
if (cfg.populateMetaFields()) {
assertEquals(200, HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of("000")),
"Must contain 200 records");
} else {
assertEquals(200, HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.empty()));
}
}
}
@Test
public void testSimpleInsertAndUpdateHFile() throws Exception {
clean();
init(HoodieFileFormat.HFILE);
HoodieWriteConfig cfg = getConfig(true);
init(HoodieFileFormat.HFILE, true);
HoodieWriteConfig cfg = getConfigBuilder(true).build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
/**
@@ -227,26 +248,35 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
assertTrue(HoodieTimeline.compareTimestamps("000", HoodieTimeline.LESSER_THAN, latestCompactionCommitTime));
assertEquals(200, HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, "000"),
assertEquals(200, HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of("000")),
"Must contain 200 records");
}
}
@Test
public void testSimpleClusteringNoUpdates() throws Exception {
testClustering(false);
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testSimpleClusteringNoUpdates(boolean populateMetaFields) throws Exception {
clean();
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
testClustering(false, populateMetaFields);
}
@Test
public void testSimpleClusteringWithUpdates() throws Exception {
testClustering(true);
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testSimpleClusteringWithUpdates(boolean populateMetaFields) throws Exception {
clean();
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
testClustering(true, populateMetaFields);
}
private void testClustering(boolean doUpdates) throws Exception {
private void testClustering(boolean doUpdates, boolean populateMetaFields) throws Exception {
// set low compaction small File Size to generate more file groups.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
HoodieWriteConfig cfg = getConfigBuilder(true, 10L, clusteringConfig).build();
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true, 10L, clusteringConfig);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
/**
@@ -302,8 +332,12 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
"Expecting a single commit.");
assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp());
assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction());
assertEquals(400, HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, "000"),
"Must contain 200 records");
if (cfg.populateMetaFields()) {
assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of("000")),
"Must contain 200 records");
} else {
assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.empty()));
}
}
}
@@ -431,9 +465,14 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
}
}
@Test
public void testSimpleInsertUpdateAndDelete() throws Exception {
HoodieWriteConfig cfg = getConfig(true);
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testSimpleInsertUpdateAndDelete(boolean populateMetaFields) throws Exception {
clean();
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
/**
@@ -505,7 +544,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
assertTrue(dataFilesToRead.findAny().isPresent());
List<String> dataFiles = tableView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles, basePath);
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles, basePath, new JobConf(hadoopConf), true, false);
// Wrote 20 records and deleted 20 records, so remaining 20-20 = 0
assertEquals(0, recordsRead.size(), "Must contain 0 records");
}
@@ -575,8 +614,10 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
testCOWToMORConvertedTableRollback(true);
}
private void testRollbackWithDeltaAndCompactionCommit(Boolean rollbackUsingMarkers) throws Exception {
HoodieWriteConfig cfg = getConfig(false, rollbackUsingMarkers);
private void testRollbackWithDeltaAndCompactionCommit(Boolean rollbackUsingMarkers, boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false, rollbackUsingMarkers, IndexType.SIMPLE);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
@@ -621,7 +662,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
*/
final String commitTime1 = "002";
// WriteClient with custom config (disable small file handling)
try (SparkRDDWriteClient secondClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());) {
try (SparkRDDWriteClient secondClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff(false));) {
secondClient.startCommitWithTime(commitTime1);
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
@@ -723,19 +764,31 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
}
}
@Test
public void testRollbackWithDeltaAndCompactionCommitUsingFileList() throws Exception {
testRollbackWithDeltaAndCompactionCommit(false);
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testRollbackWithDeltaAndCompactionCommitUsingFileList(boolean populateMetaFields) throws Exception {
clean();
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
testRollbackWithDeltaAndCompactionCommit(false, populateMetaFields);
}
@Test
public void testRollbackWithDeltaAndCompactionCommitUsingMarkers() throws Exception {
testRollbackWithDeltaAndCompactionCommit(true);
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testRollbackWithDeltaAndCompactionCommitUsingMarkers(boolean populateMetaFields) throws Exception {
clean();
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
testRollbackWithDeltaAndCompactionCommit(true, populateMetaFields);
}
@Test
public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
HoodieWriteConfig cfg = getConfig(false);
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testMultiRollbackWithDeltaAndCompactionCommit(boolean populateMetaFields) throws Exception {
clean();
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
try (final SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
/**
* Write 1 (only inserts)
@@ -776,7 +829,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
*/
newCommitTime = "002";
// WriteClient with custom config (disable small file handling)
SparkRDDWriteClient nClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());
SparkRDDWriteClient nClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff(populateMetaFields));
nClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
@@ -885,20 +938,29 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
}
}
protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff(boolean populateMetaFields) {
HoodieWriteConfig.Builder cfgBuilder = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withDeleteParallelism(2)
.withAutoCommit(false)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withEmbeddedTimelineServerEnabled(true)
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024).parquetMaxFileSize(1024).build()).forTable("test-trip-table")
.build();
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024).parquetMaxFileSize(1024).build()).forTable("test-trip-table");
if (!populateMetaFields) {
addConfigsForPopulateMetaFields(cfgBuilder, false);
}
return cfgBuilder.build();
}
@Test
public void testUpsertPartitioner() throws Exception {
HoodieWriteConfig cfg = getConfig(true);
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testUpsertPartitioner(boolean populateMetaFields) throws Exception {
clean();
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
/**
@@ -969,16 +1031,20 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
List<String> dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles,
basePath);
basePath, new JobConf(hadoopConf), true, false);
// Wrote 20 records in 2 batches
assertEquals(40, recordsRead.size(), "Must contain 40 records");
}
}
@Test
public void testLogFileCountsAfterCompaction() throws Exception {
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testLogFileCountsAfterCompaction(boolean populateMetaFields) throws Exception {
// insert 100 records
HoodieWriteConfig config = getConfig(true);
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig config = cfgBuilder.build();
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
@@ -1052,6 +1118,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
@@ -1094,6 +1161,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
// insert 100 records
// Setting IndexType to be InMemory to simulate Global Index nature
HoodieWriteConfig config = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY).build();
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) {
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);
@@ -1253,6 +1321,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Exception {
HoodieWriteConfig cfg = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY)
.withAutoCommit(false).build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
metaClient = getHoodieMetaClient(hadoopConf, basePath);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);

View File

@@ -198,7 +198,7 @@ public class CompactionTestBase extends HoodieClientTestBase {
assertEquals(latestCompactionCommitTime, compactionInstantTime,
"Expect compaction instant time to be the latest commit time");
assertEquals(expectedNumRecs,
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, "000"),
HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of("000")),
"Must contain expected records");
}

View File

@@ -518,12 +518,12 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
if (filterForCommitTimeWithAssert) {
// Check that the incremental consumption from prevCommitTime
assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime),
HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of(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.countRecordsSince(jsc, basePath, sqlContext, timeline, ct),
HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of(ct)),
"Incremental consumption from " + ct + " should give all records in latest commit");
});
}
@@ -590,7 +590,7 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
if (filerForCommitTimeWithAssert) {
// Check that the incremental consumption from prevCommitTime
assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime),
HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of(prevCommitTime)),
"Incremental consumption from " + prevCommitTime + " should give no records in latest commit,"
+ " since it is a delete operation");
}

View File

@@ -44,6 +44,7 @@ import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.table.WorkloadStat;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -234,10 +235,13 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
properties.put(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), "false");
properties.put("hoodie.datasource.write.recordkey.field","_row_key");
properties.put("hoodie.datasource.write.partitionpath.field","partition_path");
properties.put(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key(), "_row_key");
properties.put(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key(), "partition_path");
properties.put(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key(), SimpleKeyGenerator.class.getName());
return properties;
}
protected void addAppropriatePropsForPopulateMetaFields(HoodieWriteConfig.Builder configBuilder, boolean populateMetaFields) {
protected void addConfigsForPopulateMetaFields(HoodieWriteConfig.Builder configBuilder, boolean populateMetaFields) {
if (!populateMetaFields) {
configBuilder.withProperties(getPropertiesForKeyGen())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.SIMPLE).build());

View File

@@ -141,33 +141,46 @@ public class HoodieClientTestUtils {
}
/**
* Obtain all new data written into the Hoodie table since the given timestamp.
* Obtain all new data written into the Hoodie table with an optional from timestamp.
*/
public static long countRecordsSince(JavaSparkContext jsc, String basePath, SQLContext sqlContext,
HoodieTimeline commitTimeline, String lastCommitTime) {
public static long countRecordsOptionallySince(JavaSparkContext jsc, String basePath, SQLContext sqlContext,
HoodieTimeline commitTimeline, Option<String> lastCommitTimeOpt) {
List<HoodieInstant> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE).getInstants().collect(Collectors.toList());
lastCommitTimeOpt.isPresent() ? commitTimeline.findInstantsAfter(lastCommitTimeOpt.get(), Integer.MAX_VALUE).getInstants().collect(Collectors.toList()) :
commitTimeline.getInstants().collect(Collectors.toList());
try {
// 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()]);
if (paths[0].endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
return sqlContext.read().parquet(paths)
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime))
.count();
Dataset<Row> rows = sqlContext.read().parquet(paths);
if (lastCommitTimeOpt.isPresent()) {
return rows.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTimeOpt.get()))
.count();
} else {
return rows.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();
Stream<GenericRecord> genericRecordStream = readHFile(jsc, paths);
if (lastCommitTimeOpt.isPresent()) {
return genericRecordStream.filter(gr -> HoodieTimeline.compareTimestamps(lastCommitTimeOpt.get(), HoodieActiveTimeline.LESSER_THAN,
gr.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()))
.count();
} else {
return genericRecordStream.count();
}
} else if (paths[0].endsWith(HoodieFileFormat.ORC.getFileExtension())) {
return sqlContext.read().orc(paths)
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime))
.count();
Dataset<Row> rows = sqlContext.read().orc(paths);
if (lastCommitTimeOpt.isPresent()) {
return rows.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTimeOpt.get()))
.count();
} else {
return rows.count();
}
}
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);
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTimeOpt.get(), e);
}
}

View File

@@ -58,20 +58,30 @@ public class HoodieMergeOnReadTestUtils {
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths,
String basePath, JobConf jobConf, boolean realtime) {
return getRecordsUsingInputFormat(conf, inputPaths, basePath, jobConf, realtime, true);
}
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths,
String basePath, JobConf jobConf, boolean realtime, boolean populateMetaFieldsConfigValue) {
Schema schema = new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA);
return getRecordsUsingInputFormat(conf, inputPaths, basePath, jobConf, realtime, schema,
HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>(), populateMetaFieldsConfigValue);
}
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths, String basePath, JobConf jobConf, boolean realtime, Schema rawSchema,
String rawHiveColumnTypes, boolean projectCols, List<String> projectedColumns) {
return getRecordsUsingInputFormat(conf, inputPaths, basePath, jobConf, realtime, rawSchema, rawHiveColumnTypes, projectCols, projectedColumns, true);
}
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths, String basePath, JobConf jobConf, boolean realtime, Schema rawSchema,
String rawHiveColumnTypes, boolean projectCols, List<String> projectedColumns, boolean populateMetaFieldsConfigValue) {
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build();
FileInputFormat inputFormat = HoodieInputFormatUtils.getInputFormat(metaClient.getTableConfig().getBaseFileFormat(), realtime, jobConf);
Schema schema = HoodieAvroUtils.addMetadataFields(rawSchema);
String hiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(rawHiveColumnTypes);
setPropsForInputFormat(inputFormat, jobConf, schema, hiveColumnTypes, projectCols, projectedColumns);
setPropsForInputFormat(inputFormat, jobConf, schema, hiveColumnTypes, projectCols, projectedColumns, populateMetaFieldsConfigValue);
final List<Field> fields;
if (projectCols) {
fields = schema.getFields().stream().filter(f -> projectedColumns.contains(f.name()))
@@ -112,6 +122,11 @@ public class HoodieMergeOnReadTestUtils {
}
private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf jobConf, Schema schema, String hiveColumnTypes, boolean projectCols, List<String> projectedCols) {
setPropsForInputFormat(inputFormat, jobConf, schema, hiveColumnTypes, projectCols, projectedCols, true);
}
private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf jobConf, Schema schema, String hiveColumnTypes, boolean projectCols, List<String> projectedCols,
boolean populateMetaFieldsConfigValue) {
List<Schema.Field> fields = schema.getFields();
final List<String> projectedColNames;
if (!projectCols) {