[HUDI-1352] Add FileSystemView APIs to query pending clustering operations (#2202)
This commit is contained in:
@@ -21,10 +21,12 @@ package org.apache.hudi.common.table.view;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieFSPermission;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.avro.model.HoodiePath;
|
||||
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
|
||||
import org.apache.hudi.common.bootstrap.FileStatusUtils;
|
||||
import org.apache.hudi.common.bootstrap.index.BootstrapIndex.IndexWriter;
|
||||
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
|
||||
@@ -49,6 +51,7 @@ import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.ClusteringUtils;
|
||||
import org.apache.hudi.common.util.CommitUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
@@ -1424,6 +1427,63 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
assertEquals(actualReplacedFileIds, allReplacedFileIds);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPendingClusteringOperations() throws IOException {
|
||||
String partitionPath1 = "2020/06/27";
|
||||
new File(basePath + "/" + partitionPath1).mkdirs();
|
||||
|
||||
// create 2 fileId in partition1 - fileId1 is replaced later on.
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
assertFalse(roView.getLatestBaseFiles(partitionPath1)
|
||||
.anyMatch(dfile -> dfile.getFileId().equals(fileId1) || dfile.getFileId().equals(fileId2) || dfile.getFileId().equals(fileId3)),
|
||||
"No commit, should not find any data file");
|
||||
// Only one commit
|
||||
String commitTime1 = "1";
|
||||
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
|
||||
String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
|
||||
String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
|
||||
new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile();
|
||||
new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile();
|
||||
new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile();
|
||||
|
||||
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
|
||||
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
|
||||
saveAsComplete(commitTimeline, instant1, Option.empty());
|
||||
refreshFsView();
|
||||
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
|
||||
.filter(dfile -> dfile.getFileId().equals(fileId1)).count());
|
||||
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
|
||||
.filter(dfile -> dfile.getFileId().equals(fileId2)).count());
|
||||
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
|
||||
.filter(dfile -> dfile.getFileId().equals(fileId3)).count());
|
||||
|
||||
List<FileSlice>[] fileSliceGroups = new List[] {
|
||||
Collections.singletonList(fsView.getLatestFileSlice(partitionPath1, fileId1).get()),
|
||||
Collections.singletonList(fsView.getLatestFileSlice(partitionPath1, fileId2).get())
|
||||
};
|
||||
|
||||
// create pending clustering operation - fileId1, fileId2 are being clustered in different groups
|
||||
HoodieClusteringPlan plan = ClusteringUtils.createClusteringPlan("strategy", new HashMap<>(),
|
||||
fileSliceGroups, Collections.emptyMap());
|
||||
|
||||
String clusterTime = "2";
|
||||
HoodieInstant instant2 = new HoodieInstant(State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, clusterTime);
|
||||
HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder()
|
||||
.setClusteringPlan(plan).setOperationType(WriteOperationType.CLUSTER.name()).build();
|
||||
metaClient.getActiveTimeline().saveToPendingReplaceCommit(instant2, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata));
|
||||
|
||||
//make sure view doesnt include fileId1
|
||||
refreshFsView();
|
||||
Set<String> fileIds =
|
||||
fsView.getFileGroupsInPendingClustering().map(e -> e.getLeft().getFileId()).collect(Collectors.toSet());
|
||||
assertTrue(fileIds.contains(fileId1));
|
||||
assertTrue(fileIds.contains(fileId2));
|
||||
assertFalse(fileIds.contains(fileId3));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
|
||||
@@ -0,0 +1,142 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.common.util;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
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.testutils.HoodieCommonTestHarness;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
/**
|
||||
* Tests for {@link ClusteringUtils}.
|
||||
*/
|
||||
public class TestClusteringUtils extends HoodieCommonTestHarness {
|
||||
|
||||
private static final String CLUSTERING_STRATEGY_CLASS = "org.apache.hudi.DefaultClusteringStrategy";
|
||||
private static final Map<String, String> STRATEGY_PARAMS = new HashMap<String, String>() {
|
||||
{
|
||||
put("sortColumn", "record_key");
|
||||
}
|
||||
};
|
||||
|
||||
@BeforeEach
|
||||
public void init() throws IOException {
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClusteringPlanMultipleInstants() throws Exception {
|
||||
String partitionPath1 = "partition1";
|
||||
List<String> fileIds1 = new ArrayList<>();
|
||||
fileIds1.add(UUID.randomUUID().toString());
|
||||
fileIds1.add(UUID.randomUUID().toString());
|
||||
String clusterTime1 = "1";
|
||||
createRequestedReplaceInstant(partitionPath1, clusterTime1, fileIds1);
|
||||
|
||||
List<String> fileIds2 = new ArrayList<>();
|
||||
fileIds2.add(UUID.randomUUID().toString());
|
||||
fileIds2.add(UUID.randomUUID().toString());
|
||||
fileIds2.add(UUID.randomUUID().toString());
|
||||
|
||||
List<String> fileIds3 = new ArrayList<>();
|
||||
fileIds3.add(UUID.randomUUID().toString());
|
||||
|
||||
String clusterTime = "2";
|
||||
createRequestedReplaceInstant(partitionPath1, clusterTime, fileIds2, fileIds3);
|
||||
|
||||
// create replace.requested without clustering plan. this instant should be ignored by ClusteringUtils
|
||||
createRequestedReplaceInstantNotClustering("3");
|
||||
|
||||
// create replace.requested without any metadata content. This instant should be ignored by ClusteringUtils
|
||||
metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "4"));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
assertEquals(4, metaClient.getActiveTimeline().filterPendingReplaceTimeline().countInstants());
|
||||
|
||||
Map<HoodieFileGroupId, HoodieInstant> fileGroupToInstantMap =
|
||||
ClusteringUtils.getAllFileGroupsInPendingClusteringPlans(metaClient);
|
||||
assertEquals(fileIds1.size() + fileIds2.size() + fileIds3.size(), fileGroupToInstantMap.size());
|
||||
validateClusteringInstant(fileIds1, partitionPath1, clusterTime1, fileGroupToInstantMap);
|
||||
validateClusteringInstant(fileIds2, partitionPath1, clusterTime, fileGroupToInstantMap);
|
||||
validateClusteringInstant(fileIds3, partitionPath1, clusterTime, fileGroupToInstantMap);
|
||||
}
|
||||
|
||||
private void validateClusteringInstant(List<String> fileIds, String partitionPath,
|
||||
String expectedInstantTime, Map<HoodieFileGroupId, HoodieInstant> fileGroupToInstantMap) {
|
||||
for (String fileId : fileIds) {
|
||||
assertEquals(expectedInstantTime, fileGroupToInstantMap.get(new HoodieFileGroupId(partitionPath, fileId)).getTimestamp());
|
||||
}
|
||||
}
|
||||
|
||||
private HoodieInstant createRequestedReplaceInstantNotClustering(String instantTime) throws IOException {
|
||||
HoodieInstant newRequestedInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime);
|
||||
HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder()
|
||||
.setOperationType(WriteOperationType.UNKNOWN.name()).build();
|
||||
metaClient.getActiveTimeline().saveToPendingReplaceCommit(newRequestedInstant, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata));
|
||||
return newRequestedInstant;
|
||||
}
|
||||
|
||||
private HoodieInstant createRequestedReplaceInstant(String partitionPath1, String clusterTime, List<String>... fileIds) throws IOException {
|
||||
List<FileSlice>[] fileSliceGroups = new List[fileIds.length];
|
||||
for (int i = 0; i < fileIds.length; i++) {
|
||||
fileSliceGroups[i] = fileIds[i].stream().map(fileId -> generateFileSlice(partitionPath1, fileId, "0")).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
HoodieClusteringPlan clusteringPlan =
|
||||
ClusteringUtils.createClusteringPlan(CLUSTERING_STRATEGY_CLASS, STRATEGY_PARAMS, fileSliceGroups, 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;
|
||||
}
|
||||
|
||||
private FileSlice generateFileSlice(String partitionPath, String fileId, String baseInstant) {
|
||||
FileSlice fs = new FileSlice(new HoodieFileGroupId(partitionPath, fileId), baseInstant);
|
||||
fs.setBaseFile(new HoodieBaseFile(FSUtils.makeDataFileName(baseInstant, "1-0-1", fileId)));
|
||||
return fs;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user