Repository: flink
Updated Branches:
  refs/heads/master 2b8303cdc -> 676854b92


[FLINK-2800] [kryo] Fix Kryo serialization to clear buffered data

The Kryo serializer uses Kryo's Output class to buffer individual write 
operations before
it is written to the underlying output stream. This Output class is flushed by 
Flink's
KryoSerializer upon finishing its serialize call. However, in case of an 
exception when
flushing the Output, the buffered data is kept in the buffer. Since Flink uses 
EOFExceptions
to mark that an underlying buffer is full and has to be spilled, for example, 
it can happen
that the record triggering the spilling is written twice after it is rewritten. 
The reason
is that Kryo's Output buffer still contains the serialization data of the 
failed attempt which
is also flushed to the emptied output stream.

This duplication of records can lead to corrupted data which eventually let's 
the Flink program
crash. The problem is solved by clearing Kryo's Output when the flush operation 
was not successful.

This closes #1308


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b654e989
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b654e989
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b654e989

Branch: refs/heads/master
Commit: b654e989b60c6607f5f5bf757130c86bd0facd8c
Parents: 2b8303c
Author: Till Rohrmann <trohrm...@apache.org>
Authored: Wed Oct 28 18:40:41 2015 +0100
Committer: Fabian Hueske <fhue...@apache.org>
Committed: Thu Oct 29 10:18:07 2015 +0100

----------------------------------------------------------------------
 .../typeutils/runtime/kryo/KryoSerializer.java  |  17 +-
 .../runtime/kryo/KryoClearedBufferTest.java     | 283 +++++++++++++++++++
 2 files changed, 297 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b654e989/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
----------------------------------------------------------------------
diff --git 
a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
 
