yegangy0718 commented on code in PR #7360: URL: https://github.com/apache/iceberg/pull/7360#discussion_r1171927828
########## 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 double checked gateway interface ``` CompletableFuture<Acknowledge> sendEvent(OperatorEvent evt); ``` The `sendEvent` is an async call. it returns `CompletableFuture`. If there is any failure for sending the event, `callInCoordinatorThread` won't fail. ########## 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 checked how SourceCoordiantorContext handles that. It calls CoordinatryThread for every subtask ``` void sendEventToSourceOperator(int subtaskId, OperatorEvent event) { checkSubtaskIndex(subtaskId); callInCoordinatorThread( () -> { final OperatorCoordinator.SubtaskGateway gateway = subtaskGateways.getOnlyGatewayAndCheckReady(subtaskId); gateway.sendEvent(event); return null; }, String.format("Failed to send event %s to subtask %d", event, subtaskId)); } ``` For the two options, execute the loop for all subtasks in coordinator thread vs define loop outside and then call coordiantor thread for every subtask, I feel that there is no mush difference. Since callInCoordinatorThread is sync call, any one fails/throws exception, the remaining one won't be executed. -- 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]
