[HUDI-2258] Metadata table for flink (#3381)
This commit is contained in:
@@ -248,7 +248,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
||||
if (!latestMetadataInstant.isPresent()) {
|
||||
LOG.warn("Metadata Table will need to be re-bootstrapped as no instants were found");
|
||||
rebootstrap = true;
|
||||
} else if (datasetMetaClient.getActiveTimeline().isBeforeTimelineStarts(latestMetadataInstant.get().getTimestamp())) {
|
||||
} else if (!latestMetadataInstant.get().getTimestamp().equals(SOLO_COMMIT_TIMESTAMP)
|
||||
&& datasetMetaClient.getActiveTimeline().isBeforeTimelineStarts(latestMetadataInstant.get().getTimestamp())) {
|
||||
LOG.warn("Metadata Table will need to be re-bootstrapped as un-synced instants have been archived."
|
||||
+ " latestMetadataInstant=" + latestMetadataInstant.get().getTimestamp()
|
||||
+ ", latestDatasetInstant=" + datasetMetaClient.getActiveTimeline().firstInstant().get().getTimestamp());
|
||||
|
||||
@@ -40,7 +40,9 @@ import org.apache.hudi.common.util.CommitUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieMetadataException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.index.FlinkHoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
@@ -50,6 +52,8 @@ import org.apache.hudi.io.FlinkMergeAndReplaceHandle;
|
||||
import org.apache.hudi.io.FlinkMergeHandle;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.io.MiniBatchHandle;
|
||||
import org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
@@ -153,7 +157,15 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public List<WriteStatus> upsertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
|
||||
throw new HoodieNotSupportedException("UpsertPrepped operation is not supported yet");
|
||||
// only used for metadata table, the upsert happens in single thread
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
|
||||
getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
|
||||
table.validateUpsertSchema();
|
||||
preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, table.getMetaClient());
|
||||
final HoodieWriteHandle<?, ?, ?, ?> writeHandle = getOrCreateWriteHandle(preppedRecords.get(0), getConfig(),
|
||||
instantTime, table, preppedRecords.listIterator());
|
||||
HoodieWriteMetadata<List<WriteStatus>> result = ((HoodieFlinkTable<T>) table).upsertPrepped(context, writeHandle, instantTime, preppedRecords);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -355,7 +367,14 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
protected List<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported yet");
|
||||
// only used for metadata table, the compaction happens in single thread
|
||||
try {
|
||||
List<WriteStatus> writeStatuses = FlinkCompactHelpers.compact(compactionInstantTime, this);
|
||||
commitCompaction(compactionInstantTime, writeStatuses, Option.empty());
|
||||
return writeStatuses;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error while compacting instant: " + compactionInstantTime);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -370,6 +389,16 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
return getTableAndInitCtx(metaClient, operationType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void syncTableMetadata() {
|
||||
// Open up the metadata table again, for syncing
|
||||
try (HoodieTableMetadataWriter writer = FlinkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) {
|
||||
LOG.info("Successfully synced to metadata table");
|
||||
} catch (Exception e) {
|
||||
throw new HoodieMetadataException("Error syncing to metadata table.", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean the write handles within a checkpoint interval.
|
||||
* All the handles should have been closed already.
|
||||
|
||||
@@ -32,6 +32,7 @@ import org.apache.flink.api.common.functions.RuntimeContext;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -47,7 +48,13 @@ import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper
|
||||
* A flink engine implementation of HoodieEngineContext.
|
||||
*/
|
||||
public class HoodieFlinkEngineContext extends HoodieEngineContext {
|
||||
private RuntimeContext runtimeContext;
|
||||
public static final HoodieFlinkEngineContext DEFAULT = new HoodieFlinkEngineContext();
|
||||
|
||||
private final RuntimeContext runtimeContext;
|
||||
|
||||
private HoodieFlinkEngineContext() {
|
||||
this(new SerializableConfiguration(FlinkClientUtil.getHadoopConf()), new DefaultTaskContextSupplier());
|
||||
}
|
||||
|
||||
public HoodieFlinkEngineContext(TaskContextSupplier taskContextSupplier) {
|
||||
this(new SerializableConfiguration(FlinkClientUtil.getHadoopConf()), taskContextSupplier);
|
||||
@@ -97,4 +104,34 @@ public class HoodieFlinkEngineContext extends HoodieEngineContext {
|
||||
public void setJobStatus(String activeModule, String activityDescription) {
|
||||
// no operation for now
|
||||
}
|
||||
|
||||
/**
|
||||
* Override the flink context supplier to return constant write token.
|
||||
*/
|
||||
private static class DefaultTaskContextSupplier extends FlinkTaskContextSupplier {
|
||||
|
||||
public DefaultTaskContextSupplier() {
|
||||
this(null);
|
||||
}
|
||||
|
||||
public DefaultTaskContextSupplier(RuntimeContext flinkRuntimeContext) {
|
||||
super(flinkRuntimeContext);
|
||||
}
|
||||
|
||||
public Supplier<Integer> getPartitionIdSupplier() {
|
||||
return () -> 0;
|
||||
}
|
||||
|
||||
public Supplier<Integer> getStageIdSupplier() {
|
||||
return () -> 1;
|
||||
}
|
||||
|
||||
public Supplier<Long> getAttemptIdSupplier() {
|
||||
return () -> 0L;
|
||||
}
|
||||
|
||||
public Option<String> getProperty(EngineProperty prop) {
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,191 @@
|
||||
/*
|
||||
* 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.metadata;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.metrics.Registry;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieMetadataException;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FlinkHoodieBackedTableMetadataWriter.class);
|
||||
|
||||
public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig, HoodieEngineContext context) {
|
||||
return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, context);
|
||||
}
|
||||
|
||||
FlinkHoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) {
|
||||
super(hadoopConf, writeConfig, engineContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void initRegistry() {
|
||||
if (metadataWriteConfig.isMetricsOn()) {
|
||||
// should support executor metrics
|
||||
Registry registry = Registry.getRegistry("HoodieMetadata");
|
||||
this.metrics = Option.of(new HoodieMetadataMetrics(registry));
|
||||
} else {
|
||||
this.metrics = Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) {
|
||||
try {
|
||||
if (enabled) {
|
||||
bootstrapIfNeeded(engineContext, datasetMetaClient);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to initialize metadata table. Disabling the writer.", e);
|
||||
enabled = false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime) {
|
||||
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
|
||||
List<HoodieRecord> recordRDD = prepRecords(records, partitionName);
|
||||
|
||||
try (HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient(engineContext, metadataWriteConfig, true)) {
|
||||
writeClient.startCommitWithTime(instantTime);
|
||||
writeClient.transitionRequestedToInflight(HoodieActiveTimeline.DELTA_COMMIT_ACTION, instantTime);
|
||||
|
||||
List<WriteStatus> statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime);
|
||||
statuses.forEach(writeStatus -> {
|
||||
if (writeStatus.hasErrors()) {
|
||||
throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime);
|
||||
}
|
||||
});
|
||||
writeClient.commit(instantTime, statuses, Option.empty(), HoodieActiveTimeline.DELTA_COMMIT_ACTION, Collections.emptyMap());
|
||||
// trigger cleaning, compaction, with suffixes based on the same instant time. This ensures that any future
|
||||
// delta commits synced over will not have an instant time lesser than the last completed instant on the
|
||||
// metadata table.
|
||||
if (writeClient.scheduleCompactionAtInstant(instantTime + "001", Option.empty())) {
|
||||
writeClient.compact(instantTime + "001");
|
||||
}
|
||||
writeClient.clean(instantTime + "002");
|
||||
}
|
||||
|
||||
// Update total size of the metadata and count of base/log files
|
||||
metrics.ifPresent(m -> {
|
||||
try {
|
||||
Map<String, String> stats = m.getStats(false, metaClient, metadata);
|
||||
m.updateMetrics(Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE)),
|
||||
Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE)),
|
||||
Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES)),
|
||||
Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES)));
|
||||
} catch (HoodieIOException e) {
|
||||
LOG.error("Could not publish metadata size metrics", e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the timestamp of the latest instant synced.
|
||||
* <p>
|
||||
* To sync a instant on dataset, we create a corresponding delta-commit on the metadata table. So return the latest
|
||||
* delta-commit.
|
||||
*/
|
||||
@Override
|
||||
public Option<String> getLatestSyncedInstantTime() {
|
||||
if (!enabled) {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline();
|
||||
return timeline.getDeltaCommitTimeline().filterCompletedInstants()
|
||||
.lastInstant().map(HoodieInstant::getTimestamp);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tag each record with the location.
|
||||
* <p>
|
||||
* Since we only read the latest base file in a partition, we tag the records with the instant time of the latest
|
||||
* base file.
|
||||
*/
|
||||
private List<HoodieRecord> prepRecords(List<HoodieRecord> records, String partitionName) {
|
||||
HoodieTable table = HoodieFlinkTable.create(metadataWriteConfig, (HoodieFlinkEngineContext) engineContext);
|
||||
TableFileSystemView.SliceView fsView = table.getSliceView();
|
||||
List<HoodieBaseFile> baseFiles = fsView.getLatestFileSlices(partitionName)
|
||||
.map(FileSlice::getBaseFile)
|
||||
.filter(Option::isPresent)
|
||||
.map(Option::get)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// All the metadata fits within a single base file
|
||||
if (partitionName.equals(MetadataPartitionType.FILES.partitionPath())) {
|
||||
if (baseFiles.size() > 1) {
|
||||
throw new HoodieMetadataException("Multiple base files found in metadata partition");
|
||||
}
|
||||
}
|
||||
|
||||
String fileId;
|
||||
String instantTime;
|
||||
if (!baseFiles.isEmpty()) {
|
||||
fileId = baseFiles.get(0).getFileId();
|
||||
instantTime = "U";
|
||||
} else {
|
||||
// If there is a log file then we can assume that it has the data
|
||||
List<HoodieLogFile> logFiles = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath())
|
||||
.map(FileSlice::getLatestLogFile)
|
||||
.filter(Option::isPresent)
|
||||
.map(Option::get)
|
||||
.collect(Collectors.toList());
|
||||
if (logFiles.isEmpty()) {
|
||||
// No base and log files. All are new inserts
|
||||
fileId = FSUtils.createNewFileIdPfx();
|
||||
instantTime = "I";
|
||||
} else {
|
||||
fileId = logFiles.get(0).getFileId();
|
||||
instantTime = "U";
|
||||
}
|
||||
}
|
||||
|
||||
return records.stream().map(r -> r.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId))).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
@@ -34,6 +34,7 @@ import org.apache.hudi.io.FlinkAppendHandle;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.delta.FlinkUpsertDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.delta.FlinkUpsertPreppedDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.FlinkScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.FlinkMergeOnReadRollbackActionExecutor;
|
||||
@@ -63,6 +64,18 @@ public class HoodieFlinkMergeOnReadTable<T extends HoodieRecordPayload>
|
||||
return new FlinkUpsertDeltaCommitActionExecutor<>(context, appendHandle, config, this, instantTime, hoodieRecords).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> preppedRecords) {
|
||||
ValidationUtils.checkArgument(writeHandle instanceof FlinkAppendHandle,
|
||||
"MOR write handle should always be a FlinkAppendHandle");
|
||||
FlinkAppendHandle<?, ?, ?, ?> appendHandle = (FlinkAppendHandle<?, ?, ?, ?>) writeHandle;
|
||||
return new FlinkUpsertPreppedDeltaCommitActionExecutor<>(context, appendHandle, config, this, instantTime, preppedRecords).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> insert(
|
||||
HoodieEngineContext context,
|
||||
|
||||
@@ -32,7 +32,7 @@ import java.util.List;
|
||||
|
||||
public class FlinkUpsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseFlinkDeltaCommitActionExecutor<T> {
|
||||
private List<HoodieRecord<T>> inputRecords;
|
||||
private final List<HoodieRecord<T>> inputRecords;
|
||||
|
||||
public FlinkUpsertDeltaCommitActionExecutor(HoodieEngineContext context,
|
||||
FlinkAppendHandle<?, ?, ?, ?> writeHandle,
|
||||
|
||||
@@ -0,0 +1,52 @@
|
||||
/*
|
||||
* 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.commit.delta;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.FlinkAppendHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class FlinkUpsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseFlinkDeltaCommitActionExecutor<T> {
|
||||
|
||||
private final List<HoodieRecord<T>> preppedRecords;
|
||||
|
||||
public FlinkUpsertPreppedDeltaCommitActionExecutor(HoodieEngineContext context,
|
||||
FlinkAppendHandle<?, ?, ?, ?> writeHandle,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> preppedRecords) {
|
||||
super(context, writeHandle, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
|
||||
this.preppedRecords = preppedRecords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> execute() {
|
||||
return super.execute(preppedRecords);
|
||||
}
|
||||
}
|
||||
@@ -19,20 +19,33 @@
|
||||
package org.apache.hudi.table.action.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
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.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* A flink implementation of {@link AbstractCompactHelpers}.
|
||||
*
|
||||
@@ -40,6 +53,7 @@ import java.util.stream.Collectors;
|
||||
*/
|
||||
public class FlinkCompactHelpers<T extends HoodieRecordPayload> extends
|
||||
AbstractCompactHelpers<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(FlinkCompactHelpers.class);
|
||||
|
||||
private FlinkCompactHelpers() {
|
||||
}
|
||||
@@ -71,5 +85,63 @@ public class FlinkCompactHelpers<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
return metadata;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked, rawtypes")
|
||||
public static List<WriteStatus> compact(
|
||||
HoodieFlinkWriteClient writeClient,
|
||||
String compactInstantTime,
|
||||
CompactionOperation compactionOperation) throws IOException {
|
||||
HoodieFlinkMergeOnReadTableCompactor compactor = new HoodieFlinkMergeOnReadTableCompactor();
|
||||
return compactor.compact(
|
||||
new HoodieFlinkCopyOnWriteTable<>(
|
||||
writeClient.getConfig(),
|
||||
writeClient.getEngineContext(),
|
||||
writeClient.getHoodieTable().getMetaClient()),
|
||||
writeClient.getHoodieTable().getMetaClient(),
|
||||
writeClient.getConfig(),
|
||||
compactionOperation,
|
||||
compactInstantTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the metadata table compactor code path.
|
||||
*/
|
||||
@SuppressWarnings("unchecked, rawtypes")
|
||||
public static List<WriteStatus> compact(String compactionInstantTime, HoodieFlinkWriteClient writeClient) throws IOException {
|
||||
HoodieFlinkTable table = writeClient.getHoodieTable();
|
||||
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
|
||||
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
|
||||
if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
|
||||
writeClient.rollbackInflightCompaction(inflightInstant, table);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
|
||||
// generate compaction plan
|
||||
// should support configurable commit metadata
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(
|
||||
table.getMetaClient(), compactionInstantTime);
|
||||
|
||||
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|
||||
|| (compactionPlan.getOperations().isEmpty())) {
|
||||
// do nothing.
|
||||
LOG.info("No compaction plan for instant " + compactionInstantTime);
|
||||
return Collections.emptyList();
|
||||
} else {
|
||||
HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
// Mark instant as compaction inflight
|
||||
table.getActiveTimeline().transitionCompactionRequestedToInflight(instant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
List<CompactionOperation> operations = compactionPlan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
LOG.info("Compacting " + operations + " files");
|
||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||
for (CompactionOperation operation : operations) {
|
||||
List<WriteStatus> statuses = compact(writeClient, compactionInstantTime, operation);
|
||||
writeStatusList.addAll(statuses);
|
||||
}
|
||||
return writeStatusList;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -21,7 +21,10 @@ package org.apache.hudi.table.action.rollback;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
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.IOType;
|
||||
@@ -31,7 +34,11 @@ import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
@@ -71,4 +78,11 @@ public class FlinkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> ext
|
||||
throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected Map<FileStatus, Long> getWrittenLogFileSizeMap(String partitionPathStr, String baseCommitTime, String fileId) throws IOException {
|
||||
// collect all log files that is supposed to be deleted with this rollback
|
||||
return FSUtils.getAllLogFiles(table.getMetaClient().getFs(),
|
||||
FSUtils.getPartitionPath(config.getBasePath(), partitionPathStr), fileId, HoodieFileFormat.HOODIE_LOG.getFileExtension(), baseCommitTime)
|
||||
.collect(Collectors.toMap(HoodieLogFile::getFileStatus, value -> value.getFileStatus().getLen()));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.table.action.rollback;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
@@ -116,12 +117,22 @@ public class ListingBasedRollbackHelper implements Serializable {
|
||||
.withDeletedFileResults(filesToDeletedStatus).build());
|
||||
}
|
||||
case APPEND_ROLLBACK_BLOCK: {
|
||||
String fileId = rollbackRequest.getFileId().get();
|
||||
String latestBaseInstant = rollbackRequest.getLatestBaseInstant().get();
|
||||
|
||||
// collect all log files that is supposed to be deleted with this rollback
|
||||
Map<FileStatus, Long> writtenLogFileSizeMap = FSUtils.getAllLogFiles(metaClient.getFs(),
|
||||
FSUtils.getPartitionPath(config.getBasePath(), rollbackRequest.getPartitionPath()),
|
||||
fileId, HoodieFileFormat.HOODIE_LOG.getFileExtension(), latestBaseInstant)
|
||||
.collect(Collectors.toMap(HoodieLogFile::getFileStatus, value -> value.getFileStatus().getLen()));
|
||||
|
||||
HoodieLogFormat.Writer writer = null;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(FSUtils.getPartitionPath(metaClient.getBasePath(), rollbackRequest.getPartitionPath()))
|
||||
.withFileId(rollbackRequest.getFileId().get())
|
||||
.overBaseCommit(rollbackRequest.getLatestBaseInstant().get()).withFs(metaClient.getFs())
|
||||
.withFileId(fileId)
|
||||
.overBaseCommit(latestBaseInstant)
|
||||
.withFs(metaClient.getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
|
||||
// generate metadata
|
||||
@@ -151,7 +162,8 @@ public class ListingBasedRollbackHelper implements Serializable {
|
||||
);
|
||||
return new ImmutablePair<>(rollbackRequest.getPartitionPath(),
|
||||
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build());
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback)
|
||||
.withWrittenLogFileSizeMap(writtenLogFileSizeMap).build());
|
||||
}
|
||||
default:
|
||||
throw new IllegalStateException("Unknown Rollback action " + rollbackRequest);
|
||||
|
||||
@@ -38,14 +38,6 @@ public class FlinkClientUtil {
|
||||
return HoodieTableMetaClient.builder().setBasePath(basePath).setConf(FlinkClientUtil.getHadoopConf()).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses the file name from path.
|
||||
*/
|
||||
public static String parseFileName(String path) {
|
||||
int slash = path.lastIndexOf(Path.SEPARATOR);
|
||||
return path.substring(slash + 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the hadoop configuration with possible hadoop conf paths.
|
||||
* E.G. the configurations under path $HADOOP_CONF_DIR and $HADOOP_HOME.
|
||||
|
||||
Reference in New Issue
Block a user