This is an automated email from the ASF dual-hosted git repository.
fokko pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/avro.git
The following commit(s) were added to refs/heads/main by this push:
new 9f9023cd0 AVRO-3871: Support nested lists/maps in
BlockingDirectBinaryEncoder (#2732)
9f9023cd0 is described below
commit 9f9023cd03d65b6c4793d592037776a88206423c
Author: Fokko Driesprong <[email protected]>
AuthorDate: Sun May 5 22:48:23 2024 +0200
AVRO-3871: Support nested lists/maps in BlockingDirectBinaryEncoder (#2732)
* Support nested lists/maps
* Add some tests
---
.../avro/io/BlockingDirectBinaryEncoder.java | 38 +-
.../org/apache/avro/io/DirectBinaryEncoder.java | 2 +-
.../avro/io/TestBlockingDirectBinaryEncoder.java | 77 ++-
.../avro/specific/TestRecordWithMapsAndArrays.java | 532 +++++++++++++++++----
.../resources/TestRecordWithMapsAndArrays.avsc | 24 +
5 files changed, 544 insertions(+), 129 deletions(-)
diff --git
a/lang/java/avro/src/main/java/org/apache/avro/io/BlockingDirectBinaryEncoder.java
b/lang/java/avro/src/main/java/org/apache/avro/io/BlockingDirectBinaryEncoder.java
index b029034d0..2ef2375e6 100644
---
a/lang/java/avro/src/main/java/org/apache/avro/io/BlockingDirectBinaryEncoder.java
+++
b/lang/java/avro/src/main/java/org/apache/avro/io/BlockingDirectBinaryEncoder.java
@@ -21,6 +21,8 @@ import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
/**
* An {@link Encoder} for Avro's binary encoding that does not buffer output.
@@ -46,13 +48,13 @@ import java.nio.ByteBuffer;
* @see Decoder
*/
public class BlockingDirectBinaryEncoder extends DirectBinaryEncoder {
- private final BufferOutputStream buffer;
+ private final ArrayList<BufferOutputStream> buffers;
- private OutputStream originalStream;
+ private final ArrayDeque<OutputStream> stashedBuffers;
- private boolean inBlock = false;
+ private int depth = 0;
- private long blockItemCount;
+ private final ArrayDeque<Long> blockItemCounts;
/**
* Create a writer that sends its output to the underlying stream
@@ -62,24 +64,30 @@ public class BlockingDirectBinaryEncoder extends
DirectBinaryEncoder {
*/
public BlockingDirectBinaryEncoder(OutputStream out) {
super(out);
- this.buffer = new BufferOutputStream();
+ this.buffers = new ArrayList<>();
+ this.stashedBuffers = new ArrayDeque<>();
+ this.blockItemCounts = new ArrayDeque<>();
}
private void startBlock() {
- if (inBlock) {
- throw new RuntimeException("Nested Maps/Arrays are not supported by the
BlockingDirectBinaryEncoder");
+ stashedBuffers.push(out);
+ if (this.buffers.size() <= depth) {
+ this.buffers.add(new BufferOutputStream());
}
- originalStream = out;
- buffer.reset();
- out = buffer;
- inBlock = true;
+ BufferOutputStream buf = buffers.get(depth);
+ buf.reset();
+ this.depth += 1;
+ this.out = buf;
}
private void endBlock() {
- if (!inBlock) {
+ if (depth == 0) {
throw new RuntimeException("Called endBlock, while not buffering a
block");
}
- out = originalStream;
+ this.depth -= 1;
+ out = stashedBuffers.pop();
+ BufferOutputStream buffer = this.buffers.get(depth);
+ long blockItemCount = blockItemCounts.pop();
if (blockItemCount > 0) {
try {
// Make it negative, so the reader knows that the number of bytes is
coming
@@ -90,13 +98,11 @@ public class BlockingDirectBinaryEncoder extends
DirectBinaryEncoder {
throw new RuntimeException(e);
}
}
- inBlock = false;
- buffer.reset();
}
@Override
public void setItemCount(long itemCount) throws IOException {
- blockItemCount = itemCount;
+ blockItemCounts.push(itemCount);
}
@Override
diff --git
a/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryEncoder.java
b/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryEncoder.java
index 8d8172bc2..df7c118b6 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryEncoder.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/io/DirectBinaryEncoder.java
@@ -48,7 +48,7 @@ public class DirectBinaryEncoder extends BinaryEncoder {
* Create a writer that sends its output to the underlying stream
* <code>out</code>.
**/
- public DirectBinaryEncoder(OutputStream out) {
+ protected DirectBinaryEncoder(OutputStream out) {
configure(out);
}
diff --git
a/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingDirectBinaryEncoder.java
b/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingDirectBinaryEncoder.java
index da2976d18..fee4c23e1 100644
---
a/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingDirectBinaryEncoder.java
+++
b/lang/java/avro/src/test/java/org/apache/avro/io/TestBlockingDirectBinaryEncoder.java
@@ -22,6 +22,7 @@ import org.apache.avro.SchemaNormalization;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.message.BinaryMessageDecoder;
import org.apache.avro.specific.TestRecordWithMapsAndArrays;
+import org.hamcrest.Matchers;
import org.junit.jupiter.api.Test;
import java.io.ByteArrayOutputStream;
@@ -31,6 +32,7 @@ import java.util.Arrays;
import java.util.Map;
import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.hasEntry;
import static org.hamcrest.Matchers.is;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
@@ -38,6 +40,27 @@ import static org.mockito.Mockito.verify;
public class TestBlockingDirectBinaryEncoder {
+ private void writeToArray(BinaryEncoder encoder, int[] numbers) throws
IOException {
+ encoder.writeArrayStart();
+ encoder.setItemCount(numbers.length);
+ for (int number : numbers) {
+ encoder.startItem();
+ encoder.writeString(Integer.toString(number));
+ }
+ encoder.writeArrayEnd();
+ }
+
+ private void writeToMap(BinaryEncoder encoder, long[] numbers) throws
IOException {
+ encoder.writeMapStart();
+ encoder.setItemCount(numbers.length);
+ for (long number : numbers) {
+ encoder.startItem();
+ encoder.writeString(Long.toString(number));
+ encoder.writeLong(number);
+ }
+ encoder.writeMapEnd();
+ }
+
@Test
void blockingDirectBinaryEncoder() throws IOException,
NoSuchAlgorithmException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -49,35 +72,57 @@ public class TestBlockingDirectBinaryEncoder {
encoder.writeFixed(new byte[] { (byte) 0xC3, (byte) 0x01 });
encoder.writeFixed(SchemaNormalization.parsingFingerprint("CRC-64-AVRO",
TestRecordWithMapsAndArrays.SCHEMA$));
- int len = 5;
+ // Array
+ this.writeToArray(encoder, new int[] { 1, 2, 3, 4, 5 });
+
+ // Map
+ writeToMap(encoder, new long[] { 1L, 2L, 3L, 4L, 5L });
+
+ // Nested Array
encoder.writeArrayStart();
- encoder.setItemCount(len);
- for (int i = 0; i < len; i++) {
- encoder.startItem();
- encoder.writeString(Integer.toString(i));
- }
+ encoder.setItemCount(2);
+ this.writeToArray(encoder, new int[] { 1, 2 });
+ this.writeToArray(encoder, new int[] { 3, 4, 5 });
encoder.writeArrayEnd();
+ // Nested Map
+
encoder.writeMapStart();
- encoder.setItemCount(len);
- for (long i = 0; i < len; i++) {
- encoder.startItem();
- encoder.writeString(Long.toString(i));
- encoder.writeLong(i);
- }
+ encoder.setItemCount(2);
+ encoder.writeString("first");
+ this.writeToMap(encoder, new long[] { 1L, 2L });
+ encoder.writeString("second");
+ this.writeToMap(encoder, new long[] { 3L, 4L, 5L });
encoder.writeMapEnd();
+
+ // Read
+
encoder.flush();
BinaryMessageDecoder<TestRecordWithMapsAndArrays> decoder =
TestRecordWithMapsAndArrays.getDecoder();
TestRecordWithMapsAndArrays r = decoder.decode(baos.toByteArray());
- assertThat(r.getArr(), is(Arrays.asList("0", "1", "2", "3", "4")));
+ assertThat(r.getArr(), is(Arrays.asList("1", "2", "3", "4", "5")));
Map<String, Long> map = r.getMap();
assertThat(map.size(), is(5));
- for (long i = 0; i < len; i++) {
+ for (long i = 1; i <= 5; i++) {
assertThat(map.get(Long.toString(i)), is(i));
}
+
+ assertThat(r.getNestedArr(), is(Arrays.asList(Arrays.asList("1", "2"),
Arrays.asList("3", "4", "5"))));
+
+ Map<String, Map<String, Long>> nestedMap = r.getNestedMap();
+ assertThat(nestedMap.size(), is(2));
+
+ assertThat(nestedMap.get("first").size(), is(2));
+ assertThat(nestedMap.get("first").get("1"), is(1L));
+ assertThat(nestedMap.get("first").get("2"), is(2L));
+
+ assertThat(nestedMap.get("second").size(), is(3));
+ assertThat(nestedMap.get("second").get("3"), is(3L));
+ assertThat(nestedMap.get("second").get("4"), is(4L));
+ assertThat(nestedMap.get("second").get("5"), is(5L));
}
@Test
@@ -93,8 +138,8 @@ public class TestBlockingDirectBinaryEncoder {
in.read(null, mockDecoder);
}
- verify(mockDecoder, times(1)).skipMap();
- verify(mockDecoder, times(1)).skipArray();
+ verify(mockDecoder, times(2)).skipMap();
+ verify(mockDecoder, times(2)).skipArray();
verify(mockDecoder, times(0)).readString();
verify(mockDecoder, times(0)).readLong();
}
diff --git
a/lang/java/avro/src/test/java/org/apache/avro/specific/TestRecordWithMapsAndArrays.java
b/lang/java/avro/src/test/java/org/apache/avro/specific/TestRecordWithMapsAndArrays.java
index 7150f5001..81572bc22 100644
---
a/lang/java/avro/src/test/java/org/apache/avro/specific/TestRecordWithMapsAndArrays.java
+++
b/lang/java/avro/src/test/java/org/apache/avro/specific/TestRecordWithMapsAndArrays.java
@@ -24,13 +24,12 @@ import org.apache.avro.message.BinaryMessageEncoder;
import org.apache.avro.message.BinaryMessageDecoder;
import org.apache.avro.message.SchemaStore;
[email protected]
-public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.SpecificRecordBase
- implements org.apache.avro.specific.SpecificRecord {
- private static final long serialVersionUID = 3113266652594662627L;
+@AvroGenerated
+public class TestRecordWithMapsAndArrays extends SpecificRecordBase implements
SpecificRecord {
+ private static final long serialVersionUID = -3823801533006425147L;
public static final org.apache.avro.Schema SCHEMA$ = new
org.apache.avro.Schema.Parser().parse(
-
"{\"type\":\"record\",\"name\":\"TestRecordWithMapsAndArrays\",\"namespace\":\"org.apache.avro.specific\",\"fields\":[{\"name\":\"arr\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"default\":[]}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"values\":\"long\",\"avro.java.string\":\"String\",\"default\":{}}}]}");
+
"{\"type\":\"record\",\"name\":\"TestRecordWithMapsAndArrays\",\"namespace\":\"org.apache.avro.specific\",\"fields\":[{\"name\":\"arr\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"default\":[]}},{\"name\":\"map\",\"type\":{\"type\":\"map\",\"values\":\"long\",\"avro.java.string\":\"String\",\"default\":{}}},{\"name\":\"nested_arr\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java
[...]
public static org.apache.avro.Schema getClassSchema() {
return SCHEMA$;
@@ -46,7 +45,7 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Return the BinaryMessageEncoder instance used by this class.
- *
+ *
* @return the message encoder used by this class
*/
public static BinaryMessageEncoder<TestRecordWithMapsAndArrays> getEncoder()
{
@@ -55,7 +54,7 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Return the BinaryMessageDecoder instance used by this class.
- *
+ *
* @return the message decoder used by this class
*/
public static BinaryMessageDecoder<TestRecordWithMapsAndArrays> getDecoder()
{
@@ -65,7 +64,7 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Create a new BinaryMessageDecoder instance for this class that uses the
* specified {@link SchemaStore}.
- *
+ *
* @param resolver a {@link SchemaStore} used to find schemas by fingerprint
* @return a BinaryMessageDecoder instance for this class backed by the given
* SchemaStore
@@ -76,7 +75,7 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Serializes this TestRecordWithMapsAndArrays to a ByteBuffer.
- *
+ *
* @return a buffer holding the serialized data for this instance
* @throws java.io.IOException if this instance could not be serialized
*/
@@ -86,7 +85,7 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Deserializes a TestRecordWithMapsAndArrays from a ByteBuffer.
- *
+ *
* @param b a byte buffer holding serialized data for an instance of this
class
* @return a TestRecordWithMapsAndArrays instance decoded from the given
buffer
* @throws java.io.IOException if the given bytes could not be deserialized
into
@@ -96,8 +95,10 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
return DECODER.decode(b);
}
- private java.util.List<java.lang.String> arr;
- private java.util.Map<java.lang.String, java.lang.Long> map;
+ private java.util.List<String> arr;
+ private java.util.Map<String, Long> map;
+ private java.util.List<java.util.List<String>> nested_arr;
+ private java.util.Map<String, java.util.Map<String, Long>> nested_map;
/**
* Default constructor. Note that this does not initialize fields to their
@@ -109,18 +110,23 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* All-args constructor.
- *
- * @param arr The new value for arr
- * @param map The new value for map
+ *
+ * @param arr The new value for arr
+ * @param map The new value for map
+ * @param nested_arr The new value for nested_arr
+ * @param nested_map The new value for nested_map
*/
- public TestRecordWithMapsAndArrays(java.util.List<java.lang.String> arr,
- java.util.Map<java.lang.String, java.lang.Long> map) {
+ public TestRecordWithMapsAndArrays(java.util.List<String> arr,
java.util.Map<String, Long> map,
+ java.util.List<java.util.List<String>> nested_arr,
+ java.util.Map<String, java.util.Map<String, Long>> nested_map) {
this.arr = arr;
this.map = map;
+ this.nested_arr = nested_arr;
+ this.nested_map = nested_map;
}
@Override
- public org.apache.avro.specific.SpecificData getSpecificData() {
+ public SpecificData getSpecificData() {
return MODEL$;
}
@@ -131,12 +137,16 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
// Used by DatumWriter. Applications should not call.
@Override
- public java.lang.Object get(int field$) {
+ public Object get(int field$) {
switch (field$) {
case 0:
return arr;
case 1:
return map;
+ case 2:
+ return nested_arr;
+ case 3:
+ return nested_map;
default:
throw new IndexOutOfBoundsException("Invalid index: " + field$);
}
@@ -145,13 +155,19 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
// Used by DatumReader. Applications should not call.
@Override
@SuppressWarnings(value = "unchecked")
- public void put(int field$, java.lang.Object value$) {
+ public void put(int field$, Object value$) {
switch (field$) {
case 0:
- arr = (java.util.List<java.lang.String>) value$;
+ arr = (java.util.List<String>) value$;
break;
case 1:
- map = (java.util.Map<java.lang.String, java.lang.Long>) value$;
+ map = (java.util.Map<String, Long>) value$;
+ break;
+ case 2:
+ nested_arr = (java.util.List<java.util.List<String>>) value$;
+ break;
+ case 3:
+ nested_map = (java.util.Map<String, java.util.Map<String, Long>>) value$;
break;
default:
throw new IndexOutOfBoundsException("Invalid index: " + field$);
@@ -160,90 +176,126 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Gets the value of the 'arr' field.
- *
+ *
* @return The value of the 'arr' field.
*/
- public java.util.List<java.lang.String> getArr() {
+ public java.util.List<String> getArr() {
return arr;
}
/**
* Sets the value of the 'arr' field.
- *
+ *
* @param value the value to set.
*/
- public void setArr(java.util.List<java.lang.String> value) {
+ public void setArr(java.util.List<String> value) {
this.arr = value;
}
/**
* Gets the value of the 'map' field.
- *
+ *
* @return The value of the 'map' field.
*/
- public java.util.Map<java.lang.String, java.lang.Long> getMap() {
+ public java.util.Map<String, Long> getMap() {
return map;
}
/**
* Sets the value of the 'map' field.
- *
+ *
* @param value the value to set.
*/
- public void setMap(java.util.Map<java.lang.String, java.lang.Long> value) {
+ public void setMap(java.util.Map<String, Long> value) {
this.map = value;
}
+ /**
+ * Gets the value of the 'nested_arr' field.
+ *
+ * @return The value of the 'nested_arr' field.
+ */
+ public java.util.List<java.util.List<String>> getNestedArr() {
+ return nested_arr;
+ }
+
+ /**
+ * Sets the value of the 'nested_arr' field.
+ *
+ * @param value the value to set.
+ */
+ public void setNestedArr(java.util.List<java.util.List<String>> value) {
+ this.nested_arr = value;
+ }
+
+ /**
+ * Gets the value of the 'nested_map' field.
+ *
+ * @return The value of the 'nested_map' field.
+ */
+ public java.util.Map<String, java.util.Map<String, Long>> getNestedMap() {
+ return nested_map;
+ }
+
+ /**
+ * Sets the value of the 'nested_map' field.
+ *
+ * @param value the value to set.
+ */
+ public void setNestedMap(java.util.Map<String, java.util.Map<String, Long>>
value) {
+ this.nested_map = value;
+ }
+
/**
* Creates a new TestRecordWithMapsAndArrays RecordBuilder.
- *
+ *
* @return A new TestRecordWithMapsAndArrays RecordBuilder
*/
- public static org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder
newBuilder() {
- return new org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder();
+ public static Builder newBuilder() {
+ return new Builder();
}
/**
* Creates a new TestRecordWithMapsAndArrays RecordBuilder by copying an
* existing Builder.
- *
+ *
* @param other The existing builder to copy.
* @return A new TestRecordWithMapsAndArrays RecordBuilder
*/
- public static org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder
newBuilder(
- org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder other) {
+ public static Builder newBuilder(Builder other) {
if (other == null) {
- return new
org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder();
+ return new Builder();
} else {
- return new
org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder(other);
+ return new Builder(other);
}
}
/**
* Creates a new TestRecordWithMapsAndArrays RecordBuilder by copying an
* existing TestRecordWithMapsAndArrays instance.
- *
+ *
* @param other The existing instance to copy.
* @return A new TestRecordWithMapsAndArrays RecordBuilder
*/
- public static org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder
newBuilder(
- org.apache.avro.specific.TestRecordWithMapsAndArrays other) {
+ public static Builder newBuilder(TestRecordWithMapsAndArrays other) {
if (other == null) {
- return new
org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder();
+ return new Builder();
} else {
- return new
org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder(other);
+ return new Builder(other);
}
}
/**
* RecordBuilder for TestRecordWithMapsAndArrays instances.
*/
- @org.apache.avro.specific.AvroGenerated
- public static class Builder extends
org.apache.avro.specific.SpecificRecordBuilderBase<TestRecordWithMapsAndArrays>
+ @AvroGenerated
+ public static class Builder extends
SpecificRecordBuilderBase<TestRecordWithMapsAndArrays>
implements
org.apache.avro.data.RecordBuilder<TestRecordWithMapsAndArrays> {
- private java.util.List<java.lang.String> arr;
- private java.util.Map<java.lang.String, java.lang.Long> map;
+ private java.util.List<String> arr;
+ private java.util.Map<String, Long> map;
+ private java.util.List<java.util.List<String>> nested_arr;
+ private java.util.Map<String, java.util.Map<String, Long>> nested_map;
/** Creates a new Builder */
private Builder() {
@@ -252,10 +304,10 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Creates a Builder by copying an existing Builder.
- *
+ *
* @param other The existing Builder to copy.
*/
- private
Builder(org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder other) {
+ private Builder(Builder other) {
super(other);
if (isValidValue(fields()[0], other.arr)) {
this.arr = data().deepCopy(fields()[0].schema(), other.arr);
@@ -265,14 +317,22 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
this.map = data().deepCopy(fields()[1].schema(), other.map);
fieldSetFlags()[1] = other.fieldSetFlags()[1];
}
+ if (isValidValue(fields()[2], other.nested_arr)) {
+ this.nested_arr = data().deepCopy(fields()[2].schema(),
other.nested_arr);
+ fieldSetFlags()[2] = other.fieldSetFlags()[2];
+ }
+ if (isValidValue(fields()[3], other.nested_map)) {
+ this.nested_map = data().deepCopy(fields()[3].schema(),
other.nested_map);
+ fieldSetFlags()[3] = other.fieldSetFlags()[3];
+ }
}
/**
* Creates a Builder by copying an existing TestRecordWithMapsAndArrays
instance
- *
+ *
* @param other The existing instance to copy.
*/
- private Builder(org.apache.avro.specific.TestRecordWithMapsAndArrays
other) {
+ private Builder(TestRecordWithMapsAndArrays other) {
super(SCHEMA$, MODEL$);
if (isValidValue(fields()[0], other.arr)) {
this.arr = data().deepCopy(fields()[0].schema(), other.arr);
@@ -282,24 +342,32 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
this.map = data().deepCopy(fields()[1].schema(), other.map);
fieldSetFlags()[1] = true;
}
+ if (isValidValue(fields()[2], other.nested_arr)) {
+ this.nested_arr = data().deepCopy(fields()[2].schema(),
other.nested_arr);
+ fieldSetFlags()[2] = true;
+ }
+ if (isValidValue(fields()[3], other.nested_map)) {
+ this.nested_map = data().deepCopy(fields()[3].schema(),
other.nested_map);
+ fieldSetFlags()[3] = true;
+ }
}
/**
* Gets the value of the 'arr' field.
- *
+ *
* @return The value.
*/
- public java.util.List<java.lang.String> getArr() {
+ public java.util.List<String> getArr() {
return arr;
}
/**
* Sets the value of the 'arr' field.
- *
+ *
* @param value The value of 'arr'.
* @return This builder.
*/
- public org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder
setArr(java.util.List<java.lang.String> value) {
+ public Builder setArr(java.util.List<String> value) {
validate(fields()[0], value);
this.arr = value;
fieldSetFlags()[0] = true;
@@ -308,7 +376,7 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Checks whether the 'arr' field has been set.
- *
+ *
* @return True if the 'arr' field has been set, false otherwise.
*/
public boolean hasArr() {
@@ -317,10 +385,10 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Clears the value of the 'arr' field.
- *
+ *
* @return This builder.
*/
- public org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder
clearArr() {
+ public Builder clearArr() {
arr = null;
fieldSetFlags()[0] = false;
return this;
@@ -328,21 +396,20 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Gets the value of the 'map' field.
- *
+ *
* @return The value.
*/
- public java.util.Map<java.lang.String, java.lang.Long> getMap() {
+ public java.util.Map<String, Long> getMap() {
return map;
}
/**
* Sets the value of the 'map' field.
- *
+ *
* @param value The value of 'map'.
* @return This builder.
*/
- public org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder setMap(
- java.util.Map<java.lang.String, java.lang.Long> value) {
+ public Builder setMap(java.util.Map<String, Long> value) {
validate(fields()[1], value);
this.map = value;
fieldSetFlags()[1] = true;
@@ -351,7 +418,7 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Checks whether the 'map' field has been set.
- *
+ *
* @return True if the 'map' field has been set, false otherwise.
*/
public boolean hasMap() {
@@ -360,27 +427,114 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
/**
* Clears the value of the 'map' field.
- *
+ *
* @return This builder.
*/
- public org.apache.avro.specific.TestRecordWithMapsAndArrays.Builder
clearMap() {
+ public Builder clearMap() {
map = null;
fieldSetFlags()[1] = false;
return this;
}
+ /**
+ * Gets the value of the 'nested_arr' field.
+ *
+ * @return The value.
+ */
+ public java.util.List<java.util.List<String>> getNestedArr() {
+ return nested_arr;
+ }
+
+ /**
+ * Sets the value of the 'nested_arr' field.
+ *
+ * @param value The value of 'nested_arr'.
+ * @return This builder.
+ */
+ public Builder setNestedArr(java.util.List<java.util.List<String>> value) {
+ validate(fields()[2], value);
+ this.nested_arr = value;
+ fieldSetFlags()[2] = true;
+ return this;
+ }
+
+ /**
+ * Checks whether the 'nested_arr' field has been set.
+ *
+ * @return True if the 'nested_arr' field has been set, false otherwise.
+ */
+ public boolean hasNestedArr() {
+ return fieldSetFlags()[2];
+ }
+
+ /**
+ * Clears the value of the 'nested_arr' field.
+ *
+ * @return This builder.
+ */
+ public Builder clearNestedArr() {
+ nested_arr = null;
+ fieldSetFlags()[2] = false;
+ return this;
+ }
+
+ /**
+ * Gets the value of the 'nested_map' field.
+ *
+ * @return The value.
+ */
+ public java.util.Map<String, java.util.Map<String, Long>> getNestedMap() {
+ return nested_map;
+ }
+
+ /**
+ * Sets the value of the 'nested_map' field.
+ *
+ * @param value The value of 'nested_map'.
+ * @return This builder.
+ */
+ public Builder setNestedMap(java.util.Map<String, java.util.Map<String,
Long>> value) {
+ validate(fields()[3], value);
+ this.nested_map = value;
+ fieldSetFlags()[3] = true;
+ return this;
+ }
+
+ /**
+ * Checks whether the 'nested_map' field has been set.
+ *
+ * @return True if the 'nested_map' field has been set, false otherwise.
+ */
+ public boolean hasNestedMap() {
+ return fieldSetFlags()[3];
+ }
+
+ /**
+ * Clears the value of the 'nested_map' field.
+ *
+ * @return This builder.
+ */
+ public Builder clearNestedMap() {
+ nested_map = null;
+ fieldSetFlags()[3] = false;
+ return this;
+ }
+
@Override
@SuppressWarnings("unchecked")
public TestRecordWithMapsAndArrays build() {
try {
TestRecordWithMapsAndArrays record = new TestRecordWithMapsAndArrays();
- record.arr = fieldSetFlags()[0] ? this.arr :
(java.util.List<java.lang.String>) defaultValue(fields()[0]);
- record.map = fieldSetFlags()[1] ? this.map
- : (java.util.Map<java.lang.String, java.lang.Long>)
defaultValue(fields()[1]);
+ record.arr = fieldSetFlags()[0] ? this.arr : (java.util.List<String>)
defaultValue(fields()[0]);
+ record.map = fieldSetFlags()[1] ? this.map : (java.util.Map<String,
Long>) defaultValue(fields()[1]);
+ record.nested_arr = fieldSetFlags()[2] ? this.nested_arr
+ : (java.util.List<java.util.List<String>>)
defaultValue(fields()[2]);
+ record.nested_map = fieldSetFlags()[3] ? this.nested_map
+ : (java.util.Map<String, java.util.Map<String, Long>>)
defaultValue(fields()[3]);
return record;
} catch (org.apache.avro.AvroMissingFieldException e) {
throw e;
- } catch (java.lang.Exception e) {
+ } catch (Exception e) {
throw new org.apache.avro.AvroRuntimeException(e);
}
}
@@ -415,7 +569,7 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
out.writeArrayStart();
out.setItemCount(size0);
long actualSize0 = 0;
- for (java.lang.String e0 : this.arr) {
+ for (String e0 : this.arr) {
actualSize0++;
out.startItem();
out.writeString(e0);
@@ -429,11 +583,11 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
out.writeMapStart();
out.setItemCount(size1);
long actualSize1 = 0;
- for (java.util.Map.Entry<java.lang.String, java.lang.Long> e1 :
this.map.entrySet()) {
+ for (java.util.Map.Entry<String, Long> e1 : this.map.entrySet()) {
actualSize1++;
out.startItem();
out.writeString(e1.getKey());
- java.lang.Long v1 = e1.getValue();
+ Long v1 = e1.getValue();
out.writeLong(v1);
}
out.writeMapEnd();
@@ -441,6 +595,62 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
throw new java.util.ConcurrentModificationException(
"Map-size written was " + size1 + ", but element count was " +
actualSize1 + ".");
+ long size2 = this.nested_arr.size();
+ out.writeArrayStart();
+ out.setItemCount(size2);
+ long actualSize2 = 0;
+ for (java.util.List<String> e2 : this.nested_arr) {
+ actualSize2++;
+ out.startItem();
+ long size3 = e2.size();
+ out.writeArrayStart();
+ out.setItemCount(size3);
+ long actualSize3 = 0;
+ for (String e3 : e2) {
+ actualSize3++;
+ out.startItem();
+ out.writeString(e3);
+ }
+ out.writeArrayEnd();
+ if (actualSize3 != size3)
+ throw new java.util.ConcurrentModificationException(
+ "Array-size written was " + size3 + ", but element count was " +
actualSize3 + ".");
+ }
+ out.writeArrayEnd();
+ if (actualSize2 != size2)
+ throw new java.util.ConcurrentModificationException(
+ "Array-size written was " + size2 + ", but element count was " +
actualSize2 + ".");
+
+ long size4 = this.nested_map.size();
+ out.writeMapStart();
+ out.setItemCount(size4);
+ long actualSize4 = 0;
+ for (java.util.Map.Entry<String, java.util.Map<String, Long>> e4 :
this.nested_map.entrySet()) {
+ actualSize4++;
+ out.startItem();
+ out.writeString(e4.getKey());
+ java.util.Map<String, Long> v4 = e4.getValue();
+ long size5 = v4.size();
+ out.writeMapStart();
+ out.setItemCount(size5);
+ long actualSize5 = 0;
+ for (java.util.Map.Entry<String, Long> e5 : v4.entrySet()) {
+ actualSize5++;
+ out.startItem();
+ out.writeString(e5.getKey());
+ Long v5 = e5.getValue();
+ out.writeLong(v5);
+ }
+ out.writeMapEnd();
+ if (actualSize5 != size5)
+ throw new java.util.ConcurrentModificationException(
+ "Map-size written was " + size5 + ", but element count was " +
actualSize5 + ".");
+ }
+ out.writeMapEnd();
+ if (actualSize4 != size4)
+ throw new java.util.ConcurrentModificationException(
+ "Map-size written was " + size4 + ", but element count was " +
actualSize4 + ".");
+
}
@Override
@@ -448,58 +658,118 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff();
if (fieldOrder == null) {
long size0 = in.readArrayStart();
- java.util.List<java.lang.String> a0 = this.arr;
+ java.util.List<String> a0 = this.arr;
if (a0 == null) {
- a0 = new SpecificData.Array<java.lang.String>((int) size0,
SCHEMA$.getField("arr").schema());
+ a0 = new SpecificData.Array<String>((int) size0,
SCHEMA$.getField("arr").schema());
this.arr = a0;
} else
a0.clear();
- SpecificData.Array<java.lang.String> ga0 = (a0 instanceof
SpecificData.Array
- ? (SpecificData.Array<java.lang.String>) a0
- : null);
+ SpecificData.Array<String> ga0 = (a0 instanceof SpecificData.Array ?
(SpecificData.Array<String>) a0 : null);
for (; 0 < size0; size0 = in.arrayNext()) {
for (; size0 != 0; size0--) {
- java.lang.String e0 = (ga0 != null ? ga0.peek() : null);
+ String e0 = (ga0 != null ? ga0.peek() : null);
e0 = in.readString();
a0.add(e0);
}
}
long size1 = in.readMapStart();
- java.util.Map<java.lang.String, java.lang.Long> m1 = this.map; // Need
fresh name due to limitation of macro
- // system
+ java.util.Map<String, Long> m1 = this.map; // Need fresh name due to
limitation of macro system
if (m1 == null) {
- m1 = new java.util.HashMap<java.lang.String, java.lang.Long>((int)
size1);
+ m1 = new java.util.HashMap<String, Long>((int) size1);
this.map = m1;
} else
m1.clear();
for (; 0 < size1; size1 = in.mapNext()) {
for (; size1 != 0; size1--) {
- java.lang.String k1 = null;
+ String k1 = null;
k1 = in.readString();
- java.lang.Long v1 = null;
+ Long v1 = null;
v1 = in.readLong();
m1.put(k1, v1);
}
}
+ long size2 = in.readArrayStart();
+ java.util.List<java.util.List<String>> a2 = this.nested_arr;
+ if (a2 == null) {
+ a2 = new SpecificData.Array<java.util.List<String>>((int) size2,
SCHEMA$.getField("nested_arr").schema());
+ this.nested_arr = a2;
+ } else
+ a2.clear();
+ SpecificData.Array<java.util.List<String>> ga2 = (a2 instanceof
SpecificData.Array
+ ? (SpecificData.Array<java.util.List<String>>) a2
+ : null);
+ for (; 0 < size2; size2 = in.arrayNext()) {
+ for (; size2 != 0; size2--) {
+ java.util.List<String> e2 = (ga2 != null ? ga2.peek() : null);
+ long size3 = in.readArrayStart();
+ java.util.List<String> a3 = e2;
+ if (a3 == null) {
+ a3 = new SpecificData.Array<String>((int) size3,
SCHEMA$.getField("nested_arr").schema().getElementType());
+ e2 = a3;
+ } else
+ a3.clear();
+ SpecificData.Array<String> ga3 = (a3 instanceof SpecificData.Array ?
(SpecificData.Array<String>) a3 : null);
+ for (; 0 < size3; size3 = in.arrayNext()) {
+ for (; size3 != 0; size3--) {
+ String e3 = (ga3 != null ? ga3.peek() : null);
+ e3 = in.readString();
+ a3.add(e3);
+ }
+ }
+ a2.add(e2);
+ }
+ }
+
+ long size4 = in.readMapStart();
+ java.util.Map<String, java.util.Map<String, Long>> m4 = this.nested_map;
// Need fresh name due to limitation of
+
// macro system
+ if (m4 == null) {
+ m4 = new java.util.HashMap<String, java.util.Map<String, Long>>((int)
size4);
+ this.nested_map = m4;
+ } else
+ m4.clear();
+ for (; 0 < size4; size4 = in.mapNext()) {
+ for (; size4 != 0; size4--) {
+ String k4 = null;
+ k4 = in.readString();
+ java.util.Map<String, Long> v4 = null;
+ long size5 = in.readMapStart();
+ java.util.Map<String, Long> m5 = v4; // Need fresh name due to
limitation of macro system
+ if (m5 == null) {
+ m5 = new java.util.HashMap<String, Long>((int) size5);
+ v4 = m5;
+ } else
+ m5.clear();
+ for (; 0 < size5; size5 = in.mapNext()) {
+ for (; size5 != 0; size5--) {
+ String k5 = null;
+ k5 = in.readString();
+ Long v5 = null;
+ v5 = in.readLong();
+ m5.put(k5, v5);
+ }
+ }
+ m4.put(k4, v4);
+ }
+ }
+
} else {
- for (int i = 0; i < 2; i++) {
+ for (int i = 0; i < 4; i++) {
switch (fieldOrder[i].pos()) {
case 0:
long size0 = in.readArrayStart();
- java.util.List<java.lang.String> a0 = this.arr;
+ java.util.List<String> a0 = this.arr;
if (a0 == null) {
- a0 = new SpecificData.Array<java.lang.String>((int) size0,
SCHEMA$.getField("arr").schema());
+ a0 = new SpecificData.Array<String>((int) size0,
SCHEMA$.getField("arr").schema());
this.arr = a0;
} else
a0.clear();
- SpecificData.Array<java.lang.String> ga0 = (a0 instanceof
SpecificData.Array
- ? (SpecificData.Array<java.lang.String>) a0
- : null);
+ SpecificData.Array<String> ga0 = (a0 instanceof SpecificData.Array ?
(SpecificData.Array<String>) a0 : null);
for (; 0 < size0; size0 = in.arrayNext()) {
for (; size0 != 0; size0--) {
- java.lang.String e0 = (ga0 != null ? ga0.peek() : null);
+ String e0 = (ga0 != null ? ga0.peek() : null);
e0 = in.readString();
a0.add(e0);
}
@@ -508,24 +778,94 @@ public class TestRecordWithMapsAndArrays extends
org.apache.avro.specific.Specif
case 1:
long size1 = in.readMapStart();
- java.util.Map<java.lang.String, java.lang.Long> m1 = this.map; //
Need fresh name due to limitation of macro
- //
system
+ java.util.Map<String, Long> m1 = this.map; // Need fresh name due to
limitation of macro system
if (m1 == null) {
- m1 = new java.util.HashMap<java.lang.String, java.lang.Long>((int)
size1);
+ m1 = new java.util.HashMap<String, Long>((int) size1);
this.map = m1;
} else
m1.clear();
for (; 0 < size1; size1 = in.mapNext()) {
for (; size1 != 0; size1--) {
- java.lang.String k1 = null;
+ String k1 = null;
k1 = in.readString();
- java.lang.Long v1 = null;
+ Long v1 = null;
v1 = in.readLong();
m1.put(k1, v1);
}
}
break;
+ case 2:
+ long size2 = in.readArrayStart();
+ java.util.List<java.util.List<String>> a2 = this.nested_arr;
+ if (a2 == null) {
+ a2 = new SpecificData.Array<java.util.List<String>>((int) size2,
SCHEMA$.getField("nested_arr").schema());
+ this.nested_arr = a2;
+ } else
+ a2.clear();
+ SpecificData.Array<java.util.List<String>> ga2 = (a2 instanceof
SpecificData.Array
+ ? (SpecificData.Array<java.util.List<String>>) a2
+ : null);
+ for (; 0 < size2; size2 = in.arrayNext()) {
+ for (; size2 != 0; size2--) {
+ java.util.List<String> e2 = (ga2 != null ? ga2.peek() : null);
+ long size3 = in.readArrayStart();
+ java.util.List<String> a3 = e2;
+ if (a3 == null) {
+ a3 = new SpecificData.Array<String>((int) size3,
+ SCHEMA$.getField("nested_arr").schema().getElementType());
+ e2 = a3;
+ } else
+ a3.clear();
+ SpecificData.Array<String> ga3 = (a3 instanceof
SpecificData.Array ? (SpecificData.Array<String>) a3
+ : null);
+ for (; 0 < size3; size3 = in.arrayNext()) {
+ for (; size3 != 0; size3--) {
+ String e3 = (ga3 != null ? ga3.peek() : null);
+ e3 = in.readString();
+ a3.add(e3);
+ }
+ }
+ a2.add(e2);
+ }
+ }
+ break;
+
+ case 3:
+ long size4 = in.readMapStart();
+ java.util.Map<String, java.util.Map<String, Long>> m4 =
this.nested_map; // Need fresh name due to limitation
+
// of macro system
+ if (m4 == null) {
+ m4 = new java.util.HashMap<String, java.util.Map<String,
Long>>((int) size4);
+ this.nested_map = m4;
+ } else
+ m4.clear();
+ for (; 0 < size4; size4 = in.mapNext()) {
+ for (; size4 != 0; size4--) {
+ String k4 = null;
+ k4 = in.readString();
+ java.util.Map<String, Long> v4 = null;
+ long size5 = in.readMapStart();
+ java.util.Map<String, Long> m5 = v4; // Need fresh name due to
limitation of macro system
+ if (m5 == null) {
+ m5 = new java.util.HashMap<String, Long>((int) size5);
+ v4 = m5;
+ } else
+ m5.clear();
+ for (; 0 < size5; size5 = in.mapNext()) {
+ for (; size5 != 0; size5--) {
+ String k5 = null;
+ k5 = in.readString();
+ Long v5 = null;
+ v5 = in.readLong();
+ m5.put(k5, v5);
+ }
+ }
+ m4.put(k4, v4);
+ }
+ }
+ break;
+
default:
throw new java.io.IOException("Corrupt ResolvingDecoder.");
}
diff --git a/lang/java/avro/src/test/resources/TestRecordWithMapsAndArrays.avsc
b/lang/java/avro/src/test/resources/TestRecordWithMapsAndArrays.avsc
index e2bc0382b..d19c0d8df 100644
--- a/lang/java/avro/src/test/resources/TestRecordWithMapsAndArrays.avsc
+++ b/lang/java/avro/src/test/resources/TestRecordWithMapsAndArrays.avsc
@@ -18,6 +18,30 @@
"values": "long",
"default": {}
}
+ },
+ {
+ "name": "nested_arr",
+ "type": {
+ "type": "array",
+ "items": {
+ "type": "array",
+ "items": "string",
+ "default": []
+ },
+ "default": []
+ }
+ },
+ {
+ "name": "nested_map",
+ "type": {
+ "type": "map",
+ "values": {
+ "type": "map",
+ "values": "long",
+ "default": {}
+ },
+ "default": {}
+ }
}
]
}