1
0

[HUDI-1327] Introduce base implemetation of hudi-flink-client (#2176)

This commit is contained in:
wangxianghu
2020-11-18 17:57:11 +08:00
committed by GitHub
parent 430d4b428e
commit 4d05680038
48 changed files with 4597 additions and 38 deletions

View File

@@ -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;
import org.apache.hudi.client.common.EngineProperty;
import org.apache.hudi.client.common.TaskContextSupplier;
import org.apache.hudi.common.util.Option;
import org.apache.flink.api.common.functions.RuntimeContext;
import java.util.function.Supplier;
/**
* Flink task context supplier.
*/
public class FlinkTaskContextSupplier extends TaskContextSupplier {
private RuntimeContext flinkRuntimeContext;
public FlinkTaskContextSupplier(RuntimeContext flinkRuntimeContext) {
this.flinkRuntimeContext = flinkRuntimeContext;
}
public RuntimeContext getFlinkRuntimeContext() {
return flinkRuntimeContext;
}
@Override
public Supplier<Integer> getPartitionIdSupplier() {
return () -> this.flinkRuntimeContext.getIndexOfThisSubtask();
}
@Override
public Supplier<Integer> getStageIdSupplier() {
// need to check again
return () -> this.flinkRuntimeContext.getNumberOfParallelSubtasks();
}
@Override
public Supplier<Long> getAttemptIdSupplier() {
return () -> (long) this.flinkRuntimeContext.getAttemptNumber();
}
@Override
public Option<String> getProperty(EngineProperty prop) {
// no operation for now
return Option.empty();
}
}

View File

@@ -0,0 +1,213 @@
/*
* 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;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
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.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTableVersion;
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.exception.HoodieNotSupportedException;
import org.apache.hudi.index.FlinkHoodieIndex;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
import com.codahale.metrics.Timer;
import org.apache.hadoop.conf.Configuration;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@SuppressWarnings("checkstyle:LineLength")
public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
AbstractHoodieWriteClient<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
super(context, clientConfig);
}
public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
super(context, writeConfig, rollbackPending);
}
public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
Option<EmbeddedTimelineService> timelineService) {
super(context, writeConfig, rollbackPending, timelineService);
}
/**
* Complete changes performed at the given instantTime marker with specified action.
*/
@Override
protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config);
}
@Override
public boolean commit(String instantTime, List<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata, String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
List<HoodieWriteStat> writeStats = writeStatuses.parallelStream().map(WriteStatus::getStat).collect(Collectors.toList());
return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
}
@Override
protected HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> createTable(HoodieWriteConfig config, Configuration hadoopConf) {
return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
}
@Override
public List<HoodieRecord<T>> filterExists(List<HoodieRecord<T>> hoodieRecords) {
// Create a Hoodie table which encapsulated the commits and files visible
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
Timer.Context indexTimer = metrics.getIndexCtx();
List<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList());
}
@Override
public List<WriteStatus> upsert(List<HoodieRecord<T>> records, String instantTime) {
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
table.validateUpsertSchema();
setOperationType(WriteOperationType.UPSERT);
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
HoodieWriteMetadata<List<WriteStatus>> result = table.upsert(context, instantTime, records);
if (result.getIndexLookupDuration().isPresent()) {
metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
}
return postWrite(result, instantTime, table);
}
@Override
public List<WriteStatus> upsertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
throw new HoodieNotSupportedException("UpsertPrepped operation is not supported yet");
}
@Override
public List<WriteStatus> insert(List<HoodieRecord<T>> records, String instantTime) {
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
table.validateUpsertSchema();
setOperationType(WriteOperationType.INSERT);
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
HoodieWriteMetadata<List<WriteStatus>> result = table.insert(context, instantTime, records);
if (result.getIndexLookupDuration().isPresent()) {
metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
}
return postWrite(result, instantTime, table);
}
@Override
public List<WriteStatus> insertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime) {
throw new HoodieNotSupportedException("InsertPrepped operation is not supported yet");
}
@Override
public List<WriteStatus> bulkInsert(List<HoodieRecord<T>> records, String instantTime) {
throw new HoodieNotSupportedException("BulkInsert operation is not supported yet");
}
@Override
public List<WriteStatus> bulkInsert(List<HoodieRecord<T>> records, String instantTime, Option<BulkInsertPartitioner<List<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
throw new HoodieNotSupportedException("BulkInsert operation is not supported yet");
}
@Override
public List<WriteStatus> bulkInsertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime, Option<BulkInsertPartitioner<List<HoodieRecord<T>>>> bulkInsertPartitioner) {
throw new HoodieNotSupportedException("BulkInsertPrepped operation is not supported yet");
}
@Override
public List<WriteStatus> delete(List<HoodieKey> keys, String instantTime) {
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.DELETE, instantTime);
setOperationType(WriteOperationType.DELETE);
HoodieWriteMetadata<List<WriteStatus>> result = table.delete(context,instantTime, keys);
return postWrite(result, instantTime, table);
}
@Override
protected List<WriteStatus> postWrite(HoodieWriteMetadata<List<WriteStatus>> result,
String instantTime,
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) {
if (result.getIndexLookupDuration().isPresent()) {
metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis());
}
return result.getWriteStatuses();
}
@Override
public void commitCompaction(String compactionInstantTime, List<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata) throws IOException {
throw new HoodieNotSupportedException("Compaction is not supported yet");
}
@Override
protected void completeCompaction(HoodieCommitMetadata metadata, List<WriteStatus> writeStatuses, HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table, String compactionCommitTime) {
throw new HoodieNotSupportedException("Compaction is not supported yet");
}
@Override
protected List<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) {
throw new HoodieNotSupportedException("Compaction is not supported yet");
}
@Override
protected HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> getTableAndInitCtx(WriteOperationType operationType, String instantTime) {
HoodieTableMetaClient metaClient = createMetaClient(true);
new FlinkUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime);
return getTableAndInitCtx(metaClient, operationType);
}
private HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) {
if (operationType == WriteOperationType.DELETE) {
setWriteSchemaForDeletes(metaClient);
}
// Create a Hoodie table which encapsulated the commits and files visible
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context, metaClient);
if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
writeTimer = metrics.getCommitCtx();
} else {
writeTimer = metrics.getDeltaCommitCtx();
}
return table;
}
public List<String> getInflightsAndRequestedInstants(String commitType) {
HoodieFlinkTable<T> table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieTimeline unCompletedTimeline = table.getMetaClient().getCommitsTimeline().filterInflightsAndRequested();
return unCompletedTimeline.getInstants().filter(x -> x.getAction().equals(commitType)).map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
}
}

View File

