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


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregateDataStatistics.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.iceberg.flink.sink.shuffle;
+
+import java.io.Serializable;
+import java.util.Set;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+class AggregateDataStatistics<K> implements Serializable {
+
+  private final long checkpointId;
+  private final DataStatistics<K> dataStatistics;
+  private final Set<Integer> subtaskSet = Sets.newHashSet();
+
+  AggregateDataStatistics(long checkpoint, final DataStatisticsFactory<K> 
statisticsFactory) {
+    this.checkpointId = checkpoint;
+    this.dataStatistics = statisticsFactory.createDataStatistics();
+  }
+
+  long checkpointId() {
+    return checkpointId;
+  }
+
+  DataStatistics<K> dataStatistics() {
+    return dataStatistics;
+  }
+
+  void mergeDataStatistic(int subtask, DataStatisticsEvent<K> event) {
+    Preconditions.checkArgument(
+        checkpointId == event.checkpointId(),
+        "Received unexpected event from checkpoint %s. Expected checkpoint %s",
+        event.checkpointId(),
+        checkpointId);
+    if (!subtaskSet.add(subtask)) {
+      return;

Review Comment:
   maybe log a debug or info message in this case? normally it should happen.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregateDataStatistics.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.iceberg.flink.sink.shuffle;
+
+import java.io.Serializable;
+import java.util.Set;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+class AggregateDataStatistics<K> implements Serializable {
+
+  private final long checkpointId;
+  private final DataStatistics<K> dataStatistics;
+  private final Set<Integer> subtaskSet = Sets.newHashSet();
+
+  AggregateDataStatistics(long checkpoint, final DataStatisticsFactory<K> 
statisticsFactory) {
+    this.checkpointId = checkpoint;
+    this.dataStatistics = statisticsFactory.createDataStatistics();
+  }
+
+  long checkpointId() {
+    return checkpointId;
+  }
+
+  DataStatistics<K> dataStatistics() {
+    return dataStatistics;
+  }
+
+  void mergeDataStatistic(int subtask, DataStatisticsEvent<K> event) {
+    Preconditions.checkArgument(

Review Comment:
   should we fail in this case? another option is to reset the stats and set to 
start tracking for a new checkpoint. but then we should log the warning with 
some details. like for the aborted checkpoint cycle which subtasks haven't 
reported stats.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregateDataStatistics.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.iceberg.flink.sink.shuffle;
+
+import java.io.Serializable;
+import java.util.Set;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+
+class AggregateDataStatistics<K> implements Serializable {
+
+  private final long checkpointId;
+  private final DataStatistics<K> dataStatistics;
+  private final Set<Integer> subtaskSet = Sets.newHashSet();
+
+  AggregateDataStatistics(long checkpoint, final DataStatisticsFactory<K> 
statisticsFactory) {
+    this.checkpointId = checkpoint;
+    this.dataStatistics = statisticsFactory.createDataStatistics();
+  }
+
+  long checkpointId() {
+    return checkpointId;
+  }
+
+  DataStatistics<K> dataStatistics() {
+    return dataStatistics;
+  }
+
+  void mergeDataStatistic(int subtask, DataStatisticsEvent<K> event) {
+    Preconditions.checkArgument(
+        checkpointId == event.checkpointId(),
+        "Received unexpected event from checkpoint %s. Expected checkpoint %s",
+        event.checkpointId(),
+        checkpointId);
+    if (!subtaskSet.add(subtask)) {
+      return;
+    }
+
+    dataStatistics.merge(event.dataStatistics());
+  }
+
+  long aggregateSize() {

Review Comment:
   nit: the method name is not intuitive. maybe `reportedSubtasksCount`?



##########
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;
+
+  public 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();

Review Comment:
   it is a little weird that the coordinatorExecutor is not started in this 
class's `start` method, but it is shut down in the `close` method.



##########
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;
+
+  public 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);
+    }
+
+    if (incompleteAggregateDataStatistics.aggregateSize() == 
context.currentParallelism()) {
+      completeAggregateDataStatistics = incompleteAggregateDataStatistics;
+      LOG.info(
+          "Received data statistics from all {} operators for checkpoint {}. 
Sending the aggregated data statistics {} to subtasks.",
+          context.currentParallelism(),
+          incompleteAggregateDataStatistics.checkpointId(),
+          completeAggregateDataStatistics);
+      incompleteAggregateDataStatistics = null;
+      context.sendDataStatisticsToSubtasks(
+          checkpointId, completeAggregateDataStatistics.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);
+          if (event instanceof DataStatisticsEvent) {
+            handleDataStatisticRequest(subtask, ((DataStatisticsEvent<K>) 
event));
+          } else {

Review Comment:
   Iceberg style typically uses `Preconditions` for this type of check.



##########
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;
+
+  public 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) {

Review Comment:
   nit: `incompleteAggregateDataStatistics` -> `inProgressAggregation`?



##########
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;
+
+  public 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());

Review Comment:
   maybe rename `dataStatistics` to `complete`. kind of like 
`TaskWriter#complete` that returns the result.



##########
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;
+
+  public 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);
+    }
+
+    if (incompleteAggregateDataStatistics.aggregateSize() == 
context.currentParallelism()) {
+      completeAggregateDataStatistics = incompleteAggregateDataStatistics;
+      LOG.info(
+          "Received data statistics from all {} operators for checkpoint {}. 
Sending the aggregated data statistics {} to subtasks.",
+          context.currentParallelism(),
+          incompleteAggregateDataStatistics.checkpointId(),
+          completeAggregateDataStatistics);
+      incompleteAggregateDataStatistics = null;
+      context.sendDataStatisticsToSubtasks(
+          checkpointId, completeAggregateDataStatistics.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);
+          if (event instanceof DataStatisticsEvent) {
+            handleDataStatisticRequest(subtask, ((DataStatisticsEvent<K>) 
event));
+          } else {
+            throw new FlinkException("Unrecognized data statistics operator 
event: " + event);
+          }
+        },
+        "handling operator event %s from data statistics operator subtask %d 
(#%d)",
+        event,
+        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 =

Review Comment:
   this fine for now. In the future, we want a stable serializer. BTW, we only 
need to checkpoint the statistics, right? do we need to checkpoint other 
metadata fields (like checkpoint id and Set<Integer>)?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorContext.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.HashMap;
+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.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DataStatisticsCoordinatorContext<K> implements AutoCloseable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataStatisticsCoordinatorContext.class);
+  private final ExecutorService coordinatorExecutor;
+  private final OperatorCoordinator.Context operatorCoordinatorContext;
+  private final SubtaskGateways subtaskGateways;
+  private final 
DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory
+      coordinatorThreadFactory;
+
+  DataStatisticsCoordinatorContext(
+      ExecutorService coordinatorExecutor,
+      DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory 
coordinatorThreadFactory,
+      OperatorCoordinator.Context operatorCoordinatorContext) {
+    this.coordinatorExecutor = coordinatorExecutor;
+    this.coordinatorThreadFactory = coordinatorThreadFactory;
+    this.operatorCoordinatorContext = operatorCoordinatorContext;
+    this.subtaskGateways = new SubtaskGateways(currentParallelism());
+  }
+
+  @Override
+  public void close() throws Exception {
+    coordinatorExecutor.shutdown();
+    coordinatorExecutor.awaitTermination(Long.MAX_VALUE, 
TimeUnit.MILLISECONDS);
+  }
+
+  void sendDataStatisticsToSubtasks(long checkpointId, DataStatistics<K> 
globalDataStatistics) {
+    callInCoordinatorThread(
+        () -> {
+          DataStatisticsEvent<K> dataStatisticsEvent =
+              new DataStatisticsEvent<>(checkpointId, globalDataStatistics);
+          int parallelism = currentParallelism();
+          for (int i = 0; i < parallelism; ++i) {
+            
subtaskGateways.getOnlyGatewayAndCheckReady(i).sendEvent(dataStatisticsEvent);
+          }
+          return null;
+        },
+        String.format(
+            "Failed to send global data statistics %s for checkpoint %d",
+            globalDataStatistics, checkpointId));

Review Comment:
   be careful with logging. `globalDataStatistics` toString can be large, right?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorContext.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.HashMap;
+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.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DataStatisticsCoordinatorContext<K> implements AutoCloseable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataStatisticsCoordinatorContext.class);
+  private final ExecutorService coordinatorExecutor;
+  private final OperatorCoordinator.Context operatorCoordinatorContext;
+  private final SubtaskGateways subtaskGateways;
+  private final 
DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory
+      coordinatorThreadFactory;
+
+  DataStatisticsCoordinatorContext(
+      ExecutorService coordinatorExecutor,
+      DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory 
coordinatorThreadFactory,
+      OperatorCoordinator.Context operatorCoordinatorContext) {
+    this.coordinatorExecutor = coordinatorExecutor;
+    this.coordinatorThreadFactory = coordinatorThreadFactory;
+    this.operatorCoordinatorContext = operatorCoordinatorContext;
+    this.subtaskGateways = new SubtaskGateways(currentParallelism());
+  }
+
+  @Override
+  public void close() throws Exception {
+    coordinatorExecutor.shutdown();
+    coordinatorExecutor.awaitTermination(Long.MAX_VALUE, 
TimeUnit.MILLISECONDS);
+  }
+
+  void sendDataStatisticsToSubtasks(long checkpointId, DataStatistics<K> 
globalDataStatistics) {
+    callInCoordinatorThread(
+        () -> {
+          DataStatisticsEvent<K> dataStatisticsEvent =
+              new DataStatisticsEvent<>(checkpointId, globalDataStatistics);
+          int parallelism = currentParallelism();
+          for (int i = 0; i < parallelism; ++i) {
+            
subtaskGateways.getOnlyGatewayAndCheckReady(i).sendEvent(dataStatisticsEvent);
+          }
+          return null;
+        },
+        String.format(
+            "Failed to send global data statistics %s for checkpoint %d",
+            globalDataStatistics, checkpointId));
+  }
+
+  int currentParallelism() {
+    return operatorCoordinatorContext.currentParallelism();
+  }
+
+  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);
+  }
+
+  void failJob(Throwable cause) {
+    operatorCoordinatorContext.failJob(cause);
+  }
+
+  /**
+   * A helper method that delegates the callable to the coordinator thread if 
the current thread is
+   * not the coordinator thread, otherwise call the callable right away.
+   *
+   * @param callable the callable to delegate.
+   */
+  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);
+      }
+    }
+  }
+
+  private static class SubtaskGateways {
+    private final Map<Integer, OperatorCoordinator.SubtaskGateway>[] gateways;
+
+    private SubtaskGateways(int parallelism) {
+      this.gateways = new Map[parallelism];
+
+      for (int i = 0; i < parallelism; ++i) {
+        this.gateways[i] = new HashMap<>();
+      }
+    }
+
+    private void registerSubtaskGateway(OperatorCoordinator.SubtaskGateway 
gateway) {

Review Comment:
   wondering if we need debug logs for the subtask 
registrations/unregistrations/ready methods?



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

Review Comment:
   nit: aggregation -> aggregated`



##########
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;
+
+  public 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

Review Comment:
   nit: `completeAggregateDataStatistics` -> `lastCompletedAggregation`?



##########
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;
+
+  public 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()

Review Comment:
   this method is really long. seems to me some of the logic can be put into 
the `AggregateDataStatistics` class



##########
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;
+
+  public 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);

Review Comment:
   nit: Iceberg coding style doesn't use `final`



##########
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;
+
+  public 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);
+    }
+
+    if (incompleteAggregateDataStatistics.aggregateSize() == 
context.currentParallelism()) {
+      completeAggregateDataStatistics = incompleteAggregateDataStatistics;
+      LOG.info(
+          "Received data statistics from all {} operators for checkpoint {}. 
Sending the aggregated data statistics {} to subtasks.",
+          context.currentParallelism(),
+          incompleteAggregateDataStatistics.checkpointId(),
+          completeAggregateDataStatistics);
+      incompleteAggregateDataStatistics = null;
+      context.sendDataStatisticsToSubtasks(
+          checkpointId, completeAggregateDataStatistics.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);
+          if (event instanceof DataStatisticsEvent) {
+            handleDataStatisticRequest(subtask, ((DataStatisticsEvent<K>) 
event));
+          } else {
+            throw new FlinkException("Unrecognized data statistics operator 
event: " + event);
+          }
+        },
+        "handling operator event %s from data statistics operator subtask %d 
(#%d)",
+        event,

Review Comment:
   should we log just the event class name or the actually event toString? I 
thought the former is more appropriate.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorContext.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.HashMap;
+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.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DataStatisticsCoordinatorContext<K> implements AutoCloseable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataStatisticsCoordinatorContext.class);
+  private final ExecutorService coordinatorExecutor;
+  private final OperatorCoordinator.Context operatorCoordinatorContext;
+  private final SubtaskGateways subtaskGateways;
+  private final 
DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory
+      coordinatorThreadFactory;
+
+  DataStatisticsCoordinatorContext(
+      ExecutorService coordinatorExecutor,
+      DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory 
coordinatorThreadFactory,
+      OperatorCoordinator.Context operatorCoordinatorContext) {
+    this.coordinatorExecutor = coordinatorExecutor;
+    this.coordinatorThreadFactory = coordinatorThreadFactory;
+    this.operatorCoordinatorContext = operatorCoordinatorContext;
+    this.subtaskGateways = new SubtaskGateways(currentParallelism());
+  }
+
+  @Override
+  public void close() throws Exception {
+    coordinatorExecutor.shutdown();
+    coordinatorExecutor.awaitTermination(Long.MAX_VALUE, 
TimeUnit.MILLISECONDS);
+  }
+
+  void sendDataStatisticsToSubtasks(long checkpointId, DataStatistics<K> 
globalDataStatistics) {
+    callInCoordinatorThread(
+        () -> {
+          DataStatisticsEvent<K> dataStatisticsEvent =
+              new DataStatisticsEvent<>(checkpointId, globalDataStatistics);
+          int parallelism = currentParallelism();
+          for (int i = 0; i < parallelism; ++i) {
+            
subtaskGateways.getOnlyGatewayAndCheckReady(i).sendEvent(dataStatisticsEvent);
+          }
+          return null;
+        },
+        String.format(
+            "Failed to send global data statistics %s for checkpoint %d",
+            globalDataStatistics, checkpointId));
+  }
+
+  int currentParallelism() {

Review Comment:
   nit: maybe just `parallelism`?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorContext.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.HashMap;
+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.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DataStatisticsCoordinatorContext<K> implements AutoCloseable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataStatisticsCoordinatorContext.class);
+  private final ExecutorService coordinatorExecutor;
+  private final OperatorCoordinator.Context operatorCoordinatorContext;
+  private final SubtaskGateways subtaskGateways;
+  private final 
DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory
+      coordinatorThreadFactory;
+
+  DataStatisticsCoordinatorContext(
+      ExecutorService coordinatorExecutor,
+      DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory 
coordinatorThreadFactory,
+      OperatorCoordinator.Context operatorCoordinatorContext) {
+    this.coordinatorExecutor = coordinatorExecutor;
+    this.coordinatorThreadFactory = coordinatorThreadFactory;
+    this.operatorCoordinatorContext = operatorCoordinatorContext;
+    this.subtaskGateways = new SubtaskGateways(currentParallelism());
+  }
+
+  @Override
+  public void close() throws Exception {
+    coordinatorExecutor.shutdown();

Review Comment:
   coordinatorExecutor was also shutdown in the 
`DataStatisticsCoordinator#close` method. probably not necessarily to do it 
both 



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java:
##########
@@ -138,6 +137,14 @@ public void snapshotState(StateSnapshotContext context) 
throws Exception {
     localStatistics = statisticsFactory.createDataStatistics();
   }
 
+  @Override
+  public void notifyCheckpointComplete(long checkpointId) throws Exception {
+    super.notifyCheckpointComplete(checkpointId);
+    // Send global statistics to partitioners at checkpoint to update data 
distribution at the same

Review Comment:
   you meant TODO here?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorContext.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.HashMap;
+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.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DataStatisticsCoordinatorContext<K> implements AutoCloseable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataStatisticsCoordinatorContext.class);
+  private final ExecutorService coordinatorExecutor;
+  private final OperatorCoordinator.Context operatorCoordinatorContext;
+  private final SubtaskGateways subtaskGateways;
+  private final 
DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory
+      coordinatorThreadFactory;
+
+  DataStatisticsCoordinatorContext(
+      ExecutorService coordinatorExecutor,
+      DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory 
coordinatorThreadFactory,
+      OperatorCoordinator.Context operatorCoordinatorContext) {
+    this.coordinatorExecutor = coordinatorExecutor;
+    this.coordinatorThreadFactory = coordinatorThreadFactory;
+    this.operatorCoordinatorContext = operatorCoordinatorContext;
+    this.subtaskGateways = new SubtaskGateways(currentParallelism());
+  }
+
+  @Override
+  public void close() throws Exception {
+    coordinatorExecutor.shutdown();
+    coordinatorExecutor.awaitTermination(Long.MAX_VALUE, 
TimeUnit.MILLISECONDS);
+  }
+
+  void sendDataStatisticsToSubtasks(long checkpointId, DataStatistics<K> 
globalDataStatistics) {
+    callInCoordinatorThread(
+        () -> {
+          DataStatisticsEvent<K> dataStatisticsEvent =
+              new DataStatisticsEvent<>(checkpointId, globalDataStatistics);
+          int parallelism = currentParallelism();
+          for (int i = 0; i < parallelism; ++i) {
+            
subtaskGateways.getOnlyGatewayAndCheckReady(i).sendEvent(dataStatisticsEvent);

Review Comment:
   a failure to a single task will abort the for loop. wondering how should we 
handle it. we can catch and log. another option is to do the for loop outside 
and submit one Callable for each subtask. not sure which way is better.



-- 
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]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to