[HUDI-1354] Block updates and replace on file groups in clustering (#2275)
* [HUDI-1354] Block updates and replace on file groups in clustering * [HUDI-1354] Block updates and replace on file groups in clustering
This commit is contained in:
@@ -21,6 +21,8 @@ package org.apache.hudi.client;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
@@ -30,15 +32,19 @@ import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.IOType;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.ClusteringUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
@@ -52,10 +58,12 @@ import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieCorruptedDataException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
@@ -78,6 +86,7 @@ import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
@@ -86,6 +95,7 @@ import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_0;
|
||||
import static org.apache.hudi.common.testutils.FileCreateUtils.getBaseFileCountsForPaths;
|
||||
@@ -97,6 +107,8 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAM
|
||||
import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys;
|
||||
import static org.apache.hudi.common.testutils.Transformations.recordsToRecordKeySet;
|
||||
import static org.apache.hudi.common.util.ParquetUtils.readRowKeysFromParquet;
|
||||
import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY;
|
||||
import static org.apache.hudi.config.HoodieClusteringConfig.DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS;
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
@@ -110,6 +122,12 @@ import static org.mockito.Mockito.when;
|
||||
public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestHoodieClientOnCopyOnWriteStorage.class);
|
||||
private static final Map<String, String> STRATEGY_PARAMS = new HashMap<String, String>() {
|
||||
{
|
||||
put("sortColumn", "record_key");
|
||||
}
|
||||
};
|
||||
|
||||
private HoodieTestTable testTable;
|
||||
|
||||
@BeforeEach
|
||||
@@ -681,6 +699,70 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
}
|
||||
}
|
||||
|
||||
private Pair<List<WriteStatus>, List<HoodieRecord>> insertBatchRecords(SparkRDDWriteClient client, String commitTime,
|
||||
Integer recordNum, int expectStatueSize) {
|
||||
client.startCommitWithTime(commitTime);
|
||||
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime, recordNum);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
|
||||
List<WriteStatus> statuses = client.upsert(insertRecordsRDD1, commitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
assertEquals(expectStatueSize, statuses.size(), "check expect statue size.");
|
||||
return Pair.of(statuses, inserts1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateRejectForClustering() throws IOException {
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
|
||||
Properties props = new Properties();
|
||||
props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY, "true");
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(100,
|
||||
TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), props);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(config, false);
|
||||
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
//1. insert to generate 2 file group
|
||||
String commitTime1 = "001";
|
||||
Pair<List<WriteStatus>, List<HoodieRecord>> upsertResult = insertBatchRecords(client, commitTime1, 600, 2);
|
||||
List<HoodieRecord> inserts1 = upsertResult.getValue();
|
||||
List<String> fileGroupIds1 = table.getFileSystemView().getAllFileGroups(testPartitionPath)
|
||||
.map(fileGroup -> fileGroup.getFileGroupId().getFileId()).collect(Collectors.toList());
|
||||
assertEquals(2, fileGroupIds1.size());
|
||||
|
||||
// 2. generate clustering plan for fileGroupIds1 file groups
|
||||
String commitTime2 = "002";
|
||||
List<List<FileSlice>> firstInsertFileSlicesList = table.getFileSystemView().getAllFileGroups(testPartitionPath)
|
||||
.map(fileGroup -> fileGroup.getAllFileSlices().collect(Collectors.toList())).collect(Collectors.toList());
|
||||
List<FileSlice>[] fileSlices = (List<FileSlice>[])firstInsertFileSlicesList.toArray(new List[firstInsertFileSlicesList.size()]);
|
||||
createRequestedReplaceInstant(this.metaClient, commitTime2, fileSlices);
|
||||
|
||||
// 3. insert one record with no updating reject exception, and not merge the small file, just generate a new file group
|
||||
String commitTime3 = "003";
|
||||
insertBatchRecords(client, commitTime3, 1, 1).getKey();
|
||||
List<String> fileGroupIds2 = table.getFileSystemView().getAllFileGroups(testPartitionPath)
|
||||
.map(fileGroup -> fileGroup.getFileGroupId().getFileId()).collect(Collectors.toList());
|
||||
assertEquals(3, fileGroupIds2.size());
|
||||
|
||||
// 4. update one record for the clustering two file groups, throw reject update exception
|
||||
String commitTime4 = "004";
|
||||
client.startCommitWithTime(commitTime4);
|
||||
List<HoodieRecord> insertsAndUpdates3 = new ArrayList<>();
|
||||
insertsAndUpdates3.addAll(dataGen.generateUpdates(commitTime4, inserts1));
|
||||
String assertMsg = String.format("Not allowed to update the clustering files in partition: %s "
|
||||
+ "For pending clustering operations, we are not going to support update for now.", testPartitionPath);
|
||||
assertThrows(HoodieUpsertException.class, () -> {
|
||||
writeClient.upsert(jsc.parallelize(insertsAndUpdates3, 1), commitTime3).collect(); }, assertMsg);
|
||||
|
||||
// 5. insert one record with no updating reject exception, will merge the small file
|
||||
String commitTime5 = "005";
|
||||
List<WriteStatus> statuses = insertBatchRecords(client, commitTime5, 1, 1).getKey();
|
||||
fileGroupIds2.removeAll(fileGroupIds1);
|
||||
assertEquals(fileGroupIds2.get(0), statuses.get(0).getFileId());
|
||||
List<String> firstInsertFileGroupIds4 = table.getFileSystemView().getAllFileGroups(testPartitionPath)
|
||||
.map(fileGroup -> fileGroup.getFileGroupId().getFileId()).collect(Collectors.toList());
|
||||
assertEquals(3, firstInsertFileGroupIds4.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test scenario of new file-group getting added during upsert().
|
||||
*/
|
||||
@@ -1467,8 +1549,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
String schemaStr = useNullSchema ? NULL_SCHEMA : TRIP_EXAMPLE_SCHEMA;
|
||||
return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize);
|
||||
}
|
||||
|
||||
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize) {
|
||||
return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, new Properties());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, Properties props) {
|
||||
HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr);
|
||||
return builder
|
||||
.withCompactionConfig(
|
||||
@@ -1479,6 +1565,19 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
HoodieStorageConfig.newBuilder()
|
||||
.hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200))
|
||||
.parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build())
|
||||
.withProps(props)
|
||||
.build();
|
||||
}
|
||||
|
||||
protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient metaClient, String clusterTime, List<FileSlice>[] fileSlices) throws IOException {
|
||||
HoodieClusteringPlan clusteringPlan =
|
||||
ClusteringUtils.createClusteringPlan(DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS, STRATEGY_PARAMS, fileSlices, Collections.emptyMap());
|
||||
|
||||
HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, clusterTime);
|
||||
HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder()
|
||||
.setClusteringPlan(clusteringPlan).setOperationType(WriteOperationType.CLUSTER.name()).build();
|
||||
metaClient.getActiveTimeline().saveToPendingReplaceCommit(clusteringInstant, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata));
|
||||
return clusteringInstant;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user