@@ -18,14 +18,16 @@
package org.apache.hudi.client.common;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.client.FlinkTaskContextSupplier;
import org.apache.hudi.client.common.function.SerializableConsumer;
import org.apache.hudi.client.common.function.SerializableFunction;
import org.apache.hudi.client.common.function.SerializablePairFunction;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.util.Option;
import scala.Tuple2;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@@ -40,6 +42,7 @@ import static org.apache.hudi.client.common.function.FunctionWrapper.throwingMap
* A flink engine implementation of HoodieEngineContext.
*/
public class HoodieFlinkEngineContext extends HoodieEngineContext {
private RuntimeContext runtimeContext;
public HoodieFlinkEngineContext(TaskContextSupplier taskContextSupplier) {
this(new SerializableConfiguration(new Configuration()), taskContextSupplier);
@@ -47,6 +50,11 @@ public class HoodieFlinkEngineContext extends HoodieEngineContext {
public HoodieFlinkEngineContext(SerializableConfiguration hadoopConf, TaskContextSupplier taskContextSupplier) {
super(hadoopConf, taskContextSupplier);
this.runtimeContext = ((FlinkTaskContextSupplier) taskContextSupplier).getFlinkRuntimeContext();
}
public RuntimeContext getRuntimeContext() {
return this.runtimeContext;
}
@Override
@@ -66,9 +74,7 @@ public class HoodieFlinkEngineContext extends HoodieEngineContext {
@Override
public <I, K, V> Map<K, V> mapToPair(List<I> data, SerializablePairFunction<I, K, V> func, Integer parallelism) {
Map<K, V> map = new HashMap<>();
data.stream().map(throwingMapToPairWrapper(func)).forEach(x -> map.put(x._1, x._2));
return map;
return data.stream().map(throwingMapToPairWrapper(func)).collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
}
@Override

View File

@@ -19,12 +19,13 @@
package org.apache.hudi.client.common.function;
import org.apache.hudi.exception.HoodieException;
import scala.Tuple2;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Stream;
import scala.Tuple2;
/**
* Function wrapper util class, which catches the exception thrown by input function and return a similar function
* with no exception thrown.

View File

@@ -0,0 +1,80 @@
/*
* 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;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.TaskContextSupplier;
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.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.io.WriteHandleFactory;
import org.apache.hudi.table.HoodieTable;
import org.apache.avro.Schema;
import java.util.Iterator;
import java.util.List;
public class FlinkLazyInsertIterable<T extends HoodieRecordPayload> extends HoodieLazyInsertIterable<T> {
public FlinkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
boolean areRecordsSorted,
HoodieWriteConfig config,
String instantTime,
HoodieTable hoodieTable,
String idPrefix,
TaskContextSupplier taskContextSupplier) {
super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier);
}
public FlinkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
boolean areRecordsSorted,
HoodieWriteConfig config,
String instantTime,
HoodieTable hoodieTable,
String idPrefix,
TaskContextSupplier taskContextSupplier,
WriteHandleFactory writeHandleFactory) {
super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, writeHandleFactory);
}
@Override
protected List<WriteStatus> computeNext() {
// Executor service used for launching writer thread.
BoundedInMemoryExecutor<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor =
null;
try {
final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
bufferedIteratorExecutor =
new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(inputItr), Option.of(getInsertHandler()), getTransformFunction(schema));
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
return result;
} catch (Exception e) {
throw new HoodieException(e);
} finally {
if (null != bufferedIteratorExecutor) {
bufferedIteratorExecutor.shutdownNow();
}
}
}
}

View File

@@ -0,0 +1,77 @@
/*
* 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.index;
import org.apache.hudi.ApiMaturityLevel;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
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.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.index.state.FlinkInMemoryStateIndex;
import org.apache.hudi.table.HoodieTable;
import java.util.List;
/**
* Base flink implementation of {@link HoodieIndex}.
* @param <T> payload type
*/
public abstract class FlinkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
protected FlinkHoodieIndex(HoodieWriteConfig config) {
super(config);
}
public static FlinkHoodieIndex createIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) {
// first use index class config to create index.
if (!StringUtils.isNullOrEmpty(config.getIndexClass())) {
Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config);
if (!(instance instanceof HoodieIndex)) {
throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex");
}
return (FlinkHoodieIndex) instance;
}
// TODO more indexes to be added
switch (config.getIndexType()) {
case INMEMORY:
return new FlinkInMemoryStateIndex<>(context, config);
default:
throw new HoodieIndexException("Unsupported index type " + config.getIndexType());
}
}
@Override
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
public abstract List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
HoodieEngineContext context,
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
@Override
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
public abstract List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
HoodieEngineContext context,
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException;
}

View File

@@ -0,0 +1,135 @@
/*
* 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.index.state;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIndexException;
import org.apache.hudi.index.FlinkHoodieIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.flink.api.common.state.MapState;
import org.apache.flink.api.common.state.MapStateDescriptor;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.List;
/**
* Hoodie index implementation backed by flink state.
*
* @param <T> type of payload
*/
public class FlinkInMemoryStateIndex<T extends HoodieRecordPayload> extends FlinkHoodieIndex<T> {
private static final Logger LOG = LogManager.getLogger(FlinkInMemoryStateIndex.class);
private MapState<HoodieKey, HoodieRecordLocation> mapState;
public FlinkInMemoryStateIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) {
super(config);
if (context.getRuntimeContext() != null) {
MapStateDescriptor<HoodieKey, HoodieRecordLocation> indexStateDesc =
new MapStateDescriptor<>("indexState", TypeInformation.of(HoodieKey.class), TypeInformation.of(HoodieRecordLocation.class));
if (context.getRuntimeContext() != null) {
mapState = context.getRuntimeContext().getMapState(indexStateDesc);
}
}
}
@Override
public List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records,
HoodieEngineContext context,
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
return context.map(records, record -> {
try {
if (mapState.contains(record.getKey())) {
record.unseal();
record.setCurrentLocation(mapState.get(record.getKey()));
record.seal();
}
} catch (Exception e) {
LOG.error(String.format("Tag record location failed, key = %s, %s", record.getRecordKey(), e.getMessage()));
}
return record;
}, 0);
}
@Override
public List<WriteStatus> updateLocation(List<WriteStatus> writeStatuses,
HoodieEngineContext context,
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) throws HoodieIndexException {
return context.map(writeStatuses, writeStatus -> {
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
if (!writeStatus.isErrored(record.getKey())) {
HoodieKey key = record.getKey();
Option<HoodieRecordLocation> newLocation = record.getNewLocation();
if (newLocation.isPresent()) {
try {
mapState.put(key, newLocation.get());
} catch (Exception e) {
LOG.error(String.format("Update record location failed, key = %s, %s", record.getRecordKey(), e.getMessage()));
}
} else {
// Delete existing index for a deleted record
try {
mapState.remove(key);
} catch (Exception e) {
LOG.error(String.format("Remove record location failed, key = %s, %s", record.getRecordKey(), e.getMessage()));
}
}
}
}
return writeStatus;
}, 0);
}
@Override
public boolean rollbackCommit(String instantTime) {
return true;
}
/**
* Only looks up by recordKey.
*/
@Override
public boolean isGlobal() {
return true;
}
@Override
public boolean canIndexLogFiles() {
return true;
}
/**
* Index needs to be explicitly updated after storage write.
*/
@Override
public boolean isImplicitWithStorage() {
return false;
}
}

View File

