This is an automated email from the ASF dual-hosted git repository. sewen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit 8a76952dab6e9533c2815878ffde96806658161e Author: Stephan Ewen <[email protected]> AuthorDate: Wed Apr 17 17:31:42 2019 +0200 [hotfix] [tests] Consolidate mocking of ResultPartition in one utility class This also removes the use of Mockito from these classes. --- .../runtime/io/network/NetworkEnvironmentTest.java | 20 +----- .../io/network/partition/PartitionTestUtils.java | 74 ++++++++++++++++++++++ .../partition/PipelinedSubpartitionTest.java | 2 +- .../io/network/partition/ResultPartitionTest.java | 23 +------ 4 files changed, 78 insertions(+), 41 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java index 3a2014a..f20feb7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java @@ -18,18 +18,13 @@ package org.apache.flink.runtime.io.network; -import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.io.disk.iomanager.NoOpIOManager; import org.apache.flink.runtime.io.network.partition.InputChannelTestUtils; -import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; +import org.apache.flink.runtime.io.network.partition.PartitionTestUtils; import org.apache.flink.runtime.io.network.partition.ResultPartition; -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.runtime.io.network.partition.consumer.RemoteInputChannel; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; -import org.apache.flink.runtime.taskmanager.NoOpTaskActions; import org.apache.flink.runtime.taskmanager.Task; import org.junit.Rule; @@ -277,18 +272,7 @@ public class NetworkEnvironmentTest { private static ResultPartition createResultPartition( final ResultPartitionType partitionType, final int channels) { - return new ResultPartition( - "TestTask-" + partitionType + ":" + channels, - new NoOpTaskActions(), - new JobID(), - new ResultPartitionID(), - partitionType, - channels, - channels, - mock(ResultPartitionManager.class), - new NoOpResultPartitionConsumableNotifier(), - new NoOpIOManager(), - false); + return PartitionTestUtils.createPartition(partitionType, channels); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java new file mode 100644 index 0000000..3391030 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java @@ -0,0 +1,74 @@ +/* + * 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.io.network.partition; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.io.disk.iomanager.NoOpIOManager; +import org.apache.flink.runtime.taskmanager.NoOpTaskActions; + +/** + * This class should consolidate all mocking logic for ResultPartitions. + * While using Mockito internally (for now), the use of Mockito should not + * leak out of this class. + */ +public class PartitionTestUtils { + + public static ResultPartition createPartition() { + return createPartition(ResultPartitionType.PIPELINED_BOUNDED); + } + + public static ResultPartition createPartition(ResultPartitionType type) { + return createPartition( + new NoOpResultPartitionConsumableNotifier(), + type, + false); + } + + public static ResultPartition createPartition(ResultPartitionType type, int numChannels) { + return createPartition(new NoOpResultPartitionConsumableNotifier(), type, numChannels, false); + } + + public static ResultPartition createPartition( + ResultPartitionConsumableNotifier notifier, + ResultPartitionType type, + boolean sendScheduleOrUpdateConsumersMessage) { + + return createPartition(notifier, type, 1, sendScheduleOrUpdateConsumersMessage); + } + + public static ResultPartition createPartition( + ResultPartitionConsumableNotifier notifier, + ResultPartitionType type, + int numChannels, + boolean sendScheduleOrUpdateConsumersMessage) { + + return new ResultPartition( + "TestTask", + new NoOpTaskActions(), + new JobID(), + new ResultPartitionID(), + type, + numChannels, + numChannels, + new ResultPartitionManager(), + notifier, + new NoOpIOManager(), + sendScheduleOrUpdateConsumersMessage); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index 8535973..36cd353 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -73,7 +73,7 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase { @Override PipelinedSubpartition createSubpartition() { - final ResultPartition parent = mock(ResultPartition.class); + final ResultPartition parent = PartitionTestUtils.createPartition(); return new PipelinedSubpartition(0, parent); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index 9e3c117..c911df7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -19,19 +19,18 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.io.disk.iomanager.NoOpIOManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; -import org.apache.flink.runtime.taskmanager.NoOpTaskActions; import org.apache.flink.runtime.taskmanager.TaskActions; import org.junit.Assert; import org.junit.Test; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; +import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -250,24 +249,4 @@ public class ResultPartitionTest { network.shutdown(); } } - - // ------------------------------------------------------------------------ - - private static ResultPartition createPartition( - ResultPartitionConsumableNotifier notifier, - ResultPartitionType type, - boolean sendScheduleOrUpdateConsumersMessage) { - return new ResultPartition( - "TestTask", - new NoOpTaskActions(), - new JobID(), - new ResultPartitionID(), - type, - 1, - 1, - mock(ResultPartitionManager.class), - notifier, - new NoOpIOManager(), - sendScheduleOrUpdateConsumersMessage); - } }
