[HUDI-1468] Support custom clustering strategies and preserve commit metadata as part of clustering (#3419)
Co-authored-by: Satish Kotha <satishkotha@uber.com>
This commit is contained in:
@@ -98,6 +98,7 @@ import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.jetbrains.annotations.NotNull;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Test;
|
||||
@@ -165,6 +166,15 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of);
|
||||
}
|
||||
|
||||
private static Stream<Arguments> populateMetaFieldsAndPreserveMetadataParams() {
|
||||
return Arrays.stream(new Boolean[][] {
|
||||
{true, true},
|
||||
{false, true},
|
||||
{true, false},
|
||||
{false, false}
|
||||
}).map(Arguments::of);
|
||||
}
|
||||
|
||||
private static Stream<Arguments> rollbackFailedCommitsParams() {
|
||||
return Stream.of(
|
||||
Arguments.of(HoodieFailedWritesCleaningPolicy.LAZY, true),
|
||||
@@ -1292,21 +1302,23 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("populateMetaFieldsParams")
|
||||
public void testSimpleClustering(boolean populateMetaFields) throws Exception {
|
||||
@MethodSource("populateMetaFieldsAndPreserveMetadataParams")
|
||||
public void testSimpleClustering(boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception {
|
||||
// setup clustering config.
|
||||
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
|
||||
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
|
||||
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1)
|
||||
.withPreserveHoodieCommitMetadata(preserveCommitMetadata).build();
|
||||
testInsertAndClustering(clusteringConfig, populateMetaFields, true, SqlQueryEqualityPreCommitValidator.class.getName(), COUNT_SQL_QUERY_FOR_VALIDATION, "");
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("populateMetaFieldsParams")
|
||||
public void testClusteringWithSortColumns(boolean populateMetaFields) throws Exception {
|
||||
@MethodSource("populateMetaFieldsAndPreserveMetadataParams")
|
||||
public void testClusteringWithSortColumns(boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception {
|
||||
// setup clustering config.
|
||||
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
|
||||
.withClusteringSortColumns(populateMetaFields ? "_hoodie_record_key" : "_row_key")
|
||||
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
|
||||
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1)
|
||||
.withPreserveHoodieCommitMetadata(preserveCommitMetadata).build();
|
||||
testInsertAndClustering(clusteringConfig, populateMetaFields, true, SqlQueryEqualityPreCommitValidator.class.getName(), COUNT_SQL_QUERY_FOR_VALIDATION, "");
|
||||
}
|
||||
|
||||
@@ -1401,37 +1413,37 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
private List<HoodieRecord> testInsertAndClustering(HoodieClusteringConfig clusteringConfig, boolean populateMetaFields,
|
||||
boolean completeClustering, String validatorClasses,
|
||||
String sqlQueryForEqualityValidation, String sqlQueryForSingleResultValidation) throws Exception {
|
||||
List<HoodieRecord> allRecords = testInsertTwoBatches(populateMetaFields);
|
||||
Pair<List<HoodieRecord>, List<String>> allRecords = testInsertTwoBatches(populateMetaFields);
|
||||
testClustering(clusteringConfig, populateMetaFields, completeClustering, validatorClasses, sqlQueryForEqualityValidation, sqlQueryForSingleResultValidation, allRecords);
|
||||
return allRecords;
|
||||
return allRecords.getLeft();
|
||||
|
||||
}
|
||||
|
||||
private List<HoodieRecord> testInsertTwoBatches(boolean populateMetaFields) throws IOException {
|
||||
private Pair<List<HoodieRecord>, List<String>> testInsertTwoBatches(boolean populateMetaFields) throws IOException {
|
||||
// create config to not update small files.
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(2000, TRIP_EXAMPLE_SCHEMA, 10, false, populateMetaFields,
|
||||
populateMetaFields ? new Properties() : getPropertiesForKeyGen());
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(config);
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {"2015/03/16"});
|
||||
String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
List<HoodieRecord> records1 = dataGen.generateInserts(commitTime, 200);
|
||||
List<WriteStatus> statuses1 = writeAndVerifyBatch(client, records1, commitTime, populateMetaFields);
|
||||
String commitTime1 = HoodieActiveTimeline.createNewInstantTime();
|
||||
List<HoodieRecord> records1 = dataGen.generateInserts(commitTime1, 200);
|
||||
List<WriteStatus> statuses1 = writeAndVerifyBatch(client, records1, commitTime1, populateMetaFields);
|
||||
Set<HoodieFileGroupId> fileIds1 = getFileGroupIdsFromWriteStatus(statuses1);
|
||||
|
||||
commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
List<HoodieRecord> records2 = dataGen.generateInserts(commitTime, 200);
|
||||
List<WriteStatus> statuses2 = writeAndVerifyBatch(client, records2, commitTime, populateMetaFields);
|
||||
String commitTime2 = HoodieActiveTimeline.createNewInstantTime();
|
||||
List<HoodieRecord> records2 = dataGen.generateInserts(commitTime2, 200);
|
||||
List<WriteStatus> statuses2 = writeAndVerifyBatch(client, records2, commitTime2, populateMetaFields);
|
||||
Set<HoodieFileGroupId> fileIds2 = getFileGroupIdsFromWriteStatus(statuses2);
|
||||
//verify new files are created for 2nd write
|
||||
Set<HoodieFileGroupId> fileIdIntersection = new HashSet<>(fileIds1);
|
||||
fileIdIntersection.retainAll(fileIds2);
|
||||
assertEquals(0, fileIdIntersection.size());
|
||||
return Stream.concat(records1.stream(), records2.stream()).collect(Collectors.toList());
|
||||
return Pair.of(Stream.concat(records1.stream(), records2.stream()).collect(Collectors.toList()), Arrays.asList(commitTime1, commitTime2));
|
||||
}
|
||||
|
||||
private String testClustering(HoodieClusteringConfig clusteringConfig, boolean populateMetaFields, boolean completeClustering,
|
||||
String validatorClasses, String sqlQueryForEqualityValidation, String sqlQueryForSingleResultValidation,
|
||||
List<HoodieRecord> allRecords) throws IOException {
|
||||
private void testClustering(HoodieClusteringConfig clusteringConfig, boolean populateMetaFields, boolean completeClustering,
|
||||
String validatorClasses, String sqlQueryForEqualityValidation, String sqlQueryForSingleResultValidation,
|
||||
Pair<List<HoodieRecord>, List<String>> allRecords) throws IOException {
|
||||
|
||||
HoodieWriteConfig config = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).withAutoCommit(false)
|
||||
.withClusteringConfig(clusteringConfig)
|
||||
@@ -1442,10 +1454,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
if (completeClustering) {
|
||||
String clusteringCommitTime = metaClient.reloadActiveTimeline().getCompletedReplaceTimeline()
|
||||
.getReverseOrderedInstants().findFirst().get().getTimestamp();
|
||||
verifyRecordsWritten(clusteringCommitTime, populateMetaFields, allRecords, clusterMetadata.getWriteStatuses().collect(), config);
|
||||
return clusteringCommitTime;
|
||||
} else {
|
||||
return "";
|
||||
verifyRecordsWritten(clusteringCommitTime, populateMetaFields, allRecords.getLeft(), clusterMetadata.getWriteStatuses().collect(), config);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1454,7 +1463,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
boolean completeClustering,
|
||||
String validatorClasses,
|
||||
String sqlQueryForEqualityValidation, String sqlQueryForSingleResultValidation,
|
||||
List<HoodieRecord> allRecords) throws IOException {
|
||||
Pair<List<HoodieRecord>, List<String>> allRecords) throws IOException {
|
||||
HoodiePreCommitValidatorConfig validatorConfig = HoodiePreCommitValidatorConfig.newBuilder()
|
||||
.withPreCommitValidator(StringUtils.nullToEmpty(validatorClasses))
|
||||
.withPrecommitValidatorEqualitySqlQueries(sqlQueryForEqualityValidation)
|
||||
@@ -1470,7 +1479,11 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(config);
|
||||
String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString();
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> clusterMetadata = client.cluster(clusteringCommitTime, completeClustering);
|
||||
verifyRecordsWritten(clusteringCommitTime, populateMetaFields, allRecords, clusterMetadata.getWriteStatuses().collect(), config);
|
||||
if (config.isPreserveHoodieCommitMetadata() && config.populateMetaFields()) {
|
||||
verifyRecordsWrittenWithPreservedMetadata(new HashSet<>(allRecords.getRight()), allRecords.getLeft(), clusterMetadata.getWriteStatuses().collect());
|
||||
} else {
|
||||
verifyRecordsWritten(clusteringCommitTime, populateMetaFields, allRecords.getLeft(), clusterMetadata.getWriteStatuses().collect(), config);
|
||||
}
|
||||
|
||||
Set<HoodieFileGroupId> replacedFileIds = new HashSet<>();
|
||||
clusterMetadata.getPartitionToReplaceFileIds().entrySet().forEach(partitionFiles ->
|
||||
@@ -1663,13 +1676,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
private void verifyRecordsWritten(String commitTime, boolean populateMetadataField,
|
||||
List<HoodieRecord> expectedRecords, List<WriteStatus> allStatus, HoodieWriteConfig config) throws IOException {
|
||||
List<GenericRecord> records = new ArrayList<>();
|
||||
for (WriteStatus status : allStatus) {
|
||||
Path filePath = new Path(basePath, status.getStat().getPath());
|
||||
records.addAll(BaseFileUtils.getInstance(metaClient).readAvroRecords(jsc.hadoopConfiguration(), filePath));
|
||||
}
|
||||
|
||||
Set<String> expectedKeys = recordsToRecordKeySet(expectedRecords);
|
||||
assertEquals(records.size(), expectedKeys.size());
|
||||
Set<String> expectedKeys = verifyRecordKeys(expectedRecords, allStatus, records);
|
||||
if (config.populateMetaFields()) {
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
@@ -1689,6 +1696,29 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
}
|
||||
}
|
||||
|
||||
@NotNull
|
||||
private Set<String> verifyRecordKeys(List<HoodieRecord> expectedRecords, List<WriteStatus> allStatus, List<GenericRecord> records) {
|
||||
for (WriteStatus status : allStatus) {
|
||||
Path filePath = new Path(basePath, status.getStat().getPath());
|
||||
records.addAll(BaseFileUtils.getInstance(metaClient).readAvroRecords(jsc.hadoopConfiguration(), filePath));
|
||||
}
|
||||
Set<String> expectedKeys = recordsToRecordKeySet(expectedRecords);
|
||||
assertEquals(records.size(), expectedKeys.size());
|
||||
return expectedKeys;
|
||||
}
|
||||
|
||||
private void verifyRecordsWrittenWithPreservedMetadata(Set<String> commitTimes, List<HoodieRecord> expectedRecords, List<WriteStatus> allStatus) {
|
||||
List<GenericRecord> records = new ArrayList<>();
|
||||
Set<String> expectedKeys = verifyRecordKeys(expectedRecords, allStatus, records);
|
||||
Map<String, List<GenericRecord>> recordsByCommitTime = records.stream()
|
||||
.collect(Collectors.groupingBy(r -> r.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()));
|
||||
assertTrue(commitTimes.containsAll(recordsByCommitTime.keySet()));
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertTrue(expectedKeys.contains(recordKey));
|
||||
}
|
||||
}
|
||||
|
||||
private List<WriteStatus> writeAndVerifyBatch(SparkRDDWriteClient client, List<HoodieRecord> inserts, String commitTime, boolean populateMetaFields) throws IOException {
|
||||
client.startCommitWithTime(commitTime);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts, 2);
|
||||
|
||||
@@ -152,6 +152,15 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of);
|
||||
}
|
||||
|
||||
private static Stream<Arguments> populateMetaFieldsAndPreserveMetadataParams() {
|
||||
return Arrays.stream(new Boolean[][] {
|
||||
{true, true},
|
||||
{false, true},
|
||||
{true, false},
|
||||
{false, false}
|
||||
}).map(Arguments::of);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("populateMetaFieldsParams")
|
||||
public void testSimpleInsertAndUpdate(boolean populateMetaFields) throws Exception {
|
||||
@@ -254,25 +263,25 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("populateMetaFieldsParams")
|
||||
public void testSimpleClusteringNoUpdates(boolean populateMetaFields) throws Exception {
|
||||
@MethodSource("populateMetaFieldsAndPreserveMetadataParams")
|
||||
public void testSimpleClusteringNoUpdates(boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception {
|
||||
clean();
|
||||
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
|
||||
testClustering(false, populateMetaFields);
|
||||
testClustering(false, populateMetaFields, preserveCommitMetadata);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("populateMetaFieldsParams")
|
||||
public void testSimpleClusteringWithUpdates(boolean populateMetaFields) throws Exception {
|
||||
@MethodSource("populateMetaFieldsAndPreserveMetadataParams")
|
||||
public void testSimpleClusteringWithUpdates(boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception {
|
||||
clean();
|
||||
init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
|
||||
testClustering(true, populateMetaFields);
|
||||
testClustering(true, populateMetaFields, preserveCommitMetadata);
|
||||
}
|
||||
|
||||
private void testClustering(boolean doUpdates, boolean populateMetaFields) throws Exception {
|
||||
private void testClustering(boolean doUpdates, boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception {
|
||||
// set low compaction small File Size to generate more file groups.
|
||||
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
|
||||
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
|
||||
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).withPreserveHoodieCommitMetadata(preserveCommitMetadata).build();
|
||||
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true, 10L, clusteringConfig);
|
||||
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
|
||||
HoodieWriteConfig cfg = cfgBuilder.build();
|
||||
|
||||
Reference in New Issue
Block a user