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


##########
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:
   I looked into the `gateway.sendEvent` implementation. Underneath, when 
failure happens, it will trigger taskfailure asynchronously
   
https://github.com/apache/flink/blob/0104427dc9e38e898ba3865b499cc515004041c9/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/SubtaskGatewayImpl.java#L119
   ```
           final CompletableFuture<Acknowledge> result =
                   sendResult.whenCompleteAsync(
                           (success, failure) -> {
                               if (failure != null && 
subtaskAccess.isStillRunning()) {
                                   String msg =
                                           String.format(
                                                   EVENT_LOSS_ERROR_MESSAGE,
                                                   evt,
                                                   subtaskAccess.subtaskName());
                                   Runnables.assertNoException(
                                           () ->
                                                   
subtaskAccess.triggerTaskFailover(
                                                           new 
FlinkException(msg, failure)));
                               }
                           },
                           mainThreadExecutor);
   ```
   
   and cause exeuction graph fail
   
https://github.com/apache/flink/blob/0104427dc9e38e898ba3865b499cc515004041c9/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java#L1138
   ```
           if (!fromSchedulerNg) {
               vertex.getExecutionGraphAccessor()
                       .notifySchedulerNgAboutInternalTaskFailure(
                               attemptId, t, cancelTask, releasePartitions);
               return;
           }
   ```



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsAggregator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * GlobalStatisticsAggregator is used by {@link DataStatisticsCoordinator} to 
collect {@link
+ * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific 
checkpoint. It stores
+ * the merged {@link DataStatistics} result and uses set to keep a record of 
all reported subtasks.
+ */
+class GlobalStatisticsAggregator<K> implements Serializable {
+  private static final Logger LOG = 
LoggerFactory.getLogger(GlobalStatisticsAggregator.class);
+
+  private final long checkpointId;
+  private final DataStatistics<K> dataStatistics;
+  private final Set<Integer> subtaskSet = Sets.newHashSet();
+
+  GlobalStatisticsAggregator(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)) {
+      LOG.debug(
+          "Receive duplicated data statistics for checkpoint {} subtask {}. 
Ignore it.",

Review Comment:
   will update in latest commit



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java:
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingRunnable;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link
+ * DataStatisticsOperator} every subtask and then merge them together. Once 
aggregation for all
+ * subtasks data statistics completes, DataStatisticsCoordinator will send the 
aggregated
+ * result(global data statistics) back to {@link DataStatisticsOperator}. In 
the end a custom
+ * partitioner will distribute traffic based on the global data statistics to 
improve data
+ * clustering.
+ */
+@Internal
+class DataStatisticsCoordinator<K> implements OperatorCoordinator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataStatisticsCoordinator.class);
+
+  private static final double EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE = 
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 GlobalStatisticsAggregator<K> inProgressAggregator;
+  private volatile GlobalStatisticsAggregator<K> lastCompletedAggregator;
+  private volatile boolean started;
+
+  DataStatisticsCoordinator(
+      String operatorName,
+      OperatorCoordinator.Context context,
+      DataStatisticsFactory<K> statisticsFactory) {
+    this.operatorName = operatorName;
+    DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory 
coordinatorThreadFactory =
+        new DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory(
+            "DataStatisticsCoordinator-" + operatorName, 
context.getUserCodeClassloader());
+    this.coordinatorExecutor = 
Executors.newSingleThreadExecutor(coordinatorThreadFactory);
+    this.context =
+        new DataStatisticsCoordinatorContext<>(
+            coordinatorExecutor, coordinatorThreadFactory, context);
+    this.statisticsFactory = statisticsFactory;
+  }
+
+  @Override
+  public void start() throws Exception {
+    LOG.info("Starting data statistics coordinator for {}.", operatorName);
+    started = true;
+  }
+
+  @Override
+  public void close() throws Exception {
+    LOG.info("Closing data statistics coordinator for {}.", operatorName);
+    if (started) {
+      coordinatorExecutor.shutdown();

Review Comment:
   Tasks are submitted to the executor after start. So I add the check 
previously.
   For shutting down the executor, it indeed doesn't need to have the check `if 
(started)`.
   



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java:
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingRunnable;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link
+ * DataStatisticsOperator} every subtask and then merge them together. Once 
aggregation for all
+ * subtasks data statistics completes, DataStatisticsCoordinator will send the 
aggregated
+ * result(global data statistics) back to {@link DataStatisticsOperator}. In 
the end a custom
+ * partitioner will distribute traffic based on the global data statistics to 
improve data
+ * clustering.
+ */
+@Internal
+class DataStatisticsCoordinator<K> implements OperatorCoordinator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataStatisticsCoordinator.class);
+
+  private static final double EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE = 
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 GlobalStatisticsAggregator<K> inProgressAggregator;
+  private volatile GlobalStatisticsAggregator<K> lastCompletedAggregator;
+  private volatile boolean started;
+
+  DataStatisticsCoordinator(
+      String operatorName,
+      OperatorCoordinator.Context context,
+      DataStatisticsFactory<K> statisticsFactory) {
+    this.operatorName = operatorName;
+    DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory 
coordinatorThreadFactory =
+        new DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory(
+            "DataStatisticsCoordinator-" + operatorName, 
context.getUserCodeClassloader());
+    this.coordinatorExecutor = 
Executors.newSingleThreadExecutor(coordinatorThreadFactory);
+    this.context =
+        new DataStatisticsCoordinatorContext<>(
+            coordinatorExecutor, coordinatorThreadFactory, context);
+    this.statisticsFactory = statisticsFactory;
+  }
+
+  @Override
+  public void start() throws Exception {
+    LOG.info("Starting data statistics coordinator for {}.", operatorName);
+    started = true;
+  }
+
+  @Override
+  public void close() throws Exception {
+    LOG.info("Closing data statistics coordinator for {}.", operatorName);
+    if (started) {
+      coordinatorExecutor.shutdown();
+      if (!coordinatorExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
+        LOG.warn("Fail to shut down DataStatisticsCoordinator gracefully. 
Shutting down now");
+        coordinatorExecutor.shutdownNow();
+      }
+    }
+  }
+
+  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);
+
+            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) {

Review Comment:
   will replace it by Preconditions in latest commit



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink.shuffle;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingRunnable;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link
+ * DataStatisticsOperator} every subtask and then merge them together. Once 
aggregation for all
+ * subtasks data statistics completes, DataStatisticsCoordinator will send the 
aggregation
+ * result(global data statistics) back to {@link DataStatisticsOperator}. In 
the end a custom
+ * partitioner will distribute traffic based on the global data statistics to 
improve data
+ * clustering.
+ */
+class DataStatisticsCoordinator<K> implements OperatorCoordinator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataStatisticsCoordinator.class);
+
+  private static final double EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE = 
0.8;
+
+  private final String operatorName;
+  // A single-thread executor to handle all the actions for coordinator
+  private final ExecutorService coordinatorExecutor;
+  private final DataStatisticsCoordinatorContext<K> context;
+  private final DataStatisticsFactory<K> statisticsFactory;
+
+  private volatile AggregateDataStatistics<K> 
incompleteAggregateDataStatistics;
+  private volatile AggregateDataStatistics<K> completeAggregateDataStatistics;
+  private volatile boolean started;
+
+  DataStatisticsCoordinator(
+      String operatorName,
+      ExecutorService coordinatorExecutor,
+      DataStatisticsCoordinatorContext<K> context,
+      DataStatisticsFactory<K> statisticsFactory) {
+    this.operatorName = operatorName;
+    this.coordinatorExecutor = coordinatorExecutor;
+    this.context = context;
+    this.statisticsFactory = statisticsFactory;
+  }
+
+  @Override
+  public void start() throws Exception {
+    LOG.info("Starting data statistics coordinator for {}.", operatorName);
+    started = true;
+  }
+
+  @Override
+  public void close() throws Exception {
+    LOG.info("Closing data statistics coordinator for {}.", operatorName);
+    try {
+      if (started) {
+        context.close();
+      }
+    } finally {
+      coordinatorExecutor.shutdownNow();
+      // We do not expect this to actually block for long. At this point, 
there should
+      // be very few task running in the executor, if any.
+      coordinatorExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS);
+    }
+  }
+
+  private void runInCoordinatorThread(

Review Comment:
   I look into how to use `Tasks` as you suggested. We can do it like below
   ```
                 Tasks.foreach(String.format(actionName, 
actionNameFormatParameters))
                         .noRetry()
                         .executeWith(coordinatorExecutor)
                         .throwFailureWhenFinished()
                         .run(actionString -> {
                           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);
   
                             LOG.error(
                                     "Uncaught exception in the data statistics 
{} while {}. Triggering job failover.",
                                     operatorName,
                                     actionString,
                                     t);
                             context.failJob(t);
                           }
                         });
   ```
   But one thing I notice is, `Tasks.run` it's sync call. It will wait for the 
action to finish. 
   
https://github.com/apache/iceberg/blob/a9b8eff23b794c368bc596825e7c3084598c3dbd/core/src/main/java/org/apache/iceberg/util/Tasks.java#L358
   
   The behavior we would need in coordinator is async call. 
   



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

Review Comment:
   I see what you're saying. I will update the value to 80.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsAggregator.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * GlobalStatisticsAggregator is used by {@link DataStatisticsCoordinator} to 
collect {@link
+ * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific 
checkpoint. It stores
+ * the merged {@link DataStatistics} result and uses set to keep a record of 
all reported subtasks.
+ */
+class GlobalStatisticsAggregator<K> implements Serializable {
+  private static final Logger LOG = 
LoggerFactory.getLogger(GlobalStatisticsAggregator.class);
+
+  private final long checkpointId;
+  private final DataStatistics<K> dataStatistics;
+  private final Set<Integer> subtaskSet = Sets.newHashSet();
+
+  GlobalStatisticsAggregator(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)) {
+      LOG.debug(
+          "Receive duplicated data statistics for checkpoint {} subtask {}. 
Ignore it.",
+          checkpointId,
+          subtask);
+      return;
+    }
+
+    dataStatistics.merge(event.dataStatistics());
+  }
+
+  long aggregatedSubtasksCount() {

Review Comment:
   yes, you're right. Subtask id is in integer range. I will update the type



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java:
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingRunnable;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.jetbrains.annotations.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link
+ * DataStatisticsOperator} every subtask and then merge them together. Once 
aggregation for all
+ * subtasks data statistics completes, DataStatisticsCoordinator will send the 
aggregated
+ * result(global data statistics) back to {@link DataStatisticsOperator}. In 
the end a custom
+ * partitioner will distribute traffic based on the global data statistics to 
improve data
+ * clustering.
+ */
+@Internal
+class DataStatisticsCoordinator<K> implements OperatorCoordinator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataStatisticsCoordinator.class);
+
+  private static final double EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE = 
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 GlobalStatisticsAggregator<K> inProgressAggregator;
+  private volatile GlobalStatisticsAggregator<K> lastCompletedAggregator;
+  private volatile boolean started;
+
+  DataStatisticsCoordinator(
+      String operatorName,
+      OperatorCoordinator.Context context,
+      DataStatisticsFactory<K> statisticsFactory) {
+    this.operatorName = operatorName;
+    DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory 
coordinatorThreadFactory =
+        new DataStatisticsCoordinatorProvider.CoordinatorExecutorThreadFactory(
+            "DataStatisticsCoordinator-" + operatorName, 
context.getUserCodeClassloader());
+    this.coordinatorExecutor = 
Executors.newSingleThreadExecutor(coordinatorThreadFactory);
+    this.context =
+        new DataStatisticsCoordinatorContext<>(
+            coordinatorExecutor, coordinatorThreadFactory, context);
+    this.statisticsFactory = statisticsFactory;
+  }
+
+  @Override
+  public void start() throws Exception {
+    LOG.info("Starting data statistics coordinator for {}.", operatorName);
+    started = true;
+  }
+
+  @Override
+  public void close() throws Exception {
+    LOG.info("Closing data statistics coordinator for {}.", operatorName);
+    if (started) {
+      coordinatorExecutor.shutdown();
+      if (!coordinatorExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
+        LOG.warn("Fail to shut down DataStatisticsCoordinator gracefully. 
Shutting down now");
+        coordinatorExecutor.shutdownNow();
+      }
+    }
+  }
+
+  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);
+
+            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 (lastCompletedAggregator != null && 
lastCompletedAggregator.checkpointId() >= checkpointId) {
+      LOG.debug(
+          "Data statistics aggregation for checkpoint {} has completed. Ignore 
the event from subtask {} for checkpoint {}",
+          lastCompletedAggregator.checkpointId(),
+          subtask,
+          checkpointId);
+      return;
+    }
+
+    if (inProgressAggregator == null) {
+      inProgressAggregator = new GlobalStatisticsAggregator<>(checkpointId, 
statisticsFactory);
+    }
+
+    if (inProgressAggregator.checkpointId() < checkpointId) {
+      if ((double) inProgressAggregator.aggregatedSubtasksCount() / 
context.parallelism()
+          >= EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE) {
+        lastCompletedAggregator = inProgressAggregator;
+        LOG.info(
+            "Received data statistics from {} operators out of total {} for 
checkpoint {}. "
+                + "It's more than the expected percentage {}. Sending the 
aggregate data"
+                + " statistics {} to subtasks.",
+            inProgressAggregator.aggregatedSubtasksCount(),
+            context.parallelism(),
+            inProgressAggregator.checkpointId(),
+            EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE,
+            lastCompletedAggregator);
+        inProgressAggregator = new GlobalStatisticsAggregator<>(checkpointId, 
statisticsFactory);
+        inProgressAggregator.mergeDataStatistic(subtask, event);
+        context.sendDataStatisticsToSubtasks(
+            inProgressAggregator.checkpointId(), 
lastCompletedAggregator.dataStatistics());
+        return;
+      } else {
+        LOG.info(
+            "Received data statistics from {} operators out of total {} for 
checkpoint {}. "
+                + "It's less than the expected percentage {}. Dropping the 
incomplete aggregate "
+                + "data statistics {} and starting collecting data statistics 
from new checkpoint {}",
+            inProgressAggregator.aggregatedSubtasksCount(),
+            context.parallelism(),
+            inProgressAggregator.checkpointId(),
+            EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE,
+            inProgressAggregator,
+            checkpointId);
+        inProgressAggregator = new GlobalStatisticsAggregator<>(checkpointId, 
statisticsFactory);
+      }
+    } else if (inProgressAggregator.checkpointId() > checkpointId) {
+      LOG.debug(
+          "Expect data statistics for checkpoint {}, but receive event from 
older checkpoint {}. Ignore it.",
+          inProgressAggregator.checkpointId(),
+          checkpointId);
+      return;
+    }
+
+    inProgressAggregator.mergeDataStatistic(subtask, event);
+
+    if (inProgressAggregator.aggregatedSubtasksCount() == 
context.parallelism()) {
+      lastCompletedAggregator = inProgressAggregator;
+      LOG.info(
+          "Received data statistics from all {} operators for checkpoint {}. 
Sending the aggregated data statistics {} to subtasks.",
+          context.parallelism(),
+          inProgressAggregator.checkpointId(),
+          lastCompletedAggregator.dataStatistics());
+      inProgressAggregator = null;
+      context.sendDataStatisticsToSubtasks(checkpointId, 
lastCompletedAggregator.dataStatistics());
+    }
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void handleEventFromOperator(int subtask, int attemptNumber, 
OperatorEvent event) {
+    runInCoordinatorThread(
+        () -> {
+          LOG.debug(
+              "Handling event from subtask {} (#{}) of {}: {}",
+              subtask,
+              attemptNumber,
+              operatorName,
+              event);
+          Preconditions.checkArgument(event instanceof DataStatisticsEvent);
+          handleDataStatisticRequest(subtask, ((DataStatisticsEvent<K>) 
event));
+        },
+        "handling operator event %s from data statistics operator subtask %d 
(#%d)",

Review Comment:
   It's the action name parameter. Function #runInCoordinatorThread will use it 
to generate error message.
   ```
     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);
   
               String actionString = String.format(actionName, 
actionNameFormatParameters);
               LOG.error(
                   "Uncaught exception in the data statistics {} while {}. 
Triggering job failover.",
                   operatorName,
                   actionString,
                   t);
               context.failJob(t);
             }
           });
   ```



##########
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:
   I debate with myself whether we should fail the job when 
sendDataStatisticsToSubtasks fails. 
   On one side, shuffling is not on the critical path. If we fail the job, will 
that impact data streaming
   But on the other side, we cannot just always log silently for failure. User 
should be aware of the issue since they enable the feature. 
   I checked `SourceCoordinatorContext` implementation. It doesn't check the 
`gate.sendEvent` response. Not sure how it handles communication failure. Still 
investigating. 



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