You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "yegangy0718 (via GitHub)" <gi...@apache.org> on 2023/05/01 05:56:35 UTC

[GitHub] [iceberg] yegangy0718 commented on a diff in pull request #7360: Flink: Implement data statistics coordinator to aggregate data statistics from operator subtasks

yegangy0718 commented on code in PR #7360:
URL: https://github.com/apache/iceberg/pull/7360#discussion_r1181387972


##########
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(

Review Comment:
   Thanks for the feedback!
   
   We have two functions using coordinatorExecutor. 
   One is called `runInCoordinatorThread` which only calls `coordinatorExecutor` to execute the `runnable` asynchronously.   `Tasks.run` waits for tasks to finish. Adding `throwFailureWhenProcessing` won't help too much unless we add new async run for `Tasks`.
   The other one is called `callInCoordinatorThread`  which is used to call `sendEvent` for every subtask in `sendDataStatisticsToSubtasks`.  This one is sync call which can be considered to use `Tasks`. 
   Current implementation for `callInCoordinatorThread` is like below
   ```
     private void sendDataStatisticsToSubtasks(long checkpointId, DataStatistics<K> globalDataStatistics) {
       callInCoordinatorThread(
               () -> {
                 DataStatisticsEvent<K> dataStatisticsEvent =
                         new DataStatisticsEvent<>(checkpointId, globalDataStatistics);
                 int parallelism = parallelism();
                 for (int i = 0; i < parallelism; ++i) {
                   subtaskGateways.getOnlyGatewayAndCheckReady(i).sendEvent(dataStatisticsEvent);
                 }
                 return null;
               },
               String.format("Failed to send global data statistics for checkpoint %d", checkpointId));
     }
   ```
   If coordinatorExecutor has more than one thread, then we can make full use of `Tasks` to send data statstics like below
   ```
     private void sendDataStatisticsToSubtasks(long checkpointId, DataStatistics<K> globalDataStatistics) throws Exception {
       try {
         Tasks.foreach(IntStream.range(0, parallelism()).iterator())
                 .noRetry()
                 .executeWith(coordinatorExecutor)
                 .throwFailureWhenFinished()
                 .run(subtask -> {
                   DataStatisticsEvent<K> dataStatisticsEvent =
                           new DataStatisticsEvent<>(checkpointId, globalDataStatistics);
                   subtaskGateways.getOnlyGatewayAndCheckReady(subtask).sendEvent(dataStatisticsEvent);
                 }, Exception.class);
       } catch (Exception e) {
         throw new FlinkRuntimeException(e.getMessage(), e);
       }
   }
   ```
   But the coordinatorExecutor is a single thread executor to prevent concurrent race conditions.  So using `Tasks` won't help too much for this use case. 
   



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.Executors;
+import org.apache.flink.annotation.Internal;
+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.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 aggregated
+ * 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.
+ */
+@Internal
+class DataStatisticsCoordinator<K> implements OperatorCoordinator {
+  private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class);
+
+  private static final double EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE = 80;
+
+  private final String operatorName;
+  private final DataStatisticsCoordinatorContext<K> context;
+  private final DataStatisticsFactory<K> statisticsFactory;
+
+  private volatile GlobalStatisticsAggregator<K> inProgressAggregator;
+  private volatile GlobalStatisticsAggregator<K> lastCompletedAggregator;
+  private volatile boolean started;
+
+  DataStatisticsCoordinator(
+      String operatorName,
+      OperatorCoordinator.Context context,
+      DataStatisticsFactory<K> statisticsFactory) {
+    this.operatorName = operatorName;
+    DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory coordinatorThreadFactory =
+        new DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory(
+            "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader());
+    this.context =
+        new DataStatisticsCoordinatorContext<>(
+            Executors.newSingleThreadExecutor(coordinatorThreadFactory),
+            coordinatorThreadFactory,
+            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);
+    context.close();
+    LOG.info("Data statistics coordinator for {} closed.", operatorName);
+  }
+
+  private void runInCoordinatorThread(
+      ThrowingRunnable<Throwable> action, String actionName, Object... actionNameFormatParameters) {
+    ensureStarted();
+    context.runInCoordinatorThread(
+        () -> {
+          try {
+            action.run();
+          } catch (Throwable t) {
+            ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
+            String actionString = String.format(actionName, actionNameFormatParameters);
+            LOG.error(
+                "Uncaught exception in the data statistics {} while {}. Triggering job failover.",
+                operatorName,
+                actionString,
+                t);
+
+            this.context.failJob(t);
+          }
+        });
+  }
+
+  private void ensureStarted() {
+    Preconditions.checkState(started, "The coordinator has not started yet.");
+  }
+
+  private void handleDataStatisticRequest(int subtask, DataStatisticsEvent<K> event) {
+    long checkpointId = event.checkpointId();
+
+    if (lastCompletedAggregator != null && lastCompletedAggregator.checkpointId() >= checkpointId) {
+      LOG.debug(
+          "Data statistics aggregation for checkpoint {} has completed. Ignore the event from subtask {} for checkpoint {}",
+          lastCompletedAggregator.checkpointId(),
+          subtask,
+          checkpointId);
+      return;
+    }
+
+    if (inProgressAggregator == null) {
+      inProgressAggregator = new GlobalStatisticsAggregator<>(checkpointId, statisticsFactory);
+    }
+
+    if (inProgressAggregator.checkpointId() < checkpointId) {
+      if ((double) inProgressAggregator.aggregatedSubtasksCount() / context.parallelism() * 100
+          >= EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE) {
+        lastCompletedAggregator = inProgressAggregator;
+        LOG.info(
+            "Received data statistics from {} operators out of total {} for checkpoint {}. "
+                + "It's more than the expected percentage {}. Sending the aggregate data"
+                + " statistics {} to subtasks.",
+            inProgressAggregator.aggregatedSubtasksCount(),
+            context.parallelism(),
+            inProgressAggregator.checkpointId(),
+            EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE,
+            lastCompletedAggregator);
+        inProgressAggregator = new GlobalStatisticsAggregator<>(checkpointId, statisticsFactory);
+        inProgressAggregator.mergeDataStatistic(subtask, event);
+        context.sendDataStatisticsToSubtasks(
+            inProgressAggregator.checkpointId(), lastCompletedAggregator.dataStatistics());
+        return;
+      } else {
+        LOG.info(
+            "Received data statistics from {} operators out of total {} for checkpoint {}. "
+                + "It's less than the expected percentage {}. Dropping the incomplete aggregate "
+                + "data statistics {} and starting collecting data statistics from new checkpoint {}",
+            inProgressAggregator.aggregatedSubtasksCount(),
+            context.parallelism(),
+            inProgressAggregator.checkpointId(),
+            EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE,
+            inProgressAggregator,
+            checkpointId);
+        inProgressAggregator = new GlobalStatisticsAggregator<>(checkpointId, statisticsFactory);
+      }
+    } else if (inProgressAggregator.checkpointId() > checkpointId) {
+      LOG.debug(
+          "Expect data statistics for checkpoint {}, but receive event from older checkpoint {}. Ignore it.",
+          inProgressAggregator.checkpointId(),
+          checkpointId);
+      return;
+    }
+
+    inProgressAggregator.mergeDataStatistic(subtask, event);
+
+    if (inProgressAggregator.aggregatedSubtasksCount() == context.parallelism()) {
+      lastCompletedAggregator = inProgressAggregator;
+      LOG.info(
+          "Received data statistics from all {} operators for checkpoint {}. Sending the aggregated data statistics {} to subtasks.",
+          context.parallelism(),
+          inProgressAggregator.checkpointId(),
+          lastCompletedAggregator.dataStatistics());
+      inProgressAggregator = null;
+      context.sendDataStatisticsToSubtasks(checkpointId, lastCompletedAggregator.dataStatistics());
+    }
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) {
+    runInCoordinatorThread(
+        () -> {
+          LOG.debug(
+              "Handling event from subtask {} (#{}) of {}: {}",
+              subtask,
+              attemptNumber,
+              operatorName,
+              event);
+          Preconditions.checkArgument(event instanceof DataStatisticsEvent);
+          handleDataStatisticRequest(subtask, ((DataStatisticsEvent<K>) event));
+        },
+        "handling operator event %s from data statistics operator subtask %d (#%d)",
+        event.getClass(),
+        subtask,
+        attemptNumber);
+  }
+
+  @Override
+  public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> resultFuture) {
+    runInCoordinatorThread(
+        () -> {
+          LOG.debug(
+              "Taking a state snapshot on data statistics coordinator {} for checkpoint {}",
+              operatorName,
+              checkpointId);
+          try {
+            byte[] serializedDataDistributionWeight =
+                InstantiationUtil.serializeObject(lastCompletedAggregator);
+            resultFuture.complete(serializedDataDistributionWeight);
+          } catch (Throwable e) {
+            ExceptionUtils.rethrowIfFatalErrorOrOOM(e);
+            resultFuture.completeExceptionally(
+                new CompletionException(
+                    String.format("Failed to checkpoint data statistics for %s", operatorName), e));
+          }
+        },
+        "taking checkpoint %d",
+        checkpointId);
+  }
+
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) {}
+
+  @Override
+  public void notifyCheckpointAborted(long checkpointId) {}
+
+  @Override
+  public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData)
+      throws Exception {
+    if (started) {
+      throw new IllegalStateException(
+          "The coordinator can only be reset if it was not yet started");
+    }
+
+    if (checkpointData == null) {
+      return;
+    }
+
+    LOG.info(
+        "Restoring data statistic coordinator {} from checkpoint {}.", operatorName, checkpointId);
+    lastCompletedAggregator =
+        InstantiationUtil.deserializeObject(

Review Comment:
   Yup, I will have a separate PR for the serializer



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java:
##########
@@ -0,0 +1,368 @@
+/*
+ * 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.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.annotation.Internal;
+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.FlinkRuntimeException;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.ThrowableCatchingRunnable;
+import org.apache.flink.util.function.ThrowingRunnable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+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 aggregated
+ * 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.
+ */
+@Internal
+class DataStatisticsCoordinator<K> implements OperatorCoordinator {
+  private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class);
+
+  private final String operatorName;
+  private final ExecutorService coordinatorExecutor;
+  private final OperatorCoordinator.Context operatorCoordinatorContext;
+  private final SubtaskGateways subtaskGateways;
+  private final DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory
+      coordinatorThreadFactory;
+  private final GlobalStatisticsAggregatorTracker<K> globalStatisticsAggregatorTracker;
+
+  private volatile boolean started;
+
+  DataStatisticsCoordinator(
+      String operatorName,
+      OperatorCoordinator.Context context,
+      DataStatisticsFactory<K> statisticsFactory) {
+    this.operatorName = operatorName;
+    this.coordinatorThreadFactory =
+        new DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory(
+            "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader());
+    this.coordinatorExecutor = Executors.newFixedThreadPool(2, coordinatorThreadFactory);
+    this.operatorCoordinatorContext = context;
+    this.subtaskGateways = new SubtaskGateways(parallelism());
+    this.globalStatisticsAggregatorTracker =
+        new GlobalStatisticsAggregatorTracker<>(statisticsFactory, parallelism());
+  }
+
+  @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);
+    coordinatorExecutor.shutdown();
+    try {
+      if (!coordinatorExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
+        LOG.warn(
+            "Fail to shut down data statistics coordinator context gracefully. Shutting down now");
+        coordinatorExecutor.shutdownNow();
+        if (!coordinatorExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
+          LOG.warn("Fail to terminate data statistics coordinator context");
+          return;
+        }
+      }
+      LOG.info("Data statistics coordinator context closed.");
+    } catch (InterruptedException e) {
+      coordinatorExecutor.shutdownNow();
+      Thread.currentThread().interrupt();
+      LOG.error("Errors occurred while closing the data statistics coordinator context", e);
+    }
+
+    LOG.info("Data statistics coordinator for {} closed.", operatorName);
+  }
+
+  void callInCoordinatorThread(Callable<Void> callable, String errorMessage) {
+    // Ensure the task is done by the coordinator executor.
+    if (!coordinatorThreadFactory.isCurrentThreadCoordinatorThread()) {
+      try {
+        final Callable<Void> guardedCallable =
+            () -> {
+              try {
+                return callable.call();
+              } catch (Throwable t) {
+                LOG.error("Uncaught Exception in DataStatistics Coordinator Executor", t);
+                ExceptionUtils.rethrowException(t);
+                return null;
+              }
+            };
+
+        coordinatorExecutor.submit(guardedCallable).get();
+      } catch (InterruptedException | ExecutionException e) {
+        throw new FlinkRuntimeException(errorMessage, e);
+      }
+    } else {
+      try {
+        callable.call();
+      } catch (Throwable t) {
+        LOG.error("Uncaught Exception in DataStatistics coordinator executor", t);
+        throw new FlinkRuntimeException(errorMessage, t);
+      }
+    }
+  }
+
+  public void runInCoordinatorThread(Runnable runnable) {
+    this.coordinatorExecutor.execute(
+        new ThrowableCatchingRunnable(
+            (throwable) -> {
+              this.coordinatorThreadFactory.uncaughtException(Thread.currentThread(), throwable);
+            },
+            runnable));
+  }
+
+  private void runInCoordinatorThread(
+      ThrowingRunnable<Throwable> action, String actionName, Object... actionNameFormatParameters) {
+    ensureStarted();
+    runInCoordinatorThread(
+        () -> {
+          try {
+            action.run();
+          } catch (Throwable t) {
+            ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
+            String actionString = String.format(actionName, actionNameFormatParameters);
+            LOG.error(
+                "Uncaught exception in the data statistics {} while {}. Triggering job failover.",
+                operatorName,
+                actionString,
+                t);
+            operatorCoordinatorContext.failJob(t);
+          }
+        });
+  }
+
+  private void ensureStarted() {
+    Preconditions.checkState(started, "The coordinator has not started yet.");
+  }
+
+  private int parallelism() {
+    return operatorCoordinatorContext.currentParallelism();
+  }
+
+  private void handleDataStatisticRequest(int subtask, DataStatisticsEvent<K> event) {
+    if (globalStatisticsAggregatorTracker.receiveDataStatisticEventAndCheckCompletion(
+        subtask, event)) {
+      GlobalStatisticsAggregator<K> lastCompletedAggregator =
+          globalStatisticsAggregatorTracker.lastCompletedAggregator();
+      sendDataStatisticsToSubtasks(
+          lastCompletedAggregator.checkpointId(), lastCompletedAggregator.dataStatistics());
+    }
+  }
+
+  private void sendDataStatisticsToSubtasks(
+      long checkpointId, DataStatistics<K> globalDataStatistics) {
+    callInCoordinatorThread(
+        () -> {
+          DataStatisticsEvent<K> dataStatisticsEvent =
+              new DataStatisticsEvent<>(checkpointId, globalDataStatistics);
+          int parallelism = parallelism();
+          for (int i = 0; i < parallelism; ++i) {
+            subtaskGateways.getOnlyGatewayAndCheckReady(i).sendEvent(dataStatisticsEvent);
+          }
+          return null;
+        },
+        String.format("Failed to send global data statistics for checkpoint %d", checkpointId));
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) {
+    runInCoordinatorThread(
+        () -> {
+          LOG.debug(
+              "Handling event from subtask {} (#{}) of {}: {}",
+              subtask,
+              attemptNumber,
+              operatorName,
+              event);
+          Preconditions.checkArgument(event instanceof DataStatisticsEvent);
+          handleDataStatisticRequest(subtask, ((DataStatisticsEvent<K>) event));
+        },
+        "handling operator event %s from subtask %d (#%d)",
+        event.getClass(),
+        subtask,
+        attemptNumber);
+  }
+
+  @Override
+  public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> resultFuture) {
+    runInCoordinatorThread(
+        () -> {
+          LOG.debug(
+              "Taking a state snapshot on data statistics coordinator {} for checkpoint {}",
+              operatorName,
+              checkpointId);
+          try {
+            byte[] serializedDataDistributionWeight =
+                InstantiationUtil.serializeObject(
+                    globalStatisticsAggregatorTracker.lastCompletedAggregator());

Review Comment:
   Only save the `lastCompletedAggregator` during checkpoint so that when resetting checkpoint we also only restore `lastCompletedAggregator`. The reason for this is, if we serialize globalStatisticsAggregatorTracker which contains parallelism, inprogressAggregator, and lastCompletedAggregator, the parallelism may be changed when resetting checkpoint, and inprogressAggregator is useless. 



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.Executors;
+import org.apache.flink.annotation.Internal;
+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.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 aggregated
+ * 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.
+ */
+@Internal
+class DataStatisticsCoordinator<K> implements OperatorCoordinator {
+  private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class);
+
+  private static final double EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE = 80;
+
+  private final String operatorName;
+  private final DataStatisticsCoordinatorContext<K> context;
+  private final DataStatisticsFactory<K> statisticsFactory;
+
+  private volatile GlobalStatisticsAggregator<K> inProgressAggregator;
+  private volatile GlobalStatisticsAggregator<K> lastCompletedAggregator;
+  private volatile boolean started;
+
+  DataStatisticsCoordinator(
+      String operatorName,
+      OperatorCoordinator.Context context,
+      DataStatisticsFactory<K> statisticsFactory) {
+    this.operatorName = operatorName;
+    DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory coordinatorThreadFactory =
+        new DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory(
+            "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader());
+    this.context =
+        new DataStatisticsCoordinatorContext<>(
+            Executors.newSingleThreadExecutor(coordinatorThreadFactory),
+            coordinatorThreadFactory,
+            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);
+    context.close();
+    LOG.info("Data statistics coordinator for {} closed.", operatorName);
+  }
+
+  private void runInCoordinatorThread(
+      ThrowingRunnable<Throwable> action, String actionName, Object... actionNameFormatParameters) {
+    ensureStarted();
+    context.runInCoordinatorThread(
+        () -> {
+          try {
+            action.run();
+          } catch (Throwable t) {
+            ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
+            String actionString = String.format(actionName, actionNameFormatParameters);
+            LOG.error(
+                "Uncaught exception in the data statistics {} while {}. Triggering job failover.",
+                operatorName,
+                actionString,
+                t);
+
+            this.context.failJob(t);
+          }
+        });
+  }
+
+  private void ensureStarted() {
+    Preconditions.checkState(started, "The coordinator has not started yet.");
+  }
+
+  private void handleDataStatisticRequest(int subtask, DataStatisticsEvent<K> event) {
+    long checkpointId = event.checkpointId();
+
+    if (lastCompletedAggregator != null && lastCompletedAggregator.checkpointId() >= checkpointId) {

Review Comment:
   Current `GlobalStatisticsAggregator` implementation contains the information for 
   ```
     private final long checkpointId;
     private final DataStatistics<K> dataStatistics;
     private final Set<Integer> subtaskSet = Sets.newHashSet();
   ```
   In `coordinator`, we need to store both `inProgressAggregator` and  `lastCompletedAggregator`.
   If we want to move the logic into `GlobalStatisticsAggregator`, we need to update the constructor to store
   ```
     private final long inprogressCheckpointId;
     private final long lastCompletedCheckpointId;
     private final DataStatistics<K> inprogressDataStatistics;
     private final Set<Integer>  inprogressSubtaskSet;
   ```
   I end up defining a new class in `coordinator` called `GlobalStatisticsAggregatorTracker` which stores `inProgressAggregator` and `lastCompletedAggregator` 
   



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.Executors;
+import org.apache.flink.annotation.Internal;
+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.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 aggregated
+ * 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.
+ */
+@Internal
+class DataStatisticsCoordinator<K> implements OperatorCoordinator {
+  private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class);
+
+  private static final double EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE = 80;

Review Comment:
   will update it to 90%



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorContext.java:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.ThrowableCatchingRunnable;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DataStatisticsCoordinatorContext is used by {@link DataStatisticsCoordinator} to communicate with
+ * {@link DataStatisticsOperator} via {@link OperatorCoordinator.SubtaskGateway}.
+ */
+@Internal
+class DataStatisticsCoordinatorContext<K> implements AutoCloseable {

Review Comment:
   When implementing it, I have thought about it. For me, having a context class which is responsible for all the sub gateway communication makes sense.  But from the other side, this introduces tedious code for the below functions 
   ```
     void attemptReady(OperatorCoordinator.SubtaskGateway gateway) {
       Preconditions.checkState(this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread());
       this.subtaskGateways.registerSubtaskGateway(gateway);
     }
   
     void attemptFailed(int subtaskIndex, int attemptNumber) {
       Preconditions.checkState(this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread());
       this.subtaskGateways.unregisterSubtaskGateway(subtaskIndex, attemptNumber);
     }
   
     void subtaskReset(int subtaskIndex) {
       Preconditions.checkState(this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread());
       this.subtaskGateways.reset(subtaskIndex);
     }
   ```
   I can move the sub gateway logic into `coordinator` and let's see if it's cleaner



-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org