1
0

[HUDI-1951] Add bucket hash index, compatible with the hive bucket (#3173)

* [HUDI-2154] Add index key field to HoodieKey

* [HUDI-2157] Add the bucket index and its read/write implemention of Spark engine.
* revert HUDI-2154 add index key field to HoodieKey
* fix all comments and introduce a new tricky way to get index key at runtime
support double insert for bucket index
* revert spark read optimizer based on bucket index
* add the storage layout
* index tag, hash function and add ut
* fix ut
* address partial comments
* Code review feedback
* add layout config and docs
* fix ut
* rename hoodie.layout and rebase master

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Shawy Geng
2021-12-31 04:38:26 +08:00
committed by GitHub
parent 0f0088fe4b
commit a4e622ac61
46 changed files with 1335 additions and 47 deletions

View File

@@ -28,6 +28,7 @@ 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.hbase.SparkHoodieHBaseIndex;
import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex;
import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex;
@@ -55,6 +56,8 @@ 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:

View File

@@ -216,7 +216,10 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
}
protected Partitioner getPartitioner(WorkloadProfile profile) {
if (WriteOperationType.isChangingRecords(operationType)) {
Option<String> layoutPartitionerClass = table.getStorageLayout().layoutPartitionerClass();
if (layoutPartitionerClass.isPresent()) {
return getLayoutPartitioner(profile, layoutPartitionerClass.get());
} else if (WriteOperationType.isChangingRecords(operationType)) {
return getUpsertPartitioner(profile);
} else {
return getInsertPartitioner(profile);
@@ -305,7 +308,7 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
@SuppressWarnings("unchecked")
protected Iterator<List<WriteStatus>> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr,
Partitioner partitioner) {
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
SparkHoodiePartitioner upsertPartitioner = (SparkHoodiePartitioner) partitioner;
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
BucketType btype = binfo.bucketType;
try {
@@ -394,6 +397,12 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
return getUpsertPartitioner(profile);
}
public Partitioner getLayoutPartitioner(WorkloadProfile profile, String layoutPartitionerClass) {
return (Partitioner) ReflectionUtils.loadClass(layoutPartitionerClass,
new Class[] { WorkloadProfile.class, HoodieEngineContext.class, HoodieTable.class, HoodieWriteConfig.class },
profile, context, table, config);
}
@Override
protected void runPrecommitValidators(HoodieWriteMetadata<JavaRDD<WriteStatus>> writeMetadata) {
SparkValidatorUtils.runValidators(config, writeMetadata, context, table, instantTime);

View File

@@ -0,0 +1,136 @@
/*
* 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.table.action.commit;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.hudi.index.bucket.BucketIdentifier;
import scala.Tuple2;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.index.bucket.HoodieBucketIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.WorkloadStat;
/**
* Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition).
*/
public class SparkBucketIndexPartitioner<T extends HoodieRecordPayload<T>> extends
SparkHoodiePartitioner<T> {
private final int numBuckets;
private final String indexKeyField;
private final int totalPartitionPaths;
private final List<String> partitionPaths;
/**
* Helps get the RDD partition id, partition id is partition offset + bucket id.
* The partition offset is a multiple of the bucket num.
*/
private final Map<String, Integer> partitionPathOffset;
/**
* Partition path and file groups in it pair. Decide the file group an incoming update should go to.
*/
private Map<String, Set<String>> updatePartitionPathFileIds;
public SparkBucketIndexPartitioner(WorkloadProfile profile,
HoodieEngineContext context,
HoodieTable table,
HoodieWriteConfig config) {
super(profile, table);
if (!(table.getIndex() instanceof HoodieBucketIndex)) {
throw new HoodieException(
" Bucket index partitioner should only be used by BucketIndex other than "
+ table.getIndex().getClass().getSimpleName());
}
this.numBuckets = ((HoodieBucketIndex<T>) table.getIndex()).getNumBuckets();
this.indexKeyField = config.getBucketIndexHashField();
this.totalPartitionPaths = profile.getPartitionPaths().size();
partitionPaths = new ArrayList<>(profile.getPartitionPaths());
partitionPathOffset = new HashMap<>();
int i = 0;
for (Object partitionPath : profile.getPartitionPaths()) {
partitionPathOffset.put(partitionPath.toString(), i);
i += numBuckets;
}
assignUpdates(profile);
}
private void assignUpdates(WorkloadProfile profile) {
updatePartitionPathFileIds = new HashMap<>();
// each update location gets a partition
Set<Entry<String, WorkloadStat>> partitionStatEntries = profile.getPartitionPathStatMap()
.entrySet();
for (Entry<String, WorkloadStat> partitionStat : partitionStatEntries) {
if (!updatePartitionPathFileIds.containsKey(partitionStat.getKey())) {
updatePartitionPathFileIds.put(partitionStat.getKey(), new HashSet<>());
}
for (Entry<String, Pair<String, Long>> updateLocEntry :
partitionStat.getValue().getUpdateLocationToCount().entrySet()) {
updatePartitionPathFileIds.get(partitionStat.getKey()).add(updateLocEntry.getKey());
}
}
}
@Override
public BucketInfo getBucketInfo(int bucketNumber) {
String partitionPath = partitionPaths.get(bucketNumber / numBuckets);
String bucketId = BucketIdentifier.bucketIdStr(bucketNumber % numBuckets);
Option<String> fileIdOption = Option.fromJavaOptional(updatePartitionPathFileIds
.getOrDefault(partitionPath, Collections.emptySet()).stream()
.filter(e -> e.startsWith(bucketId))
.findFirst());
if (fileIdOption.isPresent()) {
return new BucketInfo(BucketType.UPDATE, fileIdOption.get(), partitionPath);
} else {
return new BucketInfo(BucketType.INSERT, BucketIdentifier.newBucketFileIdPrefix(bucketId), partitionPath);
}
}
@Override
public int numPartitions() {
return totalPartitionPaths * numBuckets;
}
@Override
public int getPartition(Object key) {
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey, Option<HoodieRecordLocation>>) key;
String partitionPath = keyLocation._1.getPartitionPath();
Option<HoodieRecordLocation> location = keyLocation._2;
int bucketId = location.isPresent()
? BucketIdentifier.bucketIdFromFileId(location.get().getFileId())
: BucketIdentifier.getBucketId(keyLocation._1, indexKeyField, numBuckets);
return partitionPathOffset.get(partitionPath) + bucketId;
}
}

View File

@@ -0,0 +1,50 @@
/*
* 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.table.action.commit;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.spark.Partitioner;
/**
* Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition).
*/
public abstract class SparkHoodiePartitioner<T extends HoodieRecordPayload<T>> extends Partitioner
implements org.apache.hudi.table.action.commit.Partitioner {
/**
* Stat for the current workload. Helps in determining inserts, upserts etc.
*/
protected WorkloadProfile profile;
protected final HoodieTable table;
public SparkHoodiePartitioner(WorkloadProfile profile, HoodieTable table) {
this.profile = profile;
this.table = table;
}
@Override
public int getNumPartitions() {
return numPartitions();
}
public abstract BucketInfo getBucketInfo(int bucketNumber);
}

View File

@@ -44,6 +44,6 @@ public class SparkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>>
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false);
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType);
}
}

