YuweiXiao commented on code in PR #4480: URL: https://github.com/apache/hudi/pull/4480#discussion_r869815401
########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java: ########## @@ -0,0 +1,206 @@ +/* + * 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.index.bucket; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.common.model.ConsistentHashingNode; +import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * Consistent hashing bucket index implementation, with auto-adjust bucket number. + * NOTE: bucket resizing is triggered by clustering. + */ +public class HoodieSparkConsistentBucketIndex extends HoodieBucketIndex { + + private static final Logger LOG = LogManager.getLogger(HoodieSparkConsistentBucketIndex.class); + + public HoodieSparkConsistentBucketIndex(HoodieWriteConfig config) { + super(config); + } + + @Override + public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> writeStatuses, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { + return writeStatuses; + } + + /** + * Do nothing. + * A failed write may create a hashing metadata for a partition. In this case, we still do nothing when rolling back + * the failed write. Because the hashing metadata created by a writer must have 00000000000000 timestamp and can be viewed + * as the initialization of a partition rather than as a part of the failed write. + */ + @Override + public boolean rollbackCommit(String instantTime) { + return true; + } + + @Override + protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List<String> partitionPath) { + return new ConsistentBucketIndexLocationMapper(table, partitionPath); + } + + /** + * Load hashing metadata of the given partition, if it is not existed, create a new one (also persist it into storage) + * + * @param table hoodie table + * @param partition table partition + * @return Consistent hashing metadata + */ + public HoodieConsistentHashingMetadata loadOrCreateMetadata(HoodieTable table, String partition) { + HoodieConsistentHashingMetadata metadata = loadMetadata(table, partition); + if (metadata != null) { + return metadata; + } + + // There is no metadata, so try to create a new one and save it. + metadata = new HoodieConsistentHashingMetadata(partition, numBuckets); + if (saveMetadata(table, metadata, false)) { + return metadata; + } + + // The creation failed, so try load metadata again. Concurrent creation of metadata should have succeeded. + // Note: the consistent problem of cloud storage is handled internal in the HoodieWrapperFileSystem, i.e., ConsistentGuard + metadata = loadMetadata(table, partition); + ValidationUtils.checkState(metadata != null, "Failed to load or create metadata, partition: " + partition); + return metadata; + } + + /** + * Load hashing metadata of the given partition, if it is not existed, return null + * + * @param table hoodie table + * @param partition table partition + * @return Consistent hashing metadata or null if it does not exist + */ + public static HoodieConsistentHashingMetadata loadMetadata(HoodieTable table, String partition) { + Path metadataPath = FSUtils.getPartitionPath(table.getMetaClient().getHashingMetadataPath(), partition); + + try { + if (!table.getMetaClient().getFs().exists(metadataPath)) { + return null; + } + FileStatus[] metaFiles = table.getMetaClient().getFs().listStatus(metadataPath); + final HoodieTimeline completedCommits = table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(); + Predicate<FileStatus> metaFilePredicate = fileStatus -> { + String filename = fileStatus.getPath().getName(); + if (!filename.contains(HoodieConsistentHashingMetadata.HASHING_METADATA_FILE_SUFFIX)) { + return false; + } + String timestamp = HoodieConsistentHashingMetadata.getTimestampFromFile(filename); + return completedCommits.containsInstant(timestamp) || timestamp.equals(HoodieTimeline.INIT_INSTANT_TS); + }; + + // Get a valid hashing metadata with the largest (latest) timestamp + FileStatus metaFile = Arrays.stream(metaFiles).filter(metaFilePredicate) + .max(Comparator.comparing(a -> a.getPath().getName())).orElse(null); + + if (metaFile == null) { + return null; + } + + byte[] content = FileIOUtils.readAsByteArray(table.getMetaClient().getFs().open(metaFile.getPath())); + return HoodieConsistentHashingMetadata.fromBytes(content); + } catch (IOException e) { + LOG.warn("Error when loading hashing metadata, partition: " + partition, e); Review Comment: Fixed. ########## hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java: ########## @@ -0,0 +1,245 @@ +/* + * 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.client.functional; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import org.apache.hudi.hadoop.RealtimeFileStatus; +import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; +import org.apache.hudi.testutils.MetadataMergeWriteStatus; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.Properties; +import java.util.Random; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Test consistent hashing index + */ +@Tag("functional") +public class TestConsistentBucketIndex extends HoodieClientTestHarness { + + private final Random random = new Random(1); + private HoodieIndex index; + private HoodieWriteConfig config; + + private static Stream<Arguments> configParams() { + // preserveMetaField, partitioned + Object[][] data = new Object[][] { + {true, false}, + {false, false}, + {true, true}, + {false, true}, + }; + return Stream.of(data).map(Arguments::of); + } + + private void setUp(boolean populateMetaFields, boolean partitioned) throws Exception { + initPath(); + initSparkContexts(); + if (partitioned) { + initTestDataGenerator(); + } else { + initTestDataGenerator(new String[] {""}); + } + initFileSystem(); + Properties props = populateMetaFields ? new Properties() : getPropertiesForKeyGen(); + props.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, props); + config = getConfigBuilder() + .withProperties(props) + .withIndexConfig(HoodieIndexConfig.newBuilder() + .fromProperties(props) + .withIndexType(HoodieIndex.IndexType.BUCKET) + .withIndexKeyField("_row_key") + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING) + .build()) + .withAutoCommit(false) + .build(); + writeClient = getHoodieWriteClient(config); + index = writeClient.getIndex(); + } + + @AfterEach + public void tearDown() throws IOException { + cleanupResources(); + } + + /** + * Test bucket index tagging (always tag regardless of the write status) + * Test bucket index tagging consistency, two tagging result should be same + * + * @param populateMetaFields + * @param partitioned + * @throws Exception + */ + @ParameterizedTest + @MethodSource("configParams") + public void testTagLocation(boolean populateMetaFields, boolean partitioned) throws Exception { + setUp(populateMetaFields, partitioned); + String newCommitTime = "001"; + int totalRecords = 20 + random.nextInt(20); + List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords); + JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 2); + + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + + // The records should be tagged anyway, even though it is the first time doing tagging + List<HoodieRecord> taggedRecord = tagLocation(index, writeRecords, hoodieTable).collect(); + Assertions.assertTrue(taggedRecord.stream().allMatch(r -> r.isCurrentLocationKnown())); + + // Tag again, the records should get the same location (hashing metadata has been persisted after the first tagging) + List<HoodieRecord> taggedRecord2 = tagLocation(index, writeRecords, hoodieTable).collect(); + for (HoodieRecord ref : taggedRecord) { + for (HoodieRecord record : taggedRecord2) { + if (ref.getRecordKey().equals(record.getRecordKey())) { + Assertions.assertEquals(ref.getCurrentLocation(), record.getCurrentLocation()); + break; + } + } + } + } + + @ParameterizedTest + @MethodSource("configParams") + public void testWriteData(boolean populateMetaFields, boolean partitioned) throws Exception { + setUp(populateMetaFields, partitioned); + String newCommitTime = "001"; + int totalRecords = 20 + random.nextInt(20); + List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords); + JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 2); + + metaClient = HoodieTableMetaClient.reload(metaClient); + + // Insert totalRecords records + writeClient.startCommitWithTime(newCommitTime); + List<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime).collect(); + org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues); + boolean success = writeClient.commitStats(newCommitTime, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType()); Review Comment: Fixed. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