@@ -0,0 +1,140 @@
/*
* 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;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.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.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
import org.apache.hudi.table.action.clean.FlinkCleanActionExecutor;
import org.apache.hudi.table.action.commit.FlinkDeleteCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkInsertCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkInsertPreppedCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkUpsertCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkUpsertPreppedCommitActionExecutor;
import org.apache.hudi.table.action.rollback.FlinkCopyOnWriteRollbackActionExecutor;
import java.util.List;
import java.util.Map;
public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieFlinkTable<T> {
protected HoodieFlinkCopyOnWriteTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
super(config, context, metaClient);
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> upsert(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> records) {
return new FlinkUpsertCommitActionExecutor<>(context, config, this, instantTime, records).execute();
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> insert(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> records) {
return new FlinkInsertCommitActionExecutor<>(context, config, this, instantTime, records).execute();
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> bulkInsert(HoodieEngineContext context,
String instantTime,
List<HoodieRecord<T>> records,
Option<BulkInsertPartitioner<List<HoodieRecord<T>>>> bulkInsertPartitioner) {
throw new HoodieNotSupportedException("BulkInsert is not supported yet");
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> delete(HoodieEngineContext context, String instantTime, List<HoodieKey> keys) {
return new FlinkDeleteCommitActionExecutor<>(context, config, this, instantTime, keys).execute();
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> preppedRecords) {
return new FlinkUpsertPreppedCommitActionExecutor<>(context, config, this, instantTime, preppedRecords).execute();
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> insertPrepped(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> preppedRecords) {
return new FlinkInsertPreppedCommitActionExecutor<>(context, config, this, instantTime, preppedRecords).execute();
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> bulkInsertPrepped(HoodieEngineContext context,
String instantTime,
List<HoodieRecord<T>> preppedRecords,
Option<BulkInsertPartitioner<List<HoodieRecord<T>>>> bulkInsertPartitioner) {
throw new HoodieNotSupportedException("BulkInsertPrepped is not supported yet");
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> insertOverwrite(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> records) {
throw new HoodieNotSupportedException("InsertOverWrite is not supported yet");
}
@Override
public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> compact(HoodieEngineContext context, String compactionInstantTime) {
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
}
@Override
public HoodieBootstrapWriteMetadata<List<WriteStatus>> bootstrap(HoodieEngineContext context, Option<Map<String, String>> extraMetadata) {
throw new HoodieNotSupportedException("Bootstrap is not supported yet");
}
@Override
public void rollbackBootstrap(HoodieEngineContext context, String instantTime) {
throw new HoodieNotSupportedException("Bootstrap is not supported yet");
}
@Override
public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) {
return new FlinkCleanActionExecutor(context, config, this, cleanInstantTime).execute();
}
@Override
public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants) {
return new FlinkCopyOnWriteRollbackActionExecutor(context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
}
@Override
public HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, String user, String comment) {
throw new HoodieNotSupportedException("Savepoint is not supported yet");
}
@Override
public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) {
throw new HoodieNotSupportedException("Savepoint and restore is not supported yet");
}
}

View File

@@ -0,0 +1,31 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.config.HoodieWriteConfig;
public class HoodieFlinkMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieFlinkCopyOnWriteTable<T> {
protected HoodieFlinkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
super(config, context, metaClient);
}
// TODO not support yet.
}

View File

@@ -0,0 +1,72 @@
/*
* 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;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
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.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.index.FlinkHoodieIndex;
import org.apache.hudi.index.HoodieIndex;
import java.util.List;
public abstract class HoodieFlinkTable<T extends HoodieRecordPayload>
extends HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
protected HoodieFlinkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
super(config, context, metaClient);
}
public static <T extends HoodieRecordPayload> HoodieFlinkTable<T> create(HoodieWriteConfig config, HoodieFlinkEngineContext context) {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(
context.getHadoopConf().get(),
config.getBasePath(),
true,
config.getConsistencyGuardConfig(),
Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))
);
return HoodieFlinkTable.create(config, context, metaClient);
}
public static <T extends HoodieRecordPayload> HoodieFlinkTable<T> create(HoodieWriteConfig config,
HoodieFlinkEngineContext context,
HoodieTableMetaClient metaClient) {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return new HoodieFlinkCopyOnWriteTable<>(config, context, metaClient);
case MERGE_ON_READ:
throw new HoodieNotSupportedException("MERGE_ON_READ is not supported yet");
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
@Override
protected HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config);
}
}

View File

@@ -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.table.action.clean;
import org.apache.hudi.avro.model.HoodieActionInstant;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.model.CleanFileInfo;
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.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import scala.Tuple2;
public class FlinkCleanActionExecutor<T extends HoodieRecordPayload> extends
BaseCleanActionExecutor<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
private static final Logger LOG = LogManager.getLogger(FlinkCleanActionExecutor.class);
public FlinkCleanActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
String instantTime) {
super(context, config, table, instantTime);
}
@Override
List<HoodieCleanStat> clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) {
Iterator<Tuple2<String, CleanFileInfo>> filesToBeDeletedPerPartition = cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream()
.flatMap(x -> x.getValue().stream().map(y -> new Tuple2<>(x.getKey(), new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile())))).iterator();
Stream<Tuple2<String, PartitionCleanStat>> partitionCleanStats =
deleteFilesFunc(filesToBeDeletedPerPartition, table)
.collect(Collectors.groupingBy(Pair::getLeft))
.entrySet().stream()
.map(x -> new Tuple2(x.getKey(), x.getValue().stream().map(y -> y.getRight()).reduce(PartitionCleanStat::merge).get()));
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
// Return PartitionCleanStat for each partition passed.
return cleanerPlan.getFilePathsToBeDeletedPerPartition().keySet().stream().map(partitionPath -> {
PartitionCleanStat partitionCleanStat = partitionCleanStatsMap.containsKey(partitionPath)
? partitionCleanStatsMap.get(partitionPath)
: new PartitionCleanStat(partitionPath);
HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain();
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
.withEarliestCommitRetained(Option.ofNullable(
actionInstant != null
? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()),
actionInstant.getAction(), actionInstant.getTimestamp())
: null))
.withDeletePathPattern(partitionCleanStat.deletePathPatterns())
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles())
.withFailedDeletes(partitionCleanStat.failedDeleteFiles())
.withDeleteBootstrapBasePathPatterns(partitionCleanStat.getDeleteBootstrapBasePathPatterns())
.withSuccessfulDeleteBootstrapBaseFiles(partitionCleanStat.getSuccessfulDeleteBootstrapBaseFiles())
.withFailedDeleteBootstrapBaseFiles(partitionCleanStat.getFailedDeleteBootstrapBaseFiles())
.build();
}).collect(Collectors.toList());
}
private static Stream<Pair<String, PartitionCleanStat>> deleteFilesFunc(Iterator<Tuple2<String, CleanFileInfo>> iter, HoodieTable table) {
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
FileSystem fs = table.getMetaClient().getFs();
while (iter.hasNext()) {
Tuple2<String, CleanFileInfo> partitionDelFileTuple = iter.next();
String partitionPath = partitionDelFileTuple._1();
Path deletePath = new Path(partitionDelFileTuple._2().getFilePath());
String deletePathStr = deletePath.toString();
Boolean deletedFileResult = null;
try {
deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
} catch (IOException e) {
LOG.error("Delete file failed");
}
if (!partitionCleanStatMap.containsKey(partitionPath)) {
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
}
boolean isBootstrapBasePathFile = partitionDelFileTuple._2().isBootstrapBaseFile();
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
if (isBootstrapBasePathFile) {
// For Bootstrap Base file deletions, store the full file path.
partitionCleanStat.addDeleteFilePatterns(deletePath.toString(), true);
partitionCleanStat.addDeletedFileResult(deletePath.toString(), deletedFileResult, true);
} else {
partitionCleanStat.addDeleteFilePatterns(deletePath.getName(), false);
partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult, false);
}
}
return partitionCleanStatMap.entrySet().stream().map(e -> Pair.of(e.getKey(), e.getValue()));
}
}

View File

@@ -0,0 +1,331 @@
/*
* 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;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieBaseFile;
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.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.CommitUtils;
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.HoodieCommitException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.execution.FlinkLazyInsertIterable;
import org.apache.hudi.io.CreateHandleFactory;
import org.apache.hudi.io.HoodieMergeHandle;
import org.apache.hudi.io.HoodieSortedMergeHandle;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.WorkloadStat;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.time.Instant;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import scala.Tuple2;
public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayload> extends
BaseCommitActionExecutor<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>, HoodieWriteMetadata> {
private static final Logger LOG = LogManager.getLogger(BaseFlinkCommitActionExecutor.class);
public BaseFlinkCommitActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable table,
String instantTime,
WriteOperationType operationType) {
super(context, config, table, instantTime, operationType, Option.empty());
}
public BaseFlinkCommitActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable table,
String instantTime,
WriteOperationType operationType,
Option extraMetadata) {
super(context, config, table, instantTime, operationType, extraMetadata);
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> execute(List<HoodieRecord<T>> inputRecords) {
HoodieWriteMetadata<List<WriteStatus>> result = new HoodieWriteMetadata<>();
WorkloadProfile profile = null;
if (isWorkloadProfileNeeded()) {
profile = new WorkloadProfile(buildProfile(inputRecords));
LOG.info("Workload profile :" + profile);
try {
saveWorkloadProfileMetadataToInflight(profile, instantTime);
} catch (Exception e) {
HoodieTableMetaClient metaClient = table.getMetaClient();
HoodieInstant inflightInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, metaClient.getCommitActionType(), instantTime);
try {
if (!metaClient.getFs().exists(new Path(metaClient.getMetaPath(), inflightInstant.getFileName()))) {
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", e);
}
} catch (IOException ex) {
LOG.error("Check file exists failed");
throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", ex);
}
}
}
final Partitioner partitioner = getPartitioner(profile);
Map<Integer, List<HoodieRecord<T>>> partitionedRecords = partition(inputRecords, partitioner);
List<WriteStatus> writeStatuses = new LinkedList<>();
partitionedRecords.forEach((partition, records) -> {
if (WriteOperationType.isChangingRecords(operationType)) {
handleUpsertPartition(instantTime, partition, records.iterator(), partitioner).forEachRemaining(writeStatuses::addAll);
} else {
handleInsertPartition(instantTime, partition, records.iterator(), partitioner).forEachRemaining(writeStatuses::addAll);
}
});
updateIndex(writeStatuses, result);
return result;
}
protected void updateIndex(List<WriteStatus> writeStatuses, HoodieWriteMetadata<List<WriteStatus>> result) {
Instant indexStartTime = Instant.now();
// Update the index back
List<WriteStatus> statuses = table.getIndex().updateLocation(writeStatuses, context, table);
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
result.setWriteStatuses(statuses);
}
@Override
protected String getCommitActionType() {
return table.getMetaClient().getCommitActionType();
}
private Partitioner getPartitioner(WorkloadProfile profile) {
if (WriteOperationType.isChangingRecords(operationType)) {
return getUpsertPartitioner(profile);
} else {
return getInsertPartitioner(profile);
}
}
private Map<Integer, List<HoodieRecord<T>>> partition(List<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
Map<Integer, List<Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>>> partitionedMidRecords = dedupedRecords
.stream()
.map(record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record))
.collect(Collectors.groupingBy(x -> partitioner.getPartition(x._1)));
Map<Integer, List<HoodieRecord<T>>> results = new LinkedHashMap<>();
partitionedMidRecords.forEach((key, value) -> results.put(key, value.stream().map(x -> x._2).collect(Collectors.toList())));
return results;
}
protected Pair<HashMap<String, WorkloadStat>, WorkloadStat> buildProfile(List<HoodieRecord<T>> inputRecords) {
HashMap<String, WorkloadStat> partitionPathStatMap = new HashMap<>();
WorkloadStat globalStat = new WorkloadStat();
Map<Pair<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = inputRecords
.stream()
.map(record -> Pair.of(
Pair.of(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record))
.collect(Collectors.groupingBy(Pair::getLeft, Collectors.counting()));
for (Map.Entry<Pair<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts.entrySet()) {
String partitionPath = e.getKey().getLeft();
Long count = e.getValue();
Option<HoodieRecordLocation> locOption = e.getKey().getRight();
if (!partitionPathStatMap.containsKey(partitionPath)) {
partitionPathStatMap.put(partitionPath, new WorkloadStat());
}
if (locOption.isPresent()) {
// update
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
globalStat.addUpdates(locOption.get(), count);
} else {
// insert
partitionPathStatMap.get(partitionPath).addInserts(count);
globalStat.addInserts(count);
}
}
return Pair.of(partitionPathStatMap, globalStat);
}
@Override
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<List<WriteStatus>> result) {
commit(extraMetadata, result, result.getWriteStatuses().stream().map(WriteStatus::getStat).collect(Collectors.toList()));
}
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<List<WriteStatus>> result, List<HoodieWriteStat> writeStats) {
String actionType = getCommitActionType();
LOG.info("Committing " + instantTime + ", action Type " + actionType);
result.setCommitted(true);
result.setWriteStats(writeStats);
// Finalize write
finalizeWrite(instantTime, writeStats, result);
try {
LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType());
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
LOG.info("Committed " + instantTime);
result.setCommitMetadata(Option.of(metadata));
} catch (IOException e) {
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
e);
}
}
protected Map<String, List<String>> getPartitionToReplacedFileIds(List<WriteStatus> writeStatuses) {
return Collections.emptyMap();
}
@Override
protected boolean isWorkloadProfileNeeded() {
return true;
}
@SuppressWarnings("unchecked")
protected Iterator<List<WriteStatus>> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr,
Partitioner partitioner) {
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
BucketType btype = binfo.bucketType;
try {
if (btype.equals(BucketType.INSERT)) {
return handleInsert(binfo.fileIdPrefix, recordItr);
} else if (btype.equals(BucketType.UPDATE)) {
return handleUpdate(binfo.partitionPath, binfo.fileIdPrefix, recordItr);
} else {
throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition);
}
} catch (Throwable t) {
String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
LOG.error(msg, t);
throw new HoodieUpsertException(msg, t);
}
}
protected Iterator<List<WriteStatus>> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr,
Partitioner partitioner) {
return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
}
@Override
public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
Iterator<HoodieRecord<T>> recordItr)
throws IOException {
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
if (!recordItr.hasNext()) {
LOG.info("Empty partition with fileId => " + fileId);
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
}
// these are updates
HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr);
return handleUpdateInternal(upsertHandle, fileId);
}
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, 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 {
FlinkMergeHelper.newInstance().runMerge(table, upsertHandle);
}
// TODO(vc): This needs to be revisited
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
+ upsertHandle.getWriteStatus());
}
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
}
protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
if (table.requireSortedRecords()) {
return new HoodieSortedMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier);
} else {
return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier);
}
}
protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId,
Map<String, HoodieRecord<T>> keyToNewRecords,
HoodieBaseFile dataFileToBeMerged) {
return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords,
partitionPath, fileId, dataFileToBeMerged, taskContextSupplier);
}
@Override
public Iterator<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr)
throws Exception {
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
if (!recordItr.hasNext()) {
LOG.info("Empty partition");
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
}
return new FlinkLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx,
taskContextSupplier, new CreateHandleFactory<>());
}
/**
* Provides a partitioner to perform the upsert operation, based on the workload profile.
*/
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
if (profile == null) {
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
}
return new UpsertPartitioner(profile, context, table, config);
}
/**
* Provides a partitioner to perform the insert operation, based on the workload profile.
*/
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
return getUpsertPartitioner(profile);
}
}

