1
0

[HUDI-2144]Bug-Fix:Offline clustering(HoodieClusteringJob) will cause insert action losing data (#3240)

* fixed

* add testUpsertPartitionerWithSmallFileHandlingAndClusteringPlan ut

* fix CheckStyle

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
This commit is contained in:
zhangyue19921010
2021-07-13 09:14:17 +08:00
committed by GitHub
parent ca440ccf88
commit c8a2033c27
3 changed files with 99 additions and 2 deletions

View File

@@ -146,7 +146,7 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends Partiti
* @return smallFiles not in clustering
*/
private List<SmallFile> filterSmallFilesInClustering(final Set<String> pendingClusteringFileGroupsId, final List<SmallFile> smallFiles) {
if (this.config.isClusteringEnabled()) {
if (!pendingClusteringFileGroupsId.isEmpty()) {
return smallFiles.stream()
.filter(smallFile -> !pendingClusteringFileGroupsId.contains(smallFile.location.getFileId())).collect(Collectors.toList());
} else {

View File

@@ -18,21 +18,26 @@
package org.apache.hudi.table.action.commit;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.testutils.ClusteringTestUtils;
import org.apache.hudi.common.testutils.CompactionTestUtils;
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieHBaseIndexConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
@@ -350,6 +355,44 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
"Should be assigned to only file id not pending compaction which is 2");
}
@Test
public void testUpsertPartitionerWithSmallFileHandlingAndClusteringPlan() throws Exception {
final String testPartitionPath = DEFAULT_PARTITION_PATHS[0];
// create HoodieWriteConfig and set inline and async clustering disable here.
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder().build())
.withClusteringConfig(HoodieClusteringConfig.newBuilder().withInlineClustering(false).withAsyncClustering(false).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1000 * 1024).parquetMaxFileSize(1000 * 1024).build())
.build();
// create file slice with instantTime 001 and build clustering plan including this created 001 file slice.
HoodieClusteringPlan clusteringPlan = ClusteringTestUtils.createClusteringPlan(metaClient, "001", "1");
// create requested replace commit
HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder()
.setClusteringPlan(clusteringPlan).setOperationType(WriteOperationType.CLUSTER.name()).build();
FileCreateUtils.createRequestedReplaceCommit(basePath,"002", Option.of(requestedReplaceMetadata));
// create file slice 002
FileCreateUtils.createBaseFile(basePath, testPartitionPath, "002", "2", 1);
FileCreateUtils.createCommit(basePath, "002");
metaClient = HoodieTableMetaClient.reload(metaClient);
// generate new data to be ingested
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath});
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("003", 100);
WorkloadProfile profile = new WorkloadProfile(buildProfile(jsc.parallelize(insertRecords)));
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
// create UpsertPartitioner
UpsertPartitioner partitioner = new UpsertPartitioner(profile, context, table, config);
// for now we have file slice1 and file slice2 and file slice1 is contained in pending clustering plan
// So that only file slice2 can be used for ingestion.
assertEquals(1, partitioner.smallFiles.size(), "Should have 1 small file to be ingested.");
}
@Test
public void testUpsertPartitionerWithSmallFileHandlingWithCanIndexLogFiles() throws Exception {
// Note this is used because it is same partition path used in CompactionTestUtils.createCompactionPlan()