View File

@@ -58,12 +58,14 @@ public class SparkInsertOverwriteCommitActionExecutor<T extends HoodieRecordPayl
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false);
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType);
}
@Override
protected Partitioner getPartitioner(WorkloadProfile profile) {
return new SparkInsertOverwritePartitioner(profile, context, table, config);
return table.getStorageLayout().layoutPartitionerClass()
.map(c -> getLayoutPartitioner(profile, c))
.orElse(new SparkInsertOverwritePartitioner(profile, context, table, config));
}
@Override

View File

@@ -44,6 +44,6 @@ public class SparkUpsertCommitActionExecutor<T extends HoodieRecordPayload<T>>
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true);
config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, operationType);
}
}

View File

@@ -37,7 +37,6 @@ import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.WorkloadStat;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.PairFunction;
@@ -57,7 +56,7 @@ import scala.Tuple2;
/**
* Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition).
*/
public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends Partitioner {
public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends SparkHoodiePartitioner<T> {
private static final Logger LOG = LogManager.getLogger(UpsertPartitioner.class);
@@ -69,10 +68,6 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends Partiti
* Total number of RDD partitions, is determined by total buckets we want to pack the incoming workload into.
*/
private int totalBuckets = 0;
/**
* Stat for the current workload. Helps in determining inserts, upserts etc.
*/
private WorkloadProfile profile;
/**
* Helps decide which bucket an incoming update should go to.
*/
@@ -86,17 +81,14 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends Partiti
*/
private HashMap<Integer, BucketInfo> bucketInfoMap;
protected final HoodieTable table;
protected final HoodieWriteConfig config;
public UpsertPartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table,
HoodieWriteConfig config) {
super(profile, table);
updateLocationToBucket = new HashMap<>();
partitionPathToInsertBucketInfos = new HashMap<>();
bucketInfoMap = new HashMap<>();
this.profile = profile;
this.table = table;
this.config = config;
assignUpdates(profile);
assignInserts(profile, context);

View File

@@ -74,8 +74,8 @@ public abstract class AbstractSparkDeltaCommitActionExecutor<T extends HoodieRec
public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
Iterator<HoodieRecord<T>> recordItr) throws IOException {
LOG.info("Merging updates for commit " + instantTime + " for file " + fileId);
if (!table.getIndex().canIndexLogFiles() && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
if (!table.getIndex().canIndexLogFiles() && mergeOnReadUpsertPartitioner != null
&& mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
LOG.info("Small file corrections for updates for commit " + instantTime + " for file " + fileId);
return super.handleUpdate(partitionPath, fileId, recordItr);
} else {

View File

@@ -45,6 +45,6 @@ public class SparkInsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, false);
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, operationType);
}
}

View File

@@ -44,6 +44,6 @@ public class SparkUpsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<
@Override
public HoodieWriteMetadata execute() {
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, true);
config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, operationType);
}
}