View File

@@ -0,0 +1,46 @@
/*
* 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;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import java.util.List;
public class FlinkDeleteCommitActionExecutor<T extends HoodieRecordPayload<T>> extends BaseFlinkCommitActionExecutor<T> {
private final List<HoodieKey> keys;
public FlinkDeleteCommitActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config, HoodieTable table,
String instantTime, List<HoodieKey> keys) {
super(context, config, table, instantTime, WriteOperationType.DELETE);
this.keys = keys;
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> execute() {
return FlinkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this);
}
}

View File

@@ -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.commit;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.WorkloadStat;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import java.time.Duration;
import java.time.Instant;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.stream.Collectors;
@SuppressWarnings("checkstyle:LineLength")
public class FlinkDeleteHelper<R> extends
AbstractDeleteHelper<EmptyHoodieRecordPayload, List<HoodieRecord<EmptyHoodieRecordPayload>>, List<HoodieKey>, List<WriteStatus>, R> {
private FlinkDeleteHelper() {
}
private static class DeleteHelperHolder {
private static final FlinkDeleteHelper FLINK_DELETE_HELPER = new FlinkDeleteHelper();
}
public static FlinkDeleteHelper newInstance() {
return DeleteHelperHolder.FLINK_DELETE_HELPER;
}
@Override
public List<HoodieKey> deduplicateKeys(List<HoodieKey> keys, HoodieTable<EmptyHoodieRecordPayload, List<HoodieRecord<EmptyHoodieRecordPayload>>, List<HoodieKey>, List<WriteStatus>> table, int parallelism) {
boolean isIndexingGlobal = table.getIndex().isGlobal();
if (isIndexingGlobal) {
HashSet<String> recordKeys = keys.stream().map(HoodieKey::getRecordKey).collect(Collectors.toCollection(HashSet::new));
List<HoodieKey> deduplicatedKeys = new LinkedList<>();
keys.forEach(x -> {
if (recordKeys.contains(x.getRecordKey())) {
deduplicatedKeys.add(x);
}
});
return deduplicatedKeys;
} else {
HashSet<HoodieKey> set = new HashSet<>(keys);
keys.clear();
keys.addAll(set);
return keys;
}
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> execute(String instantTime,
List<HoodieKey> keys,
HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable<EmptyHoodieRecordPayload, List<HoodieRecord<EmptyHoodieRecordPayload>>, List<HoodieKey>, List<WriteStatus>> table,
BaseCommitActionExecutor<EmptyHoodieRecordPayload, List<HoodieRecord<EmptyHoodieRecordPayload>>, List<HoodieKey>, List<WriteStatus>, R> deleteExecutor) {
try {
HoodieWriteMetadata<List<WriteStatus>> result = null;
List<HoodieKey> dedupedKeys = keys;
final int parallelism = config.getDeleteShuffleParallelism();
if (config.shouldCombineBeforeDelete()) {
// De-dupe/merge if needed
dedupedKeys = deduplicateKeys(keys, table, parallelism);
}
List<HoodieRecord<EmptyHoodieRecordPayload>> dedupedRecords =
dedupedKeys.stream().map(key -> new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList());
Instant beginTag = Instant.now();
// perform index loop up to get existing location of records
List<HoodieRecord<EmptyHoodieRecordPayload>> taggedRecords =
table.getIndex().tagLocation(dedupedRecords, context, table);
Duration tagLocationDuration = Duration.between(beginTag, Instant.now());
// filter out non existent keys/records
List<HoodieRecord<EmptyHoodieRecordPayload>> taggedValidRecords = taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).collect(Collectors.toList());
if (!taggedValidRecords.isEmpty()) {
result = deleteExecutor.execute(taggedValidRecords);
result.setIndexLookupDuration(tagLocationDuration);
} else {
// if entire set of keys are non existent
deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime);
result = new HoodieWriteMetadata<>();
result.setWriteStatuses(Collections.EMPTY_LIST);
deleteExecutor.commitOnAutoCommit(result);
}
return result;
} catch (Throwable e) {
if (e instanceof HoodieUpsertException) {
throw (HoodieUpsertException) e;
}
throw new HoodieUpsertException("Failed to delete for commit time " + instantTime, e);
}
}
}

View File

@@ -0,0 +1,50 @@
/*
* 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;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.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.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import java.util.List;
public class FlinkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>> extends BaseFlinkCommitActionExecutor<T> {
private List<HoodieRecord<T>> inputRecords;
public FlinkInsertCommitActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable table,
String instantTime,
List<HoodieRecord<T>> inputRecords) {
super(context, config, table, instantTime, WriteOperationType.INSERT);
this.inputRecords = inputRecords;
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> execute() {
return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table,
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false);
}
}

View File

@@ -0,0 +1,47 @@
/*
* 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;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.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.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import java.util.List;
public class FlinkInsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>> extends BaseFlinkCommitActionExecutor<T> {
private final List<HoodieRecord<T>> preppedRecords;
public FlinkInsertPreppedCommitActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config, HoodieTable table,
String instantTime, List<HoodieRecord<T>> preppedRecords) {
super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
this.preppedRecords = preppedRecords;
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> execute() {
return super.execute(preppedRecords);
}
}

View File

@@ -0,0 +1,116 @@
/*
* 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;
import org.apache.hudi.client.WriteStatus;
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.Option;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.io.HoodieMergeHandle;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
import org.apache.hudi.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.BinaryDecoder;
import org.apache.avro.io.BinaryEncoder;
import org.apache.hadoop.conf.Configuration;
import java.io.IOException;
import java.util.Iterator;
import scala.collection.immutable.List;
public class FlinkMergeHelper<T extends HoodieRecordPayload> extends AbstractMergeHelper<T, List<HoodieRecord<T>>,
List<HoodieKey>, List<WriteStatus>> {
private FlinkMergeHelper() {
}
private static class MergeHelperHolder {
private static final FlinkMergeHelper FLINK_MERGE_HELPER = new FlinkMergeHelper();
}
public static FlinkMergeHelper newInstance() {
return FlinkMergeHelper.MergeHelperHolder.FLINK_MERGE_HELPER;
}
@Override
public void runMerge(HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> upsertHandle) throws IOException {
final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation();
Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> mergeHandle = upsertHandle;
HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
final GenericDatumWriter<GenericRecord> gWriter;
final GenericDatumReader<GenericRecord> gReader;
Schema readSchema;
if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
gWriter = new GenericDatumWriter<>(readSchema);
gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetafields());
} else {
gReader = null;
gWriter = null;
readSchema = mergeHandle.getWriterSchemaWithMetafields();
}
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
HoodieFileReader<GenericRecord> reader = HoodieFileReaderFactory.<GenericRecord>getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
try {
final Iterator<GenericRecord> readerIterator;
if (baseFile.getBootstrapBaseFile().isPresent()) {
readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
} else {
readerIterator = reader.getRecordIterator(readSchema);
}
ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(readerIterator),
Option.of(new UpdateHandler(mergeHandle)), record -> {
if (!externalSchemaTransformation) {
return record;
}
return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record);
});
wrapper.execute();
} catch (Exception e) {
throw new HoodieException(e);
} finally {
if (reader != null) {
reader.close();
}
mergeHandle.close();
if (null != wrapper) {
wrapper.shutdownNow();
}
}
}
}

View File

@@ -0,0 +1,50 @@
/*
* 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;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.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.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import java.util.List;
public class FlinkUpsertCommitActionExecutor<T extends HoodieRecordPayload<T>> extends BaseFlinkCommitActionExecutor<T> {
private List<HoodieRecord<T>> inputRecords;
public FlinkUpsertCommitActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable table,
String instantTime,
List<HoodieRecord<T>> inputRecords) {
super(context, config, table, instantTime, WriteOperationType.UPSERT);
this.inputRecords = inputRecords;
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> execute() {
return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table,
config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true);
}
}

View File

@@ -0,0 +1,47 @@
/*
* 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;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.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.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import java.util.List;
public class FlinkUpsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>> extends BaseFlinkCommitActionExecutor<T> {
private final List<HoodieRecord<T>> preppedRecords;
public FlinkUpsertPreppedCommitActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config, HoodieTable table,
String instantTime, List<HoodieRecord<T>> preppedRecords) {
super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
this.preppedRecords = preppedRecords;
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> execute() {
return super.execute(preppedRecords);
}
}

View File

@@ -0,0 +1,68 @@
/*
* 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;
import org.apache.hudi.client.WriteStatus;
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.collection.Pair;
import org.apache.hudi.index.HoodieIndex;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
public class FlinkWriteHelper<T extends HoodieRecordPayload,R> extends AbstractWriteHelper<T, List<HoodieRecord<T>>,
List<HoodieKey>, List<WriteStatus>, R> {
private FlinkWriteHelper() {
}
private static class WriteHelperHolder {
private static final FlinkWriteHelper FLINK_WRITE_HELPER = new FlinkWriteHelper();
}
public static FlinkWriteHelper newInstance() {
return WriteHelperHolder.FLINK_WRITE_HELPER;
}
@Override
public List<HoodieRecord<T>> deduplicateRecords(List<HoodieRecord<T>> records,
HoodieIndex<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> index,
int parallelism) {
boolean isIndexingGlobal = index.isGlobal();
Map<Object, List<Pair<Object, HoodieRecord<T>>>> keyedRecords = records.stream().map(record -> {
HoodieKey hoodieKey = record.getKey();
// If index used is global, then records are expected to differ in their partitionPath
Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey;
return Pair.of(key, record);
}).collect(Collectors.groupingBy(Pair::getLeft));
return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> {
@SuppressWarnings("unchecked")
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
// we cannot allow the user to change the key or partitionPath, since that will affect
// everything
// so pick it from one of the records.
return new HoodieRecord<T>(rec1.getKey(), reducedData);
}).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList());
}
}

View File

@@ -0,0 +1,27 @@
/*
* 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;
import java.io.Serializable;
public interface Partitioner extends Serializable {
int getNumPartitions();
int getPartition(Object key);
}

View File

@@ -0,0 +1,316 @@
/*
* 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;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.NumericUtils;
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.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
import org.apache.hudi.table.WorkloadStat;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import scala.Tuple2;
public class UpsertPartitioner<T extends HoodieRecordPayload<T>> implements Partitioner {
private static final Logger LOG = LogManager.getLogger(UpsertPartitioner.class);
/**
* List of all small files to be corrected.
*/
protected List<SmallFile> smallFiles = new ArrayList<>();
/**
* Total number of RDD partitions, is determined by total buckets we want to pack the incoming workload into.
*/
private int totalBuckets = 0;
/**
* Stat for the current workload. Helps in determining inserts, upserts etc.
*/
private WorkloadProfile profile;
/**
* Helps decide which bucket an incoming update should go to.
*/
private HashMap<String, Integer> updateLocationToBucket;
/**
* Helps us pack inserts into 1 or more buckets depending on number of incoming records.
*/
private HashMap<String, List<InsertBucketCumulativeWeightPair>> partitionPathToInsertBucketInfos;
/**
* Remembers what type each bucket is for later.
*/
private HashMap<Integer, BucketInfo> bucketInfoMap;
protected final HoodieTable table;
protected final HoodieWriteConfig config;
public UpsertPartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table,
HoodieWriteConfig config) {
updateLocationToBucket = new HashMap<>();
partitionPathToInsertBucketInfos = new HashMap<>();
bucketInfoMap = new HashMap<>();
this.profile = profile;
this.table = table;
this.config = config;
assignUpdates(profile);
assignInserts(profile, context);
LOG.info("Total Buckets :" + totalBuckets + ", buckets info => " + bucketInfoMap + ", \n"
+ "Partition to insert buckets => " + partitionPathToInsertBucketInfos + ", \n"
+ "UpdateLocations mapped to buckets =>" + updateLocationToBucket);
}
private void assignUpdates(WorkloadProfile profile) {
// each update location gets a partition
Set<Map.Entry<String, WorkloadStat>> partitionStatEntries = profile.getPartitionPathStatMap().entrySet();
for (Map.Entry<String, WorkloadStat> partitionStat : partitionStatEntries) {
for (Map.Entry<String, Pair<String, Long>> updateLocEntry :
partitionStat.getValue().getUpdateLocationToCount().entrySet()) {
addUpdateBucket(partitionStat.getKey(), updateLocEntry.getKey());
}
}
}
private int addUpdateBucket(String partitionPath, String fileIdHint) {
int bucket = totalBuckets;
updateLocationToBucket.put(fileIdHint, bucket);
BucketInfo bucketInfo = new BucketInfo();
bucketInfo.bucketType = BucketType.UPDATE;
bucketInfo.fileIdPrefix = fileIdHint;
bucketInfo.partitionPath = partitionPath;
bucketInfoMap.put(totalBuckets, bucketInfo);
totalBuckets++;
return bucket;
}
private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) {
// for new inserts, compute buckets depending on how many records we have for each partition
Set<String> partitionPaths = profile.getPartitionPaths();
long averageRecordSize =
averageBytesPerRecord(table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(),
config);
LOG.info("AvgRecordSize => " + averageRecordSize);
Map<String, List<SmallFile>> partitionSmallFilesMap =
getSmallFilesForPartitions(new ArrayList<String>(partitionPaths), context);
for (String partitionPath : partitionPaths) {
WorkloadStat pStat = profile.getWorkloadStat(partitionPath);
if (pStat.getNumInserts() > 0) {
List<SmallFile> smallFiles = partitionSmallFilesMap.get(partitionPath);
this.smallFiles.addAll(smallFiles);
LOG.info("For partitionPath : " + partitionPath + " Small Files => " + smallFiles);
long totalUnassignedInserts = pStat.getNumInserts();
List<Integer> bucketNumbers = new ArrayList<>();
List<Long> recordsPerBucket = new ArrayList<>();
// first try packing this into one of the smallFiles
for (SmallFile smallFile : smallFiles) {
long recordsToAppend = Math.min((config.getParquetMaxFileSize() - smallFile.sizeBytes) / averageRecordSize,
totalUnassignedInserts);
if (recordsToAppend > 0 && totalUnassignedInserts > 0) {
// create a new bucket or re-use an existing bucket
int bucket;
if (updateLocationToBucket.containsKey(smallFile.location.getFileId())) {
bucket = updateLocationToBucket.get(smallFile.location.getFileId());
LOG.info("Assigning " + recordsToAppend + " inserts to existing update bucket " + bucket);
} else {
bucket = addUpdateBucket(partitionPath, smallFile.location.getFileId());
LOG.info("Assigning " + recordsToAppend + " inserts to new update bucket " + bucket);
}
bucketNumbers.add(bucket);
recordsPerBucket.add(recordsToAppend);
totalUnassignedInserts -= recordsToAppend;
}
}
// if we have anything more, create new insert buckets, like normal
if (totalUnassignedInserts > 0) {
long insertRecordsPerBucket = config.getCopyOnWriteInsertSplitSize();
if (config.shouldAutoTuneInsertSplits()) {
insertRecordsPerBucket = config.getParquetMaxFileSize() / averageRecordSize;
}
int insertBuckets = (int) Math.ceil((1.0 * totalUnassignedInserts) / insertRecordsPerBucket);
LOG.info("After small file assignment: unassignedInserts => " + totalUnassignedInserts
+ ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " + insertRecordsPerBucket);
for (int b = 0; b < insertBuckets; b++) {
bucketNumbers.add(totalBuckets);
recordsPerBucket.add(totalUnassignedInserts / insertBuckets);
BucketInfo bucketInfo = new BucketInfo();
bucketInfo.bucketType = BucketType.INSERT;
bucketInfo.partitionPath = partitionPath;
bucketInfo.fileIdPrefix = FSUtils.createNewFileIdPfx();
bucketInfoMap.put(totalBuckets, bucketInfo);
totalBuckets++;
}
}
// Go over all such buckets, and assign weights as per amount of incoming inserts.
List<InsertBucketCumulativeWeightPair> insertBuckets = new ArrayList<>();
double curentCumulativeWeight = 0;
for (int i = 0; i < bucketNumbers.size(); i++) {
InsertBucket bkt = new InsertBucket();
bkt.bucketNumber = bucketNumbers.get(i);
bkt.weight = (1.0 * recordsPerBucket.get(i)) / pStat.getNumInserts();
curentCumulativeWeight += bkt.weight;
insertBuckets.add(new InsertBucketCumulativeWeightPair(bkt, curentCumulativeWeight));
}
LOG.info("Total insert buckets for partition path " + partitionPath + " => " + insertBuckets);
partitionPathToInsertBucketInfos.put(partitionPath, insertBuckets);
}
}
}
private Map<String, List<SmallFile>> getSmallFilesForPartitions(List<String> partitionPaths, HoodieEngineContext context) {
Map<String, List<SmallFile>> partitionSmallFilesMap = new HashMap<>();
if (partitionPaths != null && partitionPaths.size() > 0) {
context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions");
partitionSmallFilesMap = context.mapToPair(partitionPaths, partitionPath -> new Tuple2<>(partitionPath, getSmallFiles(partitionPath)), 0);
}
return partitionSmallFilesMap;
}
/**
* Returns a list of small files in the given partition path.
*/
protected List<SmallFile> getSmallFiles(String partitionPath) {
// smallFiles only for partitionPath
List<SmallFile> smallFileLocations = new ArrayList<>();
HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline().filterCompletedInstants();
if (!commitTimeline.empty()) { // if we have some commits
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
List<HoodieBaseFile> allFiles = table.getBaseFileOnlyView()
.getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList());
for (HoodieBaseFile file : allFiles) {
if (file.getFileSize() < config.getParquetSmallFileLimit()) {
String filename = file.getFileName();
SmallFile sf = new SmallFile();
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
sf.sizeBytes = file.getFileSize();
smallFileLocations.add(sf);
}
}
}
return smallFileLocations;
}
public BucketInfo getBucketInfo(int bucketNumber) {
return bucketInfoMap.get(bucketNumber);
}
public List<InsertBucketCumulativeWeightPair> getInsertBuckets(String partitionPath) {
return partitionPathToInsertBucketInfos.get(partitionPath);
}
@Override
public int getNumPartitions() {
return totalBuckets;
}
@Override
public int getPartition(Object key) {
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation =
(Tuple2<HoodieKey, Option<HoodieRecordLocation>>) key;
if (keyLocation._2().isPresent()) {
HoodieRecordLocation location = keyLocation._2().get();
return updateLocationToBucket.get(location.getFileId());
} else {
String partitionPath = keyLocation._1().getPartitionPath();
List<InsertBucketCumulativeWeightPair> targetBuckets = partitionPathToInsertBucketInfos.get(partitionPath);
// pick the target bucket to use based on the weights.
final long totalInserts = Math.max(1, profile.getWorkloadStat(partitionPath).getNumInserts());
final long hashOfKey = NumericUtils.getMessageDigestHash("MD5", keyLocation._1().getRecordKey());
final double r = 1.0 * Math.floorMod(hashOfKey, totalInserts) / totalInserts;
int index = Collections.binarySearch(targetBuckets, new InsertBucketCumulativeWeightPair(new InsertBucket(), r));
if (index >= 0) {
return targetBuckets.get(index).getKey().bucketNumber;
}
if ((-1 * index - 1) < targetBuckets.size()) {
return targetBuckets.get((-1 * index - 1)).getKey().bucketNumber;
}
// return first one, by default
return targetBuckets.get(0).getKey().bucketNumber;
}
}
/**
* Obtains the average record size based on records written during previous commits. Used for estimating how many
* records pack into one file.
*/
protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, HoodieWriteConfig hoodieWriteConfig) {
long avgSize = hoodieWriteConfig.getCopyOnWriteRecordSizeEstimate();
long fileSizeThreshold = (long) (hoodieWriteConfig.getRecordSizeEstimationThreshold() * hoodieWriteConfig.getParquetSmallFileLimit());
try {
if (!commitTimeline.empty()) {
// Go over the reverse ordered commits to get a more recent estimate of average record size.
Iterator<HoodieInstant> instants = commitTimeline.getReverseOrderedInstants().iterator();
while (instants.hasNext()) {
HoodieInstant instant = instants.next();
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class);
long totalBytesWritten = commitMetadata.fetchTotalBytesWritten();
long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten();
if (totalBytesWritten > fileSizeThreshold && totalRecordsWritten > 0) {
avgSize = (long) Math.ceil((1.0 * totalBytesWritten) / totalRecordsWritten);
break;
}
}
}
} catch (Throwable t) {
// make this fail safe.
LOG.error("Error trying to compute average bytes/record ", t);
}
return avgSize;
}
}

