guoweiM commented on a change in pull request #16465:
URL: https://github.com/apache/flink/pull/16465#discussion_r668403128



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleMasterContext.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.shuffle;
+
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * Shuffle context used to create {@link ShuffleMaster}. It can work as a 
proxy to other cluster

Review comment:
       Could you give more explanation about "It can work as a proxy to other 
cluster components and hide these components from users"?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleMasterContext.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.shuffle;
+
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * Shuffle context used to create {@link ShuffleMaster}. It can work as a 
proxy to other cluster
+ * components and hide these components from users.
+ */
+public interface ShuffleMasterContext {
+
+    /** Returns the cluster configuration. */

Review comment:
       Maybe     /** @return the cluster configuration. */
   

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/JobShuffleContext.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.shuffle;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Job level shuffle context which can offer some job information like job ID 
and through it, the
+ * shuffle plugin can stop tracking the lost result partitions.
+ */
+public interface JobShuffleContext {
+
+    /** Returns the corresponding {@link JobID}. */
+    JobID getJobID();
+
+    /**
+     * Stops tracking the target result partitions, which means these 
partitions will be removed and

Review comment:
       See the comments in `ShuffleMasterContext.java` 

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java
##########
@@ -111,7 +105,7 @@ public JobManagerRunner createJobManagerRunner(
                         jobManagerJobMetricGroupFactory,
                         fatalErrorHandler,
                         userCodeClassLoader,
-                        shuffleMaster,
+                        jobManagerServices.getShuffleMaster(),

Review comment:
       Do we still need to pass the `ShuffleMaster` to the 
`DefaultJobMasterServiceFactory`?
   It could get the `ShuffleMaster` from the `JobManagerSharedServices`

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleMaster.java
##########
@@ -30,7 +31,39 @@
  * @param <T> partition shuffle descriptor used for producer/consumer 
deployment and their data
  *     exchange.
  */
-public interface ShuffleMaster<T extends ShuffleDescriptor> {
+public interface ShuffleMaster<T extends ShuffleDescriptor> extends 
AutoCloseable {
+
+    /**
+     * Starts this shuffle master as a service. One can do some initialization 
here, for example
+     * getting access and connecting to the external system.
+     */
+    default void start() throws Exception {}
+
+    /**
+     * Closes this shuffle master service which should release all resources. 
A shuffle master will
+     * only be closed when the cluster is shut down.
+     */
+    @Override
+    default void close() throws Exception {}
+
+    /**
+     * Registers the target job together with the corresponding {@link 
JobShuffleContext} to this
+     * shuffle master. Through the shuffle context, one can obtain some basic 
information like job
+     * ID, job configuration and all result partitions produced. Besides, by 
stopping tracking the

Review comment:
       "all result partitions produces" could be removed.




-- 
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...@flink.apache.org

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


Reply via email to