[HUDI-2332] Add clustering and compaction in Kafka Connect Sink (#3857)
* [HUDI-2332] Add clustering and compaction in Kafka Connect Sink * Disable validation check on instant time for compaction and adjust configs * Add javadocs * Add clustering and compaction config * Fix transaction causing missing records in the target table * Add debugging logs * Fix kafka offset sync in participant * Adjust how clustering and compaction are configured in kafka-connect * Fix clustering strategy * Remove irrelevant changes from other published PRs * Update clustering logic and others * Update README * Fix test failures * Fix indentation * Fix clustering config * Add JavaCustomColumnsSortPartitioner and make async compaction enabled by default * Add test for JavaCustomColumnsSortPartitioner * Add more changes after IDE sync * Update README with clarification * Fix clustering logic after rebasing * Remove unrelated changes
This commit is contained in:
@@ -0,0 +1,65 @@
|
||||
/*
|
||||
* 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.client.common.HoodieJavaEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieJavaCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieJavaMergeOnReadTable;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Clustering Strategy that only looks at latest 'daybased.lookback.partitions' partitions
|
||||
* for Java engine.
|
||||
*/
|
||||
public class JavaRecentDaysClusteringPlanStrategy<T extends HoodieRecordPayload<T>>
|
||||
extends JavaSizeBasedClusteringPlanStrategy<T> {
|
||||
private static final Logger LOG = LogManager.getLogger(JavaRecentDaysClusteringPlanStrategy.class);
|
||||
|
||||
public JavaRecentDaysClusteringPlanStrategy(HoodieJavaCopyOnWriteTable<T> table,
|
||||
HoodieJavaEngineContext engineContext,
|
||||
HoodieWriteConfig writeConfig) {
|
||||
super(table, engineContext, writeConfig);
|
||||
}
|
||||
|
||||
public JavaRecentDaysClusteringPlanStrategy(HoodieJavaMergeOnReadTable<T> table,
|
||||
HoodieJavaEngineContext engineContext,
|
||||
HoodieWriteConfig writeConfig) {
|
||||
super(table, engineContext, writeConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<String> filterPartitionPaths(List<String> partitionPaths) {
|
||||
int targetPartitionsForClustering = getWriteConfig().getTargetPartitionsForClustering();
|
||||
int skipPartitionsFromLatestForClustering = getWriteConfig().getSkipPartitionsFromLatestForClustering();
|
||||
return partitionPaths.stream()
|
||||
.sorted(Comparator.reverseOrder())
|
||||
.skip(Math.max(skipPartitionsFromLatestForClustering, 0))
|
||||
.limit(targetPartitionsForClustering > 0 ? targetPartitionsForClustering : partitionPaths.size())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,131 @@
|
||||
/*
|
||||
* 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.client.common.HoodieJavaEngineContext;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
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.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.action.cluster.strategy.PartitionAwareClusteringPlanStrategy;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS;
|
||||
|
||||
/**
|
||||
* Clustering Strategy for Java engine based on following.
|
||||
* 1) Creates clustering groups based on max size allowed per group.
|
||||
* 2) Excludes files that are greater than 'small.file.limit' from clustering plan.
|
||||
*/
|
||||
public class JavaSizeBasedClusteringPlanStrategy<T extends HoodieRecordPayload<T>>
|
||||
extends PartitionAwareClusteringPlanStrategy<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
private static final Logger LOG = LogManager.getLogger(JavaSizeBasedClusteringPlanStrategy.class);
|
||||
|
||||
public JavaSizeBasedClusteringPlanStrategy(HoodieJavaCopyOnWriteTable<T> table,
|
||||
HoodieJavaEngineContext engineContext,
|
||||
HoodieWriteConfig writeConfig) {
|
||||
super(table, engineContext, writeConfig);
|
||||
}
|
||||
|
||||
public JavaSizeBasedClusteringPlanStrategy(HoodieJavaMergeOnReadTable<T> table,
|
||||
HoodieJavaEngineContext engineContext,
|
||||
HoodieWriteConfig writeConfig) {
|
||||
super(table, engineContext, writeConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Stream<HoodieClusteringGroup> buildClusteringGroupsForPartition(String partitionPath, List<FileSlice> fileSlices) {
|
||||
List<Pair<List<FileSlice>, Integer>> fileSliceGroups = new ArrayList<>();
|
||||
List<FileSlice> currentGroup = new ArrayList<>();
|
||||
long totalSizeSoFar = 0;
|
||||
HoodieWriteConfig writeConfig = getWriteConfig();
|
||||
for (FileSlice currentSlice : fileSlices) {
|
||||
// assume each filegroup size is ~= parquet.max.file.size
|
||||
totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : writeConfig.getParquetMaxFileSize();
|
||||
// check if max size is reached and create new group, if needed.
|
||||
if (totalSizeSoFar >= writeConfig.getClusteringMaxBytesInGroup() && !currentGroup.isEmpty()) {
|
||||
int numOutputGroups = getNumberOfOutputFileGroups(totalSizeSoFar, writeConfig.getClusteringTargetFileMaxBytes());
|
||||
LOG.info("Adding one clustering group " + totalSizeSoFar + " max bytes: "
|
||||
+ writeConfig.getClusteringMaxBytesInGroup() + " num input slices: " + currentGroup.size() + " output groups: " + numOutputGroups);
|
||||
fileSliceGroups.add(Pair.of(currentGroup, numOutputGroups));
|
||||
currentGroup = new ArrayList<>();
|
||||
totalSizeSoFar = 0;
|
||||
}
|
||||
currentGroup.add(currentSlice);
|
||||
// totalSizeSoFar could be 0 when new group was created in the previous conditional block.
|
||||
// reset to the size of current slice, otherwise the number of output file group will become 0 even though current slice is present.
|
||||
if (totalSizeSoFar == 0) {
|
||||
totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : writeConfig.getParquetMaxFileSize();
|
||||
}
|
||||
}
|
||||
if (!currentGroup.isEmpty()) {
|
||||
int numOutputGroups = getNumberOfOutputFileGroups(totalSizeSoFar, writeConfig.getClusteringTargetFileMaxBytes());
|
||||
LOG.info("Adding final clustering group " + totalSizeSoFar + " max bytes: "
|
||||
+ writeConfig.getClusteringMaxBytesInGroup() + " num input slices: " + currentGroup.size() + " output groups: " + numOutputGroups);
|
||||
fileSliceGroups.add(Pair.of(currentGroup, numOutputGroups));
|
||||
}
|
||||
|
||||
return fileSliceGroups.stream().map(fileSliceGroup -> HoodieClusteringGroup.newBuilder()
|
||||
.setSlices(getFileSliceInfo(fileSliceGroup.getLeft()))
|
||||
.setNumOutputFileGroups(fileSliceGroup.getRight())
|
||||
.setMetrics(buildMetrics(fileSliceGroup.getLeft()))
|
||||
.build());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, String> getStrategyParams() {
|
||||
Map<String, String> params = new HashMap<>();
|
||||
if (!StringUtils.isNullOrEmpty(getWriteConfig().getClusteringSortColumns())) {
|
||||
params.put(PLAN_STRATEGY_SORT_COLUMNS.key(), getWriteConfig().getClusteringSortColumns());
|
||||
}
|
||||
return params;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<String> filterPartitionPaths(List<String> partitionPaths) {
|
||||
return partitionPaths;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Stream<FileSlice> getFileSlicesEligibleForClustering(final String partition) {
|
||||
return super.getFileSlicesEligibleForClustering(partition)
|
||||
// Only files that have basefile size smaller than small file size are eligible.
|
||||
.filter(slice -> slice.getBaseFile().map(HoodieBaseFile::getFileSize).orElse(0L) < getWriteConfig().getClusteringSmallFileLimit());
|
||||
}
|
||||
|
||||
private int getNumberOfOutputFileGroups(long groupSize, long targetFileSize) {
|
||||
return (int) Math.ceil(groupSize / (double) targetFileSize);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,242 @@
|
||||
/*
|
||||
* 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.avro.HoodieAvroUtils;
|
||||
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.common.JavaTaskContextSupplier;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.ClusteringOperation;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
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.RewriteAvroPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
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.exception.HoodieClusteringException;
|
||||
import org.apache.hudi.execution.bulkinsert.JavaCustomColumnsSortPartitioner;
|
||||
import org.apache.hudi.io.IOUtils;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
import org.apache.hudi.keygen.KeyGenUtils;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.table.log.HoodieFileSliceReader.getFileSliceReader;
|
||||
import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS;
|
||||
|
||||
/**
|
||||
* Clustering strategy for Java engine.
|
||||
*/
|
||||
public abstract class JavaExecutionStrategy<T extends HoodieRecordPayload<T>>
|
||||
extends ClusteringExecutionStrategy<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(JavaExecutionStrategy.class);
|
||||
|
||||
public JavaExecutionStrategy(
|
||||
HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) {
|
||||
super(table, engineContext, writeConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> performClustering(
|
||||
HoodieClusteringPlan clusteringPlan, Schema schema, String instantTime) {
|
||||
// execute clustering for each group and collect WriteStatus
|
||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||
clusteringPlan.getInputGroups().forEach(
|
||||
inputGroup -> writeStatusList.addAll(runClusteringForGroup(
|
||||
inputGroup, clusteringPlan.getStrategy().getStrategyParams(),
|
||||
Option.ofNullable(clusteringPlan.getPreserveHoodieMetadata()).orElse(false),
|
||||
instantTime)));
|
||||
HoodieWriteMetadata<List<WriteStatus>> writeMetadata = new HoodieWriteMetadata<>();
|
||||
writeMetadata.setWriteStatuses(writeStatusList);
|
||||
return writeMetadata;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*
|
||||
* @param inputRecords List of {@link HoodieRecord}.
|
||||
* @param numOutputGroups Number of output file groups.
|
||||
* @param instantTime Clustering (replace commit) instant time.
|
||||
* @param strategyParams Strategy parameters containing columns to sort the data by when clustering.
|
||||
* @param schema Schema of the data including metadata fields.
|
||||
* @param fileGroupIdList File group id corresponding to each out group.
|
||||
* @param preserveHoodieMetadata Whether to preserve commit metadata while clustering.
|
||||
* @return List of {@link WriteStatus}.
|
||||
*/
|
||||
public abstract List<WriteStatus> performClusteringWithRecordList(
|
||||
final List<HoodieRecord<T>> inputRecords, final int numOutputGroups, final String instantTime,
|
||||
final Map<String, String> strategyParams, final Schema schema,
|
||||
final List<HoodieFileGroupId> fileGroupIdList, final boolean preserveHoodieMetadata);
|
||||
|
||||
/**
|
||||
* Create {@link BulkInsertPartitioner} based on strategy params.
|
||||
*
|
||||
* @param strategyParams Strategy parameters containing columns to sort the data by when clustering.
|
||||
* @param schema Schema of the data including metadata fields.
|
||||
* @return empty for now.
|
||||
*/
|
||||
protected Option<BulkInsertPartitioner<T>> getPartitioner(Map<String, String> strategyParams, Schema schema) {
|
||||
if (strategyParams.containsKey(PLAN_STRATEGY_SORT_COLUMNS.key())) {
|
||||
return Option.of(new JavaCustomColumnsSortPartitioner(
|
||||
strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key()).split(","),
|
||||
HoodieAvroUtils.addMetadataFields(schema)));
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes clustering for the group.
|
||||
*/
|
||||
private List<WriteStatus> runClusteringForGroup(
|
||||
HoodieClusteringGroup clusteringGroup, Map<String, String> strategyParams,
|
||||
boolean preserveHoodieMetadata, String instantTime) {
|
||||
List<HoodieRecord<T>> inputRecords = readRecordsForGroup(clusteringGroup, instantTime);
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema()));
|
||||
List<HoodieFileGroupId> inputFileIds = clusteringGroup.getSlices().stream()
|
||||
.map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId()))
|
||||
.collect(Collectors.toList());
|
||||
return performClusteringWithRecordList(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema, inputFileIds, preserveHoodieMetadata);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of all records for the group. This includes all records from file slice
|
||||
* (Apply updates from log files, if any).
|
||||
*/
|
||||
private List<HoodieRecord<T>> readRecordsForGroup(HoodieClusteringGroup clusteringGroup, String instantTime) {
|
||||
List<ClusteringOperation> clusteringOps = clusteringGroup.getSlices().stream().map(ClusteringOperation::create).collect(Collectors.toList());
|
||||
boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0);
|
||||
if (hasLogFiles) {
|
||||
// if there are log files, we read all records into memory for a file group and apply updates.
|
||||
return readRecordsForGroupWithLogs(clusteringOps, instantTime);
|
||||
} else {
|
||||
// We want to optimize reading records for case there are no log files.
|
||||
return readRecordsForGroupBaseFiles(clusteringOps);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Read records from baseFiles and apply updates.
|
||||
*/
|
||||
private List<HoodieRecord<T>> readRecordsForGroupWithLogs(List<ClusteringOperation> clusteringOps,
|
||||
String instantTime) {
|
||||
HoodieWriteConfig config = getWriteConfig();
|
||||
HoodieTable table = getHoodieTable();
|
||||
List<HoodieRecord<T>> records = new ArrayList<>();
|
||||
|
||||
clusteringOps.forEach(clusteringOp -> {
|
||||
long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new JavaTaskContextSupplier(), config);
|
||||
LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction);
|
||||
try {
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder()
|
||||
.withFileSystem(table.getMetaClient().getFs())
|
||||
.withBasePath(table.getMetaClient().getBasePath())
|
||||
.withLogFilePaths(clusteringOp.getDeltaFilePaths())
|
||||
.withReaderSchema(readerSchema)
|
||||
.withLatestInstantTime(instantTime)
|
||||
.withMaxMemorySizeInBytes(maxMemoryPerCompaction)
|
||||
.withReadBlocksLazily(config.getCompactionLazyBlockReadEnabled())
|
||||
.withReverseReader(config.getCompactionReverseLogReadEnabled())
|
||||
.withBufferSize(config.getMaxDFSStreamBufferSize())
|
||||
.withSpillableMapBasePath(config.getSpillableMapBasePath())
|
||||
.withPartition(clusteringOp.getPartitionPath())
|
||||
.build();
|
||||
|
||||
Option<HoodieFileReader> baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath())
|
||||
? Option.empty()
|
||||
: Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())));
|
||||
HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
|
||||
Iterator<HoodieRecord<T>> fileSliceReader = getFileSliceReader(baseFileReader, scanner, readerSchema,
|
||||
tableConfig.getPayloadClass(),
|
||||
tableConfig.getPreCombineField(),
|
||||
tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(),
|
||||
tableConfig.getPartitionFieldProp())));
|
||||
fileSliceReader.forEachRemaining(records::add);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath()
|
||||
+ " and " + clusteringOp.getDeltaFilePaths(), e);
|
||||
}
|
||||
});
|
||||
return records;
|
||||
}
|
||||
|
||||
/**
|
||||
* Read records from baseFiles.
|
||||
*/
|
||||
private List<HoodieRecord<T>> readRecordsForGroupBaseFiles(List<ClusteringOperation> clusteringOps) {
|
||||
List<HoodieRecord<T>> records = new ArrayList<>();
|
||||
clusteringOps.forEach(clusteringOp -> {
|
||||
try {
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema()));
|
||||
HoodieFileReader<IndexedRecord> baseFileReader = HoodieFileReaderFactory.getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath()));
|
||||
Iterator<IndexedRecord> recordIterator = baseFileReader.getRecordIterator(readerSchema);
|
||||
recordIterator.forEachRemaining(record -> records.add(transform(record)));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath()
|
||||
+ " and " + clusteringOp.getDeltaFilePaths(), e);
|
||||
}
|
||||
});
|
||||
return records;
|
||||
}
|
||||
|
||||
/**
|
||||
* Transform IndexedRecord into HoodieRecord.
|
||||
*/
|
||||
private HoodieRecord<T> transform(IndexedRecord indexedRecord) {
|
||||
GenericRecord record = (GenericRecord) indexedRecord;
|
||||
Option<BaseKeyGenerator> keyGeneratorOpt = Option.empty();
|
||||
String key = KeyGenUtils.getRecordKeyFromGenericRecord(record, keyGeneratorOpt);
|
||||
String partition = KeyGenUtils.getPartitionPathFromGenericRecord(record, keyGeneratorOpt);
|
||||
HoodieKey hoodieKey = new HoodieKey(key, partition);
|
||||
|
||||
HoodieRecordPayload avroPayload = new RewriteAvroPayload(record);
|
||||
HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, avroPayload);
|
||||
return hoodieRecord;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* 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.avro.Schema;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
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.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.CreateHandleFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.commit.JavaBulkInsertHelper;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Clustering Strategy based on following.
|
||||
* 1) Java execution engine.
|
||||
* 2) Uses bulk_insert to write data into new files.
|
||||
*/
|
||||
public class JavaSortAndSizeExecutionStrategy<T extends HoodieRecordPayload<T>>
|
||||
extends JavaExecutionStrategy<T> {
|
||||
private static final Logger LOG = LogManager.getLogger(JavaSortAndSizeExecutionStrategy.class);
|
||||
|
||||
public JavaSortAndSizeExecutionStrategy(HoodieTable table,
|
||||
HoodieEngineContext engineContext,
|
||||
HoodieWriteConfig writeConfig) {
|
||||
super(table, engineContext, writeConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WriteStatus> performClusteringWithRecordList(
|
||||
final List<HoodieRecord<T>> inputRecords, final int numOutputGroups,
|
||||
final String instantTime, final Map<String, String> strategyParams, final Schema schema,
|
||||
final List<HoodieFileGroupId> fileGroupIdList, final boolean preserveHoodieMetadata) {
|
||||
LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime);
|
||||
Properties props = getWriteConfig().getProps();
|
||||
props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key(), String.valueOf(numOutputGroups));
|
||||
// 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());
|
||||
props.put(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key(), String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes()));
|
||||
HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder().withProps(props).build();
|
||||
return (List<WriteStatus>) JavaBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig,
|
||||
false, getPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,62 @@
|
||||
/*
|
||||
* 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.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* A partitioner that does sorting based on specified column values for Java client.
|
||||
*
|
||||
* @param <T> HoodieRecordPayload type
|
||||
*/
|
||||
public class JavaCustomColumnsSortPartitioner<T extends HoodieRecordPayload>
|
||||
implements BulkInsertPartitioner<List<HoodieRecord<T>>> {
|
||||
|
||||
private final String[] sortColumnNames;
|
||||
private final Schema schema;
|
||||
|
||||
public JavaCustomColumnsSortPartitioner(String[] columnNames, Schema schema) {
|
||||
this.sortColumnNames = columnNames;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRecord<T>> repartitionRecords(
|
||||
List<HoodieRecord<T>> records, int outputSparkPartitions) {
|
||||
return records.stream().sorted((o1, o2) -> {
|
||||
Object values1 = HoodieAvroUtils.getRecordColumnValues(o1, sortColumnNames, schema);
|
||||
Object values2 = HoodieAvroUtils.getRecordColumnValues(o2, sortColumnNames, schema);
|
||||
return values1.toString().compareTo(values2.toString());
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean arePartitionRecordsSorted() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -29,6 +29,7 @@ import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
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.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
@@ -39,17 +40,24 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.io.HoodieSortedMergeHandle;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||
import org.apache.hudi.table.action.clean.CleanActionExecutor;
|
||||
import org.apache.hudi.table.action.clean.CleanPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaDeleteCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.cluster.JavaClusteringPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.cluster.JavaExecuteClusteringCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaBulkInsertCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaBulkInsertPreppedCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaDeleteCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaInsertCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaInsertOverwriteCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaInsertOverwriteTableCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaInsertPreppedCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaMergeHelper;
|
||||
import org.apache.hudi.table.action.commit.JavaUpsertCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.JavaUpsertPreppedCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor;
|
||||
@@ -57,10 +65,20 @@ import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
|
||||
import org.apache.hudi.table.action.savepoint.SavepointActionExecutor;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class HoodieJavaCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieJavaTable<T> {
|
||||
public class HoodieJavaCopyOnWriteTable<T extends HoodieRecordPayload>
|
||||
extends HoodieJavaTable<T> implements HoodieCompactionHandler<T> {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HoodieJavaCopyOnWriteTable.class);
|
||||
|
||||
protected HoodieJavaCopyOnWriteTable(HoodieWriteConfig config,
|
||||
HoodieEngineContext context,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
@@ -160,23 +178,23 @@ public class HoodieJavaCopyOnWriteTable<T extends HoodieRecordPayload> extends H
|
||||
public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context,
|
||||
String instantTime,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
throw new HoodieNotSupportedException("ScheduleCompaction is not supported yet");
|
||||
throw new HoodieNotSupportedException("ScheduleCompaction is not supported on a CopyOnWrite table");
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> compact(HoodieEngineContext context,
|
||||
String compactionInstantTime) {
|
||||
throw new HoodieNotSupportedException("Compact is not supported yet");
|
||||
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieClusteringPlan> scheduleClustering(final HoodieEngineContext context, final String instantTime, final Option<Map<String, String>> extraMetadata) {
|
||||
throw new HoodieNotSupportedException("Clustering is not supported yet");
|
||||
return new JavaClusteringPlanActionExecutor<>(context, config, this, instantTime, extraMetadata).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> cluster(final HoodieEngineContext context, final String clusteringInstantTime) {
|
||||
throw new HoodieNotSupportedException("Clustering is not supported yet");
|
||||
return new JavaExecuteClusteringCommitActionExecutor<>(context, config, this, clusteringInstantTime).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -235,4 +253,53 @@ public class HoodieJavaCopyOnWriteTable<T extends HoodieRecordPayload> extends H
|
||||
return new CopyOnWriteRestoreActionExecutor(
|
||||
context, config, this, restoreInstantTime, instantToRestore).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(
|
||||
String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile)
|
||||
throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, keyToNewRecords, oldDataFile);
|
||||
return handleUpdateInternal(upsertHandle, instantTime, fileId);
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle<?, ?, ?, ?> upsertHandle, String instantTime,
|
||||
String fileId) throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException(
|
||||
"Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
|
||||
} else {
|
||||
JavaMergeHelper.newInstance().runMerge(this, upsertHandle);
|
||||
}
|
||||
|
||||
// TODO(yihua): This needs to be revisited
|
||||
if (upsertHandle.getPartitionPath() == null) {
|
||||
LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
|
||||
+ upsertHandle.writeStatuses());
|
||||
}
|
||||
|
||||
return Collections.singletonList(upsertHandle.writeStatuses()).iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) {
|
||||
if (requireSortedRecords()) {
|
||||
return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId,
|
||||
dataFileToBeMerged, taskContextSupplier, Option.empty());
|
||||
} else {
|
||||
return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId,
|
||||
dataFileToBeMerged, taskContextSupplier, Option.empty());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsert(
|
||||
String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap) {
|
||||
HoodieCreateHandle<?, ?, ?, ?> createHandle =
|
||||
new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier);
|
||||
createHandle.write();
|
||||
return Collections.singletonList(createHandle.close()).iterator();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieJavaEngineContext;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
@@ -29,9 +30,13 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.JavaBulkInsertPreppedCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.HoodieJavaMergeOnReadTableCompactor;
|
||||
import org.apache.hudi.table.action.compact.RunCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.ScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.JavaUpsertPreppedDeltaCommitActionExecutor;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class HoodieJavaMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieJavaCopyOnWriteTable<T> {
|
||||
protected HoodieJavaMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
|
||||
@@ -60,4 +65,21 @@ public class HoodieJavaMergeOnReadTable<T extends HoodieRecordPayload> extends H
|
||||
return new JavaBulkInsertPreppedCommitActionExecutor((HoodieJavaEngineContext) context, config,
|
||||
this, instantTime, preppedRecords, bulkInsertPartitioner).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor(
|
||||
context, config, this, instantTime, extraMetadata,
|
||||
new HoodieJavaMergeOnReadTableCompactor());
|
||||
return scheduleCompactionExecutor.execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> compact(
|
||||
HoodieEngineContext context, String compactionInstantTime) {
|
||||
RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor(
|
||||
context, config, this, compactionInstantTime, new HoodieJavaMergeOnReadTableCompactor(),
|
||||
new HoodieJavaCopyOnWriteTable(config, context, getMetaClient()));
|
||||
return convertMetadata(compactionExecutor.execute());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieJavaEngineContext;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -31,9 +32,12 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.JavaHoodieIndexFactory;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.hudi.common.data.HoodieList.getList;
|
||||
|
||||
public abstract class HoodieJavaTable<T extends HoodieRecordPayload>
|
||||
extends HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
protected HoodieJavaTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
|
||||
@@ -61,6 +65,11 @@ public abstract class HoodieJavaTable<T extends HoodieRecordPayload>
|
||||
}
|
||||
}
|
||||
|
||||
public static HoodieWriteMetadata<List<WriteStatus>> convertMetadata(
|
||||
HoodieWriteMetadata<HoodieData<WriteStatus>> metadata) {
|
||||
return metadata.clone(getList(metadata.getWriteStatuses()));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return JavaHoodieIndexFactory.createIndex(config);
|
||||
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
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.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class JavaClusteringPlanActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseClusteringPlanActionExecutor<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
|
||||
public JavaClusteringPlanActionExecutor(
|
||||
HoodieEngineContext context, HoodieWriteConfig config,
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
|
||||
String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, instantTime, extraMetadata);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,123 @@
|
||||
/*
|
||||
* 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.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringGroup;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.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.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieClusteringException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy;
|
||||
import org.apache.hudi.table.action.commit.BaseJavaCommitActionExecutor;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class JavaExecuteClusteringCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseJavaCommitActionExecutor<T> {
|
||||
|
||||
private final HoodieClusteringPlan clusteringPlan;
|
||||
|
||||
public JavaExecuteClusteringCommitActionExecutor(
|
||||
HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime) {
|
||||
super(context, config, table, instantTime, WriteOperationType.CLUSTER);
|
||||
this.clusteringPlan = ClusteringUtils.getClusteringPlan(
|
||||
table.getMetaClient(), HoodieTimeline.getReplaceCommitRequestedInstant(instantTime))
|
||||
.map(Pair::getRight).orElseThrow(() -> new HoodieClusteringException(
|
||||
"Unable to read clustering plan for instant: " + instantTime));
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> execute() {
|
||||
HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(instantTime);
|
||||
// Mark instant as clustering inflight
|
||||
table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty());
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
final Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
HoodieWriteMetadata<List<WriteStatus>> writeMetadata = (
|
||||
(ClusteringExecutionStrategy<T, List<HoodieRecord<? extends HoodieRecordPayload>>, List<HoodieKey>, List<WriteStatus>>)
|
||||
ReflectionUtils.loadClass(config.getClusteringExecutionStrategyClass(),
|
||||
new Class<?>[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config))
|
||||
.performClustering(clusteringPlan, schema, instantTime);
|
||||
List<WriteStatus> writeStatusList = writeMetadata.getWriteStatuses();
|
||||
List<WriteStatus> statuses = updateIndex(writeStatusList, writeMetadata);
|
||||
writeMetadata.setWriteStats(statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()));
|
||||
writeMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(writeMetadata));
|
||||
validateWriteResult(writeMetadata);
|
||||
commitOnAutoCommit(writeMetadata);
|
||||
if (!writeMetadata.getCommitMetadata().isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeMetadata.getWriteStats().get(), writeMetadata.getPartitionToReplaceFileIds(),
|
||||
extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
|
||||
writeMetadata.setCommitMetadata(Option.of(commitMetadata));
|
||||
}
|
||||
return writeMetadata;
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate actions taken by clustering. In the first implementation, we validate at least one new file is written.
|
||||
* But we can extend this to add more validation. E.g. number of records read = number of records written etc.
|
||||
* We can also make these validations in BaseCommitActionExecutor to reuse pre-commit hooks for multiple actions.
|
||||
*/
|
||||
private void validateWriteResult(HoodieWriteMetadata<List<WriteStatus>> writeMetadata) {
|
||||
if (writeMetadata.getWriteStatuses().isEmpty()) {
|
||||
throw new HoodieClusteringException("Clustering plan produced 0 WriteStatus for " + instantTime
|
||||
+ " #groups: " + clusteringPlan.getInputGroups().size() + " expected at least "
|
||||
+ clusteringPlan.getInputGroups().stream().mapToInt(HoodieClusteringGroup::getNumOutputFileGroups).sum()
|
||||
+ " write statuses");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCommitActionType() {
|
||||
return HoodieTimeline.REPLACE_COMMIT_ACTION;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, List<String>> getPartitionToReplacedFileIds(HoodieWriteMetadata<List<WriteStatus>> writeMetadata) {
|
||||
Set<HoodieFileGroupId> newFilesWritten = writeMetadata.getWriteStats().get().stream()
|
||||
.map(s -> new HoodieFileGroupId(s.getPartitionPath(), s.getFileId())).collect(Collectors.toSet());
|
||||
return ClusteringUtils.getFileGroupsFromClusteringPlan(clusteringPlan)
|
||||
.filter(fg -> !newFilesWritten.contains(fg))
|
||||
.collect(Collectors.groupingBy(fg -> fg.getPartitionPath(), Collectors.mapping(fg -> fg.getFileId(), Collectors.toList())));
|
||||
}
|
||||
}
|
||||
@@ -126,13 +126,14 @@ public abstract class BaseJavaCommitActionExecutor<T extends HoodieRecordPayload
|
||||
return result;
|
||||
}
|
||||
|
||||
protected void updateIndex(List<WriteStatus> writeStatuses, HoodieWriteMetadata<List<WriteStatus>> result) {
|
||||
protected List<WriteStatus> updateIndex(List<WriteStatus> writeStatuses, HoodieWriteMetadata<List<WriteStatus>> result) {
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
List<WriteStatus> statuses = HoodieList.getList(
|
||||
table.getIndex().updateLocation(HoodieList.of(writeStatuses), context, table));
|
||||
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
|
||||
result.setWriteStatuses(statuses);
|
||||
return statuses;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.action.compact;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
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.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Compacts a hoodie table with merge on read storage in Java engine. Computes all possible
|
||||
* compactions, passes it through a CompactionFilter and executes all the compactions and
|
||||
* writes a new version of base files and make a normal commit.
|
||||
*/
|
||||
public class HoodieJavaMergeOnReadTableCompactor<T extends HoodieRecordPayload>
|
||||
extends HoodieCompactor<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
|
||||
@Override
|
||||
public void preCompact(
|
||||
HoodieTable table, HoodieTimeline pendingCompactionTimeline, String compactionInstantTime) {
|
||||
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
|
||||
if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
|
||||
table.rollbackInflightCompaction(inflightInstant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maybePersist(HoodieData<WriteStatus> writeStatus, HoodieWriteConfig config) {
|
||||
// No OP
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,98 @@
|
||||
/*
|
||||
* 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.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.testutils.HoodieJavaClientTestBase;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestJavaBulkInsertInternalPartitioner extends HoodieJavaClientTestBase {
|
||||
private static final Comparator<HoodieRecord> KEY_COMPARATOR =
|
||||
Comparator.comparing(o -> (o.getPartitionPath() + "+" + o.getRecordKey()));
|
||||
|
||||
public static List<HoodieRecord> generateTestRecordsForBulkInsert(int numRecords) {
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
|
||||
List<HoodieRecord> records = dataGenerator.generateInserts("0", numRecords);
|
||||
return records;
|
||||
}
|
||||
|
||||
public static Map<String, Long> generatePartitionNumRecords(List<HoodieRecord> records) {
|
||||
return records.stream().map(record -> record.getPartitionPath())
|
||||
.collect(Collectors.groupingBy(Function.identity(), Collectors.counting()));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {"rider", "rider,driver"})
|
||||
public void testCustomColumnSortPartitioner(String sortColumnString) throws Exception {
|
||||
String[] sortColumns = sortColumnString.split(",");
|
||||
Comparator<HoodieRecord> columnComparator =
|
||||
getCustomColumnComparator(HoodieTestDataGenerator.AVRO_SCHEMA, sortColumns);
|
||||
|
||||
List<HoodieRecord> records = generateTestRecordsForBulkInsert(1000);
|
||||
testBulkInsertInternalPartitioner(
|
||||
new JavaCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA),
|
||||
records, true, generatePartitionNumRecords(records), Option.of(columnComparator));
|
||||
}
|
||||
|
||||
private Comparator<HoodieRecord> getCustomColumnComparator(Schema schema, String[] sortColumns) {
|
||||
return Comparator.comparing(
|
||||
record -> HoodieAvroUtils.getRecordColumnValues(record, sortColumns, schema).toString());
|
||||
}
|
||||
|
||||
private void verifyRecordAscendingOrder(List<HoodieRecord> records,
|
||||
Option<Comparator<HoodieRecord>> comparator) {
|
||||
List<HoodieRecord> expectedRecords = new ArrayList<>(records);
|
||||
Collections.sort(expectedRecords, comparator.orElse(KEY_COMPARATOR));
|
||||
assertEquals(expectedRecords, records);
|
||||
}
|
||||
|
||||
private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner,
|
||||
List<HoodieRecord> records,
|
||||
boolean isSorted,
|
||||
Map<String, Long> expectedPartitionNumRecords,
|
||||
Option<Comparator<HoodieRecord>> comparator) {
|
||||
List<HoodieRecord> actualRecords =
|
||||
(List<HoodieRecord>) partitioner.repartitionRecords(records, 1);
|
||||
if (isSorted) {
|
||||
// Verify global order
|
||||
verifyRecordAscendingOrder(actualRecords, comparator);
|
||||
}
|
||||
|
||||
// Verify number of records per partition path
|
||||
assertEquals(expectedPartitionNumRecords, generatePartitionNumRecords(actualRecords));
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user