[HUDI-2501] Add HoodieData abstraction and refactor compaction actions in hudi-client module (#3741)
This commit is contained in:
@@ -40,6 +40,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
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.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieClusteringException;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
@@ -51,7 +52,7 @@ import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.compact.SparkCompactHelpers;
|
||||
import org.apache.hudi.table.action.compact.CompactHelpers;
|
||||
import org.apache.hudi.table.marker.WriteMarkersFactory;
|
||||
import org.apache.hudi.table.upgrade.SparkUpgradeDowngradeHelper;
|
||||
import org.apache.hudi.table.upgrade.UpgradeDowngrade;
|
||||
@@ -291,8 +292,8 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
@Override
|
||||
public void commitCompaction(String compactionInstantTime, JavaRDD<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata) throws IOException {
|
||||
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
|
||||
HoodieCommitMetadata metadata = SparkCompactHelpers.newInstance().createCompactionMetadata(
|
||||
table, compactionInstantTime, writeStatuses, config.getSchema());
|
||||
HoodieCommitMetadata metadata = CompactHelpers.getInstance().createCompactionMetadata(
|
||||
table, compactionInstantTime, HoodieJavaRDD.of(writeStatuses), config.getSchema());
|
||||
extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata));
|
||||
completeCompaction(metadata, writeStatuses, table, compactionInstantTime);
|
||||
}
|
||||
@@ -307,7 +308,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
// commit to data table after committing to metadata table.
|
||||
finalizeWrite(table, compactionCommitTime, writeStats);
|
||||
LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata);
|
||||
SparkCompactHelpers.newInstance().completeInflightCompaction(table, compactionCommitTime, metadata);
|
||||
CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata);
|
||||
WriteMarkersFactory.get(config.getMarkersType(), table, compactionCommitTime)
|
||||
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||
if (compactionTimer != null) {
|
||||
@@ -330,11 +331,12 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
|
||||
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
|
||||
if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
|
||||
rollbackInflightCompaction(inflightInstant, table);
|
||||
table.rollbackInflightCompaction(inflightInstant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
compactionTimer = metrics.getCompactionCtx();
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = table.compact(context, compactionInstantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata =
|
||||
table.compact(context, compactionInstantTime);
|
||||
JavaRDD<WriteStatus> statuses = compactionMetadata.getWriteStatuses();
|
||||
if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) {
|
||||
completeTableService(TableServiceType.COMPACT, compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime);
|
||||
|
||||
@@ -20,6 +20,8 @@ package org.apache.hudi.client.common;
|
||||
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.data.HoodieAccumulator;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.engine.EngineProperty;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.function.SerializableBiFunction;
|
||||
@@ -30,6 +32,8 @@ import org.apache.hudi.common.function.SerializablePairFunction;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.data.HoodieSparkLongAccumulator;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -74,6 +78,23 @@ public class HoodieSparkEngineContext extends HoodieEngineContext {
|
||||
return ((HoodieSparkEngineContext) context).getJavaSparkContext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieAccumulator newAccumulator() {
|
||||
HoodieSparkLongAccumulator accumulator = HoodieSparkLongAccumulator.create();
|
||||
javaSparkContext.sc().register(accumulator.getAccumulator());
|
||||
return accumulator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> HoodieData<T> emptyHoodieData() {
|
||||
return HoodieJavaRDD.of(javaSparkContext.emptyRDD());
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> HoodieData<T> parallelize(List<T> data) {
|
||||
return HoodieJavaRDD.of(javaSparkContext.parallelize(data, data.size()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism) {
|
||||
return javaSparkContext.parallelize(data, parallelism).map(func::apply).collect();
|
||||
|
||||
@@ -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.data;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.function.SerializableFunction;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Holds a {@link JavaRDD} of objects.
|
||||
*
|
||||
* @param <T> type of object.
|
||||
*/
|
||||
public class HoodieJavaRDD<T> extends HoodieData<T> {
|
||||
|
||||
private final JavaRDD<T> rddData;
|
||||
|
||||
private HoodieJavaRDD(JavaRDD<T> rddData) {
|
||||
this.rddData = rddData;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param rddData a {@link JavaRDD} of objects in type T.
|
||||
* @param <T> type of object.
|
||||
* @return a new instance containing the {@link JavaRDD<T>} reference.
|
||||
*/
|
||||
public static <T> HoodieJavaRDD<T> of(JavaRDD<T> rddData) {
|
||||
return new HoodieJavaRDD<>(rddData);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param data a {@link List} of objects in type T.
|
||||
* @param context {@link HoodieSparkEngineContext} to use.
|
||||
* @param parallelism parallelism for the {@link JavaRDD<T>}.
|
||||
* @param <T> type of object.
|
||||
* @return a new instance containing the {@link JavaRDD<T>} instance.
|
||||
*/
|
||||
public static <T> HoodieJavaRDD<T> of(
|
||||
List<T> data, HoodieSparkEngineContext context, int parallelism) {
|
||||
return new HoodieJavaRDD<>(context.getJavaSparkContext().parallelize(data, parallelism));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param hoodieData {@link HoodieJavaRDD <T>} instance containing the {@link JavaRDD} of objects.
|
||||
* @param <T> type of object.
|
||||
* @return the a {@link JavaRDD} of objects in type T.
|
||||
*/
|
||||
public static <T> JavaRDD<T> getJavaRDD(HoodieData<T> hoodieData) {
|
||||
return ((HoodieJavaRDD<T>) hoodieData).get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<T> get() {
|
||||
return rddData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty() {
|
||||
return rddData.isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> map(SerializableFunction<T, O> func) {
|
||||
return HoodieJavaRDD.of(rddData.map(func::apply));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func) {
|
||||
return HoodieJavaRDD.of(rddData.flatMap(func::apply));
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<T> collectAsList() {
|
||||
return rddData.collect();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* 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.data;
|
||||
|
||||
import org.apache.hudi.common.data.HoodieAccumulator;
|
||||
|
||||
import org.apache.spark.util.AccumulatorV2;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
|
||||
/**
|
||||
* An accumulator on counts based on Spark {@link AccumulatorV2} implementation.
|
||||
*/
|
||||
public class HoodieSparkLongAccumulator extends HoodieAccumulator {
|
||||
|
||||
private final AccumulatorV2<Long, Long> accumulator;
|
||||
|
||||
private HoodieSparkLongAccumulator() {
|
||||
accumulator = new LongAccumulator();
|
||||
}
|
||||
|
||||
public static HoodieSparkLongAccumulator create() {
|
||||
return new HoodieSparkLongAccumulator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long value() {
|
||||
return accumulator.value();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(long increment) {
|
||||
accumulator.add(increment);
|
||||
}
|
||||
|
||||
public AccumulatorV2<Long, Long> getAccumulator() {
|
||||
return accumulator;
|
||||
}
|
||||
}
|
||||
@@ -87,7 +87,8 @@ import java.util.Map;
|
||||
* <p>
|
||||
* UPDATES - Produce a new version of the file, just replacing the updated records with new values
|
||||
*/
|
||||
public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieSparkTable<T> {
|
||||
public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload>
|
||||
extends HoodieSparkTable<T> implements HoodieCompactionHandler<T> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieSparkCopyOnWriteTable.class);
|
||||
|
||||
@@ -157,7 +158,8 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> compact(HoodieEngineContext context, String compactionInstantTime) {
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> compact(
|
||||
HoodieEngineContext context, String compactionInstantTime) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
|
||||
}
|
||||
|
||||
@@ -191,20 +193,22 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public Option<HoodieRollbackPlan> scheduleRollback(HoodieEngineContext context,
|
||||
String instantTime,
|
||||
HoodieInstant instantToRollback, boolean skipTimelinePublish) {
|
||||
String instantTime,
|
||||
HoodieInstant instantToRollback, boolean skipTimelinePublish) {
|
||||
return new BaseRollbackPlanActionExecutor<>(context, config, this, instantTime, instantToRollback, skipTimelinePublish).execute();
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
|
||||
@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 {
|
||||
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);
|
||||
@@ -241,9 +245,11 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap) {
|
||||
HoodieCreateHandle<?,?,?,?> createHandle =
|
||||
@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();
|
||||
|
||||
@@ -39,9 +39,9 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||
import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.SparkRunCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.SparkScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor;
|
||||
import org.apache.hudi.table.action.compact.RunCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.ScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkBulkInsertPreppedDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor;
|
||||
@@ -123,15 +123,19 @@ public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
BaseScheduleCompactionActionExecutor scheduleCompactionExecutor = new SparkScheduleCompactionActionExecutor(
|
||||
context, config, this, instantTime, extraMetadata);
|
||||
ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor(
|
||||
context, config, this, instantTime, extraMetadata,
|
||||
new HoodieSparkMergeOnReadTableCompactor());
|
||||
return scheduleCompactionExecutor.execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> compact(HoodieEngineContext context, String compactionInstantTime) {
|
||||
SparkRunCompactionActionExecutor compactionExecutor = new SparkRunCompactionActionExecutor((HoodieSparkEngineContext) context, config, this, compactionInstantTime);
|
||||
return compactionExecutor.execute();
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> compact(
|
||||
HoodieEngineContext context, String compactionInstantTime) {
|
||||
RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor(
|
||||
context, config, this, compactionInstantTime, new HoodieSparkMergeOnReadTableCompactor(),
|
||||
new HoodieSparkCopyOnWriteTable(config, context, getMetaClient()));
|
||||
return convertMetadata(compactionExecutor.execute());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -35,12 +36,15 @@ import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.apache.hudi.data.HoodieJavaRDD.getJavaRDD;
|
||||
|
||||
public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
|
||||
extends HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
@@ -91,6 +95,11 @@ public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
|
||||
return hoodieSparkTable;
|
||||
}
|
||||
|
||||
public static HoodieWriteMetadata<JavaRDD<WriteStatus>> convertMetadata(
|
||||
HoodieWriteMetadata<HoodieData<WriteStatus>> metadata) {
|
||||
return metadata.clone(getJavaRDD(metadata.getWriteStatuses()));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return SparkHoodieIndex.createIndex(config);
|
||||
|
||||
@@ -18,248 +18,41 @@
|
||||
|
||||
package org.apache.hudi.table.action.compact;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.client.utils.SparkMemoryUtils;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
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.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.IOUtils;
|
||||
import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.util.AccumulatorV2;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* Compacts a hoodie table with merge on read storage. 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
|
||||
*
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class HoodieSparkMergeOnReadTableCompactor<T extends HoodieRecordPayload> implements HoodieCompactor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieSparkMergeOnReadTableCompactor.class);
|
||||
// Accumulator to keep track of total log files for a table
|
||||
private AccumulatorV2<Long, Long> totalLogFiles;
|
||||
// Accumulator to keep track of total log file slices for a table
|
||||
private AccumulatorV2<Long, Long> totalFileSlices;
|
||||
public class HoodieSparkMergeOnReadTableCompactor<T extends HoodieRecordPayload>
|
||||
extends HoodieCompactor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(HoodieEngineContext context, HoodieCompactionPlan compactionPlan,
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionInstantTime) throws IOException {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|
||||
|| (compactionPlan.getOperations().isEmpty())) {
|
||||
return jsc.emptyRDD();
|
||||
public void preCompact(
|
||||
HoodieTable table, HoodieTimeline pendingCompactionTimeline, String compactionInstantTime) {
|
||||
HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
if (!pendingCompactionTimeline.containsInstant(instant)) {
|
||||
throw new IllegalStateException(
|
||||
"No Compaction request available at " + compactionInstantTime + " to run compaction");
|
||||
}
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient);
|
||||
|
||||
// Here we firstly use the table schema as the reader schema to read
|
||||
// log file.That is because in the case of MergeInto, the config.getSchema may not
|
||||
// the same with the table schema.
|
||||
try {
|
||||
Schema readerSchema = schemaUtil.getTableAvroSchema(false);
|
||||
config.setSchema(readerSchema.toString());
|
||||
} catch (Exception e) {
|
||||
// If there is no commit in the table, just ignore the exception.
|
||||
}
|
||||
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
HoodieSparkCopyOnWriteTable table = new HoodieSparkCopyOnWriteTable(config, context, metaClient);
|
||||
List<CompactionOperation> operations = compactionPlan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
LOG.info("Compactor compacting " + operations + " files");
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices");
|
||||
return jsc.parallelize(operations, operations.size())
|
||||
.map(s -> compact(table, metaClient, config, s, compactionInstantTime)).flatMap(List::iterator);
|
||||
}
|
||||
|
||||
private List<WriteStatus> compact(HoodieSparkCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config, CompactionOperation operation, String instantTime) throws IOException {
|
||||
FileSystem fs = metaClient.getFs();
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
LOG.info("Compacting base " + operation.getDataFileName() + " with delta files " + operation.getDeltaFileNames()
|
||||
+ " for commit " + instantTime);
|
||||
// TODO - FIX THIS
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file
|
||||
// (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be
|
||||
// loaded and load it using CompositeAvroLogReader
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
String maxInstantTime = metaClient
|
||||
.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
|
||||
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config);
|
||||
LOG.info("MaxMemoryPerCompaction => " + maxMemoryPerCompaction);
|
||||
|
||||
List<String> logFiles = operation.getDeltaFileNames().stream().map(
|
||||
p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), p).toString())
|
||||
.collect(toList());
|
||||
HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder()
|
||||
.withFileSystem(fs)
|
||||
.withBasePath(metaClient.getBasePath())
|
||||
.withLogFilePaths(logFiles)
|
||||
.withReaderSchema(readerSchema)
|
||||
.withLatestInstantTime(maxInstantTime)
|
||||
.withMaxMemorySizeInBytes(maxMemoryPerCompaction)
|
||||
.withReadBlocksLazily(config.getCompactionLazyBlockReadEnabled())
|
||||
.withReverseReader(config.getCompactionReverseLogReadEnabled())
|
||||
.withBufferSize(config.getMaxDFSStreamBufferSize())
|
||||
.withSpillableMapBasePath(config.getSpillableMapBasePath())
|
||||
.withDiskMapType(config.getCommonConfig().getSpillableDiskMapType())
|
||||
.withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
|
||||
.build();
|
||||
if (!scanner.iterator().hasNext()) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
Option<HoodieBaseFile> oldDataFileOpt =
|
||||
operation.getBaseFile(metaClient.getBasePath(), operation.getPartitionPath());
|
||||
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
Iterator<List<WriteStatus>> result;
|
||||
// If the dataFile is present, perform updates else perform inserts into a new base file.
|
||||
if (oldDataFileOpt.isPresent()) {
|
||||
result = hoodieCopyOnWriteTable.handleUpdate(instantTime, operation.getPartitionPath(),
|
||||
operation.getFileId(), scanner.getRecords(),
|
||||
oldDataFileOpt.get());
|
||||
} else {
|
||||
result = hoodieCopyOnWriteTable.handleInsert(instantTime, operation.getPartitionPath(), operation.getFileId(),
|
||||
scanner.getRecords());
|
||||
}
|
||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream).peek(s -> {
|
||||
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
|
||||
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
|
||||
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
|
||||
s.getStat().setPartitionPath(operation.getPartitionPath());
|
||||
s.getStat()
|
||||
.setTotalLogSizeCompacted(operation.getMetrics().get(CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue());
|
||||
s.getStat().setTotalLogBlocks(scanner.getTotalLogBlocks());
|
||||
s.getStat().setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks());
|
||||
s.getStat().setTotalRollbackBlocks(scanner.getTotalRollbacks());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
|
||||
s.getStat().setRuntimeStats(runtimeStats);
|
||||
scanner.close();
|
||||
}).collect(toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
|
||||
HoodieWriteConfig config, String compactionCommitTime,
|
||||
Set<HoodieFileGroupId> fgIdsInPendingCompactionAndClustering)
|
||||
throws IOException {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
totalLogFiles = new LongAccumulator();
|
||||
totalFileSlices = new LongAccumulator();
|
||||
jsc.sc().register(totalLogFiles);
|
||||
jsc.sc().register(totalFileSlices);
|
||||
|
||||
ValidationUtils.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
|
||||
"Can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not "
|
||||
+ hoodieTable.getMetaClient().getTableType().name());
|
||||
|
||||
// TODO : check if maxMemory is not greater than JVM or spark.executor memory
|
||||
// TODO - rollback any compactions in flight
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
LOG.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
||||
|
||||
// filter the partition paths if needed to reduce list status
|
||||
partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths);
|
||||
|
||||
if (partitionPaths.isEmpty()) {
|
||||
// In case no partitions could be picked, return no compaction plan
|
||||
return null;
|
||||
}
|
||||
|
||||
SliceView fileSystemView = hoodieTable.getSliceView();
|
||||
LOG.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact");
|
||||
|
||||
List<HoodieCompactionOperation> operations = context.flatMap(partitionPaths, partitionPath -> {
|
||||
return fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.filter(slice -> !fgIdsInPendingCompactionAndClustering.contains(slice.getFileGroupId()))
|
||||
.map(s -> {
|
||||
List<HoodieLogFile> logFiles =
|
||||
s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
|
||||
totalLogFiles.add((long) logFiles.size());
|
||||
totalFileSlices.add(1L);
|
||||
// Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
|
||||
// for spark Map operations and collecting them finally in Avro generated classes for storing
|
||||
// into meta files.
|
||||
Option<HoodieBaseFile> dataFile = s.getBaseFile();
|
||||
return new CompactionOperation(dataFile, partitionPath, logFiles,
|
||||
config.getCompactionStrategy().captureMetrics(config, s));
|
||||
})
|
||||
.filter(c -> !c.getDeltaFileNames().isEmpty());
|
||||
}, partitionPaths.size()).stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList());
|
||||
|
||||
LOG.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
LOG.info("Total number of latest files slices " + totalFileSlices.value());
|
||||
LOG.info("Total number of log files " + totalLogFiles.value());
|
||||
LOG.info("Total number of file slices " + totalFileSlices.value());
|
||||
// Filter the compactions with the passed in filter. This lets us choose most effective
|
||||
// compactions only
|
||||
HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations,
|
||||
CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList()));
|
||||
ValidationUtils.checkArgument(
|
||||
compactionPlan.getOperations().stream().noneMatch(
|
||||
op -> fgIdsInPendingCompactionAndClustering.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))),
|
||||
"Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. "
|
||||
+ "Please fix your strategy implementation. FileIdsWithPendingCompactions :" + fgIdsInPendingCompactionAndClustering
|
||||
+ ", Selected workload :" + compactionPlan);
|
||||
if (compactionPlan.getOperations().isEmpty()) {
|
||||
LOG.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
|
||||
}
|
||||
return compactionPlan;
|
||||
public void maybePersist(HoodieData<WriteStatus> writeStatus, HoodieWriteConfig config) {
|
||||
HoodieJavaRDD.getJavaRDD(writeStatus).persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,75 +0,0 @@
|
||||
/*
|
||||
* 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.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
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.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A spark implementation of {@link AbstractCompactHelpers}.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class SparkCompactHelpers<T extends HoodieRecordPayload> extends
|
||||
AbstractCompactHelpers<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private SparkCompactHelpers() {
|
||||
}
|
||||
|
||||
private static class CompactHelperHolder {
|
||||
private static final SparkCompactHelpers SPARK_COMPACT_HELPERS = new SparkCompactHelpers();
|
||||
}
|
||||
|
||||
public static SparkCompactHelpers newInstance() {
|
||||
return CompactHelperHolder.SPARK_COMPACT_HELPERS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCommitMetadata createCompactionMetadata(HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String compactionInstantTime,
|
||||
JavaRDD<WriteStatus> writeStatuses,
|
||||
String schema) throws IOException {
|
||||
byte[] planBytes = table.getActiveTimeline().readCompactionPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get();
|
||||
HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan(planBytes);
|
||||
List<HoodieWriteStat> updateStatusMap = writeStatuses.map(WriteStatus::getStat).collect();
|
||||
org.apache.hudi.common.model.HoodieCommitMetadata metadata = new org.apache.hudi.common.model.HoodieCommitMetadata(true);
|
||||
for (HoodieWriteStat stat : updateStatusMap) {
|
||||
metadata.addWriteStat(stat.getPartitionPath(), stat);
|
||||
}
|
||||
metadata.addMetadata(org.apache.hudi.common.model.HoodieCommitMetadata.SCHEMA_KEY, schema);
|
||||
if (compactionPlan.getExtraMetadata() != null) {
|
||||
compactionPlan.getExtraMetadata().forEach(metadata::addMetadata);
|
||||
}
|
||||
return metadata;
|
||||
}
|
||||
}
|
||||
@@ -1,96 +0,0 @@
|
||||
/*
|
||||
* 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.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.client.utils.SparkMemoryUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
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.HoodieWriteStat;
|
||||
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.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCompactionException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.BaseActionExecutor;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkRunCompactionActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, HoodieWriteMetadata<JavaRDD<WriteStatus>>> {
|
||||
|
||||
public SparkRunCompactionActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime) {
|
||||
super(context, config, table, instantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(instantTime);
|
||||
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
|
||||
if (!pendingCompactionTimeline.containsInstant(instant)) {
|
||||
throw new IllegalStateException(
|
||||
"No Compaction request available at " + instantTime + " to run compaction");
|
||||
}
|
||||
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = new HoodieWriteMetadata<>();
|
||||
try {
|
||||
HoodieActiveTimeline timeline = table.getActiveTimeline();
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.getCompactionPlan(table.getMetaClient(), instantTime);
|
||||
// Mark instant as compaction inflight
|
||||
timeline.transitionCompactionRequestedToInflight(instant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
|
||||
JavaRDD<WriteStatus> statuses = compactor.compact(context, compactionPlan, table, config, instantTime);
|
||||
|
||||
statuses.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Preparing compaction metadata");
|
||||
List<HoodieWriteStat> updateStatusMap = statuses.map(WriteStatus::getStat).collect();
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
|
||||
for (HoodieWriteStat stat : updateStatusMap) {
|
||||
metadata.addWriteStat(stat.getPartitionPath(), stat);
|
||||
}
|
||||
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema());
|
||||
|
||||
compactionMetadata.setWriteStatuses(statuses);
|
||||
compactionMetadata.setCommitted(false);
|
||||
compactionMetadata.setCommitMetadata(Option.of(metadata));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
|
||||
}
|
||||
|
||||
return compactionMetadata;
|
||||
}
|
||||
}
|
||||
@@ -1,152 +0,0 @@
|
||||
/*
|
||||
* 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.avro.model.HoodieCompactionPlan;
|
||||
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.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
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.table.view.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCompactionException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.ParseException;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkScheduleCompactionActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseScheduleCompactionActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkScheduleCompactionActionExecutor.class);
|
||||
|
||||
public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, instantTime, extraMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieCompactionPlan scheduleCompaction() {
|
||||
LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||
// judge if we need to compact according to num delta commits and time elapsed
|
||||
boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
|
||||
if (compactable) {
|
||||
LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
|
||||
HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
|
||||
try {
|
||||
SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
|
||||
Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
|
||||
.map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
|
||||
.collect(Collectors.toSet());
|
||||
// exclude files in pending clustering from compaction.
|
||||
fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
|
||||
return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
return new HoodieCompactionPlan();
|
||||
}
|
||||
|
||||
public Pair<Integer, String> getLatestDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
|
||||
Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
|
||||
|
||||
String latestInstantTs;
|
||||
int deltaCommitsSinceLastCompaction = 0;
|
||||
if (lastCompaction.isPresent()) {
|
||||
latestInstantTs = lastCompaction.get().getTimestamp();
|
||||
deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(latestInstantTs, Integer.MAX_VALUE).countInstants();
|
||||
} else {
|
||||
latestInstantTs = deltaCommits.firstInstant().get().getTimestamp();
|
||||
deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(latestInstantTs, Integer.MAX_VALUE).countInstants();
|
||||
}
|
||||
return Pair.of(deltaCommitsSinceLastCompaction, latestInstantTs);
|
||||
}
|
||||
|
||||
public boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) {
|
||||
boolean compactable;
|
||||
// get deltaCommitsSinceLastCompaction and lastCompactionTs
|
||||
Pair<Integer, String> latestDeltaCommitInfo = getLatestDeltaCommitInfo(compactionTriggerStrategy);
|
||||
int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax();
|
||||
int inlineCompactDeltaSecondsMax = config.getInlineCompactDeltaSecondsMax();
|
||||
switch (compactionTriggerStrategy) {
|
||||
case NUM_COMMITS:
|
||||
compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft();
|
||||
if (compactable) {
|
||||
LOG.info(String.format("The delta commits >= %s, trigger compaction scheduler.", inlineCompactDeltaCommitMax));
|
||||
}
|
||||
break;
|
||||
case TIME_ELAPSED:
|
||||
compactable = inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight());
|
||||
if (compactable) {
|
||||
LOG.info(String.format("The elapsed time >=%ss, trigger compaction scheduler.", inlineCompactDeltaSecondsMax));
|
||||
}
|
||||
break;
|
||||
case NUM_OR_TIME:
|
||||
compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft()
|
||||
|| inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight());
|
||||
if (compactable) {
|
||||
LOG.info(String.format("The delta commits >= %s or elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax,
|
||||
inlineCompactDeltaSecondsMax));
|
||||
}
|
||||
break;
|
||||
case NUM_AND_TIME:
|
||||
compactable = inlineCompactDeltaCommitMax <= latestDeltaCommitInfo.getLeft()
|
||||
&& inlineCompactDeltaSecondsMax <= parsedToSeconds(instantTime) - parsedToSeconds(latestDeltaCommitInfo.getRight());
|
||||
if (compactable) {
|
||||
LOG.info(String.format("The delta commits >= %s and elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax,
|
||||
inlineCompactDeltaSecondsMax));
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new HoodieCompactionException("Unsupported compaction trigger strategy: " + config.getInlineCompactTriggerStrategy());
|
||||
}
|
||||
return compactable;
|
||||
}
|
||||
|
||||
public Long parsedToSeconds(String time) {
|
||||
long timestamp;
|
||||
try {
|
||||
timestamp = HoodieActiveTimeline.COMMIT_FORMATTER.parse(time).getTime() / 1000;
|
||||
} catch (ParseException e) {
|
||||
throw new HoodieCompactionException(e.getMessage(), e);
|
||||
}
|
||||
return timestamp;
|
||||
}
|
||||
}
|
||||
@@ -89,8 +89,8 @@ public class TestAsyncCompaction extends CompactionTestBase {
|
||||
metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build();
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
|
||||
client.rollbackInflightCompaction(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
|
||||
hoodieTable.rollbackInflightCompaction(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime));
|
||||
metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build();
|
||||
pendingCompactionInstant = metaClient.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline()
|
||||
.getInstants().findFirst().get();
|
||||
|
||||
@@ -208,7 +208,8 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
String compactionInstantTime = "102";
|
||||
table.scheduleCompaction(context, compactionInstantTime, Option.empty());
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
JavaRDD<WriteStatus> result = (JavaRDD<WriteStatus>) table.compact(context, compactionInstantTime).getWriteStatuses();
|
||||
JavaRDD<WriteStatus> result = (JavaRDD<WriteStatus>) table.compact(
|
||||
context, compactionInstantTime).getWriteStatuses();
|
||||
|
||||
// Verify that all partition paths are present in the WriteStatus result
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
|
||||
@@ -279,8 +279,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
||||
final String compactedCommitTime = metaClient.getActiveTimeline().reload().lastInstant().get().getTimestamp();
|
||||
assertTrue(Arrays.stream(listAllBaseFilesInPath(hoodieTable))
|
||||
.anyMatch(file -> compactedCommitTime.equals(new HoodieBaseFile(file).getCommitTime())));
|
||||
thirdClient.rollbackInflightCompaction(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactedCommitTime),
|
||||
hoodieTable);
|
||||
hoodieTable.rollbackInflightCompaction(new HoodieInstant(
|
||||
HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactedCommitTime));
|
||||
allFiles = listAllBaseFilesInPath(hoodieTable);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
tableView = getHoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
|
||||
@@ -611,7 +611,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
||||
//writeClient.commitCompaction(newCommitTime, statuses, Option.empty());
|
||||
// Trigger a rollback of compaction
|
||||
table.getActiveTimeline().reload();
|
||||
writeClient.rollbackInflightCompaction(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, newCommitTime), table);
|
||||
table.rollbackInflightCompaction(new HoodieInstant(
|
||||
HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, newCommitTime));
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieSparkTable.create(config, context(), metaClient);
|
||||
@@ -619,7 +620,7 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
||||
((SyncableFileSystemView) tableRTFileSystemView).reset();
|
||||
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> fileSlices = getFileSystemViewWithUnCommittedSlices(metaClient)
|
||||
List<FileSlice> fileSlices = getFileSystemViewWithUnCommittedSlices(metaClient)
|
||||
.getAllFileSlices(partitionPath).filter(fs -> fs.getBaseInstantTime().equals("100")).collect(Collectors.toList());
|
||||
assertTrue(fileSlices.stream().noneMatch(fileSlice -> fileSlice.getBaseFile().isPresent()));
|
||||
assertTrue(fileSlices.stream().anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0));
|
||||
|
||||
Reference in New Issue
Block a user