diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index 1180845a6ed8a..9f5f49a5086e1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; import javax.annotation.Nonnull; @@ -53,10 +54,14 @@ public class HoodieClusteringConfig extends HoodieConfig { "org.apache.hudi.client.clustering.plan.strategy.SparkSizeBasedClusteringPlanStrategy"; public static final String FLINK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = "org.apache.hudi.client.clustering.plan.strategy.FlinkSizeBasedClusteringPlanStrategy"; + public static final String SPARK_CONSISTENT_BUCKET_CLUSTERING_PLAN_STRATEGY = + "org.apache.hudi.client.clustering.plan.strategy.SparkConsistentBucketClusteringPlanStrategy"; public static final String JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY = "org.apache.hudi.client.clustering.plan.strategy.JavaSizeBasedClusteringPlanStrategy"; public static final String SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY = "org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy"; + public static final String SPARK_CONSISTENT_BUCKET_EXECUTION_STRATEGY = + "org.apache.hudi.client.clustering.run.strategy.SparkConsistentBucketClusteringExecutionStrategy"; public static final String JAVA_SORT_AND_SIZE_EXECUTION_STRATEGY = "org.apache.hudi.client.clustering.run.strategy.JavaSortAndSizeExecutionStrategy"; public static final String PLAN_PARTITION_FILTER_MODE = @@ -589,18 +594,46 @@ public Builder withDataOptimizeBuildCurveSampleNumber(int sampleNumber) { } public HoodieClusteringConfig build() { - clusteringConfig.setDefaultValue( - PLAN_STRATEGY_CLASS_NAME, getDefaultPlanStrategyClassName(engineType)); - clusteringConfig.setDefaultValue( - EXECUTION_STRATEGY_CLASS_NAME, getDefaultExecutionStrategyClassName(engineType)); + setDefaults(); + validate(); + + return clusteringConfig; + } + + private void setDefaults() { + // Consistent hashing bucket index + if (clusteringConfig.contains(HoodieIndexConfig.INDEX_TYPE.key()) + && clusteringConfig.contains(HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE.key()) + && clusteringConfig.getString(HoodieIndexConfig.INDEX_TYPE.key()).equalsIgnoreCase(HoodieIndex.IndexType.BUCKET.name()) + && clusteringConfig.getString(HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE.key()).equalsIgnoreCase(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING.name())) { + clusteringConfig.setDefaultValue(PLAN_STRATEGY_CLASS_NAME, SPARK_CONSISTENT_BUCKET_CLUSTERING_PLAN_STRATEGY); + clusteringConfig.setDefaultValue(EXECUTION_STRATEGY_CLASS_NAME, SPARK_CONSISTENT_BUCKET_EXECUTION_STRATEGY); + } else { + clusteringConfig.setDefaultValue( + PLAN_STRATEGY_CLASS_NAME, getDefaultPlanStrategyClassName(engineType)); + clusteringConfig.setDefaultValue( + EXECUTION_STRATEGY_CLASS_NAME, getDefaultExecutionStrategyClassName(engineType)); + } clusteringConfig.setDefaults(HoodieClusteringConfig.class.getName()); + } + private void validate() { boolean inlineCluster = clusteringConfig.getBoolean(HoodieClusteringConfig.INLINE_CLUSTERING); boolean inlineClusterSchedule = clusteringConfig.getBoolean(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING); ValidationUtils.checkArgument(!(inlineCluster && inlineClusterSchedule), String.format("Either of inline clustering (%s) or " + "schedule inline clustering (%s) can be enabled. Both can't be set to true at the same time. %s,%s", HoodieClusteringConfig.INLINE_CLUSTERING.key(), HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING.key(), inlineCluster, inlineClusterSchedule)); - return clusteringConfig; + + // Consistent hashing bucket index + if (clusteringConfig.contains(HoodieIndexConfig.INDEX_TYPE.key()) + && clusteringConfig.contains(HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE.key()) + && clusteringConfig.getString(HoodieIndexConfig.INDEX_TYPE.key()).equalsIgnoreCase(HoodieIndex.IndexType.BUCKET.name()) + && clusteringConfig.getString(HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE.key()).equalsIgnoreCase(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING.name())) { + ValidationUtils.checkArgument(clusteringConfig.getString(PLAN_STRATEGY_CLASS_NAME).equals(SPARK_CONSISTENT_BUCKET_CLUSTERING_PLAN_STRATEGY), + "Consistent hashing bucket index only supports clustering plan strategy : " + SPARK_CONSISTENT_BUCKET_CLUSTERING_PLAN_STRATEGY); + ValidationUtils.checkArgument(clusteringConfig.getString(EXECUTION_STRATEGY_CLASS_NAME).equals(SPARK_CONSISTENT_BUCKET_EXECUTION_STRATEGY), + "Consistent hashing bucket index only supports clustering execution strategy : " + SPARK_CONSISTENT_BUCKET_EXECUTION_STRATEGY); + } } private String getDefaultPlanStrategyClassName(EngineType engineType) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java index 0cef5550af8b7..57e201bb21eed 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java @@ -30,6 +30,9 @@ import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + import javax.annotation.concurrent.Immutable; import java.io.File; @@ -62,6 +65,8 @@ + "which tags incoming records as either inserts or updates to older records.") public class HoodieIndexConfig extends HoodieConfig { + private static final Logger LOG = LogManager.getLogger(HoodieIndexConfig.class); + public static final ConfigProperty INDEX_TYPE = ConfigProperty .key("hoodie.index.type") // Builder#getDefaultIndexType has already set it according to engine type @@ -263,12 +268,37 @@ public class HoodieIndexConfig extends HoodieConfig { .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 BUCKET_INDEX_MAX_NUM_BUCKETS = ConfigProperty + .key("hoodie.bucket.index.max.num.buckets") + .noDefaultValue() + .withDocumentation("Only applies if bucket index engine is consistent hashing. Determine the upper bound of " + + "the number of buckets in the hudi table. Bucket resizing cannot be done higher than this max limit."); + + public static final ConfigProperty BUCKET_INDEX_MIN_NUM_BUCKETS = ConfigProperty + .key("hoodie.bucket.index.min.num.buckets") + .noDefaultValue() + .withDocumentation("Only applies if bucket index engine is consistent hashing. Determine the lower bound of " + + "the number of buckets in the hudi table. Bucket resizing cannot be done lower than this min limit."); + public static final ConfigProperty 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"); + public static final ConfigProperty BUCKET_SPLIT_THRESHOLD = ConfigProperty + .key("hoodie.bucket.index.split.threshold") + .defaultValue(2.0) + .withDocumentation("Control if the bucket should be split when using consistent hashing bucket index." + + "Specifically, if a file slice size reaches `hoodie.xxxx.max.file.size` * threshold, then split will be carried out."); + + public static final ConfigProperty BUCKET_MERGE_THRESHOLD = ConfigProperty + .key("hoodie.bucket.index.merge.threshold") + .defaultValue(0.2) + .withDocumentation("Control if buckets should be merged when using consistent hashing bucket index" + + "Specifically, if a file slice size is smaller than `hoodie.xxxx.max.file.size` * threshold, then it will be considered" + + "as a merge candidate."); + /** * Deprecated configs. These are now part of {@link HoodieHBaseIndexConfig}. */ @@ -600,6 +630,16 @@ public Builder withBucketNum(String bucketNum) { return this; } + public Builder withBucketMaxNum(int bucketMaxNum) { + hoodieIndexConfig.setValue(BUCKET_INDEX_MAX_NUM_BUCKETS, String.valueOf(bucketMaxNum)); + return this; + } + + public Builder withBucketMinNum(int bucketMinNum) { + hoodieIndexConfig.setValue(BUCKET_INDEX_MIN_NUM_BUCKETS, String.valueOf(bucketMinNum)); + return this; + } + public Builder withIndexKeyField(String keyField) { hoodieIndexConfig.setValue(BUCKET_INDEX_HASH_FIELD, keyField); return this; @@ -650,6 +690,20 @@ private void validateBucketIndexConfig() { if (hoodieIndexConfig.getIntOrDefault(BUCKET_INDEX_NUM_BUCKETS) <= 0) { throw new HoodieIndexException("When using bucket index, hoodie.bucket.index.num.buckets cannot be negative."); } + int bucketNum = hoodieIndexConfig.getInt(BUCKET_INDEX_NUM_BUCKETS); + if (StringUtils.isNullOrEmpty(hoodieIndexConfig.getString(BUCKET_INDEX_MAX_NUM_BUCKETS))) { + hoodieIndexConfig.setValue(BUCKET_INDEX_MAX_NUM_BUCKETS, Integer.toString(bucketNum)); + } else if (hoodieIndexConfig.getInt(BUCKET_INDEX_MAX_NUM_BUCKETS) < bucketNum) { + LOG.warn("Maximum bucket number is smaller than bucket number, maximum: " + hoodieIndexConfig.getInt(BUCKET_INDEX_MAX_NUM_BUCKETS) + ", bucketNum: " + bucketNum); + hoodieIndexConfig.setValue(BUCKET_INDEX_MAX_NUM_BUCKETS, Integer.toString(bucketNum)); + } + + if (StringUtils.isNullOrEmpty(hoodieIndexConfig.getString(BUCKET_INDEX_MIN_NUM_BUCKETS))) { + hoodieIndexConfig.setValue(BUCKET_INDEX_MIN_NUM_BUCKETS, Integer.toString(bucketNum)); + } else if (hoodieIndexConfig.getInt(BUCKET_INDEX_MIN_NUM_BUCKETS) > bucketNum) { + LOG.warn("Minimum bucket number is larger than the bucket number, minimum: " + hoodieIndexConfig.getInt(BUCKET_INDEX_MIN_NUM_BUCKETS) + ", bucketNum: " + bucketNum); + hoodieIndexConfig.setValue(BUCKET_INDEX_MIN_NUM_BUCKETS, Integer.toString(bucketNum)); + } } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 6178e63e3606c..90a468368f1ae 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1644,6 +1644,22 @@ public int getBucketIndexNumBuckets() { return getIntOrDefault(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS); } + public int getBucketIndexMaxNumBuckets() { + return getInt(HoodieIndexConfig.BUCKET_INDEX_MAX_NUM_BUCKETS); + } + + public int getBucketIndexMinNumBuckets() { + return getInt(HoodieIndexConfig.BUCKET_INDEX_MIN_NUM_BUCKETS); + } + + public double getBucketSplitThreshold() { + return getDouble(HoodieIndexConfig.BUCKET_SPLIT_THRESHOLD); + } + + public double getBucketMergeThreshold() { + return getDouble(HoodieIndexConfig.BUCKET_MERGE_THRESHOLD); + } + public String getBucketIndexHashField() { return getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD); } @@ -1651,6 +1667,19 @@ public String getBucketIndexHashField() { /** * storage properties. */ + public long getMaxFileSize(HoodieFileFormat format) { + switch (format) { + case PARQUET: + return getParquetMaxFileSize(); + case HFILE: + return getHFileMaxFileSize(); + case ORC: + return getOrcMaxFileSize(); + default: + throw new HoodieNotSupportedException("Unknown file format: " + format); + } + } + public long getParquetMaxFileSize() { return getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java index 1182c45c72479..3ede648e23440 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java @@ -88,6 +88,18 @@ public abstract HoodieData updateLocation( HoodieData writeStatuses, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException; + + /** + * Extracts the location of written records, and updates the index. + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) + public HoodieData updateLocation( + HoodieData writeStatuses, HoodieEngineContext context, + HoodieTable hoodieTable, String instant) throws HoodieIndexException { + return updateLocation(writeStatuses, context, hoodieTable); + } + + /** * Rollback the effects of the commit made at instantTime. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.java index 4955087333a25..1ce68ef97bf29 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.java @@ -28,8 +28,8 @@ public interface BucketIndexLocationMapper extends Serializable { /** - * Get record location given hoodie key and partition path + * Get record location given hoodie key */ - Option getRecordLocation(HoodieKey key, String partitionPath); + Option getRecordLocation(HoodieKey key); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIdentifier.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIdentifier.java index c44a8a6ccfb0c..d7007f11626d8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIdentifier.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIdentifier.java @@ -22,14 +22,22 @@ 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.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.hash.HashID; +import org.apache.hudi.exception.HoodieClusteringException; +import org.jetbrains.annotations.NotNull; + +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.SortedMap; import java.util.TreeMap; +import java.util.stream.Collectors; public class ConsistentBucketIdentifier extends BucketIdentifier { @@ -68,7 +76,7 @@ public int getNumBuckets() { /** * Get bucket of the given file group * - * @param fileId the file group id. NOTE: not filePfx (i.e., uuid) + * @param fileId the file group id. NOTE: not filePrefix (i.e., uuid) */ public ConsistentHashingNode getBucketByFileId(String fileId) { return fileIdToBucket.get(fileId); @@ -88,11 +96,81 @@ protected ConsistentHashingNode getBucket(int hashValue) { return tailMap.isEmpty() ? ring.firstEntry().getValue() : tailMap.get(tailMap.firstKey()); } + /** + * Get the former node of the given node (inferred from file id). + */ + public ConsistentHashingNode getFormerBucket(String fileId) { + return getFormerBucket(getBucketByFileId(fileId).getValue()); + } + + /** + * Get the former node of the given node (inferred from hash value). + */ + public ConsistentHashingNode getFormerBucket(int hashValue) { + SortedMap headMap = ring.headMap(hashValue); + return headMap.isEmpty() ? ring.lastEntry().getValue() : headMap.get(headMap.lastKey()); + } + + public List mergeBucket(List fileIds) { + ValidationUtils.checkArgument(fileIds.size() >= 2, "At least two file groups should be provided for merging"); + // Get nodes using fileIds + List nodes = fileIds.stream().map(this::getBucketByFileId).collect(Collectors.toList()); + + // Validate the input + for (int i = 0; i < nodes.size() - 1; ++i) { + ValidationUtils.checkState(getFormerBucket(nodes.get(i + 1).getValue()).getValue() == nodes.get(i).getValue(), "Cannot merge discontinuous hash range"); + } + + // Create child nodes with proper tag (keep the last one and delete other nodes) + List childNodes = new ArrayList<>(nodes.size()); + for (int i = 0; i < nodes.size() - 1; ++i) { + childNodes.add(new ConsistentHashingNode(nodes.get(i).getValue(), null, ConsistentHashingNode.NodeTag.DELETE)); + } + childNodes.add(new ConsistentHashingNode(nodes.get(nodes.size() - 1).getValue(), FSUtils.createNewFileIdPfx(), ConsistentHashingNode.NodeTag.REPLACE)); + return childNodes; + } + + public Option> splitBucket(String fileId) { + ConsistentHashingNode bucket = getBucketByFileId(fileId); + ValidationUtils.checkState(bucket != null, "FileId has no corresponding bucket"); + return splitBucket(bucket); + } + + /** + * Split bucket in the range middle, also generate the corresponding file ids + * + * TODO support different split criteria, e.g., distribute records evenly using statistics + * + * @param bucket parent bucket + * @return lists of children buckets + */ + public Option> splitBucket(@NotNull ConsistentHashingNode bucket) { + ConsistentHashingNode formerBucket = getFormerBucket(bucket.getValue()); + + long mid = (long) formerBucket.getValue() + bucket.getValue() + + (formerBucket.getValue() < bucket.getValue() ? 0 : (HoodieConsistentHashingMetadata.HASH_VALUE_MASK + 1L)); + mid = (mid >> 1) & HoodieConsistentHashingMetadata.HASH_VALUE_MASK; + + // Cannot split as it already is the smallest bucket range + if (mid == formerBucket.getValue() || mid == bucket.getValue()) { + return Option.empty(); + } + + return Option.of(Arrays.asList( + new ConsistentHashingNode((int) mid, FSUtils.createNewFileIdPfx(), ConsistentHashingNode.NodeTag.REPLACE), + new ConsistentHashingNode(bucket.getValue(), FSUtils.createNewFileIdPfx(), ConsistentHashingNode.NodeTag.REPLACE)) + ); + } + /** * 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. + *

+ * Children nodes are also considered, and will override the original nodes, + * which is used during bucket resizing (i.e., children nodes take the place + * of the original nodes) */ private void initialize() { for (ConsistentHashingNode p : metadata.getNodes()) { @@ -100,5 +178,25 @@ private void initialize() { // One bucket has only one file group, so append 0 directly fileIdToBucket.put(FSUtils.createNewFileId(p.getFileIdPrefix(), 0), p); } + + // Handle children nodes, i.e., replace or delete the original nodes + ConsistentHashingNode tmp; + for (ConsistentHashingNode p : metadata.getChildrenNodes()) { + switch (p.getTag()) { + case REPLACE: + tmp = ring.put(p.getValue(), p); + if (tmp != null) { + fileIdToBucket.remove(FSUtils.createNewFileId(tmp.getFileIdPrefix(), 0)); + } + fileIdToBucket.put(FSUtils.createNewFileId(p.getFileIdPrefix(), 0), p); + break; + case DELETE: + tmp = ring.remove(p.getValue()); + fileIdToBucket.remove(FSUtils.createNewFileId(tmp.getFileIdPrefix(), 0)); + break; + default: + throw new HoodieClusteringException("Children node is tagged as NORMAL or unknown tag: " + p); + } + } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java index cbb3b07f4457f..3b3eee37e5602 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java @@ -69,9 +69,9 @@ public HoodieData> tagLocation( HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { - // Initialize necessary information before tagging. e.g., hashing metadata + // Get bucket location mapper for the given partitions List partitions = records.map(HoodieRecord::getPartitionPath).distinct().collectAsList(); - LOG.info("Initializing hashing metadata for partitions: " + partitions); + LOG.info("Get BucketIndexLocationMapper for partitions: " + partitions); BucketIndexLocationMapper mapper = getLocationMapper(hoodieTable, partitions); return records.mapPartitions(iterator -> @@ -80,7 +80,7 @@ public HoodieData> tagLocation( protected HoodieRecord computeNext() { // TODO maybe batch the operation to improve performance HoodieRecord record = inputItr.next(); - Option loc = mapper.getRecordLocation(record.getKey(), record.getPartitionPath()); + Option loc = mapper.getRecordLocation(record.getKey()); return HoodieIndexUtils.getTaggedRecord(record, loc); } }, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java index 92ac4f69b2c42..2ccebb472f277 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java @@ -90,9 +90,9 @@ public SimpleBucketIndexLocationMapper(HoodieTable table, List partition } @Override - public Option getRecordLocation(HoodieKey key, String partitionPath) { + public Option getRecordLocation(HoodieKey key) { int bucketId = BucketIdentifier.getBucketId(key, indexKeyFields, numBuckets); - Map bucketIdToFileIdMapping = partitionPathFileIDList.get(partitionPath); + Map bucketIdToFileIdMapping = partitionPathFileIDList.get(key.getPartitionPath()); return Option.ofNullable(bucketIdToFileIdMapping.getOrDefault(bucketId, null)); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index e0d40642a6a24..44a0bcc33e719 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -49,6 +49,7 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock; +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.view.TableFileSystemView.SliceView; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Option; @@ -151,6 +152,11 @@ private void init(HoodieRecord record) { logFiles = fileSlice.get().getLogFiles().map(HoodieLogFile::getFileName).collect(Collectors.toList()); } else { baseInstantTime = instantTime; + // Handle log file only case. This is necessary for the concurrent clustering and writer case (e.g., consistent hashing bucket index). + // NOTE: flink engine use instantTime to mark operation type, check BaseFlinkCommitActionExecutor::execute + if (record.getCurrentLocation() != null && HoodieInstantTimeGenerator.isValidInstantTime(record.getCurrentLocation().getInstantTime())) { + baseInstantTime = record.getCurrentLocation().getInstantTime(); + } // This means there is no base data file, start appending to a new log file fileSlice = Option.of(new FileSlice(partitionPath, baseInstantTime, this.fileId)); LOG.info("New AppendHandle for partition :" + partitionPath); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java index 15ead5efb0080..64f68e56f5066 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java @@ -78,8 +78,9 @@ protected Option createClusteringPlan() { } LOG.info("Generating clustering plan for table " + config.getBasePath()); - ClusteringPlanStrategy strategy = (ClusteringPlanStrategy) - ReflectionUtils.loadClass(ClusteringPlanStrategy.checkAndGetClusteringPlanStrategy(config), table, context, config); + ClusteringPlanStrategy strategy = (ClusteringPlanStrategy) ReflectionUtils.loadClass( + ClusteringPlanStrategy.checkAndGetClusteringPlanStrategy(config), + new Class[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config); return strategy.generateClusteringPlan(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java index 34b35d2ba946d..dd827ff5a94c0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -104,6 +104,13 @@ public ClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineConte */ public abstract Option generateClusteringPlan(); + /** + * Check if the clustering can proceed. If not (i.e., return false), the PlanStrategy will generate an empty plan to stop the scheduling. + */ + public boolean checkPrecondition() { + return true; + } + /** * Return file slices eligible for clustering. FileIds in pending clustering/compaction are not eligible for clustering. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java index 5d62ef390233f..7042585f59bb7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java @@ -68,6 +68,10 @@ protected List filterPartitionPaths(List partitionPaths) { @Override public Option generateClusteringPlan() { + if (!checkPrecondition()) { + return Option.empty(); + } + HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient(); LOG.info("Scheduling clustering for " + metaClient.getBasePath()); HoodieWriteConfig config = getWriteConfig(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java index 4e33eb06038cd..c08c3f312dbc8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java @@ -22,19 +22,23 @@ import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.table.HoodieTable; +import java.io.Serializable; import java.util.Set; /** * When file groups in clustering, write records to these file group need to check. */ -public abstract class UpdateStrategy { +public abstract class UpdateStrategy implements Serializable { - protected final HoodieEngineContext engineContext; - protected Set fileGroupsInPendingClustering; + protected final transient HoodieEngineContext engineContext; + protected final HoodieTable table; + protected final Set fileGroupsInPendingClustering; - protected UpdateStrategy(HoodieEngineContext engineContext, Set fileGroupsInPendingClustering) { + public UpdateStrategy(HoodieEngineContext engineContext, HoodieTable table, Set fileGroupsInPendingClustering) { this.engineContext = engineContext; + this.table = table; this.fileGroupsInPendingClustering = fileGroupsInPendingClustering; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java index 5355194ff75bf..aa7196e3dbbed 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java @@ -38,6 +38,8 @@ public abstract HoodieWriteMetadata bulkInsert(I inputRecords, String instant /** * Only write input records. Does not change timeline/index. Return information about new files created. + * + * @param writeHandleFactory default write handle factory writing records. */ public abstract O bulkInsert(I inputRecords, String instantTime, HoodieTable table, HoodieWriteConfig config, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 31c8bbd6d30d2..495cef117a8c8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -262,7 +262,7 @@ protected HoodieWriteMetadata> executeClustering(HoodieC private HoodieData updateIndex(HoodieData writeStatuses, HoodieWriteMetadata> result) { Instant indexStartTime = Instant.now(); // Update the index back - HoodieData statuses = table.getIndex().updateLocation(writeStatuses, context, table); + HoodieData statuses = table.getIndex().updateLocation(writeStatuses, context, table, instantTime); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); return statuses; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieConsistentBucketLayout.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieConsistentBucketLayout.java index 0ed2b9c939a7b..acea61177a5f9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieConsistentBucketLayout.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieConsistentBucketLayout.java @@ -32,12 +32,15 @@ public class HoodieConsistentBucketLayout extends HoodieStorageLayout { public static final Set SUPPORTED_OPERATIONS = CollectionUtils.createImmutableSet( WriteOperationType.INSERT, WriteOperationType.INSERT_PREPPED, + WriteOperationType.BULK_INSERT, + WriteOperationType.BULK_INSERT_PREPPED, WriteOperationType.UPSERT, WriteOperationType.UPSERT_PREPPED, WriteOperationType.INSERT_OVERWRITE, WriteOperationType.DELETE, WriteOperationType.COMPACT, - WriteOperationType.DELETE_PARTITION + WriteOperationType.DELETE_PARTITION, + WriteOperationType.CLUSTER ); public HoodieConsistentBucketLayout(HoodieWriteConfig config) { diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java index 0adbb998a0658..ac9585366f50d 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java @@ -46,15 +46,16 @@ import java.util.Properties; import java.util.function.Function; -import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE; import static org.apache.hudi.config.HoodieArchivalConfig.ASYNC_ARCHIVE; import static org.apache.hudi.config.HoodieCleanConfig.ASYNC_CLEAN; import static org.apache.hudi.config.HoodieCleanConfig.AUTO_CLEAN; import static org.apache.hudi.config.HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY; +import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE; import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT; import static org.apache.hudi.config.HoodieWriteConfig.TABLE_SERVICES_ENABLED; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestHoodieWriteConfig { @@ -371,6 +372,32 @@ public void testAutoConcurrencyConfigAdjustmentWithMetadataTableDisabled(HoodieT HoodieFailedWritesCleaningPolicy.LAZY, FileSystemBasedLockProviderTestClass.class.getName()); } + @Test + public void testConsistentBucketIndexDefaultClusteringConfig() { + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BUCKET) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING).build()) + .build(); + assertEquals(HoodieClusteringConfig.SPARK_CONSISTENT_BUCKET_CLUSTERING_PLAN_STRATEGY, writeConfig.getClusteringPlanStrategyClass()); + assertEquals(HoodieClusteringConfig.SPARK_CONSISTENT_BUCKET_EXECUTION_STRATEGY, writeConfig.getClusteringExecutionStrategyClass()); + } + + @Test + public void testConsistentBucketIndexInvalidClusteringConfig() { + TypedProperties consistentBucketIndexProps = HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BUCKET) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING).build().getProps(); + HoodieWriteConfig.Builder writeConfigBuilder = HoodieWriteConfig.newBuilder().withPath("/tmp"); + + assertThrows(IllegalArgumentException.class, + () -> writeConfigBuilder.withClusteringConfig(HoodieClusteringConfig.newBuilder() + .fromProperties(consistentBucketIndexProps) + .withClusteringPlanStrategyClass(HoodieClusteringConfig.JAVA_SIZED_BASED_CLUSTERING_PLAN_STRATEGY).build())); + assertThrows(IllegalArgumentException.class, + () -> writeConfigBuilder.withClusteringConfig(HoodieClusteringConfig.newBuilder() + .fromProperties(consistentBucketIndexProps) + .withClusteringExecutionStrategyClass(HoodieClusteringConfig.SPARK_SORT_AND_SIZE_EXECUTION_STRATEGY).build())); + } + private HoodieWriteConfig createWriteConfig(Map configs) { final Properties properties = new Properties(); configs.forEach(properties::setProperty); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java index 31f33890ad318..1d8d069f4f1c3 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java @@ -26,12 +26,13 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.List; +import static org.junit.jupiter.api.Assertions.assertEquals; + public class TestBucketIdentifier { public static final String NESTED_COL_SCHEMA = "{\"type\":\"record\", \"name\":\"nested_col\",\"fields\": [" @@ -103,20 +104,20 @@ public void testBucketIdWithComplexRecordKey() { public void testGetHashKeys() { BucketIdentifier identifier = new BucketIdentifier(); List keys = identifier.getHashKeys(new HoodieKey("abc", "partition"), ""); - Assertions.assertEquals(1, keys.size()); - Assertions.assertEquals("abc", keys.get(0)); + assertEquals(1, keys.size()); + assertEquals("abc", keys.get(0)); keys = identifier.getHashKeys(new HoodieKey("f1:abc", "partition"), "f1"); - Assertions.assertEquals(1, keys.size()); - Assertions.assertEquals("abc", keys.get(0)); + assertEquals(1, keys.size()); + assertEquals("abc", keys.get(0)); keys = identifier.getHashKeys(new HoodieKey("f1:abc,f2:bcd", "partition"), "f2"); - Assertions.assertEquals(1, keys.size()); - Assertions.assertEquals("bcd", keys.get(0)); + assertEquals(1, keys.size()); + assertEquals("bcd", keys.get(0)); keys = identifier.getHashKeys(new HoodieKey("f1:abc,f2:bcd", "partition"), "f1,f2"); - Assertions.assertEquals(2, keys.size()); - Assertions.assertEquals("abc", keys.get(0)); - Assertions.assertEquals("bcd", keys.get(1)); + assertEquals(2, keys.size()); + assertEquals("abc", keys.get(0)); + assertEquals("bcd", keys.get(1)); } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestConsistentBucketIdIdentifier.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestConsistentBucketIdIdentifier.java index 3ffe6ded188b8..150b7a2f053f3 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestConsistentBucketIdIdentifier.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestConsistentBucketIdIdentifier.java @@ -21,20 +21,42 @@ 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.util.Option; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; import static org.apache.hudi.common.model.HoodieConsistentHashingMetadata.HASH_VALUE_MASK; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Unit test of consistent bucket identifier */ public class TestConsistentBucketIdIdentifier { + private static Stream splitBucketParams() { + Object[][] data = new Object[][] { + {HASH_VALUE_MASK, 0xf, (int) (((long) 0xf + HASH_VALUE_MASK) >> 1)}, + {1, HASH_VALUE_MASK, 0}, + {0, HASH_VALUE_MASK, -1}, + {1, HASH_VALUE_MASK - 10, HASH_VALUE_MASK - 4}, + {9, HASH_VALUE_MASK - 2, 3}, + {0, HASH_VALUE_MASK - 1, HASH_VALUE_MASK} + }; + return Stream.of(data).map(Arguments::of); + } + @Test public void testGetBucket() { List nodes = Arrays.asList( @@ -44,36 +66,135 @@ public void testGetBucket() { HoodieConsistentHashingMetadata meta = new HoodieConsistentHashingMetadata((short) 0, "", "", 3, 0, nodes); ConsistentBucketIdentifier identifier = new ConsistentBucketIdentifier(meta); - Assertions.assertEquals(3, identifier.getNumBuckets()); + assertEquals(3, identifier.getNumBuckets()); // Get bucket by hash keys - Assertions.assertEquals(nodes.get(2), identifier.getBucket(Arrays.asList("Hudi"))); - Assertions.assertEquals(nodes.get(1), identifier.getBucket(Arrays.asList("bucket_index"))); - Assertions.assertEquals(nodes.get(1), identifier.getBucket(Arrays.asList("consistent_hashing"))); - Assertions.assertEquals(nodes.get(1), identifier.getBucket(Arrays.asList("bucket_index", "consistent_hashing"))); + assertEquals(nodes.get(2), identifier.getBucket(Arrays.asList("Hudi"))); + assertEquals(nodes.get(1), identifier.getBucket(Arrays.asList("bucket_index"))); + assertEquals(nodes.get(1), identifier.getBucket(Arrays.asList("consistent_hashing"))); + assertEquals(nodes.get(1), identifier.getBucket(Arrays.asList("bucket_index", "consistent_hashing"))); int[] ref1 = {2, 2, 1, 1, 0, 1, 1, 1, 0, 1}; int[] ref2 = {1, 0, 1, 0, 1, 1, 1, 0, 1, 2}; for (int i = 0; i < 10; ++i) { - Assertions.assertEquals(nodes.get(ref1[i]), identifier.getBucket(Arrays.asList(Integer.toString(i)))); - Assertions.assertEquals(nodes.get(ref2[i]), identifier.getBucket(Arrays.asList(Integer.toString(i), Integer.toString(i + 1)))); + assertEquals(nodes.get(ref1[i]), identifier.getBucket(Arrays.asList(Integer.toString(i)))); + assertEquals(nodes.get(ref2[i]), identifier.getBucket(Arrays.asList(Integer.toString(i), Integer.toString(i + 1)))); } // Get bucket by hash value - Assertions.assertEquals(nodes.get(0), identifier.getBucket(0)); - Assertions.assertEquals(nodes.get(0), identifier.getBucket(50)); - Assertions.assertEquals(nodes.get(0), identifier.getBucket(100)); - Assertions.assertEquals(nodes.get(1), identifier.getBucket(101)); - Assertions.assertEquals(nodes.get(1), identifier.getBucket(0x1fffffff)); - Assertions.assertEquals(nodes.get(1), identifier.getBucket(0x2fffffff)); - Assertions.assertEquals(nodes.get(2), identifier.getBucket(0x40000000)); - Assertions.assertEquals(nodes.get(2), identifier.getBucket(0x40000001)); - Assertions.assertEquals(nodes.get(2), identifier.getBucket(0x4fffffff)); - Assertions.assertEquals(nodes.get(0), identifier.getBucket(0x50000000)); - Assertions.assertEquals(nodes.get(0), identifier.getBucket(HASH_VALUE_MASK)); + assertEquals(nodes.get(0), identifier.getBucket(0)); + assertEquals(nodes.get(0), identifier.getBucket(50)); + assertEquals(nodes.get(0), identifier.getBucket(100)); + assertEquals(nodes.get(1), identifier.getBucket(101)); + assertEquals(nodes.get(1), identifier.getBucket(0x1fffffff)); + assertEquals(nodes.get(1), identifier.getBucket(0x2fffffff)); + assertEquals(nodes.get(2), identifier.getBucket(0x40000000)); + assertEquals(nodes.get(2), identifier.getBucket(0x40000001)); + assertEquals(nodes.get(2), identifier.getBucket(0x4fffffff)); + assertEquals(nodes.get(0), identifier.getBucket(0x50000000)); + assertEquals(nodes.get(0), identifier.getBucket(HASH_VALUE_MASK)); // Get bucket by file id - Assertions.assertEquals(nodes.get(0), identifier.getBucketByFileId(FSUtils.createNewFileId("0", 0))); - Assertions.assertEquals(nodes.get(1), identifier.getBucketByFileId(FSUtils.createNewFileId("1", 0))); - Assertions.assertEquals(nodes.get(2), identifier.getBucketByFileId(FSUtils.createNewFileId("2", 0))); + assertEquals(nodes.get(0), identifier.getBucketByFileId(FSUtils.createNewFileId("0", 0))); + assertEquals(nodes.get(1), identifier.getBucketByFileId(FSUtils.createNewFileId("1", 0))); + assertEquals(nodes.get(2), identifier.getBucketByFileId(FSUtils.createNewFileId("2", 0))); + } + + /** + * @param v0 first node hash value + * @param v1 second node hash value + * @param mid mid node hash value generated by split the first bucket v0 + */ + @ParameterizedTest + @MethodSource("splitBucketParams") + public void testSplitBucket(int v0, int v1, int mid) { + // Hash range mapping:: [0, 0xf], (0xf, MAX] + List nodes = Arrays.asList( + new ConsistentHashingNode(v0, "0"), + new ConsistentHashingNode(v1, "1")); + HoodieConsistentHashingMetadata meta = new HoodieConsistentHashingMetadata((short) 0, "", "", 4, 0, nodes); + Option> res = new ConsistentBucketIdentifier(meta).splitBucket(nodes.get(0)); + if (mid < 0) { + assertTrue(!res.isPresent()); + return; + } + + List childNodes = res.get(); + assertEquals(2, childNodes.size()); + assertTrue(childNodes.stream().allMatch(c -> c.getTag() == ConsistentHashingNode.NodeTag.REPLACE)); + assertEquals(mid, childNodes.get(0).getValue()); + assertEquals(nodes.get(0).getValue(), childNodes.get(1).getValue()); + } + + @Test + public void testMerge() { + HoodieConsistentHashingMetadata meta = new HoodieConsistentHashingMetadata("partition", 8); + List nodes = meta.getNodes(); + + List fileIds = IntStream.range(0, 3).mapToObj(i -> FSUtils.createNewFileId(nodes.get(i).getFileIdPrefix(), 0)).collect(Collectors.toList()); + List childNodes = new ConsistentBucketIdentifier(meta).mergeBucket(fileIds); + assertEquals(ConsistentHashingNode.NodeTag.DELETE, childNodes.get(0).getTag()); + assertEquals(ConsistentHashingNode.NodeTag.DELETE, childNodes.get(1).getTag()); + assertEquals(ConsistentHashingNode.NodeTag.REPLACE, childNodes.get(2).getTag()); + assertEquals(nodes.get(2).getValue(), childNodes.get(2).getValue()); + assertNotEquals(nodes.get(2).getFileIdPrefix(), childNodes.get(2).getFileIdPrefix()); + + fileIds = Arrays.asList(nodes.get(7), nodes.get(0), nodes.get(1)).stream() + .map(ConsistentHashingNode::getFileIdPrefix).map(f -> FSUtils.createNewFileId(f, 0)).collect(Collectors.toList()); + childNodes = new ConsistentBucketIdentifier(meta).mergeBucket(fileIds); + assertEquals(ConsistentHashingNode.NodeTag.DELETE, childNodes.get(0).getTag()); + assertEquals(ConsistentHashingNode.NodeTag.DELETE, childNodes.get(1).getTag()); + assertEquals(ConsistentHashingNode.NodeTag.REPLACE, childNodes.get(2).getTag()); + assertEquals(nodes.get(1).getValue(), childNodes.get(2).getValue()); + assertNotEquals(nodes.get(1).getFileIdPrefix(), childNodes.get(2).getFileIdPrefix()); + + } + + @Test + public void testNonContinuousBucketMerge() { + HoodieConsistentHashingMetadata meta = new HoodieConsistentHashingMetadata("partition", 8); + List nodes = meta.getNodes(); + + boolean exception = false; + try { + List fileIds = IntStream.range(0, 2).mapToObj(i -> FSUtils.createNewFileId(nodes.get(i * 2).getFileIdPrefix(), 0)).collect(Collectors.toList()); + new ConsistentBucketIdentifier(meta).mergeBucket(fileIds); + } catch (Exception e) { + exception = true; + } + assertTrue(exception); + } + + @Test + public void testChildrenNodesInitialization() { + HoodieConsistentHashingMetadata metadata = new HoodieConsistentHashingMetadata("partition", 8); + List childrenNodes = new ArrayList<>(); + childrenNodes.add(new ConsistentHashingNode(metadata.getNodes().get(0).getValue(), "d1", ConsistentHashingNode.NodeTag.DELETE)); + childrenNodes.add(new ConsistentHashingNode(1024, "a1", ConsistentHashingNode.NodeTag.REPLACE)); + childrenNodes.add(new ConsistentHashingNode(metadata.getNodes().get(1).getValue(), "a2", ConsistentHashingNode.NodeTag.REPLACE)); + metadata.setChildrenNodes(childrenNodes); + + ConsistentBucketIdentifier identifier = new ConsistentBucketIdentifier(metadata); + List nodes = new ArrayList<>(identifier.getNodes()); + assertEquals(1024, nodes.get(0).getValue()); + assertEquals("a1", nodes.get(0).getFileIdPrefix()); + assertEquals(metadata.getNodes().get(1).getValue(), nodes.get(1).getValue()); + assertEquals("a2", nodes.get(1).getFileIdPrefix()); + } + + @Test + public void testInvalidChildrenNodesInitialization() { + HoodieConsistentHashingMetadata metadata = new HoodieConsistentHashingMetadata("partition", 8); + List childrenNodes = new ArrayList<>(); + ConsistentBucketIdentifier identifier = new ConsistentBucketIdentifier(metadata); + childrenNodes = new ArrayList<>(); + childrenNodes.add(new ConsistentHashingNode(metadata.getNodes().get(0).getValue(), "d1", ConsistentHashingNode.NodeTag.NORMAL)); + metadata.setChildrenNodes(childrenNodes); + boolean isException = false; + try { + identifier = new ConsistentBucketIdentifier(metadata); + } catch (Exception e) { + isException = true; + } + assertEquals(true, isException); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java index 6ff063f49d445..891ddf89937e2 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategy.java @@ -20,7 +20,7 @@ import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -29,8 +29,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable; -import org.apache.hudi.table.HoodieFlinkMergeOnReadTable; +import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy; import org.apache.log4j.LogManager; @@ -53,14 +52,8 @@ public class FlinkSizeBasedClusteringPlanStrategy>, List, List> { private static final Logger LOG = LogManager.getLogger(FlinkSizeBasedClusteringPlanStrategy.class); - public FlinkSizeBasedClusteringPlanStrategy(HoodieFlinkCopyOnWriteTable table, - HoodieFlinkEngineContext engineContext, - HoodieWriteConfig writeConfig) { - super(table, engineContext, writeConfig); - } - - public FlinkSizeBasedClusteringPlanStrategy(HoodieFlinkMergeOnReadTable table, - HoodieFlinkEngineContext engineContext, + public FlinkSizeBasedClusteringPlanStrategy(HoodieTable table, + HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { super(table, engineContext, writeConfig); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java index ec7202f4d8622..b2291652413c6 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/JavaSizeBasedClusteringPlanStrategy.java @@ -21,7 +21,7 @@ import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -30,8 +30,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieJavaCopyOnWriteTable; -import org.apache.hudi.table.HoodieJavaMergeOnReadTable; +import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -53,14 +52,8 @@ public class JavaSizeBasedClusteringPlanStrategy>, List, List> { private static final Logger LOG = LogManager.getLogger(JavaSizeBasedClusteringPlanStrategy.class); - public JavaSizeBasedClusteringPlanStrategy(HoodieJavaCopyOnWriteTable table, - HoodieJavaEngineContext engineContext, - HoodieWriteConfig writeConfig) { - super(table, engineContext, writeConfig); - } - - public JavaSizeBasedClusteringPlanStrategy(HoodieJavaMergeOnReadTable table, - HoodieJavaEngineContext engineContext, + public JavaSizeBasedClusteringPlanStrategy(HoodieTable table, + HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { super(table, engineContext, writeConfig); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkConsistentBucketClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkConsistentBucketClusteringPlanStrategy.java new file mode 100644 index 0000000000000..e8985f2e096a4 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkConsistentBucketClusteringPlanStrategy.java @@ -0,0 +1,322 @@ +/* + * 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.clustering.plan.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.ConsistentHashingNode; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; +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.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.TableFileSystemView; +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.common.util.collection.Triple; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.index.bucket.ConsistentBucketIdentifier; +import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +/** + * Clustering plan strategy specifically for consistent bucket index + */ +public class SparkConsistentBucketClusteringPlanStrategy> + extends PartitionAwareClusteringPlanStrategy>, JavaRDD, JavaRDD> { + + private static final Logger LOG = LogManager.getLogger(SparkConsistentBucketClusteringPlanStrategy.class); + + public static final String METADATA_PARTITION_KEY = "clustering.group.partition"; + public static final String METADATA_CHILD_NODE_KEY = "clustering.group.child.node"; + public static final String METADATA_SEQUENCE_NUMBER_KEY = "clustering.group.sequence.no"; + + public SparkConsistentBucketClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + validate(); + } + + /** + * TODO maybe add force config to schedule the clustering. It could allow clustering on partitions that are not doing write operation. + * Block clustering if there is any ongoing concurrent writers + * + * @return true if the schedule can proceed + */ + @Override + public boolean checkPrecondition() { + HoodieTimeline timeline = getHoodieTable().getActiveTimeline().getDeltaCommitTimeline().filterInflightsAndRequested(); + if (!timeline.empty()) { + LOG.warn("When using consistent bucket, clustering cannot be scheduled async if there are concurrent writers. " + + "Writer instant: " + timeline.getInstants().collect(Collectors.toList())); + return false; + } + return true; + } + + /** + * Generate candidate clustering file slices of the given partition. + * If there is inflight / requested clustering working on the partition, then return empty list + * to ensure serialized update to the hashing metadata. + * + * @return candidate file slices to be clustered (i.e., sort, bucket split or merge) + */ + @Override + protected Stream getFileSlicesEligibleForClustering(String partition) { + TableFileSystemView fileSystemView = getHoodieTable().getFileSystemView(); + boolean isPartitionInClustering = fileSystemView.getFileGroupsInPendingClustering().anyMatch(p -> p.getLeft().getPartitionPath().equals(partition)); + if (isPartitionInClustering) { + LOG.info("Partition: " + partition + " is already in clustering, skip"); + return Stream.empty(); + } + + return super.getFileSlicesEligibleForClustering(partition); + } + + @Override + protected Map getStrategyParams() { + Map params = new HashMap<>(); + if (!StringUtils.isNullOrEmpty(getWriteConfig().getClusteringSortColumns())) { + params.put(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key(), getWriteConfig().getClusteringSortColumns()); + } + return params; + } + + /** + * Generate cluster group based on split, merge and sort rules + */ + @Override + protected Stream buildClusteringGroupsForPartition(String partitionPath, List fileSlices) { + ValidationUtils.checkArgument(getHoodieTable().getIndex() instanceof HoodieSparkConsistentBucketIndex, + "Mismatch of index type and the clustering strategy, index: " + getHoodieTable().getIndex().getClass().getSimpleName()); + Option metadata = HoodieSparkConsistentBucketIndex.loadMetadata(getHoodieTable(), partitionPath); + ValidationUtils.checkArgument(metadata.isPresent(), "Metadata is empty for partition: " + partitionPath); + ConsistentBucketIdentifier identifier = new ConsistentBucketIdentifier(metadata.get()); + + // Apply split rule + int splitSlot = getWriteConfig().getBucketIndexMaxNumBuckets() - identifier.getNumBuckets(); + Triple, Integer, List> splitResult = + buildSplitClusteringGroups(identifier, fileSlices, splitSlot); + List ret = new ArrayList<>(splitResult.getLeft()); + + // Apply merge rule + int mergeSlot = identifier.getNumBuckets() - getWriteConfig().getBucketIndexMinNumBuckets() + splitResult.getMiddle(); + Triple, Integer, List> mergeResult = + buildMergeClusteringGroup(identifier, splitResult.getRight(), mergeSlot); + ret.addAll(mergeResult.getLeft()); + + // Apply sort only to the remaining file groups + ret.addAll(mergeResult.getRight().stream().map(fs -> { + ConsistentHashingNode oldNode = identifier.getBucketByFileId(fs.getFileId()); + ConsistentHashingNode newNode = new ConsistentHashingNode(oldNode.getValue(), FSUtils.createNewFileIdPfx(), ConsistentHashingNode.NodeTag.REPLACE); + return HoodieClusteringGroup.newBuilder() + .setSlices(getFileSliceInfo(Collections.singletonList(fs))) + .setNumOutputFileGroups(1) + .setMetrics(buildMetrics(Collections.singletonList(fs))) + .setExtraMetadata(constructExtraMetadata(fs.getPartitionPath(), Collections.singletonList(newNode), identifier.getMetadata().getSeqNo())) + .build(); + }).collect(Collectors.toList())); + + return ret.stream(); + } + + /** + * Generate clustering groups according to split rules. + * Currently, we always split bucket into two sub-buckets. + * + * @param identifier bucket identifier + * @param fileSlices file slice candidate to be built as split clustering groups + * @param splitSlot number of new bucket allowed to produce, in order to constrain the upper bound of the total number of bucket + * @return list of clustering group, number of new buckets generated, remaining file slice (that does not split) + */ + protected Triple, Integer, List> buildSplitClusteringGroups( + ConsistentBucketIdentifier identifier, List fileSlices, int splitSlot) { + List retGroup = new ArrayList<>(); + List fsUntouched = new ArrayList<>(); + long splitSize = getSplitSize(); + int remainingSplitSlot = splitSlot; + for (FileSlice fs : fileSlices) { + boolean needSplit = fs.getTotalFileSize() > splitSize; + if (!needSplit || remainingSplitSlot == 0) { + fsUntouched.add(fs); + continue; + } + + Option> nodes = identifier.splitBucket(fs.getFileId()); + + // Bucket cannot be split + if (!nodes.isPresent()) { + fsUntouched.add(fs); + continue; + } + + remainingSplitSlot--; + List fsList = Collections.singletonList(fs); + retGroup.add(HoodieClusteringGroup.newBuilder() + .setSlices(getFileSliceInfo(fsList)) + .setNumOutputFileGroups(2) + .setMetrics(buildMetrics(fsList)) + .setExtraMetadata(constructExtraMetadata(fs.getPartitionPath(), nodes.get(), identifier.getMetadata().getSeqNo())) + .build()); + } + return Triple.of(retGroup, splitSlot - remainingSplitSlot, fsUntouched); + } + + /** + * Generate clustering group according to merge rules + * + * @param identifier bucket identifier + * @param fileSlices file slice candidates to be built as merge clustering groups + * @param mergeSlot number of bucket allowed to be merged, in order to guarantee the lower bound of the total number of bucket + * @return list of clustering group, number of buckets merged (removed), remaining file slice (that does not be merged) + */ + protected Triple, Integer, List> buildMergeClusteringGroup( + ConsistentBucketIdentifier identifier, List fileSlices, int mergeSlot) { + if (fileSlices.size() <= 1) { + return Triple.of(Collections.emptyList(), 0, fileSlices); + } + + long mergeSize = getMergeSize(); + int remainingMergeSlot = mergeSlot; + List groups = new ArrayList<>(); + boolean[] added = new boolean[fileSlices.size()]; + + fileSlices.sort(Comparator.comparingInt(a -> identifier.getBucketByFileId(a.getFileId()).getValue())); + // In each round, we check if the ith file slice can be merged with its predecessors and successors + for (int i = 0; i < fileSlices.size(); ++i) { + if (added[i] || fileSlices.get(i).getTotalFileSize() > mergeSize) { + continue; + } + + // 0: startIdx, 1: endIdx + int[] rangeIdx = {i, i}; + long totalSize = fileSlices.get(i).getTotalFileSize(); + // Do backward check first (k == 0), and then forward check (k == 1) + for (int k = 0; k < 2; ++k) { + boolean forward = k == 1; + do { + int nextIdx = forward ? (rangeIdx[k] + 1 < fileSlices.size() ? rangeIdx[k] + 1 : 0) : (rangeIdx[k] >= 1 ? rangeIdx[k] - 1 : fileSlices.size() - 1); + boolean isNeighbour = identifier.getBucketByFileId(fileSlices.get(nextIdx).getFileId()) == identifier.getFormerBucket(fileSlices.get(rangeIdx[k]).getFileId()); + /** + * Merge condition: + * 1. there is still slot to merge bucket + * 2. the previous file slices is not merged + * 3. the previous file slice and current file slice are neighbour in the hash ring + * 4. Both the total file size up to now and the previous file slice size are smaller than merge size threshold + */ + if (remainingMergeSlot == 0 || added[nextIdx] || !isNeighbour || totalSize > mergeSize || fileSlices.get(nextIdx).getTotalFileSize() > mergeSize) { + break; + } + + // Mark preIdx as merge candidate + totalSize += fileSlices.get(nextIdx).getTotalFileSize(); + rangeIdx[k] = nextIdx; + remainingMergeSlot--; + } while (rangeIdx[k] != i); + } + + int startIdx = rangeIdx[0]; + int endIdx = rangeIdx[1]; + if (endIdx == i && startIdx == i) { + continue; + } + + // Construct merge group if there is at least two file slices + List fs = new ArrayList<>(); + while (true) { + added[startIdx] = true; + fs.add(fileSlices.get(startIdx)); + if (startIdx == endIdx) { + break; + } + startIdx = startIdx + 1 < fileSlices.size() ? startIdx + 1 : 0; + } + + groups.add(HoodieClusteringGroup.newBuilder() + .setSlices(getFileSliceInfo(fs)) + .setNumOutputFileGroups(1) + .setMetrics(buildMetrics(fs)) + .setExtraMetadata( + constructExtraMetadata( + fs.get(0).getPartitionPath(), + identifier.mergeBucket(fs.stream().map(FileSlice::getFileId).collect(Collectors.toList())), + identifier.getMetadata().getSeqNo())) + .build()); + } + + // Collect file slices that are not involved in merge + List fsUntouched = IntStream.range(0, fileSlices.size()).filter(i -> !added[i]) + .mapToObj(fileSlices::get).collect(Collectors.toList()); + + return Triple.of(groups, mergeSlot - remainingMergeSlot, fsUntouched); + } + + /** + * Construct extra metadata for clustering group + */ + private Map constructExtraMetadata(String partition, List nodes, int seqNo) { + Map extraMetadata = new HashMap<>(); + try { + extraMetadata.put(METADATA_PARTITION_KEY, partition); + extraMetadata.put(METADATA_CHILD_NODE_KEY, ConsistentHashingNode.toJsonString(nodes)); + extraMetadata.put(METADATA_SEQUENCE_NUMBER_KEY, Integer.toString(seqNo)); + } catch (IOException e) { + LOG.error("Failed to construct extra metadata, partition: " + partition + ", nodes:" + nodes); + throw new HoodieClusteringException("Failed to construct extra metadata, partition: " + partition + ", nodes:" + nodes); + } + return extraMetadata; + } + + private long getSplitSize() { + HoodieFileFormat format = getHoodieTable().getMetaClient().getTableConfig().getBaseFileFormat(); + return (long) (getWriteConfig().getMaxFileSize(format) * getWriteConfig().getBucketSplitThreshold()); + } + + private long getMergeSize() { + HoodieFileFormat format = getHoodieTable().getMetaClient().getTableConfig().getBaseFileFormat(); + return (long) (getWriteConfig().getMaxFileSize(format) * getWriteConfig().getBucketMergeThreshold()); + } + + private void validate() { + ValidationUtils.checkArgument(getHoodieTable().getIndex() instanceof HoodieSparkConsistentBucketIndex, + "SparConsistentBucketClusteringPlanStrategy is only applicable to table with consistent hash index"); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java index 88c3057f2f0ad..acb8dadf0a7bb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSingleFileSortPlanStrategy.java @@ -20,13 +20,12 @@ package org.apache.hudi.client.clustering.plan.strategy; import org.apache.hudi.avro.model.HoodieClusteringGroup; -import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; -import org.apache.hudi.table.HoodieSparkMergeOnReadTable; +import org.apache.hudi.table.HoodieTable; import java.util.Collections; import java.util.List; @@ -40,11 +39,7 @@ public class SparkSingleFileSortPlanStrategy> extends SparkSizeBasedClusteringPlanStrategy { - public SparkSingleFileSortPlanStrategy(HoodieSparkCopyOnWriteTable table, HoodieSparkEngineContext engineContext, HoodieWriteConfig writeConfig) { - super(table, engineContext, writeConfig); - } - - public SparkSingleFileSortPlanStrategy(HoodieSparkMergeOnReadTable table, HoodieSparkEngineContext engineContext, HoodieWriteConfig writeConfig) { + public SparkSingleFileSortPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { super(table, engineContext, writeConfig); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java index 6629569d096b3..46d1851d131b8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java @@ -20,7 +20,7 @@ import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -29,8 +29,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; -import org.apache.hudi.table.HoodieSparkMergeOnReadTable; +import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy; import org.apache.log4j.LogManager; @@ -54,14 +53,8 @@ public class SparkSizeBasedClusteringPlanStrategy>, JavaRDD, JavaRDD> { private static final Logger LOG = LogManager.getLogger(SparkSizeBasedClusteringPlanStrategy.class); - public SparkSizeBasedClusteringPlanStrategy(HoodieSparkCopyOnWriteTable table, - HoodieSparkEngineContext engineContext, - HoodieWriteConfig writeConfig) { - super(table, engineContext, writeConfig); - } - - public SparkSizeBasedClusteringPlanStrategy(HoodieSparkMergeOnReadTable table, - HoodieSparkEngineContext engineContext, + public SparkSizeBasedClusteringPlanStrategy(HoodieTable table, + HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { super(table, engineContext, writeConfig); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index df0ad6e2b82da..5bd2da23451aa 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -129,7 +129,8 @@ public HoodieWriteMetadata> performClustering(final Hood */ public abstract HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, final int numOutputGroups, final String instantTime, final Map strategyParams, final Schema schema, - final List fileGroupIdList, final boolean preserveHoodieMetadata); + final List fileGroupIdList, final boolean preserveHoodieMetadata, + final Map extraMetadata); /** * Create {@link BulkInsertPartitioner} based on strategy params. @@ -175,7 +176,8 @@ private CompletableFuture> runClusteringForGroupAsync(Ho List inputFileIds = clusteringGroup.getSlices().stream() .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) .collect(Collectors.toList()); - return performClusteringWithRecordsRDD(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema, inputFileIds, preserveHoodieMetadata); + return performClusteringWithRecordsRDD(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema, inputFileIds, preserveHoodieMetadata, + clusteringGroup.getExtraMetadata()); }); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java new file mode 100644 index 0000000000000..9caea8508cfd8 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkConsistentBucketClusteringExecutionStrategy.java @@ -0,0 +1,79 @@ +/* + * 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.clustering.run.strategy; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.clustering.plan.strategy.SparkConsistentBucketClusteringPlanStrategy; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.ConsistentHashingNode; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.execution.bulkinsert.RDDConsistentBucketPartitioner; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; + +import org.apache.avro.Schema; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Clustering execution strategy specifically for consistent hashing index + */ +public class SparkConsistentBucketClusteringExecutionStrategy> + extends MultipleSparkJobExecutionStrategy { + + private static final Logger LOG = LogManager.getLogger(SparkConsistentBucketClusteringExecutionStrategy.class); + + public SparkConsistentBucketClusteringExecutionStrategy(HoodieTable table, HoodieEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + public HoodieData performClusteringWithRecordsRDD(HoodieData> inputRecords, int numOutputGroups, String instantTime, + Map strategyParams, Schema schema, List fileGroupIdList, + boolean preserveHoodieMetadata, Map extraMetadata) { + + LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); + Properties props = getWriteConfig().getProps(); + // We are calling another action executor - disable auto commit. Strategy is only expected to write data in new files. + props.put(HoodieWriteConfig.AUTO_COMMIT_ENABLE.key(), Boolean.FALSE.toString()); + HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder().withProps(props).build(); + + RDDConsistentBucketPartitioner partitioner = new RDDConsistentBucketPartitioner<>(getHoodieTable(), strategyParams, preserveHoodieMetadata); + try { + List nodes = ConsistentHashingNode.fromJsonString(extraMetadata.get(SparkConsistentBucketClusteringPlanStrategy.METADATA_CHILD_NODE_KEY)); + partitioner.addHashingChildrenNodes(extraMetadata.get(SparkConsistentBucketClusteringPlanStrategy.METADATA_PARTITION_KEY), nodes); + } catch (Exception e) { + LOG.error("Failed to add hashing children nodes", e); + throw new HoodieClusteringException("Failed to add hashing children nodes", e); + } + + return (HoodieData) SparkBulkInsertHelper.newInstance() + .bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, partitioner, true, numOutputGroups); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index b61017c34ce41..b994244c5314f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -61,7 +61,8 @@ public HoodieData performClusteringWithRecordsRDD(HoodieData strategyParams, Schema schema, List fileGroupIdList, - boolean preserveHoodieMetadata) { + boolean preserveHoodieMetadata, + Map extraMetadata) { if (numOutputGroups != 1 || fileGroupIdList.size() != 1) { throw new HoodieClusteringException("Expect only one file group for strategy: " + getClass().getName()); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index 7db63d4169fc1..cee923cf9b96d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -55,7 +55,8 @@ public SparkSortAndSizeExecutionStrategy(HoodieTable table, @Override public HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, final int numOutputGroups, final String instantTime, final Map strategyParams, final Schema schema, - final List fileGroupIdList, final boolean preserveHoodieMetadata) { + final List fileGroupIdList, final boolean preserveHoodieMetadata, + final Map extraMetadata) { LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder() diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java index 3eadba25bbcfb..8263c209218f7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java @@ -18,11 +18,12 @@ package org.apache.hudi.client.clustering.update.strategy; -import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; import java.util.List; @@ -34,9 +35,9 @@ */ public abstract class BaseSparkUpdateStrategy> extends UpdateStrategy>> { - public BaseSparkUpdateStrategy(HoodieSparkEngineContext engineContext, + public BaseSparkUpdateStrategy(HoodieEngineContext engineContext, HoodieTable table, Set fileGroupsInPendingClustering) { - super(engineContext, fileGroupsInPendingClustering); + super(engineContext, table, fileGroupsInPendingClustering); } /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java index 59040629f718e..acb6d82ae189c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java @@ -18,14 +18,14 @@ package org.apache.hudi.client.clustering.update.strategy; -import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.table.HoodieTable; -import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -35,9 +35,9 @@ */ public class SparkAllowUpdateStrategy> extends BaseSparkUpdateStrategy { - public SparkAllowUpdateStrategy(HoodieSparkEngineContext engineContext, - HashSet fileGroupsInPendingClustering) { - super(engineContext, fileGroupsInPendingClustering); + public SparkAllowUpdateStrategy( + HoodieEngineContext engineContext, HoodieTable table, Set fileGroupsInPendingClustering) { + super(engineContext, table, fileGroupsInPendingClustering); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkConsistentBucketDuplicateUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkConsistentBucketDuplicateUpdateStrategy.java new file mode 100644 index 0000000000000..089151cb9551a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkConsistentBucketDuplicateUpdateStrategy.java @@ -0,0 +1,150 @@ +/* + * 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.clustering.update.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.clustering.plan.strategy.SparkConsistentBucketClusteringPlanStrategy; +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.model.ConsistentHashingNode; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; +import org.apache.hudi.common.model.HoodieFileGroupId; +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.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.index.bucket.ConsistentBucketIdentifier; +import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.util.ValidationUtils.checkState; +import static org.apache.hudi.index.HoodieIndexUtils.getTaggedRecord; + +/** + * Update strategy for (consistent hashing) bucket index + * If updates to file groups that are under clustering are identified, then generate + * two same records for each update, routing to both old and new file groups + */ +public class SparkConsistentBucketDuplicateUpdateStrategy> extends UpdateStrategy>> { + + private static final Logger LOG = LogManager.getLogger(SparkConsistentBucketDuplicateUpdateStrategy.class); + + public SparkConsistentBucketDuplicateUpdateStrategy(HoodieEngineContext engineContext, HoodieTable table, Set fileGroupsInPendingClustering) { + super(engineContext, table, fileGroupsInPendingClustering); + } + + @Override + public Pair>, Set> handleUpdate(HoodieData> taggedRecordsRDD) { + if (fileGroupsInPendingClustering.isEmpty()) { + return Pair.of(taggedRecordsRDD, Collections.emptySet()); + } + + HoodieData> filteredRecordsRDD = taggedRecordsRDD.filter(r -> { + checkState(r.getCurrentLocation() != null); + return fileGroupsInPendingClustering.contains(new HoodieFileGroupId(r.getPartitionPath(), r.getCurrentLocation().getFileId())); + }); + + if (filteredRecordsRDD.count() == 0) { + return Pair.of(taggedRecordsRDD, Collections.emptySet()); + } + + // Read all pending/ongoing clustering plans + List> instantPlanPairs = + table.getMetaClient().getActiveTimeline().filterInflightsAndRequested().filter(instant -> instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)).getInstants() + .map(instant -> ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant)) + .flatMap(o -> o.isPresent() ? Stream.of(o.get()) : Stream.empty()) + .collect(Collectors.toList()); + + // Construct child node for each partition & build the bucket identifier + final Set partitions = new HashSet<>(filteredRecordsRDD.map(HoodieRecord::getPartitionPath).distinct().collectAsList()); + Map partitionToHashingMeta = new HashMap<>(); + Map partitionToInstant = new HashMap<>(); + for (Pair pair : instantPlanPairs) { + String instant = pair.getLeft().getTimestamp(); + HoodieClusteringPlan plan = pair.getRight(); + extractHashingMetadataFromClusteringPlan(instant, plan, partitions, partitionToHashingMeta, partitionToInstant); + } + Map partitionToIdentifier = partitionToHashingMeta.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new ConsistentBucketIdentifier(e.getValue()))); + + // Produce records tagged with new record location + List indexKeyFields = Arrays.asList(table.getConfig().getBucketIndexHashField().split(",")); + HoodieData> redirectedRecordsRDD = filteredRecordsRDD.map(r -> { + ConsistentHashingNode node = partitionToIdentifier.get(r.getPartitionPath()).getBucket(r.getKey(), indexKeyFields); + return getTaggedRecord(new HoodieAvroRecord(r.getKey(), r.getData(), r.getOperation()), + Option.ofNullable(new HoodieRecordLocation(partitionToInstant.get(r.getPartitionPath()), FSUtils.createNewFileId(node.getFileIdPrefix(), 0)))); + }); + + // Return combined iterator (the original and records with new location) + return Pair.of(taggedRecordsRDD.union(redirectedRecordsRDD), Collections.emptySet()); + } + + private void extractHashingMetadataFromClusteringPlan(String instant, HoodieClusteringPlan plan, final Set recordPartitions, + Map partitionToHashingMeta, Map partitionToInstant) { + for (HoodieClusteringGroup group : plan.getInputGroups()) { + Map groupMeta = group.getExtraMetadata(); + String p = groupMeta.get(SparkConsistentBucketClusteringPlanStrategy.METADATA_PARTITION_KEY); + checkState(p != null, "Clustering plan does not has partition info, plan: " + plan); + // Skip unrelated clustering group + if (!recordPartitions.contains(p)) { + return; + } + + String preInstant = partitionToInstant.putIfAbsent(p, instant); + checkState(preInstant == null || preInstant.equals(instant), "Find a partition: " + p + " with two clustering instants"); + if (!partitionToHashingMeta.containsKey(p)) { + Option metadataOption = HoodieSparkConsistentBucketIndex.loadMetadata(table, p); + checkState(metadataOption.isPresent(), "Failed to load consistent hashing metadata for partition: " + p); + partitionToHashingMeta.put(p, metadataOption.get()); + } + + try { + String nodeJson = group.getExtraMetadata().get(SparkConsistentBucketClusteringPlanStrategy.METADATA_CHILD_NODE_KEY); + List nodes = ConsistentHashingNode.fromJsonString(nodeJson); + partitionToHashingMeta.get(p).getChildrenNodes().addAll(nodes); + } catch (Exception e) { + LOG.error("Failed to parse child nodes in clustering plan", e); + throw new HoodieException("Failed to parse child nodes in clustering plan, partition: " + p + ", cluster group: " + group, e); + } + } + } + +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java index d09422ee77242..2b05942f00a74 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java @@ -18,19 +18,19 @@ package org.apache.hudi.client.clustering.update.strategy; -import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieClusteringUpdateException; +import org.apache.hudi.table.HoodieTable; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Set; @@ -41,9 +41,8 @@ public class SparkRejectUpdateStrategy> extends BaseSparkUpdateStrategy { private static final Logger LOG = LogManager.getLogger(SparkRejectUpdateStrategy.class); - public SparkRejectUpdateStrategy(HoodieSparkEngineContext engineContext, - HashSet fileGroupsInPendingClustering) { - super(engineContext, fileGroupsInPendingClustering); + public SparkRejectUpdateStrategy(HoodieEngineContext engineContext, HoodieTable table, Set fileGroupsInPendingClustering) { + super(engineContext, table, fileGroupsInPendingClustering); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java index aaa7b5b4bfab4..2851c05204c39 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java @@ -18,8 +18,11 @@ package org.apache.hudi.execution.bulkinsert; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieTable; /** * A factory to generate built-in partitioner to repartition input records into at least @@ -27,6 +30,14 @@ */ public abstract class BulkInsertInternalPartitionerFactory { + public static BulkInsertPartitioner get(HoodieTable table, HoodieWriteConfig config) { + if (config.getIndexType().equals(HoodieIndex.IndexType.BUCKET) + && config.getBucketIndexEngineType().equals(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING)) { + return new RDDConsistentBucketPartitioner(table); + } + return get(config.getBulkInsertSortMode()); + } + public static BulkInsertPartitioner get(BulkInsertSortMode sortMode) { switch (sortMode) { case NONE: diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java new file mode 100644 index 0000000000000..dc5317f7d8a33 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDBucketIndexPartitioner.java @@ -0,0 +1,33 @@ +/* + * 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.execution.bulkinsert; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.table.BulkInsertPartitioner; + +import org.apache.spark.api.java.JavaRDD; + +/** + * Abstract of bucket index bulk_insert partitioner + * TODO implement partitioner for SIMPLE BUCKET INDEX + */ +public abstract class RDDBucketIndexPartitioner + implements BulkInsertPartitioner>> { +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java new file mode 100644 index 0000000000000..825c22264dbfc --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java @@ -0,0 +1,277 @@ +/* + * 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.execution.bulkinsert; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.config.SerializableSchema; +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.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.index.bucket.ConsistentBucketIdentifier; +import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex; +import org.apache.hudi.io.AppendHandleFactory; +import org.apache.hudi.io.SingleFileHandleCreateFactory; +import org.apache.hudi.io.WriteHandleFactory; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaRDD; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import scala.Tuple2; + +import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS; + +/** + * A partitioner for (consistent hashing) bucket index used in bulk_insert + */ +public class RDDConsistentBucketPartitioner extends RDDBucketIndexPartitioner { + + private static final Logger LOG = LogManager.getLogger(RDDConsistentBucketPartitioner.class); + + private final HoodieTable table; + private final List indexKeyFields; + private final Map> hashingChildrenNodes; + private final String[] sortColumnNames; + private final boolean preserveHoodieMetadata; + private final boolean consistentLogicalTimestampEnabled; + + private List doAppend; + private List fileIdPfxList; + + public RDDConsistentBucketPartitioner(HoodieTable table, Map strategyParams, boolean preserveHoodieMetadata) { + this.table = table; + this.indexKeyFields = Arrays.asList(table.getConfig().getBucketIndexHashField().split(",")); + this.hashingChildrenNodes = new HashMap<>(); + this.consistentLogicalTimestampEnabled = table.getConfig().isConsistentLogicalTimestampEnabled(); + this.preserveHoodieMetadata = preserveHoodieMetadata; + + if (strategyParams.containsKey(PLAN_STRATEGY_SORT_COLUMNS.key())) { + sortColumnNames = strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key()).split(","); + } else { + sortColumnNames = null; + } + } + + public RDDConsistentBucketPartitioner(HoodieTable table) { + this(table, Collections.emptyMap(), false); + ValidationUtils.checkArgument(table.getIndex() instanceof HoodieSparkConsistentBucketIndex, + "RDDConsistentBucketPartitioner can only be used together with consistent hashing bucket index"); + ValidationUtils.checkArgument(table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ), + "CoW table with bucket index doesn't support bulk_insert"); + } + + /** + * Repartition the records to conform the bucket index storage layout constraints. + * Specifically, partition the records based on consistent bucket index, which is computed + * using hashing metadata and records' key. + * + * @param records Input Hoodie records + * @param outputSparkPartitions Not used, the actual parallelism is determined by the bucket number + * @return partitioned records, each partition of data corresponds to a bucket (i.e., file group) + */ + @Override + public JavaRDD> repartitionRecords(JavaRDD> records, int outputSparkPartitions) { + Map partitionToIdentifier = initializeBucketIdentifier(records); + Map> partitionToFileIdPfxIdxMap = generateFileIdPfx(partitionToIdentifier); + return doPartition(records, new Partitioner() { + @Override + public int numPartitions() { + return fileIdPfxList.size(); + } + + @Override + public int getPartition(Object key) { + HoodieKey hoodieKey = (HoodieKey) key; + String partition = hoodieKey.getPartitionPath(); + ConsistentHashingNode node = partitionToIdentifier.get(partition).getBucket(hoodieKey, indexKeyFields); + return partitionToFileIdPfxIdxMap.get(partition).get(node.getFileIdPrefix()); + } + }); + } + + @Override + public boolean arePartitionRecordsSorted() { + return (sortColumnNames != null && sortColumnNames.length > 0) + || table.requireSortedRecords() || table.getConfig().getBulkInsertSortMode() != BulkInsertSortMode.NONE; + } + + @Override + public Option getWriteHandleFactory(int idx) { + return doAppend.get(idx) ? Option.of(new AppendHandleFactory()) : + Option.of(new SingleFileHandleCreateFactory(FSUtils.createNewFileId(getFileIdPfx(idx), 0), this.preserveHoodieMetadata)); + } + + @Override + public String getFileIdPfx(int partitionId) { + return fileIdPfxList.get(partitionId); + } + + public void addHashingChildrenNodes(String partition, List nodes) { + ValidationUtils.checkState(nodes.stream().noneMatch(n -> n.getTag() == ConsistentHashingNode.NodeTag.NORMAL), "children nodes should not be tagged as NORMAL"); + hashingChildrenNodes.put(partition, nodes); + } + + /** + * Get (construct) the bucket identifier of the given partition + */ + private ConsistentBucketIdentifier getBucketIdentifier(String partition) { + HoodieSparkConsistentBucketIndex index = (HoodieSparkConsistentBucketIndex) table.getIndex(); + HoodieConsistentHashingMetadata metadata = index.loadOrCreateMetadata(this.table, partition); + if (hashingChildrenNodes.containsKey(partition)) { + metadata.setChildrenNodes(hashingChildrenNodes.get(partition)); + } + return new ConsistentBucketIdentifier(metadata); + } + + /** + * Initialize hashing metadata of input records. The metadata of all related partitions will be loaded, and + * the mapping from partition to its bucket identifier is constructed. + */ + private Map initializeBucketIdentifier(JavaRDD> records) { + return records.map(HoodieRecord::getPartitionPath).distinct().collect().stream() + .collect(Collectors.toMap(p -> p, p -> getBucketIdentifier(p))); + } + + /** + * Initialize fileIdPfx for each data partition. Specifically, the following fields is constructed: + * - fileIdPfxList: the Nth element corresponds to the Nth data partition, indicating its fileIdPfx + * - doAppend: represents if the Nth data partition should use AppendHandler + * - partitionToFileIdPfxIdxMap (return value): (table partition) -> (fileIdPfx -> idx) mapping + * + * @param partitionToIdentifier Mapping from table partition to bucket identifier + */ + private Map> generateFileIdPfx(Map partitionToIdentifier) { + Map> partitionToFileIdPfxIdxMap = new HashMap(partitionToIdentifier.size() * 2); + doAppend = new ArrayList<>(); + fileIdPfxList = new ArrayList<>(); + int count = 0; + for (ConsistentBucketIdentifier identifier : partitionToIdentifier.values()) { + Map fileIdPfxToIdx = new HashMap(); + for (ConsistentHashingNode node : identifier.getNodes()) { + fileIdPfxToIdx.put(node.getFileIdPrefix(), count++); + } + fileIdPfxList.addAll(identifier.getNodes().stream().map(ConsistentHashingNode::getFileIdPrefix).collect(Collectors.toList())); + if (identifier.getMetadata().isFirstCreated()) { + // Create new file group when the hashing metadata is new (i.e., first write to the partition) + doAppend.addAll(Collections.nCopies(identifier.getNodes().size(), false)); + } else { + // Child node requires generating a fresh new base file, rather than log file + doAppend.addAll(identifier.getNodes().stream().map(n -> n.getTag() == ConsistentHashingNode.NodeTag.NORMAL).collect(Collectors.toList())); + } + partitionToFileIdPfxIdxMap.put(identifier.getMetadata().getPartitionPath(), fileIdPfxToIdx); + } + + ValidationUtils.checkState(fileIdPfxList.size() == partitionToIdentifier.values().stream().mapToInt(ConsistentBucketIdentifier::getNumBuckets).sum(), + "Error state after constructing fileId & idx mapping"); + return partitionToFileIdPfxIdxMap; + } + + /** + * Execute partition using the given partitioner. + * If sorting is required, will do it within each data partition: + * - if sortColumnNames is specified, apply sort to the column (the behaviour is the same as `RDDCustomColumnsSortPartitioner` + * - if table requires sort or BulkInsertSortMode is not None, then sort by record key within partition. + * By default, do partition only. + * + * @param records + * @param partitioner a default partition that accepts `HoodieKey` as the partition key + * @return + */ + private JavaRDD> doPartition(JavaRDD> records, Partitioner partitioner) { + if (sortColumnNames != null && sortColumnNames.length > 0) { + return doPartitionAndCustomColumnSort(records, partitioner); + } else if (table.requireSortedRecords() || table.getConfig().getBulkInsertSortMode() != BulkInsertSortMode.NONE) { + return doPartitionAndSortByRecordKey(records, partitioner); + } else { + // By default, do partition only + return records.mapToPair(record -> new Tuple2<>(record.getKey(), record)) + .partitionBy(partitioner).map(Tuple2::_2); + } + } + + /** + * Sort by specified column value. The behaviour is the same as `RDDCustomColumnsSortPartitioner` + * + * @param records + * @param partitioner + * @return + */ + private JavaRDD> doPartitionAndCustomColumnSort(JavaRDD> records, Partitioner partitioner) { + final String[] sortColumns = sortColumnNames; + final SerializableSchema schema = new SerializableSchema(HoodieAvroUtils.addMetadataFields((new Schema.Parser().parse(table.getConfig().getSchema())))); + Comparator> comparator = (Comparator> & Serializable) (t1, t2) -> { + Object obj1 = HoodieAvroUtils.getRecordColumnValues(t1, sortColumns, schema, consistentLogicalTimestampEnabled); + Object obj2 = HoodieAvroUtils.getRecordColumnValues(t2, sortColumns, schema, consistentLogicalTimestampEnabled); + return ((Comparable) obj1).compareTo(obj2); + }; + + return records.mapToPair(record -> new Tuple2<>(record, record)) + .repartitionAndSortWithinPartitions(new Partitioner() { + @Override + public int numPartitions() { + return partitioner.numPartitions(); + } + + @Override + public int getPartition(Object key) { + return partitioner.getPartition(((HoodieRecord) key).getKey()); + } + }, comparator).map(Tuple2::_2); + } + + /** + * Sort by record key within each partition. The behaviour is the same as BulkInsertSortMode.PARTITION_SORT. + * + * @param records + * @param partitioner + * @return + */ + private JavaRDD> doPartitionAndSortByRecordKey(JavaRDD> records, Partitioner partitioner) { + if (table.getConfig().getBulkInsertSortMode() == BulkInsertSortMode.GLOBAL_SORT) { + LOG.warn("Consistent bucket does not support global sort mode, the sort will only be done within each data partition"); + } + + Comparator comparator = (Comparator & Serializable) (t1, t2) -> { + return t1.getRecordKey().compareTo(t2.getRecordKey()); + }; + + return records.mapToPair(record -> new Tuple2<>(record.getKey(), record)) + .repartitionAndSortWithinPartitions(partitioner, comparator) + .map(Tuple2::_2); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java index ca6bf0fc7d990..cbc822f9acf34 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java @@ -18,7 +18,10 @@ package org.apache.hudi.index.bucket; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.clustering.plan.strategy.SparkConsistentBucketClusteringPlanStrategy; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; @@ -27,12 +30,16 @@ 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.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; 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.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.table.HoodieTable; @@ -42,7 +49,9 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.FileNotFoundException; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -50,6 +59,8 @@ import java.util.function.Predicate; import java.util.stream.Collectors; +import scala.Tuple2; + /** * Consistent hashing bucket index implementation, with auto-adjust bucket number. * NOTE: bucket resizing is triggered by clustering. @@ -67,6 +78,63 @@ public HoodieData updateLocation(HoodieData writeStatu HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { + throw new HoodieIndexException("Consistent hashing index does not support update location without the instant parameter"); + } + + /** + * Persist hashing metadata to storage. Only clustering operations will modify the metadata. + * For example, splitting & merging buckets, or just sorting and producing a new bucket. + */ + @Override + public HoodieData updateLocation(HoodieData writeStatuses, + HoodieEngineContext context, + HoodieTable hoodieTable, + String instantTime) + throws HoodieIndexException { + HoodieInstant instant = hoodieTable.getMetaClient().getActiveTimeline().findInstantsAfterOrEquals(instantTime, 1).firstInstant().get(); + ValidationUtils.checkState(instant.getTimestamp().equals(instantTime), "Cannot get the same instant, instantTime: " + instantTime); + if (!instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) { + return writeStatuses; + } + + // Double-check if it is a clustering operation by trying to obtain the clustering plan + Option> instantPlanPair = + ClusteringUtils.getClusteringPlan(hoodieTable.getMetaClient(), HoodieTimeline.getReplaceCommitRequestedInstant(instantTime)); + if (!instantPlanPair.isPresent()) { + return writeStatuses; + } + + HoodieClusteringPlan plan = instantPlanPair.get().getRight(); + HoodieJavaRDD.getJavaRDD(context.parallelize(plan.getInputGroups().stream().map(HoodieClusteringGroup::getExtraMetadata).collect(Collectors.toList()))) + .mapToPair(m -> new Tuple2<>(m.get(SparkConsistentBucketClusteringPlanStrategy.METADATA_PARTITION_KEY), m) + ).groupByKey().foreach((input) -> { + // Process each partition + String partition = input._1(); + List childNodes = new ArrayList<>(); + int seqNo = 0; + for (Map m: input._2()) { + String nodesJson = m.get(SparkConsistentBucketClusteringPlanStrategy.METADATA_CHILD_NODE_KEY); + childNodes.addAll(ConsistentHashingNode.fromJsonString(nodesJson)); + seqNo = Integer.parseInt(m.get(SparkConsistentBucketClusteringPlanStrategy.METADATA_SEQUENCE_NUMBER_KEY)); + } + + Option metadataOption = loadMetadata(hoodieTable, partition); + ValidationUtils.checkState(metadataOption.isPresent(), "Failed to load metadata for partition: " + partition); + HoodieConsistentHashingMetadata meta = metadataOption.get(); + ValidationUtils.checkState(meta.getSeqNo() == seqNo, + "Non serialized update to hashing metadata, old seq: " + meta.getSeqNo() + ", new seq: " + seqNo); + + // Get new metadata and save + meta.setChildrenNodes(childNodes); + List newNodes = (new ConsistentBucketIdentifier(meta)).getNodes().stream() + .map(n -> new ConsistentHashingNode(n.getValue(), n.getFileIdPrefix(), ConsistentHashingNode.NodeTag.NORMAL)) + .collect(Collectors.toList()); + HoodieConsistentHashingMetadata newMeta = new HoodieConsistentHashingMetadata(meta.getVersion(), meta.getPartitionPath(), + instantTime, meta.getNumBuckets(), seqNo + 1, newNodes); + // Overwrite to tolerate re-run of clustering operation + saveMetadata(hoodieTable, newMeta, true); + }); + return writeStatuses; } @@ -94,22 +162,22 @@ protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List metadataOption = loadMetadata(table, partition); + if (metadataOption.isPresent()) { + return metadataOption.get(); } // There is no metadata, so try to create a new one and save it. - metadata = new HoodieConsistentHashingMetadata(partition, numBuckets); + HoodieConsistentHashingMetadata 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; + metadataOption = loadMetadata(table, partition); + ValidationUtils.checkState(metadataOption.isPresent(), "Failed to load or create metadata, partition: " + partition); + return metadataOption.get(); } /** @@ -119,13 +187,10 @@ public HoodieConsistentHashingMetadata loadOrCreateMetadata(HoodieTable table, S * @param partition table partition * @return Consistent hashing metadata or null if it does not exist */ - public static HoodieConsistentHashingMetadata loadMetadata(HoodieTable table, String partition) { + public static Option 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 metaFilePredicate = fileStatus -> { @@ -142,11 +207,13 @@ public static HoodieConsistentHashingMetadata loadMetadata(HoodieTable table, St .max(Comparator.comparing(a -> a.getPath().getName())).orElse(null); if (metaFile == null) { - return null; + return Option.empty(); } byte[] content = FileIOUtils.readAsByteArray(table.getMetaClient().getFs().open(metaFile.getPath())); - return HoodieConsistentHashingMetadata.fromBytes(content); + return Option.of(HoodieConsistentHashingMetadata.fromBytes(content)); + } catch (FileNotFoundException e) { + return Option.empty(); } catch (IOException e) { LOG.error("Error when loading hashing metadata, partition: " + partition, e); throw new HoodieIndexException("Error while loading hashing metadata", e); @@ -192,7 +259,8 @@ public ConsistentBucketIndexLocationMapper(HoodieTable table, List parti } @Override - public Option getRecordLocation(HoodieKey key, String partitionPath) { + public Option getRecordLocation(HoodieKey key) { + String partitionPath = key.getPartitionPath(); ConsistentHashingNode node = partitionToIdentifier.get(partitionPath).getBucket(key, indexKeyFields); if (!StringUtils.isNullOrEmpty(node.getFileIdPrefix())) { /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index f8e4b31ff687e..af72c14efc99c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -43,6 +43,7 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.execution.SparkLazyInsertIterable; +import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.io.HoodieConcatHandle; import org.apache.hudi.io.HoodieMergeHandle; @@ -112,10 +113,18 @@ public BaseSparkCommitActionExecutor(HoodieEngineContext context, } } - private HoodieData> clusteringHandleUpdate(HoodieData> inputRecords, Set fileGroupsInPendingClustering) { + private HoodieData> clusteringHandleUpdate(HoodieData> inputRecords) { context.setJobStatus(this.getClass().getSimpleName(), "Handling updates which are under clustering: " + config.getTableName()); + Set fileGroupsInPendingClustering = + table.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet()); + // Skip processing if there is no inflight clustering + if (fileGroupsInPendingClustering.isEmpty()) { + return inputRecords; + } + UpdateStrategy>> updateStrategy = (UpdateStrategy>>) ReflectionUtils - .loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering); + .loadClass(config.getClusteringUpdatesStrategyClass(), new Class[] {HoodieEngineContext.class, HoodieTable.class, Set.class}, + this.context, table, fileGroupsInPendingClustering); Pair>, Set> recordsAndPendingClusteringFileGroups = updateStrategy.handleUpdate(inputRecords); Set fileGroupsWithUpdatesAndPendingClustering = recordsAndPendingClusteringFileGroups.getRight(); @@ -150,10 +159,13 @@ public HoodieWriteMetadata> execute(HoodieData> inputRecordsWithClusteringUpdate = clusteringHandleUpdate(inputRecords); + WorkloadProfile workloadProfile = null; if (isWorkloadProfileNeeded()) { - context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile: " + config.getTableName()); - workloadProfile = new WorkloadProfile(buildProfile(inputRecords), operationType, table.getIndex().canIndexLogFiles()); + context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile:" + config.getTableName()); + workloadProfile = new WorkloadProfile(buildProfile(inputRecordsWithClusteringUpdate), operationType, table.getIndex().canIndexLogFiles()); LOG.info("Input workload profile :" + workloadProfile); } @@ -163,11 +175,6 @@ public HoodieWriteMetadata> execute(HoodieData fileGroupsInPendingClustering = - table.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet()); - HoodieData> inputRecordsWithClusteringUpdate = fileGroupsInPendingClustering.isEmpty() ? inputRecords : clusteringHandleUpdate(inputRecords, fileGroupsInPendingClustering); - context.setJobStatus(this.getClass().getSimpleName(), "Doing partition and writing data: " + config.getTableName()); HoodieData writeStatuses = mapPartitionsAsRDD(inputRecordsWithClusteringUpdate, partitioner); HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); @@ -175,6 +182,9 @@ public HoodieWriteMetadata> execute(HoodieData, WorkloadStat> buildProfile(HoodieData> inputRecords) { HashMap partitionPathStatMap = new HashMap<>(); WorkloadStat globalStat = new WorkloadStat(); @@ -228,7 +238,7 @@ private HoodieData mapPartitionsAsRDD(HoodieData> d if (table.requireSortedRecords()) { // Partition and sort within each partition as a single step. This is faster than partitioning first and then // applying a sort. - Comparator>> comparator = (Comparator>> & Serializable)(t1, t2) -> { + Comparator>> comparator = (Comparator>> & Serializable) (t1, t2) -> { HoodieKey key1 = t1._1; HoodieKey key2 = t2._1; return key1.getRecordKey().compareTo(key2.getRecordKey()); @@ -254,7 +264,7 @@ protected HoodieData updateIndex(HoodieData writeStatu writeStatuses.persist(config.getString(WRITE_STATUS_STORAGE_LEVEL_VALUE)); Instant indexStartTime = Instant.now(); // Update the index back - HoodieData statuses = table.getIndex().updateLocation(writeStatuses, context, table); + HoodieData statuses = table.getIndex().updateLocation(writeStatuses, context, table, instantTime); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); return statuses; @@ -268,7 +278,7 @@ protected void updateIndexAndCommitIfNeeded(HoodieData writeStatusR @Override protected String getCommitActionType() { - return table.getMetaClient().getCommitActionType(); + return table.getMetaClient().getCommitActionType(); } @Override @@ -344,6 +354,13 @@ public Iterator> handleUpdate(String partitionPath, String fil LOG.info("Empty partition with fileId => " + fileId); return Collections.emptyIterator(); } + + // Pre-check: if the old file does not exist (which may happen in bucket index case), fallback to insert + if (!table.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).isPresent() + && HoodieIndex.IndexType.BUCKET.equals(config.getIndexType())) { + return handleInsert(fileId, recordItr); + } + // these are updates HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); return handleUpdateInternal(upsertHandle, fileId); @@ -379,8 +396,7 @@ protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, } @Override - public Iterator> handleInsert(String idPfx, Iterator> recordItr) - throws Exception { + public Iterator> handleInsert(String idPfx, Iterator> recordItr) { // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records if (!recordItr.hasNext()) { LOG.info("Empty partition"); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java index 1652c35eb63e6..5768520a05fb7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java @@ -74,7 +74,7 @@ public HoodieWriteMetadata> bulkInsert(final HoodieData< executor.getCommitActionType(), instantTime), Option.empty(), config.shouldAllowMultiWriteOnSameInstant()); - BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.orElse(BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode())); + BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.orElse(BulkInsertInternalPartitionerFactory.get(table, config)); // write new files HoodieData writeStatuses = bulkInsert(inputRecords, instantTime, table, config, performDedupe, partitioner, false, @@ -84,6 +84,20 @@ public HoodieWriteMetadata> bulkInsert(final HoodieData< return result; } + /** + * Do bulk insert using WriteHandleFactory from the partitioner (i.e., partitioner.getWriteHandleFactory) + */ + public HoodieData bulkInsert(HoodieData> inputRecords, + String instantTime, + HoodieTable>, HoodieData, HoodieData> table, + HoodieWriteConfig config, + boolean performDedupe, + BulkInsertPartitioner partitioner, + boolean useWriterSchema, + int parallelism) { + return bulkInsert(inputRecords, instantTime, table, config, performDedupe, partitioner, useWriterSchema, parallelism, null); + } + @Override public HoodieData bulkInsert(HoodieData> inputRecords, String instantTime, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java index dd545d5262846..57668aada1ca7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java @@ -44,6 +44,7 @@ public SparkInsertOverwritePartitioner(WorkloadProfile profile, HoodieEngineCont /** * Returns a list of small files in the given partition path. */ + @Override protected List getSmallFiles(String partitionPath) { // for overwrite, we ignore all existing files. So do not consider any file to be smallFiles return Collections.emptyList(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java index 61e6f25af9429..fe467ddbffbeb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java @@ -87,8 +87,7 @@ public Iterator> handleUpdate(String partitionPath, String fil } @Override - public Iterator> handleInsert(String idPfx, Iterator> recordItr) - throws Exception { + public Iterator> handleInsert(String idPfx, Iterator> recordItr) { // If canIndexLogFiles, write inserts to log files else write inserts to base files if (table.getIndex().canIndexLogFiles()) { return new SparkLazyInsertIterable<>(recordItr, true, config, instantTime, table, diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java new file mode 100644 index 0000000000000..810dbc5bc75cc --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java @@ -0,0 +1,190 @@ +/* + * 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.clustering.plan.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.ConsistentHashingNode; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.collection.Triple; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.bucket.ConsistentBucketIdentifier; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieClientTestHarness; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.List; +import java.util.Random; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.hudi.config.HoodieIndexConfig.BUCKET_MERGE_THRESHOLD; +import static org.apache.hudi.config.HoodieIndexConfig.BUCKET_SPLIT_THRESHOLD; + +public class TestSparkConsistentBucketClusteringPlanStrategy extends HoodieClientTestHarness { + + private final Random random = new Random(); + + private void setup() throws IOException { + initPath(); + initSparkContexts(); + initFileSystem(); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); + } + + @AfterEach + public void tearDown() throws IOException { + cleanupResources(); + } + + @Test + public void testBuildSplitClusteringGroup() throws IOException { + setup(); + int maxFileSize = 5120; + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BUCKET) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING) + .withBucketMaxNum(6) + .withBucketNum("4").build()) + .withStorageConfig(HoodieStorageConfig.newBuilder() + .parquetMaxFileSize(maxFileSize).build()) + .build(); + + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + SparkConsistentBucketClusteringPlanStrategy planStrategy = new SparkConsistentBucketClusteringPlanStrategy(hoodieTable, context, config); + + HoodieConsistentHashingMetadata metadata = new HoodieConsistentHashingMetadata("partition", config.getBucketIndexNumBuckets()); + ConsistentBucketIdentifier identifier = new ConsistentBucketIdentifier(metadata); + + int[] fsSize = {maxFileSize * 5, (int) (maxFileSize * BUCKET_SPLIT_THRESHOLD.defaultValue() + 1), maxFileSize, maxFileSize * 5}; + List fileSlices = IntStream.range(0, metadata.getNodes().size()).mapToObj( + i -> createFileSliceWithSize(metadata.getNodes().get(i).getFileIdPrefix(), 1024, fsSize[i] - 1024) + ).collect(Collectors.toList()); + + /** + * 1. Test split candidate selection based on file size + * 2. Test the effectiveness of split slot + */ + Triple res = planStrategy.buildSplitClusteringGroups(identifier, fileSlices, 2); + Assertions.assertEquals(2, res.getMiddle()); + List groups = (List) res.getLeft(); + Assertions.assertEquals(2, groups.size()); + Assertions.assertEquals(fileSlices.get(0).getFileId(), groups.get(0).getSlices().get(0).getFileId()); + Assertions.assertEquals(fileSlices.get(1).getFileId(), groups.get(1).getSlices().get(0).getFileId()); + List fsUntouched = (List) res.getRight(); + Assertions.assertEquals(2, fsUntouched.size()); + Assertions.assertEquals(fileSlices.get(2), fsUntouched.get(0)); + Assertions.assertEquals(fileSlices.get(3), fsUntouched.get(1)); + } + + @Test + public void testBuildMergeClusteringGroup() throws Exception { + setup(); + int maxFileSize = 5120; + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BUCKET) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING) + .withBucketMinNum(4) + .withBucketNum("4").build()) + .withStorageConfig(HoodieStorageConfig.newBuilder() + .parquetMaxFileSize(maxFileSize).build()) + .build(); + + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + SparkConsistentBucketClusteringPlanStrategy planStrategy = new SparkConsistentBucketClusteringPlanStrategy(hoodieTable, context, config); + + HoodieConsistentHashingMetadata metadata = new HoodieConsistentHashingMetadata("partition", 8); + ConsistentBucketIdentifier identifier = new ConsistentBucketIdentifier(metadata); + + int mergeSize = (int) (maxFileSize * BUCKET_MERGE_THRESHOLD.defaultValue()); + int[] fsSize = {0, maxFileSize, mergeSize / 2, mergeSize / 2, mergeSize / 2, maxFileSize, mergeSize / 4, mergeSize / 4}; + List fileSlices = IntStream.range(0, metadata.getNodes().size()).mapToObj( + i -> createFileSliceWithSize(metadata.getNodes().get(i).getFileIdPrefix(), fsSize[i] / 2, fsSize[i] / 2) + ).collect(Collectors.toList()); + + /** + * 1. Test merge candidate selection based on file size + * 2. Test empty file size + * 3. Test merge slot + */ + Triple res = planStrategy.buildMergeClusteringGroup(identifier, fileSlices, 4); + Assertions.assertEquals(3, res.getMiddle()); + List groups = (List) res.getLeft(); + Assertions.assertEquals(2, groups.size()); + + // Check group 0 + Assertions.assertEquals(fileSlices.get(0).getFileId(), groups.get(0).getSlices().get(2).getFileId()); + Assertions.assertEquals(fileSlices.get(7).getFileId(), groups.get(0).getSlices().get(1).getFileId()); + Assertions.assertEquals(fileSlices.get(6).getFileId(), groups.get(0).getSlices().get(0).getFileId()); + Assertions.assertEquals(3, groups.get(0).getSlices().size()); + List nodes = ConsistentHashingNode.fromJsonString(groups.get(0).getExtraMetadata().get(SparkConsistentBucketClusteringPlanStrategy.METADATA_CHILD_NODE_KEY)); + Assertions.assertEquals(3, nodes.size()); + Assertions.assertEquals(ConsistentHashingNode.NodeTag.DELETE, nodes.get(0).getTag()); + Assertions.assertEquals(ConsistentHashingNode.NodeTag.DELETE, nodes.get(1).getTag()); + Assertions.assertEquals(ConsistentHashingNode.NodeTag.REPLACE, nodes.get(2).getTag()); + Assertions.assertEquals(metadata.getNodes().get(0).getValue(), nodes.get(2).getValue()); + + // Check group 1 + Assertions.assertEquals(fileSlices.get(2).getFileId(), groups.get(1).getSlices().get(0).getFileId()); + Assertions.assertEquals(fileSlices.get(3).getFileId(), groups.get(1).getSlices().get(1).getFileId()); + Assertions.assertEquals(2, groups.get(1).getSlices().size()); + nodes = ConsistentHashingNode.fromJsonString(groups.get(1).getExtraMetadata().get(SparkConsistentBucketClusteringPlanStrategy.METADATA_CHILD_NODE_KEY)); + Assertions.assertEquals(2, nodes.size()); + Assertions.assertEquals(ConsistentHashingNode.NodeTag.DELETE, nodes.get(0).getTag()); + Assertions.assertEquals(ConsistentHashingNode.NodeTag.REPLACE, nodes.get(1).getTag()); + Assertions.assertEquals(metadata.getNodes().get(3).getValue(), nodes.get(1).getValue()); + } + + private FileSlice createFileSliceWithSize(String fileIdPfx, long baseFileSize, long totalLogFileSize) { + String fileId = FSUtils.createNewFileId(fileIdPfx, 0); + FileSlice fs = new FileSlice("partition", "001", fileId); + if (baseFileSize > 0) { + HoodieBaseFile f = new HoodieBaseFile(fileId); + f.setFileLen(baseFileSize); + fs.setBaseFile(f); + } + + int numLogFiles = random.nextInt(10) + 1; + if (totalLogFileSize < numLogFiles) { + numLogFiles = (int) totalLogFileSize; + } + long logFileSize = (totalLogFileSize + numLogFiles - 1) / Math.max(numLogFiles, 1); + for (int i = 0; i < numLogFiles; ++i) { + HoodieLogFile f = new HoodieLogFile(String.format(".%s_%s.log.%d", fileId, "12345678", i)); + f.setFileLen(logFileSize); + fs.addLogFile(f); + } + + return fs; + } + +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java index 05617301936eb..d970d593d0fe8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java @@ -23,7 +23,9 @@ 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.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -33,6 +35,7 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.RealtimeFileStatus; import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; @@ -41,11 +44,12 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; 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; @@ -67,6 +71,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS; +import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_TRIGGER_STRATEGY; + /** * Test consistent hashing index */ @@ -107,6 +114,7 @@ private void setUp(boolean populateMetaFields, boolean partitioned) throws Excep .withIndexType(HoodieIndex.IndexType.BUCKET) .withIndexKeyField("_row_key") .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING) + .withBucketNum("8") .build()) .withAutoCommit(false) .build(); @@ -164,31 +172,15 @@ public void testWriteData(boolean populateMetaFields, boolean partitioned) throw List records = dataGen.generateInserts(newCommitTime, totalRecords); JavaRDD writeRecords = jsc.parallelize(records, 2); - metaClient = HoodieTableMetaClient.reload(metaClient); - // Insert totalRecords records - writeClient.startCommitWithTime(newCommitTime); - List 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); + List writeStatues = writeData(writeRecords, newCommitTime, WriteOperationType.UPSERT, true); // 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()); + Assertions.assertEquals(totalRecords, readRecordsNum(dataGen.getPartitionPaths(), populateMetaFields)); // 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); + writeData(writeRecords, "002", WriteOperationType.UPSERT, true); // The number of log file should double after this insertion long numberOfLogFiles = Arrays.stream(dataGen.getPartitionPaths()) .mapToInt(p -> { @@ -197,27 +189,99 @@ public void testWriteData(boolean populateMetaFields, boolean partitioned) throw }).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); + Assertions.assertEquals(totalRecords, readRecordsNum(dataGen.getPartitionPaths(), populateMetaFields)); // 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()); + writeData("003", totalRecords, true); + Assertions.assertEquals(totalRecords * 2, readRecordsNum(dataGen.getPartitionPaths(), populateMetaFields)); } - private List readRecords(String[] partitions, boolean populateMetaFields) { + @ParameterizedTest + @MethodSource("configParams") + public void testWriteDataWithCompaction(boolean populateMetaFields, boolean partitioned) throws Exception { + setUp(populateMetaFields, partitioned); + writeData(HoodieActiveTimeline.createNewInstantTime(), 200, true); + config.setValue(INLINE_COMPACT_NUM_DELTA_COMMITS, "1"); + config.setValue(INLINE_COMPACT_TRIGGER_STRATEGY, CompactionTriggerStrategy.NUM_COMMITS.name()); + String compactionTime = (String) writeClient.scheduleCompaction(Option.empty()).get(); + Assertions.assertEquals(200, readRecordsNum(dataGen.getPartitionPaths(), populateMetaFields)); + writeData(HoodieActiveTimeline.createNewInstantTime(), 200, true); + Assertions.assertEquals(400, readRecordsNum(dataGen.getPartitionPaths(), populateMetaFields)); + HoodieWriteMetadata> compactionMetadata = writeClient.compact(compactionTime); + writeClient.commitCompaction(compactionTime, compactionMetadata.getCommitMetadata().get(), Option.empty()); + Assertions.assertEquals(400, readRecordsNum(dataGen.getPartitionPaths(), populateMetaFields)); + } + + @ParameterizedTest + @MethodSource("configParams") + public void testBulkInsertData(boolean populateMetaFields, boolean partitioned) throws Exception { + setUp(populateMetaFields, partitioned); + String newCommitTime = "001"; + int totalRecords = 20 + random.nextInt(20); + List records = dataGen.generateInserts(newCommitTime, totalRecords); + JavaRDD writeRecords = jsc.parallelize(records, 2); + + // Bulk insert totalRecords records + List writeStatues = writeData(writeRecords, newCommitTime, WriteOperationType.BULK_INSERT, true); + // The number of distinct fileId should be the same as total file group numbers + long numFilesCreated = writeStatues.stream().map(WriteStatus::getFileId).distinct().count(); + Assertions.assertEquals(numFilesCreated, + Arrays.stream(dataGen.getPartitionPaths()).mapToInt(p -> Objects.requireNonNull(listStatus(p, true)).length).sum()); + + // BulkInsert again. + writeData(writeRecords, "002", WriteOperationType.BULK_INSERT,true); + // The total number of file group should be the same, but each file group will have a log file. + Assertions.assertEquals(numFilesCreated, + Arrays.stream(dataGen.getPartitionPaths()).mapToInt(p -> Objects.requireNonNull(listStatus(p, true)).length).sum()); + 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(numFilesCreated, numberOfLogFiles); + // The record number should be doubled if we disable the merge + hadoopConf.set("hoodie.realtime.merge.skip", "true"); + Assertions.assertEquals(totalRecords * 2, readRecordsNum(dataGen.getPartitionPaths(), populateMetaFields)); + } + + private int readRecordsNum(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); + Arrays.stream(partitions).map(p -> Paths.get(basePath, p).toString()).collect(Collectors.toList()), basePath, new JobConf(hadoopConf), true, populateMetaFields).size(); + } + + /** + * Insert `num` records into table given the commitTime + * + * @param commitTime + * @param totalRecords + */ + private List writeData(String commitTime, int totalRecords, boolean doCommit) { + List records = dataGen.generateInserts(commitTime, totalRecords); + JavaRDD writeRecords = jsc.parallelize(records, 2); + return writeData(writeRecords, commitTime, WriteOperationType.UPSERT, doCommit); + } + + private List writeData(JavaRDD records, String commitTime, WriteOperationType op, boolean doCommit) { + metaClient = HoodieTableMetaClient.reload(metaClient); + writeClient.startCommitWithTime(commitTime); + List writeStatues; + switch (op) { + case UPSERT: + writeStatues = writeClient.upsert(records, commitTime).collect(); + break; + case BULK_INSERT: + writeStatues = writeClient.bulkInsert(records, commitTime).collect(); + break; + default: + throw new HoodieException("Unsupported write operations: " + op); + } + org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues); + if (doCommit) { + boolean success = writeClient.commitStats(commitTime, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType()); + Assertions.assertTrue(success); + } + metaClient = HoodieTableMetaClient.reload(metaClient); + return writeStatues; } private FileStatus[] listStatus(String p, boolean realtime) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java new file mode 100644 index 0000000000000..6763cb154f614 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java @@ -0,0 +1,292 @@ +/* + * 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.client.clustering.plan.strategy.SparkConsistentBucketClusteringPlanStrategy; +import org.apache.hudi.client.clustering.run.strategy.SparkConsistentBucketClusteringExecutionStrategy; +import org.apache.hudi.client.clustering.update.strategy.SparkConsistentBucketDuplicateUpdateStrategy; +import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; +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.timeline.HoodieActiveTimeline; +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.HoodieClusteringConfig; +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.exception.HoodieException; +import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; +import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; +import org.apache.hudi.testutils.MetadataMergeWriteStatus; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +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.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + +import static org.apache.hudi.config.HoodieClusteringConfig.DAYBASED_LOOKBACK_PARTITIONS; +import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE; +import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST; + +@Tag("functional") +public class TestSparkConsistentBucketClustering extends HoodieClientTestHarness { + + private HoodieWriteConfig config; + private HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(0); + + public void setup(int maxFileSize) throws IOException { + setup(maxFileSize, Collections.emptyMap()); + } + + public void setup(int maxFileSize, Map options) throws IOException { + initPath(); + initSparkContexts(); + initTestDataGenerator(); + initFileSystem(); + Properties props = getPropertiesForKeyGen(); + props.putAll(options); + props.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, props); + config = getConfigBuilder().withProps(props) + .withAutoCommit(false) + .withIndexConfig(HoodieIndexConfig.newBuilder().fromProperties(props) + .withIndexType(HoodieIndex.IndexType.BUCKET).withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING) + .withBucketNum("8").withBucketMaxNum(14).withBucketMinNum(4).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().parquetMaxFileSize(maxFileSize).build()) + .withClusteringConfig(HoodieClusteringConfig.newBuilder() + .withClusteringPlanStrategyClass(SparkConsistentBucketClusteringPlanStrategy.class.getName()) + .withClusteringExecutionStrategyClass(SparkConsistentBucketClusteringExecutionStrategy.class.getName()) + .withClusteringUpdatesStrategy(SparkConsistentBucketDuplicateUpdateStrategy.class.getName()).build()) + .build(); + + writeClient = getHoodieWriteClient(config); + } + + @AfterEach + public void tearDown() throws IOException { + cleanupResources(); + } + + /** + * Test resizing with bucket number upper bound and lower bound + * + * @throws IOException + */ + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testResizing(boolean isSplit) throws IOException { + final int maxFileSize = isSplit ? 5120 : 128 * 1024 * 1024; + final int targetBucketNum = isSplit ? 14 : 4; + setup(maxFileSize); + writeData(HoodieActiveTimeline.createNewInstantTime(), 2000, true); + String clusteringTime = (String) writeClient.scheduleClustering(Option.empty()).get(); + writeClient.cluster(clusteringTime, true); + + metaClient = HoodieTableMetaClient.reload(metaClient); + final HoodieTable table = HoodieSparkTable.create(config, context, metaClient); + Assertions.assertEquals(2000, readRecords(dataGen.getPartitionPaths()).size()); + + Arrays.stream(dataGen.getPartitionPaths()).forEach(p -> { + HoodieConsistentHashingMetadata metadata = HoodieSparkConsistentBucketIndex.loadMetadata(table, p).get(); + Assertions.assertEquals(targetBucketNum, metadata.getNodes().size()); + + // The file slice has no log files + table.getSliceView().getLatestFileSlices(p).forEach(fs -> { + Assertions.assertTrue(fs.getBaseFile().isPresent()); + Assertions.assertTrue(fs.getLogFiles().count() == 0); + }); + }); + } + + /** + * 1. Test PARTITION_SORT mode, i.e., sort by the record key + * 2. Test custom column sort + * + * @throws IOException + */ + @ParameterizedTest + @ValueSource(strings = {"_row_key", "begin_lat"}) + public void testClusteringColumnSort(String sortColumn) throws IOException { + Map options = new HashMap<>(); + // Record key is handled specially + if (sortColumn.equals("_row_key")) { + options.put(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.PARTITION_SORT.toString()); + } else { + options.put(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key(), sortColumn); + } + setup(128 * 1024 * 1024, options); + + writeData(HoodieActiveTimeline.createNewInstantTime(), 500, true); + writeData(HoodieActiveTimeline.createNewInstantTime(), 500, true); + String clusteringTime = (String) writeClient.scheduleClustering(Option.empty()).get(); + writeClient.cluster(clusteringTime, true); + + // Check the specified column is in sort order + metaClient = HoodieTableMetaClient.reload(metaClient); + List inputPaths = Arrays.stream(dataGen.getPartitionPaths()).map(p -> Paths.get(basePath, p).toString()).collect(Collectors.toList()); + + // Get record reader for file groups and check each file group independently + List readers = HoodieMergeOnReadTestUtils.getRecordReadersUsingInputFormat(hadoopConf, inputPaths, basePath, new JobConf(hadoopConf), true, false); + Schema rawSchema = new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); + Schema.Field field = rawSchema.getField(sortColumn); + Comparator comparator; + if (field.schema().getType() == Schema.Type.DOUBLE) { + comparator = Comparator.comparingDouble(o -> ((DoubleWritable) o).get()); + } else if (field.schema().getType() == Schema.Type.STRING) { + comparator = Comparator.comparing(Object::toString, String::compareTo); + } else { + throw new HoodieException("Cannot get comparator: unsupported data type, " + field.schema().getType()); + } + + for (RecordReader recordReader: readers) { + Object key = recordReader.createKey(); + ArrayWritable writable = (ArrayWritable) recordReader.createValue(); + // The target column in a single file group should be in sorted order + Object lastValue = null; + while (recordReader.next(key, writable)) { + Object rowKey = writable.get()[field.pos()]; + Assertions.assertTrue(lastValue == null || comparator.compare(lastValue, rowKey) <= 0); + lastValue = rowKey; + } + } + } + + /** + * Only one clustering job is allowed on each partition + */ + @Test + public void testConcurrentClustering() throws IOException { + setup(5120); + writeData(HoodieActiveTimeline.createNewInstantTime(), 2000, true); + String clusteringTime = (String) writeClient.scheduleClustering(Option.empty()).get(); + // Schedule again, it should not be scheduled as the previous one are doing clustering to all partitions + Assertions.assertFalse(writeClient.scheduleClustering(Option.empty()).isPresent()); + writeClient.cluster(clusteringTime, true); + + // Schedule two clustering, each working on a single partition + config.setValue(DAYBASED_LOOKBACK_PARTITIONS, "1"); + config.setValue(PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST, "0"); + config.setValue(PLAN_PARTITION_FILTER_MODE, ClusteringPlanPartitionFilterMode.RECENT_DAYS.toString()); + Assertions.assertTrue(writeClient.scheduleClustering(Option.empty()).isPresent()); + config.setValue(DAYBASED_LOOKBACK_PARTITIONS, "1"); + config.setValue(PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST, "1"); + Assertions.assertTrue(writeClient.scheduleClustering(Option.empty()).isPresent()); + } + + /** + * 1. If there is any ongoing writing, cannot schedule clustering + * 2. If the clustering is scheduled, it cannot block incoming new writers + */ + @Test + public void testConcurrentWrite() throws IOException { + setup(5120); + String writeTime = HoodieActiveTimeline.createNewInstantTime(); + List writeStatues = writeData(writeTime, 2000, false); + // Cannot schedule clustering if there is in-flight writer + Assertions.assertFalse(writeClient.scheduleClustering(Option.empty()).isPresent()); + Assertions.assertTrue(writeClient.commitStats(writeTime, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + Option.empty(), metaClient.getCommitActionType())); + metaClient = HoodieTableMetaClient.reload(metaClient); + + // Schedule clustering + String clusteringTime = (String) writeClient.scheduleClustering(Option.empty()).get(); + // Concurrent is not blocked by the clustering + writeData(HoodieActiveTimeline.createNewInstantTime(), 2000, true); + // The records are immediately visible when the writer completes + Assertions.assertEquals(4000, readRecords(dataGen.getPartitionPaths()).size()); + // Clustering finished, check the number of records (there will be file group switch in the background) + writeClient.cluster(clusteringTime, true); + Assertions.assertEquals(4000, readRecords(dataGen.getPartitionPaths()).size()); + } + + private List readRecords(String[] partitions) { + return HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, + Arrays.stream(partitions).map(p -> Paths.get(basePath, p).toString()).collect(Collectors.toList()), + basePath, new JobConf(hadoopConf), true, false); + } + + /** + * Insert `num` records into table given the commitTime + * + * @param commitTime + * @param totalRecords + */ + private List writeData(String commitTime, int totalRecords, boolean doCommit) { + List records = dataGen.generateInserts(commitTime, totalRecords); + JavaRDD writeRecords = jsc.parallelize(records, 2); + metaClient = HoodieTableMetaClient.reload(metaClient); + + writeClient.startCommitWithTime(commitTime); + List writeStatues = writeClient.upsert(writeRecords, commitTime).collect(); + org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues); + if (doCommit) { + Assertions.assertTrue(writeClient.commitStats(commitTime, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + Option.empty(), metaClient.getCommitActionType())); + } + metaClient = HoodieTableMetaClient.reload(metaClient); + return writeStatues; + } + + 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()); + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java index b843546799479..4f32f0ec10617 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java @@ -93,7 +93,7 @@ public void testCreateIndex(IndexType indexType) { .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE).build()).build(); assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieSimpleBucketIndex); - config = clientConfigBuilder.withPath(basePath) + config = HoodieWriteConfig.newBuilder().withPath(basePath) .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.BUCKET) .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING).build()) .build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java index c8b877cecad11..a96ce04077088 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java @@ -108,7 +108,7 @@ public void testTagLocation() throws Exception { HoodieRecord record3 = new HoodieAvroRecord( new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); - HoodieRecord record4 = new HoodieAvroRecord( + HoodieAvroRecord record4 = new HoodieAvroRecord( new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); JavaRDD> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4)); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java index 931714fd2fcf6..8d3d99a4aac6b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java @@ -40,6 +40,8 @@ import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.ArrayList; @@ -51,6 +53,47 @@ */ public class HoodieMergeOnReadTestUtils { + private static final Logger LOG = LogManager.getLogger(HoodieMergeOnReadTestUtils.class); + + public static List getRecordReadersUsingInputFormat(Configuration conf, List inputPaths, + String basePath, JobConf jobConf, boolean realtime, boolean populateMetaField) { + Schema schema = new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); + return getRecordReadersUsingInputFormat(conf, inputPaths, basePath, jobConf, realtime, schema, + HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>(), populateMetaField); + } + + public static List getRecordReadersUsingInputFormat(Configuration conf, List inputPaths, String basePath, JobConf jobConf, boolean realtime, Schema rawSchema, + String rawHiveColumnTypes, boolean projectCols, List projectedColumns, boolean populateMetaFields) { + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build(); + FileInputFormat inputFormat = HoodieInputFormatUtils.getInputFormat(metaClient.getTableConfig().getBaseFileFormat(), realtime, jobConf); + Schema schema; + String hiveColumnTypes; + + if (populateMetaFields) { + schema = HoodieAvroUtils.addMetadataFields(rawSchema); + hiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(rawHiveColumnTypes); + } else { + schema = rawSchema; + hiveColumnTypes = rawHiveColumnTypes; + } + + setPropsForInputFormat(inputFormat, jobConf, schema, hiveColumnTypes, projectCols, projectedColumns, populateMetaFields); + + try { + FileInputFormat.setInputPaths(jobConf, String.join(",", inputPaths)); + InputSplit[] splits = inputFormat.getSplits(jobConf, inputPaths.size()); + + List recordReaders = new ArrayList<>(); + for (InputSplit split : splits) { + recordReaders.add(inputFormat.getRecordReader(split, jobConf, null)); + } + return recordReaders; + } catch (IOException e) { + e.printStackTrace(); + } + return null; + } + public static List getRecordsUsingInputFormat(Configuration conf, List inputPaths, String basePath) { return getRecordsUsingInputFormat(conf, inputPaths, basePath, new JobConf(conf), true); @@ -125,7 +168,7 @@ public static List getRecordsUsingInputFormat(Configuration conf, } } } catch (IOException ie) { - ie.printStackTrace(); + LOG.error(ie); } return records; } diff --git a/hudi-common/src/main/avro/HoodieClusteringGroup.avsc b/hudi-common/src/main/avro/HoodieClusteringGroup.avsc index f2af6b68db873..dbf70faaab01f 100644 --- a/hudi-common/src/main/avro/HoodieClusteringGroup.avsc +++ b/hudi-common/src/main/avro/HoodieClusteringGroup.avsc @@ -44,6 +44,16 @@ "type":["int", "null"], "default": 1 }, + { + /* Used to track extra metadata to facilitate clustering execution + */ + "name":"extraMetadata", + "type":["null", { + "type":"map", + "values":"string" + }], + "default": null + }, { "name":"version", "type":["int", "null"], diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ConsistentHashingNode.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ConsistentHashingNode.java index 262bb963223bb..66eb8deab2af8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/ConsistentHashingNode.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ConsistentHashingNode.java @@ -39,11 +39,17 @@ public class ConsistentHashingNode implements Serializable { private final int value; private final String fileIdPrefix; + private final NodeTag tag; + + public ConsistentHashingNode(int value, String fileIdPrefix) { + this(value, fileIdPrefix, NodeTag.NORMAL); + } @JsonCreator - public ConsistentHashingNode(@JsonProperty("value") int value, @JsonProperty("fileIdPrefix") String fileIdPrefix) { + public ConsistentHashingNode(@JsonProperty("value") int value, @JsonProperty("fileIdPrefix") String fileIdPrefix, @JsonProperty("tag") NodeTag tag) { this.value = value; this.fileIdPrefix = fileIdPrefix; + this.tag = tag; } public static String toJsonString(List nodes) throws IOException { @@ -67,6 +73,10 @@ public String getFileIdPrefix() { return fileIdPrefix; } + public NodeTag getTag() { + return tag; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder("ConsistentHashingNode{"); @@ -75,4 +85,19 @@ public String toString() { sb.append('}'); return sb.toString(); } + + public enum NodeTag { + /** + * Standard node. + */ + NORMAL, + /** + * Node that is new, or is used to replace a normal node in the hash ring. Used in bucket split. + */ + REPLACE, + /** + * To mark the deletion of a node in the hash ring. Used in bucket merge. + */ + DELETE + } } \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java index 0fc580db0b657..5d808e48ff94f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java @@ -107,6 +107,11 @@ public Option getLatestLogFile() { return Option.fromJavaOptional(logFiles.stream().findFirst()); } + public long getTotalFileSize() { + return getBaseFile().map(HoodieBaseFile::getFileSize).orElse(0L) + + getLogFiles().mapToLong(HoodieLogFile::getFileSize).sum(); + } + /** * Returns true if there is no data file and no log files. Happens as part of pending compaction * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java index 46f115262745f..e0f51fcbaa37c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.util.JsonUtils; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.log4j.LogManager; @@ -31,6 +32,7 @@ import java.io.IOException; import java.io.Serializable; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -54,6 +56,13 @@ public class HoodieConsistentHashingMetadata implements Serializable { private final int numBuckets; private final int seqNo; private final List nodes; + @JsonIgnore + protected List childrenNodes = new ArrayList<>(); + /** + * Used to indicate if the metadata is newly created, rather than read from the storage. + */ + @JsonIgnore + private boolean firstCreated; @JsonCreator public HoodieConsistentHashingMetadata(@JsonProperty("version") short version, @JsonProperty("partitionPath") String partitionPath, @@ -65,13 +74,16 @@ public HoodieConsistentHashingMetadata(@JsonProperty("version") short version, @ this.numBuckets = numBuckets; this.seqNo = seqNo; this.nodes = nodes; + this.firstCreated = false; } /** + * Only used for creating new hashing metadata. * Construct default metadata with all bucket's file group uuid initialized */ public HoodieConsistentHashingMetadata(String partitionPath, int numBuckets) { this((short) 0, partitionPath, HoodieTimeline.INIT_INSTANT_TS, numBuckets, 0, constructDefaultHashingNodes(numBuckets)); + this.firstCreated = true; } private static List constructDefaultHashingNodes(int numBuckets) { @@ -100,10 +112,22 @@ public int getSeqNo() { return seqNo; } + public boolean isFirstCreated() { + return firstCreated; + } + public List getNodes() { return nodes; } + public List getChildrenNodes() { + return childrenNodes; + } + + public void setChildrenNodes(List childrenNodes) { + this.childrenNodes = childrenNodes; + } + public String getFilename() { return instant + HASHING_METADATA_FILE_SUFFIX; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java index f2d2d7e29dcb1..de18c4fbaba6c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java @@ -134,4 +134,13 @@ private static TemporalAccessor convertDateToTemporalAccessor(Date d) { public static void setCommitTimeZone(HoodieTimelineTimeZone commitTimeZone) { HoodieInstantTimeGenerator.commitTimeZone = commitTimeZone; } + + public static boolean isValidInstantTime(String instantTime) { + try { + Long.parseLong(instantTime); + return true; + } catch (NumberFormatException e) { + return false; + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java index 88a0f70cd831c..feebf6c48224a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java @@ -50,9 +50,7 @@ public long sizeEstimate(HoodieRecord hoodieRecord) { // note the sizes and differences. A correct estimation in such cases is handled in /** {@link ExternalSpillableMap} **/ long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord); - if (LOG.isDebugEnabled()) { - LOG.debug("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema); - } + LOG.debug("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema); return sizeOfRecord; } }