diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java index f8d889640cfe6..408029126eba8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java @@ -39,7 +39,7 @@ public abstract class HoodieEngineContext { */ private SerializableConfiguration hadoopConf; - private TaskContextSupplier taskContextSupplier; + protected TaskContextSupplier taskContextSupplier; public HoodieEngineContext(SerializableConfiguration hadoopConf, TaskContextSupplier taskContextSupplier) { this.hadoopConf = hadoopConf; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index afd1a36649409..0413f75ce066d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -102,11 +102,11 @@ protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, Hoo this.viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration, config.getViewStorageConfig()); this.metaClient = metaClient; - this.index = getIndex(config); + this.index = getIndex(config, context); this.taskContextSupplier = context.getTaskContextSupplier(); } - protected abstract HoodieIndex getIndex(HoodieWriteConfig config); + protected abstract HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context); private synchronized FileSystemViewManager getViewManager() { if (null == viewManager) { diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml index 6fe6482a0c94f..e07f0c672262e 100644 --- a/hudi-client/hudi-flink-client/pom.xml +++ b/hudi-client/hudi-flink-client/pom.xml @@ -38,6 +38,16 @@ ${parent.version} + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + + + org.apache.flink + flink-clients_${scala.binary.version} + + org.apache.parquet diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java new file mode 100644 index 0000000000000..bb405e21f961f --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java @@ -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 getPartitionIdSupplier() { + return () -> this.flinkRuntimeContext.getIndexOfThisSubtask(); + } + + @Override + public Supplier getStageIdSupplier() { + // need to check again + return () -> this.flinkRuntimeContext.getNumberOfParallelSubtasks(); + } + + @Override + public Supplier getAttemptIdSupplier() { + return () -> (long) this.flinkRuntimeContext.getAttemptNumber(); + } + + @Override + public Option getProperty(EngineProperty prop) { + // no operation for now + return Option.empty(); + } + +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java new file mode 100644 index 0000000000000..4e08003404022 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -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 extends + AbstractHoodieWriteClient>, List, List> { + + 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 timelineService) { + super(context, writeConfig, rollbackPending, timelineService); + } + + /** + * Complete changes performed at the given instantTime marker with specified action. + */ + @Override + protected HoodieIndex>, List, List> createIndex(HoodieWriteConfig writeConfig) { + return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config); + } + + @Override + public boolean commit(String instantTime, List writeStatuses, Option> extraMetadata, String commitActionType, Map> partitionToReplacedFileIds) { + List writeStats = writeStatuses.parallelStream().map(WriteStatus::getStat).collect(Collectors.toList()); + return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds); + } + + @Override + protected HoodieTable>, List, List> createTable(HoodieWriteConfig config, Configuration hadoopConf) { + return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context); + } + + @Override + public List> filterExists(List> hoodieRecords) { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieFlinkTable table = HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context); + Timer.Context indexTimer = metrics.getIndexCtx(); + List> 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 upsert(List> records, String instantTime) { + HoodieTable>, List, List> table = + getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); + table.validateUpsertSchema(); + setOperationType(WriteOperationType.UPSERT); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata> 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 upsertPreppedRecords(List> preppedRecords, String instantTime) { + throw new HoodieNotSupportedException("UpsertPrepped operation is not supported yet"); + } + + @Override + public List insert(List> records, String instantTime) { + HoodieTable>, List, List> table = + getTableAndInitCtx(WriteOperationType.INSERT, instantTime); + table.validateUpsertSchema(); + setOperationType(WriteOperationType.INSERT); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata> 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 insertPreppedRecords(List> preppedRecords, String instantTime) { + throw new HoodieNotSupportedException("InsertPrepped operation is not supported yet"); + } + + @Override + public List bulkInsert(List> records, String instantTime) { + throw new HoodieNotSupportedException("BulkInsert operation is not supported yet"); + } + + @Override + public List bulkInsert(List> records, String instantTime, Option>>> userDefinedBulkInsertPartitioner) { + throw new HoodieNotSupportedException("BulkInsert operation is not supported yet"); + } + + @Override + public List bulkInsertPreppedRecords(List> preppedRecords, String instantTime, Option>>> bulkInsertPartitioner) { + throw new HoodieNotSupportedException("BulkInsertPrepped operation is not supported yet"); + } + + @Override + public List delete(List keys, String instantTime) { + HoodieTable>, List, List> table = + getTableAndInitCtx(WriteOperationType.DELETE, instantTime); + setOperationType(WriteOperationType.DELETE); + HoodieWriteMetadata> result = table.delete(context,instantTime, keys); + return postWrite(result, instantTime, table); + } + + @Override + protected List postWrite(HoodieWriteMetadata> result, + String instantTime, + HoodieTable>, List, List> hoodieTable) { + if (result.getIndexLookupDuration().isPresent()) { + metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); + } + return result.getWriteStatuses(); + } + + @Override + public void commitCompaction(String compactionInstantTime, List writeStatuses, Option> extraMetadata) throws IOException { + throw new HoodieNotSupportedException("Compaction is not supported yet"); + } + + @Override + protected void completeCompaction(HoodieCommitMetadata metadata, List writeStatuses, HoodieTable>, List, List> table, String compactionCommitTime) { + throw new HoodieNotSupportedException("Compaction is not supported yet"); + } + + @Override + protected List compact(String compactionInstantTime, boolean shouldComplete) { + throw new HoodieNotSupportedException("Compaction is not supported yet"); + } + + @Override + protected HoodieTable>, List, List> 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>, List, List> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) { + if (operationType == WriteOperationType.DELETE) { + setWriteSchemaForDeletes(metaClient); + } + // Create a Hoodie table which encapsulated the commits and files visible + HoodieFlinkTable 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 getInflightsAndRequestedInstants(String commitType) { + HoodieFlinkTable 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()); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java index 5c52b58141681..52052fd368529 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java @@ -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 @@ * 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 HoodieFlinkEngineContext(TaskContextSupplier taskContextSupplier) { 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 void foreach(List data, SerializableConsumer consumer, int para @Override public Map mapToPair(List data, SerializablePairFunction func, Integer parallelism) { - Map 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 diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/function/FunctionWrapper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/function/FunctionWrapper.java index 31b11a0d10fc9..4e91bd29d676b 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/function/FunctionWrapper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/function/FunctionWrapper.java @@ -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. diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java new file mode 100644 index 0000000000000..f0e51220d7c98 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java @@ -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 extends HoodieLazyInsertIterable { + public FlinkLazyInsertIterable(Iterator> recordItr, + boolean areRecordsSorted, + HoodieWriteConfig config, + String instantTime, + HoodieTable hoodieTable, + String idPrefix, + TaskContextSupplier taskContextSupplier) { + super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier); + } + + public FlinkLazyInsertIterable(Iterator> 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 computeNext() { + // Executor service used for launching writer thread. + BoundedInMemoryExecutor, HoodieInsertValueGenResult, List> 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 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(); + } + } + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java new file mode 100644 index 0000000000000..83b76fc8930f7 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java @@ -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 payload type + */ +public abstract class FlinkHoodieIndex extends HoodieIndex>, List, List> { + 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 updateLocation(List writeStatuses, + HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + public abstract List> tagLocation(List> records, + HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java new file mode 100644 index 0000000000000..d3fdf67d76a92 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java @@ -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 type of payload + */ +public class FlinkInMemoryStateIndex extends FlinkHoodieIndex { + + private static final Logger LOG = LogManager.getLogger(FlinkInMemoryStateIndex.class); + private MapState mapState; + + public FlinkInMemoryStateIndex(HoodieFlinkEngineContext context, HoodieWriteConfig config) { + super(config); + if (context.getRuntimeContext() != null) { + MapStateDescriptor indexStateDesc = + new MapStateDescriptor<>("indexState", TypeInformation.of(HoodieKey.class), TypeInformation.of(HoodieRecordLocation.class)); + if (context.getRuntimeContext() != null) { + mapState = context.getRuntimeContext().getMapState(indexStateDesc); + } + } + } + + @Override + public List> tagLocation(List> records, + HoodieEngineContext context, + HoodieTable>, List, List> 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 updateLocation(List writeStatuses, + HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) throws HoodieIndexException { + return context.map(writeStatuses, writeStatus -> { + for (HoodieRecord record : writeStatus.getWrittenRecords()) { + if (!writeStatus.isErrored(record.getKey())) { + HoodieKey key = record.getKey(); + Option 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; + } +} \ No newline at end of file diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java new file mode 100644 index 0000000000000..6d4c570790e0a --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -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 extends HoodieFlinkTable { + + protected HoodieFlinkCopyOnWriteTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { + super(config, context, metaClient); + } + + @Override + public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, List> records) { + return new FlinkUpsertCommitActionExecutor<>(context, config, this, instantTime, records).execute(); + } + + @Override + public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, List> records) { + return new FlinkInsertCommitActionExecutor<>(context, config, this, instantTime, records).execute(); + } + + @Override + public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, + String instantTime, + List> records, + Option>>> bulkInsertPartitioner) { + throw new HoodieNotSupportedException("BulkInsert is not supported yet"); + } + + @Override + public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, List keys) { + return new FlinkDeleteCommitActionExecutor<>(context, config, this, instantTime, keys).execute(); + } + + @Override + public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords) { + return new FlinkUpsertPreppedCommitActionExecutor<>(context, config, this, instantTime, preppedRecords).execute(); + } + + @Override + public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords) { + return new FlinkInsertPreppedCommitActionExecutor<>(context, config, this, instantTime, preppedRecords).execute(); + } + + @Override + public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, + String instantTime, + List> preppedRecords, + Option>>> bulkInsertPartitioner) { + throw new HoodieNotSupportedException("BulkInsertPrepped is not supported yet"); + } + + @Override + public HoodieWriteMetadata> insertOverwrite(HoodieEngineContext context, String instantTime, List> records) { + throw new HoodieNotSupportedException("InsertOverWrite is not supported yet"); + } + + @Override + public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { + throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); + } + + @Override + public HoodieWriteMetadata> compact(HoodieEngineContext context, String compactionInstantTime) { + throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); + } + + @Override + public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> 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"); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java new file mode 100644 index 0000000000000..51ce54a42da4f --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java @@ -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 extends HoodieFlinkCopyOnWriteTable { + protected HoodieFlinkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { + super(config, context, metaClient); + } + // TODO not support yet. +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java new file mode 100644 index 0000000000000..79fb376be61b4 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -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 + extends HoodieTable>, List, List> { + protected HoodieFlinkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { + super(config, context, metaClient); + } + + public static HoodieFlinkTable 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 HoodieFlinkTable 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>, List, List> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { + return FlinkHoodieIndex.createIndex((HoodieFlinkEngineContext) context, config); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java new file mode 100644 index 0000000000000..010e2a16af4a7 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java @@ -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 extends + BaseCleanActionExecutor>, List, List> { + + private static final Logger LOG = LogManager.getLogger(FlinkCleanActionExecutor.class); + + public FlinkCleanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, List, List> table, + String instantTime) { + super(context, config, table, instantTime); + } + + @Override + List clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) { + + Iterator> filesToBeDeletedPerPartition = cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream() + .flatMap(x -> x.getValue().stream().map(y -> new Tuple2<>(x.getKey(), new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile())))).iterator(); + + Stream> 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 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> deleteFilesFunc(Iterator> iter, HoodieTable table) { + Map partitionCleanStatMap = new HashMap<>(); + FileSystem fs = table.getMetaClient().getFs(); + + while (iter.hasNext()) { + Tuple2 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())); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java new file mode 100644 index 0000000000000..0b98b11049a24 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java @@ -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 extends + BaseCommitActionExecutor>, List, List, 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> execute(List> inputRecords) { + HoodieWriteMetadata> 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>> partitionedRecords = partition(inputRecords, partitioner); + + List 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 writeStatuses, HoodieWriteMetadata> result) { + Instant indexStartTime = Instant.now(); + // Update the index back + List 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>> partition(List> dedupedRecords, Partitioner partitioner) { + Map>, HoodieRecord>>> 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>> results = new LinkedHashMap<>(); + partitionedMidRecords.forEach((key, value) -> results.put(key, value.stream().map(x -> x._2).collect(Collectors.toList()))); + return results; + } + + protected Pair, WorkloadStat> buildProfile(List> inputRecords) { + HashMap partitionPathStatMap = new HashMap<>(); + WorkloadStat globalStat = new WorkloadStat(); + + Map>, 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>, Long> e : partitionLocationCounts.entrySet()) { + String partitionPath = e.getKey().getLeft(); + Long count = e.getValue(); + Option 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> extraMetadata, HoodieWriteMetadata> result) { + commit(extraMetadata, result, result.getWriteStatuses().stream().map(WriteStatus::getStat).collect(Collectors.toList())); + } + + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List 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> getPartitionToReplacedFileIds(List writeStatuses) { + return Collections.emptyMap(); + } + + @Override + protected boolean isWorkloadProfileNeeded() { + return true; + } + + @SuppressWarnings("unchecked") + protected Iterator> 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> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr, + Partitioner partitioner) { + return handleUpsertPartition(instantTime, partition, recordItr, partitioner); + } + + @Override + public Iterator> handleUpdate(String partitionPath, String fileId, + Iterator> 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) Collections.EMPTY_LIST).iterator(); + } + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); + return handleUpdateInternal(upsertHandle, fileId); + } + + protected Iterator> 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> 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> keyToNewRecords, + HoodieBaseFile dataFileToBeMerged) { + return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords, + partitionPath, fileId, dataFileToBeMerged, taskContextSupplier); + } + + @Override + public Iterator> handleInsert(String idPfx, Iterator> 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) 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); + } + +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java new file mode 100644 index 0000000000000..a31ab4ee15d70 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java @@ -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> extends BaseFlinkCommitActionExecutor { + private final List keys; + + public FlinkDeleteCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, List keys) { + super(context, config, table, instantTime, WriteOperationType.DELETE); + this.keys = keys; + } + + @Override + public HoodieWriteMetadata> execute() { + return FlinkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java new file mode 100644 index 0000000000000..bc1293a16b71a --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java @@ -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 extends + AbstractDeleteHelper>, List, List, 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 deduplicateKeys(List keys, HoodieTable>, List, List> table, int parallelism) { + boolean isIndexingGlobal = table.getIndex().isGlobal(); + if (isIndexingGlobal) { + HashSet recordKeys = keys.stream().map(HoodieKey::getRecordKey).collect(Collectors.toCollection(HashSet::new)); + List deduplicatedKeys = new LinkedList<>(); + keys.forEach(x -> { + if (recordKeys.contains(x.getRecordKey())) { + deduplicatedKeys.add(x); + } + }); + return deduplicatedKeys; + } else { + HashSet set = new HashSet<>(keys); + keys.clear(); + keys.addAll(set); + return keys; + } + } + + @Override + public HoodieWriteMetadata> execute(String instantTime, + List keys, + HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, List, List> table, + BaseCommitActionExecutor>, List, List, R> deleteExecutor) { + try { + HoodieWriteMetadata> result = null; + List dedupedKeys = keys; + final int parallelism = config.getDeleteShuffleParallelism(); + if (config.shouldCombineBeforeDelete()) { + // De-dupe/merge if needed + dedupedKeys = deduplicateKeys(keys, table, parallelism); + } + + List> 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> taggedRecords = + table.getIndex().tagLocation(dedupedRecords, context, table); + Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); + + // filter out non existent keys/records + List> 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); + } + } + +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java new file mode 100644 index 0000000000000..1752960321e0b --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java @@ -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> extends BaseFlinkCommitActionExecutor { + + private List> inputRecords; + + public FlinkInsertCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + List> inputRecords) { + super(context, config, table, instantTime, WriteOperationType.INSERT); + this.inputRecords = inputRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java new file mode 100644 index 0000000000000..96fcd06a64ecc --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java @@ -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> extends BaseFlinkCommitActionExecutor { + + private final List> preppedRecords; + + public FlinkInsertPreppedCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, List> preppedRecords) { + super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); + this.preppedRecords = preppedRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return super.execute(preppedRecords); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java new file mode 100644 index 0000000000000..d34aca22f049c --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java @@ -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 extends AbstractMergeHelper>, + List, List> { + + 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>, List, List> table, + HoodieMergeHandle>, List, List> upsertHandle) throws IOException { + final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation(); + Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf()); + HoodieMergeHandle>, List, List> mergeHandle = upsertHandle; + HoodieBaseFile baseFile = mergeHandle.baseFileForMerge(); + + final GenericDatumWriter gWriter; + final GenericDatumReader 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 wrapper = null; + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); + try { + final Iterator readerIterator; + if (baseFile.getBootstrapBaseFile().isPresent()) { + readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation); + } else { + readerIterator = reader.getRecordIterator(readSchema); + } + + ThreadLocal encoderCache = new ThreadLocal<>(); + ThreadLocal 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(); + } + } + } + +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java new file mode 100644 index 0000000000000..24659c5fe40cd --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java @@ -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> extends BaseFlinkCommitActionExecutor { + + private List> inputRecords; + + public FlinkUpsertCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + List> inputRecords) { + super(context, config, table, instantTime, WriteOperationType.UPSERT); + this.inputRecords = inputRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return FlinkWriteHelper.newInstance().write(instantTime, inputRecords, context, table, + config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java new file mode 100644 index 0000000000000..89540804598af --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java @@ -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> extends BaseFlinkCommitActionExecutor { + + private final List> preppedRecords; + + public FlinkUpsertPreppedCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, List> preppedRecords) { + super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); + this.preppedRecords = preppedRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return super.execute(preppedRecords); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java new file mode 100644 index 0000000000000..df106ce8d585c --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java @@ -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 extends AbstractWriteHelper>, + List, List, 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> deduplicateRecords(List> records, + HoodieIndex>, List, List> index, + int parallelism) { + boolean isIndexingGlobal = index.isGlobal(); + Map>>> 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(rec1.getKey(), reducedData); + }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList()); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/Partitioner.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/Partitioner.java new file mode 100644 index 0000000000000..2d52c50fad085 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/Partitioner.java @@ -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); +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java new file mode 100644 index 0000000000000..4bfaab4456513 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -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> implements Partitioner { + + private static final Logger LOG = LogManager.getLogger(UpsertPartitioner.class); + + /** + * List of all small files to be corrected. + */ + protected List 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 updateLocationToBucket; + /** + * Helps us pack inserts into 1 or more buckets depending on number of incoming records. + */ + private HashMap> partitionPathToInsertBucketInfos; + /** + * Remembers what type each bucket is for later. + */ + private HashMap 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> partitionStatEntries = profile.getPartitionPathStatMap().entrySet(); + for (Map.Entry partitionStat : partitionStatEntries) { + for (Map.Entry> 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 partitionPaths = profile.getPartitionPaths(); + long averageRecordSize = + averageBytesPerRecord(table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(), + config); + LOG.info("AvgRecordSize => " + averageRecordSize); + + Map> partitionSmallFilesMap = + getSmallFilesForPartitions(new ArrayList(partitionPaths), context); + + for (String partitionPath : partitionPaths) { + WorkloadStat pStat = profile.getWorkloadStat(partitionPath); + if (pStat.getNumInserts() > 0) { + + List smallFiles = partitionSmallFilesMap.get(partitionPath); + this.smallFiles.addAll(smallFiles); + + LOG.info("For partitionPath : " + partitionPath + " Small Files => " + smallFiles); + + long totalUnassignedInserts = pStat.getNumInserts(); + List bucketNumbers = new ArrayList<>(); + List 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 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> getSmallFilesForPartitions(List partitionPaths, HoodieEngineContext context) { + Map> 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 getSmallFiles(String partitionPath) { + + // smallFiles only for partitionPath + List smallFileLocations = new ArrayList<>(); + + HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline().filterCompletedInstants(); + + if (!commitTimeline.empty()) { // if we have some commits + HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); + List 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 getInsertBuckets(String partitionPath) { + return partitionPathToInsertBucketInfos.get(partitionPath); + } + + @Override + public int getNumPartitions() { + return totalBuckets; + } + + @Override + public int getPartition(Object key) { + Tuple2> keyLocation = + (Tuple2>) key; + if (keyLocation._2().isPresent()) { + HoodieRecordLocation location = keyLocation._2().get(); + return updateLocationToBucket.get(location.getFileId()); + } else { + String partitionPath = keyLocation._1().getPartitionPath(); + List 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 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; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java new file mode 100644 index 0000000000000..28b713b24b9ab --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java @@ -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 extends + BaseCopyOnWriteRollbackActionExecutor>, List, List> { + public FlinkCopyOnWriteRollbackActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, List, List> table, + String instantTime, + HoodieInstant commitInstant, + boolean deleteInstants) { + super(context, config, table, instantTime, commitInstant, deleteInstants); + } + + public FlinkCopyOnWriteRollbackActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, List, List> 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 executeRollbackUsingFileListing(HoodieInstant instantToRollback) { + List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), + config.shouldAssumeDatePartitioning()); + return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkMarkerBasedRollbackStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkMarkerBasedRollbackStrategy.java new file mode 100644 index 0000000000000..1e666b117c654 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkMarkerBasedRollbackStrategy.java @@ -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 extends AbstractMarkerBasedRollbackStrategy>, List, List> { + public FlinkMarkerBasedRollbackStrategy(HoodieTable>, List, List> table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) { + super(table, context, config, instantTime); + } + + @Override + public List execute(HoodieInstant instantToRollback) { + try { + MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp()); + List 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); + } + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java new file mode 100644 index 0000000000000..2b619fb04acd8 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java @@ -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 performRollback(HoodieEngineContext context, HoodieInstant instantToRollback, List rollbackRequests) { + Map partitionPathRollbackStatsPairs = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, true); + + Map>> 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 collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback, List rollbackRequests) { + Map 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 maybeDeleteAndCollectStats(HoodieEngineContext context, + HoodieInstant instantToRollback, + List rollbackRequests, + boolean doDelete) { + return context.mapToPair(rollbackRequests, rollbackRequest -> { + switch (rollbackRequest.getType()) { + case DELETE_DATA_FILES_ONLY: { + final Map 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 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 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 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 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 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 deleteBaseFiles(HoodieTableMetaClient metaClient, HoodieWriteConfig config, + String commit, String partitionPath, boolean doDelete) throws IOException { + final Map 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 generateHeader(String commit) { + // generate metadata + Map 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 { + + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java new file mode 100644 index 0000000000000..a96a14aa7800a --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java @@ -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); + } + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java new file mode 100644 index 0000000000000..2d2e06e5979cb --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java @@ -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 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()); + } + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java new file mode 100644 index 0000000000000..1fa3ad0a32b7c --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -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 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 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 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 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()); + } +} diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/common/TestHoodieFlinkEngineContext.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/common/TestHoodieFlinkEngineContext.java index 5d752ada03fe3..41a8b96199395 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/common/TestHoodieFlinkEngineContext.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/common/TestHoodieFlinkEngineContext.java @@ -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 void testForeach() { @Test public void testMapToPair() { - List mapList = Arrays.asList("hudi_flink", "hudi_spark"); + List mapList = Arrays.asList("spark_hudi", "flink_hudi"); Map 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 getPartitionIdSupplier() { - return null; - } - - @Override - public Supplier getStageIdSupplier() { - return null; - } - - @Override - public Supplier getAttemptIdSupplier() { - return null; - } - - @Override - public Option getProperty(EngineProperty prop) { - return null; - } - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index 4292da75fcf90..d5fb4ee018a69 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -66,7 +66,7 @@ public static HoodieSparkTable create(HoodieW } @Override - protected HoodieIndex>, JavaRDD, JavaRDD> getIndex(HoodieWriteConfig config) { + protected HoodieIndex>, JavaRDD, JavaRDD> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { return SparkHoodieIndex.createIndex(config); } } diff --git a/hudi-flink/pom.xml b/hudi-flink/pom.xml new file mode 100644 index 0000000000000..4b8cfd78e1009 --- /dev/null +++ b/hudi-flink/pom.xml @@ -0,0 +1,177 @@ + + + + + + + hudi + org.apache.hudi + 0.6.1-SNAPSHOT + + 4.0.0 + + hudi-flink_${scala.binary.version} + jar + + + ${project.parent.basedir} + + + + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + org.apache.maven.plugins + maven-jar-plugin + 3.1.2 + + + + test-jar + + + + + + org.apache.rat + apache-rat-plugin + + + + + + src/main/resources + + + src/test/resources + + + + + + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hudi + hudi-client-common + ${project.version} + + + org.apache.hudi + hudi-flink-client + ${project.version} + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + compile + + + org.apache.flink + flink-clients_${scala.binary.version} + compile + + + com.esotericsoftware.kryo + kryo + + + com.esotericsoftware.minlog + minlog + + + + + org.apache.flink + flink-connector-kafka_${scala.binary.version} + compile + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + + + org.apache.hadoop + hadoop-common + compile + + + org.apache.hadoop + hadoop-hdfs + compile + + + org.apache.hadoop + hadoop-auth + compile + + + + + org.apache.avro + avro + compile + + + + + org.apache.parquet + parquet-avro + compile + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + compile + + + + com.beust + jcommander + compile + + + com.twitter + bijection-avro_${scala.binary.version} + 0.9.7 + + + diff --git a/hudi-flink/src/main/java/org/apache/hudi/HudiFlinkStreamer.java b/hudi-flink/src/main/java/org/apache/hudi/HudiFlinkStreamer.java new file mode 100644 index 0000000000000..1206cbe512989 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/HudiFlinkStreamer.java @@ -0,0 +1,188 @@ +/* + * 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; + +import com.beust.jcommander.IStringConverter; +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParameterException; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.operator.InstantGenerateOperator; +import org.apache.hudi.operator.KeyedWriteProcessFunction; +import org.apache.hudi.operator.KeyedWriteProcessOperator; +import org.apache.hudi.sink.CommitSink; +import org.apache.hudi.source.JsonStringToHoodieRecordMapFunction; +import org.apache.hudi.util.StreamerUtil; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Properties; + +/** + * An Utility which can incrementally consume data from Kafka and apply it to the target table. + * currently, it only support COW table and insert, upsert operation. + */ +public class HudiFlinkStreamer { + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + env.enableCheckpointing(cfg.checkpointInterval); + env.getConfig().setGlobalJobParameters(cfg); + // We use checkpoint to trigger write operation, including instant generating and committing, + // There can only be one checkpoint at one time. + env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); + env.disableOperatorChaining(); + + if (cfg.flinkCheckPointPath != null) { + env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); + } + + Properties kafkaProps = StreamerUtil.getKafkaProps(cfg); + + // Read from kafka source + DataStream inputRecords = + env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps)) + .filter(Objects::nonNull) + .map(new JsonStringToHoodieRecordMapFunction(cfg)) + .name("kafka_to_hudi_record") + .uid("kafka_to_hudi_record_uid"); + + // InstantGenerateOperator helps to emit globally unique instantTime, it must be executed in one parallelism + inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator()) + .name("instant_generator") + .uid("instant_generator_id") + .setParallelism(1) + + // Keyby partition path, to avoid multiple subtasks writing to a partition at the same time + .keyBy(HoodieRecord::getPartitionPath) + + // write operator, where the write operation really happens + .transform(KeyedWriteProcessOperator.NAME, TypeInformation.of(new TypeHint, Integer>>() { + }), new KeyedWriteProcessOperator(new KeyedWriteProcessFunction())) + .name("write_process") + .uid("write_process_uid") + .setParallelism(env.getParallelism()) + + // Commit can only be executed once, so make it one parallelism + .addSink(new CommitSink()) + .name("commit_sink") + .uid("commit_sink_uid") + .setParallelism(1); + + env.execute(cfg.targetTableName); + } + + public static class Config extends Configuration { + @Parameter(names = {"--kafka-topic"}, description = "kafka topic", required = true) + public String kafkaTopic; + + @Parameter(names = {"--kafka-group-id"}, description = "kafka consumer group id", required = true) + public String kafkaGroupId; + + @Parameter(names = {"--kafka-bootstrap-servers"}, description = "kafka bootstrap.servers", required = true) + public String kafkaBootstrapServers; + + @Parameter(names = {"--flink-checkpoint-path"}, description = "flink checkpoint path") + public String flinkCheckPointPath; + + @Parameter(names = {"--flink-block-retry-times"}, description = "Times to retry when latest instant has not completed") + public String blockRetryTime = "10"; + + @Parameter(names = {"--flink-block-retry-interval"}, description = "Seconds between two tries when latest instant has not completed") + public String blockRetryInterval = "1"; + + @Parameter(names = {"--target-base-path"}, + description = "base path for the target hoodie table. " + + "(Will be created if did not exist first time around. If exists, expected to be a hoodie table)", + required = true) + public String targetBasePath; + + @Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true) + public String targetTableName; + + @Parameter(names = {"--table-type"}, description = "Type of table. COPY_ON_WRITE (or) MERGE_ON_READ", required = true) + public String tableType; + + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are " + + "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer" + + "to individual classes, for supported properties.") + public String propsFilePath = + "file://" + System.getProperty("user.dir") + "/src/test/resources/delta-streamer-config/dfs-source.properties"; + + @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " + + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.") + public List configs = new ArrayList<>(); + + @Parameter(names = {"--source-ordering-field"}, description = "Field within source record to decide how" + + " to break ties between records with same key in input data. Default: 'ts' holding unix timestamp of record") + public String sourceOrderingField = "ts"; + + @Parameter(names = {"--payload-class"}, description = "subclass of HoodieRecordPayload, that works off " + + "a GenericRecord. Implement your own, if you want to do something other than overwriting existing value") + public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName(); + + @Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input " + + "is purely new data/inserts to gain speed)", converter = OperationConverter.class) + public WriteOperationType operation = WriteOperationType.UPSERT; + + @Parameter(names = {"--filter-dupes"}, + description = "Should duplicate records from source be dropped/filtered out before insert/bulk-insert") + public Boolean filterDupes = false; + + @Parameter(names = {"--commit-on-errors"}, description = "Commit even when some records failed to be written") + public Boolean commitOnErrors = false; + + /** + * Flink checkpoint interval. + */ + @Parameter(names = {"--checkpoint-interval"}, description = "Flink checkpoint interval.") + public Long checkpointInterval = 1000 * 5L; + + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + } + + private static class OperationConverter implements IStringConverter { + + @Override + public WriteOperationType convert(String value) throws ParameterException { + return WriteOperationType.valueOf(value); + } + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/exception/HoodieFlinkStreamerException.java b/hudi-flink/src/main/java/org/apache/hudi/exception/HoodieFlinkStreamerException.java new file mode 100644 index 0000000000000..0aadce83dedf1 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/exception/HoodieFlinkStreamerException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.exception; + +public class HoodieFlinkStreamerException extends HoodieException { + + public HoodieFlinkStreamerException(String msg, Throwable e) { + super(msg, e); + } + + public HoodieFlinkStreamerException(String msg) { + super(msg); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java b/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java new file mode 100644 index 0000000000000..b242276085724 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java @@ -0,0 +1,224 @@ +/* + * 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.operator; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.HudiFlinkStreamer; +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.util.StreamerUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new + * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is + * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until + * time out and throw an exception. + */ +public class InstantGenerateOperator extends AbstractStreamOperator implements OneInputStreamOperator { + + private static final Logger LOG = LoggerFactory.getLogger(InstantGenerateOperator.class); + public static final String NAME = "InstantGenerateOperator"; + + private HudiFlinkStreamer.Config cfg; + private HoodieFlinkWriteClient writeClient; + private SerializableConfiguration serializableHadoopConf; + private transient FileSystem fs; + private String latestInstant = ""; + private List latestInstantList = new ArrayList<>(1); + private transient ListState latestInstantState; + private List bufferedRecords = new LinkedList(); + private transient ListState recordsState; + private Integer retryTimes; + private Integer retryInterval; + + @Override + public void processElement(StreamRecord streamRecord) throws Exception { + if (streamRecord.getValue() != null) { + bufferedRecords.add(streamRecord); + output.collect(streamRecord); + } + } + + @Override + public void open() throws Exception { + super.open(); + // get configs from runtimeContext + cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + + // retry times + retryTimes = Integer.valueOf(cfg.blockRetryTime); + + // retry interval + retryInterval = Integer.valueOf(cfg.blockRetryInterval); + + // hadoopConf + serializableHadoopConf = new SerializableConfiguration(StreamerUtil.getHadoopConf()); + + // Hadoop FileSystem + fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get()); + + TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null); + + // writeClient + writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg), true); + + // init table, create it if not exists. + initTable(); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + super.prepareSnapshotPreBarrier(checkpointId); + // check whether the last instant is completed, if not, wait 10s and then throws an exception + if (!StringUtils.isNullOrEmpty(latestInstant)) { + doCheck(); + // last instant completed, set it empty + latestInstant = ""; + } + + // no data no new instant + if (!bufferedRecords.isEmpty()) { + latestInstant = startNewInstant(checkpointId); + } + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + // instantState + ListStateDescriptor latestInstantStateDescriptor = new ListStateDescriptor("latestInstant", String.class); + latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor); + + // recordState + ListStateDescriptor recordsStateDescriptor = new ListStateDescriptor("recordsState", StreamRecord.class); + recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor); + + if (context.isRestored()) { + Iterator latestInstantIterator = latestInstantState.get().iterator(); + latestInstantIterator.forEachRemaining(x -> latestInstant = x); + LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant); + + Iterator recordIterator = recordsState.get().iterator(); + bufferedRecords.clear(); + recordIterator.forEachRemaining(x -> bufferedRecords.add(x)); + } + } + + @Override + public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception { + if (latestInstantList.isEmpty()) { + latestInstantList.add(latestInstant); + } else { + latestInstantList.set(0, latestInstant); + } + latestInstantState.update(latestInstantList); + LOG.info("Update latest instant [{}]", latestInstant); + + recordsState.update(bufferedRecords); + LOG.info("Update records state size = [{}]", bufferedRecords.size()); + bufferedRecords.clear(); + } + + /** + * Create a new instant. + * + * @param checkpointId + */ + private String startNewInstant(long checkpointId) { + String newTime = writeClient.startCommit(); + LOG.info("create instant [{}], at checkpoint [{}]", newTime, checkpointId); + return newTime; + } + + /** + * Check the status of last instant. + */ + private void doCheck() throws InterruptedException { + // query the requested and inflight commit/deltacommit instants + String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION; + LOG.info("Query latest instant [{}]", latestInstant); + List rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType); + int tryTimes = 0; + while (tryTimes < retryTimes) { + tryTimes++; + StringBuffer sb = new StringBuffer(); + if (rollbackPendingCommits.contains(latestInstant)) { + rollbackPendingCommits.forEach(x -> sb.append(x).append(",")); + LOG.warn("Latest transaction [{}] is not completed! unCompleted transaction:[{}],try times [{}]", latestInstant, sb.toString(), tryTimes); + TimeUnit.SECONDS.sleep(retryInterval); + rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType); + } else { + LOG.warn("Latest transaction [{}] is completed! Completed transaction, try times [{}]", latestInstant, tryTimes); + return; + } + } + throw new InterruptedException("Last instant costs more than ten second, stop task now"); + } + + + /** + * Create table if not exists. + */ + private void initTable() throws IOException { + if (!fs.exists(new Path(cfg.targetBasePath))) { + HoodieTableMetaClient.initTableType(new Configuration(serializableHadoopConf.get()), cfg.targetBasePath, + HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived", cfg.payloadClassName, 1); + LOG.info("Table initialized"); + } else { + LOG.info("Table already [{}/{}] exists, do nothing here", cfg.targetBasePath, cfg.targetTableName); + } + } + + @Override + public void close() throws Exception { + if (writeClient != null) { + writeClient.close(); + } + if (fs != null) { + fs.close(); + } + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/operator/KeyedWriteProcessFunction.java b/hudi-flink/src/main/java/org/apache/hudi/operator/KeyedWriteProcessFunction.java new file mode 100644 index 0000000000000..9a751e80e83a3 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/operator/KeyedWriteProcessFunction.java @@ -0,0 +1,164 @@ +/* + * 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.operator; + +import org.apache.hudi.HudiFlinkStreamer; +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.exception.HoodieFlinkStreamerException; +import org.apache.hudi.util.StreamerUtil; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; +import java.util.List; + +/** + * A {@link KeyedProcessFunction} where the write operations really happens. + */ +public class KeyedWriteProcessFunction extends KeyedProcessFunction, Integer>> implements CheckpointedFunction { + + private static final Logger LOG = LoggerFactory.getLogger(KeyedWriteProcessFunction.class); + /** + * Records buffer, will be processed in snapshotState function. + */ + private List bufferedRecords = new LinkedList<>(); + + /** + * Flink collector help s to send data downstream. + */ + private Collector, Integer>> output; + + /** + * Id of current subtask. + */ + private int indexOfThisSubtask; + + /** + * Instant time this batch belongs to. + */ + private String latestInstant; + + /** + * Flag indicate whether this subtask has records in. + */ + private boolean hasRecordsIn; + + /** + * Job conf. + */ + private HudiFlinkStreamer.Config cfg; + + /** + * Write Client. + */ + private transient HoodieFlinkWriteClient writeClient; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask(); + + cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + + HoodieFlinkEngineContext context = + new HoodieFlinkEngineContext(new SerializableConfiguration(new org.apache.hadoop.conf.Configuration()), new FlinkTaskContextSupplier(getRuntimeContext())); + + writeClient = new HoodieFlinkWriteClient<>(context, StreamerUtil.getHoodieClientConfig(cfg)); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) { + + // get latest requested instant + String commitType = cfg.tableType.equals(HoodieTableType.COPY_ON_WRITE.name()) ? HoodieTimeline.COMMIT_ACTION : HoodieTimeline.DELTA_COMMIT_ACTION; + List latestInstants = writeClient.getInflightsAndRequestedInstants(commitType); + latestInstant = latestInstants.isEmpty() ? null : latestInstants.get(0); + + if (bufferedRecords.size() > 0) { + hasRecordsIn = true; + if (output != null && latestInstant != null) { + String instantTimestamp = latestInstant; + LOG.info("Write records, subtask id = [{}] checkpoint_id = [{}}] instant = [{}], record size = [{}]", indexOfThisSubtask, context.getCheckpointId(), instantTimestamp, bufferedRecords.size()); + + List writeStatus; + switch (cfg.operation) { + case INSERT: + writeStatus = writeClient.insert(bufferedRecords, instantTimestamp); + break; + case UPSERT: + writeStatus = writeClient.upsert(bufferedRecords, instantTimestamp); + break; + default: + throw new HoodieFlinkStreamerException("Unknown operation : " + cfg.operation); + } + output.collect(new Tuple3<>(instantTimestamp, writeStatus, indexOfThisSubtask)); + bufferedRecords.clear(); + } + } else { + LOG.info("No data in subtask [{}]", indexOfThisSubtask); + hasRecordsIn = false; + } + } + + @Override + public void initializeState(FunctionInitializationContext functionInitializationContext) { + // no operation + } + + @Override + public void processElement(HoodieRecord hoodieRecord, Context context, Collector, Integer>> collector) { + if (output == null) { + output = collector; + } + + // buffer the records + bufferedRecords.add(hoodieRecord); + } + + public boolean hasRecordsIn() { + return hasRecordsIn; + } + + public String getLatestInstant() { + return latestInstant; + } + + @Override + public void close() { + if (writeClient != null) { + writeClient.close(); + } + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/operator/KeyedWriteProcessOperator.java b/hudi-flink/src/main/java/org/apache/hudi/operator/KeyedWriteProcessOperator.java new file mode 100644 index 0000000000000..beefe26fb26e1 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/operator/KeyedWriteProcessOperator.java @@ -0,0 +1,66 @@ +/* + * 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.operator; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * Operator helps to mock empty write results and deliver downstream when no data flow in some subtask. + */ +public class KeyedWriteProcessOperator extends KeyedProcessOperator, Integer>> { + + public static final String NAME = "WriteProcessOperator"; + private static final Logger LOG = LoggerFactory.getLogger(KeyedWriteProcessOperator.class); + private KeyedWriteProcessFunction writeProcessFunction; + + public KeyedWriteProcessOperator(KeyedProcessFunction, Integer>> function) { + super(function); + this.writeProcessFunction = (KeyedWriteProcessFunction) function; + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + // This super.snapshotState(context) triggers `writeProcessFunction.snapshotState()` method. which means the logic + // below will be executed after `writeProcessFunction.snapshotState()` method. + + // If there is no data flows in `writeProcessFunction`, it will never send anything downstream. so, in order to make + // sure each subtask will send a write status downstream, we implement this operator`s snapshotState() to mock empty + // write status and send it downstream when there is no data flows in some subtasks. + super.snapshotState(context); + + // make up an empty result and send downstream + if (!writeProcessFunction.hasRecordsIn() && writeProcessFunction.getLatestInstant() != null) { + String instantTime = writeProcessFunction.getLatestInstant(); + LOG.info("Mock empty writeStatus, subtaskId = [{}], instant = [{}]", getRuntimeContext().getIndexOfThisSubtask(), instantTime); + output.collect(new StreamRecord<>(new Tuple3(instantTime, new ArrayList(), getRuntimeContext().getIndexOfThisSubtask()))); + } + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java b/hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java new file mode 100644 index 0000000000000..f1746313f8e6d --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java @@ -0,0 +1,79 @@ +/* + * 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.schema; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.util.StreamerUtil; + +import java.io.IOException; +import java.util.Collections; + +/** + * A simple schema provider, that reads off files on DFS. + */ +public class FilebasedSchemaProvider extends SchemaProvider { + + /** + * Configs supported. + */ + public static class Config { + private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.file"; + private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.file"; + } + + private final FileSystem fs; + + private final Schema sourceSchema; + + private Schema targetSchema; + + public FilebasedSchemaProvider(TypedProperties props) { + super(props); + StreamerUtil.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_FILE_PROP)); + this.fs = FSUtils.getFs(props.getString(Config.SOURCE_SCHEMA_FILE_PROP), StreamerUtil.getHadoopConf()); + try { + this.sourceSchema = new Schema.Parser().parse(fs.open(new Path(props.getString(Config.SOURCE_SCHEMA_FILE_PROP)))); + if (props.containsKey(Config.TARGET_SCHEMA_FILE_PROP)) { + this.targetSchema = + new Schema.Parser().parse(fs.open(new Path(props.getString(Config.TARGET_SCHEMA_FILE_PROP)))); + } + } catch (IOException ioe) { + throw new HoodieIOException("Error reading schema", ioe); + } + } + + @Override + public Schema getSourceSchema() { + return sourceSchema; + } + + @Override + public Schema getTargetSchema() { + if (targetSchema != null) { + return targetSchema; + } else { + return super.getTargetSchema(); + } + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaProvider.java b/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaProvider.java new file mode 100644 index 0000000000000..f429a4cb4b202 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaProvider.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.schema; + +import org.apache.avro.Schema; +import org.apache.hudi.common.config.TypedProperties; + +import java.io.Serializable; + +/** + * Class to provide schema for reading data and also writing into a Hoodie table. + */ +public abstract class SchemaProvider implements Serializable { + + protected TypedProperties config; + + protected SchemaProvider(TypedProperties props) { + this.config = props; + } + + public abstract Schema getSourceSchema(); + + public Schema getTargetSchema() { + // by default, use source schema as target for hoodie table as well + return getSourceSchema(); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/CommitSink.java b/hudi-flink/src/main/java/org/apache/hudi/sink/CommitSink.java new file mode 100644 index 0000000000000..0660211b325c4 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/CommitSink.java @@ -0,0 +1,153 @@ +/* + * 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.sink; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.hudi.HudiFlinkStreamer; +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieFlinkStreamerException; +import org.apache.hudi.util.StreamerUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Function helps to execute commit operation. this operation should be executed only once. + */ +public class CommitSink extends RichSinkFunction, Integer>> { + + private static final Logger LOG = LoggerFactory.getLogger(CommitSink.class); + /** + * Job conf. + */ + private HudiFlinkStreamer.Config cfg; + + /** + * Write client. + */ + private transient HoodieFlinkWriteClient writeClient; + + /** + * Write result buffer. + */ + private Map>> bufferedWriteStatus = new HashMap<>(); + + /** + * Parallelism of this job. + */ + private Integer writeParallelSize = 0; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + // Get configs from runtimeContext + cfg = (HudiFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + + writeParallelSize = getRuntimeContext().getExecutionConfig().getParallelism(); + + // writeClient + writeClient = new HoodieFlinkWriteClient<>(new HoodieFlinkEngineContext(new FlinkTaskContextSupplier(null)), StreamerUtil.getHoodieClientConfig(cfg)); + } + + @Override + public void invoke(Tuple3, Integer> writeStatues, Context context) { + LOG.info("Receive records, instantTime = [{}], subtaskId = [{}], records size = [{}]", writeStatues.f0, writeStatues.f2, writeStatues.f1.size()); + try { + if (bufferedWriteStatus.containsKey(writeStatues.f0)) { + bufferedWriteStatus.get(writeStatues.f0).add(writeStatues.f1); + } else { + List> oneBatchData = new ArrayList<>(writeParallelSize); + oneBatchData.add(writeStatues.f1); + bufferedWriteStatus.put(writeStatues.f0, oneBatchData); + } + // check and commit + checkAndCommit(writeStatues.f0); + } catch (Exception e) { + throw new HoodieFlinkStreamerException("Invoke sink error", e); + } + } + + /** + * Check and commit if all subtask completed. + * + * @throws Exception + */ + private void checkAndCommit(String instantTime) throws Exception { + if (bufferedWriteStatus.get(instantTime).size() == writeParallelSize) { + LOG.info("Instant [{}] process complete, start commit!", instantTime); + doCommit(instantTime); + bufferedWriteStatus.clear(); + LOG.info("Instant [{}] commit completed!", instantTime); + } else { + LOG.info("Instant [{}], can not commit yet, subtask completed : [{}/{}]", instantTime, bufferedWriteStatus.get(instantTime).size(), writeParallelSize); + } + } + + private void doCommit(String instantTime) { + // get the records to commit + List writeResults = bufferedWriteStatus.get(instantTime).stream().flatMap(Collection::stream).collect(Collectors.toList()); + + // commit and rollback + long totalErrorRecords = writeResults.stream().map(WriteStatus::getTotalErrorRecords).reduce(Long::sum).orElse(0L); + long totalRecords = writeResults.stream().map(WriteStatus::getTotalRecords).reduce(Long::sum).orElse(0L); + boolean hasErrors = totalErrorRecords > 0; + + if (!hasErrors || cfg.commitOnErrors) { + HashMap checkpointCommitMetadata = new HashMap<>(); + if (hasErrors) { + LOG.warn("Some records failed to be merged but forcing commit since commitOnErrors set. Errors/Total=" + + totalErrorRecords + "/" + totalRecords); + } + + boolean success = writeClient.commit(instantTime, writeResults, Option.of(checkpointCommitMetadata)); + if (success) { + LOG.warn("Commit " + instantTime + " successful!"); + } else { + LOG.warn("Commit " + instantTime + " failed!"); + throw new HoodieException("Commit " + instantTime + " failed!"); + } + } else { + LOG.error("Streamer sync found errors when writing. Errors/Total=" + totalErrorRecords + "/" + totalRecords); + LOG.error("Printing out the top 100 errors"); + writeResults.stream().filter(WriteStatus::hasErrors).limit(100).forEach(ws -> { + LOG.error("Global error :", ws.getGlobalError()); + if (ws.getErrors().size() > 0) { + ws.getErrors().forEach((key, value) -> LOG.trace("Error for key:" + key + " is " + value)); + } + }); + // Rolling back instant + writeClient.rollback(instantTime); + throw new HoodieException("Commit " + instantTime + " failed and rolled-back !"); + } + } +} \ No newline at end of file diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/JsonStringToHoodieRecordMapFunction.java b/hudi-flink/src/main/java/org/apache/hudi/source/JsonStringToHoodieRecordMapFunction.java new file mode 100644 index 0000000000000..01d5cafc9b4c6 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/source/JsonStringToHoodieRecordMapFunction.java @@ -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.source; + +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.hudi.HudiFlinkStreamer; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.keygen.KeyGenerator; +import org.apache.hudi.schema.FilebasedSchemaProvider; +import org.apache.hudi.util.AvroConvertor; +import org.apache.hudi.util.StreamerUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * Function helps to transfer json string to {@link HoodieRecord}. + */ +public class JsonStringToHoodieRecordMapFunction implements MapFunction { + + private static Logger LOG = LoggerFactory.getLogger(JsonStringToHoodieRecordMapFunction.class); + + private final HudiFlinkStreamer.Config cfg; + private TypedProperties props; + private KeyGenerator keyGenerator; + private AvroConvertor avroConvertor; + + public JsonStringToHoodieRecordMapFunction(HudiFlinkStreamer.Config cfg) { + this.cfg = cfg; + init(); + } + + @Override + public HoodieRecord map(String value) throws Exception { + GenericRecord gr = avroConvertor.fromJson(value); + HoodieRecordPayload payload = StreamerUtil.createPayload(cfg.payloadClassName, gr, + (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false)); + + return new HoodieRecord<>(keyGenerator.getKey(gr), payload); + } + + private void init() { + this.props = StreamerUtil.getProps(cfg); + avroConvertor = new AvroConvertor(new FilebasedSchemaProvider(props).getSourceSchema()); + try { + keyGenerator = StreamerUtil.createKeyGenerator(props); + } catch (IOException e) { + LOG.error("Init keyGenerator failed ", e); + } + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/AvroConvertor.java b/hudi-flink/src/main/java/org/apache/hudi/util/AvroConvertor.java new file mode 100644 index 0000000000000..cf2071ee5561b --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/util/AvroConvertor.java @@ -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.util; + +import com.twitter.bijection.Injection; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.avro.MercifulJsonConverter; + +import java.io.Serializable; + +/** + * Convert a variety of datum into Avro GenericRecords. Has a bunch of lazy fields to circumvent issues around + * serializing these objects from driver to executors + */ +public class AvroConvertor implements Serializable { + + private static final long serialVersionUID = 1L; + /** + * To be lazily inited on executors. + */ + private transient Schema schema; + + private final String schemaStr; + + /** + * To be lazily inited on executors. + */ + private transient MercifulJsonConverter jsonConverter; + + + /** + * To be lazily inited on executors. + */ + private transient Injection recordInjection; + + public AvroConvertor(String schemaStr) { + this.schemaStr = schemaStr; + } + + public AvroConvertor(Schema schema) { + this.schemaStr = schema.toString(); + this.schema = schema; + } + + private void initSchema() { + if (schema == null) { + Schema.Parser parser = new Schema.Parser(); + schema = parser.parse(schemaStr); + } + } + + private void initJsonConvertor() { + if (jsonConverter == null) { + jsonConverter = new MercifulJsonConverter(); + } + } + + public GenericRecord fromJson(String json) { + initSchema(); + initJsonConvertor(); + return jsonConverter.convert(json, schema); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java new file mode 100644 index 0000000000000..c5f62691316fe --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -0,0 +1,149 @@ +/* + * 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.util; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.HudiFlinkStreamer; +import org.apache.hudi.common.config.DFSPropertiesConfiguration; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.KeyGenerator; +import org.apache.hudi.keygen.SimpleAvroKeyGenerator; +import org.apache.hudi.schema.FilebasedSchemaProvider; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.StringReader; +import java.util.List; +import java.util.Properties; + +public class StreamerUtil { + + private static Logger LOG = LoggerFactory.getLogger(StreamerUtil.class); + + public static Properties getKafkaProps(HudiFlinkStreamer.Config cfg) { + Properties result = new Properties(); + result.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cfg.kafkaBootstrapServers); + result.put(ConsumerConfig.GROUP_ID_CONFIG, cfg.kafkaGroupId); + return result; + } + + public static TypedProperties getProps(HudiFlinkStreamer.Config cfg) { + return readConfig( + FSUtils.getFs(cfg.propsFilePath, getHadoopConf()), + new Path(cfg.propsFilePath), cfg.configs).getConfig(); + } + + + /** + * Read conig from files. + */ + public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List overriddenProps) { + DFSPropertiesConfiguration conf; + try { + conf = new DFSPropertiesConfiguration(cfgPath.getFileSystem(fs.getConf()), cfgPath); + } catch (Exception e) { + conf = new DFSPropertiesConfiguration(); + LOG.warn("Unexpected error read props file at :" + cfgPath, e); + } + + try { + if (!overriddenProps.isEmpty()) { + LOG.info("Adding overridden properties to file properties."); + conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps)))); + } + } catch (IOException ioe) { + throw new HoodieIOException("Unexpected error adding config overrides", ioe); + } + + return conf; + } + + public static Configuration getHadoopConf() { + return new Configuration(); + } + + public static void checkRequiredProperties(TypedProperties props, List checkPropNames) { + checkPropNames.forEach(prop -> { + if (!props.containsKey(prop)) { + throw new HoodieNotSupportedException("Required property " + prop + " is missing"); + } + }); + } + + /** + * Create a key generator class via reflection, passing in any configs needed. + *

+ * If the class name of key generator is configured through the properties file, i.e., {@code props}, use the corresponding key generator class; otherwise, use the default key generator class + * specified in {@code DataSourceWriteOptions}. + */ + public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException { + String keyGeneratorClass = props.getString("hoodie.datasource.write.keygenerator.class", + SimpleAvroKeyGenerator.class.getName()); + try { + return (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props); + } catch (Throwable e) { + throw new IOException("Could not load key generator class " + keyGeneratorClass, e); + } + } + + /** + * Create a payload class via reflection, passing in an ordering/precombine value. + */ + public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal) + throws IOException { + try { + return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass, + new Class[] {GenericRecord.class, Comparable.class}, record, orderingVal); + } catch (Throwable e) { + throw new IOException("Could not create payload for class: " + payloadClass, e); + } + } + + public static HoodieWriteConfig getHoodieClientConfig(HudiFlinkStreamer.Config cfg) { + FileSystem fs = FSUtils.getFs(cfg.targetBasePath, getHadoopConf()); + HoodieWriteConfig.Builder builder = + HoodieWriteConfig.newBuilder().withPath(cfg.targetBasePath).combineInput(cfg.filterDupes, true) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build()) + .forTable(cfg.targetTableName) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) + .withAutoCommit(false) + .withProps(readConfig(fs, new Path(cfg.propsFilePath), cfg.configs) + .getConfig()); + + builder = builder.withSchema(new FilebasedSchemaProvider(getProps(cfg)).getTargetSchema().toString()); + HoodieWriteConfig config = builder.build(); + return config; + } + +} diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml new file mode 100644 index 0000000000000..80e828bf48d72 --- /dev/null +++ b/packaging/hudi-flink-bundle/pom.xml @@ -0,0 +1,286 @@ + + + + + + + hudi + org.apache.hudi + 0.6.1-SNAPSHOT + ../../pom.xml + + 4.0.0 + hudi-flink-bundle_${scala.binary.version} + jar + + + true + ${project.parent.basedir} + + + + + + org.apache.rat + apache-rat-plugin + + + org.apache.maven.plugins + maven-shade-plugin + ${maven-shade-plugin.version} + + + package + + shade + + + true + ${project.build.directory}/dependency-reduced-pom.xml + + + + + + true + + + META-INF/LICENSE + target/classes/META-INF/LICENSE + + + + + org.apache.hudi:hudi-common + org.apache.hudi:hudi-client-common + org.apache.hudi:hudi-flink-client + org.apache.hudi:hudi-flink_${scala.binary.version} + org.apache.hudi:hudi-timeline-service + + com.yammer.metrics:metrics-core + com.beust:jcommander + io.javalin:javalin + org.jetbrains.kotlin:* + org.rocksdb:rocksdbjni + org.apache.httpcomponents:httpclient + org.apache.httpcomponents:httpcore + org.apache.httpcomponents:fluent-hc + org.antlr:stringtemplate + org.apache.parquet:parquet-avro + org.apache.avro:avro + joda-time:joda-time + com.fasterxml.jackson.core:jackson-annotations + com.fasterxml.jackson.core:jackson-databind + com.fasterxml.jackson.core:jackson-core + + com.twitter:bijection-avro_${scala.binary.version} + com.twitter:bijection-core_${scala.binary.version} + io.confluent:kafka-avro-serializer + io.confluent:common-config + io.confluent:common-utils + io.confluent:kafka-schema-registry-client + org.apache.kafka:kafka-clients + io.dropwizard.metrics:metrics-core + io.dropwizard.metrics:metrics-graphite + io.prometheus:simpleclient + io.prometheus:simpleclient_httpserver + io.prometheus:simpleclient_dropwizard + io.prometheus:simpleclient_pushgateway + io.prometheus:simpleclient_common + com.yammer.metrics:metrics-core + org.apache.flink:flink-connector-kafka_${scala.binary.version} + org.apache.flink:flink-connector-kafka-base_${scala.binary.version} + org.apache.kafka:kafka_${scala.binary.version} + com.101tec:zkclient + org.apache.kafka:kafka-clients + + org.apache.hbase:hbase-client + org.apache.hbase:hbase-common + org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-server + org.apache.htrace:htrace-core + + + + + com.yammer.metrics. + org.apache.hudi.com.yammer.metrics. + + + com.beust.jcommander. + org.apache.hudi.com.beust.jcommander. + + + com.codahale.metrics. + org.apache.hudi.com.codahale.metrics. + + + org.apache.commons.codec. + org.apache.hudi.org.apache.commons.codec. + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/services/javax.* + + + + + + + + + + + + src/main/resources + + + src/test/resources + + + + + + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hudi + hudi-client-common + ${project.version} + + + org.apache.hudi + hudi-flink-client + ${project.version} + + + org.apache.hudi + hudi-flink_${scala.binary.version} + ${project.version} + + + + + org.apache.flink + flink-connector-kafka_${scala.binary.version} + compile + + + org.apache.flink + flink-connector-kafka-base_${scala.binary.version} + ${flink.version} + compile + + + + + org.apache.parquet + parquet-avro + compile + + + + + org.apache.avro + avro + compile + + + + joda-time + joda-time + compile + + + + + com.fasterxml.jackson.core + jackson-core + compile + + + com.fasterxml.jackson.core + jackson-databind + compile + + + com.fasterxml.jackson.core + jackson-annotations + compile + + + + org.apache.htrace + htrace-core + ${htrace.version} + compile + + + + + org.apache.hbase + hbase-common + ${hbase.version} + + + org.apache.hbase + hbase-server + ${hbase.version} + compile + + + javax.servlet + * + + + org.codehaus.jackson + * + + + org.mortbay.jetty + * + + + tomcat + * + + + + + org.apache.hbase + hbase-client + ${hbase.version} + + + org.apache.hbase + hbase-protocol + ${hbase.version} + + + \ No newline at end of file diff --git a/packaging/hudi-flink-bundle/src/main/java/org/apache/hudi/flink/bundle/Main.java b/packaging/hudi-flink-bundle/src/main/java/org/apache/hudi/flink/bundle/Main.java new file mode 100644 index 0000000000000..14c79a611999b --- /dev/null +++ b/packaging/hudi-flink-bundle/src/main/java/org/apache/hudi/flink/bundle/Main.java @@ -0,0 +1,36 @@ +/* + * 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.flink.bundle; + +import org.apache.hudi.common.util.ReflectionUtils; + +/** + * A simple main class to dump all classes loaded in current classpath + * + * This is a workaround for generating sources and javadoc jars for packaging modules. The maven plugins for generating + * javadoc and sources plugins do not generate corresponding jars if there are no source files. + * + * This class does not have anything to do with Hudi but is there to keep mvn javadocs/source plugin happy. + */ +public class Main { + + public static void main(String[] args) { + ReflectionUtils.getTopLevelClassesInClasspath(Main.class).forEach(System.out::println); + } +} diff --git a/pom.xml b/pom.xml index 2c786cfad81bc..80876cf11e2cc 100644 --- a/pom.xml +++ b/pom.xml @@ -53,6 +53,8 @@ hudi-integ-test packaging/hudi-integ-test-bundle hudi-examples + hudi-flink + packaging/hudi-flink-bundle @@ -100,6 +102,7 @@ 0.8.0 4.4.1 2.4.4 + 1.11.2 1.8.2 2.11.12 2.11 @@ -497,6 +500,26 @@ provided + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + provided + + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + provided + + + org.apache.flink + flink-connector-kafka_${scala.binary.version} + ${flink.version} + provided + + io.dropwizard.metrics @@ -980,7 +1003,7 @@ - +