1
0

[HUDI-3123] consistent hashing index: basic write path (upsert/insert) (#4480)

1. basic write path(insert/upsert) implementation
 2. adapt simple bucket index
This commit is contained in:
Yuwei XIAO
2022-05-16 11:07:01 +08:00
committed by GitHub
parent 1fded18dff
commit 61030d8e7a
41 changed files with 1510 additions and 237 deletions

View File

@@ -179,7 +179,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
initTable(WriteOperationType.INSERT, Option.ofNullable(instantTime));
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient());
HoodieWriteMetadata<HoodieData<WriteStatus>> result = table.insert(context,instantTime, HoodieJavaRDD.of(records));
HoodieWriteMetadata<HoodieData<WriteStatus>> result = table.insert(context, instantTime, HoodieJavaRDD.of(records));
HoodieWriteMetadata<JavaRDD<WriteStatus>> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses()));
return postWrite(resultRDD, instantTime, table);
}

View File

@@ -112,6 +112,11 @@ public class HoodieJavaRDD<T> extends HoodieData<T> {
return HoodieJavaRDD.of(rddData.mapPartitions(func::apply, preservesPartitioning));
}
@Override
public <O> HoodieData<O> mapPartitions(SerializableFunction<Iterator<T>, Iterator<O>> func) {
return HoodieJavaRDD.of(rddData.mapPartitions(func::apply));
}
@Override
public <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func) {
return HoodieJavaRDD.of(rddData.flatMap(e -> func.apply(e)));

View File

@@ -28,7 +28,8 @@ import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.index.bloom.HoodieBloomIndex;
import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
import org.apache.hudi.index.bloom.SparkHoodieBloomIndexHelper;
import org.apache.hudi.index.bucket.HoodieBucketIndex;
import org.apache.hudi.index.bucket.HoodieSimpleBucketIndex;
import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex;
import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex;
import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex;
@@ -56,8 +57,6 @@ public final class SparkHoodieIndexFactory {
return new SparkHoodieHBaseIndex(config);
case INMEMORY:
return new HoodieInMemoryHashIndex(config);
case BUCKET:
return new HoodieBucketIndex(config);
case BLOOM:
return new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
case GLOBAL_BLOOM:
@@ -66,6 +65,15 @@ public final class SparkHoodieIndexFactory {
return new HoodieSimpleIndex(config, getKeyGeneratorForSimpleIndex(config));
case GLOBAL_SIMPLE:
return new HoodieGlobalSimpleIndex(config, getKeyGeneratorForSimpleIndex(config));
case BUCKET:
switch (config.getBucketIndexEngineType()) {
case SIMPLE:
return new HoodieSimpleBucketIndex(config);
case CONSISTENT_HASHING:
return new HoodieSparkConsistentBucketIndex(config);
default:
throw new HoodieIndexException("Unknown bucket index engine type: " + config.getBucketIndexEngineType());
}
default:
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
}
@@ -90,6 +98,8 @@ public final class SparkHoodieIndexFactory {
return false;
case GLOBAL_SIMPLE:
return true;
case BUCKET:
return false;
default:
return createIndex(config).isGlobal();
}

View File

@@ -0,0 +1,210 @@
/*
* 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.Option;
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.error("Error when loading hashing metadata, partition: " + partition, e);
throw new HoodieIndexException("Error while loading hashing metadata", e);
}
}
/**
* Save metadata into storage
*
* @param table hoodie table
* @param metadata hashing metadata to be saved
* @param overwrite whether to overwrite existing metadata
* @return true if the metadata is saved successfully
*/
private static boolean saveMetadata(HoodieTable table, HoodieConsistentHashingMetadata metadata, boolean overwrite) {
HoodieWrapperFileSystem fs = table.getMetaClient().getFs();
Path dir = FSUtils.getPartitionPath(table.getMetaClient().getHashingMetadataPath(), metadata.getPartitionPath());
Path fullPath = new Path(dir, metadata.getFilename());
try (FSDataOutputStream fsOut = fs.create(fullPath, overwrite)) {
byte[] bytes = metadata.toBytes();
fsOut.write(bytes);
fsOut.close();
return true;
} catch (IOException e) {
LOG.warn("Failed to update bucket metadata: " + metadata, e);
}
return false;
}
public class ConsistentBucketIndexLocationMapper implements BucketIndexLocationMapper {
/**
* Mapping from partitionPath -> bucket identifier
*/
private final Map<String, ConsistentBucketIdentifier> partitionToIdentifier;
public ConsistentBucketIndexLocationMapper(HoodieTable table, List<String> partitions) {
// TODO maybe parallel
partitionToIdentifier = partitions.stream().collect(Collectors.toMap(p -> p, p -> {
HoodieConsistentHashingMetadata metadata = loadOrCreateMetadata(table, p);
return new ConsistentBucketIdentifier(metadata);
}));
}
@Override
public Option<HoodieRecordLocation> getRecordLocation(HoodieKey key, String partitionPath) {
ConsistentHashingNode node = partitionToIdentifier.get(partitionPath).getBucket(key, indexKeyFields);
if (!StringUtils.isNullOrEmpty(node.getFileIdPrefix())) {
/**
* Dynamic Bucket Index doesn't need the instant time of the latest file group.
* We add suffix 0 here to the file uuid, following the naming convention, i.e., fileId = [uuid]_[numWrites]
*/
return Option.of(new HoodieRecordLocation(null, FSUtils.createNewFileId(node.getFileIdPrefix(), 0)));
}
LOG.error("Consistent hashing node has no file group, partition: " + partitionPath + ", meta: "
+ partitionToIdentifier.get(partitionPath).getMetadata().getFilename() + ", record_key: " + key.toString());
throw new HoodieIndexException("Failed to getBucket as hashing node has no file group");
}
}
}

View File

@@ -0,0 +1,250 @@
/*
* 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());
Assertions.assertTrue(success);
metaClient = HoodieTableMetaClient.reload(metaClient);
// The number of distinct fileId should be the same as total log file numbers
Assertions.assertEquals(writeStatues.stream().map(WriteStatus::getFileId).distinct().count(),
Arrays.stream(dataGen.getPartitionPaths()).mapToInt(p -> Objects.requireNonNull(listStatus(p, true)).length).sum());
Assertions.assertEquals(totalRecords, readRecords(dataGen.getPartitionPaths(), populateMetaFields).size());
// Upsert the same set of records, the number of records should be same
newCommitTime = "002";
writeClient.startCommitWithTime(newCommitTime);
writeStatues = writeClient.upsert(writeRecords, newCommitTime).collect();
org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues);
success = writeClient.commitStats(newCommitTime, writeStatues.stream()
.map(WriteStatus::getStat)
.collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType());
Assertions.assertTrue(success);
// The number of log file should double after this insertion
long numberOfLogFiles = Arrays.stream(dataGen.getPartitionPaths())
.mapToInt(p -> {
return Arrays.stream(listStatus(p, true)).mapToInt(fs ->
fs instanceof RealtimeFileStatus ? ((RealtimeFileStatus) fs).getDeltaLogFiles().size() : 1).sum();
}).sum();
Assertions.assertEquals(writeStatues.stream().map(WriteStatus::getFileId).distinct().count() * 2, numberOfLogFiles);
// The record number should remain same because of deduplication
Assertions.assertEquals(totalRecords, readRecords(dataGen.getPartitionPaths(), populateMetaFields).size());
metaClient = HoodieTableMetaClient.reload(metaClient);
// Upsert new set of records, and validate the total number of records
newCommitTime = "003";
records = dataGen.generateInserts(newCommitTime, totalRecords);
writeRecords = jsc.parallelize(records, 2);
writeClient.startCommitWithTime(newCommitTime);
writeStatues = writeClient.upsert(writeRecords, newCommitTime).collect();
org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues);
success = writeClient.commitStats(newCommitTime, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()),
Option.empty(), metaClient.getCommitActionType());
Assertions.assertTrue(success);
Assertions.assertEquals(totalRecords * 2, readRecords(dataGen.getPartitionPaths(), populateMetaFields).size());
}
private List<GenericRecord> readRecords(String[] partitions, boolean populateMetaFields) {
return HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf,
Arrays.stream(partitions).map(p -> Paths.get(basePath, p).toString()).collect(Collectors.toList()),
basePath, new JobConf(hadoopConf), true, populateMetaFields);
}
private FileStatus[] listStatus(String p, boolean realtime) {
JobConf jobConf = new JobConf(hadoopConf);
FileInputFormat.setInputPaths(jobConf, Paths.get(basePath, p).toString());
FileInputFormat format = HoodieInputFormatUtils.getInputFormat(HoodieFileFormat.PARQUET, realtime, jobConf);
try {
if (realtime) {
return ((HoodieParquetRealtimeInputFormat) format).listStatus(jobConf);
} else {
return ((HoodieParquetInputFormat) format).listStatus(jobConf);
}
} catch (IOException e) {
e.printStackTrace();
return null;
}
}
private HoodieWriteConfig.Builder getConfigBuilder() {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
.withWriteStatusClass(MetadataMergeWriteStatus.class)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
.forTable("test-trip-table")
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
}
}

View File

@@ -131,6 +131,9 @@ public class TestHoodieIndex extends TestHoodieMetadataBase {
HoodieIndexConfig.Builder indexBuilder = HoodieIndexConfig.newBuilder().withIndexType(indexType)
.fromProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen())
.withIndexType(indexType);
if (indexType == IndexType.BUCKET) {
indexBuilder.withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE);
}
config = getConfigBuilder()
.withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen())
.withRollbackUsingMarkers(rollbackUsingMarkers)

View File

@@ -26,7 +26,8 @@ import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.hudi.index.bloom.HoodieBloomIndex;
import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
import org.apache.hudi.index.bucket.HoodieBucketIndex;
import org.apache.hudi.index.bucket.HoodieSimpleBucketIndex;
import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex;
import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex;
import org.apache.hudi.index.simple.HoodieSimpleIndex;
@@ -88,8 +89,15 @@ public class TestHoodieIndexConfigs {
break;
case BUCKET:
config = clientConfigBuilder.withPath(basePath)
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.BUCKET).build()).build();
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieBucketIndex);
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.BUCKET)
.withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE).build()).build();
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieSimpleBucketIndex);
config = clientConfigBuilder.withPath(basePath)
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.BUCKET)
.withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING).build())
.build();
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieSparkConsistentBucketIndex);
break;
default:
// no -op. just for checkstyle errors

View File

@@ -52,10 +52,10 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieBucketIndex extends HoodieClientTestHarness {
public class TestHoodieSimpleBucketIndex extends HoodieClientTestHarness {
private static final Logger LOG = LogManager.getLogger(TestHoodieBucketIndex.class);
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBucketIndex.class, "/exampleSchema.avsc", true);
private static final Logger LOG = LogManager.getLogger(TestHoodieSimpleBucketIndex.class);
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieSimpleBucketIndex.class, "/exampleSchema.avsc", true);
private static final int NUM_BUCKET = 8;
@BeforeEach
@@ -78,11 +78,15 @@ public class TestHoodieBucketIndex extends HoodieClientTestHarness {
props.setProperty(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key(), "_row_key");
assertThrows(HoodieIndexException.class, () -> {
HoodieIndexConfig.newBuilder().fromProperties(props)
.withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("8").build();
.withIndexType(HoodieIndex.IndexType.BUCKET)
.withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE)
.withBucketNum("8").build();
});
props.setProperty(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key(), "uuid");
HoodieIndexConfig.newBuilder().fromProperties(props)
.withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("8").build();
.withIndexType(HoodieIndex.IndexType.BUCKET)
.withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE)
.withBucketNum("8").build();
}
@Test
@@ -110,7 +114,7 @@ public class TestHoodieBucketIndex extends HoodieClientTestHarness {
HoodieWriteConfig config = makeConfig();
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
HoodieBucketIndex bucketIndex = new HoodieBucketIndex(config);
HoodieSimpleBucketIndex bucketIndex = new HoodieSimpleBucketIndex(config);
HoodieData<HoodieRecord<HoodieAvroRecord>> taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table);
assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(r -> r.isCurrentLocationKnown()));
@@ -133,6 +137,7 @@ public class TestHoodieBucketIndex extends HoodieClientTestHarness {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString())
.withIndexConfig(HoodieIndexConfig.newBuilder().fromProperties(props)
.withIndexType(HoodieIndex.IndexType.BUCKET)
.withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE)
.withIndexKeyField("_row_key")
.withBucketNum(String.valueOf(NUM_BUCKET)).build()).build();
}

View File

@@ -148,7 +148,10 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
props.putAll(indexConfig.build().getProps());
if (indexType.equals(HoodieIndex.IndexType.BUCKET)) {
props.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
indexConfig.fromProperties(props).withIndexKeyField("_row_key").withBucketNum("1");
indexConfig.fromProperties(props)
.withIndexKeyField("_row_key")
.withBucketNum("1")
.withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE);
props.putAll(indexConfig.build().getProps());
props.putAll(HoodieLayoutConfig.newBuilder().fromProperties(props)
.withLayoutType(HoodieStorageLayout.LayoutType.BUCKET.name())