zhijiangW commented on a change in pull request #11155: 
[FLINK-14818][benchmark] Fix receiving InputGate setup of 
StreamNetworkBenchmarkEnvironment.
URL: https://github.com/apache/flink/pull/11155#discussion_r382390260
 
 

 ##########
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java
 ##########
 @@ -272,11 +287,161 @@ private static ShuffleDescriptor 
createShuffleDescriptor(
                        ResultPartitionID resultPartitionID,
                        ResourceID location,
                        TaskManagerLocation senderLocation,
-                       int channel) {
+                       int connectionIndex) {
                final NettyShuffleDescriptorBuilder builder = 
NettyShuffleDescriptorBuilder.newBuilder()
                        .setId(resultPartitionID)
                        .setProducerInfoFromTaskManagerLocation(senderLocation)
-                       .setConnectionIndex(channel);
+                       .setConnectionIndex(connectionIndex);
                return localMode ? 
builder.setProducerLocation(location).buildLocal() : builder.buildRemote();
        }
+
+       /**
+        * A {@link SingleInputGateFactory} which replaces the default {@link 
RemoteInputChannel} and
+        * {@link LocalInputChannel} implementation with costume ones.
+        */
+       private static class TestSingleInputGateFactory extends 
SingleInputGateFactory {
+
+               private final ResourceID taskExecutorResourceId;
+               private final int partitionRequestInitialBackoff;
+               private final int partitionRequestMaxBackoff;
+               private final ConnectionManager connectionManager;
+               private final ResultPartitionManager partitionManager;
+               private final TaskEventPublisher taskEventPublisher;
+               private final NetworkBufferPool networkBufferPool;
+
+               public TestSingleInputGateFactory(
+                               @Nonnull ResourceID taskExecutorResourceId,
+                               @Nonnull NettyShuffleEnvironmentConfiguration 
networkConfig,
+                               @Nonnull ConnectionManager connectionManager,
+                               @Nonnull ResultPartitionManager 
partitionManager,
+                               @Nonnull TaskEventPublisher taskEventPublisher,
+                               @Nonnull NetworkBufferPool networkBufferPool) {
+                       super(
+                               taskExecutorResourceId,
+                               networkConfig,
+                               connectionManager,
+                               partitionManager,
+                               taskEventPublisher,
+                               networkBufferPool);
+                       this.networkBufferPool = networkBufferPool;
+                       this.taskEventPublisher = taskEventPublisher;
+                       this.partitionManager = partitionManager;
+                       this.connectionManager = connectionManager;
+                       this.partitionRequestMaxBackoff = 
networkConfig.partitionRequestMaxBackoff();
+                       this.partitionRequestInitialBackoff = 
networkConfig.partitionRequestInitialBackoff();
+                       this.taskExecutorResourceId = taskExecutorResourceId;
+               }
+
+               @Override
+               protected InputChannel createKnownInputChannel(
+                       SingleInputGate inputGate,
+                       int index,
+                       NettyShuffleDescriptor inputChannelDescriptor,
+                       ChannelStatistics channelStatistics,
+                       InputChannelMetrics metrics) {
+                       ResultPartitionID partitionId = 
inputChannelDescriptor.getResultPartitionID();
+                       if 
(inputChannelDescriptor.isLocalTo(taskExecutorResourceId)) {
+                               return new TestLocalInputChannel(
+                                       inputGate,
+                                       index,
+                                       partitionId,
+                                       partitionManager,
+                                       taskEventPublisher,
+                                       partitionRequestInitialBackoff,
+                                       partitionRequestMaxBackoff,
+                                       metrics);
+                       } else {
+                               return new TestRemoteInputChannel(
+                                       inputGate,
+                                       index,
+                                       partitionId,
+                                       
inputChannelDescriptor.getConnectionId(),
+                                       connectionManager,
+                                       partitionRequestInitialBackoff,
+                                       partitionRequestMaxBackoff,
+                                       metrics,
+                                       networkBufferPool);
+                       }
+               }
+       }
+
+       /**
+        * A {@link LocalInputChannel} which ignores the given subpartition 
index and uses channel index
+        * instead when requesting subpartition.
+        */
+       private static class TestLocalInputChannel extends LocalInputChannel {
+
+               private ResultPartitionID newPartitionID = new 
ResultPartitionID();
+
+               public TestLocalInputChannel(
+                               SingleInputGate inputGate,
+                               int channelIndex,
+                               ResultPartitionID partitionId,
+                               ResultPartitionManager partitionManager,
+                               TaskEventPublisher taskEventPublisher,
+                               int initialBackoff,
+                               int maxBackoff,
+                               InputChannelMetrics metrics) {
+                       super(
+                               inputGate,
+                               channelIndex,
+                               partitionId,
+                               partitionManager,
+                               taskEventPublisher,
+                               initialBackoff,
+                               maxBackoff,
+                               metrics);
+               }
+
+               @Override
+               public void requestSubpartition(int subpartitionIndex) throws 
IOException, InterruptedException {
+                       super.requestSubpartition(channelIndex);
+               }
+
+               @Override
+               public ResultPartitionID getPartitionId() {
 
 Review comment:
   Why we need to override this method? I guess that the original `partitionId` 
while constructing the `SingleInputGate` is valid.

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to