rkhachatryan commented on a change in pull request #13351:
URL: https://github.com/apache/flink/pull/13351#discussion_r495112681



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/SequentialChannelStateReaderImplTest.java
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.checkpoint.channel;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.StateObjectCollection;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+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.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionBuilder;
+import 
org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
+import 
org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateBuilder;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.InputChannelStateHandle;
+import org.apache.flink.runtime.state.ResultSubpartitionStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * {@link SequentialChannelStateReaderImpl} Test.
+ */
+@RunWith(Parameterized.class)
+public class SequentialChannelStateReaderImplTest {
+
+       @Parameterized.Parameters(name = "{0}: stateParLevel={1}, 
statePartsPerChannel={1}, stateBytesPerPart={2},  parLevel={4}, bufferSize={5}")
+       public static Object[][] parameters() {
+               return new Object[][]{
+                       {"NoStateAndNoChannels", 0, 0, 0, 0, 0},
+                       {"NoState", 0, 10, 10, 10, 10},
+                       {"ReadPermutedStateWithEqualBuffer", 10, 10, 10, 10, 
10},
+                       {"ReadPermutedStateWithReducedBuffer", 10, 10, 10, 20, 
10},
+                       {"ReadPermutedStateWithIncreasedBuffer", 10, 10, 10, 
10, 20},
+               };
+       }
+
+       private final ChannelStateSerializer serializer;
+       private final Random random;
+       private final int parLevel;
+       private final int statePartsPerChannel;
+       private final int stateBytesPerPart;
+       private final int bufferSize;
+       private final int stateParLevel;
+       private final boolean expectToHaveState;
+       private final int buffersPerChannel;
+
+       public SequentialChannelStateReaderImplTest(String desc, int 
stateParLevel, int statePartsPerChannel, int stateBytesPerPart, int parLevel, 
int bufferSize) {
+               this.serializer = new ChannelStateSerializerImpl();
+               this.random = new Random();
+               this.parLevel = parLevel;
+               this.statePartsPerChannel = statePartsPerChannel;
+               this.stateBytesPerPart = stateBytesPerPart;
+               this.bufferSize = bufferSize;
+               this.stateParLevel = stateParLevel;
+               this.expectToHaveState = stateParLevel * statePartsPerChannel * 
stateBytesPerPart > 0;
+               // will read without waiting for consumption
+               this.buffersPerChannel = Math.max(1, statePartsPerChannel * 
(bufferSize >= stateBytesPerPart ? 1 : stateBytesPerPart / bufferSize));
+       }
+
+       @Test
+       public void testReadPermutedState() throws Exception {
+               Map<InputChannelInfo, List<byte[]>> inputChannelsData = 
generateState(InputChannelInfo::new);
+               Map<ResultSubpartitionInfo, List<byte[]>> resultPartitionsData 
= generateState(ResultSubpartitionInfo::new);
+
+               SequentialChannelStateReader reader = new 
SequentialChannelStateReaderImpl(buildSnapshot(writePermuted(inputChannelsData, 
resultPartitionsData)));
+               assertEquals(expectToHaveState, reader.hasChannelStates());
+
+               withResultPartitions(resultPartitions -> {
+                       reader.readOutputData(resultPartitions);
+                       assertBuffersEquals(resultPartitionsData, 
collectBuffers(resultPartitions));
+               });
+
+               withInputGates(gates -> {
+                       reader.readInputData(gates);
+                       assertBuffersEquals(inputChannelsData, 
collectBuffers(gates));
+                       assertConsumed(gates);
+               });
+       }
+
+       private Map<ResultSubpartitionInfo, List<Buffer>> 
collectBuffers(ResultPartition[] resultPartitions) throws IOException {
+               Map<ResultSubpartitionInfo, List<Buffer>> actual = new 
HashMap<>();
+               for (ResultPartition resultPartition : resultPartitions) {
+                       for (int i = 0; i < 
resultPartition.getNumberOfSubpartitions(); i++) {
+                               ResultSubpartitionInfo info = 
resultPartition.getSubpartitionInfo(i);
+                               ResultSubpartitionView view = 
resultPartition.createSubpartitionView(info.getSubPartitionIdx(), () -> { /**/ 
});
+                               for (BufferAndBacklog buffer = 
view.getNextBuffer(); buffer != null; buffer = view.getNextBuffer()) {
+                                       actual.computeIfAbsent(info, unused -> 
new ArrayList<>()).add(buffer.buffer());
+                               }
+                       }
+               }
+               return actual;
+       }
+
+       private Map<InputChannelInfo, List<Buffer>> collectBuffers(InputGate[] 
gates) throws Exception {
+               Map<InputChannelInfo, List<Buffer>> actual = new HashMap<>();
+               for (InputGate gate : gates) {
+                       for (Optional<BufferOrEvent> next = gate.pollNext(); 
next.isPresent(); next = gate.pollNext()) {
+                               actual.computeIfAbsent(
+                                       next.get().getChannelInfo(),
+                                       unused -> new 
ArrayList<>()).add(next.get().getBuffer());
+                       }
+               }
+               return actual;
+       }
+
+       private void assertConsumed(InputGate[] gates) throws 
InterruptedException, java.util.concurrent.ExecutionException {
+               for (InputGate gate: gates) {
+                       assertTrue(gate.getStateConsumedFuture().isDone());
+                       gate.getStateConsumedFuture().get();
+               }
+       }
+
+       private void withInputGates(ThrowingConsumer<InputGate[], Exception> 
action) throws Exception {
+               SingleInputGate[] gates = new SingleInputGate[parLevel];
+               final int segmentsToAllocate = parLevel * parLevel * 
buffersPerChannel;
+               NetworkBufferPool networkBufferPool = new 
NetworkBufferPool(segmentsToAllocate, bufferSize);
+               try {
+                       for (int i = 0; i < parLevel; i++) {
+                               gates[i] = new SingleInputGateBuilder()
+                                       .setNumberOfChannels(parLevel)
+                                       .setSingleInputGateIndex(i)
+                                       
.setBufferPoolFactory(networkBufferPool.createBufferPool(0, buffersPerChannel))
+                                       .setSegmentProvider(networkBufferPool)
+                                       .setChannelFactory((builder, gate) -> 
builder
+                                       
.setNetworkBuffersPerChannel(buffersPerChannel)
+                                       .buildRemoteRecoveredChannel(gate))
+                                       .build();
+                               gates[i].setup();
+                       }
+                       action.accept(gates);
+               } finally {
+                       for (InputGate inputGate: gates) {
+                               inputGate.close();
+                       }
+                       try {
+                               assertEquals(segmentsToAllocate, 
networkBufferPool.getNumberOfAvailableMemorySegments());
+                       } finally {
+                               networkBufferPool.destroyAllBufferPools();
+                               networkBufferPool.destroy();
+                       }
+               }
+       }
+
+       private void withResultPartitions(ThrowingConsumer<ResultPartition[], 
Exception> action) throws Exception {
+               int segmentsToAllocate = parLevel * parLevel * 
buffersPerChannel;
+               NetworkBufferPool networkBufferPool = new 
NetworkBufferPool(segmentsToAllocate, bufferSize);
+               ResultPartition[] resultPartitions = range(0, parLevel)
+                       .mapToObj(i -> new 
ResultPartitionBuilder().setResultPartitionIndex(i).setNumberOfSubpartitions(parLevel).setNetworkBufferPool(networkBufferPool).build())
+                       .toArray(ResultPartition[]::new);
+               try {
+                       for (ResultPartition resultPartition: resultPartitions) 
{
+                               resultPartition.setup();
+                       }
+                       action.accept(resultPartitions);
+               } finally {
+                       for (ResultPartition resultPartition: resultPartitions) 
{
+                               resultPartition.close();
+                       }
+                       try {
+                               assertEquals(segmentsToAllocate, 
networkBufferPool.getNumberOfAvailableMemorySegments());
+                       } finally {
+                               networkBufferPool.destroyAllBufferPools();
+                               networkBufferPool.destroy();
+                       }
+               }
+       }
+
+       private TaskStateSnapshot 
buildSnapshot(Tuple2<List<InputChannelStateHandle>, 
List<ResultSubpartitionStateHandle>> handles) {
+               return new TaskStateSnapshot(
+                       Collections.singletonMap(new OperatorID(), new 
OperatorSubtaskState(
+                               StateObjectCollection.empty(),
+                               StateObjectCollection.empty(),
+                               StateObjectCollection.empty(),
+                               StateObjectCollection.empty(),
+                               new StateObjectCollection<>(handles.f0),
+                               new StateObjectCollection<>(handles.f1)
+                       ))
+               );
+       }
+
+       private <T> Map<T, List<byte[]>> generateState(BiFunction<Integer, 
Integer, T> descriptorCreator) {
+               return range(0, stateParLevel).boxed().flatMap(
+                       gateId -> range(0, stateParLevel).mapToObj(
+                               channelId ->
+                                       descriptorCreator.apply(gateId, 
channelId))).collect(toMap(

Review comment:
       Good idea.
   I'll extract `generateSingleChannelState` method.




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


Reply via email to