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


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

Review Comment:
   Just out of curiosity, as stated in the above log, this results in a full 
job failure and recovery to the latest complete checkpoint. Does it cause some 
additional burden?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

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


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

Reply via email to