You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by sr...@apache.org on 2018/01/08 13:03:56 UTC
[02/15] flink git commit: [FLINK-8214][streaming-tests] Collect
results into proper mock in StreamMockEnvironment
[FLINK-8214][streaming-tests] Collect results into proper mock in StreamMockEnvironment
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/0888bb62
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0888bb62
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0888bb62
Branch: refs/heads/master
Commit: 0888bb622e275ac6ff2408c2ae5014fd787b5dbd
Parents: af6bdb6
Author: Piotr Nowojski <pi...@gmail.com>
Authored: Wed Dec 6 16:02:26 2017 +0100
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Mon Jan 8 11:46:00 2018 +0100
----------------------------------------------------------------------
...dOrEventCollectingResultPartitionWriter.java | 103 ++++++++++++++++++
.../runtime/tasks/StreamMockEnvironment.java | 108 ++-----------------
2 files changed, 111 insertions(+), 100 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/0888bb62/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordOrEventCollectingResultPartitionWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordOrEventCollectingResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordOrEventCollectingResultPartitionWriter.java
new file mode 100644
index 0000000..c9ec6df
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordOrEventCollectingResultPartitionWriter.java
@@ -0,0 +1,103 @@
+/*
+ * 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.api.writer;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.event.AbstractEvent;
+import org.apache.flink.runtime.io.network.api.serialization.AdaptiveSpanningRecordDeserializer;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.plugable.DeserializationDelegate;
+import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link ResultPartitionWriter} that collects records or events on the List.
+ */
+public class RecordOrEventCollectingResultPartitionWriter<T> implements ResultPartitionWriter {
+ private final Collection<Object> output;
+ private final BufferProvider bufferProvider;
+ private final NonReusingDeserializationDelegate<T> delegate;
+ private final RecordDeserializer<DeserializationDelegate<T>> deserializer = new AdaptiveSpanningRecordDeserializer<>();
+
+ public RecordOrEventCollectingResultPartitionWriter(
+ Collection<Object> output,
+ BufferProvider bufferProvider,
+ TypeSerializer<T> serializer) {
+
+ this.output = checkNotNull(output);
+ this.bufferProvider = checkNotNull(bufferProvider);
+ this.delegate = new NonReusingDeserializationDelegate<>(checkNotNull(serializer));
+ }
+
+ @Override
+ public BufferProvider getBufferProvider() {
+ return bufferProvider;
+ }
+
+ @Override
+ public ResultPartitionID getPartitionId() {
+ return new ResultPartitionID();
+ }
+
+ @Override
+ public int getNumberOfSubpartitions() {
+ return 1;
+ }
+
+ @Override
+ public int getNumTargetKeyGroups() {
+ return 1;
+ }
+
+ @Override
+ public void writeBuffer(Buffer buffer, int targetChannel) throws IOException {
+ checkState(targetChannel < getNumberOfSubpartitions());
+
+ if (buffer.isBuffer()) {
+ deserializer.setNextBuffer(buffer);
+
+ while (deserializer.hasUnfinishedData()) {
+ RecordDeserializer.DeserializationResult result =
+ deserializer.getNextRecord(delegate);
+
+ if (result.isFullRecord()) {
+ output.add(delegate.getInstance());
+ }
+
+ if (result == RecordDeserializer.DeserializationResult.LAST_RECORD_FROM_BUFFER
+ || result == RecordDeserializer.DeserializationResult.PARTIAL_RECORD) {
+ break;
+ }
+ }
+ } else {
+ // is event
+ AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
+ output.add(event);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/0888bb62/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
index ee7337c..71371f0 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
@@ -24,54 +24,39 @@ import org.apache.flink.api.common.TaskInfo;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
-import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.io.network.TaskEventDispatcher;
-import org.apache.flink.runtime.io.network.api.serialization.AdaptiveSpanningRecordDeserializer;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
+import org.apache.flink.runtime.io.network.api.writer.RecordOrEventCollectingResultPartitionWriter;
import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
import org.apache.flink.runtime.query.KvStateRegistry;
import org.apache.flink.runtime.query.TaskKvStateRegistry;
import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
+import java.util.Collection;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
-import java.util.Queue;
import java.util.concurrent.Future;
import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
/**
* Mock {@link Environment}.
@@ -144,51 +129,12 @@ public class StreamMockEnvironment implements Environment {
inputs.add(gate);
}
- public <T> void addOutput(final Queue<Object> outputList, final TypeSerializer<T> serializer) {
+ public <T> void addOutput(final Collection<Object> outputList, final TypeSerializer<T> serializer) {
try {
- // The record-oriented writers wrap the buffer writer. We mock it
- // to collect the returned buffers and deserialize the content to
- // the output list
- BufferProvider mockBufferProvider = mock(BufferProvider.class);
- when(mockBufferProvider.requestBufferBlocking()).thenAnswer(new Answer<Buffer>() {
-
- @Override
- public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
- return new Buffer(
- MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
- mock(BufferRecycler.class));
- }
- });
-
- ResultPartitionWriter mockWriter = mock(ResultPartitionWriter.class);
- when(mockWriter.getNumberOfSubpartitions()).thenReturn(1);
- when(mockWriter.getBufferProvider()).thenReturn(mockBufferProvider);
-
- final RecordDeserializer<DeserializationDelegate<T>> recordDeserializer = new AdaptiveSpanningRecordDeserializer<DeserializationDelegate<T>>();
- final NonReusingDeserializationDelegate<T> delegate = new NonReusingDeserializationDelegate<T>(serializer);
-
- // Add records and events from the buffer to the output list
- doAnswer(new Answer<Void>() {
-
- @Override
- public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
- Buffer buffer = (Buffer) invocationOnMock.getArguments()[0];
- addBufferToOutputList(recordDeserializer, delegate, buffer, outputList);
- return null;
- }
- }).when(mockWriter).writeBuffer(any(Buffer.class), anyInt());
-
- doAnswer(new Answer<Void>() {
-
- @Override
- public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
- Buffer buffer = (Buffer) invocationOnMock.getArguments()[0];
- addBufferToOutputList(recordDeserializer, delegate, buffer, outputList);
- return null;
- }
- }).when(mockWriter).writeBufferToAllSubpartitions(any(Buffer.class));
-
- outputs.add(mockWriter);
+ outputs.add(new RecordOrEventCollectingResultPartitionWriter<T>(
+ outputList,
+ new TestPooledBufferProvider(Integer.MAX_VALUE),
+ serializer));
}
catch (Throwable t) {
t.printStackTrace();
@@ -196,44 +142,6 @@ public class StreamMockEnvironment implements Environment {
}
}
- /**
- * Adds the object behind the given <tt>buffer</tt> to the <tt>outputList</tt>.
- *
- * @param recordDeserializer de-serializer to use for the buffer
- * @param delegate de-serialization delegate to use for non-event buffers
- * @param buffer the buffer to add
- * @param outputList the output list to add the object to
- * @param <T> type of the objects behind the non-event buffers
- *
- * @throws java.io.IOException
- */
- private <T> void addBufferToOutputList(
- RecordDeserializer<DeserializationDelegate<T>> recordDeserializer,
- NonReusingDeserializationDelegate<T> delegate, Buffer buffer,
- final Queue<Object> outputList) throws java.io.IOException {
- if (buffer.isBuffer()) {
- recordDeserializer.setNextBuffer(buffer);
-
- while (recordDeserializer.hasUnfinishedData()) {
- RecordDeserializer.DeserializationResult result =
- recordDeserializer.getNextRecord(delegate);
-
- if (result.isFullRecord()) {
- outputList.add(delegate.getInstance());
- }
-
- if (result == RecordDeserializer.DeserializationResult.LAST_RECORD_FROM_BUFFER
- || result == RecordDeserializer.DeserializationResult.PARTIAL_RECORD) {
- break;
- }
- }
- } else {
- // is event
- AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
- outputList.add(event);
- }
- }
-
@Override
public Configuration getTaskConfiguration() {
return this.taskConfiguration;