You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/08/10 15:19:00 UTC

[jira] [Commented] (FLINK-9913) Improve output serialization only once in RecordWriter

    [ https://issues.apache.org/jira/browse/FLINK-9913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16576423#comment-16576423 ] 

ASF GitHub Bot commented on FLINK-9913:
---------------------------------------

pnowojski commented on a change in pull request #6417: [FLINK-9913][runtime] Improve output serialization only once in RecordWriter
URL: https://github.com/apache/flink/pull/6417#discussion_r209289603
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java
 ##########
 @@ -377,6 +388,86 @@ public void testBroadcastEmitBufferIndependence() throws Exception {
 		assertEquals("Buffer 2 shares the same reader index as buffer 1", 0, buffer2.getReaderIndex());
 	}
 
+	/**
+	 * Tests that records are broadcast via {@link ChannelSelector} and
+	 * {@link RecordWriter#emit(IOReadableWritable)}.
+	 */
+	@Test
+	public void testEmitRecordWithBroadcastPartitioner() throws Exception {
+		emitRecordWithBroadcastPartitionerOrBroadcastEmitRecord(false);
+	}
+
+	/**
+	 * Tests that records are broadcast via {@link RecordWriter#broadcastEmit(IOReadableWritable)}.
+	 */
+	@Test
+	public void testBroadcastEmitRecord() throws Exception {
+		emitRecordWithBroadcastPartitionerOrBroadcastEmitRecord(true);
+	}
+
+	/**
+	 * The results of emitting records via BroadcastPartitioner or broadcasting records directly are the same,
+	 * that is all the target channels can receive the whole outputs.
+	 *
+	 * @param isBroadcastEmit whether using {@link RecordWriter#broadcastEmit(IOReadableWritable)} or not
+	 */
+	private void emitRecordWithBroadcastPartitionerOrBroadcastEmitRecord(boolean isBroadcastEmit) throws Exception {
+		final int numChannels = 4;
+		final int bufferSize = 32;
+		final int numValues = 8;
+		final int serializationLength = 4;
+
+		@SuppressWarnings("unchecked")
+		final Queue<BufferConsumer>[] queues = new Queue[numChannels];
+		for (int i = 0; i < numChannels; i++) {
+			queues[i] = new ArrayDeque<>();
+		}
+
+		final TestPooledBufferProvider bufferProvider = new TestPooledBufferProvider(Integer.MAX_VALUE, bufferSize);
+		final ResultPartitionWriter partitionWriter = new CollectingPartitionWriter(queues, bufferProvider);
+		final RecordWriter<SerializationTestType> writer = new RecordWriter<>(partitionWriter, new Broadcast<>());
+		final RecordDeserializer<SerializationTestType> deserializer = new SpillingAdaptiveSpanningRecordDeserializer<>(
+			new String[]{ tempFolder.getRoot().getAbsolutePath() });
+
+		final ArrayDeque<SerializationTestType> serializedRecords = new ArrayDeque<>();
+		final Iterable<SerializationTestType> records = Util.randomRecords(numValues, SerializationTestTypeFactory.INT);
+		for (SerializationTestType record : records) {
+			serializedRecords.add(record);
+
+			if (isBroadcastEmit) {
+				writer.broadcastEmit(record);
+			} else {
+				writer.emit(record);
+			}
+		}
+
+		final int requiredBuffers = numValues / (bufferSize / (4 + serializationLength));
+		for (int i = 0; i < numChannels; i++) {
 
 Review comment:
   can you somehow extract common logic of this method and `org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializationTest#testSerializationRoundTrip(java.lang.Iterable<org.apache.flink.testutils.serialization.types.SerializationTestType>, int, org.apache.flink.runtime.io.network.api.serialization.RecordSerializer<org.apache.flink.testutils.serialization.types.SerializationTestType>, org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer<org.apache.flink.testutils.serialization.types.SerializationTestType>)`? They share a lot of core.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Improve output serialization only once in RecordWriter
> ------------------------------------------------------
>
>                 Key: FLINK-9913
>                 URL: https://issues.apache.org/jira/browse/FLINK-9913
>             Project: Flink
>          Issue Type: Improvement
>          Components: Network
>    Affects Versions: 1.6.0
>            Reporter: zhijiang
>            Assignee: zhijiang
>            Priority: Minor
>              Labels: pull-request-available
>             Fix For: 1.7.0
>
>
> Currently the {{RecordWriter}} emits output into multi channels via {{ChannelSelector}}  or broadcasts output to all channels directly. Each channel has a separate {{RecordSerializer}} for serializing outputs, that means the output will be serialized as many times as the number of selected channels.
> As we know, data serialization is a high cost operation, so we can get good benefits by improving the serialization only once.
> I would suggest the following changes for realizing it.
>  # Only one {{RecordSerializer}} is created in {{RecordWriter}} for all the channels.
>  # The output is serialized into the intermediate data buffer only once for different channels.
>  # The intermediate serialization results are copied into different {{BufferBuilder}}s for different channels.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)