Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ public LazyIterableIterator(Iterator<I> in) {
/**
* Called once, before any elements are processed.
*/
protected abstract void start();
protected void start() {}

/**
* Block computation to be overwritten by sub classes.
Expand All @@ -55,7 +55,7 @@ public LazyIterableIterator(Iterator<I> in) {
/**
* Called once, after all elements are processed.
*/
protected abstract void end();
protected void end() {}

//////////////////
// iterable implementation
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -216,19 +216,40 @@ public class HoodieIndexConfig extends HoodieConfig {
/**
* ***** Bucket Index Configs *****
* Bucket Index is targeted to locate the record fast by hash in big data scenarios.
* The current implementation is a basic version, so there are some constraints:
* 1. Unsupported operation: bulk insert, cluster and so on.
* 2. Bucket num change requires rewriting the partition.
* 3. Predict the table size and future data growth well to set a reasonable bucket num.
* 4. A bucket size is recommended less than 3GB and avoid bing too small.
* more details and progress see [HUDI-3039].
*/
// Bucket num equals file groups num in each partition.
// Bucket num can be set according to partition size and file group size.
* A bucket size is recommended less than 3GB to avoid being too small.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

with the consistent_hashing, do we still have the constraint?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For consistent hashing, users can config the bucket split/merge threshold to control the bucket size. 3GB may not be a hard constraint. it is a recommendation.

* For more details and progress, see [HUDI-3039].
*/

/**
* Bucket Index Engine Type: implementation of bucket index
*
* SIMPLE:
* 0. Check `HoodieSimpleBucketLayout` for its supported operations.
* 1. Bucket num is fixed and requires rewriting the partition if we want to change it.
*
* CONSISTENT_HASHING:
* 0. Check `HoodieConsistentBucketLayout` for its supported operations.
* 1. Bucket num will auto-adjust by running clustering (still in progress)
*/
public static final ConfigProperty<String> BUCKET_INDEX_ENGINE_TYPE = ConfigProperty
.key("hoodie.index.bucket.engine")
.defaultValue("SIMPLE")
.sinceVersion("0.11.0")
.withDocumentation("Type of bucket index engine to use. Default is SIMPLE bucket index, with fixed number of bucket."
+ "Possible options are [SIMPLE | CONSISTENT_HASHING]."
+ "Consistent hashing supports dynamic resizing of the number of bucket, solving potential data skew and file size "
+ "issues of the SIMPLE hashing engine.");

/**
* Bucket num equals file groups num in each partition.
* Bucket num can be set according to partition size and file group size.
*
* In dynamic bucket index cases (e.g., using CONSISTENT_HASHING), this config of number of bucket serves as a initial bucket size
*/
public static final ConfigProperty<Integer> BUCKET_INDEX_NUM_BUCKETS = ConfigProperty
.key("hoodie.bucket.index.num.buckets")
.defaultValue(256)
.withDocumentation("Only applies if index type is BUCKET_INDEX. Determine the number of buckets in the hudi table, "
.withDocumentation("Only applies if index type is BUCKET. Determine the number of buckets in the hudi table, "
+ "and each partition is divided to N buckets.");

public static final ConfigProperty<String> BUCKET_INDEX_HASH_FIELD = ConfigProperty
Expand Down Expand Up @@ -463,6 +484,11 @@ public Builder withIndexType(HoodieIndex.IndexType indexType) {
return this;
}

public Builder withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType bucketType) {
hoodieIndexConfig.setValue(BUCKET_INDEX_ENGINE_TYPE, bucketType.name());
return this;
}

public Builder withIndexClass(String indexClass) {
hoodieIndexConfig.setValue(INDEX_CLASS_NAME, indexClass);
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1428,6 +1428,10 @@ public String getIndexClass() {
return getString(HoodieIndexConfig.INDEX_CLASS_NAME);
}

public HoodieIndex.BucketIndexEngineType getBucketIndexEngineType() {
return HoodieIndex.BucketIndexEngineType.valueOf(getString(HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE));
}

public int getBloomFilterNumEntries() {
return getInt(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES_VALUE);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ public abstract HoodieData<WriteStatus> updateLocation(
public abstract boolean isImplicitWithStorage();

/**
* If the `getCustomizedPartitioner` returns a partitioner, it has to be true.
* To indicate if a operation type requires location tagging before writing
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public boolean requiresTagging(WriteOperationType operationType) {
Expand All @@ -143,4 +143,8 @@ public void close() {
public enum IndexType {
HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE, BUCKET, FLINK_STATE
}

public enum BucketIndexEngineType {
SIMPLE, CONSISTENT_HASHING
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,43 +22,58 @@
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;

import java.io.Serializable;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
import java.util.stream.Collectors;

public class BucketIdentifier {
// compatible with the spark bucket name
public class BucketIdentifier implements Serializable {
// Compatible with the spark bucket name
private static final Pattern BUCKET_NAME = Pattern.compile(".*_(\\d+)(?:\\..*)?$");

public static int getBucketId(HoodieRecord record, String indexKeyFields, int numBuckets) {
return getBucketId(record.getKey(), indexKeyFields, numBuckets);
}

public static int getBucketId(HoodieKey hoodieKey, String indexKeyFields, int numBuckets) {
return getBucketId(hoodieKey.getRecordKey(), indexKeyFields, numBuckets);
return (getHashKeys(hoodieKey, indexKeyFields).hashCode() & Integer.MAX_VALUE) % numBuckets;
}

public static int getBucketId(HoodieKey hoodieKey, List<String> indexKeyFields, int numBuckets) {
return (getHashKeys(hoodieKey.getRecordKey(), indexKeyFields).hashCode() & Integer.MAX_VALUE) % numBuckets;
}

public static int getBucketId(String recordKey, String indexKeyFields, int numBuckets) {
List<String> hashKeyFields;
if (!recordKey.contains(":")) {
hashKeyFields = Collections.singletonList(recordKey);
} else {
Map<String, String> recordKeyPairs = Arrays.stream(recordKey.split(","))
.map(p -> p.split(":"))
.collect(Collectors.toMap(p -> p[0], p -> p[1]));
hashKeyFields = Arrays.stream(indexKeyFields.split(","))
.map(f -> recordKeyPairs.get(f))
.collect(Collectors.toList());
}
return (hashKeyFields.hashCode() & Integer.MAX_VALUE) % numBuckets;
return getBucketId(getHashKeys(recordKey, indexKeyFields), numBuckets);
}

// only for test
public static int getBucketId(List<String> hashKeyFields, int numBuckets) {
return hashKeyFields.hashCode() % numBuckets;
return (hashKeyFields.hashCode() & Integer.MAX_VALUE) % numBuckets;
}

public static List<String> getHashKeys(HoodieKey hoodieKey, String indexKeyFields) {
return getHashKeys(hoodieKey.getRecordKey(), indexKeyFields);
}

protected static List<String> getHashKeys(String recordKey, String indexKeyFields) {
return !recordKey.contains(":") ? Collections.singletonList(recordKey) :
getHashKeysUsingIndexFields(recordKey, Arrays.asList(indexKeyFields.split(",")));
}

protected static List<String> getHashKeys(String recordKey, List<String> indexKeyFields) {
return !recordKey.contains(":") ? Collections.singletonList(recordKey) :
getHashKeysUsingIndexFields(recordKey, indexKeyFields);
}

private static List<String> getHashKeysUsingIndexFields(String recordKey, List<String> indexKeyFields) {
Map<String, String> recordKeyPairs = Arrays.stream(recordKey.split(","))
.map(p -> p.split(":"))
.collect(Collectors.toMap(p -> p[0], p -> p[1]));
return indexKeyFields.stream()
.map(f -> recordKeyPairs.get(f)).collect(Collectors.toList());
}

public static String partitionBucketIdStr(String partition, int bucketId) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.util.Option;

import java.io.Serializable;

public interface BucketIndexLocationMapper extends Serializable {

/**
* Get record location given hoodie key and partition path
*/
Option<HoodieRecordLocation> getRecordLocation(HoodieKey key, String partitionPath);

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hey, the HoodieKey already contains partition path field. Is there any possibility that the partitionPath does not equal to the partition path of the HoodieKey ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah you are right. Will simplify the interface in the follow-up PR.

ps. the partition path here is used to get hashing metadata for the given partition.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hey danny, I am also planning to introduce this consistent hashing index feature into hudi flink engine, in order to support dynamic bucket number. Does flink engine have any roadmap for 'dynamic bucket num' currently?

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
* 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.common.fs.FSUtils;
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.util.hash.HashID;

import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.TreeMap;

public class ConsistentBucketIdentifier extends BucketIdentifier {

/**
* Hashing metadata of a partition
*/
private final HoodieConsistentHashingMetadata metadata;
/**
* In-memory structure to speed up ring mapping (hashing value -> hashing node)
*/
private final TreeMap<Integer, ConsistentHashingNode> ring;
/**
* Mapping from fileId -> hashing node
*/
private final Map<String, ConsistentHashingNode> fileIdToBucket;

public ConsistentBucketIdentifier(HoodieConsistentHashingMetadata metadata) {
this.metadata = metadata;
this.fileIdToBucket = new HashMap<>();
this.ring = new TreeMap<>();
initialize();
}

public Collection<ConsistentHashingNode> getNodes() {
return ring.values();
}

public HoodieConsistentHashingMetadata getMetadata() {
return metadata;
}

public int getNumBuckets() {
return ring.size();
}

/**
* Get bucket of the given file group
*
* @param fileId the file group id. NOTE: not filePfx (i.e., uuid)
*/
public ConsistentHashingNode getBucketByFileId(String fileId) {
return fileIdToBucket.get(fileId);
}

public ConsistentHashingNode getBucket(HoodieKey hoodieKey, List<String> indexKeyFields) {
return getBucket(getHashKeys(hoodieKey.getRecordKey(), indexKeyFields));
}

protected ConsistentHashingNode getBucket(List<String> hashKeys) {
int hashValue = HashID.getXXHash32(String.join("", hashKeys), 0);
return getBucket(hashValue & HoodieConsistentHashingMetadata.HASH_VALUE_MASK);
}

protected ConsistentHashingNode getBucket(int hashValue) {
SortedMap<Integer, ConsistentHashingNode> tailMap = ring.tailMap(hashValue);
return tailMap.isEmpty() ? ring.firstEntry().getValue() : tailMap.get(tailMap.firstKey());
}

/**
* Initialize necessary data structure to facilitate bucket identifying.
* Specifically, we construct:
* - An in-memory tree (ring) to speed up range mapping searching.
* - A hash table (fileIdToBucket) to allow lookup of bucket using fileId.
*/
private void initialize() {
for (ConsistentHashingNode p : metadata.getNodes()) {
ring.put(p.getValue(), p);
// One bucket has only one file group, so append 0 directly
fileIdToBucket.put(FSUtils.createNewFileId(p.getFileIdPrefix(), 0), p);
}
}
}
Loading