View File

@@ -0,0 +1,71 @@
/*
* 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.rollback;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.HoodieRollbackStat;
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.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import java.util.List;
@SuppressWarnings("checkstyle:LineLength")
public class FlinkCopyOnWriteRollbackActionExecutor<T extends HoodieRecordPayload> extends
BaseCopyOnWriteRollbackActionExecutor<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
public FlinkCopyOnWriteRollbackActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
String instantTime,
HoodieInstant commitInstant,
boolean deleteInstants) {
super(context, config, table, instantTime, commitInstant, deleteInstants);
}
public FlinkCopyOnWriteRollbackActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
String instantTime,
HoodieInstant commitInstant,
boolean deleteInstants,
boolean skipTimelinePublish,
boolean useMarkerBasedStrategy) {
super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
}
@Override
protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() {
if (useMarkerBasedStrategy) {
return new FlinkMarkerBasedRollbackStrategy(table, context, config, instantTime);
} else {
return this::executeRollbackUsingFileListing;
}
}
@Override
protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant instantToRollback) {
List<ListingBasedRollbackRequest> rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning());
return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests);
}
}

View File

@@ -0,0 +1,73 @@
/*
* 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.rollback;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.HoodieRollbackStat;
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.IOType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.MarkerFiles;
import scala.Tuple2;
import java.util.List;
import java.util.stream.Collectors;
@SuppressWarnings("checkstyle:LineLength")
public class FlinkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> extends AbstractMarkerBasedRollbackStrategy<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
public FlinkMarkerBasedRollbackStrategy(HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) {
super(table, context, config, instantTime);
}
@Override
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
try {
MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp());
List<HoodieRollbackStat> rollbackStats = context.map(markerFiles.allMarkerFilePaths(), markerFilePath -> {
String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1);
IOType type = IOType.valueOf(typeStr);
switch (type) {
case MERGE:
return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath));
case APPEND:
return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback);
case CREATE:
return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath));
default:
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
}
}, 0);
return rollbackStats.stream().map(rollbackStat -> new Tuple2<>(rollbackStat.getPartitionPath(), rollbackStat))
.collect(Collectors.groupingBy(Tuple2::_1))
.values()
.stream()
.map(x -> x.stream().map(y -> y._2).reduce(RollbackUtils::mergeRollbackStat).get())
.collect(Collectors.toList());
} catch (Exception e) {
throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e);
}
}
}

View File

@@ -0,0 +1,236 @@
/*
* 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.rollback;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
import scala.Tuple2;
public class ListingBasedRollbackHelper implements Serializable {
private static final Logger LOG = LogManager.getLogger(ListingBasedRollbackHelper.class);
private final HoodieTableMetaClient metaClient;
private final HoodieWriteConfig config;
public ListingBasedRollbackHelper(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
this.metaClient = metaClient;
this.config = config;
}
/**
* Performs all rollback actions that we have collected in parallel.
*/
public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
Map<String, HoodieRollbackStat> partitionPathRollbackStatsPairs = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, true);
Map<String, List<Pair<String, HoodieRollbackStat>>> collect = partitionPathRollbackStatsPairs.entrySet()
.stream()
.map(x -> Pair.of(x.getKey(), x.getValue())).collect(Collectors.groupingBy(Pair::getLeft));
return collect.values().stream()
.map(pairs -> pairs.stream().map(Pair::getRight).reduce(RollbackUtils::mergeRollbackStat).orElse(null))
.filter(Objects::nonNull)
.collect(Collectors.toList());
}
/**
* Collect all file info that needs to be rollbacked.
*/
public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
Map<String, HoodieRollbackStat> partitionPathRollbackStatsPairs = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, false);
return new ArrayList<>(partitionPathRollbackStatsPairs.values());
}
/**
* May be delete interested files and collect stats or collect stats only.
*
* @param context instance of {@link HoodieEngineContext} to use.
* @param instantToRollback {@link HoodieInstant} of interest for which deletion or collect stats is requested.
* @param rollbackRequests List of {@link ListingBasedRollbackRequest} to be operated on.
* @param doDelete {@code true} if deletion has to be done. {@code false} if only stats are to be collected w/o performing any deletes.
* @return stats collected with or w/o actual deletions.
*/
Map<String, HoodieRollbackStat> maybeDeleteAndCollectStats(HoodieEngineContext context,
HoodieInstant instantToRollback,
List<ListingBasedRollbackRequest> rollbackRequests,
boolean doDelete) {
return context.mapToPair(rollbackRequests, rollbackRequest -> {
switch (rollbackRequest.getType()) {
case DELETE_DATA_FILES_ONLY: {
final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseFiles(metaClient, config, instantToRollback.getTimestamp(),
rollbackRequest.getPartitionPath(), doDelete);
return new Tuple2<>(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
.withDeletedFileResults(filesToDeletedStatus).build());
}
case DELETE_DATA_AND_LOG_FILES: {
final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseAndLogFiles(metaClient, config, instantToRollback.getTimestamp(), rollbackRequest.getPartitionPath(), doDelete);
return new Tuple2<>(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
.withDeletedFileResults(filesToDeletedStatus).build());
}
case APPEND_ROLLBACK_BLOCK: {
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())
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
// generate metadata
if (doDelete) {
Map<HoodieLogBlock.HeaderMetadataType, String> header = generateHeader(instantToRollback.getTimestamp());
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(header));
}
} catch (IOException | InterruptedException io) {
throw new HoodieRollbackException("Failed to rollback for instant " + instantToRollback, io);
} finally {
try {
if (writer != null) {
writer.close();
}
} catch (IOException io) {
throw new HoodieIOException("Error appending rollback block..", io);
}
}
// This step is intentionally done after writer is closed. Guarantees that
// getFileStatus would reflect correct stats and FileNotFoundException is not thrown in
// cloud-storage : HUDI-168
Map<FileStatus, Long> filesToNumBlocksRollback = Collections.singletonMap(
metaClient.getFs().getFileStatus(Objects.requireNonNull(writer).getLogFile().getPath()),
1L
);
return new Tuple2<>(rollbackRequest.getPartitionPath(),
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build());
}
default:
throw new IllegalStateException("Unknown Rollback action " + rollbackRequest);
}
}, 0);
}
/**
* Common method used for cleaning out base files under a partition path during rollback of a set of commits.
*/
private Map<FileStatus, Boolean> deleteBaseAndLogFiles(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
String commit, String partitionPath, boolean doDelete) throws IOException {
LOG.info("Cleaning path " + partitionPath);
String basefileExtension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
SerializablePathFilter filter = (path) -> {
if (path.toString().endsWith(basefileExtension)) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commit.equals(fileCommitTime);
} else if (FSUtils.isLogFile(path)) {
// Since the baseCommitTime is the only commit for new log files, it's okay here
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
return commit.equals(fileCommitTime);
}
return false;
};
final Map<FileStatus, Boolean> results = new HashMap<>();
FileSystem fs = metaClient.getFs();
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
for (FileStatus file : toBeDeleted) {
if (doDelete) {
boolean success = fs.delete(file.getPath(), false);
results.put(file, success);
LOG.info("Delete file " + file.getPath() + "\t" + success);
} else {
results.put(file, true);
}
}
return results;
}
/**
* Common method used for cleaning out base files under a partition path during rollback of a set of commits.
*/
private Map<FileStatus, Boolean> deleteBaseFiles(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
String commit, String partitionPath, boolean doDelete) throws IOException {
final Map<FileStatus, Boolean> results = new HashMap<>();
LOG.info("Cleaning path " + partitionPath);
FileSystem fs = metaClient.getFs();
String basefileExtension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
PathFilter filter = (path) -> {
if (path.toString().contains(basefileExtension)) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commit.equals(fileCommitTime);
}
return false;
};
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
for (FileStatus file : toBeDeleted) {
if (doDelete) {
boolean success = fs.delete(file.getPath(), false);
results.put(file, success);
LOG.info("Delete file " + file.getPath() + "\t" + success);
} else {
results.put(file, true);
}
}
return results;
}
private Map<HoodieLogBlock.HeaderMetadataType, String> generateHeader(String commit) {
// generate metadata
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(3);
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, commit);
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
return header;
}
public interface SerializablePathFilter extends PathFilter, Serializable {
}
}

