Repository: avro Updated Branches: refs/heads/master 0b74bd5e7 -> c9f2c4924
AVRO-1857: GenericDatumWriter.write using BufferedBinaryEncoder leaves ByteBuffer in indeterminate state This closes #198 Project: http://git-wip-us.apache.org/repos/asf/avro/repo Commit: http://git-wip-us.apache.org/repos/asf/avro/commit/c9f2c492 Tree: http://git-wip-us.apache.org/repos/asf/avro/tree/c9f2c492 Diff: http://git-wip-us.apache.org/repos/asf/avro/diff/c9f2c492 Branch: refs/heads/master Commit: c9f2c49241397fb3967bd281225f7cce353c32fc Parents: 0b74bd5 Author: Nandor Kollar <[email protected]> Authored: Mon Feb 20 16:21:39 2017 +0100 Committer: Gabor Szadovszky <[email protected]> Committed: Thu May 25 09:41:06 2017 +0200 ---------------------------------------------------------------------- CHANGES.txt | 3 + .../apache/avro/io/BufferedBinaryEncoder.java | 5 +- .../java/org/apache/avro/io/TestEncoders.java | 60 ++++++++++++++++++++ 3 files changed, 66 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/avro/blob/c9f2c492/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 49266cb..72b3133 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -151,6 +151,9 @@ Trunk (not yet released) AVRO-2033. GenericUnions constructor defaults to a broken state (Philip Rizk via thiru) + AVRO-1857: GenericDatumWriter.write using BufferedBinaryEncoder leaves ByteBuffer in indeterminate state + (Nandor Kollar via gabor) + Avro 1.8.1 (14 May 2016) INCOMPATIBLE CHANGES http://git-wip-us.apache.org/repos/asf/avro/blob/c9f2c492/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java ---------------------------------------------------------------------- diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java b/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java index 82a36f9..e99f43f 100644 --- a/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java +++ b/lang/java/avro/src/main/java/org/apache/avro/io/BufferedBinaryEncoder.java @@ -157,11 +157,12 @@ public class BufferedBinaryEncoder extends BinaryEncoder { @Override public void writeFixed(ByteBuffer bytes) throws IOException { + ByteBuffer readOnlyBytes = bytes.asReadOnlyBuffer(); if (!bytes.hasArray() && bytes.remaining() > bulkLimit) { flushBuffer(); - sink.innerWrite(bytes); // bypass the buffer + sink.innerWrite(readOnlyBytes); // bypass the readOnlyBytes } else { - super.writeFixed(bytes); + super.writeFixed(readOnlyBytes); } } http://git-wip-us.apache.org/repos/asf/avro/blob/c9f2c492/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java ---------------------------------------------------------------------- diff --git a/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java b/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java index 4d16f16..c54e2c6 100644 --- a/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java +++ b/lang/java/avro/src/test/java/org/apache/avro/io/TestEncoders.java @@ -20,6 +20,12 @@ package org.apache.avro.io; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; import org.apache.avro.AvroTypeException; import org.apache.avro.Schema; @@ -32,7 +38,15 @@ import org.codehaus.jackson.JsonGenerator; import org.junit.Assert; import org.junit.Test; +import static java.util.Arrays.asList; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + public class TestEncoders { + private static final int ENCODER_BUFFER_SIZE = 32; + private static final int EXAMPLE_DATA_SIZE = 17; + private static EncoderFactory factory = EncoderFactory.get(); @Test @@ -193,4 +207,50 @@ public class TestEncoders { Assert.assertEquals("{\"a\": {\"a1\": null, \"a2\": true}}", o.toString()); } + @Test + public void testArrayBackedByteBuffer() throws IOException { + ByteBuffer buffer = ByteBuffer.wrap(someBytes(EXAMPLE_DATA_SIZE)); + + testWithBuffer(buffer); + } + + @Test + public void testMappedByteBuffer() throws IOException { + Path file = Files.createTempFile("test", "data"); + Files.write(file, someBytes(EXAMPLE_DATA_SIZE)); + MappedByteBuffer buffer = FileChannel.open(file, StandardOpenOption.READ).map(FileChannel.MapMode.READ_ONLY, 0, EXAMPLE_DATA_SIZE); + + testWithBuffer(buffer); + } + + private void testWithBuffer(ByteBuffer buffer) throws IOException { + assertThat(asList(buffer.position(), buffer.remaining()), is(asList(0, EXAMPLE_DATA_SIZE))); + + ByteArrayOutputStream output = new ByteArrayOutputStream(EXAMPLE_DATA_SIZE * 2); + EncoderFactory encoderFactory = new EncoderFactory(); + encoderFactory.configureBufferSize(ENCODER_BUFFER_SIZE); + + Encoder encoder = encoderFactory.binaryEncoder(output, null); + new GenericDatumWriter<ByteBuffer>(Schema.create(Schema.Type.BYTES)).write(buffer, encoder); + encoder.flush(); + + assertThat(output.toByteArray(), equalTo(avroEncoded(someBytes(EXAMPLE_DATA_SIZE)))); + assertThat(asList(buffer.position(), buffer.remaining()), is(asList(0, EXAMPLE_DATA_SIZE))); // fails if buffer is not array-backed and buffer overflow occurs + } + + private byte[] someBytes(int size) { + byte[] result = new byte[size]; + for (int i = 0; i < size; i++) { + result[i] = (byte) i; + } + return result; + } + + private byte[] avroEncoded(byte[] bytes) { + assert bytes.length < 64; + byte[] result = new byte[1 + bytes.length]; + result[0] = (byte) (bytes.length * 2); // zig-zag encoding + System.arraycopy(bytes, 0, result, 1, bytes.length); + return result; + } }
