1
0

[HUDI-2550] Expand File-Group candidates list for appending for MOR tables (#3986)

This commit is contained in:
Alexey Kudinkin
2021-11-22 16:19:59 -08:00
committed by GitHub
parent fe57e9beea
commit 3bdab01a49
4 changed files with 124 additions and 57 deletions

View File

@@ -56,6 +56,7 @@ import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
@@ -336,7 +337,6 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
HoodieCompactionPlan plan = CompactionTestUtils.createCompactionPlan(metaClient, "001", "002", 1, true, false);
FileCreateUtils.createRequestedCompactionCommit(basePath, "002", plan);
// Simulate one more commit so that inflight compaction is considered when building file groups in file system view
//
FileCreateUtils.createBaseFile(basePath, testPartitionPath, "003", "2", 1);
FileCreateUtils.createCommit(basePath, "003");
@@ -434,6 +434,49 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
"Insert should be assigned to fg1");
}
@Test
public void testUpsertPartitionerWithSmallFileHandlingPickingMultipleCandidates() throws Exception {
final String partitionPath = DEFAULT_PARTITION_PATHS[0];
HoodieWriteConfig config =
makeHoodieClientConfigBuilder()
.withMergeSmallFileGroupCandidatesLimit(3)
.withStorageConfig(
HoodieStorageConfig.newBuilder()
.parquetMaxFileSize(2048)
.build()
)
.build();
// Bootstrap base files ("small-file targets")
FileCreateUtils.createBaseFile(basePath, partitionPath, "002", "fg-1", 1024);
FileCreateUtils.createBaseFile(basePath, partitionPath, "002", "fg-2", 1024);
FileCreateUtils.createBaseFile(basePath, partitionPath, "002", "fg-3", 1024);
FileCreateUtils.createCommit(basePath, "002");
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {partitionPath});
// Default estimated record size will be 1024 based on last file group created.
// Only 1 record can be added to small file
WorkloadProfile profile =
new WorkloadProfile(buildProfile(jsc.parallelize(dataGenerator.generateInserts("003", 3))));
HoodieTableMetaClient reloadedMetaClient = HoodieTableMetaClient.reload(this.metaClient);
HoodieSparkTable<?> table = HoodieSparkTable.create(config, context, reloadedMetaClient);
SparkUpsertDeltaCommitPartitioner<?> partitioner = new SparkUpsertDeltaCommitPartitioner<>(profile, context, table, config);
assertEquals(3, partitioner.numPartitions());
assertEquals(
Arrays.asList(
new BucketInfo(BucketType.UPDATE, "fg-1", partitionPath),
new BucketInfo(BucketType.UPDATE, "fg-2", partitionPath),
new BucketInfo(BucketType.UPDATE, "fg-3", partitionPath)
),
partitioner.getBucketInfos());
}
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() {
// Prepare the AvroParquetIO
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString());