1
0

[HUDI-1075] Implement simple clustering strategies to create ClusteringPlan and to run the plan

This commit is contained in:
Satish Kotha
2020-11-07 17:08:55 -08:00
parent e4e2fbc3bb
commit 6dc03b65bf
47 changed files with 2383 additions and 128 deletions

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.client;
import com.codahale.metrics.Timer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
@@ -38,9 +39,12 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
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.CommitUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
@@ -88,6 +92,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
protected transient Timer.Context writeTimer = null;
protected transient Timer.Context compactionTimer;
protected transient Timer.Context clusteringTimer;
private transient WriteOperationType operationType;
private transient HoodieWriteCommitCallback commitCallback;
@@ -389,6 +394,16 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
} else {
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false");
}
// Do an inline clustering if enabled
if (config.isInlineClustering()) {
runAnyPendingClustering(table);
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true");
inlineCluster(extraMetadata);
} else {
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false");
}
// We cannot have unbounded commit files. Archive commits if we have to archive
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
archiveLog.archiveIfRequired(context);
@@ -406,6 +421,16 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
});
}
protected void runAnyPendingClustering(HoodieTable<T, I, K, O> table) {
table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().forEach(instant -> {
Option<Pair<HoodieInstant, HoodieClusteringPlan>> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant);
if (instantPlan.isPresent()) {
LOG.info("Running pending clustering at instant " + instantPlan.get().getLeft());
cluster(instant.getTimestamp(), true);
}
});
}
/**
* Handle auto clean during commit.
*
@@ -725,6 +750,54 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
return compactionInstantTimeOpt;
}
/**
* Schedules a new clustering instant.
*
* @param extraMetadata Extra Metadata to be stored
*/
public Option<String> scheduleClustering(Option<Map<String, String>> extraMetadata) throws HoodieIOException {
String instantTime = HoodieActiveTimeline.createNewInstantTime();
return scheduleClusteringAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty();
}
/**
* Schedules a new clustering instant with passed-in instant time.
*
* @param instantTime clustering Instant Time
* @param extraMetadata Extra Metadata to be stored
*/
public boolean scheduleClusteringAtInstant(String instantTime, Option<Map<String, String>> extraMetadata) throws HoodieIOException {
LOG.info("Scheduling clustering at instant time :" + instantTime);
Option<HoodieClusteringPlan> plan = createTable(config, hadoopConf)
.scheduleClustering(context, instantTime, extraMetadata);
return plan.isPresent();
}
/**
* Ensures clustering instant is in expected state and performs clustering for the plan stored in metadata.
*
* @param clusteringInstant Clustering Instant Time
* @return Collection of Write Status
*/
public abstract HoodieWriteMetadata<O> cluster(String clusteringInstant, boolean shouldComplete);
/**
* Executes a clustering plan on a table, serially before or after an insert/upsert action.
*/
protected Option<String> inlineCluster(Option<Map<String, String>> extraMetadata) {
Option<String> clusteringInstantOpt = scheduleClustering(extraMetadata);
clusteringInstantOpt.ifPresent(clusteringInstant -> {
// inline cluster should auto commit as the user is never given control
cluster(clusteringInstant, true);
});
return clusteringInstantOpt;
}
protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable<T, I, K, O> table) {
table.rollback(context, HoodieActiveTimeline.createNewInstantTime(), inflightInstant, false);
table.getActiveTimeline().revertReplaceCommitInflightToRequested(inflightInstant);
}
/**
* Finalize Write operation.
*

View File

@@ -0,0 +1,60 @@
/*
* 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.utils;
import org.apache.hudi.common.util.ValidationUtils;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
/**
* Provides iterator interface over List of iterators. Consumes all records from first iterator element
* before moving to next iterator in the list. That is concatenate elements across multiple iterators.
*
* @param <T>
*/
public class ConcatenatingIterator<T> implements Iterator<T> {
private final Queue<Iterator<T>> allIterators;
public ConcatenatingIterator(List<Iterator<T>> iterators) {
allIterators = new LinkedList<>(iterators);
}
@Override
public boolean hasNext() {
while (!allIterators.isEmpty()) {
if (allIterators.peek().hasNext()) {
return true;
}
// iterator at current head is done. move ahead
allIterators.poll();
}
return false;
}
@Override
public T next() {
ValidationUtils.checkArgument(hasNext(), "No more elements left");
return allIterators.peek().next();
}
}

View File

@@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.client.utils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieLogFile;
import java.util.List;
import java.util.Map;
/**
* A utility class for calculating metrics related to FileSlice.
*/
public class FileSliceMetricUtils {
public static final String TOTAL_IO_READ_MB = "TOTAL_IO_READ_MB";
public static final String TOTAL_IO_WRITE_MB = "TOTAL_IO_WRITE_MB";
public static final String TOTAL_IO_MB = "TOTAL_IO_MB";
public static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILES_SIZE";
public static final String TOTAL_LOG_FILES = "TOTAL_LOG_FILES";
public static void addFileSliceCommonMetrics(List<FileSlice> fileSlices, Map<String, Double> metrics, long defaultBaseFileSize) {
int numLogFiles = 0;
long totalLogFileSize = 0;
long totalIORead = 0;
long totalIOWrite = 0;
long totalIO = 0;
for (FileSlice slice : fileSlices) {
numLogFiles += slice.getLogFiles().count();
// Total size of all the log files
totalLogFileSize += slice.getLogFiles().map(HoodieLogFile::getFileSize).filter(size -> size >= 0)
.reduce(Long::sum).orElse(0L);
long baseFileSize = slice.getBaseFile().isPresent() ? slice.getBaseFile().get().getFileSize() : 0L;
// Total read will be the base file + all the log files
totalIORead = FSUtils.getSizeInMB(baseFileSize + totalLogFileSize);
// Total write will be similar to the size of the base file
totalIOWrite = FSUtils.getSizeInMB(baseFileSize > 0 ? baseFileSize : defaultBaseFileSize);
// Total IO will the the IO for read + write
totalIO = totalIORead + totalIOWrite;
}
metrics.put(TOTAL_IO_READ_MB, (double) totalIORead);
metrics.put(TOTAL_IO_WRITE_MB, (double) totalIOWrite);
metrics.put(TOTAL_IO_MB, (double) totalIO);
metrics.put(TOTAL_LOG_FILE_SIZE, (double) totalLogFileSize);
metrics.put(TOTAL_LOG_FILES, (double) numLogFiles);
}
}

View File

@@ -0,0 +1,176 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.config;
import org.apache.hudi.common.config.DefaultHoodieConfig;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
/**
* Clustering specific configs.
*/
public class HoodieClusteringConfig extends DefaultHoodieConfig {
// Config to provide a strategy class to create ClusteringPlan. Class has to be subclass of ClusteringPlanStrategy
public static final String CLUSTERING_PLAN_STRATEGY_CLASS = "hoodie.clustering.plan.strategy.class";
public static final String DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS =
"org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy";
// Config to provide a strategy class to execute a ClusteringPlan. Class has to be subclass of RunClusteringStrategy
public static final String CLUSTERING_EXECUTION_STRATEGY_CLASS = "hoodie.clustering.execution.strategy.class";
public static final String DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS =
"org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy";
// Turn on inline clustering - clustering will be run after write operation is complete.
public static final String INLINE_CLUSTERING_PROP = "hoodie.clustering.inline";
private static final String DEFAULT_INLINE_CLUSTERING = "false";
// Config to control frequency of clustering
public static final String INLINE_CLUSTERING_MAX_COMMIT_PROP = "hoodie.clustering.inline.max.commits";
private static final String DEFAULT_INLINE_CLUSTERING_NUM_COMMITS = "4";
// Any strategy specific params can be saved with this prefix
public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy.";
// Number of partitions to list to create ClusteringPlan.
public static final String CLUSTERING_TARGET_PARTITIONS = CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.lookback.partitions";
public static final String DEFAULT_CLUSTERING_TARGET_PARTITIONS = String.valueOf(2);
// Files smaller than the size specified here are candidates for clustering.
public static final String CLUSTERING_PLAN_SMALL_FILE_LIMIT = CLUSTERING_STRATEGY_PARAM_PREFIX + "small.file.limit";
public static final String DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT = String.valueOf(600 * 1024 * 1024L); // 600MB
// Each clustering operation can create multiple groups. Total amount of data processed by clustering operation
// is defined by below two properties (CLUSTERING_MAX_BYTES_PER_GROUP * CLUSTERING_MAX_NUM_GROUPS).
// Max amount of data to be included in one group
public static final String CLUSTERING_MAX_BYTES_PER_GROUP = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group";
public static final String DEFAULT_CLUSTERING_MAX_GROUP_SIZE = String.valueOf(2 * 1024 * 1024 * 1024L);
// Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism.
public static final String CLUSTERING_MAX_NUM_GROUPS = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.num.groups";
public static final String DEFAULT_CLUSTERING_MAX_NUM_GROUPS = "30";
// Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups.
public static final String CLUSTERING_TARGET_FILE_MAX_BYTES = CLUSTERING_STRATEGY_PARAM_PREFIX + "target.file.max.bytes";
public static final String DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES = String.valueOf(1 * 1024 * 1024 * 1024L); // 1GB
// constants related to clustering that may be used by more than 1 strategy.
public static final String CLUSTERING_SORT_COLUMNS_PROPERTY = HoodieClusteringConfig.CLUSTERING_STRATEGY_PARAM_PREFIX + "sort.columns";
public HoodieClusteringConfig(Properties props) {
super(props);
}
public static Builder newBuilder() {
return new Builder();
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
try (FileReader reader = new FileReader(propertiesFile)) {
this.props.load(reader);
return this;
}
}
public Builder withClusteringPlanStrategyClass(String clusteringStrategyClass) {
props.setProperty(CLUSTERING_PLAN_STRATEGY_CLASS, clusteringStrategyClass);
return this;
}
public Builder withClusteringExecutionStrategyClass(String runClusteringStrategyClass) {
props.setProperty(CLUSTERING_EXECUTION_STRATEGY_CLASS, runClusteringStrategyClass);
return this;
}
public Builder withClusteringTargetPartitions(int clusteringTargetPartitions) {
props.setProperty(CLUSTERING_TARGET_PARTITIONS, String.valueOf(clusteringTargetPartitions));
return this;
}
public Builder withClusteringPlanSmallFileLimit(long clusteringSmallFileLimit) {
props.setProperty(CLUSTERING_PLAN_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit));
return this;
}
public Builder withClusteringSortColumns(String sortColumns) {
props.setProperty(CLUSTERING_SORT_COLUMNS_PROPERTY, sortColumns);
return this;
}
public Builder withClusteringMaxBytesInGroup(long clusteringMaxGroupSize) {
props.setProperty(CLUSTERING_MAX_BYTES_PER_GROUP, String.valueOf(clusteringMaxGroupSize));
return this;
}
public Builder withClusteringMaxNumGroups(int maxNumGroups) {
props.setProperty(CLUSTERING_MAX_NUM_GROUPS, String.valueOf(maxNumGroups));
return this;
}
public Builder withClusteringTargetFileMaxBytes(long targetFileSize) {
props.setProperty(CLUSTERING_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize));
return this;
}
public Builder withInlineClustering(Boolean inlineCompaction) {
props.setProperty(INLINE_CLUSTERING_PROP, String.valueOf(inlineCompaction));
return this;
}
public Builder withInlineClusteringNumCommits(int numCommits) {
props.setProperty(INLINE_CLUSTERING_MAX_COMMIT_PROP, String.valueOf(numCommits));
return this;
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public HoodieClusteringConfig build() {
HoodieClusteringConfig config = new HoodieClusteringConfig(props);
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_STRATEGY_CLASS),
CLUSTERING_PLAN_STRATEGY_CLASS, DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS);
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_EXECUTION_STRATEGY_CLASS),
CLUSTERING_EXECUTION_STRATEGY_CLASS, DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS);
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_BYTES_PER_GROUP), CLUSTERING_MAX_BYTES_PER_GROUP,
DEFAULT_CLUSTERING_MAX_GROUP_SIZE);
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_NUM_GROUPS), CLUSTERING_MAX_NUM_GROUPS,
DEFAULT_CLUSTERING_MAX_NUM_GROUPS);
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_FILE_MAX_BYTES), CLUSTERING_TARGET_FILE_MAX_BYTES,
DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_PROP), INLINE_CLUSTERING_PROP,
DEFAULT_INLINE_CLUSTERING);
setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_MAX_COMMIT_PROP), INLINE_CLUSTERING_MAX_COMMIT_PROP,
DEFAULT_INLINE_CLUSTERING_NUM_COMMITS);
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_PARTITIONS), CLUSTERING_TARGET_PARTITIONS,
DEFAULT_CLUSTERING_TARGET_PARTITIONS);
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_SMALL_FILE_LIMIT), CLUSTERING_PLAN_SMALL_FILE_LIMIT,
DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT);
return config;
}
}
}

View File

@@ -390,6 +390,14 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
}
public boolean isInlineClustering() {
return Boolean.parseBoolean(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_PROP));
}
public int getInlineClusterMaxCommits() {
return Integer.parseInt(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP));
}
public String getPayloadClass() {
return props.getProperty(HoodieCompactionConfig.PAYLOAD_CLASS_PROP);
}
@@ -406,6 +414,41 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLED));
}
/**
* Clustering properties.
*/
public String getClusteringPlanStrategyClass() {
return props.getProperty(HoodieClusteringConfig.CLUSTERING_PLAN_STRATEGY_CLASS);
}
public String getClusteringExecutionStrategyClass() {
return props.getProperty(HoodieClusteringConfig.CLUSTERING_EXECUTION_STRATEGY_CLASS);
}
public long getClusteringMaxBytesInGroup() {
return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_BYTES_PER_GROUP));
}
public long getClusteringSmallFileLimit() {
return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_PLAN_SMALL_FILE_LIMIT));
}
public int getClusteringMaxNumGroups() {
return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_NUM_GROUPS));
}
public long getClusteringTargetFileMaxBytes() {
return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_FILE_MAX_BYTES));
}
public int getTargetPartitionsForClustering() {
return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_PARTITIONS));
}
public String getClusteringSortColumns() {
return props.getProperty(HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY);
}
/**
* index properties.
*/
@@ -804,6 +847,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private boolean isIndexConfigSet = false;
private boolean isStorageConfigSet = false;
private boolean isCompactionConfigSet = false;
private boolean isClusteringConfigSet = false;
private boolean isMetricsConfigSet = false;
private boolean isBootstrapConfigSet = false;
private boolean isMemoryConfigSet = false;
@@ -933,6 +977,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return this;
}
public Builder withClusteringConfig(HoodieClusteringConfig clusteringConfig) {
props.putAll(clusteringConfig.getProps());
isClusteringConfigSet = true;
return this;
}
public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
props.putAll(metricsConfig.getProps());
isMetricsConfigSet = true;
@@ -1087,6 +1137,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
setDefaultOnCondition(props, !isStorageConfigSet, HoodieStorageConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isCompactionConfigSet,
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isClusteringConfigSet,
HoodieClusteringConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isMetricsConfigSet, HoodieMetricsConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isBootstrapConfigSet,
HoodieBootstrapConfig.newBuilder().fromProperties(props).build());

View File

@@ -0,0 +1,31 @@
/*
* 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.exception;
public class HoodieClusteringException extends HoodieException {
public HoodieClusteringException(String msg) {
super(msg);
}
public HoodieClusteringException(String msg, Throwable e) {
super(msg, e);
}
}

View File

@@ -37,6 +37,7 @@ public class HoodieMetrics {
public String cleanTimerName = null;
public String commitTimerName = null;
public String deltaCommitTimerName = null;
public String replaceCommitTimerName = null;
public String finalizeTimerName = null;
public String compactionTimerName = null;
public String indexTimerName = null;
@@ -48,6 +49,7 @@ public class HoodieMetrics {
private Timer deltaCommitTimer = null;
private Timer finalizeTimer = null;
private Timer compactionTimer = null;
private Timer clusteringTimer = null;
private Timer indexTimer = null;
public HoodieMetrics(HoodieWriteConfig config, String tableName) {
@@ -59,6 +61,7 @@ public class HoodieMetrics {
this.cleanTimerName = getMetricsName("timer", HoodieTimeline.CLEAN_ACTION);
this.commitTimerName = getMetricsName("timer", HoodieTimeline.COMMIT_ACTION);
this.deltaCommitTimerName = getMetricsName("timer", HoodieTimeline.DELTA_COMMIT_ACTION);
this.replaceCommitTimerName = getMetricsName("timer", HoodieTimeline.REPLACE_COMMIT_ACTION);
this.finalizeTimerName = getMetricsName("timer", "finalize");
this.compactionTimerName = getMetricsName("timer", HoodieTimeline.COMPACTION_ACTION);
this.indexTimerName = getMetricsName("timer", "index");
@@ -83,6 +86,13 @@ public class HoodieMetrics {
return compactionTimer == null ? null : compactionTimer.time();
}
public Timer.Context getClusteringCtx() {
if (config.isMetricsOn() && clusteringTimer == null) {
clusteringTimer = createTimer(replaceCommitTimerName);
}
return clusteringTimer == null ? null : clusteringTimer.time();
}
public Timer.Context getCleanCtx() {
if (config.isMetricsOn() && cleanTimer == null) {
cleanTimer = createTimer(cleanTimerName);

View File

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
@@ -337,6 +338,27 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
public abstract HoodieWriteMetadata<O> compact(HoodieEngineContext context,
String compactionInstantTime);
/**
* Schedule clustering for the instant time.
*
* @param context HoodieEngineContext
* @param instantTime Instant Time for scheduling clustering
* @param extraMetadata additional metadata to write into plan
* @return HoodieClusteringPlan, if there is enough data for clustering.
*/
public abstract Option<HoodieClusteringPlan> scheduleClustering(HoodieEngineContext context,
String instantTime,
Option<Map<String, String>> extraMetadata);
/**
* Execute Clustering on the table. Clustering re-arranges the data so that it is optimized for data access.
*
* @param context HoodieEngineContext
* @param clusteringInstantTime Instant Time
*/
public abstract HoodieWriteMetadata<O> cluster(HoodieEngineContext context, String clusteringInstantTime);
/**
* Perform metadata/full bootstrap of a Hudi table.
* @param context HoodieEngineContext

View File

@@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.cluster;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.BaseActionExecutor;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
public abstract class BaseClusteringPlanActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, Option<HoodieClusteringPlan>> {
private final Option<Map<String, String>> extraMetadata;
public BaseClusteringPlanActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, I, K, O> table,
String instantTime,
Option<Map<String, String>> extraMetadata) {
super(context, config, table, instantTime);
this.extraMetadata = extraMetadata;
}
protected abstract Option<HoodieClusteringPlan> createClusteringPlan();
@Override
public Option<HoodieClusteringPlan> execute() {
Option<HoodieClusteringPlan> planOption = createClusteringPlan();
if (planOption.isPresent()) {
HoodieInstant clusteringInstant =
new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime);
try {
HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder()
.setOperationType(WriteOperationType.CLUSTER.name())
.setExtraMetadata(extraMetadata.orElse(Collections.emptyMap()))
.setClusteringPlan(planOption.get())
.build();
table.getActiveTimeline().saveToPendingReplaceCommit(clusteringInstant,
TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata));
} catch (IOException ioe) {
throw new HoodieIOException("Exception scheduling clustering", ioe);
}
}
return planOption;
}
}

View File

@@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.cluster.strategy;
import org.apache.avro.Schema;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.Serializable;
import java.util.Map;
/**
* Pluggable implementation for writing data into new file groups based on ClusteringPlan.
*/
public abstract class ClusteringExecutionStrategy<T extends HoodieRecordPayload,I,K,O> implements Serializable {
private static final Logger LOG = LogManager.getLogger(ClusteringExecutionStrategy.class);
private final HoodieTable<T,I,K,O> hoodieTable;
private final HoodieEngineContext engineContext;
private final HoodieWriteConfig writeConfig;
public ClusteringExecutionStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) {
this.writeConfig = writeConfig;
this.hoodieTable = table;
this.engineContext = engineContext;
}
/**
* Execute clustering to write inputRecords into new files as defined by rules in strategy parameters. The number of new
* file groups created is bounded by numOutputGroups.
* Note that commit is not done as part of strategy. commit is callers responsibility.
*/
public abstract O performClustering(final I inputRecords, final int numOutputGroups, final String instantTime,
final Map<String, String> strategyParams, final Schema schema);
protected HoodieTable<T,I,K, O> getHoodieTable() {
return this.hoodieTable;
}
protected HoodieEngineContext getEngineContext() {
return this.engineContext;
}
protected HoodieWriteConfig getWriteConfig() {
return this.writeConfig;
}
}

View File

@@ -0,0 +1,142 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.cluster.strategy;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.avro.model.HoodieSliceInfo;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.utils.FileSliceMetricUtils;
import org.apache.hudi.common.model.BaseFile;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.view.SyncableFileSystemView;
import org.apache.hudi.common.util.Option;
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.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.Serializable;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Pluggable implementation for scheduling clustering and creating ClusteringPlan.
*/
public abstract class ClusteringPlanStrategy<T extends HoodieRecordPayload,I,K,O> implements Serializable {
private static final Logger LOG = LogManager.getLogger(ClusteringPlanStrategy.class);
public static final int CLUSTERING_PLAN_VERSION_1 = 1;
private final HoodieTable<T,I,K,O> hoodieTable;
private final transient HoodieEngineContext engineContext;
private final HoodieWriteConfig writeConfig;
public ClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) {
this.writeConfig = writeConfig;
this.hoodieTable = table;
this.engineContext = engineContext;
}
/**
* Generate metadata for grouping eligible files and create a plan. Note that data is not moved around
* as part of this step.
*
* If there is no data available to cluster, return None.
*/
public abstract Option<HoodieClusteringPlan> generateClusteringPlan();
/**
* Return file slices eligible for clustering. FileIds in pending clustering/compaction are not eligible for clustering.
*/
protected Stream<FileSlice> getFileSlicesEligibleForClustering(String partition) {
SyncableFileSystemView fileSystemView = (SyncableFileSystemView) getHoodieTable().getSliceView();
Set<HoodieFileGroupId> fgIdsInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
.map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
.collect(Collectors.toSet());
fgIdsInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet()));
return hoodieTable.getSliceView().getLatestFileSlices(partition)
// file ids already in clustering are not eligible
.filter(slice -> !fgIdsInPendingCompactionAndClustering.contains(slice.getFileGroupId()));
}
/**
* Get parameters specific to strategy. These parameters are passed from 'schedule clustering' step to
* 'execute clustering' step. 'execute clustering' step is typically async. So these params help with passing any required
* context from schedule to run step.
*/
protected abstract Map<String, String> getStrategyParams();
/**
* Returns any specific parameters to be stored as part of clustering metadata.
*/
protected Map<String, String> getExtraMetadata() {
return Collections.emptyMap();
}
/**
* Version to support future changes for plan.
*/
protected int getPlanVersion() {
return CLUSTERING_PLAN_VERSION_1;
}
/**
* Transform {@link FileSlice} to {@link HoodieSliceInfo}.
*/
protected static List<HoodieSliceInfo> getFileSliceInfo(List<FileSlice> slices) {
return slices.stream().map(slice -> new HoodieSliceInfo().newBuilder()
.setPartitionPath(slice.getPartitionPath())
.setFileId(slice.getFileId())
.setDataFilePath(slice.getBaseFile().map(BaseFile::getPath).orElse(StringUtils.EMPTY_STRING))
.setDeltaFilePaths(slice.getLogFiles().map(f -> f.getPath().toString()).collect(Collectors.toList()))
.setBootstrapFilePath(slice.getBaseFile().map(bf -> bf.getBootstrapBaseFile().map(bbf -> bbf.getPath()).orElse(StringUtils.EMPTY_STRING)).orElse(StringUtils.EMPTY_STRING))
.build()).collect(Collectors.toList());
}
/**
* Generate metrics for the data to be clustered.
*/
protected Map<String, Double> buildMetrics(List<FileSlice> fileSlices) {
Map<String, Double> metrics = new HashMap<>();
FileSliceMetricUtils.addFileSliceCommonMetrics(fileSlices, metrics, getWriteConfig().getParquetMaxFileSize());
return metrics;
}
protected HoodieTable<T,I,K, O> getHoodieTable() {
return this.hoodieTable;
}
protected HoodieEngineContext getEngineContext() {
return this.engineContext;
}
protected HoodieWriteConfig getWriteConfig() {
return this.writeConfig;
}
}

View File

@@ -0,0 +1,108 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.cluster.strategy;
import org.apache.hudi.avro.model.HoodieClusteringGroup;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.avro.model.HoodieClusteringStrategy;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Scheduling strategy with restriction that clustering groups can only contain files from same partition.
*/
public abstract class PartitionAwareClusteringPlanStrategy<T extends HoodieRecordPayload,I,K,O> extends ClusteringPlanStrategy<T,I,K,O> {
private static final Logger LOG = LogManager.getLogger(PartitionAwareClusteringPlanStrategy.class);
public PartitionAwareClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) {
super(table, engineContext, writeConfig);
}
/**
* Create Clustering group based on files eligible for clustering in the partition.
*/
protected abstract Stream<HoodieClusteringGroup> buildClusteringGroupsForPartition(String partitionPath,
List<FileSlice> fileSlices);
/**
* Return list of partition paths to be considered for clustering.
*/
protected List<String> filterPartitionPaths(List<String> partitionPaths) {
return partitionPaths;
}
@Override
public Option<HoodieClusteringPlan> generateClusteringPlan() {
try {
HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient();
LOG.info("Scheduling clustering for " + metaClient.getBasePath());
List<String> partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
getWriteConfig().shouldAssumeDatePartitioning());
// filter the partition paths if needed to reduce list status
partitionPaths = filterPartitionPaths(partitionPaths);
if (partitionPaths.isEmpty()) {
// In case no partitions could be picked, return no clustering plan
return Option.empty();
}
List<HoodieClusteringGroup> clusteringGroups = getEngineContext().flatMap(partitionPaths,
partitionPath -> {
List<FileSlice> fileSlicesEligible = getFileSlicesEligibleForClustering(partitionPath).collect(Collectors.toList());
return buildClusteringGroupsForPartition(partitionPath, fileSlicesEligible).limit(getWriteConfig().getClusteringMaxNumGroups());
},
partitionPaths.size())
.stream().limit(getWriteConfig().getClusteringMaxNumGroups()).collect(Collectors.toList());
if (clusteringGroups.isEmpty()) {
LOG.info("No data available to cluster");
return Option.empty();
}
HoodieClusteringStrategy strategy = HoodieClusteringStrategy.newBuilder()
.setStrategyClassName(getWriteConfig().getClusteringExecutionStrategyClass())
.setStrategyParams(getStrategyParams())
.build();
return Option.of(HoodieClusteringPlan.newBuilder()
.setStrategy(strategy)
.setInputGroups(clusteringGroups)
.setExtraMetadata(getExtraMetadata())
.setVersion(getPlanVersion())
.build());
} catch (IOException e) {
throw new HoodieIOException("Unable to create clustering plan", e);
}
}
}

View File

@@ -27,8 +27,21 @@ import org.apache.hudi.table.action.HoodieWriteMetadata;
public abstract class AbstractBulkInsertHelper<T extends HoodieRecordPayload, I, K, O, R> {
/**
* Mark instant as inflight, write input records, update index and return result.
*/
public abstract HoodieWriteMetadata<O> bulkInsert(I inputRecords, String instantTime,
HoodieTable<T, I, K, O> table, HoodieWriteConfig config,
BaseCommitActionExecutor<T, I, K, O, R> executor, boolean performDedupe,
Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner);
/**
* Only write input records. Does not change timeline/index. Return information about new files created.
*/
public abstract O bulkInsert(I inputRecords, String instantTime,
HoodieTable<T, I, K, O> table, HoodieWriteConfig config,
boolean performDedupe,
Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner,
boolean addMetadataFields,
int parallelism);
}

View File

@@ -20,14 +20,13 @@ package org.apache.hudi.table.action.compact.strategy;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.client.utils.FileSliceMetricUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import java.io.Serializable;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -50,32 +49,14 @@ public abstract class CompactionStrategy implements Serializable {
* Callback hook when a HoodieCompactionOperation is created. Individual strategies can capture the metrics they need
* to decide on the priority.
*
* @param dataFile - Base file to compact
* @param partitionPath - Partition path
* @param logFiles - List of log files to compact with the base file
* @param writeConfig write configuration.
* @param slice fileSlice to capture metrics for.
* @return Map[String, Object] - metrics captured
*/
public Map<String, Double> captureMetrics(HoodieWriteConfig writeConfig, Option<HoodieBaseFile> dataFile,
String partitionPath, List<HoodieLogFile> logFiles) {
public Map<String, Double> captureMetrics(HoodieWriteConfig writeConfig, FileSlice slice) {
Map<String, Double> metrics = new HashMap<>();
long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize();
// Total size of all the log files
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0)
.reduce(Long::sum).orElse(0L);
// Total read will be the base file + all the log files
Long totalIORead =
FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L) + totalLogFileSize);
// Total write will be similar to the size of the base file
Long totalIOWrite =
FSUtils.getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize);
// Total IO will the the IO for read + write
long totalIO = totalIORead + totalIOWrite;
// Save these metrics and we will use during the filter
metrics.put(TOTAL_IO_READ_MB, totalIORead.doubleValue());
metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue());
metrics.put(TOTAL_IO_MB, (double) totalIO);
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
metrics.put(TOTAL_LOG_FILES, (double) logFiles.size());
FileSliceMetricUtils.addFileSliceCommonMetrics(Collections.singletonList(slice), metrics, defaultMaxParquetFileSize);
return metrics;
}

View File

@@ -20,14 +20,10 @@ package org.apache.hudi.table.action.compact.strategy;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
@@ -40,21 +36,6 @@ import java.util.stream.Collectors;
public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrategy
implements Comparator<HoodieCompactionOperation> {
private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE";
@Override
public Map<String, Double> captureMetrics(HoodieWriteConfig config, Option<HoodieBaseFile> dataFile,
String partitionPath, List<HoodieLogFile> logFiles) {
Map<String, Double> metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles);
// Total size of all the log files
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0)
.reduce(Long::sum).orElse(0L);
// save the metrics needed during the order
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
return metrics;
}
@Override
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {