You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by pn...@apache.org on 2019/07/04 11:30:21 UTC
[flink] 03/06: [FLINK-13016][network] Fix
StreamTaskNetworkInput#isAvailable
This is an automated email from the ASF dual-hosted git repository.
pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 76eb11fdc4561e15c7f3ad614b416cd2a84bf50e
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jun 27 13:39:32 2019 +0200
[FLINK-13016][network] Fix StreamTaskNetworkInput#isAvailable
Before this method was ignoring data/records buffered in the currentRecordDeserializer
---
.../io/network/buffer/BufferBuilderTestUtils.java | 4 +
.../runtime/io/StreamTaskNetworkInput.java | 3 +
.../flink/streaming/runtime/io/MockInputGate.java | 17 +++-
.../runtime/io/StreamTaskNetworkInputTest.java | 101 +++++++++++++++++++++
4 files changed, 123 insertions(+), 2 deletions(-)
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java
index 7a68368..7696e08 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java
@@ -112,4 +112,8 @@ public class BufferBuilderTestUtils {
final MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(size);
return new NetworkBuffer(seg, MemorySegment::free, true, size);
}
+
+ public static BufferBuilder createEmptyBufferBuilder(int bufferSize) {
+ return new BufferBuilder(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), FreeingBufferRecycler.INSTANCE);
+ }
}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
index 8c37141..b10e892 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java
@@ -148,6 +148,9 @@ public final class StreamTaskNetworkInput implements StreamTaskInput {
@Override
public CompletableFuture<?> isAvailable() {
+ if (currentRecordDeserializer != null) {
+ return AVAILABLE;
+ }
return checkpointedInputGate.isAvailable();
}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
index 65984f4..8cb6848 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
@@ -39,10 +39,20 @@ public class MockInputGate extends InputGate {
private final boolean[] closed;
- MockInputGate(int numberOfChannels, List<BufferOrEvent> bufferOrEvents) {
+ private final boolean finishAfterLastBuffer;
+
+ public MockInputGate(int numberOfChannels, List<BufferOrEvent> bufferOrEvents) {
+ this(numberOfChannels, bufferOrEvents, true);
+ }
+
+ public MockInputGate(
+ int numberOfChannels,
+ List<BufferOrEvent> bufferOrEvents,
+ boolean finishAfterLastBuffer) {
this.numberOfChannels = numberOfChannels;
this.bufferOrEvents = new ArrayDeque<BufferOrEvent>(bufferOrEvents);
this.closed = new boolean[numberOfChannels];
+ this.finishAfterLastBuffer = finishAfterLastBuffer;
isAvailable = AVAILABLE;
}
@@ -58,12 +68,15 @@ public class MockInputGate extends InputGate {
@Override
public boolean isFinished() {
- return bufferOrEvents.isEmpty();
+ return finishAfterLastBuffer && bufferOrEvents.isEmpty();
}
@Override
public Optional<BufferOrEvent> getNext() {
BufferOrEvent next = bufferOrEvents.poll();
+ if (!finishAfterLastBuffer && bufferOrEvents.isEmpty()) {
+ resetIsAvailable();
+ }
if (next == null) {
return Optional.empty();
}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java
new file mode 100644
index 0000000..202b699
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.streaming.runtime.io;
+
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+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.api.serialization.RecordSerializer;
+import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+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.partition.consumer.BufferOrEvent;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import org.junit.After;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link StreamTaskNetworkInput}.
+ */
+public class StreamTaskNetworkInputTest {
+
+ private static final int PAGE_SIZE = 1000;
+
+ private final IOManager ioManager = new IOManagerAsync();
+
+ @After
+ public void tearDown() throws Exception {
+ ioManager.close();
+ }
+
+ @Test
+ public void testIsAvailableWithBufferedDataInDeserializer() throws Exception {
+ BufferBuilder bufferBuilder = BufferBuilderTestUtils.createEmptyBufferBuilder(PAGE_SIZE);
+
+ serializeRecord(42L, bufferBuilder);
+ serializeRecord(44L, bufferBuilder);
+
+ Buffer buffer = bufferBuilder.createBufferConsumer().build();
+
+ List<BufferOrEvent> buffers = Collections.singletonList(new BufferOrEvent(buffer, 0, false));
+
+ StreamTaskNetworkInput input = new StreamTaskNetworkInput(
+ new CheckpointedInputGate(
+ new MockInputGate(1, buffers, false),
+ new EmptyBufferStorage(),
+ new CheckpointBarrierTracker(1)),
+ LongSerializer.INSTANCE,
+ ioManager,
+ 0);
+
+ assertHasNextElement(input);
+ assertHasNextElement(input);
+ }
+
+ private void serializeRecord(long value, BufferBuilder bufferBuilder) throws IOException {
+ RecordSerializer<SerializationDelegate<StreamElement>> serializer = new SpanningRecordSerializer<>();
+ SerializationDelegate<StreamElement> serializationDelegate =
+ new SerializationDelegate<>(
+ new StreamElementSerializer<>(LongSerializer.INSTANCE));
+ serializationDelegate.setInstance(new StreamRecord<>(value));
+ serializer.serializeRecord(serializationDelegate);
+
+ assertFalse(serializer.copyToBufferBuilder(bufferBuilder).isFullBuffer());
+ }
+
+ private static void assertHasNextElement(StreamTaskNetworkInput input) throws Exception {
+ assertTrue(input.isAvailable().isDone());
+ StreamElement element = input.pollNextNullable();
+ assertNotNull(element);
+ assertTrue(element.isRecord());
+ }
+}