b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
index f825fc6..8549e26 100644
--- 
a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
+++ 
b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
@@ -58,7 +58,7 @@ import java.util.Objects;
  * @param <T> The type to be serialized.
  */
 public class KryoSerializer<T> extends TypeSerializer<T> {
-       
+
        private static final long serialVersionUID = 3L;
 
        // 
------------------------------------------------------------------------
@@ -82,7 +82,7 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
        
        private transient Input input;
        private transient Output output;
-       
+
        // 
------------------------------------------------------------------------
 
        public KryoSerializer(Class<T> type, ExecutionConfig executionConfig){
@@ -182,11 +182,22 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
                        previousOut = target;
                }
 
+               // Sanity check: Make sure that the output is cleared/has been 
flushed by the last call
+               // otherwise data might be written multiple times in case of a 
previous EOFException
+               if (output.position() != 0) {
+                       throw new IllegalStateException("The Kryo Output still 
contains data from a previous " +
+                               "serialize call. It has to be flushed or 
cleared at the end of the serialize call.");
+               }
+
                try {
                        kryo.writeClassAndObject(output, record);
                        output.flush();
                }
                catch (KryoException ke) {
+                       // make sure that the Kryo output buffer is cleared in 
case that we can recover from
+                       // the exception (e.g. EOFException which denotes 
buffer full)
+                       output.clear();
+
                        Throwable cause = ke.getCause();
                        if (cause instanceof EOFException) {
                                throw (EOFException) cause;
@@ -212,7 +223,7 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
                } catch (KryoException ke) {
                        Throwable cause = ke.getCause();
 
-                       if(cause instanceof EOFException) {
+                       if (cause instanceof EOFException) {
                                throw (EOFException) cause;
                        } else {
                                throw ke;

http://git-wip-us.apache.org/repos/asf/flink/blob/b654e989/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java
----------------------------------------------------------------------
diff --git 
a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java
 
b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java
new file mode 100644
index 0000000..ab2e45f
--- /dev/null
+++ 
b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java
@@ -0,0 +1,283 @@
+/*
+ * 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.api.java.typeutils.runtime.kryo;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.typeutils.runtime.ByteArrayInputView;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+
+public class KryoClearedBufferTest {
+
+       /**
+        * Tests that the kryo output buffer is cleared in case of an 
exception. Flink uses the
+        * EOFException to signal that a buffer is full. In such a case, the 
record which was tried
+        * to be written will be rewritten. Therefore, eventually buffered data 
of this record has
+        * to be cleared.
+        */
+       @Test
+       public void testOutputBufferedBeingClearedInCaseOfException() throws 
Exception {
+               ExecutionConfig executionConfig = new ExecutionConfig();
+               
executionConfig.registerTypeWithKryoSerializer(TestRecord.class, new 
TestRecordSerializer());
+               executionConfig.registerKryoType(TestRecord.class);
+
+               KryoSerializer<TestRecord> kryoSerializer = new 
KryoSerializer<TestRecord>(
+                       TestRecord.class,
+                       executionConfig);
+
+               int size = 94;
+               int bufferSize = 150;
+
+               TestRecord testRecord = new TestRecord(size);
+
+               TestDataOutputView target = new TestDataOutputView(bufferSize);
+
+               kryoSerializer.serialize(testRecord, target);
+
+               try {
+                       kryoSerializer.serialize(testRecord, target);
+                       Assert.fail("Expected an EOFException.");
+               } catch(EOFException eofException) {
+                       // expected exception
+                       // now the Kryo Output should have been cleared
+               }
+
+               TestRecord actualRecord = kryoSerializer.deserialize(new 
ByteArrayInputView(target.getBuffer()));
+
+               Assert.assertEquals(testRecord, actualRecord);
+
+               target.clear();
+
+               // if the kryo output has been cleared then we can serialize 
our test record into the target
+               // because the target buffer 150 bytes can host one TestRecord 
(total serialization size 100)
+               kryoSerializer.serialize(testRecord, target);
+
+               byte[] buffer = target.getBuffer();
+               int counter = 0;
+
+               for (int i = 0; i < buffer.length; i++) {
+                       if(buffer[i] == 42) {
+                               counter++;
+                       }
+               }
+
+               Assert.assertEquals(size, counter);
+       }
+
+       public static class TestRecord {
+               private byte[] buffer;
+
+               public TestRecord(int size) {
+                       buffer = new byte[size];
+
+                       Arrays.fill(buffer, (byte)42);
+               }
+
+               public TestRecord(byte[] buffer){
+                       this.buffer = buffer;
+               }
+
+               @Override
+               public boolean equals(Object obj) {
+                       if (obj instanceof TestRecord) {
+                               TestRecord record = (TestRecord) obj;
+
+                               return Arrays.equals(buffer, record.buffer);
+                       } else {
+                               return false;
+                       }
+               }
+       }
+
+       public static class TestRecordSerializer extends Serializer<TestRecord> 
implements Serializable {
+
+               private static final long serialVersionUID = 
6971996565421454985L;
+
+               @Override
+               public void write(Kryo kryo, Output output, TestRecord object) {
+                       output.writeInt(object.buffer.length);
+                       output.write(object.buffer);
+               }
+
+               @Override
+               public TestRecord read(Kryo kryo, Input input, 
Class<TestRecord> type) {
+                       int length = input.readInt();
+                       byte[] buffer = input.readBytes(length);
+
+                       return new TestRecord(buffer);
+               }
+       }
+
+       public static class TestDataOutputView implements DataOutputView {
+
+               private byte[] buffer;
+               private int position;
+
+               public TestDataOutputView(int size) {
+                       buffer = new byte[size];
+                       position = 0;
+               }
+
+               public void clear() {
+                       position = 0;
+               }
+
+               public byte[] getBuffer() {
+                       return buffer;
+               }
+
+               public void checkSize(int numBytes) throws EOFException {
+                       if (position + numBytes > buffer.length) {
+                               throw new EOFException();
+                       }
+               }
+
+               @Override
+               public void skipBytesToWrite(int numBytes) throws IOException {
+                       checkSize(numBytes);
+
+                       position += numBytes;
+               }
+
+               @Override
+               public void write(DataInputView source, int numBytes) throws 
IOException {
+                       checkSize(numBytes);
+
+                       byte[] tempBuffer = new byte[numBytes];
+
+                       source.read(tempBuffer);
+
+                       System.arraycopy(tempBuffer, 0, buffer, position, 
numBytes);
+
+                       position += numBytes;
+               }
+
+               @Override
+               public void write(int b) throws IOException {
+                       checkSize(4);
+
+                       position += 4;
+               }
+
+               @Override
+               public void write(byte[] b) throws IOException {
+                       checkSize(b.length);
+
+                       System.arraycopy(b, 0, buffer, position, b.length);
+                       position += b.length;
+               }
+
+               @Override
+               public void write(byte[] b, int off, int len) throws 
IOException {
+                       checkSize(len);
+
+                       System.arraycopy(b, off, buffer, position, len);
+
+                       position += len;
+               }
+
+               @Override
+               public void writeBoolean(boolean v) throws IOException {
+                       checkSize(1);
+                       position += 1;
+               }
+
+               @Override
+               public void writeByte(int v) throws IOException {
+                       checkSize(1);
+
+                       buffer[position] = (byte)v;
+
+                       position++;
+               }
+
+               @Override
+               public void writeShort(int v) throws IOException {
+                       checkSize(2);
+
+                       position += 2;
+               }
+
+               @Override
+               public void writeChar(int v) throws IOException {
+                       checkSize(1);
+                       position++;
+               }
+
+               @Override
+               public void writeInt(int v) throws IOException {
+                       checkSize(4);
+
+                       position += 4;
+               }
+
+               @Override
+               public void writeLong(long v) throws IOException {
+                       checkSize(8);
+                       position += 8;
+               }
+
+               @Override
+               public void writeFloat(float v) throws IOException {
+                       checkSize(4);
+                       position += 4;
+               }
+
+               @Override
+               public void writeDouble(double v) throws IOException {
+                       checkSize(8);
+                       position += 8;
+               }
+
+               @Override
+               public void writeBytes(String s) throws IOException {
+                       byte[] sBuffer = s.getBytes();
+                       checkSize(sBuffer.length);
+                       System.arraycopy(sBuffer, 0, buffer, position, 
sBuffer.length);
+                       position += sBuffer.length;
+               }
+
+               @Override
+               public void writeChars(String s) throws IOException {
+                       byte[] sBuffer = s.getBytes();
+                       checkSize(sBuffer.length);
+                       System.arraycopy(sBuffer, 0, buffer, position, 
sBuffer.length);
+                       position += sBuffer.length;
+               }
+
+               @Override
+               public void writeUTF(String s) throws IOException {
+                       byte[] sBuffer = s.getBytes();
+                       checkSize(sBuffer.length);
+                       System.arraycopy(sBuffer, 0, buffer, position, 
sBuffer.length);
+                       position += sBuffer.length;
+               }
+       }
+}

Reply via email to