zhuzhurk commented on code in PR #24771:
URL: https://github.com/apache/flink/pull/24771#discussion_r1607711905
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IOMetrics.java:
##########
@@ -145,6 +143,8 @@ public long getAccumulateIdleTime() {
}
public Map<IntermediateResultPartitionID, ResultPartitionBytes>
getResultPartitionBytes() {
- return Collections.unmodifiableMap(checkNotNull(resultPartitionBytes));
+ return resultPartitionBytes == null
Review Comment:
In which case it can be `null` when this method is used?
##########
flink-core-api/src/main/java/org/apache/flink/util/function/ConsumerWithException.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.flink.util.function;
+
+import java.util.function.Consumer;
+
+/**
+ * A checked extension of the {@link Consumer} interface.
+ *
+ * @param <T> type of the argument
+ * @param <E> type of the thrown exception
+ */
+@FunctionalInterface
+public interface ConsumerWithException<T, E extends Throwable> {
+
+ /**
+ * Performs this operation on the given arguments.
+ *
+ * @param t the first input argument
Review Comment:
arguments -> argument
the first input argument -> the input argument
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java:
##########
@@ -499,8 +516,12 @@ public CompletableFuture<Void>
registerProducedPartitions(TaskManagerLocation lo
});
}
- private static int getPartitionMaxParallelism(IntermediateResultPartition
partition) {
- return partition.getIntermediateResult().getConsumersMaxParallelism();
+ public static ResultPartitionDeploymentDescriptor
createResultPartitionDeploymentDescriptor(
+ IntermediateResultPartition partition, ShuffleDescriptor
shuffleDescriptor) {
Review Comment:
I prefer to reuse the `partitionDescriptor` which was already created.
Looks to me that the creation of `partitionDescriptor` is not very light
weighted(e.g. `isNumberOfPartitionConsumersUndefined()`) so I would avoid doing
it twice if possible.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java:
##########
@@ -453,6 +453,26 @@ public CompletableFuture<Void>
registerProducedPartitions(TaskManagerLocation lo
});
}
+ private void recoverAttempt(ExecutionAttemptID newId) {
+ if (!this.attemptId.equals(newId)) {
+ getVertex().getExecutionGraphAccessor().deregisterExecution(this);
+ this.attemptId = newId;
+ getVertex().getExecutionGraphAccessor().registerExecution(this);
+ }
+ }
+
+ /** Recover the execution vertex status after JM failover. */
+ public void recoverExecution(ExecutionAttemptID attemptID,
TaskManagerLocation location) {
Review Comment:
execution vertex -> execution attempt
attemptID -> attemptId
##########
flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java:
##########
@@ -229,6 +229,10 @@ void sendEventToSourceOperator(int subtaskId,
OperatorEvent event) {
String.format("Failed to send event %s to subtask %d", event,
subtaskId));
}
+ ScheduledExecutorService getCoordinatorExecutor() {
Review Comment:
better to annotate it with `@VisibleForTesting`.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptivebatch/BatchJobRecoveryHandler.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.flink.runtime.scheduler.adaptivebatch;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.JobVertexInputInfo;
+import org.apache.flink.runtime.executiongraph.ResultPartitionBytes;
+import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition;
+import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.jobmaster.event.JobEventReplayHandler;
+import org.apache.flink.runtime.scheduler.ExecutionVertexVersioner;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.ShuffleMaster;
+import org.apache.flink.runtime.shuffle.ShuffleMasterSnapshot;
+import org.apache.flink.util.function.ConsumerWithException;
+import org.apache.flink.util.function.TriConsumer;
+
+import org.slf4j.Logger;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+
+/** Interface for handling batch job recovery. */
+public interface BatchJobRecoveryHandler {
+
+ /**
+ * Stops the job event manager and optionally clears events.
+ *
+ * @param clearEvents whether to clear the job events that have been
recorded in the store.
+ */
+ void stopJobEventManager(boolean clearEvents);
Review Comment:
stopJobEventManager -> stop
clearEvents -> cleanUp
`stop job event manager` is implementation details.
--
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]