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

@@ -24,15 +24,20 @@ import org.apache.hudi.common.config.ConfigGroups;
import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.common.config.HoodieConfig;
import org.apache.hudi.common.engine.EngineType;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Arrays;
import java.util.Properties;
import java.util.stream.Collectors;
import static org.apache.hudi.config.HoodieHBaseIndexConfig.GET_BATCH_SIZE;
import static org.apache.hudi.config.HoodieHBaseIndexConfig.PUT_BATCH_SIZE;
@@ -54,7 +59,7 @@ public class HoodieIndexConfig extends HoodieConfig {
.key("hoodie.index.type")
.noDefaultValue()
.withDocumentation("Type of index to use. Default is Bloom filter. "
+ "Possible options are [BLOOM | GLOBAL_BLOOM |SIMPLE | GLOBAL_SIMPLE | INMEMORY | HBASE]. "
+ "Possible options are [BLOOM | GLOBAL_BLOOM |SIMPLE | GLOBAL_SIMPLE | INMEMORY | HBASE | BUCKET]. "
+ "Bloom filters removes the dependency on a external system "
+ "and is stored in the footer of the Parquet Data Files");
@@ -200,6 +205,30 @@ public class HoodieIndexConfig extends HoodieConfig {
.defaultValue("true")
.withDocumentation("Similar to " + BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE + ", but for simple index.");
/**
* ***** 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.
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, "
+ "and each partition is divided to N buckets.");
public static final ConfigProperty<String> BUCKET_INDEX_HASH_FIELD = ConfigProperty
.key("hoodie.bucket.index.hash.field")
.noDefaultValue()
.withDocumentation("Index key. It is used to index the record and find its file group. "
+ "If not set, use record key field as default");
/**
* Deprecated configs. These are now part of {@link HoodieHBaseIndexConfig}.
*/
@@ -516,12 +545,23 @@ public class HoodieIndexConfig extends HoodieConfig {
return this;
}
public Builder withBucketNum(String bucketNum) {
hoodieIndexConfig.setValue(BUCKET_INDEX_NUM_BUCKETS, bucketNum);
return this;
}
public Builder withIndexKeyField(String keyField) {
hoodieIndexConfig.setValue(BUCKET_INDEX_HASH_FIELD, keyField);
return this;
}
public HoodieIndexConfig build() {
hoodieIndexConfig.setDefaultValue(INDEX_TYPE, getDefaultIndexType(engineType));
hoodieIndexConfig.setDefaults(HoodieIndexConfig.class.getName());
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
HoodieIndex.IndexType.valueOf(hoodieIndexConfig.getString(INDEX_TYPE));
validateBucketIndexConfig();
return hoodieIndexConfig;
}
@@ -540,5 +580,27 @@ public class HoodieIndexConfig extends HoodieConfig {
public EngineType getEngineType() {
return engineType;
}
private void validateBucketIndexConfig() {
if (hoodieIndexConfig.getString(INDEX_TYPE).equalsIgnoreCase(HoodieIndex.IndexType.BUCKET.toString())) {
// check the bucket index hash field
if (StringUtils.isNullOrEmpty(hoodieIndexConfig.getString(BUCKET_INDEX_HASH_FIELD))) {
hoodieIndexConfig.setValue(BUCKET_INDEX_HASH_FIELD,
hoodieIndexConfig.getStringOrDefault(KeyGeneratorOptions.RECORDKEY_FIELD_NAME));
} else {
boolean valid = Arrays
.stream(hoodieIndexConfig.getStringOrDefault(KeyGeneratorOptions.RECORDKEY_FIELD_NAME).split(","))
.collect(Collectors.toSet())
.containsAll(Arrays.asList(hoodieIndexConfig.getString(BUCKET_INDEX_HASH_FIELD).split(",")));
if (!valid) {
throw new HoodieIndexException("Bucket index key (if configured) must be subset of record key.");
}
}
// check the bucket num
if (hoodieIndexConfig.getIntOrDefault(BUCKET_INDEX_NUM_BUCKETS) <= 0) {
throw new HoodieIndexException("When using bucket index, hoodie.bucket.index.num.buckets cannot be negative.");
}
}
}
}
}

View File

@@ -0,0 +1,89 @@
/*
* 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.config;
import org.apache.hudi.common.config.ConfigClassProperty;
import org.apache.hudi.common.config.ConfigGroups;
import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.common.config.HoodieConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.storage.HoodieStorageLayout;
import javax.annotation.concurrent.Immutable;
import java.util.Properties;
/**
* Storage layout related config.
*/
@Immutable
@ConfigClassProperty(name = "Layout Configs",
groupName = ConfigGroups.Names.WRITE_CLIENT,
description = "Configurations that control storage layout and data distribution, "
+ "which defines how the files are organized within a table.")
public class HoodieLayoutConfig extends HoodieConfig {
public static final ConfigProperty<String> LAYOUT_TYPE = ConfigProperty
.key("hoodie.storage.layout.type")
.defaultValue("DEFAULT")
.withDocumentation("Type of storage layout. Possible options are [DEFAULT | BUCKET]");
public static final ConfigProperty<String> LAYOUT_PARTITIONER_CLASS_NAME = ConfigProperty
.key("hoodie.storage.layout.partitioner.class")
.noDefaultValue()
.withDocumentation("Partitioner class, it is used to distribute data in a specific way.");
private HoodieLayoutConfig() {
super();
}
public static HoodieLayoutConfig.Builder newBuilder() {
return new Builder();
}
public static class Builder {
public HoodieLayoutConfig layoutConfig = new HoodieLayoutConfig();
public Builder fromProperties(Properties props) {
this.layoutConfig.getProps().putAll(props);
return this;
}
public Builder withLayoutType(String type) {
layoutConfig.setValue(LAYOUT_TYPE, type);
return this;
}
public Builder withLayoutPartitioner(String partitionerClass) {
layoutConfig.setValue(LAYOUT_PARTITIONER_CLASS_NAME, partitionerClass);
return this;
}
public HoodieLayoutConfig build() {
setDefault();
return layoutConfig;
}
private void setDefault() {
if (layoutConfig.getString(HoodieIndexConfig.INDEX_TYPE.key()).equals(HoodieIndex.IndexType.BUCKET.name())) {
layoutConfig.setDefaultValue(LAYOUT_TYPE, HoodieStorageLayout.LayoutType.BUCKET.name());
}
layoutConfig.setDefaultValue(LAYOUT_TYPE, LAYOUT_TYPE.defaultValue());
}
}
}

View File

@@ -57,6 +57,7 @@ import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hudi.table.storage.HoodieStorageLayout;
import org.apache.orc.CompressionKind;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
@@ -1426,6 +1427,14 @@ public class HoodieWriteConfig extends HoodieConfig {
return getBoolean(HoodieIndexConfig.SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE);
}
public int getBucketIndexNumBuckets() {
return getIntOrDefault(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS);
}
public String getBucketIndexHashField() {
return getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD);
}
/**
* storage properties.
*/
@@ -1834,6 +1843,13 @@ public class HoodieWriteConfig extends HoodieConfig {
return getString(FILEID_PREFIX_PROVIDER_CLASS);
}
/**
* Layout configs.
*/
public HoodieStorageLayout.LayoutType getLayoutType() {
return HoodieStorageLayout.LayoutType.valueOf(getString(HoodieLayoutConfig.LAYOUT_TYPE));
}
public static class Builder {
protected final HoodieWriteConfig writeConfig = new HoodieWriteConfig();
@@ -1855,6 +1871,7 @@ public class HoodieWriteConfig extends HoodieConfig {
private boolean isPreCommitValidationConfigSet = false;
private boolean isMetricsJmxConfigSet = false;
private boolean isMetricsGraphiteConfigSet = false;
private boolean isLayoutConfigSet = false;
public Builder withEngineType(EngineType engineType) {
this.engineType = engineType;
@@ -2085,6 +2102,12 @@ public class HoodieWriteConfig extends HoodieConfig {
return this;
}
public Builder withLayoutConfig(HoodieLayoutConfig layoutConfig) {
writeConfig.getProps().putAll(layoutConfig.getProps());
isLayoutConfigSet = true;
return this;
}
public Builder withFinalizeWriteParallelism(int parallelism) {
writeConfig.setValue(FINALIZE_WRITE_PARALLELISM_VALUE, String.valueOf(parallelism));
return this;
@@ -2225,6 +2248,8 @@ public class HoodieWriteConfig extends HoodieConfig {
HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
writeConfig.setDefaultOnCondition(!isPreCommitValidationConfigSet,
HoodiePreCommitValidatorConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
writeConfig.setDefaultOnCondition(!isLayoutConfigSet,
HoodieLayoutConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION_NUM, String.valueOf(TimelineLayoutVersion.CURR_VERSION));
}

View File

@@ -28,6 +28,7 @@ import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.exception.HoodieNotSupportedException;
@@ -122,6 +123,20 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> implem
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
public abstract boolean isImplicitWithStorage();
/**
* If the `getCustomizedPartitioner` returns a partitioner, it has to be true.
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public boolean requiresTagging(WriteOperationType operationType) {
switch (operationType) {
case DELETE:
case UPSERT:
return true;
default:
return false;
}
}
/**
* Each index type should implement it's own logic to release any resources acquired during the process.
*/
@@ -129,6 +144,6 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> implem
}
public enum IndexType {
HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE
HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE, BUCKET
}
}

View File

@@ -0,0 +1,74 @@
/*
* 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.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import java.util.Arrays;
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
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) {
List<String> hashKeyFields;
if (!hoodieKey.getRecordKey().contains(":")) {
hashKeyFields = Arrays.asList(hoodieKey.getRecordKey());
} else {
Map<String, String> recordKeyPairs = Arrays.stream(hoodieKey.getRecordKey().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;
}
// only for test
public static int getBucketId(List<String> hashKeyFields, int numBuckets) {
return hashKeyFields.hashCode() % numBuckets;
}
public static int bucketIdFromFileId(String fileId) {
return Integer.parseInt(fileId.substring(0, 8));
}
public static String bucketIdStr(int n) {
return String.format("%08d", n);
}
public static String newBucketFileIdPrefix(String bucketId) {
return FSUtils.createNewFileIdPfx().replaceFirst(".{8}", bucketId);
}
public static boolean isBucketFileName(String name) {
return BUCKET_NAME.matcher(name).matches();
}
}

View File

@@ -0,0 +1,166 @@
/*
* 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.client.utils.LazyIterableIterator;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
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.HoodieIOException;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.index.HoodieIndexUtils;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.HashMap;
import java.util.Map;
/**
* Hash indexing mechanism.
* @param <T>
*/
public class HoodieBucketIndex<T extends HoodieRecordPayload<T>>
extends HoodieIndex<T, Object, Object, Object> {
private static final Logger LOG = LogManager.getLogger(HoodieBucketIndex.class);
private final int numBuckets;
public HoodieBucketIndex(HoodieWriteConfig config) {
super(config);
numBuckets = config.getBucketIndexNumBuckets();
LOG.info("use bucket index, numBuckets=" + numBuckets);
}
@Override
public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> writeStatuses,
HoodieEngineContext context,
HoodieTable hoodieTable)
throws HoodieIndexException {
return writeStatuses;
}
@Override
public HoodieData<HoodieRecord<T>> tagLocation(HoodieData<HoodieRecord<T>> records,
HoodieEngineContext context,
HoodieTable hoodieTable)
throws HoodieIndexException {
HoodieData<HoodieRecord<T>> taggedRecords = records.mapPartitions(recordIter -> {
// partitionPath -> bucketId -> fileInfo
Map<String, Map<Integer, Pair<String, String>>> partitionPathFileIDList = new HashMap<>();
return new LazyIterableIterator<HoodieRecord<T>, HoodieRecord<T>>(recordIter) {
@Override
protected void start() {
}
@Override
protected HoodieRecord<T> computeNext() {
HoodieRecord record = recordIter.next();
int bucketId = BucketIdentifier.getBucketId(record, config.getBucketIndexHashField(), numBuckets);
String partitionPath = record.getPartitionPath();
if (!partitionPathFileIDList.containsKey(partitionPath)) {
partitionPathFileIDList.put(partitionPath, loadPartitionBucketIdFileIdMapping(hoodieTable, partitionPath));
}
if (partitionPathFileIDList.get(partitionPath).containsKey(bucketId)) {
Pair<String, String> fileInfo = partitionPathFileIDList.get(partitionPath).get(bucketId);
return HoodieIndexUtils.getTaggedRecord(record, Option.of(
new HoodieRecordLocation(fileInfo.getRight(), fileInfo.getLeft())
));
}
return record;
}
@Override
protected void end() {
}
};
}, true);
return taggedRecords;
}
private Map<Integer, Pair<String, String>> loadPartitionBucketIdFileIdMapping(
HoodieTable hoodieTable,
String partition) {
// bucketId -> fileIds
Map<Integer, Pair<String, String>> fileIDList = new HashMap<>();
HoodieIndexUtils
.getLatestBaseFilesForPartition(partition, hoodieTable)
.forEach(file -> {
String fileId = file.getFileId();
String commitTime = file.getCommitTime();
int bucketId = BucketIdentifier.bucketIdFromFileId(fileId);
if (!fileIDList.containsKey(bucketId)) {
fileIDList.put(bucketId, Pair.of(fileId, commitTime));
} else {
// check if bucket data is valid
throw new HoodieIOException("Find multiple files at partition path="
+ partition + " belongs to the same bucket id = " + bucketId);
}
});
return fileIDList;
}
@Override
public boolean rollbackCommit(String instantTime) {
return true;
}
@Override
public boolean isGlobal() {
return false;
}
@Override
public boolean canIndexLogFiles() {
return false;
}
@Override
public boolean isImplicitWithStorage() {
return true;
}
@Override
public boolean requiresTagging(WriteOperationType operationType) {
switch (operationType) {
case INSERT:
case INSERT_OVERWRITE:
case UPSERT:
return true;
default:
return false;
}
}
public int getNumBuckets() {
return numBuckets;
}
}

View File

@@ -119,7 +119,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
@Override
public boolean canWrite(HoodieRecord record) {
return fileWriter.canWrite() && record.getPartitionPath().equals(writeStatus.getPartitionPath());
return (fileWriter.canWrite() && record.getPartitionPath().equals(writeStatus.getPartitionPath()))
|| layoutControlsNumFiles();
}
/**

View File

@@ -195,6 +195,10 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
return false;
}
boolean layoutControlsNumFiles() {
return hoodieTable.getStorageLayout().determinesNumFileGroups();
}
/**
* Perform the actual writing of the given record into the backing file.
*/

View File

@@ -74,6 +74,8 @@ import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.table.storage.HoodieLayoutFactory;
import org.apache.hudi.table.storage.HoodieStorageLayout;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -106,6 +108,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
private SerializableConfiguration hadoopConfiguration;
protected final TaskContextSupplier taskContextSupplier;
private final HoodieTableMetadata metadata;
private final HoodieStorageLayout storageLayout;
private transient FileSystemViewManager viewManager;
protected final transient HoodieEngineContext context;
@@ -123,11 +126,16 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
this.viewManager = FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadata);
this.metaClient = metaClient;
this.index = getIndex(config, context);
this.storageLayout = getStorageLayout(config);
this.taskContextSupplier = context.getTaskContextSupplier();
}
protected abstract HoodieIndex<T, ?, ?, ?> getIndex(HoodieWriteConfig config, HoodieEngineContext context);
protected HoodieStorageLayout getStorageLayout(HoodieWriteConfig config) {
return HoodieLayoutFactory.createLayout(config);
}
private synchronized FileSystemViewManager getViewManager() {
if (null == viewManager) {
viewManager = FileSystemViewManager.createViewManager(getContext(), config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadata);
@@ -365,6 +373,10 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
return index;
}
public HoodieStorageLayout getStorageLayout() {
return storageLayout;
}
/**
* Schedule compaction for the instant time.
*

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.table.action.commit;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
@@ -38,7 +39,7 @@ public abstract class AbstractWriteHelper<T extends HoodieRecordPayload, I, K, O
boolean shouldCombine,
int shuffleParallelism,
BaseCommitActionExecutor<T, I, K, O, R> executor,
boolean performTagging) {
WriteOperationType operationType) {
try {
// De-dupe/merge if needed
I dedupedRecords =
@@ -46,7 +47,7 @@ public abstract class AbstractWriteHelper<T extends HoodieRecordPayload, I, K, O
Instant lookupBegin = Instant.now();
I taggedRecords = dedupedRecords;
if (performTagging) {
if (table.getIndex().requiresTagging(operationType)) {
// perform index loop up to get existing location of records
context.setJobStatus(this.getClass().getSimpleName(), "Tagging");
taggedRecords = tag(dedupedRecords, context, table);

View File

@@ -76,6 +76,10 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
// TODO : Remove this once we refactor and move out autoCommit method from here, since the TxnManager is held in {@link AbstractHoodieWriteClient}.
this.txnManager = new TransactionManager(config, table.getMetaClient().getFs());
this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient());
if (table.getStorageLayout().doesNotSupport(operationType)) {
throw new UnsupportedOperationException("Executor " + this.getClass().getSimpleName()
+ " is not compatible with table layout " + table.getStorageLayout().getClass().getSimpleName());
}
}
public abstract HoodieWriteMetadata<O> execute(I inputRecords);

View File

@@ -0,0 +1,67 @@
/*
* 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.storage;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieLayoutConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import java.util.HashSet;
import java.util.Set;
/**
* Storage layout when using bucket index. Data distribution and files organization are in a specific way.
*/
public class HoodieBucketLayout extends HoodieStorageLayout {
public static final Set<WriteOperationType> SUPPORTED_OPERATIONS = new HashSet<WriteOperationType>() {{
add(WriteOperationType.INSERT);
add(WriteOperationType.INSERT_PREPPED);
add(WriteOperationType.UPSERT);
add(WriteOperationType.UPSERT_PREPPED);
add(WriteOperationType.INSERT_OVERWRITE);
add(WriteOperationType.DELETE);
add(WriteOperationType.COMPACT);
add(WriteOperationType.DELETE_PARTITION);
}};
public HoodieBucketLayout(HoodieWriteConfig config) {
super(config);
}
/**
* Bucketing controls the number of file groups directly.
*/
@Override
public boolean determinesNumFileGroups() {
return true;
}
public Option<String> layoutPartitionerClass() {
return config.contains(HoodieLayoutConfig.LAYOUT_PARTITIONER_CLASS_NAME)
? Option.of(config.getString(HoodieLayoutConfig.LAYOUT_PARTITIONER_CLASS_NAME.key()))
: Option.empty();
}
@Override
public boolean doesNotSupport(WriteOperationType operationType) {
return !SUPPORTED_OPERATIONS.contains(operationType);
}
}

View File

@@ -0,0 +1,45 @@
/*
* 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.storage;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
/**
* Default storage layout with non-constraints.
*/
public class HoodieDefaultLayout extends HoodieStorageLayout {
public HoodieDefaultLayout(HoodieWriteConfig config) {
super(config);
}
public boolean determinesNumFileGroups() {
return false;
}
public Option<String> layoutPartitionerClass() {
return Option.empty();
}
public boolean doesNotSupport(WriteOperationType operationType) {
return false;
}
}

View File

@@ -0,0 +1,38 @@
/*
* 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.storage;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieNotSupportedException;
/**
* A factory to generate layout.
*/
public final class HoodieLayoutFactory {
public static HoodieStorageLayout createLayout(HoodieWriteConfig config) {
switch (config.getLayoutType()) {
case DEFAULT:
return new HoodieDefaultLayout(config);
case BUCKET:
return new HoodieBucketLayout(config);
default:
throw new HoodieNotSupportedException("Unknown layout type, set " + config.getLayoutType());
}
}
}

View File

@@ -0,0 +1,56 @@
/*
* 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.storage;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import java.io.Serializable;
/**
* Storage layout defines how the files are organized within a table.
*/
public abstract class HoodieStorageLayout implements Serializable {
protected final HoodieWriteConfig config;
public HoodieStorageLayout(HoodieWriteConfig config) {
this.config = config;
}
/**
* By default, layout does not directly control the total number of files.
*/
public abstract boolean determinesNumFileGroups();
/**
* Return the layout specific partitioner for writing data, if any.
*/
public abstract Option<String> layoutPartitionerClass();
/**
* Determines if the operation is supported by the layout.
*/
public abstract boolean doesNotSupport(WriteOperationType operationType);
public enum LayoutType {
DEFAULT, BUCKET
}
}