View File

@@ -0,0 +1,61 @@
/*
* 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.upgrade;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTableVersion;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieUpgradeDowngradeException;
import java.io.IOException;
public class FlinkUpgradeDowngrade extends AbstractUpgradeDowngrade {
public FlinkUpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, HoodieEngineContext context) {
super(metaClient, config, context);
}
@Override
public void run(HoodieTableMetaClient metaClient, HoodieTableVersion toVersion, HoodieWriteConfig config,
HoodieEngineContext context, String instantTime) {
try {
new FlinkUpgradeDowngrade(metaClient, config, context).run(toVersion, instantTime);
} catch (IOException e) {
throw new HoodieUpgradeDowngradeException("Error during upgrade/downgrade to version:" + toVersion, e);
}
}
@Override
protected void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) {
if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) {
new ZeroToOneUpgradeHandler().upgrade(config, context, instantTime);
} else {
throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true);
}
}
@Override
protected void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) {
if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) {
new OneToZeroDowngradeHandler().downgrade(config, context, instantTime);
} else {
throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false);
}
}
}

View File

@@ -0,0 +1,49 @@
/*
* 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.upgrade;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
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.HoodieFlinkTable;
import org.apache.hudi.table.MarkerFiles;
import java.util.List;
import java.util.stream.Collectors;
/**
* Downgrade handle to assist in downgrading hoodie table from version 1 to 0.
*/
public class OneToZeroDowngradeHandler implements DowngradeHandler {
@Override
public void downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) {
// fetch pending commit info
HoodieFlinkTable table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
List<HoodieInstant> commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList());
for (HoodieInstant commitInstant : commits) {
// delete existing marker files
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
markerFiles.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
}
}
}

