hililiwei commented on code in PR #7360: URL: https://github.com/apache/iceberg/pull/7360#discussion_r1170049999
########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java: ########## @@ -0,0 +1,324 @@ +/* + * 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.iceberg.flink.sink.shuffle; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.jetbrains.annotations.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link + * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all + * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregation + * result(global data statistics) back to {@link DataStatisticsOperator}. In the end a custom + * partitioner will distribute traffic based on the global data statistics to improve data + * clustering. + */ +class DataStatisticsCoordinator<K> implements OperatorCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); + + private static final double EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE = 0.8; + + private final String operatorName; + // A single-thread executor to handle all the actions for coordinator + private final ExecutorService coordinatorExecutor; + private final DataStatisticsCoordinatorContext<K> context; + private final DataStatisticsFactory<K> statisticsFactory; + + private volatile AggregateDataStatistics<K> incompleteAggregateDataStatistics; + private volatile AggregateDataStatistics<K> completeAggregateDataStatistics; + private volatile boolean started; + + DataStatisticsCoordinator( + String operatorName, + ExecutorService coordinatorExecutor, + DataStatisticsCoordinatorContext<K> context, + DataStatisticsFactory<K> statisticsFactory) { + this.operatorName = operatorName; + this.coordinatorExecutor = coordinatorExecutor; + this.context = context; + this.statisticsFactory = statisticsFactory; + } + + @Override + public void start() throws Exception { + LOG.info("Starting data statistics coordinator for {}.", operatorName); + started = true; + } + + @Override + public void close() throws Exception { + LOG.info("Closing data statistics coordinator for {}.", operatorName); + try { + if (started) { + context.close(); + } + } finally { + coordinatorExecutor.shutdownNow(); + // We do not expect this to actually block for long. At this point, there should + // be very few task running in the executor, if any. + coordinatorExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS); + } + } + + private void runInCoordinatorThread( + ThrowingRunnable<Throwable> action, String actionName, Object... actionNameFormatParameters) { + ensureStarted(); + coordinatorExecutor.execute( + () -> { + try { + action.run(); + } catch (Throwable t) { + // if we have a JVM critical error, promote it immediately, there is a good + // chance the logging or job failing will not succeed anymore + ExceptionUtils.rethrowIfFatalErrorOrOOM(t); + + final String actionString = String.format(actionName, actionNameFormatParameters); + LOG.error( + "Uncaught exception in the data statistics {} while {}. Triggering job failover.", + operatorName, + actionString, + t); + context.failJob(t); + } + }); + } + + private void ensureStarted() { + if (!this.started) { + throw new IllegalStateException("The coordinator has not started yet."); + } + } + + private void handleDataStatisticRequest(int subtask, DataStatisticsEvent<K> event) { + long checkpointId = event.checkpointId(); + + if (completeAggregateDataStatistics != null + && completeAggregateDataStatistics.checkpointId() >= checkpointId) { + LOG.debug( + "Data statistics aggregation for checkpoint {} has completed. Ignore the event from subtask {} for checkpoint {}", + completeAggregateDataStatistics.checkpointId(), + subtask, + checkpointId); + return; + } + + if (incompleteAggregateDataStatistics == null) { + incompleteAggregateDataStatistics = + new AggregateDataStatistics<>(checkpointId, statisticsFactory); + } + + if (incompleteAggregateDataStatistics.checkpointId() < checkpointId) { + if ((double) incompleteAggregateDataStatistics.aggregateSize() / context.currentParallelism() + >= EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE) { + completeAggregateDataStatistics = incompleteAggregateDataStatistics; + LOG.info( + "Received data statistics from {} operators out of total {} for checkpoint {}. It's more than the expected percentage {}. Thus sending the aggregate data statistics {} to subtasks.", + incompleteAggregateDataStatistics.aggregateSize(), + context.currentParallelism(), + incompleteAggregateDataStatistics.checkpointId(), + EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE, + completeAggregateDataStatistics); + incompleteAggregateDataStatistics = + new AggregateDataStatistics<>(checkpointId, statisticsFactory); + incompleteAggregateDataStatistics.mergeDataStatistic(subtask, event); + context.sendDataStatisticsToSubtasks( + incompleteAggregateDataStatistics.checkpointId(), + completeAggregateDataStatistics.dataStatistics()); + return; + } else { + LOG.info( + "Received data statistics from {} operators out of total {} for checkpoint {}. It's less than the expected percentage {}. Thus dropping the incomplete aggregate data statistics {} and starting collecting data statistics from new checkpoint {}", + incompleteAggregateDataStatistics.aggregateSize(), + context.currentParallelism(), + incompleteAggregateDataStatistics.checkpointId(), + EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE, + incompleteAggregateDataStatistics, + checkpointId); + incompleteAggregateDataStatistics = + new AggregateDataStatistics<>(checkpointId, statisticsFactory); + incompleteAggregateDataStatistics.mergeDataStatistic(subtask, event); + } + } else if (incompleteAggregateDataStatistics.checkpointId() > checkpointId) { + LOG.debug( + "Expect data statistics for checkpoint {}, but receive event from older checkpoint {}. Ignore it.", + incompleteAggregateDataStatistics.checkpointId(), + checkpointId); + return; + } else { + incompleteAggregateDataStatistics.mergeDataStatistic(subtask, event); Review Comment: It looks like we can delete 168 line, take this out of the last else condition? ``` ……………… return; } incompleteAggregateDataStatistics.mergeDataStatistic(subtask, event); ``` -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org