1
0

[HUDI-2442] Change default values for certin clustering configs (#3875)

This commit is contained in:
Sagar Sumit
2021-11-10 14:23:24 +05:30
committed by GitHub
parent a40ac62e0c
commit 187bedf795
5 changed files with 18 additions and 23 deletions

View File

@@ -750,7 +750,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringSortColumns("_row_key")
.withClusteringSortColumns("_row_key").withInlineClustering(true)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
HoodieWriteConfig newWriteConfig = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER)

View File

@@ -1356,7 +1356,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
public void testSimpleClustering(boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).withInlineClustering(true)
.withPreserveHoodieCommitMetadata(preserveCommitMetadata).build();
testInsertAndClustering(clusteringConfig, populateMetaFields, true, false, SqlQueryEqualityPreCommitValidator.class.getName(), COUNT_SQL_QUERY_FOR_VALIDATION, "");
}
@@ -1367,7 +1367,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringSortColumns(populateMetaFields ? "_hoodie_record_key" : "_row_key")
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).withInlineClustering(true)
.withPreserveHoodieCommitMetadata(preserveCommitMetadata).build();
testInsertAndClustering(clusteringConfig, populateMetaFields, true, false, SqlQueryEqualityPreCommitValidator.class.getName(), COUNT_SQL_QUERY_FOR_VALIDATION, "");
}
@@ -1391,7 +1391,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
public void testPendingClusteringRollback(boolean populateMetaFields) throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).withInlineClustering(true).build();
// start clustering, but don't commit
List<HoodieRecord> allRecords = testInsertAndClustering(clusteringConfig, populateMetaFields, false);
@@ -1422,7 +1422,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
public void testClusteringWithFailingValidator() throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringSortColumns("_hoodie_record_key")
.withClusteringSortColumns("_hoodie_record_key").withInlineClustering(true)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
try {
testInsertAndClustering(clusteringConfig, true, true, false, FailingPreCommitValidator.class.getName(), COUNT_SQL_QUERY_FOR_VALIDATION, "");
@@ -1436,7 +1436,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
public void testClusteringInvalidConfigForSqlQueryValidator() throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).withInlineClustering(true).build();
try {
testInsertAndClustering(clusteringConfig, false, true, false, SqlQueryEqualityPreCommitValidator.class.getName(), "", "");
fail("expected pre-commit clustering validation to fail because sql query is not configured");
@@ -1449,7 +1449,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
public void testClusteringInvalidConfigForSqlQuerySingleResultValidator() throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).withInlineClustering(true).build();
testInsertAndClustering(clusteringConfig, false, true, false, SqlQuerySingleResultPreCommitValidator.class.getName(),
"", COUNT_SQL_QUERY_FOR_VALIDATION + "#400");
@@ -1459,7 +1459,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
public void testClusteringInvalidConfigForSqlQuerySingleResultValidatorFailure() throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).withInlineClustering(true).build();
try {
testInsertAndClustering(clusteringConfig, false, true, false, SqlQuerySingleResultPreCommitValidator.class.getName(),

View File

@@ -93,6 +93,7 @@ class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTes
.withClusteringConfig(HoodieClusteringConfig.newBuilder()
.withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0)
.withInlineClustering(true)
.withInlineClusteringNumCommits(1)
.withPreserveHoodieCommitMetadata(preserveCommitMetadata).build())
.withRollbackUsingMarkers(false);