venkata91 commented on code in PR #2619:
URL: https://github.com/apache/celeborn/pull/2619#discussion_r1676864998


##########
client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGateFactory.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.celeborn.plugin.flink;
+
+import java.io.IOException;
+
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.io.network.buffer.BufferDecompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.apache.celeborn.common.CelebornConf;
+
+/** Factory class to create {@link RemoteShuffleInputGate}. */
+public class RemoteShuffleInputGateFactory extends 
AbstractRemoteShuffleInputGateFactory {
+
+  public RemoteShuffleInputGateFactory(
+      CelebornConf conf, NetworkBufferPool networkBufferPool, int 
networkBufferSize) {
+    super(conf, networkBufferPool, networkBufferSize);
+  }
+
+  // For testing.

Review Comment:
   nit: `@VisibleForTesting`?



##########
client-flink/flink-1.16/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartitionSuiteJ.java:
##########
@@ -0,0 +1,618 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferDecompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.util.function.SupplierWithException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.celeborn.common.CelebornConf;
+import org.apache.celeborn.plugin.flink.buffer.BufferPacker;
+import org.apache.celeborn.plugin.flink.buffer.DataBuffer;
+import org.apache.celeborn.plugin.flink.readclient.FlinkShuffleClientImpl;
+import org.apache.celeborn.plugin.flink.utils.BufferUtils;
+
+public class RemoteShuffleResultPartitionSuiteJ {
+  private final int networkBufferSize = 32 * 1024;
+  private BufferCompressor bufferCompressor = new 
BufferCompressor(networkBufferSize, "lz4");
+  private RemoteShuffleOutputGate remoteShuffleOutputGate = 
mock(RemoteShuffleOutputGate.class);
+  private final String compressCodec = "LZ4";
+  private final CelebornConf conf = new CelebornConf();
+  BufferDecompressor bufferDecompressor = new 
BufferDecompressor(networkBufferSize, "LZ4");
+
+  private static final int totalBuffers = 1000;
+
+  private static final int bufferSize = 1024;
+
+  private NetworkBufferPool globalBufferPool;
+
+  private BufferPool dataBufferPool;
+
+  private BufferPool nettyBufferPool;
+
+  private RemoteShuffleResultPartition partitionWriter;
+
+  private FakedRemoteShuffleOutputGate outputGate;
+
+  @Before
+  public void setup() {
+    globalBufferPool = new NetworkBufferPool(totalBuffers, bufferSize);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (outputGate != null) {
+      outputGate.release();
+    }
+
+    if (dataBufferPool != null) {
+      dataBufferPool.lazyDestroy();
+    }
+    if (nettyBufferPool != null) {
+      nettyBufferPool.lazyDestroy();
+    }
+    assertEquals(totalBuffers, 
globalBufferPool.getNumberOfAvailableMemorySegments());
+    globalBufferPool.destroy();
+  }
+
+  @Test
+  public void tesSimpleFlush() throws IOException, InterruptedException {
+    List<SupplierWithException<BufferPool, IOException>> bufferPool = 
createBufferPoolFactory();
+    RemoteShuffleResultPartition remoteShuffleResultPartition =
+        new RemoteShuffleResultPartition(
+            "test",
+            0,
+            new ResultPartitionID(),
+            ResultPartitionType.BLOCKING,
+            2,
+            2,
+            32 * 1024,
+            new ResultPartitionManager(),
+            bufferCompressor,
+            bufferPool.get(0),
+            remoteShuffleOutputGate);
+    remoteShuffleResultPartition.setup();
+    doNothing().when(remoteShuffleOutputGate).regionStart(anyBoolean());
+    doNothing().when(remoteShuffleOutputGate).regionFinish();
+    
when(remoteShuffleOutputGate.getBufferPool()).thenReturn(bufferPool.get(1).get());
+    DataBuffer dataBuffer = 
remoteShuffleResultPartition.getDelegation().getUnicastDataBuffer();
+    ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[] {1, 2, 3});
+    dataBuffer.append(byteBuffer, 0, Buffer.DataType.DATA_BUFFER);
+    remoteShuffleResultPartition.getDelegation().flushDataBuffer(dataBuffer, 
true);
+  }
+
+  private List<SupplierWithException<BufferPool, IOException>> 
createBufferPoolFactory() {
+    NetworkBufferPool networkBufferPool =
+        new NetworkBufferPool(256 * 8, 32 * 1024, Duration.ofMillis(1000));

Review Comment:
   `Duration.ofSeconds(1)`?



##########
pom.xml:
##########
@@ -1592,6 +1592,26 @@
       </properties>
     </profile>
 
+    <profile>
+      <id>flink-1.16</id>
+      <modules>
+        <module>client-flink/common</module>
+        <module>client-flink/flink-1.16</module>
+        <module>client-flink/flink-1.16-shaded</module>
+        <module>tests/flink-it</module>
+      </modules>
+      <properties>
+        <flink.version>1.16.3</flink.version>
+        <flink.binary.version>1.16</flink.binary.version>
+        <scala.binary.version>2.12</scala.binary.version>
+        
<celeborn.flink.plugin.artifact>celeborn-client-flink-1.16_${scala.binary.version}</celeborn.flink.plugin.artifact>
+        <flink.streamig.artifact>flink-streaming-java</flink.streamig.artifact>

Review Comment:
   orthogonal to this change - typo in `flink.streamig.artifact` -> 
`flink.streaming.artifact`?



##########
client-flink/flink-1.16/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.flink.api.common.BatchShuffleMode;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraphID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+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.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.JobShuffleContext;
+import org.apache.flink.runtime.shuffle.PartitionDescriptor;
+import org.apache.flink.runtime.shuffle.ProducerDescriptor;
+import org.apache.flink.runtime.shuffle.ShuffleMasterContext;
+import org.apache.flink.runtime.shuffle.TaskInputsOutputsDescriptor;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.CelebornConf;
+import org.apache.celeborn.plugin.flink.utils.FlinkUtils;
+
+public class RemoteShuffleMasterTest {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RemoteShuffleMasterTest.class);
+  private RemoteShuffleMaster remoteShuffleMaster;
+  private Configuration configuration;
+
+  @Before
+  public void setUp() {
+    configuration = new Configuration();
+    remoteShuffleMaster = createShuffleMaster(configuration);
+  }
+
+  @Test
+  public void testRegisterJob() {
+    JobShuffleContext jobShuffleContext = 
createJobShuffleContext(JobID.generate());
+    remoteShuffleMaster.registerJob(jobShuffleContext);
+
+    // reRunRegister job
+    try {
+      remoteShuffleMaster.registerJob(jobShuffleContext);
+    } catch (Exception e) {
+      Assert.assertTrue(true);
+    }
+
+    // unRegister job
+    remoteShuffleMaster.unregisterJob(jobShuffleContext.getJobId());
+    remoteShuffleMaster.registerJob(jobShuffleContext);
+  }
+
+  @Test
+  public void testRegisterPartitionWithProducer()
+      throws UnknownHostException, ExecutionException, InterruptedException {
+    JobID jobID = JobID.generate();
+    JobShuffleContext jobShuffleContext = createJobShuffleContext(jobID);
+    remoteShuffleMaster.registerJob(jobShuffleContext);
+
+    IntermediateDataSetID intermediateDataSetID = new IntermediateDataSetID();
+    PartitionDescriptor partitionDescriptor = 
createPartitionDescriptor(intermediateDataSetID, 0);
+    ProducerDescriptor producerDescriptor = createProducerDescriptor();
+    RemoteShuffleDescriptor remoteShuffleDescriptor =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID, partitionDescriptor, 
producerDescriptor)
+            .get();
+    ShuffleResource shuffleResource = 
remoteShuffleDescriptor.getShuffleResource();
+    ShuffleResourceDescriptor mapPartitionShuffleDescriptor =
+        shuffleResource.getMapPartitionShuffleDescriptor();
+
+    LOG.info("remoteShuffleDescriptor:{}", remoteShuffleDescriptor);
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getShuffleId());
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getPartitionId());
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getAttemptId());
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getMapId());
+
+    // use same dataset id
+    partitionDescriptor = createPartitionDescriptor(intermediateDataSetID, 1);
+    remoteShuffleDescriptor =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID, partitionDescriptor, 
producerDescriptor)
+            .get();
+    mapPartitionShuffleDescriptor =
+        
remoteShuffleDescriptor.getShuffleResource().getMapPartitionShuffleDescriptor();
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getShuffleId());
+    Assert.assertEquals(1, mapPartitionShuffleDescriptor.getMapId());
+
+    // use another attemptId
+    producerDescriptor = createProducerDescriptor();
+    remoteShuffleDescriptor =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID, partitionDescriptor, 
producerDescriptor)
+            .get();
+    mapPartitionShuffleDescriptor =
+        
remoteShuffleDescriptor.getShuffleResource().getMapPartitionShuffleDescriptor();
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getShuffleId());
+    Assert.assertEquals(1, mapPartitionShuffleDescriptor.getAttemptId());
+    Assert.assertEquals(1, mapPartitionShuffleDescriptor.getMapId());
+  }
+
+  @Test
+  public void testRegisterMultipleJobs()
+      throws UnknownHostException, ExecutionException, InterruptedException {
+    JobID jobID1 = JobID.generate();
+    JobShuffleContext jobShuffleContext1 = createJobShuffleContext(jobID1);
+    remoteShuffleMaster.registerJob(jobShuffleContext1);
+
+    JobID jobID2 = JobID.generate();
+    JobShuffleContext jobShuffleContext2 = createJobShuffleContext(jobID2);
+    remoteShuffleMaster.registerJob(jobShuffleContext2);
+
+    IntermediateDataSetID intermediateDataSetID = new IntermediateDataSetID();
+    PartitionDescriptor partitionDescriptor = 
createPartitionDescriptor(intermediateDataSetID, 0);
+    ProducerDescriptor producerDescriptor = createProducerDescriptor();
+    RemoteShuffleDescriptor remoteShuffleDescriptor1 =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID1, partitionDescriptor, 
producerDescriptor)
+            .get();
+
+    // use same datasetId but different jobId
+    RemoteShuffleDescriptor remoteShuffleDescriptor2 =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID2, partitionDescriptor, 
producerDescriptor)
+            .get();
+
+    Assert.assertEquals(
+        remoteShuffleDescriptor1
+            .getShuffleResource()
+            .getMapPartitionShuffleDescriptor()
+            .getShuffleId(),
+        0);
+    Assert.assertEquals(
+        remoteShuffleDescriptor2
+            .getShuffleResource()
+            .getMapPartitionShuffleDescriptor()
+            .getShuffleId(),
+        1);
+  }
+
+  @Test
+  public void testShuffleMemoryAnnouncing() {
+    Map<IntermediateDataSetID, Integer> numberOfInputGateChannels = new 
HashMap<>();
+    Map<IntermediateDataSetID, Integer> numbersOfResultSubpartitions = new 
HashMap<>();
+    Map<IntermediateDataSetID, Integer> subPartitionNums = new HashMap<>();
+    Map<IntermediateDataSetID, ResultPartitionType> inputPartitionTypes = new 
HashMap<>();

Review Comment:
   Again, not used at all, remove?



##########
client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentProvider;
+import org.apache.flink.metrics.SimpleCounter;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter;
+import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo;
+import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.SubpartitionIndexRange;
+import org.apache.flink.runtime.event.TaskEvent;
+import org.apache.flink.runtime.io.network.ConnectionID;
+import org.apache.flink.runtime.io.network.LocalConnectionManager;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferDecompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate;
+import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
+import 
org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.throughput.ThroughputCalculator;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.clock.SystemClock;
+import org.apache.flink.util.function.SupplierWithException;
+
+import org.apache.celeborn.common.CelebornConf;
+
+/** A {@link IndexedInputGate} which ingest data from remote shuffle workers. 
*/
+public class RemoteShuffleInputGate extends IndexedInputGate {
+
+  private final RemoteShuffleInputGateDelegation inputGateDelegation;
+
+  public RemoteShuffleInputGate(
+      CelebornConf celebornConf,
+      String taskName,
+      int gateIndex,
+      InputGateDeploymentDescriptor gateDescriptor,
+      SupplierWithException<BufferPool, IOException> bufferPoolFactory,
+      BufferDecompressor bufferDecompressor,
+      int numConcurrentReading) {
+
+    inputGateDelegation =
+        new RemoteShuffleInputGateDelegation(
+            celebornConf,
+            taskName,
+            gateIndex,
+            gateDescriptor,
+            bufferPoolFactory,
+            bufferDecompressor,
+            numConcurrentReading,
+            availabilityHelper,
+            gateDescriptor.getConsumedSubpartitionIndexRange().getStartIndex(),
+            gateDescriptor.getConsumedSubpartitionIndexRange().getEndIndex());
+  }
+
+  /** Setup gate and build network connections. */
+  @Override
+  public void setup() throws IOException {
+    inputGateDelegation.setup();
+  }
+
+  /** Index of the gate of the corresponding computing task. */
+  @Override
+  public int getGateIndex() {
+    return inputGateDelegation.getGateIndex();
+  }
+
+  /** Get number of input channels. A channel is a data flow from one shuffle 
worker. */
+  @Override
+  public int getNumberOfInputChannels() {
+    return inputGateDelegation.getBufferReaders().size();
+  }
+
+  /** Whether reading is finished -- all channels are finished and cached 
buffers are drained. */
+  @Override
+  public boolean isFinished() {
+    return inputGateDelegation.isFinished();
+  }
+
+  @Override
+  public Optional<BufferOrEvent> getNext() {
+    throw new UnsupportedOperationException("Not implemented (DataSet API is 
not supported).");
+  }
+
+  /** Poll a received {@link BufferOrEvent}. */
+  @Override
+  public Optional<BufferOrEvent> pollNext() throws IOException {
+    return inputGateDelegation.pollNext();
+  }
+
+  /** Close all reading channels inside this {@link RemoteShuffleInputGate}. */
+  @Override
+  public void close() throws Exception {
+    inputGateDelegation.close();
+  }
+
+  /** Get {@link InputChannelInfo}s of this {@link RemoteShuffleInputGate}. */
+  @Override
+  public List<InputChannelInfo> getChannelInfos() {
+    return inputGateDelegation.getChannelsInfo();
+  }
+
+  @Override
+  public void requestPartitions() {
+    // do-nothing
+  }
+
+  @Override
+  public void checkpointStarted(CheckpointBarrier barrier) {
+    // do-nothing.
+  }
+
+  @Override
+  public void checkpointStopped(long cancelledCheckpointId) {
+    // do-nothing.
+  }
+
+  @Override
+  public void triggerDebloating() {
+    // do-nothing.
+  }
+
+  @Override
+  public List<InputChannelInfo> getUnfinishedChannels() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public EndOfDataStatus hasReceivedEndOfData() {
+    if (inputGateDelegation.getPendingEndOfDataEvents() > 0) {
+      return EndOfDataStatus.NOT_END_OF_DATA;
+    } else {
+      // Keep compatibility with streaming mode.
+      return EndOfDataStatus.DRAINED;
+    }
+  }
+
+  @Override
+  public void finishReadRecoveredState() {
+    // do-nothing.
+  }
+
+  @Override
+  public InputChannel getChannel(int channelIndex) {
+    return new FakedRemoteInputChannel(channelIndex);
+  }
+
+  @Override
+  public void sendTaskEvent(TaskEvent event) {
+    throw new FlinkRuntimeException("Method should not be called.");
+  }
+
+  @Override
+  public void resumeConsumption(InputChannelInfo channelInfo) {
+    throw new FlinkRuntimeException("Method should not be called.");
+  }
+
+  @Override
+  public void acknowledgeAllRecordsProcessed(InputChannelInfo 
inputChannelInfo) {}
+
+  @Override
+  public CompletableFuture<Void> getStateConsumedFuture() {
+    return CompletableFuture.completedFuture(null);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "ReadGate [owning task: %s, gate index: %d, descriptor: %s]",
+        inputGateDelegation.getTaskName(),
+        inputGateDelegation.getGateIndex(),
+        inputGateDelegation.getGateDescriptor().toString());
+  }
+
+  /** Accommodation for the incompleteness of Flink pluggable shuffle service. 
*/
+  private class FakedRemoteInputChannel extends RemoteInputChannel {
+    FakedRemoteInputChannel(int channelIndex) {
+      super(
+          new SingleInputGate(
+              inputGateDelegation.getTaskName(),
+              inputGateDelegation.getGateIndex(),
+              new IntermediateDataSetID(),
+              ResultPartitionType.BLOCKING,
+              new SubpartitionIndexRange(0, 0),
+              1,
+              (a, b, c) -> {},
+              () -> null,
+              null,
+              new FakedMemorySegmentProvider(),
+              0,
+              new ThroughputCalculator(SystemClock.getInstance()),
+              null),
+          channelIndex,
+          new ResultPartitionID(),
+          0,
+          new ConnectionID(
+              new TaskManagerLocation(ResourceID.generate(), 
InetAddress.getLoopbackAddress(), 1),
+              0),
+          new LocalConnectionManager(),
+          0,
+          0,
+          0,
+          new SimpleCounter(),
+          new SimpleCounter(),
+          new FakedChannelStateWriter());
+    }
+  }
+
+  /** Accommodation for the incompleteness of Flink pluggable shuffle service. 
*/
+  private static class FakedMemorySegmentProvider implements 
MemorySegmentProvider {
+
+    @Override
+    public Collection<MemorySegment> requestUnpooledMemorySegments(int i) 
throws IOException {
+      return null;
+    }
+
+    @Override
+    public void recycleUnpooledMemorySegments(Collection<MemorySegment> 
collection)
+        throws IOException {}
+  }
+
+  /** Accommodation for the incompleteness of Flink pluggable shuffle service. 
*/
+  private static class FakedChannelStateWriter implements ChannelStateWriter {
+
+    @Override
+    public void start(long cpId, CheckpointOptions checkpointOptions) {}

Review Comment:
   super nit: comment `// do-nothing.` similar to above overridden methods? Can 
be added to other overridden methods below as well.



##########
client-flink/flink-1.16/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.flink.api.common.BatchShuffleMode;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraphID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+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.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.JobShuffleContext;
+import org.apache.flink.runtime.shuffle.PartitionDescriptor;
+import org.apache.flink.runtime.shuffle.ProducerDescriptor;
+import org.apache.flink.runtime.shuffle.ShuffleMasterContext;
+import org.apache.flink.runtime.shuffle.TaskInputsOutputsDescriptor;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.CelebornConf;
+import org.apache.celeborn.plugin.flink.utils.FlinkUtils;
+
+public class RemoteShuffleMasterTest {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RemoteShuffleMasterTest.class);
+  private RemoteShuffleMaster remoteShuffleMaster;
+  private Configuration configuration;
+
+  @Before
+  public void setUp() {
+    configuration = new Configuration();
+    remoteShuffleMaster = createShuffleMaster(configuration);
+  }
+
+  @Test
+  public void testRegisterJob() {
+    JobShuffleContext jobShuffleContext = 
createJobShuffleContext(JobID.generate());
+    remoteShuffleMaster.registerJob(jobShuffleContext);
+
+    // reRunRegister job

Review Comment:
   super nit: `rerun registerJob`?



##########
client-flink/flink-1.16/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterTest.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.celeborn.plugin.flink;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.flink.api.common.BatchShuffleMode;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraphID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+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.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.JobShuffleContext;
+import org.apache.flink.runtime.shuffle.PartitionDescriptor;
+import org.apache.flink.runtime.shuffle.ProducerDescriptor;
+import org.apache.flink.runtime.shuffle.ShuffleMasterContext;
+import org.apache.flink.runtime.shuffle.TaskInputsOutputsDescriptor;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.CelebornConf;
+import org.apache.celeborn.plugin.flink.utils.FlinkUtils;
+
+public class RemoteShuffleMasterTest {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RemoteShuffleMasterTest.class);
+  private RemoteShuffleMaster remoteShuffleMaster;
+  private Configuration configuration;
+
+  @Before
+  public void setUp() {
+    configuration = new Configuration();
+    remoteShuffleMaster = createShuffleMaster(configuration);
+  }
+
+  @Test
+  public void testRegisterJob() {
+    JobShuffleContext jobShuffleContext = 
createJobShuffleContext(JobID.generate());
+    remoteShuffleMaster.registerJob(jobShuffleContext);
+
+    // reRunRegister job
+    try {
+      remoteShuffleMaster.registerJob(jobShuffleContext);
+    } catch (Exception e) {
+      Assert.assertTrue(true);
+    }
+
+    // unRegister job
+    remoteShuffleMaster.unregisterJob(jobShuffleContext.getJobId());
+    remoteShuffleMaster.registerJob(jobShuffleContext);
+  }
+
+  @Test
+  public void testRegisterPartitionWithProducer()
+      throws UnknownHostException, ExecutionException, InterruptedException {
+    JobID jobID = JobID.generate();
+    JobShuffleContext jobShuffleContext = createJobShuffleContext(jobID);
+    remoteShuffleMaster.registerJob(jobShuffleContext);
+
+    IntermediateDataSetID intermediateDataSetID = new IntermediateDataSetID();
+    PartitionDescriptor partitionDescriptor = 
createPartitionDescriptor(intermediateDataSetID, 0);
+    ProducerDescriptor producerDescriptor = createProducerDescriptor();
+    RemoteShuffleDescriptor remoteShuffleDescriptor =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID, partitionDescriptor, 
producerDescriptor)
+            .get();
+    ShuffleResource shuffleResource = 
remoteShuffleDescriptor.getShuffleResource();
+    ShuffleResourceDescriptor mapPartitionShuffleDescriptor =
+        shuffleResource.getMapPartitionShuffleDescriptor();
+
+    LOG.info("remoteShuffleDescriptor:{}", remoteShuffleDescriptor);
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getShuffleId());
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getPartitionId());
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getAttemptId());
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getMapId());
+
+    // use same dataset id
+    partitionDescriptor = createPartitionDescriptor(intermediateDataSetID, 1);
+    remoteShuffleDescriptor =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID, partitionDescriptor, 
producerDescriptor)
+            .get();
+    mapPartitionShuffleDescriptor =
+        
remoteShuffleDescriptor.getShuffleResource().getMapPartitionShuffleDescriptor();
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getShuffleId());
+    Assert.assertEquals(1, mapPartitionShuffleDescriptor.getMapId());
+
+    // use another attemptId
+    producerDescriptor = createProducerDescriptor();
+    remoteShuffleDescriptor =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID, partitionDescriptor, 
producerDescriptor)
+            .get();
+    mapPartitionShuffleDescriptor =
+        
remoteShuffleDescriptor.getShuffleResource().getMapPartitionShuffleDescriptor();
+    Assert.assertEquals(0, mapPartitionShuffleDescriptor.getShuffleId());
+    Assert.assertEquals(1, mapPartitionShuffleDescriptor.getAttemptId());
+    Assert.assertEquals(1, mapPartitionShuffleDescriptor.getMapId());
+  }
+
+  @Test
+  public void testRegisterMultipleJobs()
+      throws UnknownHostException, ExecutionException, InterruptedException {
+    JobID jobID1 = JobID.generate();
+    JobShuffleContext jobShuffleContext1 = createJobShuffleContext(jobID1);
+    remoteShuffleMaster.registerJob(jobShuffleContext1);
+
+    JobID jobID2 = JobID.generate();
+    JobShuffleContext jobShuffleContext2 = createJobShuffleContext(jobID2);
+    remoteShuffleMaster.registerJob(jobShuffleContext2);
+
+    IntermediateDataSetID intermediateDataSetID = new IntermediateDataSetID();
+    PartitionDescriptor partitionDescriptor = 
createPartitionDescriptor(intermediateDataSetID, 0);
+    ProducerDescriptor producerDescriptor = createProducerDescriptor();
+    RemoteShuffleDescriptor remoteShuffleDescriptor1 =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID1, partitionDescriptor, 
producerDescriptor)
+            .get();
+
+    // use same datasetId but different jobId
+    RemoteShuffleDescriptor remoteShuffleDescriptor2 =
+        remoteShuffleMaster
+            .registerPartitionWithProducer(jobID2, partitionDescriptor, 
producerDescriptor)
+            .get();
+
+    Assert.assertEquals(
+        remoteShuffleDescriptor1
+            .getShuffleResource()
+            .getMapPartitionShuffleDescriptor()
+            .getShuffleId(),
+        0);
+    Assert.assertEquals(
+        remoteShuffleDescriptor2
+            .getShuffleResource()
+            .getMapPartitionShuffleDescriptor()
+            .getShuffleId(),
+        1);
+  }
+
+  @Test
+  public void testShuffleMemoryAnnouncing() {
+    Map<IntermediateDataSetID, Integer> numberOfInputGateChannels = new 
HashMap<>();
+    Map<IntermediateDataSetID, Integer> numbersOfResultSubpartitions = new 
HashMap<>();
+    Map<IntermediateDataSetID, Integer> subPartitionNums = new HashMap<>();

Review Comment:
   Looks like this is not used at all, remove?



-- 
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]


Reply via email to