nsivabalan commented on a change in pull request #3727: URL: https://github.com/apache/hudi/pull/3727#discussion_r718849052
########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java ########## @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.compact; + +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.client.AbstractHoodieWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.CompactionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCompactionException; +import org.apache.hudi.table.HoodieCopyOnWriteTableOperation; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.avro.Schema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; + +import static java.util.stream.Collectors.toList; + +/** + * Base class helps to perform compact. + * + * @param <T> Type of payload in {@link org.apache.hudi.common.model.HoodieRecord} + */ +public class CompactHelpers<T extends HoodieRecordPayload> { + + private static final Logger LOG = LoggerFactory.getLogger(CompactHelpers.class); + + private CompactHelpers() { + } + + private static class CompactHelperHolder { + private static final CompactHelpers COMPACT_HELPERS = new CompactHelpers(); + } + + public static CompactHelpers newInstance() { + return CompactHelperHolder.COMPACT_HELPERS; + } + + public HoodieWriteMetadata<HoodieData<WriteStatus>> compact( + HoodieEngineContext context, HoodieTable table, + HoodieCopyOnWriteTableOperation copyOnWriteTableOperation, HoodieWriteConfig config, + String compactionInstantTime, AbstractHoodieWriteClient writeClient, + TaskContextSupplier taskContextSupplier) { + HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); + HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); + if (pendingCompactionTimeline.containsInstant(inflightInstant)) { + writeClient.rollbackInflightCompaction(inflightInstant, table); + table.getMetaClient().reloadActiveTimeline(); + } + + HoodieWriteMetadata<HoodieData<WriteStatus>> compactionMetadata = new HoodieWriteMetadata<>(); + compactionMetadata.setWriteStatuses(context.createEmptyHoodieData()); + try { + // generate compaction plan + // should support configurable commit metadata + HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan( + table.getMetaClient(), compactionInstantTime); + + if (compactionPlan == null || (compactionPlan.getOperations() == null) + || (compactionPlan.getOperations().isEmpty())) { + // do nothing. + LOG.info("No compaction plan for instant " + compactionInstantTime); + return compactionMetadata; + } + + HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + // Mark instant as compaction inflight + table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); + table.getMetaClient().reloadActiveTimeline(); + + HoodieTableMetaClient metaClient = table.getMetaClient(); Review comment: this method is too lengthy. Can we try to add 1 or 2 private methods. ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java ########## @@ -321,15 +324,14 @@ protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD<WriteSt protected JavaRDD<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) { HoodieSparkTable<T> table = HoodieSparkTable.create(config, context); preWrite(compactionInstantTime, WriteOperationType.COMPACT, table.getMetaClient()); - HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); - HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); - if (pendingCompactionTimeline.containsInstant(inflightInstant)) { - rollbackInflightCompaction(inflightInstant, table); - table.getMetaClient().reloadActiveTimeline(); - } compactionTimer = metrics.getCompactionCtx(); - HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = table.compact(context, compactionInstantTime); - JavaRDD<WriteStatus> statuses = compactionMetadata.getWriteStatuses(); + HoodieSparkCopyOnWriteTable sparkCopyOnWriteTable = + new HoodieSparkCopyOnWriteTable(config, context, table.getMetaClient()); + HoodieWriteMetadata<HoodieData<WriteStatus>> compactionMetadata = + CompactHelpers.newInstance().compact(context, table, sparkCopyOnWriteTable, + config, compactionInstantTime, this, new SparkTaskContextSupplier()); Review comment: same here, why not use existing context ? ########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java ########## @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.compact; + +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.client.AbstractHoodieWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.CompactionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCompactionException; +import org.apache.hudi.table.HoodieCopyOnWriteTableOperation; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.avro.Schema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; + +import static java.util.stream.Collectors.toList; + +/** + * Base class helps to perform compact. + * + * @param <T> Type of payload in {@link org.apache.hudi.common.model.HoodieRecord} + */ +public class CompactHelpers<T extends HoodieRecordPayload> { + + private static final Logger LOG = LoggerFactory.getLogger(CompactHelpers.class); + + private CompactHelpers() { + } + + private static class CompactHelperHolder { + private static final CompactHelpers COMPACT_HELPERS = new CompactHelpers(); + } + + public static CompactHelpers newInstance() { + return CompactHelperHolder.COMPACT_HELPERS; + } + + public HoodieWriteMetadata<HoodieData<WriteStatus>> compact( + HoodieEngineContext context, HoodieTable table, + HoodieCopyOnWriteTableOperation copyOnWriteTableOperation, HoodieWriteConfig config, + String compactionInstantTime, AbstractHoodieWriteClient writeClient, + TaskContextSupplier taskContextSupplier) { + HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); + HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); + if (pendingCompactionTimeline.containsInstant(inflightInstant)) { + writeClient.rollbackInflightCompaction(inflightInstant, table); + table.getMetaClient().reloadActiveTimeline(); + } + + HoodieWriteMetadata<HoodieData<WriteStatus>> compactionMetadata = new HoodieWriteMetadata<>(); + compactionMetadata.setWriteStatuses(context.createEmptyHoodieData()); + try { + // generate compaction plan + // should support configurable commit metadata + HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan( + table.getMetaClient(), compactionInstantTime); + + if (compactionPlan == null || (compactionPlan.getOperations() == null) + || (compactionPlan.getOperations().isEmpty())) { + // do nothing. + LOG.info("No compaction plan for instant " + compactionInstantTime); + return compactionMetadata; + } + + HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + // Mark instant as compaction inflight + table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); + table.getMetaClient().reloadActiveTimeline(); + + HoodieTableMetaClient metaClient = table.getMetaClient(); + TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); + + // Here we firstly use the table schema as the reader schema to read + // log file.That is because in the case of MergeInto, the config.getSchema may not + // the same with the table schema. + try { Review comment: I guess these are not invoked in flink engine? can you double check that please. HoodieFlinkMergeOnReadTableCompactor ########## File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java ########## @@ -68,6 +77,21 @@ public HoodieFlinkEngineContext(SerializableConfiguration hadoopConf, TaskContex this.runtimeContext = ((FlinkTaskContextSupplier) taskContextSupplier).getFlinkRuntimeContext(); } + @Override + public HoodieAccumulator createNewAccumulator() { Review comment: createNewLongAccumulator ########## File path: hudi-common/src/main/java/org/apache/hudi/common/function/SerializablePairFlatMapFunction.java ########## @@ -0,0 +1,33 @@ +/* + * 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.common.function; + +import org.apache.hudi.common.util.collection.Pair; + +import java.io.Serializable; +import java.util.stream.Stream; + +/** + * A function that returns a stream of key-value pairs (Pair<K, V>). Review comment: minor: can you fix java docs "<" ">" ########## File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java ########## @@ -86,6 +110,17 @@ public RuntimeContext getRuntimeContext() { .collect(Collectors.toList()); } + @Override + public <I, K, V> Stream<ImmutablePair<K, V>> mapPartitionsToPairAndReduceByKey( + Stream<I> data, SerializablePairFlatMapFunction<Iterator<I>, K, V> flatMapToPairFunc, + SerializableBiFunction<V, V, V> reduceFunc, int parallelism) { + return throwingFlatMapToPairWrapper(flatMapToPairFunc).apply(data.parallel().iterator()) + .collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream() Review comment: may I know where have we used the "parallelism" within this method? ########## File path: hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java ########## @@ -369,7 +372,14 @@ public void completeCompaction( protected List<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) { // only used for metadata table, the compaction happens in single thread try { - List<WriteStatus> writeStatuses = FlinkCompactHelpers.compact(compactionInstantTime, this); + HoodieFlinkCopyOnWriteTable flinkCopyOnWriteTable = new HoodieFlinkCopyOnWriteTable( + getConfig(), getEngineContext(), getHoodieTable().getMetaClient()); + HoodieWriteMetadata<HoodieData<WriteStatus>> writeMetadata = + CompactHelpers.newInstance().compact(context, getHoodieTable(), flinkCopyOnWriteTable, + config, compactionInstantTime, this, new FlinkTaskContextSupplier(null)); Review comment: why create a new FlinkTaskContextSupplier? dont' we have one already locally ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