View File

@@ -0,0 +1,137 @@
/*
* 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.upgrade;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.IOType;
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.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.MarkerFiles;
import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper;
import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest;
import org.apache.hudi.table.action.rollback.RollbackUtils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import java.util.List;
import java.util.stream.Collectors;
/**
* Upgrade handle to assist in upgrading hoodie table from version 0 to 1.
*/
public class ZeroToOneUpgradeHandler implements UpgradeHandler {
@Override
public void upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) {
// fetch pending commit info
HoodieFlinkTable table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
if (commits.size() > 0 && instantTime != null) {
// ignore the latest inflight commit since a new commit would have been started and we need to fix any pending commits from previous launch
commits.remove(instantTime);
}
for (String commit : commits) {
// for every pending commit, delete old marker files and re-create marker files in new format
recreateMarkerFiles(commit, table, context, config.getMarkersDeleteParallelism());
}
}
/**
* Recreate marker files in new format.
* Step1: Delete existing marker files
* Step2: Collect all rollback file info.
* Step3: recreate marker files for all interested files.
*
* @param commitInstantTime instant of interest for which marker files need to be recreated.
* @param table instance of {@link HoodieFlinkTable} to use
* @param context instance of {@link HoodieEngineContext} to use
* @throws HoodieRollbackException on any exception during upgrade.
*/
private static void recreateMarkerFiles(final String commitInstantTime,
HoodieFlinkTable table,
HoodieEngineContext context,
int parallelism) throws HoodieRollbackException {
try {
// fetch hoodie instant
Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
.findFirst());
if (commitInstantOpt.isPresent()) {
// delete existing marker files
MarkerFiles markerFiles = new MarkerFiles(table, commitInstantTime);
markerFiles.quietDeleteMarkerDir(context, parallelism);
// generate rollback stats
List<ListingBasedRollbackRequest> rollbackRequests;
if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) {
rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
table.getConfig().shouldAssumeDatePartitioning());
} else {
rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context);
}
List<HoodieRollbackStat> rollbackStats = new ListingBasedRollbackHelper(table.getMetaClient(), table.getConfig())
.collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests);
// recreate marker files adhering to marker based rollback
for (HoodieRollbackStat rollbackStat : rollbackStats) {
for (String path : rollbackStat.getSuccessDeleteFiles()) {
String dataFileName = path.substring(path.lastIndexOf("/") + 1);
// not feasible to differentiate MERGE from CREATE. hence creating with MERGE IOType for all base files.
markerFiles.create(rollbackStat.getPartitionPath(), dataFileName, IOType.MERGE);
}
for (FileStatus fileStatus : rollbackStat.getCommandBlocksCount().keySet()) {
markerFiles.create(rollbackStat.getPartitionPath(), getFileNameForMarkerFromLogFile(fileStatus.getPath().toString(), table), IOType.APPEND);
}
}
}
} catch (Exception e) {
throw new HoodieRollbackException("Exception thrown while upgrading Hoodie Table from version 0 to 1", e);
}
}
/**
* Curates file name for marker from existing log file path.
* log file format : partitionpath/.fileid_baseInstant.log.writetoken
* marker file format : partitionpath/fileId_writetoken_baseinstant.basefileExtn.marker.APPEND
*
* @param logFilePath log file path for which marker file name needs to be generated.
* @return the marker file name thus curated.
*/
private static String getFileNameForMarkerFromLogFile(String logFilePath, HoodieTable table) {
Path logPath = new Path(table.getMetaClient().getBasePath(), logFilePath);
String fileId = FSUtils.getFileIdFromLogPath(logPath);
String baseInstant = FSUtils.getBaseCommitTimeFromLogPath(logPath);
String writeToken = FSUtils.getWriteTokenFromLogPath(logPath);
return FSUtils.makeDataFileName(baseInstant, writeToken, fileId, table.getBaseFileFormat().getFileExtension());
}
}

