zhijiangW commented on a change in pull request #8362: [FLINK-11391] Introduce 
shuffle master interface
URL: https://github.com/apache/flink/pull/8362#discussion_r289269558
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleMaster.java
 ##########
 @@ -0,0 +1,52 @@
+/*
+ * 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.runtime.io.network.partition.ResultPartitionID;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Default {@link ShuffleMaster} for netty and local file based shuffle 
implementation.
+ */
+public enum NettyShuffleMaster implements 
ShuffleMaster<NettyShuffleDescriptor> {
+       INSTANCE;
+
+       @Override
+       public CompletableFuture<NettyShuffleDescriptor> 
registerPartitionWithProducer(
+                       PartitionDescriptor partitionDescriptor,
+                       ProducerDescriptor producerDescriptor) {
+               ResultPartitionID resultPartitionID = new ResultPartitionID(
+                       partitionDescriptor.getPartitionId(),
+                       producerDescriptor.getProducerExecutionId());
+               NettyShuffleDescriptor shuffleDeploymentDescriptor = new 
NettyShuffleDescriptor(
+                       producerDescriptor.getProducerResourceId(),
+                       createProducerLocation(producerDescriptor, 
partitionDescriptor.getConnectionIndex()),
+                       resultPartitionID);
+               return 
CompletableFuture.completedFuture(shuffleDeploymentDescriptor);
+       }
+
+       private static NettyShuffleDescriptor.PartitionLocation 
createProducerLocation(
+                       ProducerDescriptor producerDescriptor,
+                       int connectionIndex) {
+               return producerDescriptor.getDataPort() >= 0 ?
 
 Review comment:
   It seems a bit confusing here to distinguish local or remove based on data 
port. 
   
   Whether local or remote should be from the view of consumer side which has 
corresponding local/remote channel. From producer side it is not caring about 
the consumer location, always has the same partition. 
   
   Also the data port here should be always more than 0 after TM starts to 
register to JM. It does not indicate the producer and consumer are in the same 
TM or not. 
   
   And in the end the consumer still checks the resource id to generate 
local/remote channel, not based on `LocalPartition/RemotetPartition`.  I 
suggest it is no need to define a `PartitionLocation` interface, we could 
always have a general `PartitionLocation` here to cover the relevant info of 
connection. And whether this connection info is actually used based on consumer 
side to check the resource id.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to