View File

@@ -18,18 +18,19 @@
package org.apache.hudi.client.common;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.client.FlinkTaskContextSupplier;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import scala.Tuple2;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
import scala.Tuple2;
/**
* Unit test against HoodieFlinkEngineContext.
@@ -39,7 +40,7 @@ public class TestHoodieFlinkEngineContext {
@BeforeEach
public void init() {
context = new HoodieFlinkEngineContext(new DummyTaskContextSupplier());
context = new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null));
}
@Test
@@ -80,36 +81,14 @@ public class TestHoodieFlinkEngineContext {
@Test
public void testMapToPair() {
List<String> mapList = Arrays.asList("hudi_flink", "hudi_spark");
List<String> mapList = Arrays.asList("spark_hudi", "flink_hudi");
Map<String, String> resultMap = context.mapToPair(mapList, x -> {
String[] splits = x.split("_");
return Tuple2.apply(splits[0], splits[1]);
}, 2);
Assertions.assertNotNull(resultMap.get("hudi"));
Assertions.assertEquals(resultMap.get("spark"), resultMap.get("flink"));
}
public static class DummyTaskContextSupplier extends TaskContextSupplier {
@Override
public Supplier<Integer> getPartitionIdSupplier() {
return null;
}
@Override
public Supplier<Integer> getStageIdSupplier() {
return null;
}
@Override
public Supplier<Long> getAttemptIdSupplier() {
return null;
}
@Override
public Option<String> getProperty(EngineProperty prop) {
return null;
}
}
}