From 2799f83be48ba4cba45a9e7c69b05be5e4212812 Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Mon, 30 Jan 2023 15:55:17 +0100 Subject: [PATCH 1/6] Run Avro extension tests against different Avro versions --- .../job_PostCommit_Java_Avro_Versions.groovy | 47 +++++++++++ build.gradle.kts | 4 + .../sdk/schemas/utils/SchemaTestUtils.java | 2 + sdks/java/extensions/avro/build.gradle | 50 +++++++++++- .../sdk/extensions/avro/coders/AvroCoder.java | 2 +- .../extensions/avro/coders/AvroCoderTest.java | 55 +------------ .../avro/coders/SpecificRecordTest.java | 79 +++++++++++++++++++ .../avro/schemas/AvroSchemaTest.java | 8 +- .../avro/schemas/utils/AvroUtilsTest.java | 33 +++++--- 9 files changed, 212 insertions(+), 68 deletions(-) create mode 100644 .test-infra/jenkins/job_PostCommit_Java_Avro_Versions.groovy create mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/SpecificRecordTest.java diff --git a/.test-infra/jenkins/job_PostCommit_Java_Avro_Versions.groovy b/.test-infra/jenkins/job_PostCommit_Java_Avro_Versions.groovy new file mode 100644 index 000000000000..ea07cb263928 --- /dev/null +++ b/.test-infra/jenkins/job_PostCommit_Java_Avro_Versions.groovy @@ -0,0 +1,47 @@ +/* + * 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. + */ + +import CommonJobProperties as commonJobProperties +import PostcommitJobBuilder + + +// This job runs the Java tests that depends on Avro against different Avro API versions +PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Avro_Versions', 'Run PostCommit_Java_Avro_Versions', + 'Java Avro Versions Post Commit Tests', this) { + + description('Java Avro Versions Post Commit Tests') + + // Set common parameters. + commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 240) + + // Publish all test results to Jenkins + publishers { + archiveJunit('**/build/test-results/**/*.xml') + } + + // Gradle goals for this job. + steps { + gradle { + rootBuildScriptDir(commonJobProperties.checkoutDir) + tasks(":javaAvroVersionsTest") + commonJobProperties.setGradleSwitches(delegate) + // Specify maven home on Jenkins, needed by Maven archetype integration tests. + switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') + } + } + } diff --git a/build.gradle.kts b/build.gradle.kts index 85088a262760..995e55de4543 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -393,6 +393,10 @@ tasks.register("javaHadoopVersionsTest") { dependsOn(":runners:spark:3:hadoopVersionsTest") } +tasks.register("javaAvroVersionsTest") { + dependsOn(":sdks:java:extensions:avro:avroVersionsTest") +} + tasks.register("sqlPostCommit") { dependsOn(":sdks:java:extensions:sql:postCommit") dependsOn(":sdks:java:extensions:sql:jdbc:postCommit") diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SchemaTestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SchemaTestUtils.java index 494c47ee259c..1f039b1989c8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SchemaTestUtils.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SchemaTestUtils.java @@ -126,6 +126,8 @@ private static boolean fieldsEquivalent(Object expected, Object actual, FieldTyp (Map) expected, (Map) actual, fieldType.getMapValueType()); + } else if (fieldType.getTypeName() == TypeName.ROW) { + return rowsEquivalent((Row) expected, (Row) actual); } else { return Objects.equals(expected, actual); } diff --git a/sdks/java/extensions/avro/build.gradle b/sdks/java/extensions/avro/build.gradle index f73a9efccfb5..2b4649164101 100644 --- a/sdks/java/extensions/avro/build.gradle +++ b/sdks/java/extensions/avro/build.gradle @@ -1,3 +1,5 @@ +import java.util.stream.Collectors + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -27,6 +29,15 @@ applyAvroNature() description = "Apache Beam :: SDKs :: Java :: Extensions :: Avro" +def avroVersions = [ + '182': "1.8.2", + '192': "1.9.2", + '1102': "1.10.2", + '1111': "1.11.1", +] + +avroVersions.each{k,v -> configurations.create("avroVersion$k")} + // Exclude tests that need a runner test { systemProperty "beamUseDummyRunner", "true" @@ -53,4 +64,41 @@ dependencies { testImplementation library.java.junit testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") testRuntimeOnly library.java.slf4j_jdk14 -} \ No newline at end of file + avroVersions.each {"avroVersion$it.key" "org.apache.avro:avro:$it.value"} +} + +avroVersions.each { kv -> + configurations."avroVersion$kv.key" { + resolutionStrategy { + force "org.apache.avro:avro:$kv.value" + } + } +} + +avroVersions.each {kv -> + task "avroVersion${kv.key}Test"(type: Test) { + group = "Verification" + description = "Runs Avro extension tests with Avro API $kv.value" + outputs.upToDateWhen { false } + testClassesDirs = sourceSets.test.output.classesDirs + classpath = configurations."avroVersion${kv.key}" + sourceSets.test.runtimeClasspath + include '**/*.class' + exclude '**/AvroIOTest$NeedsRunnerTests$*.class' + if (!kv.value.equals("1.8.2")) { + // Don't include this test for recent Avro versions because of AVRO-2943 + exclude '**/SpecificRecordTest.class' + } + } +} + +task avroVersionsTest { + group = "Verification" + description = 'Runs Avro extension tests with different Avro API versions' + dependsOn createTaskNames(avroVersions, "Test") +} + +static def createTaskNames(Map prefixMap, String suffix) { + return prefixMap.keySet().stream() + .map{version -> "avroVersion${version}${suffix}"} + .collect(Collectors.toList()) +} diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java index 4687eb566424..186a9a76557b 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java @@ -256,7 +256,7 @@ private Object readResolve() throws IOException, ClassNotFoundException { * Serializable}'s usage of the {@link #writeReplace} method. Kryo doesn't utilize Java's * serialization and hence is able to encode the {@link Schema} object directly. */ - private static class SerializableSchemaSupplier implements Serializable, Supplier { + static class SerializableSchemaSupplier implements Serializable, Supplier { // writeReplace makes this object serializable. This is a limitation of FindBugs as discussed // here: // http://stackoverflow.com/questions/26156523/is-writeobject-not-neccesary-using-the-serialization-proxy-pattern diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java index 730ccf60e0b9..7dfc8049eb33 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java @@ -28,11 +28,11 @@ import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; +import com.esotericsoftware.kryo.serializers.JavaSerializer; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; @@ -55,16 +55,12 @@ import org.apache.avro.reflect.Stringable; import org.apache.avro.reflect.Union; import org.apache.avro.specific.SpecificData; -import org.apache.avro.specific.SpecificRecord; import org.apache.avro.util.Utf8; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.extensions.avro.schemas.TestAvro; -import org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested; -import org.apache.beam.sdk.extensions.avro.schemas.TestEnum; -import org.apache.beam.sdk.extensions.avro.schemas.fixed4; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.InterceptingUrlClassLoader; import org.apache.beam.sdk.testing.NeedsRunner; @@ -78,8 +74,6 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -87,7 +81,6 @@ import org.hamcrest.TypeSafeMatcher; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; -import org.joda.time.LocalDate; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -103,23 +96,6 @@ public class AvroCoderTest { new DateTime().withDate(1994, 10, 31).withZone(DateTimeZone.UTC); public static final DateTime DATETIME_B = new DateTime().withDate(1997, 4, 25).withZone(DateTimeZone.UTC); - private static final TestAvroNested AVRO_NESTED_SPECIFIC_RECORD = new TestAvroNested(true, 42); - private static final TestAvro AVRO_SPECIFIC_RECORD = - new TestAvro( - true, - 43, - 44L, - 44.1f, - 44.2d, - "mystring", - ByteBuffer.wrap(new byte[] {1, 2, 3, 4}), - new fixed4(new byte[] {1, 2, 3, 4}), - new LocalDate(1979, 3, 14), - new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4), - TestEnum.abc, - AVRO_NESTED_SPECIFIC_RECORD, - ImmutableList.of(AVRO_NESTED_SPECIFIC_RECORD, AVRO_NESTED_SPECIFIC_RECORD), - ImmutableMap.of("k1", AVRO_NESTED_SPECIFIC_RECORD, "k2", AVRO_NESTED_SPECIFIC_RECORD)); @DefaultCoder(AvroCoder.class) private static class Pojo { @@ -286,6 +262,7 @@ public void testKryoSerialization() throws Exception { // Kryo instantiation Kryo kryo = new Kryo(); kryo.setInstantiatorStrategy(new StdInstantiatorStrategy()); + kryo.addDefaultSerializer(AvroCoder.SerializableSchemaSupplier.class, JavaSerializer.class); // Serialization of object without any memoization ByteArrayOutputStream coderWithoutMemoizationBos = new ByteArrayOutputStream(); @@ -324,28 +301,6 @@ public void testPojoEncoding() throws Exception { CoderProperties.coderDecodeEncodeEqual(coder, value); } - @Test - public void testSpecificRecordEncoding() throws Exception { - AvroCoder coder = - AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), false); - - assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); - CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); - } - - @Test - public void testReflectRecordEncoding() throws Exception { - AvroCoder coder = AvroCoder.of(TestAvro.class, true); - AvroCoder coderWithSchema = - AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), true); - - assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); - assertTrue(SpecificRecord.class.isAssignableFrom(coderWithSchema.getType())); - - CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); - CoderProperties.coderDecodeEncodeEqual(coderWithSchema, AVRO_SPECIFIC_RECORD); - } - @Test public void testDisableReflectionEncoding() { try { @@ -353,10 +308,8 @@ public void testDisableReflectionEncoding() { fail("When userReclectApi is disable, schema should not be generated through reflection"); } catch (AvroRuntimeException e) { String message = - "avro.shaded.com.google.common.util.concurrent.UncheckedExecutionException: " - + "org.apache.avro.AvroRuntimeException: " - + "Not a Specific class: class org.apache.beam.sdk.extensions.avro.coders.AvroCoderTest$Pojo"; - assertEquals(message, e.getMessage()); + "Not a Specific class: class org.apache.beam.sdk.extensions.avro.coders.AvroCoderTest$Pojo"; + assertTrue(e.getMessage().contains(message)); } } diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/SpecificRecordTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/SpecificRecordTest.java new file mode 100644 index 000000000000..8063437b8465 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/SpecificRecordTest.java @@ -0,0 +1,79 @@ +/* + * 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.beam.sdk.extensions.avro.coders; + +import static org.junit.Assert.assertTrue; + +import java.nio.ByteBuffer; +import org.apache.avro.specific.SpecificRecord; +import org.apache.beam.sdk.extensions.avro.schemas.TestAvro; +import org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested; +import org.apache.beam.sdk.extensions.avro.schemas.TestEnum; +import org.apache.beam.sdk.extensions.avro.schemas.fixed4; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.DateTime; +import org.joda.time.LocalDate; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class SpecificRecordTest { + private static final TestAvroNested AVRO_NESTED_SPECIFIC_RECORD = new TestAvroNested(true, 42); + + private static final TestAvro AVRO_SPECIFIC_RECORD = + new TestAvro( + true, + 43, + 44L, + 44.1f, + 44.2d, + "mystring", + ByteBuffer.wrap(new byte[] {1, 2, 3, 4}), + new fixed4(new byte[] {1, 2, 3, 4}), + new LocalDate(1979, 3, 14), + new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4), + TestEnum.abc, + AVRO_NESTED_SPECIFIC_RECORD, + ImmutableList.of(AVRO_NESTED_SPECIFIC_RECORD, AVRO_NESTED_SPECIFIC_RECORD), + ImmutableMap.of("k1", AVRO_NESTED_SPECIFIC_RECORD, "k2", AVRO_NESTED_SPECIFIC_RECORD)); + + @Test + public void testSpecificRecordEncoding() throws Exception { + AvroCoder coder = + AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), false); + + assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); + CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); + } + + @Test + public void testReflectRecordEncoding() throws Exception { + AvroCoder coder = AvroCoder.of(TestAvro.class, true); + AvroCoder coderWithSchema = + AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), true); + + assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); + assertTrue(SpecificRecord.class.isAssignableFrom(coderWithSchema.getType())); + + CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); + CoderProperties.coderDecodeEncodeEqual(coderWithSchema, AVRO_SPECIFIC_RECORD); + } +} diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java index 066739ade69f..f08c51b09711 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.avro.schemas; +import static org.apache.beam.sdk.schemas.utils.SchemaTestUtils.equivalentTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import java.nio.ByteBuffer; @@ -377,7 +379,9 @@ public void testSpecificRecordSchema() { @Test public void testPojoSchema() { - assertEquals(POJO_SCHEMA, new AvroRecordSchema().schemaFor(TypeDescriptor.of(AvroPojo.class))); + assertThat( + new AvroRecordSchema().schemaFor(TypeDescriptor.of(AvroPojo.class)), + equivalentTo(POJO_SCHEMA)); } @Test @@ -449,7 +453,7 @@ public void testRowToGenericRecord() { public void testPojoRecordToRow() { SerializableFunction toRow = new AvroRecordSchema().toRowFunction(TypeDescriptor.of(AvroPojo.class)); - assertEquals(ROW_FOR_POJO, toRow.apply(AVRO_POJO)); + assertThat(toRow.apply(AVRO_POJO), equivalentTo(ROW_FOR_POJO)); } @Test diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java index 4e282fb7094b..ac2ec14a2900 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java @@ -156,7 +156,7 @@ public void testNullableArrayFieldToBeamArrayField() { ReflectData.makeNullable( org.apache.avro.Schema.createArray(org.apache.avro.Schema.create(Type.INT))), "", - null); + (Object) null); Field expectedBeamField = Field.nullable("arrayField", FieldType.array(FieldType.INT32)); @@ -174,7 +174,7 @@ public void testNullableBeamArrayFieldToAvroField() { ReflectData.makeNullable( org.apache.avro.Schema.createArray(org.apache.avro.Schema.create(Type.INT))), "", - null); + (Object) null); org.apache.avro.Schema.Field avroField = AvroUtils.toAvroField(beamField, "ignored"); assertEquals(expectedAvroField, avroField); @@ -184,9 +184,10 @@ private static List getAvroSubSchemaFields() { List fields = Lists.newArrayList(); fields.add( new org.apache.avro.Schema.Field( - "bool", org.apache.avro.Schema.create(Type.BOOLEAN), "", null)); + "bool", org.apache.avro.Schema.create(Type.BOOLEAN), "", (Object) null)); fields.add( - new org.apache.avro.Schema.Field("int", org.apache.avro.Schema.create(Type.INT), "", null)); + new org.apache.avro.Schema.Field( + "int", org.apache.avro.Schema.create(Type.INT), "", (Object) null)); return fields; } @@ -402,21 +403,24 @@ public void testNullableFieldInAvroSchema() { List fields = Lists.newArrayList(); fields.add( new org.apache.avro.Schema.Field( - "int", ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT)), "", null)); + "int", + ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT)), + "", + (Object) null)); fields.add( new org.apache.avro.Schema.Field( "array", org.apache.avro.Schema.createArray( ReflectData.makeNullable(org.apache.avro.Schema.create(Type.BYTES))), "", - null)); + (Object) null)); fields.add( new org.apache.avro.Schema.Field( "map", org.apache.avro.Schema.createMap( ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT))), "", - null)); + (Object) null)); fields.add( new org.apache.avro.Schema.Field( "enum", @@ -424,7 +428,7 @@ public void testNullableFieldInAvroSchema() { org.apache.avro.Schema.createEnum( "fruit", "", "", ImmutableList.of("banana", "apple", "pear"))), "", - null)); + (Object) null)); org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord("topLevelRecord", null, null, false, fields); @@ -472,21 +476,24 @@ public void testNullableFieldsInBeamSchema() { List fields = Lists.newArrayList(); fields.add( new org.apache.avro.Schema.Field( - "int", ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT)), "", null)); + "int", + ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT)), + "", + (Object) null)); fields.add( new org.apache.avro.Schema.Field( "array", org.apache.avro.Schema.createArray( ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT))), "", - null)); + (Object) null)); fields.add( new org.apache.avro.Schema.Field( "map", org.apache.avro.Schema.createMap( ReflectData.makeNullable(org.apache.avro.Schema.create(Type.INT))), "", - null)); + (Object) null)); org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord("topLevelRecord", null, null, false, fields); assertEquals(avroSchema, AvroUtils.toAvroSchema(beamSchema)); @@ -522,7 +529,7 @@ public void testUnionFieldInAvroSchema() { fields.add( new org.apache.avro.Schema.Field( - "union", org.apache.avro.Schema.createUnion(unionFields), "", null)); + "union", org.apache.avro.Schema.createUnion(unionFields), "", (Object) null)); org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord("topLevelRecord", null, null, false, fields); OneOfType oneOfType = @@ -549,7 +556,7 @@ public void testUnionFieldInBeamSchema() { unionFields.add(org.apache.avro.Schema.create(Type.STRING)); fields.add( new org.apache.avro.Schema.Field( - "union", org.apache.avro.Schema.createUnion(unionFields), "", null)); + "union", org.apache.avro.Schema.createUnion(unionFields), "", (Object) null)); org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord("topLevelRecord", null, null, false, fields); GenericRecord expectedGenericRecord = From d5ac9a7189935896f5258d51f9bc77d94648f89f Mon Sep 17 00:00:00 2001 From: Moritz Mack Date: Tue, 21 Mar 2023 15:56:31 +0100 Subject: [PATCH 2/6] Use separate test sourcesets per avro version for generated Avro classes --- sdks/java/extensions/avro/build.gradle | 28 +++++++++++++++++--------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/sdks/java/extensions/avro/build.gradle b/sdks/java/extensions/avro/build.gradle index 2b4649164101..108c11214d8a 100644 --- a/sdks/java/extensions/avro/build.gradle +++ b/sdks/java/extensions/avro/build.gradle @@ -30,7 +30,6 @@ applyAvroNature() description = "Apache Beam :: SDKs :: Java :: Extensions :: Avro" def avroVersions = [ - '182': "1.8.2", '192': "1.9.2", '1102': "1.10.2", '1111': "1.11.1", @@ -67,27 +66,36 @@ dependencies { avroVersions.each {"avroVersion$it.key" "org.apache.avro:avro:$it.value"} } -avroVersions.each { kv -> +avroVersions.each {kv -> configurations."avroVersion$kv.key" { resolutionStrategy { force "org.apache.avro:avro:$kv.value" } } -} -avroVersions.each {kv -> + sourceSets { + "avro${kv.key}" { + java.srcDirs "src/test/avro${kv.key}" + compileClasspath = configurations."avroVersion$kv.key" + sourceSets.test.output + sourceSets.test.compileClasspath + runtimeClasspath += compileClasspath + sourceSets.test.runtimeClasspath + } + } + + "compileAvro${kv.key}Java" { + options.warnings = false + checkerFramework { + skipCheckerFramework = true + } + } + task "avroVersion${kv.key}Test"(type: Test) { group = "Verification" description = "Runs Avro extension tests with Avro API $kv.value" outputs.upToDateWhen { false } - testClassesDirs = sourceSets.test.output.classesDirs - classpath = configurations."avroVersion${kv.key}" + sourceSets.test.runtimeClasspath + classpath = sourceSets."avro${kv.key}".runtimeClasspath + include '**/*.class' exclude '**/AvroIOTest$NeedsRunnerTests$*.class' - if (!kv.value.equals("1.8.2")) { - // Don't include this test for recent Avro versions because of AVRO-2943 - exclude '**/SpecificRecordTest.class' - } } } From db2d2d68e59f52adfabad24eead7431f7902f2ec Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Wed, 22 Mar 2023 16:58:04 +0100 Subject: [PATCH 3/6] Added avro-generated POJOs for different Avro versions --- .../sdk/extensions/avro/schemas/TestAvro.java | 1342 ++++++++++++++++ .../avro/schemas/TestAvroNested.java | 416 +++++ .../sdk/extensions/avro/schemas/TestEnum.java | 25 + .../sdk/extensions/avro/schemas/fixed4.java | 56 + .../sdk/extensions/avro/schemas/TestAvro.java | 1350 +++++++++++++++++ .../avro/schemas/TestAvroNested.java | 424 ++++++ .../sdk/extensions/avro/schemas/TestEnum.java | 27 + .../sdk/extensions/avro/schemas/fixed4.java | 56 + .../sdk/extensions/avro/schemas/TestAvro.java | 1341 ++++++++++++++++ .../avro/schemas/TestAvroNested.java | 416 +++++ .../sdk/extensions/avro/schemas/TestEnum.java | 25 + .../sdk/extensions/avro/schemas/fixed4.java | 56 + 12 files changed, 5534 insertions(+) create mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java create mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java create mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java create mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java create mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java create mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java create mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java create mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java create mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java create mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java create mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java create mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java new file mode 100644 index 000000000000..519ebd573979 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java @@ -0,0 +1,1342 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; + +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.SchemaStore; +import org.apache.avro.specific.SpecificData; + +import java.time.Instant; +import java.time.LocalDate; +import java.util.Map; + +@org.apache.avro.specific.AvroGenerated +public class TestAvro extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 27902431178981259L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvro\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"bool_non_nullable\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]},{\"name\":\"long\",\"type\":[\"long\",\"null\"]},{\"name\":\"float\",\"type\":[\"float\",\"null\"]},{\"name\":\"double\",\"type\":[\"double\",\"null\"]},{\"name\":\"string\",\"type\":[\"string\",\"null\"]},{\"name\":\"bytes\",\"type\":[\"bytes\",\"null\"]},{\"name\":\"fixed\",\"type\":{\"type\":\"fixed\",\"name\":\"fixed4\",\"size\":4}},{\"name\":\"date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},{\"name\":\"timestampMillis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}},{\"name\":\"TestEnum\",\"type\":{\"type\":\"enum\",\"name\":\"TestEnum\",\"symbols\":[\"abc\",\"cde\"]}},{\"name\":\"row\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"TestAvroNested\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}]},{\"name\":\"array\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\",\"TestAvroNested\"]}]},{\"name\":\"map\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"TestAvroNested\"]}]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static SpecificData MODEL$ = new SpecificData(); +static { + MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.DateConversion()); + MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.TimestampMillisConversion()); + } + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageEncoder instance used by this class. + * @return the message encoder used by this class + */ + public static BinaryMessageEncoder getEncoder() { + return ENCODER; + } + + /** + * Return the BinaryMessageDecoder instance used by this class. + * @return the message decoder used by this class + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * 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 + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); + } + + /** + * Serializes this TestAvro to a ByteBuffer. + * @return a buffer holding the serialized data for this instance + * @throws java.io.IOException if this instance could not be serialized + */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** + * Deserializes a TestAvro from a ByteBuffer. + * @param b a byte buffer holding serialized data for an instance of this class + * @return a TestAvro instance decoded from the given buffer + * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class + */ + public static TestAvro fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + private boolean bool_non_nullable; + private Integer int$; + private Long long$; + private Float float$; + private Double double$; + private CharSequence string; + private java.nio.ByteBuffer bytes; + private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; + private java.time.LocalDate date; + private java.time.Instant timestampMillis; + private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; + private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; + private java.util.List array; + private java.util.Map map; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public TestAvro() {} + + /** + * All-args constructor. + * @param bool_non_nullable The new value for bool_non_nullable + * @param int$ The new value for int + * @param long$ The new value for long + * @param float$ The new value for float + * @param double$ The new value for double + * @param string The new value for string + * @param bytes The new value for bytes + * @param fixed The new value for fixed + * @param date The new value for date + * @param timestampMillis The new value for timestampMillis + * @param TestEnum The new value for TestEnum + * @param row The new value for row + * @param array The new value for array + * @param map The new value for map + */ + public TestAvro(Boolean bool_non_nullable, Integer int$, Long long$, Float float$, Double double$, CharSequence string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { + this.bool_non_nullable = bool_non_nullable; + this.int$ = int$; + this.long$ = long$; + this.float$ = float$; + this.double$ = double$; + this.string = string; + this.bytes = bytes; + this.fixed = fixed; + this.date = date; + this.timestampMillis = timestampMillis.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); + this.TestEnum = TestEnum; + this.row = row; + this.array = array; + this.map = map; + } + + /** + * Manually added a ompatible with Avro v1.8.2 API constructor + * + * @param bool_non_nullable + * @param int$ + * @param long$ + * @param float$ + * @param double$ + * @param string + * @param bytes + * @param fixed + * @param date + * @param timestampMillis + * @param TestEnum + * @param row + * @param array + * @param map + */ + public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.String string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, org.joda.time.LocalDate date, org.joda.time.DateTime timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { + this.bool_non_nullable = bool_non_nullable; + this.int$ = int$; + this.long$ = long$; + this.float$ = float$; + this.double$ = double$; + this.string = string; + this.bytes = bytes; + this.fixed = fixed; + this.date = LocalDate.of(date.getYear(), date.getMonthOfYear(), date.getDayOfMonth()); + this.timestampMillis = Instant.ofEpochMilli(timestampMillis.getMillis()); + this.TestEnum = TestEnum; + this.row = row; + this.array = array; + this.map = (Map)map; + } + + public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public Object get(int field$) { + switch (field$) { + case 0: return bool_non_nullable; + case 1: return int$; + case 2: return long$; + case 3: return float$; + case 4: return double$; + case 5: return string; + case 6: return bytes; + case 7: return fixed; + case 8: return date; + case 9: return timestampMillis; + case 10: return TestEnum; + case 11: return row; + case 12: return array; + case 13: return map; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + private static final org.apache.avro.Conversion[] conversions = + new org.apache.avro.Conversion[] { + null, + null, + null, + null, + null, + null, + null, + null, + new org.apache.avro.data.TimeConversions.DateConversion(), + new org.apache.avro.data.TimeConversions.TimestampMillisConversion(), + null, + null, + null, + null, + null + }; + + @Override + public org.apache.avro.Conversion getConversion(int field) { + return conversions[field]; + } + + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, Object value$) { + switch (field$) { + case 0: bool_non_nullable = (Boolean)value$; break; + case 1: int$ = (Integer)value$; break; + case 2: long$ = (Long)value$; break; + case 3: float$ = (Float)value$; break; + case 4: double$ = (Double)value$; break; + case 5: string = (CharSequence)value$; break; + case 6: bytes = (java.nio.ByteBuffer)value$; break; + case 7: fixed = (org.apache.beam.sdk.extensions.avro.schemas.fixed4)value$; break; + case 8: date = (java.time.LocalDate)value$; break; + case 9: timestampMillis = (java.time.Instant)value$; break; + case 10: TestEnum = (org.apache.beam.sdk.extensions.avro.schemas.TestEnum)value$; break; + case 11: row = (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested)value$; break; + case 12: array = (java.util.List)value$; break; + case 13: map = (java.util.Map)value$; break; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + /** + * Gets the value of the 'bool_non_nullable' field. + * @return The value of the 'bool_non_nullable' field. + */ + public boolean getBoolNonNullable() { + return bool_non_nullable; + } + + + /** + * Sets the value of the 'bool_non_nullable' field. + * @param value the value to set. + */ + public void setBoolNonNullable(boolean value) { + this.bool_non_nullable = value; + } + + /** + * Gets the value of the 'int$' field. + * @return The value of the 'int$' field. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value the value to set. + */ + public void setInt$(Integer value) { + this.int$ = value; + } + + /** + * Gets the value of the 'long$' field. + * @return The value of the 'long$' field. + */ + public Long getLong$() { + return long$; + } + + + /** + * Sets the value of the 'long$' field. + * @param value the value to set. + */ + public void setLong$(Long value) { + this.long$ = value; + } + + /** + * Gets the value of the 'float$' field. + * @return The value of the 'float$' field. + */ + public Float getFloat$() { + return float$; + } + + + /** + * Sets the value of the 'float$' field. + * @param value the value to set. + */ + public void setFloat$(Float value) { + this.float$ = value; + } + + /** + * Gets the value of the 'double$' field. + * @return The value of the 'double$' field. + */ + public Double getDouble$() { + return double$; + } + + + /** + * Sets the value of the 'double$' field. + * @param value the value to set. + */ + public void setDouble$(Double value) { + this.double$ = value; + } + + /** + * Gets the value of the 'string' field. + * @return The value of the 'string' field. + */ + public CharSequence getString() { + return string; + } + + + /** + * Sets the value of the 'string' field. + * @param value the value to set. + */ + public void setString(CharSequence value) { + this.string = value; + } + + /** + * Gets the value of the 'bytes' field. + * @return The value of the 'bytes' field. + */ + public java.nio.ByteBuffer getBytes() { + return bytes; + } + + + /** + * Sets the value of the 'bytes' field. + * @param value the value to set. + */ + public void setBytes(java.nio.ByteBuffer value) { + this.bytes = value; + } + + /** + * Gets the value of the 'fixed' field. + * @return The value of the 'fixed' field. + */ + public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { + return fixed; + } + + + /** + * Sets the value of the 'fixed' field. + * @param value the value to set. + */ + public void setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { + this.fixed = value; + } + + /** + * Gets the value of the 'date' field. + * @return The value of the 'date' field. + */ + public java.time.LocalDate getDate() { + return date; + } + + + /** + * Sets the value of the 'date' field. + * @param value the value to set. + */ + public void setDate(java.time.LocalDate value) { + this.date = value; + } + + /** + * Gets the value of the 'timestampMillis' field. + * @return The value of the 'timestampMillis' field. + */ + public java.time.Instant getTimestampMillis() { + return timestampMillis; + } + + + /** + * Sets the value of the 'timestampMillis' field. + * @param value the value to set. + */ + public void setTimestampMillis(java.time.Instant value) { + this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); + } + + /** + * Gets the value of the 'TestEnum' field. + * @return The value of the 'TestEnum' field. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { + return TestEnum; + } + + + /** + * Sets the value of the 'TestEnum' field. + * @param value the value to set. + */ + public void setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { + this.TestEnum = value; + } + + /** + * Gets the value of the 'row' field. + * @return The value of the 'row' field. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { + return row; + } + + + /** + * Sets the value of the 'row' field. + * @param value the value to set. + */ + public void setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { + this.row = value; + } + + /** + * Gets the value of the 'array' field. + * @return The value of the 'array' field. + */ + public java.util.List getArray() { + return array; + } + + + /** + * Sets the value of the 'array' field. + * @param value the value to set. + */ + public void setArray(java.util.List value) { + this.array = value; + } + + /** + * Gets the value of the 'map' field. + * @return The value of the 'map' field. + */ + public java.util.Map getMap() { + return map; + } + + + /** + * Sets the value of the 'map' field. + * @param value the value to set. + */ + public void setMap(java.util.Map value) { + this.map = value; + } + + /** + * Creates a new TestAvro RecordBuilder. + * @return A new TestAvro RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder() { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); + } + + /** + * Creates a new TestAvro RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new TestAvro RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); + } + } + + /** + * Creates a new TestAvro RecordBuilder by copying an existing TestAvro instance. + * @param other The existing instance to copy. + * @return A new TestAvro RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); + } + } + + /** + * RecordBuilder for TestAvro instances. + */ + @org.apache.avro.specific.AvroGenerated + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean bool_non_nullable; + private Integer int$; + private Long long$; + private Float float$; + private Double double$; + private CharSequence string; + private java.nio.ByteBuffer bytes; + private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; + private java.time.LocalDate date; + private java.time.Instant timestampMillis; + private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; + private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; + private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder rowBuilder; + private java.util.List array; + private java.util.Map map; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { + super(other); + if (isValidValue(fields()[0], other.bool_non_nullable)) { + this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); + fieldSetFlags()[0] = other.fieldSetFlags()[0]; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = other.fieldSetFlags()[1]; + } + if (isValidValue(fields()[2], other.long$)) { + this.long$ = data().deepCopy(fields()[2].schema(), other.long$); + fieldSetFlags()[2] = other.fieldSetFlags()[2]; + } + if (isValidValue(fields()[3], other.float$)) { + this.float$ = data().deepCopy(fields()[3].schema(), other.float$); + fieldSetFlags()[3] = other.fieldSetFlags()[3]; + } + if (isValidValue(fields()[4], other.double$)) { + this.double$ = data().deepCopy(fields()[4].schema(), other.double$); + fieldSetFlags()[4] = other.fieldSetFlags()[4]; + } + if (isValidValue(fields()[5], other.string)) { + this.string = data().deepCopy(fields()[5].schema(), other.string); + fieldSetFlags()[5] = other.fieldSetFlags()[5]; + } + if (isValidValue(fields()[6], other.bytes)) { + this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); + fieldSetFlags()[6] = other.fieldSetFlags()[6]; + } + if (isValidValue(fields()[7], other.fixed)) { + this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); + fieldSetFlags()[7] = other.fieldSetFlags()[7]; + } + if (isValidValue(fields()[8], other.date)) { + this.date = data().deepCopy(fields()[8].schema(), other.date); + fieldSetFlags()[8] = other.fieldSetFlags()[8]; + } + if (isValidValue(fields()[9], other.timestampMillis)) { + this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); + fieldSetFlags()[9] = other.fieldSetFlags()[9]; + } + if (isValidValue(fields()[10], other.TestEnum)) { + this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); + fieldSetFlags()[10] = other.fieldSetFlags()[10]; + } + if (isValidValue(fields()[11], other.row)) { + this.row = data().deepCopy(fields()[11].schema(), other.row); + fieldSetFlags()[11] = other.fieldSetFlags()[11]; + } + if (other.hasRowBuilder()) { + this.rowBuilder = org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(other.getRowBuilder()); + } + if (isValidValue(fields()[12], other.array)) { + this.array = data().deepCopy(fields()[12].schema(), other.array); + fieldSetFlags()[12] = other.fieldSetFlags()[12]; + } + if (isValidValue(fields()[13], other.map)) { + this.map = data().deepCopy(fields()[13].schema(), other.map); + fieldSetFlags()[13] = other.fieldSetFlags()[13]; + } + } + + /** + * Creates a Builder by copying an existing TestAvro instance + * @param other The existing instance to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { + super(SCHEMA$); + if (isValidValue(fields()[0], other.bool_non_nullable)) { + this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.long$)) { + this.long$ = data().deepCopy(fields()[2].schema(), other.long$); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.float$)) { + this.float$ = data().deepCopy(fields()[3].schema(), other.float$); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.double$)) { + this.double$ = data().deepCopy(fields()[4].schema(), other.double$); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.string)) { + this.string = data().deepCopy(fields()[5].schema(), other.string); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.bytes)) { + this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); + fieldSetFlags()[6] = true; + } + if (isValidValue(fields()[7], other.fixed)) { + this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); + fieldSetFlags()[7] = true; + } + if (isValidValue(fields()[8], other.date)) { + this.date = data().deepCopy(fields()[8].schema(), other.date); + fieldSetFlags()[8] = true; + } + if (isValidValue(fields()[9], other.timestampMillis)) { + this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); + fieldSetFlags()[9] = true; + } + if (isValidValue(fields()[10], other.TestEnum)) { + this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); + fieldSetFlags()[10] = true; + } + if (isValidValue(fields()[11], other.row)) { + this.row = data().deepCopy(fields()[11].schema(), other.row); + fieldSetFlags()[11] = true; + } + this.rowBuilder = null; + if (isValidValue(fields()[12], other.array)) { + this.array = data().deepCopy(fields()[12].schema(), other.array); + fieldSetFlags()[12] = true; + } + if (isValidValue(fields()[13], other.map)) { + this.map = data().deepCopy(fields()[13].schema(), other.map); + fieldSetFlags()[13] = true; + } + } + + /** + * Gets the value of the 'bool_non_nullable' field. + * @return The value. + */ + public boolean getBoolNonNullable() { + return bool_non_nullable; + } + + + /** + * Sets the value of the 'bool_non_nullable' field. + * @param value The value of 'bool_non_nullable'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBoolNonNullable(boolean value) { + validate(fields()[0], value); + this.bool_non_nullable = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'bool_non_nullable' field has been set. + * @return True if the 'bool_non_nullable' field has been set, false otherwise. + */ + public boolean hasBoolNonNullable() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'bool_non_nullable' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNonNullable() { + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'int$' field. + * @return The value. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value The value of 'int$'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setInt$(Integer value) { + validate(fields()[1], value); + this.int$ = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'int$' field has been set. + * @return True if the 'int$' field has been set, false otherwise. + */ + public boolean hasInt$() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'int$' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearInt$() { + int$ = null; + fieldSetFlags()[1] = false; + return this; + } + + /** + * Gets the value of the 'long$' field. + * @return The value. + */ + public Long getLong$() { + return long$; + } + + + /** + * Sets the value of the 'long$' field. + * @param value The value of 'long$'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setLong$(Long value) { + validate(fields()[2], value); + this.long$ = value; + fieldSetFlags()[2] = true; + return this; + } + + /** + * Checks whether the 'long$' field has been set. + * @return True if the 'long$' field has been set, false otherwise. + */ + public boolean hasLong$() { + return fieldSetFlags()[2]; + } + + + /** + * Clears the value of the 'long$' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearLong$() { + long$ = null; + fieldSetFlags()[2] = false; + return this; + } + + /** + * Gets the value of the 'float$' field. + * @return The value. + */ + public Float getFloat$() { + return float$; + } + + + /** + * Sets the value of the 'float$' field. + * @param value The value of 'float$'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFloat$(Float value) { + validate(fields()[3], value); + this.float$ = value; + fieldSetFlags()[3] = true; + return this; + } + + /** + * Checks whether the 'float$' field has been set. + * @return True if the 'float$' field has been set, false otherwise. + */ + public boolean hasFloat$() { + return fieldSetFlags()[3]; + } + + + /** + * Clears the value of the 'float$' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFloat$() { + float$ = null; + fieldSetFlags()[3] = false; + return this; + } + + /** + * Gets the value of the 'double$' field. + * @return The value. + */ + public Double getDouble$() { + return double$; + } + + + /** + * Sets the value of the 'double$' field. + * @param value The value of 'double$'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDouble$(Double value) { + validate(fields()[4], value); + this.double$ = value; + fieldSetFlags()[4] = true; + return this; + } + + /** + * Checks whether the 'double$' field has been set. + * @return True if the 'double$' field has been set, false otherwise. + */ + public boolean hasDouble$() { + return fieldSetFlags()[4]; + } + + + /** + * Clears the value of the 'double$' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDouble$() { + double$ = null; + fieldSetFlags()[4] = false; + return this; + } + + /** + * Gets the value of the 'string' field. + * @return The value. + */ + public CharSequence getString() { + return string; + } + + + /** + * Sets the value of the 'string' field. + * @param value The value of 'string'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setString(CharSequence value) { + validate(fields()[5], value); + this.string = value; + fieldSetFlags()[5] = true; + return this; + } + + /** + * Checks whether the 'string' field has been set. + * @return True if the 'string' field has been set, false otherwise. + */ + public boolean hasString() { + return fieldSetFlags()[5]; + } + + + /** + * Clears the value of the 'string' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearString() { + string = null; + fieldSetFlags()[5] = false; + return this; + } + + /** + * Gets the value of the 'bytes' field. + * @return The value. + */ + public java.nio.ByteBuffer getBytes() { + return bytes; + } + + + /** + * Sets the value of the 'bytes' field. + * @param value The value of 'bytes'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBytes(java.nio.ByteBuffer value) { + validate(fields()[6], value); + this.bytes = value; + fieldSetFlags()[6] = true; + return this; + } + + /** + * Checks whether the 'bytes' field has been set. + * @return True if the 'bytes' field has been set, false otherwise. + */ + public boolean hasBytes() { + return fieldSetFlags()[6]; + } + + + /** + * Clears the value of the 'bytes' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBytes() { + bytes = null; + fieldSetFlags()[6] = false; + return this; + } + + /** + * Gets the value of the 'fixed' field. + * @return The value. + */ + public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { + return fixed; + } + + + /** + * Sets the value of the 'fixed' field. + * @param value The value of 'fixed'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { + validate(fields()[7], value); + this.fixed = value; + fieldSetFlags()[7] = true; + return this; + } + + /** + * Checks whether the 'fixed' field has been set. + * @return True if the 'fixed' field has been set, false otherwise. + */ + public boolean hasFixed() { + return fieldSetFlags()[7]; + } + + + /** + * Clears the value of the 'fixed' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFixed() { + fixed = null; + fieldSetFlags()[7] = false; + return this; + } + + /** + * Gets the value of the 'date' field. + * @return The value. + */ + public java.time.LocalDate getDate() { + return date; + } + + + /** + * Sets the value of the 'date' field. + * @param value The value of 'date'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDate(java.time.LocalDate value) { + validate(fields()[8], value); + this.date = value; + fieldSetFlags()[8] = true; + return this; + } + + /** + * Checks whether the 'date' field has been set. + * @return True if the 'date' field has been set, false otherwise. + */ + public boolean hasDate() { + return fieldSetFlags()[8]; + } + + + /** + * Clears the value of the 'date' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDate() { + fieldSetFlags()[8] = false; + return this; + } + + /** + * Gets the value of the 'timestampMillis' field. + * @return The value. + */ + public java.time.Instant getTimestampMillis() { + return timestampMillis; + } + + + /** + * Sets the value of the 'timestampMillis' field. + * @param value The value of 'timestampMillis'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTimestampMillis(java.time.Instant value) { + validate(fields()[9], value); + this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); + fieldSetFlags()[9] = true; + return this; + } + + /** + * Checks whether the 'timestampMillis' field has been set. + * @return True if the 'timestampMillis' field has been set, false otherwise. + */ + public boolean hasTimestampMillis() { + return fieldSetFlags()[9]; + } + + + /** + * Clears the value of the 'timestampMillis' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTimestampMillis() { + fieldSetFlags()[9] = false; + return this; + } + + /** + * Gets the value of the 'TestEnum' field. + * @return The value. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { + return TestEnum; + } + + + /** + * Sets the value of the 'TestEnum' field. + * @param value The value of 'TestEnum'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { + validate(fields()[10], value); + this.TestEnum = value; + fieldSetFlags()[10] = true; + return this; + } + + /** + * Checks whether the 'TestEnum' field has been set. + * @return True if the 'TestEnum' field has been set, false otherwise. + */ + public boolean hasTestEnum() { + return fieldSetFlags()[10]; + } + + + /** + * Clears the value of the 'TestEnum' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTestEnum() { + TestEnum = null; + fieldSetFlags()[10] = false; + return this; + } + + /** + * Gets the value of the 'row' field. + * @return The value. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { + return row; + } + + + /** + * Sets the value of the 'row' field. + * @param value The value of 'row'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { + validate(fields()[11], value); + this.rowBuilder = null; + this.row = value; + fieldSetFlags()[11] = true; + return this; + } + + /** + * Checks whether the 'row' field has been set. + * @return True if the 'row' field has been set, false otherwise. + */ + public boolean hasRow() { + return fieldSetFlags()[11]; + } + + /** + * Gets the Builder instance for the 'row' field and creates one if it doesn't exist yet. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder getRowBuilder() { + if (rowBuilder == null) { + if (hasRow()) { + setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(row)); + } else { + setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder()); + } + } + return rowBuilder; + } + + /** + * Sets the Builder instance for the 'row' field + * @param value The builder instance that must be set. + * @return This builder. + */ + + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder value) { + clearRow(); + rowBuilder = value; + return this; + } + + /** + * Checks whether the 'row' field has an active Builder instance + * @return True if the 'row' field has an active Builder instance + */ + public boolean hasRowBuilder() { + return rowBuilder != null; + } + + /** + * Clears the value of the 'row' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearRow() { + row = null; + rowBuilder = null; + fieldSetFlags()[11] = false; + return this; + } + + /** + * Gets the value of the 'array' field. + * @return The value. + */ + public java.util.List getArray() { + return array; + } + + + /** + * Sets the value of the 'array' field. + * @param value The value of 'array'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setArray(java.util.List value) { + validate(fields()[12], value); + this.array = value; + fieldSetFlags()[12] = true; + return this; + } + + /** + * Checks whether the 'array' field has been set. + * @return True if the 'array' field has been set, false otherwise. + */ + public boolean hasArray() { + return fieldSetFlags()[12]; + } + + + /** + * Clears the value of the 'array' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearArray() { + array = null; + fieldSetFlags()[12] = false; + return this; + } + + /** + * Gets the value of the 'map' field. + * @return The value. + */ + public java.util.Map getMap() { + return map; + } + + + /** + * Sets the value of the 'map' field. + * @param value The value of 'map'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setMap(java.util.Map value) { + validate(fields()[13], value); + this.map = value; + fieldSetFlags()[13] = true; + return this; + } + + /** + * Checks whether the 'map' field has been set. + * @return True if the 'map' field has been set, false otherwise. + */ + public boolean hasMap() { + return fieldSetFlags()[13]; + } + + + /** + * Clears the value of the 'map' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearMap() { + map = null; + fieldSetFlags()[13] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public TestAvro build() { + try { + TestAvro record = new TestAvro(); + record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); + record.long$ = fieldSetFlags()[2] ? this.long$ : (Long) defaultValue(fields()[2]); + record.float$ = fieldSetFlags()[3] ? this.float$ : (Float) defaultValue(fields()[3]); + record.double$ = fieldSetFlags()[4] ? this.double$ : (Double) defaultValue(fields()[4]); + record.string = fieldSetFlags()[5] ? this.string : (CharSequence) defaultValue(fields()[5]); + record.bytes = fieldSetFlags()[6] ? this.bytes : (java.nio.ByteBuffer) defaultValue(fields()[6]); + record.fixed = fieldSetFlags()[7] ? this.fixed : (org.apache.beam.sdk.extensions.avro.schemas.fixed4) defaultValue(fields()[7]); + record.date = fieldSetFlags()[8] ? this.date : (java.time.LocalDate) defaultValue(fields()[8]); + record.timestampMillis = fieldSetFlags()[9] ? this.timestampMillis : (java.time.Instant) defaultValue(fields()[9]); + record.TestEnum = fieldSetFlags()[10] ? this.TestEnum : (org.apache.beam.sdk.extensions.avro.schemas.TestEnum) defaultValue(fields()[10]); + if (rowBuilder != null) { + try { + record.row = this.rowBuilder.build(); + } catch (org.apache.avro.AvroMissingFieldException e) { + e.addParentField(record.getSchema().getField("row")); + throw e; + } + } else { + record.row = fieldSetFlags()[11] ? this.row : (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested) defaultValue(fields()[11]); + } + record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); + record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); + return record; + } catch (org.apache.avro.AvroMissingFieldException e) { + throw e; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + +} + + + + + + + + + + diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java new file mode 100644 index 000000000000..cf1c2d7af3f0 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java @@ -0,0 +1,416 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; + +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.SchemaStore; +import org.apache.avro.specific.SpecificData; + +@org.apache.avro.specific.AvroGenerated +public class TestAvroNested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 4633138088036298925L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvroNested\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static SpecificData MODEL$ = new SpecificData(); + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageEncoder instance used by this class. + * @return the message encoder used by this class + */ + public static BinaryMessageEncoder getEncoder() { + return ENCODER; + } + + /** + * Return the BinaryMessageDecoder instance used by this class. + * @return the message decoder used by this class + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * 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 + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); + } + + /** + * Serializes this TestAvroNested to a ByteBuffer. + * @return a buffer holding the serialized data for this instance + * @throws java.io.IOException if this instance could not be serialized + */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** + * Deserializes a TestAvroNested from a ByteBuffer. + * @param b a byte buffer holding serialized data for an instance of this class + * @return a TestAvroNested instance decoded from the given buffer + * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class + */ + public static TestAvroNested fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + private boolean BOOL_NON_NULLABLE; + private Integer int$; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public TestAvroNested() {} + + /** + * All-args constructor. + * @param BOOL_NON_NULLABLE The new value for BOOL_NON_NULLABLE + * @param int$ The new value for int + */ + public TestAvroNested(Boolean BOOL_NON_NULLABLE, Integer int$) { + this.BOOL_NON_NULLABLE = BOOL_NON_NULLABLE; + this.int$ = int$; + } + + public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public Object get(int field$) { + switch (field$) { + case 0: return BOOL_NON_NULLABLE; + case 1: return int$; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, Object value$) { + switch (field$) { + case 0: BOOL_NON_NULLABLE = (Boolean)value$; break; + case 1: int$ = (Integer)value$; break; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + /** + * Gets the value of the 'BOOL_NON_NULLABLE' field. + * @return The value of the 'BOOL_NON_NULLABLE' field. + */ + public boolean getBOOLNONNULLABLE() { + return BOOL_NON_NULLABLE; + } + + + /** + * Sets the value of the 'BOOL_NON_NULLABLE' field. + * @param value the value to set. + */ + public void setBOOLNONNULLABLE(boolean value) { + this.BOOL_NON_NULLABLE = value; + } + + /** + * Gets the value of the 'int$' field. + * @return The value of the 'int$' field. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value the value to set. + */ + public void setInt$(Integer value) { + this.int$ = value; + } + + /** + * Creates a new TestAvroNested RecordBuilder. + * @return A new TestAvroNested RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder() { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); + } + + /** + * Creates a new TestAvroNested RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new TestAvroNested RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); + } + } + + /** + * Creates a new TestAvroNested RecordBuilder by copying an existing TestAvroNested instance. + * @param other The existing instance to copy. + * @return A new TestAvroNested RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); + } + } + + /** + * RecordBuilder for TestAvroNested instances. + */ + @org.apache.avro.specific.AvroGenerated + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean BOOL_NON_NULLABLE; + private Integer int$; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { + super(other); + if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { + this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); + fieldSetFlags()[0] = other.fieldSetFlags()[0]; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = other.fieldSetFlags()[1]; + } + } + + /** + * Creates a Builder by copying an existing TestAvroNested instance + * @param other The existing instance to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { + super(SCHEMA$); + if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { + this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = true; + } + } + + /** + * Gets the value of the 'BOOL_NON_NULLABLE' field. + * @return The value. + */ + public boolean getBOOLNONNULLABLE() { + return BOOL_NON_NULLABLE; + } + + + /** + * Sets the value of the 'BOOL_NON_NULLABLE' field. + * @param value The value of 'BOOL_NON_NULLABLE'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setBOOLNONNULLABLE(boolean value) { + validate(fields()[0], value); + this.BOOL_NON_NULLABLE = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'BOOL_NON_NULLABLE' field has been set. + * @return True if the 'BOOL_NON_NULLABLE' field has been set, false otherwise. + */ + public boolean hasBOOLNONNULLABLE() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'BOOL_NON_NULLABLE' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearBOOLNONNULLABLE() { + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'int$' field. + * @return The value. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value The value of 'int$'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setInt$(Integer value) { + validate(fields()[1], value); + this.int$ = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'int$' field has been set. + * @return True if the 'int$' field has been set, false otherwise. + */ + public boolean hasInt$() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'int$' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearInt$() { + int$ = null; + fieldSetFlags()[1] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public TestAvroNested build() { + try { + TestAvroNested record = new TestAvroNested(); + record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); + return record; + } catch (org.apache.avro.AvroMissingFieldException e) { + throw e; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + + @Override protected boolean hasCustomCoders() { return true; } + + @Override public void customEncode(org.apache.avro.io.Encoder out) + throws java.io.IOException + { + out.writeBoolean(this.BOOL_NON_NULLABLE); + + if (this.int$ == null) { + out.writeIndex(1); + out.writeNull(); + } else { + out.writeIndex(0); + out.writeInt(this.int$); + } + + } + + @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) + throws java.io.IOException + { + org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); + if (fieldOrder == null) { + this.BOOL_NON_NULLABLE = in.readBoolean(); + + if (in.readIndex() != 0) { + in.readNull(); + this.int$ = null; + } else { + this.int$ = in.readInt(); + } + + } else { + for (int i = 0; i < 2; i++) { + switch (fieldOrder[i].pos()) { + case 0: + this.BOOL_NON_NULLABLE = in.readBoolean(); + break; + + case 1: + if (in.readIndex() != 0) { + in.readNull(); + this.int$ = null; + } else { + this.int$ = in.readInt(); + } + break; + + default: + throw new java.io.IOException("Corrupt ResolvingDecoder."); + } + } + } + } +} + + + + + + + + + + diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java new file mode 100644 index 000000000000..8d1e1d7dffe9 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java @@ -0,0 +1,25 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; +@org.apache.avro.specific.AvroGenerated +public enum TestEnum implements org.apache.avro.generic.GenericEnumSymbol { + abc, cde ; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"enum\",\"name\":\"TestEnum\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"symbols\":[\"abc\",\"cde\"]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } +} diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java new file mode 100644 index 000000000000..24c5be9ca18d --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java @@ -0,0 +1,56 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; +@org.apache.avro.specific.FixedSize(4) +@org.apache.avro.specific.AvroGenerated +public class fixed4 extends org.apache.avro.specific.SpecificFixed { + private static final long serialVersionUID = -5646354132642432749L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"fixed\",\"name\":\"fixed4\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"size\":4}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + + /** Creates a new fixed4 */ + public fixed4() { + super(); + } + + /** + * Creates a new fixed4 with the given bytes. + * @param bytes The bytes to create the new fixed4. + */ + public fixed4(byte[] bytes) { + super(bytes); + } + + private static final org.apache.avro.io.DatumWriter + WRITER$ = new org.apache.avro.specific.SpecificDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, org.apache.avro.specific.SpecificData.getEncoder(out)); + } + + private static final org.apache.avro.io.DatumReader + READER$ = new org.apache.avro.specific.SpecificDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, org.apache.avro.specific.SpecificData.getDecoder(in)); + } + +} diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java new file mode 100644 index 000000000000..926bfa9db9b4 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java @@ -0,0 +1,1350 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; + +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.SchemaStore; +import org.apache.avro.specific.SpecificData; + +import java.time.Instant; +import java.time.LocalDate; +import java.util.Map; + +@org.apache.avro.specific.AvroGenerated +public class TestAvro extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 27902431178981259L; + + + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvro\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"bool_non_nullable\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]},{\"name\":\"long\",\"type\":[\"long\",\"null\"]},{\"name\":\"float\",\"type\":[\"float\",\"null\"]},{\"name\":\"double\",\"type\":[\"double\",\"null\"]},{\"name\":\"string\",\"type\":[\"string\",\"null\"]},{\"name\":\"bytes\",\"type\":[\"bytes\",\"null\"]},{\"name\":\"fixed\",\"type\":{\"type\":\"fixed\",\"name\":\"fixed4\",\"size\":4}},{\"name\":\"date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},{\"name\":\"timestampMillis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}},{\"name\":\"TestEnum\",\"type\":{\"type\":\"enum\",\"name\":\"TestEnum\",\"symbols\":[\"abc\",\"cde\"]}},{\"name\":\"row\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"TestAvroNested\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}]},{\"name\":\"array\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\",\"TestAvroNested\"]}]},{\"name\":\"map\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"TestAvroNested\"]}]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static final SpecificData MODEL$ = new SpecificData(); + static { + MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.DateConversion()); + MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.TimestampMillisConversion()); + } + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder<>(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder<>(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageEncoder instance used by this class. + * @return the message encoder used by this class + */ + public static BinaryMessageEncoder getEncoder() { + return ENCODER; + } + + /** + * Return the BinaryMessageDecoder instance used by this class. + * @return the message decoder used by this class + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * 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 + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder<>(MODEL$, SCHEMA$, resolver); + } + + /** + * Serializes this TestAvro to a ByteBuffer. + * @return a buffer holding the serialized data for this instance + * @throws java.io.IOException if this instance could not be serialized + */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** + * Deserializes a TestAvro from a ByteBuffer. + * @param b a byte buffer holding serialized data for an instance of this class + * @return a TestAvro instance decoded from the given buffer + * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class + */ + public static TestAvro fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + private boolean bool_non_nullable; + private Integer int$; + private Long long$; + private Float float$; + private Double double$; + private CharSequence string; + private java.nio.ByteBuffer bytes; + private fixed4 fixed; + private java.time.LocalDate date; + private java.time.Instant timestampMillis; + private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; + private TestAvroNested row; + private java.util.List array; + private java.util.Map map; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public TestAvro() {} + + /** + * All-args constructor. + * @param bool_non_nullable The new value for bool_non_nullable + * @param int$ The new value for int + * @param long$ The new value for long + * @param float$ The new value for float + * @param double$ The new value for double + * @param string The new value for string + * @param bytes The new value for bytes + * @param fixed The new value for fixed + * @param date The new value for date + * @param timestampMillis The new value for timestampMillis + * @param TestEnum The new value for TestEnum + * @param row The new value for row + * @param array The new value for array + * @param map The new value for map + */ + public TestAvro(Boolean bool_non_nullable, Integer int$, Long long$, Float float$, Double double$, CharSequence string, java.nio.ByteBuffer bytes, fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, TestAvroNested row, java.util.List array, java.util.Map map) { + this.bool_non_nullable = bool_non_nullable; + this.int$ = int$; + this.long$ = long$; + this.float$ = float$; + this.double$ = double$; + this.string = string; + this.bytes = bytes; + this.fixed = fixed; + this.date = date; + this.timestampMillis = timestampMillis.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); + this.TestEnum = TestEnum; + this.row = row; + this.array = array; + this.map = map; + } + + /** + * Manually added a ompatible with Avro v1.8.2 API constructor + * + * @param bool_non_nullable + * @param int$ + * @param long$ + * @param float$ + * @param double$ + * @param string + * @param bytes + * @param fixed + * @param date + * @param timestampMillis + * @param TestEnum + * @param row + * @param array + * @param map + */ + public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.String string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, org.joda.time.LocalDate date, org.joda.time.DateTime timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { + this.bool_non_nullable = bool_non_nullable; + this.int$ = int$; + this.long$ = long$; + this.float$ = float$; + this.double$ = double$; + this.string = string; + this.bytes = bytes; + this.fixed = fixed; + this.date = LocalDate.of(date.getYear(), date.getMonthOfYear(), date.getDayOfMonth()); + this.timestampMillis = Instant.ofEpochMilli(timestampMillis.getMillis()); + this.TestEnum = TestEnum; + this.row = row; + this.array = array; + this.map = (Map)map; + } + + @Override + public SpecificData getSpecificData() { return MODEL$; } + + @Override + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + + // Used by DatumWriter. Applications should not call. + @Override + public Object get(int field$) { + switch (field$) { + case 0: return bool_non_nullable; + case 1: return int$; + case 2: return long$; + case 3: return float$; + case 4: return double$; + case 5: return string; + case 6: return bytes; + case 7: return fixed; + case 8: return date; + case 9: return timestampMillis; + case 10: return TestEnum; + case 11: return row; + case 12: return array; + case 13: return map; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + private static final org.apache.avro.Conversion[] conversions = + new org.apache.avro.Conversion[] { + null, + null, + null, + null, + null, + null, + null, + null, + new org.apache.avro.data.TimeConversions.DateConversion(), + new org.apache.avro.data.TimeConversions.TimestampMillisConversion(), + null, + null, + null, + null, + null + }; + + @Override + public org.apache.avro.Conversion getConversion(int field) { + return conversions[field]; + } + + // Used by DatumReader. Applications should not call. + @Override + @SuppressWarnings(value="unchecked") + public void put(int field$, Object value$) { + switch (field$) { + case 0: bool_non_nullable = (Boolean)value$; break; + case 1: int$ = (Integer)value$; break; + case 2: long$ = (Long)value$; break; + case 3: float$ = (Float)value$; break; + case 4: double$ = (Double)value$; break; + case 5: string = (CharSequence)value$; break; + case 6: bytes = (java.nio.ByteBuffer)value$; break; + case 7: fixed = (fixed4)value$; break; + case 8: date = (java.time.LocalDate)value$; break; + case 9: timestampMillis = (java.time.Instant)value$; break; + case 10: TestEnum = (org.apache.beam.sdk.extensions.avro.schemas.TestEnum)value$; break; + case 11: row = (TestAvroNested)value$; break; + case 12: array = (java.util.List)value$; break; + case 13: map = (java.util.Map)value$; break; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + /** + * Gets the value of the 'bool_non_nullable' field. + * @return The value of the 'bool_non_nullable' field. + */ + public boolean getBoolNonNullable() { + return bool_non_nullable; + } + + + /** + * Sets the value of the 'bool_non_nullable' field. + * @param value the value to set. + */ + public void setBoolNonNullable(boolean value) { + this.bool_non_nullable = value; + } + + /** + * Gets the value of the 'int$' field. + * @return The value of the 'int$' field. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value the value to set. + */ + public void setInt$(Integer value) { + this.int$ = value; + } + + /** + * Gets the value of the 'long$' field. + * @return The value of the 'long$' field. + */ + public Long getLong$() { + return long$; + } + + + /** + * Sets the value of the 'long$' field. + * @param value the value to set. + */ + public void setLong$(Long value) { + this.long$ = value; + } + + /** + * Gets the value of the 'float$' field. + * @return The value of the 'float$' field. + */ + public Float getFloat$() { + return float$; + } + + + /** + * Sets the value of the 'float$' field. + * @param value the value to set. + */ + public void setFloat$(Float value) { + this.float$ = value; + } + + /** + * Gets the value of the 'double$' field. + * @return The value of the 'double$' field. + */ + public Double getDouble$() { + return double$; + } + + + /** + * Sets the value of the 'double$' field. + * @param value the value to set. + */ + public void setDouble$(Double value) { + this.double$ = value; + } + + /** + * Gets the value of the 'string' field. + * @return The value of the 'string' field. + */ + public CharSequence getString() { + return string; + } + + + /** + * Sets the value of the 'string' field. + * @param value the value to set. + */ + public void setString(CharSequence value) { + this.string = value; + } + + /** + * Gets the value of the 'bytes' field. + * @return The value of the 'bytes' field. + */ + public java.nio.ByteBuffer getBytes() { + return bytes; + } + + + /** + * Sets the value of the 'bytes' field. + * @param value the value to set. + */ + public void setBytes(java.nio.ByteBuffer value) { + this.bytes = value; + } + + /** + * Gets the value of the 'fixed' field. + * @return The value of the 'fixed' field. + */ + public fixed4 getFixed() { + return fixed; + } + + + /** + * Sets the value of the 'fixed' field. + * @param value the value to set. + */ + public void setFixed(fixed4 value) { + this.fixed = value; + } + + /** + * Gets the value of the 'date' field. + * @return The value of the 'date' field. + */ + public java.time.LocalDate getDate() { + return date; + } + + + /** + * Sets the value of the 'date' field. + * @param value the value to set. + */ + public void setDate(java.time.LocalDate value) { + this.date = value; + } + + /** + * Gets the value of the 'timestampMillis' field. + * @return The value of the 'timestampMillis' field. + */ + public java.time.Instant getTimestampMillis() { + return timestampMillis; + } + + + /** + * Sets the value of the 'timestampMillis' field. + * @param value the value to set. + */ + public void setTimestampMillis(java.time.Instant value) { + this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); + } + + /** + * Gets the value of the 'TestEnum' field. + * @return The value of the 'TestEnum' field. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { + return TestEnum; + } + + + /** + * Sets the value of the 'TestEnum' field. + * @param value the value to set. + */ + public void setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { + this.TestEnum = value; + } + + /** + * Gets the value of the 'row' field. + * @return The value of the 'row' field. + */ + public TestAvroNested getRow() { + return row; + } + + + /** + * Sets the value of the 'row' field. + * @param value the value to set. + */ + public void setRow(TestAvroNested value) { + this.row = value; + } + + /** + * Gets the value of the 'array' field. + * @return The value of the 'array' field. + */ + public java.util.List getArray() { + return array; + } + + + /** + * Sets the value of the 'array' field. + * @param value the value to set. + */ + public void setArray(java.util.List value) { + this.array = value; + } + + /** + * Gets the value of the 'map' field. + * @return The value of the 'map' field. + */ + public java.util.Map getMap() { + return map; + } + + + /** + * Sets the value of the 'map' field. + * @param value the value to set. + */ + public void setMap(java.util.Map value) { + this.map = value; + } + + /** + * Creates a new TestAvro RecordBuilder. + * @return A new TestAvro RecordBuilder + */ + public static Builder newBuilder() { + return new Builder(); + } + + /** + * Creates a new TestAvro RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new TestAvro RecordBuilder + */ + public static Builder newBuilder(Builder other) { + if (other == null) { + return new Builder(); + } else { + return new Builder(other); + } + } + + /** + * Creates a new TestAvro RecordBuilder by copying an existing TestAvro instance. + * @param other The existing instance to copy. + * @return A new TestAvro RecordBuilder + */ + public static Builder newBuilder(TestAvro other) { + if (other == null) { + return new Builder(); + } else { + return new Builder(other); + } + } + + /** + * RecordBuilder for TestAvro instances. + */ + @org.apache.avro.specific.AvroGenerated + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean bool_non_nullable; + private Integer int$; + private Long long$; + private Float float$; + private Double double$; + private CharSequence string; + private java.nio.ByteBuffer bytes; + private fixed4 fixed; + private java.time.LocalDate date; + private java.time.Instant timestampMillis; + private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; + private TestAvroNested row; + private TestAvroNested.Builder rowBuilder; + private java.util.List array; + private java.util.Map map; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$, MODEL$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(Builder other) { + super(other); + if (isValidValue(fields()[0], other.bool_non_nullable)) { + this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); + fieldSetFlags()[0] = other.fieldSetFlags()[0]; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = other.fieldSetFlags()[1]; + } + if (isValidValue(fields()[2], other.long$)) { + this.long$ = data().deepCopy(fields()[2].schema(), other.long$); + fieldSetFlags()[2] = other.fieldSetFlags()[2]; + } + if (isValidValue(fields()[3], other.float$)) { + this.float$ = data().deepCopy(fields()[3].schema(), other.float$); + fieldSetFlags()[3] = other.fieldSetFlags()[3]; + } + if (isValidValue(fields()[4], other.double$)) { + this.double$ = data().deepCopy(fields()[4].schema(), other.double$); + fieldSetFlags()[4] = other.fieldSetFlags()[4]; + } + if (isValidValue(fields()[5], other.string)) { + this.string = data().deepCopy(fields()[5].schema(), other.string); + fieldSetFlags()[5] = other.fieldSetFlags()[5]; + } + if (isValidValue(fields()[6], other.bytes)) { + this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); + fieldSetFlags()[6] = other.fieldSetFlags()[6]; + } + if (isValidValue(fields()[7], other.fixed)) { + this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); + fieldSetFlags()[7] = other.fieldSetFlags()[7]; + } + if (isValidValue(fields()[8], other.date)) { + this.date = data().deepCopy(fields()[8].schema(), other.date); + fieldSetFlags()[8] = other.fieldSetFlags()[8]; + } + if (isValidValue(fields()[9], other.timestampMillis)) { + this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); + fieldSetFlags()[9] = other.fieldSetFlags()[9]; + } + if (isValidValue(fields()[10], other.TestEnum)) { + this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); + fieldSetFlags()[10] = other.fieldSetFlags()[10]; + } + if (isValidValue(fields()[11], other.row)) { + this.row = data().deepCopy(fields()[11].schema(), other.row); + fieldSetFlags()[11] = other.fieldSetFlags()[11]; + } + if (other.hasRowBuilder()) { + this.rowBuilder = TestAvroNested.newBuilder(other.getRowBuilder()); + } + if (isValidValue(fields()[12], other.array)) { + this.array = data().deepCopy(fields()[12].schema(), other.array); + fieldSetFlags()[12] = other.fieldSetFlags()[12]; + } + if (isValidValue(fields()[13], other.map)) { + this.map = data().deepCopy(fields()[13].schema(), other.map); + fieldSetFlags()[13] = other.fieldSetFlags()[13]; + } + } + + /** + * Creates a Builder by copying an existing TestAvro instance + * @param other The existing instance to copy. + */ + private Builder(TestAvro other) { + super(SCHEMA$, MODEL$); + if (isValidValue(fields()[0], other.bool_non_nullable)) { + this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.long$)) { + this.long$ = data().deepCopy(fields()[2].schema(), other.long$); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.float$)) { + this.float$ = data().deepCopy(fields()[3].schema(), other.float$); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.double$)) { + this.double$ = data().deepCopy(fields()[4].schema(), other.double$); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.string)) { + this.string = data().deepCopy(fields()[5].schema(), other.string); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.bytes)) { + this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); + fieldSetFlags()[6] = true; + } + if (isValidValue(fields()[7], other.fixed)) { + this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); + fieldSetFlags()[7] = true; + } + if (isValidValue(fields()[8], other.date)) { + this.date = data().deepCopy(fields()[8].schema(), other.date); + fieldSetFlags()[8] = true; + } + if (isValidValue(fields()[9], other.timestampMillis)) { + this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); + fieldSetFlags()[9] = true; + } + if (isValidValue(fields()[10], other.TestEnum)) { + this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); + fieldSetFlags()[10] = true; + } + if (isValidValue(fields()[11], other.row)) { + this.row = data().deepCopy(fields()[11].schema(), other.row); + fieldSetFlags()[11] = true; + } + this.rowBuilder = null; + if (isValidValue(fields()[12], other.array)) { + this.array = data().deepCopy(fields()[12].schema(), other.array); + fieldSetFlags()[12] = true; + } + if (isValidValue(fields()[13], other.map)) { + this.map = data().deepCopy(fields()[13].schema(), other.map); + fieldSetFlags()[13] = true; + } + } + + /** + * Gets the value of the 'bool_non_nullable' field. + * @return The value. + */ + public boolean getBoolNonNullable() { + return bool_non_nullable; + } + + + /** + * Sets the value of the 'bool_non_nullable' field. + * @param value The value of 'bool_non_nullable'. + * @return This builder. + */ + public Builder setBoolNonNullable(boolean value) { + validate(fields()[0], value); + this.bool_non_nullable = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'bool_non_nullable' field has been set. + * @return True if the 'bool_non_nullable' field has been set, false otherwise. + */ + public boolean hasBoolNonNullable() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'bool_non_nullable' field. + * @return This builder. + */ + public Builder clearBoolNonNullable() { + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'int$' field. + * @return The value. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value The value of 'int$'. + * @return This builder. + */ + public Builder setInt$(Integer value) { + validate(fields()[1], value); + this.int$ = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'int$' field has been set. + * @return True if the 'int$' field has been set, false otherwise. + */ + public boolean hasInt$() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'int$' field. + * @return This builder. + */ + public Builder clearInt$() { + int$ = null; + fieldSetFlags()[1] = false; + return this; + } + + /** + * Gets the value of the 'long$' field. + * @return The value. + */ + public Long getLong$() { + return long$; + } + + + /** + * Sets the value of the 'long$' field. + * @param value The value of 'long$'. + * @return This builder. + */ + public Builder setLong$(Long value) { + validate(fields()[2], value); + this.long$ = value; + fieldSetFlags()[2] = true; + return this; + } + + /** + * Checks whether the 'long$' field has been set. + * @return True if the 'long$' field has been set, false otherwise. + */ + public boolean hasLong$() { + return fieldSetFlags()[2]; + } + + + /** + * Clears the value of the 'long$' field. + * @return This builder. + */ + public Builder clearLong$() { + long$ = null; + fieldSetFlags()[2] = false; + return this; + } + + /** + * Gets the value of the 'float$' field. + * @return The value. + */ + public Float getFloat$() { + return float$; + } + + + /** + * Sets the value of the 'float$' field. + * @param value The value of 'float$'. + * @return This builder. + */ + public Builder setFloat$(Float value) { + validate(fields()[3], value); + this.float$ = value; + fieldSetFlags()[3] = true; + return this; + } + + /** + * Checks whether the 'float$' field has been set. + * @return True if the 'float$' field has been set, false otherwise. + */ + public boolean hasFloat$() { + return fieldSetFlags()[3]; + } + + + /** + * Clears the value of the 'float$' field. + * @return This builder. + */ + public Builder clearFloat$() { + float$ = null; + fieldSetFlags()[3] = false; + return this; + } + + /** + * Gets the value of the 'double$' field. + * @return The value. + */ + public Double getDouble$() { + return double$; + } + + + /** + * Sets the value of the 'double$' field. + * @param value The value of 'double$'. + * @return This builder. + */ + public Builder setDouble$(Double value) { + validate(fields()[4], value); + this.double$ = value; + fieldSetFlags()[4] = true; + return this; + } + + /** + * Checks whether the 'double$' field has been set. + * @return True if the 'double$' field has been set, false otherwise. + */ + public boolean hasDouble$() { + return fieldSetFlags()[4]; + } + + + /** + * Clears the value of the 'double$' field. + * @return This builder. + */ + public Builder clearDouble$() { + double$ = null; + fieldSetFlags()[4] = false; + return this; + } + + /** + * Gets the value of the 'string' field. + * @return The value. + */ + public CharSequence getString() { + return string; + } + + + /** + * Sets the value of the 'string' field. + * @param value The value of 'string'. + * @return This builder. + */ + public Builder setString(CharSequence value) { + validate(fields()[5], value); + this.string = value; + fieldSetFlags()[5] = true; + return this; + } + + /** + * Checks whether the 'string' field has been set. + * @return True if the 'string' field has been set, false otherwise. + */ + public boolean hasString() { + return fieldSetFlags()[5]; + } + + + /** + * Clears the value of the 'string' field. + * @return This builder. + */ + public Builder clearString() { + string = null; + fieldSetFlags()[5] = false; + return this; + } + + /** + * Gets the value of the 'bytes' field. + * @return The value. + */ + public java.nio.ByteBuffer getBytes() { + return bytes; + } + + + /** + * Sets the value of the 'bytes' field. + * @param value The value of 'bytes'. + * @return This builder. + */ + public Builder setBytes(java.nio.ByteBuffer value) { + validate(fields()[6], value); + this.bytes = value; + fieldSetFlags()[6] = true; + return this; + } + + /** + * Checks whether the 'bytes' field has been set. + * @return True if the 'bytes' field has been set, false otherwise. + */ + public boolean hasBytes() { + return fieldSetFlags()[6]; + } + + + /** + * Clears the value of the 'bytes' field. + * @return This builder. + */ + public Builder clearBytes() { + bytes = null; + fieldSetFlags()[6] = false; + return this; + } + + /** + * Gets the value of the 'fixed' field. + * @return The value. + */ + public fixed4 getFixed() { + return fixed; + } + + + /** + * Sets the value of the 'fixed' field. + * @param value The value of 'fixed'. + * @return This builder. + */ + public Builder setFixed(fixed4 value) { + validate(fields()[7], value); + this.fixed = value; + fieldSetFlags()[7] = true; + return this; + } + + /** + * Checks whether the 'fixed' field has been set. + * @return True if the 'fixed' field has been set, false otherwise. + */ + public boolean hasFixed() { + return fieldSetFlags()[7]; + } + + + /** + * Clears the value of the 'fixed' field. + * @return This builder. + */ + public Builder clearFixed() { + fixed = null; + fieldSetFlags()[7] = false; + return this; + } + + /** + * Gets the value of the 'date' field. + * @return The value. + */ + public java.time.LocalDate getDate() { + return date; + } + + + /** + * Sets the value of the 'date' field. + * @param value The value of 'date'. + * @return This builder. + */ + public Builder setDate(java.time.LocalDate value) { + validate(fields()[8], value); + this.date = value; + fieldSetFlags()[8] = true; + return this; + } + + /** + * Checks whether the 'date' field has been set. + * @return True if the 'date' field has been set, false otherwise. + */ + public boolean hasDate() { + return fieldSetFlags()[8]; + } + + + /** + * Clears the value of the 'date' field. + * @return This builder. + */ + public Builder clearDate() { + fieldSetFlags()[8] = false; + return this; + } + + /** + * Gets the value of the 'timestampMillis' field. + * @return The value. + */ + public java.time.Instant getTimestampMillis() { + return timestampMillis; + } + + + /** + * Sets the value of the 'timestampMillis' field. + * @param value The value of 'timestampMillis'. + * @return This builder. + */ + public Builder setTimestampMillis(java.time.Instant value) { + validate(fields()[9], value); + this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); + fieldSetFlags()[9] = true; + return this; + } + + /** + * Checks whether the 'timestampMillis' field has been set. + * @return True if the 'timestampMillis' field has been set, false otherwise. + */ + public boolean hasTimestampMillis() { + return fieldSetFlags()[9]; + } + + + /** + * Clears the value of the 'timestampMillis' field. + * @return This builder. + */ + public Builder clearTimestampMillis() { + fieldSetFlags()[9] = false; + return this; + } + + /** + * Gets the value of the 'TestEnum' field. + * @return The value. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { + return TestEnum; + } + + + /** + * Sets the value of the 'TestEnum' field. + * @param value The value of 'TestEnum'. + * @return This builder. + */ + public Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { + validate(fields()[10], value); + this.TestEnum = value; + fieldSetFlags()[10] = true; + return this; + } + + /** + * Checks whether the 'TestEnum' field has been set. + * @return True if the 'TestEnum' field has been set, false otherwise. + */ + public boolean hasTestEnum() { + return fieldSetFlags()[10]; + } + + + /** + * Clears the value of the 'TestEnum' field. + * @return This builder. + */ + public Builder clearTestEnum() { + TestEnum = null; + fieldSetFlags()[10] = false; + return this; + } + + /** + * Gets the value of the 'row' field. + * @return The value. + */ + public TestAvroNested getRow() { + return row; + } + + + /** + * Sets the value of the 'row' field. + * @param value The value of 'row'. + * @return This builder. + */ + public Builder setRow(TestAvroNested value) { + validate(fields()[11], value); + this.rowBuilder = null; + this.row = value; + fieldSetFlags()[11] = true; + return this; + } + + /** + * Checks whether the 'row' field has been set. + * @return True if the 'row' field has been set, false otherwise. + */ + public boolean hasRow() { + return fieldSetFlags()[11]; + } + + /** + * Gets the Builder instance for the 'row' field and creates one if it doesn't exist yet. + * @return This builder. + */ + public TestAvroNested.Builder getRowBuilder() { + if (rowBuilder == null) { + if (hasRow()) { + setRowBuilder(TestAvroNested.newBuilder(row)); + } else { + setRowBuilder(TestAvroNested.newBuilder()); + } + } + return rowBuilder; + } + + /** + * Sets the Builder instance for the 'row' field + * @param value The builder instance that must be set. + * @return This builder. + */ + + public Builder setRowBuilder(TestAvroNested.Builder value) { + clearRow(); + rowBuilder = value; + return this; + } + + /** + * Checks whether the 'row' field has an active Builder instance + * @return True if the 'row' field has an active Builder instance + */ + public boolean hasRowBuilder() { + return rowBuilder != null; + } + + /** + * Clears the value of the 'row' field. + * @return This builder. + */ + public Builder clearRow() { + row = null; + rowBuilder = null; + fieldSetFlags()[11] = false; + return this; + } + + /** + * Gets the value of the 'array' field. + * @return The value. + */ + public java.util.List getArray() { + return array; + } + + + /** + * Sets the value of the 'array' field. + * @param value The value of 'array'. + * @return This builder. + */ + public Builder setArray(java.util.List value) { + validate(fields()[12], value); + this.array = value; + fieldSetFlags()[12] = true; + return this; + } + + /** + * Checks whether the 'array' field has been set. + * @return True if the 'array' field has been set, false otherwise. + */ + public boolean hasArray() { + return fieldSetFlags()[12]; + } + + + /** + * Clears the value of the 'array' field. + * @return This builder. + */ + public Builder clearArray() { + array = null; + fieldSetFlags()[12] = false; + return this; + } + + /** + * Gets the value of the 'map' field. + * @return The value. + */ + public java.util.Map getMap() { + return map; + } + + + /** + * Sets the value of the 'map' field. + * @param value The value of 'map'. + * @return This builder. + */ + public Builder setMap(java.util.Map value) { + validate(fields()[13], value); + this.map = value; + fieldSetFlags()[13] = true; + return this; + } + + /** + * Checks whether the 'map' field has been set. + * @return True if the 'map' field has been set, false otherwise. + */ + public boolean hasMap() { + return fieldSetFlags()[13]; + } + + + /** + * Clears the value of the 'map' field. + * @return This builder. + */ + public Builder clearMap() { + map = null; + fieldSetFlags()[13] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public TestAvro build() { + try { + TestAvro record = new TestAvro(); + record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); + record.long$ = fieldSetFlags()[2] ? this.long$ : (Long) defaultValue(fields()[2]); + record.float$ = fieldSetFlags()[3] ? this.float$ : (Float) defaultValue(fields()[3]); + record.double$ = fieldSetFlags()[4] ? this.double$ : (Double) defaultValue(fields()[4]); + record.string = fieldSetFlags()[5] ? this.string : (CharSequence) defaultValue(fields()[5]); + record.bytes = fieldSetFlags()[6] ? this.bytes : (java.nio.ByteBuffer) defaultValue(fields()[6]); + record.fixed = fieldSetFlags()[7] ? this.fixed : (fixed4) defaultValue(fields()[7]); + record.date = fieldSetFlags()[8] ? this.date : (java.time.LocalDate) defaultValue(fields()[8]); + record.timestampMillis = fieldSetFlags()[9] ? this.timestampMillis : (java.time.Instant) defaultValue(fields()[9]); + record.TestEnum = fieldSetFlags()[10] ? this.TestEnum : (org.apache.beam.sdk.extensions.avro.schemas.TestEnum) defaultValue(fields()[10]); + if (rowBuilder != null) { + try { + record.row = this.rowBuilder.build(); + } catch (org.apache.avro.AvroMissingFieldException e) { + e.addParentField(record.getSchema().getField("row")); + throw e; + } + } else { + record.row = fieldSetFlags()[11] ? this.row : (TestAvroNested) defaultValue(fields()[11]); + } + record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); + record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); + return record; + } catch (org.apache.avro.AvroMissingFieldException e) { + throw e; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + +} + + + + + + + + + + diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java new file mode 100644 index 000000000000..cfbe2380cda8 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java @@ -0,0 +1,424 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; + +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.SchemaStore; +import org.apache.avro.specific.SpecificData; + +@org.apache.avro.specific.AvroGenerated +public class TestAvroNested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 4633138088036298925L; + + + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvroNested\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static final SpecificData MODEL$ = new SpecificData(); + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder<>(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder<>(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageEncoder instance used by this class. + * @return the message encoder used by this class + */ + public static BinaryMessageEncoder getEncoder() { + return ENCODER; + } + + /** + * Return the BinaryMessageDecoder instance used by this class. + * @return the message decoder used by this class + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * 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 + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder<>(MODEL$, SCHEMA$, resolver); + } + + /** + * Serializes this TestAvroNested to a ByteBuffer. + * @return a buffer holding the serialized data for this instance + * @throws java.io.IOException if this instance could not be serialized + */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** + * Deserializes a TestAvroNested from a ByteBuffer. + * @param b a byte buffer holding serialized data for an instance of this class + * @return a TestAvroNested instance decoded from the given buffer + * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class + */ + public static TestAvroNested fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + private boolean BOOL_NON_NULLABLE; + private Integer int$; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public TestAvroNested() {} + + /** + * All-args constructor. + * @param BOOL_NON_NULLABLE The new value for BOOL_NON_NULLABLE + * @param int$ The new value for int + */ + public TestAvroNested(Boolean BOOL_NON_NULLABLE, Integer int$) { + this.BOOL_NON_NULLABLE = BOOL_NON_NULLABLE; + this.int$ = int$; + } + + @Override + public SpecificData getSpecificData() { return MODEL$; } + + @Override + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + + // Used by DatumWriter. Applications should not call. + @Override + public Object get(int field$) { + switch (field$) { + case 0: return BOOL_NON_NULLABLE; + case 1: return int$; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + // Used by DatumReader. Applications should not call. + @Override + @SuppressWarnings(value="unchecked") + public void put(int field$, Object value$) { + switch (field$) { + case 0: BOOL_NON_NULLABLE = (Boolean)value$; break; + case 1: int$ = (Integer)value$; break; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + /** + * Gets the value of the 'BOOL_NON_NULLABLE' field. + * @return The value of the 'BOOL_NON_NULLABLE' field. + */ + public boolean getBOOLNONNULLABLE() { + return BOOL_NON_NULLABLE; + } + + + /** + * Sets the value of the 'BOOL_NON_NULLABLE' field. + * @param value the value to set. + */ + public void setBOOLNONNULLABLE(boolean value) { + this.BOOL_NON_NULLABLE = value; + } + + /** + * Gets the value of the 'int$' field. + * @return The value of the 'int$' field. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value the value to set. + */ + public void setInt$(Integer value) { + this.int$ = value; + } + + /** + * Creates a new TestAvroNested RecordBuilder. + * @return A new TestAvroNested RecordBuilder + */ + public static Builder newBuilder() { + return new Builder(); + } + + /** + * Creates a new TestAvroNested RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new TestAvroNested RecordBuilder + */ + public static Builder newBuilder(Builder other) { + if (other == null) { + return new Builder(); + } else { + return new Builder(other); + } + } + + /** + * Creates a new TestAvroNested RecordBuilder by copying an existing TestAvroNested instance. + * @param other The existing instance to copy. + * @return A new TestAvroNested RecordBuilder + */ + public static Builder newBuilder(TestAvroNested other) { + if (other == null) { + return new Builder(); + } else { + return new Builder(other); + } + } + + /** + * RecordBuilder for TestAvroNested instances. + */ + @org.apache.avro.specific.AvroGenerated + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean BOOL_NON_NULLABLE; + private Integer int$; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$, MODEL$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(Builder other) { + super(other); + if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { + this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); + fieldSetFlags()[0] = other.fieldSetFlags()[0]; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = other.fieldSetFlags()[1]; + } + } + + /** + * Creates a Builder by copying an existing TestAvroNested instance + * @param other The existing instance to copy. + */ + private Builder(TestAvroNested other) { + super(SCHEMA$, MODEL$); + if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { + this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = true; + } + } + + /** + * Gets the value of the 'BOOL_NON_NULLABLE' field. + * @return The value. + */ + public boolean getBOOLNONNULLABLE() { + return BOOL_NON_NULLABLE; + } + + + /** + * Sets the value of the 'BOOL_NON_NULLABLE' field. + * @param value The value of 'BOOL_NON_NULLABLE'. + * @return This builder. + */ + public Builder setBOOLNONNULLABLE(boolean value) { + validate(fields()[0], value); + this.BOOL_NON_NULLABLE = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'BOOL_NON_NULLABLE' field has been set. + * @return True if the 'BOOL_NON_NULLABLE' field has been set, false otherwise. + */ + public boolean hasBOOLNONNULLABLE() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'BOOL_NON_NULLABLE' field. + * @return This builder. + */ + public Builder clearBOOLNONNULLABLE() { + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'int$' field. + * @return The value. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value The value of 'int$'. + * @return This builder. + */ + public Builder setInt$(Integer value) { + validate(fields()[1], value); + this.int$ = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'int$' field has been set. + * @return True if the 'int$' field has been set, false otherwise. + */ + public boolean hasInt$() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'int$' field. + * @return This builder. + */ + public Builder clearInt$() { + int$ = null; + fieldSetFlags()[1] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public TestAvroNested build() { + try { + TestAvroNested record = new TestAvroNested(); + record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); + return record; + } catch (org.apache.avro.AvroMissingFieldException e) { + throw e; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + + @Override protected boolean hasCustomCoders() { return true; } + + @Override public void customEncode(org.apache.avro.io.Encoder out) + throws java.io.IOException + { + out.writeBoolean(this.BOOL_NON_NULLABLE); + + if (this.int$ == null) { + out.writeIndex(1); + out.writeNull(); + } else { + out.writeIndex(0); + out.writeInt(this.int$); + } + + } + + @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) + throws java.io.IOException + { + org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); + if (fieldOrder == null) { + this.BOOL_NON_NULLABLE = in.readBoolean(); + + if (in.readIndex() != 0) { + in.readNull(); + this.int$ = null; + } else { + this.int$ = in.readInt(); + } + + } else { + for (int i = 0; i < 2; i++) { + switch (fieldOrder[i].pos()) { + case 0: + this.BOOL_NON_NULLABLE = in.readBoolean(); + break; + + case 1: + if (in.readIndex() != 0) { + in.readNull(); + this.int$ = null; + } else { + this.int$ = in.readInt(); + } + break; + + default: + throw new java.io.IOException("Corrupt ResolvingDecoder."); + } + } + } + } +} + + + + + + + + + + diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java new file mode 100644 index 000000000000..eb52356a472b --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java @@ -0,0 +1,27 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; +@org.apache.avro.specific.AvroGenerated +public enum TestEnum implements org.apache.avro.generic.GenericEnumSymbol { + abc, cde ; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"enum\",\"name\":\"TestEnum\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"symbols\":[\"abc\",\"cde\"]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + @Override + public org.apache.avro.Schema getSchema() { return SCHEMA$; } +} diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java new file mode 100644 index 000000000000..24c5be9ca18d --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java @@ -0,0 +1,56 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; +@org.apache.avro.specific.FixedSize(4) +@org.apache.avro.specific.AvroGenerated +public class fixed4 extends org.apache.avro.specific.SpecificFixed { + private static final long serialVersionUID = -5646354132642432749L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"fixed\",\"name\":\"fixed4\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"size\":4}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + + /** Creates a new fixed4 */ + public fixed4() { + super(); + } + + /** + * Creates a new fixed4 with the given bytes. + * @param bytes The bytes to create the new fixed4. + */ + public fixed4(byte[] bytes) { + super(bytes); + } + + private static final org.apache.avro.io.DatumWriter + WRITER$ = new org.apache.avro.specific.SpecificDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, org.apache.avro.specific.SpecificData.getEncoder(out)); + } + + private static final org.apache.avro.io.DatumReader + READER$ = new org.apache.avro.specific.SpecificDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, org.apache.avro.specific.SpecificData.getDecoder(in)); + } + +} diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java new file mode 100644 index 000000000000..9493c7976c85 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java @@ -0,0 +1,1341 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; + +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.SchemaStore; +import org.apache.avro.specific.SpecificData; + +import java.time.Instant; +import java.time.LocalDate; +import java.util.Map; + +@org.apache.avro.specific.AvroGenerated +public class TestAvro extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 27902431178981259L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvro\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"bool_non_nullable\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]},{\"name\":\"long\",\"type\":[\"long\",\"null\"]},{\"name\":\"float\",\"type\":[\"float\",\"null\"]},{\"name\":\"double\",\"type\":[\"double\",\"null\"]},{\"name\":\"string\",\"type\":[\"string\",\"null\"]},{\"name\":\"bytes\",\"type\":[\"bytes\",\"null\"]},{\"name\":\"fixed\",\"type\":{\"type\":\"fixed\",\"name\":\"fixed4\",\"size\":4}},{\"name\":\"date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},{\"name\":\"timestampMillis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}},{\"name\":\"TestEnum\",\"type\":{\"type\":\"enum\",\"name\":\"TestEnum\",\"symbols\":[\"abc\",\"cde\"]}},{\"name\":\"row\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"TestAvroNested\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}]},{\"name\":\"array\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\",\"TestAvroNested\"]}]},{\"name\":\"map\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"TestAvroNested\"]}]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static SpecificData MODEL$ = new SpecificData(); +static { + MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.DateConversion()); + MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.TimestampMillisConversion()); + } + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageEncoder instance used by this class. + * @return the message encoder used by this class + */ + public static BinaryMessageEncoder getEncoder() { + return ENCODER; + } + + /** + * Return the BinaryMessageDecoder instance used by this class. + * @return the message decoder used by this class + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * 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 + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); + } + + /** + * Serializes this TestAvro to a ByteBuffer. + * @return a buffer holding the serialized data for this instance + * @throws java.io.IOException if this instance could not be serialized + */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** + * Deserializes a TestAvro from a ByteBuffer. + * @param b a byte buffer holding serialized data for an instance of this class + * @return a TestAvro instance decoded from the given buffer + * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class + */ + public static TestAvro fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + private boolean bool_non_nullable; + private Integer int$; + private Long long$; + private Float float$; + private Double double$; + private CharSequence string; + private java.nio.ByteBuffer bytes; + private fixed4 fixed; + private java.time.LocalDate date; + private java.time.Instant timestampMillis; + private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; + private TestAvroNested row; + private java.util.List array; + private java.util.Map map; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public TestAvro() {} + + /** + * All-args constructor. + * @param bool_non_nullable The new value for bool_non_nullable + * @param int$ The new value for int + * @param long$ The new value for long + * @param float$ The new value for float + * @param double$ The new value for double + * @param string The new value for string + * @param bytes The new value for bytes + * @param fixed The new value for fixed + * @param date The new value for date + * @param timestampMillis The new value for timestampMillis + * @param TestEnum The new value for TestEnum + * @param row The new value for row + * @param array The new value for array + * @param map The new value for map + */ + public TestAvro(Boolean bool_non_nullable, Integer int$, Long long$, Float float$, Double double$, CharSequence string, java.nio.ByteBuffer bytes, fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, TestAvroNested row, java.util.List array, java.util.Map map) { + this.bool_non_nullable = bool_non_nullable; + this.int$ = int$; + this.long$ = long$; + this.float$ = float$; + this.double$ = double$; + this.string = string; + this.bytes = bytes; + this.fixed = fixed; + this.date = date; + this.timestampMillis = timestampMillis.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); + this.TestEnum = TestEnum; + this.row = row; + this.array = array; + this.map = map; + } + + /** + * Manually added a ompatible with Avro v1.8.2 API constructor + * + * @param bool_non_nullable + * @param int$ + * @param long$ + * @param float$ + * @param double$ + * @param string + * @param bytes + * @param fixed + * @param date + * @param timestampMillis + * @param TestEnum + * @param row + * @param array + * @param map + */ + public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.String string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, org.joda.time.LocalDate date, org.joda.time.DateTime timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { + this.bool_non_nullable = bool_non_nullable; + this.int$ = int$; + this.long$ = long$; + this.float$ = float$; + this.double$ = double$; + this.string = string; + this.bytes = bytes; + this.fixed = fixed; + this.date = LocalDate.of(date.getYear(), date.getMonthOfYear(), date.getDayOfMonth()); + this.timestampMillis = Instant.ofEpochMilli(timestampMillis.getMillis()); + this.TestEnum = TestEnum; + this.row = row; + this.array = array; + this.map = (Map)map; + } + + public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public Object get(int field$) { + switch (field$) { + case 0: return bool_non_nullable; + case 1: return int$; + case 2: return long$; + case 3: return float$; + case 4: return double$; + case 5: return string; + case 6: return bytes; + case 7: return fixed; + case 8: return date; + case 9: return timestampMillis; + case 10: return TestEnum; + case 11: return row; + case 12: return array; + case 13: return map; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + private static final org.apache.avro.Conversion[] conversions = + new org.apache.avro.Conversion[] { + null, + null, + null, + null, + null, + null, + null, + null, + new org.apache.avro.data.TimeConversions.DateConversion(), + new org.apache.avro.data.TimeConversions.TimestampMillisConversion(), + null, + null, + null, + null, + null + }; + + @Override + public org.apache.avro.Conversion getConversion(int field) { + return conversions[field]; + } + + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, Object value$) { + switch (field$) { + case 0: bool_non_nullable = (Boolean)value$; break; + case 1: int$ = (Integer)value$; break; + case 2: long$ = (Long)value$; break; + case 3: float$ = (Float)value$; break; + case 4: double$ = (Double)value$; break; + case 5: string = (CharSequence)value$; break; + case 6: bytes = (java.nio.ByteBuffer)value$; break; + case 7: fixed = (fixed4)value$; break; + case 8: date = (java.time.LocalDate)value$; break; + case 9: timestampMillis = (java.time.Instant)value$; break; + case 10: TestEnum = (org.apache.beam.sdk.extensions.avro.schemas.TestEnum)value$; break; + case 11: row = (TestAvroNested)value$; break; + case 12: array = (java.util.List)value$; break; + case 13: map = (java.util.Map)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'bool_non_nullable' field. + * @return The value of the 'bool_non_nullable' field. + */ + public boolean getBoolNonNullable() { + return bool_non_nullable; + } + + + /** + * Sets the value of the 'bool_non_nullable' field. + * @param value the value to set. + */ + public void setBoolNonNullable(boolean value) { + this.bool_non_nullable = value; + } + + /** + * Gets the value of the 'int$' field. + * @return The value of the 'int$' field. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value the value to set. + */ + public void setInt$(Integer value) { + this.int$ = value; + } + + /** + * Gets the value of the 'long$' field. + * @return The value of the 'long$' field. + */ + public Long getLong$() { + return long$; + } + + + /** + * Sets the value of the 'long$' field. + * @param value the value to set. + */ + public void setLong$(Long value) { + this.long$ = value; + } + + /** + * Gets the value of the 'float$' field. + * @return The value of the 'float$' field. + */ + public Float getFloat$() { + return float$; + } + + + /** + * Sets the value of the 'float$' field. + * @param value the value to set. + */ + public void setFloat$(Float value) { + this.float$ = value; + } + + /** + * Gets the value of the 'double$' field. + * @return The value of the 'double$' field. + */ + public Double getDouble$() { + return double$; + } + + + /** + * Sets the value of the 'double$' field. + * @param value the value to set. + */ + public void setDouble$(Double value) { + this.double$ = value; + } + + /** + * Gets the value of the 'string' field. + * @return The value of the 'string' field. + */ + public CharSequence getString() { + return string; + } + + + /** + * Sets the value of the 'string' field. + * @param value the value to set. + */ + public void setString(CharSequence value) { + this.string = value; + } + + /** + * Gets the value of the 'bytes' field. + * @return The value of the 'bytes' field. + */ + public java.nio.ByteBuffer getBytes() { + return bytes; + } + + + /** + * Sets the value of the 'bytes' field. + * @param value the value to set. + */ + public void setBytes(java.nio.ByteBuffer value) { + this.bytes = value; + } + + /** + * Gets the value of the 'fixed' field. + * @return The value of the 'fixed' field. + */ + public fixed4 getFixed() { + return fixed; + } + + + /** + * Sets the value of the 'fixed' field. + * @param value the value to set. + */ + public void setFixed(fixed4 value) { + this.fixed = value; + } + + /** + * Gets the value of the 'date' field. + * @return The value of the 'date' field. + */ + public java.time.LocalDate getDate() { + return date; + } + + + /** + * Sets the value of the 'date' field. + * @param value the value to set. + */ + public void setDate(java.time.LocalDate value) { + this.date = value; + } + + /** + * Gets the value of the 'timestampMillis' field. + * @return The value of the 'timestampMillis' field. + */ + public java.time.Instant getTimestampMillis() { + return timestampMillis; + } + + + /** + * Sets the value of the 'timestampMillis' field. + * @param value the value to set. + */ + public void setTimestampMillis(java.time.Instant value) { + this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); + } + + /** + * Gets the value of the 'TestEnum' field. + * @return The value of the 'TestEnum' field. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { + return TestEnum; + } + + + /** + * Sets the value of the 'TestEnum' field. + * @param value the value to set. + */ + public void setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { + this.TestEnum = value; + } + + /** + * Gets the value of the 'row' field. + * @return The value of the 'row' field. + */ + public TestAvroNested getRow() { + return row; + } + + + /** + * Sets the value of the 'row' field. + * @param value the value to set. + */ + public void setRow(TestAvroNested value) { + this.row = value; + } + + /** + * Gets the value of the 'array' field. + * @return The value of the 'array' field. + */ + public java.util.List getArray() { + return array; + } + + + /** + * Sets the value of the 'array' field. + * @param value the value to set. + */ + public void setArray(java.util.List value) { + this.array = value; + } + + /** + * Gets the value of the 'map' field. + * @return The value of the 'map' field. + */ + public java.util.Map getMap() { + return map; + } + + + /** + * Sets the value of the 'map' field. + * @param value the value to set. + */ + public void setMap(java.util.Map value) { + this.map = value; + } + + /** + * Creates a new TestAvro RecordBuilder. + * @return A new TestAvro RecordBuilder + */ + public static Builder newBuilder() { + return new Builder(); + } + + /** + * Creates a new TestAvro RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new TestAvro RecordBuilder + */ + public static Builder newBuilder(Builder other) { + if (other == null) { + return new Builder(); + } else { + return new Builder(other); + } + } + + /** + * Creates a new TestAvro RecordBuilder by copying an existing TestAvro instance. + * @param other The existing instance to copy. + * @return A new TestAvro RecordBuilder + */ + public static Builder newBuilder(TestAvro other) { + if (other == null) { + return new Builder(); + } else { + return new Builder(other); + } + } + + /** + * RecordBuilder for TestAvro instances. + */ + @org.apache.avro.specific.AvroGenerated + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean bool_non_nullable; + private Integer int$; + private Long long$; + private Float float$; + private Double double$; + private CharSequence string; + private java.nio.ByteBuffer bytes; + private fixed4 fixed; + private java.time.LocalDate date; + private java.time.Instant timestampMillis; + private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; + private TestAvroNested row; + private TestAvroNested.Builder rowBuilder; + private java.util.List array; + private java.util.Map map; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(Builder other) { + super(other); + if (isValidValue(fields()[0], other.bool_non_nullable)) { + this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); + fieldSetFlags()[0] = other.fieldSetFlags()[0]; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = other.fieldSetFlags()[1]; + } + if (isValidValue(fields()[2], other.long$)) { + this.long$ = data().deepCopy(fields()[2].schema(), other.long$); + fieldSetFlags()[2] = other.fieldSetFlags()[2]; + } + if (isValidValue(fields()[3], other.float$)) { + this.float$ = data().deepCopy(fields()[3].schema(), other.float$); + fieldSetFlags()[3] = other.fieldSetFlags()[3]; + } + if (isValidValue(fields()[4], other.double$)) { + this.double$ = data().deepCopy(fields()[4].schema(), other.double$); + fieldSetFlags()[4] = other.fieldSetFlags()[4]; + } + if (isValidValue(fields()[5], other.string)) { + this.string = data().deepCopy(fields()[5].schema(), other.string); + fieldSetFlags()[5] = other.fieldSetFlags()[5]; + } + if (isValidValue(fields()[6], other.bytes)) { + this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); + fieldSetFlags()[6] = other.fieldSetFlags()[6]; + } + if (isValidValue(fields()[7], other.fixed)) { + this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); + fieldSetFlags()[7] = other.fieldSetFlags()[7]; + } + if (isValidValue(fields()[8], other.date)) { + this.date = data().deepCopy(fields()[8].schema(), other.date); + fieldSetFlags()[8] = other.fieldSetFlags()[8]; + } + if (isValidValue(fields()[9], other.timestampMillis)) { + this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); + fieldSetFlags()[9] = other.fieldSetFlags()[9]; + } + if (isValidValue(fields()[10], other.TestEnum)) { + this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); + fieldSetFlags()[10] = other.fieldSetFlags()[10]; + } + if (isValidValue(fields()[11], other.row)) { + this.row = data().deepCopy(fields()[11].schema(), other.row); + fieldSetFlags()[11] = other.fieldSetFlags()[11]; + } + if (other.hasRowBuilder()) { + this.rowBuilder = TestAvroNested.newBuilder(other.getRowBuilder()); + } + if (isValidValue(fields()[12], other.array)) { + this.array = data().deepCopy(fields()[12].schema(), other.array); + fieldSetFlags()[12] = other.fieldSetFlags()[12]; + } + if (isValidValue(fields()[13], other.map)) { + this.map = data().deepCopy(fields()[13].schema(), other.map); + fieldSetFlags()[13] = other.fieldSetFlags()[13]; + } + } + + /** + * Creates a Builder by copying an existing TestAvro instance + * @param other The existing instance to copy. + */ + private Builder(TestAvro other) { + super(SCHEMA$); + if (isValidValue(fields()[0], other.bool_non_nullable)) { + this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.long$)) { + this.long$ = data().deepCopy(fields()[2].schema(), other.long$); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.float$)) { + this.float$ = data().deepCopy(fields()[3].schema(), other.float$); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.double$)) { + this.double$ = data().deepCopy(fields()[4].schema(), other.double$); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.string)) { + this.string = data().deepCopy(fields()[5].schema(), other.string); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.bytes)) { + this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); + fieldSetFlags()[6] = true; + } + if (isValidValue(fields()[7], other.fixed)) { + this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); + fieldSetFlags()[7] = true; + } + if (isValidValue(fields()[8], other.date)) { + this.date = data().deepCopy(fields()[8].schema(), other.date); + fieldSetFlags()[8] = true; + } + if (isValidValue(fields()[9], other.timestampMillis)) { + this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); + fieldSetFlags()[9] = true; + } + if (isValidValue(fields()[10], other.TestEnum)) { + this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); + fieldSetFlags()[10] = true; + } + if (isValidValue(fields()[11], other.row)) { + this.row = data().deepCopy(fields()[11].schema(), other.row); + fieldSetFlags()[11] = true; + } + this.rowBuilder = null; + if (isValidValue(fields()[12], other.array)) { + this.array = data().deepCopy(fields()[12].schema(), other.array); + fieldSetFlags()[12] = true; + } + if (isValidValue(fields()[13], other.map)) { + this.map = data().deepCopy(fields()[13].schema(), other.map); + fieldSetFlags()[13] = true; + } + } + + /** + * Gets the value of the 'bool_non_nullable' field. + * @return The value. + */ + public boolean getBoolNonNullable() { + return bool_non_nullable; + } + + + /** + * Sets the value of the 'bool_non_nullable' field. + * @param value The value of 'bool_non_nullable'. + * @return This builder. + */ + public Builder setBoolNonNullable(boolean value) { + validate(fields()[0], value); + this.bool_non_nullable = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'bool_non_nullable' field has been set. + * @return True if the 'bool_non_nullable' field has been set, false otherwise. + */ + public boolean hasBoolNonNullable() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'bool_non_nullable' field. + * @return This builder. + */ + public Builder clearBoolNonNullable() { + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'int$' field. + * @return The value. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value The value of 'int$'. + * @return This builder. + */ + public Builder setInt$(Integer value) { + validate(fields()[1], value); + this.int$ = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'int$' field has been set. + * @return True if the 'int$' field has been set, false otherwise. + */ + public boolean hasInt$() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'int$' field. + * @return This builder. + */ + public Builder clearInt$() { + int$ = null; + fieldSetFlags()[1] = false; + return this; + } + + /** + * Gets the value of the 'long$' field. + * @return The value. + */ + public Long getLong$() { + return long$; + } + + + /** + * Sets the value of the 'long$' field. + * @param value The value of 'long$'. + * @return This builder. + */ + public Builder setLong$(Long value) { + validate(fields()[2], value); + this.long$ = value; + fieldSetFlags()[2] = true; + return this; + } + + /** + * Checks whether the 'long$' field has been set. + * @return True if the 'long$' field has been set, false otherwise. + */ + public boolean hasLong$() { + return fieldSetFlags()[2]; + } + + + /** + * Clears the value of the 'long$' field. + * @return This builder. + */ + public Builder clearLong$() { + long$ = null; + fieldSetFlags()[2] = false; + return this; + } + + /** + * Gets the value of the 'float$' field. + * @return The value. + */ + public Float getFloat$() { + return float$; + } + + + /** + * Sets the value of the 'float$' field. + * @param value The value of 'float$'. + * @return This builder. + */ + public Builder setFloat$(Float value) { + validate(fields()[3], value); + this.float$ = value; + fieldSetFlags()[3] = true; + return this; + } + + /** + * Checks whether the 'float$' field has been set. + * @return True if the 'float$' field has been set, false otherwise. + */ + public boolean hasFloat$() { + return fieldSetFlags()[3]; + } + + + /** + * Clears the value of the 'float$' field. + * @return This builder. + */ + public Builder clearFloat$() { + float$ = null; + fieldSetFlags()[3] = false; + return this; + } + + /** + * Gets the value of the 'double$' field. + * @return The value. + */ + public Double getDouble$() { + return double$; + } + + + /** + * Sets the value of the 'double$' field. + * @param value The value of 'double$'. + * @return This builder. + */ + public Builder setDouble$(Double value) { + validate(fields()[4], value); + this.double$ = value; + fieldSetFlags()[4] = true; + return this; + } + + /** + * Checks whether the 'double$' field has been set. + * @return True if the 'double$' field has been set, false otherwise. + */ + public boolean hasDouble$() { + return fieldSetFlags()[4]; + } + + + /** + * Clears the value of the 'double$' field. + * @return This builder. + */ + public Builder clearDouble$() { + double$ = null; + fieldSetFlags()[4] = false; + return this; + } + + /** + * Gets the value of the 'string' field. + * @return The value. + */ + public CharSequence getString() { + return string; + } + + + /** + * Sets the value of the 'string' field. + * @param value The value of 'string'. + * @return This builder. + */ + public Builder setString(CharSequence value) { + validate(fields()[5], value); + this.string = value; + fieldSetFlags()[5] = true; + return this; + } + + /** + * Checks whether the 'string' field has been set. + * @return True if the 'string' field has been set, false otherwise. + */ + public boolean hasString() { + return fieldSetFlags()[5]; + } + + + /** + * Clears the value of the 'string' field. + * @return This builder. + */ + public Builder clearString() { + string = null; + fieldSetFlags()[5] = false; + return this; + } + + /** + * Gets the value of the 'bytes' field. + * @return The value. + */ + public java.nio.ByteBuffer getBytes() { + return bytes; + } + + + /** + * Sets the value of the 'bytes' field. + * @param value The value of 'bytes'. + * @return This builder. + */ + public Builder setBytes(java.nio.ByteBuffer value) { + validate(fields()[6], value); + this.bytes = value; + fieldSetFlags()[6] = true; + return this; + } + + /** + * Checks whether the 'bytes' field has been set. + * @return True if the 'bytes' field has been set, false otherwise. + */ + public boolean hasBytes() { + return fieldSetFlags()[6]; + } + + + /** + * Clears the value of the 'bytes' field. + * @return This builder. + */ + public Builder clearBytes() { + bytes = null; + fieldSetFlags()[6] = false; + return this; + } + + /** + * Gets the value of the 'fixed' field. + * @return The value. + */ + public fixed4 getFixed() { + return fixed; + } + + + /** + * Sets the value of the 'fixed' field. + * @param value The value of 'fixed'. + * @return This builder. + */ + public Builder setFixed(fixed4 value) { + validate(fields()[7], value); + this.fixed = value; + fieldSetFlags()[7] = true; + return this; + } + + /** + * Checks whether the 'fixed' field has been set. + * @return True if the 'fixed' field has been set, false otherwise. + */ + public boolean hasFixed() { + return fieldSetFlags()[7]; + } + + + /** + * Clears the value of the 'fixed' field. + * @return This builder. + */ + public Builder clearFixed() { + fixed = null; + fieldSetFlags()[7] = false; + return this; + } + + /** + * Gets the value of the 'date' field. + * @return The value. + */ + public java.time.LocalDate getDate() { + return date; + } + + + /** + * Sets the value of the 'date' field. + * @param value The value of 'date'. + * @return This builder. + */ + public Builder setDate(java.time.LocalDate value) { + validate(fields()[8], value); + this.date = value; + fieldSetFlags()[8] = true; + return this; + } + + /** + * Checks whether the 'date' field has been set. + * @return True if the 'date' field has been set, false otherwise. + */ + public boolean hasDate() { + return fieldSetFlags()[8]; + } + + + /** + * Clears the value of the 'date' field. + * @return This builder. + */ + public Builder clearDate() { + fieldSetFlags()[8] = false; + return this; + } + + /** + * Gets the value of the 'timestampMillis' field. + * @return The value. + */ + public java.time.Instant getTimestampMillis() { + return timestampMillis; + } + + + /** + * Sets the value of the 'timestampMillis' field. + * @param value The value of 'timestampMillis'. + * @return This builder. + */ + public Builder setTimestampMillis(java.time.Instant value) { + validate(fields()[9], value); + this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); + fieldSetFlags()[9] = true; + return this; + } + + /** + * Checks whether the 'timestampMillis' field has been set. + * @return True if the 'timestampMillis' field has been set, false otherwise. + */ + public boolean hasTimestampMillis() { + return fieldSetFlags()[9]; + } + + + /** + * Clears the value of the 'timestampMillis' field. + * @return This builder. + */ + public Builder clearTimestampMillis() { + fieldSetFlags()[9] = false; + return this; + } + + /** + * Gets the value of the 'TestEnum' field. + * @return The value. + */ + public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { + return TestEnum; + } + + + /** + * Sets the value of the 'TestEnum' field. + * @param value The value of 'TestEnum'. + * @return This builder. + */ + public Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { + validate(fields()[10], value); + this.TestEnum = value; + fieldSetFlags()[10] = true; + return this; + } + + /** + * Checks whether the 'TestEnum' field has been set. + * @return True if the 'TestEnum' field has been set, false otherwise. + */ + public boolean hasTestEnum() { + return fieldSetFlags()[10]; + } + + + /** + * Clears the value of the 'TestEnum' field. + * @return This builder. + */ + public Builder clearTestEnum() { + TestEnum = null; + fieldSetFlags()[10] = false; + return this; + } + + /** + * Gets the value of the 'row' field. + * @return The value. + */ + public TestAvroNested getRow() { + return row; + } + + + /** + * Sets the value of the 'row' field. + * @param value The value of 'row'. + * @return This builder. + */ + public Builder setRow(TestAvroNested value) { + validate(fields()[11], value); + this.rowBuilder = null; + this.row = value; + fieldSetFlags()[11] = true; + return this; + } + + /** + * Checks whether the 'row' field has been set. + * @return True if the 'row' field has been set, false otherwise. + */ + public boolean hasRow() { + return fieldSetFlags()[11]; + } + + /** + * Gets the Builder instance for the 'row' field and creates one if it doesn't exist yet. + * @return This builder. + */ + public TestAvroNested.Builder getRowBuilder() { + if (rowBuilder == null) { + if (hasRow()) { + setRowBuilder(TestAvroNested.newBuilder(row)); + } else { + setRowBuilder(TestAvroNested.newBuilder()); + } + } + return rowBuilder; + } + + /** + * Sets the Builder instance for the 'row' field + * @param value The builder instance that must be set. + * @return This builder. + */ + public Builder setRowBuilder(TestAvroNested.Builder value) { + clearRow(); + rowBuilder = value; + return this; + } + + /** + * Checks whether the 'row' field has an active Builder instance + * @return True if the 'row' field has an active Builder instance + */ + public boolean hasRowBuilder() { + return rowBuilder != null; + } + + /** + * Clears the value of the 'row' field. + * @return This builder. + */ + public Builder clearRow() { + row = null; + rowBuilder = null; + fieldSetFlags()[11] = false; + return this; + } + + /** + * Gets the value of the 'array' field. + * @return The value. + */ + public java.util.List getArray() { + return array; + } + + + /** + * Sets the value of the 'array' field. + * @param value The value of 'array'. + * @return This builder. + */ + public Builder setArray(java.util.List value) { + validate(fields()[12], value); + this.array = value; + fieldSetFlags()[12] = true; + return this; + } + + /** + * Checks whether the 'array' field has been set. + * @return True if the 'array' field has been set, false otherwise. + */ + public boolean hasArray() { + return fieldSetFlags()[12]; + } + + + /** + * Clears the value of the 'array' field. + * @return This builder. + */ + public Builder clearArray() { + array = null; + fieldSetFlags()[12] = false; + return this; + } + + /** + * Gets the value of the 'map' field. + * @return The value. + */ + public java.util.Map getMap() { + return map; + } + + + /** + * Sets the value of the 'map' field. + * @param value The value of 'map'. + * @return This builder. + */ + public Builder setMap(java.util.Map value) { + validate(fields()[13], value); + this.map = value; + fieldSetFlags()[13] = true; + return this; + } + + /** + * Checks whether the 'map' field has been set. + * @return True if the 'map' field has been set, false otherwise. + */ + public boolean hasMap() { + return fieldSetFlags()[13]; + } + + + /** + * Clears the value of the 'map' field. + * @return This builder. + */ + public Builder clearMap() { + map = null; + fieldSetFlags()[13] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public TestAvro build() { + try { + TestAvro record = new TestAvro(); + record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); + record.long$ = fieldSetFlags()[2] ? this.long$ : (Long) defaultValue(fields()[2]); + record.float$ = fieldSetFlags()[3] ? this.float$ : (Float) defaultValue(fields()[3]); + record.double$ = fieldSetFlags()[4] ? this.double$ : (Double) defaultValue(fields()[4]); + record.string = fieldSetFlags()[5] ? this.string : (CharSequence) defaultValue(fields()[5]); + record.bytes = fieldSetFlags()[6] ? this.bytes : (java.nio.ByteBuffer) defaultValue(fields()[6]); + record.fixed = fieldSetFlags()[7] ? this.fixed : (fixed4) defaultValue(fields()[7]); + record.date = fieldSetFlags()[8] ? this.date : (java.time.LocalDate) defaultValue(fields()[8]); + record.timestampMillis = fieldSetFlags()[9] ? this.timestampMillis : (java.time.Instant) defaultValue(fields()[9]); + record.TestEnum = fieldSetFlags()[10] ? this.TestEnum : (org.apache.beam.sdk.extensions.avro.schemas.TestEnum) defaultValue(fields()[10]); + if (rowBuilder != null) { + try { + record.row = this.rowBuilder.build(); + } catch (org.apache.avro.AvroMissingFieldException e) { + e.addParentField(record.getSchema().getField("row")); + throw e; + } + } else { + record.row = fieldSetFlags()[11] ? this.row : (TestAvroNested) defaultValue(fields()[11]); + } + record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); + record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); + return record; + } catch (org.apache.avro.AvroMissingFieldException e) { + throw e; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + +} + + + + + + + + + + diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java new file mode 100644 index 000000000000..179fe0c29736 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java @@ -0,0 +1,416 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; + +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.SchemaStore; +import org.apache.avro.specific.SpecificData; + +@org.apache.avro.specific.AvroGenerated +public class TestAvroNested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 4633138088036298925L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvroNested\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static SpecificData MODEL$ = new SpecificData(); + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageEncoder instance used by this class. + * @return the message encoder used by this class + */ + public static BinaryMessageEncoder getEncoder() { + return ENCODER; + } + + /** + * Return the BinaryMessageDecoder instance used by this class. + * @return the message decoder used by this class + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * 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 + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); + } + + /** + * Serializes this TestAvroNested to a ByteBuffer. + * @return a buffer holding the serialized data for this instance + * @throws java.io.IOException if this instance could not be serialized + */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** + * Deserializes a TestAvroNested from a ByteBuffer. + * @param b a byte buffer holding serialized data for an instance of this class + * @return a TestAvroNested instance decoded from the given buffer + * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class + */ + public static TestAvroNested fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + private boolean BOOL_NON_NULLABLE; + private Integer int$; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public TestAvroNested() {} + + /** + * All-args constructor. + * @param BOOL_NON_NULLABLE The new value for BOOL_NON_NULLABLE + * @param int$ The new value for int + */ + public TestAvroNested(Boolean BOOL_NON_NULLABLE, Integer int$) { + this.BOOL_NON_NULLABLE = BOOL_NON_NULLABLE; + this.int$ = int$; + } + + public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public Object get(int field$) { + switch (field$) { + case 0: return BOOL_NON_NULLABLE; + case 1: return int$; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, Object value$) { + switch (field$) { + case 0: BOOL_NON_NULLABLE = (Boolean)value$; break; + case 1: int$ = (Integer)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'BOOL_NON_NULLABLE' field. + * @return The value of the 'BOOL_NON_NULLABLE' field. + */ + public boolean getBOOLNONNULLABLE() { + return BOOL_NON_NULLABLE; + } + + + /** + * Sets the value of the 'BOOL_NON_NULLABLE' field. + * @param value the value to set. + */ + public void setBOOLNONNULLABLE(boolean value) { + this.BOOL_NON_NULLABLE = value; + } + + /** + * Gets the value of the 'int$' field. + * @return The value of the 'int$' field. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value the value to set. + */ + public void setInt$(Integer value) { + this.int$ = value; + } + + /** + * Creates a new TestAvroNested RecordBuilder. + * @return A new TestAvroNested RecordBuilder + */ + public static Builder newBuilder() { + return new Builder(); + } + + /** + * Creates a new TestAvroNested RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new TestAvroNested RecordBuilder + */ + public static Builder newBuilder(Builder other) { + if (other == null) { + return new Builder(); + } else { + return new Builder(other); + } + } + + /** + * Creates a new TestAvroNested RecordBuilder by copying an existing TestAvroNested instance. + * @param other The existing instance to copy. + * @return A new TestAvroNested RecordBuilder + */ + public static Builder newBuilder(TestAvroNested other) { + if (other == null) { + return new Builder(); + } else { + return new Builder(other); + } + } + + /** + * RecordBuilder for TestAvroNested instances. + */ + @org.apache.avro.specific.AvroGenerated + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean BOOL_NON_NULLABLE; + private Integer int$; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(Builder other) { + super(other); + if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { + this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); + fieldSetFlags()[0] = other.fieldSetFlags()[0]; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = other.fieldSetFlags()[1]; + } + } + + /** + * Creates a Builder by copying an existing TestAvroNested instance + * @param other The existing instance to copy. + */ + private Builder(TestAvroNested other) { + super(SCHEMA$); + if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { + this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int$)) { + this.int$ = data().deepCopy(fields()[1].schema(), other.int$); + fieldSetFlags()[1] = true; + } + } + + /** + * Gets the value of the 'BOOL_NON_NULLABLE' field. + * @return The value. + */ + public boolean getBOOLNONNULLABLE() { + return BOOL_NON_NULLABLE; + } + + + /** + * Sets the value of the 'BOOL_NON_NULLABLE' field. + * @param value The value of 'BOOL_NON_NULLABLE'. + * @return This builder. + */ + public Builder setBOOLNONNULLABLE(boolean value) { + validate(fields()[0], value); + this.BOOL_NON_NULLABLE = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'BOOL_NON_NULLABLE' field has been set. + * @return True if the 'BOOL_NON_NULLABLE' field has been set, false otherwise. + */ + public boolean hasBOOLNONNULLABLE() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'BOOL_NON_NULLABLE' field. + * @return This builder. + */ + public Builder clearBOOLNONNULLABLE() { + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'int$' field. + * @return The value. + */ + public Integer getInt$() { + return int$; + } + + + /** + * Sets the value of the 'int$' field. + * @param value The value of 'int$'. + * @return This builder. + */ + public Builder setInt$(Integer value) { + validate(fields()[1], value); + this.int$ = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'int$' field has been set. + * @return True if the 'int$' field has been set, false otherwise. + */ + public boolean hasInt$() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'int$' field. + * @return This builder. + */ + public Builder clearInt$() { + int$ = null; + fieldSetFlags()[1] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public TestAvroNested build() { + try { + TestAvroNested record = new TestAvroNested(); + record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); + return record; + } catch (org.apache.avro.AvroMissingFieldException e) { + throw e; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + + @Override protected boolean hasCustomCoders() { return true; } + + @Override public void customEncode(org.apache.avro.io.Encoder out) + throws java.io.IOException + { + out.writeBoolean(this.BOOL_NON_NULLABLE); + + if (this.int$ == null) { + out.writeIndex(1); + out.writeNull(); + } else { + out.writeIndex(0); + out.writeInt(this.int$); + } + + } + + @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) + throws java.io.IOException + { + org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); + if (fieldOrder == null) { + this.BOOL_NON_NULLABLE = in.readBoolean(); + + if (in.readIndex() != 0) { + in.readNull(); + this.int$ = null; + } else { + this.int$ = in.readInt(); + } + + } else { + for (int i = 0; i < 2; i++) { + switch (fieldOrder[i].pos()) { + case 0: + this.BOOL_NON_NULLABLE = in.readBoolean(); + break; + + case 1: + if (in.readIndex() != 0) { + in.readNull(); + this.int$ = null; + } else { + this.int$ = in.readInt(); + } + break; + + default: + throw new java.io.IOException("Corrupt ResolvingDecoder."); + } + } + } + } +} + + + + + + + + + + diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java new file mode 100644 index 000000000000..8d1e1d7dffe9 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java @@ -0,0 +1,25 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; +@org.apache.avro.specific.AvroGenerated +public enum TestEnum implements org.apache.avro.generic.GenericEnumSymbol { + abc, cde ; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"enum\",\"name\":\"TestEnum\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"symbols\":[\"abc\",\"cde\"]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } +} diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java new file mode 100644 index 000000000000..24c5be9ca18d --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java @@ -0,0 +1,56 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; +@org.apache.avro.specific.FixedSize(4) +@org.apache.avro.specific.AvroGenerated +public class fixed4 extends org.apache.avro.specific.SpecificFixed { + private static final long serialVersionUID = -5646354132642432749L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"fixed\",\"name\":\"fixed4\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"size\":4}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + + /** Creates a new fixed4 */ + public fixed4() { + super(); + } + + /** + * Creates a new fixed4 with the given bytes. + * @param bytes The bytes to create the new fixed4. + */ + public fixed4(byte[] bytes) { + super(bytes); + } + + private static final org.apache.avro.io.DatumWriter + WRITER$ = new org.apache.avro.specific.SpecificDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, org.apache.avro.specific.SpecificData.getEncoder(out)); + } + + private static final org.apache.avro.io.DatumReader + READER$ = new org.apache.avro.specific.SpecificDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, org.apache.avro.specific.SpecificData.getDecoder(in)); + } + +} From 1bcb859bfc1f2e17af9fce48df23e846d617acbf Mon Sep 17 00:00:00 2001 From: Moritz Mack Date: Wed, 22 Mar 2023 18:06:14 +0100 Subject: [PATCH 4/6] Fix type converters in AvroUtils to support java time --- .../sdk/schemas/utils/ByteBuddyUtils.java | 5 + .../avro/schemas/utils/AvroUtils.java | 105 ++++++++++++++++-- 2 files changed, 101 insertions(+), 9 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java index bb0003c4d4dc..6bcb98cff4ab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java @@ -283,6 +283,11 @@ public T convert(TypeDescriptor typeDescriptor) { } } + protected StackManipulation shortCircuitReturnNull( + StackManipulation readValue, StackManipulation onNotNull) { + return new ShortCircuitReturnNull(readValue, onNotNull); + } + protected abstract T convertArray(TypeDescriptor type); protected abstract T convertIterable(TypeDescriptor type); diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java index fcd8c11089b8..f062326b2da4 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java @@ -34,14 +34,18 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import javax.annotation.Nonnull; import net.bytebuddy.description.type.TypeDescription.ForLoadedType; +import net.bytebuddy.implementation.bytecode.Division; import net.bytebuddy.implementation.bytecode.Duplication; +import net.bytebuddy.implementation.bytecode.Multiplication; import net.bytebuddy.implementation.bytecode.StackManipulation; import net.bytebuddy.implementation.bytecode.StackManipulation.Compound; import net.bytebuddy.implementation.bytecode.TypeCreation; import net.bytebuddy.implementation.bytecode.assign.TypeCasting; +import net.bytebuddy.implementation.bytecode.constant.LongConstant; import net.bytebuddy.implementation.bytecode.member.MethodInvocation; import net.bytebuddy.matcher.ElementMatchers; import org.apache.avro.AvroRuntimeException; @@ -144,6 +148,7 @@ "rawtypes" }) public class AvroUtils { + static { // This works around a bug in the Avro library (AVRO-1891) around SpecificRecord's handling // of DateTime types. @@ -151,6 +156,14 @@ public class AvroUtils { GenericData.get().addLogicalTypeConversion(new AvroCoder.JodaTimestampConversion()); } + private static final ForLoadedType BYTES = new ForLoadedType(byte[].class); + private static final ForLoadedType JAVA_INSTANT = new ForLoadedType(java.time.Instant.class); + private static final ForLoadedType JAVA_LOCALE_DATE = + new ForLoadedType(java.time.LocalDate.class); + private static final ForLoadedType JODA_READABLE_INSTANT = + new ForLoadedType(ReadableInstant.class); + private static final ForLoadedType JODA_INSTANT = new ForLoadedType(Instant.class); + // Unwrap an AVRO schema into the base type an whether it is nullable. public static class TypeWithNullability { public final org.apache.avro.Schema type; @@ -254,7 +267,10 @@ public AvroConvertType(boolean returnRawType) { @Override protected java.lang.reflect.Type convertDefault(TypeDescriptor type) { - if (type.isSubtypeOf(TypeDescriptor.of(GenericFixed.class))) { + if (type.isSubtypeOf(TypeDescriptor.of(java.time.Instant.class)) + || type.isSubtypeOf(TypeDescriptor.of(java.time.LocalDate.class))) { + return convertDateTime(type); + } else if (type.isSubtypeOf(TypeDescriptor.of(GenericFixed.class))) { return byte[].class; } else { return super.convertDefault(type); @@ -282,10 +298,46 @@ protected StackManipulation convertDefault(TypeDescriptor type) { MethodInvocation.invoke( new ForLoadedType(GenericFixed.class) .getDeclaredMethods() - .filter( - ElementMatchers.named("bytes") - .and(ElementMatchers.returns(new ForLoadedType(byte[].class)))) + .filter(ElementMatchers.named("bytes").and(ElementMatchers.returns(BYTES))) .getOnly())); + } else if (java.time.Instant.class.isAssignableFrom(type.getRawType())) { + // Generates the following code: + // return Instant.ofEpochMilli(value.toEpochMilli()) + StackManipulation onNotNull = + new Compound( + readValue, + MethodInvocation.invoke( + JAVA_INSTANT + .getDeclaredMethods() + .filter(ElementMatchers.named("toEpochMilli")) + .getOnly()), + MethodInvocation.invoke( + JODA_INSTANT + .getDeclaredMethods() + .filter( + ElementMatchers.isStatic().and(ElementMatchers.named("ofEpochMilli"))) + .getOnly())); + return shortCircuitReturnNull(readValue, onNotNull); + } else if (java.time.LocalDate.class.isAssignableFrom(type.getRawType())) { + // Generates the following code: + // return Instant.ofEpochMilli(value.toEpochDay() * TimeUnit.DAYS.toMillis(1)) + StackManipulation onNotNull = + new Compound( + readValue, + MethodInvocation.invoke( + JAVA_LOCALE_DATE + .getDeclaredMethods() + .filter(ElementMatchers.named("toEpochDay")) + .getOnly()), + LongConstant.forValue(TimeUnit.DAYS.toMillis(1)), + Multiplication.LONG, + MethodInvocation.invoke( + JODA_INSTANT + .getDeclaredMethods() + .filter( + ElementMatchers.isStatic().and(ElementMatchers.named("ofEpochMilli"))) + .getOnly())); + return shortCircuitReturnNull(readValue, onNotNull); } return super.convertDefault(type); } @@ -303,25 +355,60 @@ protected TypeConversionsFactory getFactory() { @Override protected StackManipulation convertDefault(TypeDescriptor type) { - final ForLoadedType byteArrayType = new ForLoadedType(byte[].class); if (type.isSubtypeOf(TypeDescriptor.of(GenericFixed.class))) { // Generate the following code: - // return new T((byte[]) value); + // return new T((byte[]) value); ForLoadedType loadedType = new ForLoadedType(type.getRawType()); return new Compound( TypeCreation.of(loadedType), Duplication.SINGLE, // Load the parameter and cast it to a byte[]. readValue, - TypeCasting.to(byteArrayType), + TypeCasting.to(BYTES), // Create a new instance that wraps this byte[]. MethodInvocation.invoke( loadedType .getDeclaredMethods() .filter( - ElementMatchers.isConstructor() - .and(ElementMatchers.takesArguments(byteArrayType))) + ElementMatchers.isConstructor().and(ElementMatchers.takesArguments(BYTES))) .getOnly())); + } else if (java.time.Instant.class.isAssignableFrom(type.getRawType())) { + // Generates the following code: + // return java.time.Instant.ofEpochMilli(value.getMillis()) + StackManipulation onNotNull = + new Compound( + readValue, + MethodInvocation.invoke( + JODA_READABLE_INSTANT + .getDeclaredMethods() + .filter(ElementMatchers.named("getMillis")) + .getOnly()), + MethodInvocation.invoke( + JAVA_INSTANT + .getDeclaredMethods() + .filter( + ElementMatchers.isStatic().and(ElementMatchers.named("ofEpochMilli"))) + .getOnly())); + return shortCircuitReturnNull(readValue, onNotNull); + } else if (java.time.LocalDate.class.isAssignableFrom(type.getRawType())) { + // Generates the following code: + // return java.time.LocalDate.ofEpochDay(value.getMillis() / TimeUnit.DAYS.toMillis(1)) + StackManipulation onNotNull = + new Compound( + readValue, + MethodInvocation.invoke( + JODA_READABLE_INSTANT + .getDeclaredMethods() + .filter(ElementMatchers.named("getMillis")) + .getOnly()), + LongConstant.forValue(TimeUnit.DAYS.toMillis(1)), + Division.LONG, + MethodInvocation.invoke( + JAVA_LOCALE_DATE + .getDeclaredMethods() + .filter(ElementMatchers.isStatic().and(ElementMatchers.named("ofEpochDay"))) + .getOnly())); + return shortCircuitReturnNull(readValue, onNotNull); } return super.convertDefault(type); } From 03aa4286856458c2dbbdb02423b8db6ee22e419b Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Fri, 24 Mar 2023 16:54:28 +0100 Subject: [PATCH 5/6] Add TestAvro and AvroGeneratedUser factory classes --- .test-infra/jenkins/job_PreCommit_Java.groovy | 3 + sdks/java/extensions/avro/build.gradle | 12 +- .../extensions/avro/io/AvroGeneratedUser.java | 515 +++++++++++++++++ .../sdk/extensions/avro/schemas/TestAvro.java | 155 ++---- .../avro/schemas/TestAvroNested.java | 36 +- .../extensions/avro/io/AvroGeneratedUser.java | 523 ++++++++++++++++++ .../sdk/extensions/avro/schemas/TestAvro.java | 273 ++++----- .../avro/schemas/TestAvroNested.java | 62 ++- .../extensions/avro/io/AvroGeneratedUser.java | 515 +++++++++++++++++ .../sdk/extensions/avro/schemas/TestAvro.java | 273 ++++----- .../avro/schemas/TestAvroNested.java | 62 ++- .../extensions/avro/coders/AvroCoderTest.java | 59 ++ .../avro/coders/SpecificRecordTest.java | 79 --- .../avro/io/AvroGeneratedUserFactory.java | 49 ++ .../sdk/extensions/avro/io/AvroIOTest.java | 6 +- .../avro/schemas/AvroSchemaTest.java | 2 +- .../avro/schemas/TestAvroFactory.java | 109 ++++ .../avro/schemas/utils/AvroUtilsTest.java | 3 +- 18 files changed, 2168 insertions(+), 568 deletions(-) create mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java create mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java create mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java delete mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/SpecificRecordTest.java create mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUserFactory.java create mode 100644 sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java diff --git a/.test-infra/jenkins/job_PreCommit_Java.groovy b/.test-infra/jenkins/job_PreCommit_Java.groovy index bc94a418d2e9..e1fc2ee1080c 100644 --- a/.test-infra/jenkins/job_PreCommit_Java.groovy +++ b/.test-infra/jenkins/job_PreCommit_Java.groovy @@ -112,6 +112,9 @@ builder.build { execPattern('**/build/jacoco/*.exec') exclusionPattern('**/org/apache/beam/gradle/**,**/org/apache/beam/model/**,' + '**/org/apache/beam/runners/dataflow/worker/windmill/**,**/AutoValue_*,' + + '**/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.*,' + + '**/org/apache/beam/sdk/extensions/avro/schemas/Test*,' + + '**/org/apache/beam/sdk/extensions/avro/schemas/fixed4.*,' + excludePaths.collect {entry -> getModuleNameFromProject(entry) }.join(",") ) } } diff --git a/sdks/java/extensions/avro/build.gradle b/sdks/java/extensions/avro/build.gradle index 108c11214d8a..9ff56981b432 100644 --- a/sdks/java/extensions/avro/build.gradle +++ b/sdks/java/extensions/avro/build.gradle @@ -75,7 +75,10 @@ avroVersions.each {kv -> sourceSets { "avro${kv.key}" { - java.srcDirs "src/test/avro${kv.key}" + java { + srcDirs "src/test/avro${kv.key}" + } + compileClasspath = configurations."avroVersion$kv.key" + sourceSets.test.output + sourceSets.test.compileClasspath runtimeClasspath += compileClasspath + sourceSets.test.runtimeClasspath } @@ -86,6 +89,13 @@ avroVersions.each {kv -> checkerFramework { skipCheckerFramework = true } + checkstyle { + ignoreFailures = true + } + } + + "spotbugsAvro${kv.key}" { + ignoreFailures = true } task "avroVersion${kv.key}Test"(type: Test) { diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java new file mode 100644 index 000000000000..3920ca5374d4 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java @@ -0,0 +1,515 @@ +/* + * 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.beam.sdk.extensions.avro.io; + +import org.apache.avro.generic.GenericArray; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.util.Utf8; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.SchemaStore; + +@org.apache.avro.specific.AvroGenerated +public class AvroGeneratedUser extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 3056733241446362297L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroGeneratedUser\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.io\",\"fields\":[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"favorite_number\",\"type\":[\"int\",\"null\"]},{\"name\":\"favorite_color\",\"type\":[\"string\",\"null\"]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static SpecificData MODEL$ = new SpecificData(); + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageEncoder instance used by this class. + * @return the message encoder used by this class + */ + public static BinaryMessageEncoder getEncoder() { + return ENCODER; + } + + /** + * Return the BinaryMessageDecoder instance used by this class. + * @return the message decoder used by this class + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * 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 + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); + } + + /** + * Serializes this AvroGeneratedUser to a ByteBuffer. + * @return a buffer holding the serialized data for this instance + * @throws java.io.IOException if this instance could not be serialized + */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** + * Deserializes a AvroGeneratedUser from a ByteBuffer. + * @param b a byte buffer holding serialized data for an instance of this class + * @return a AvroGeneratedUser instance decoded from the given buffer + * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class + */ + public static AvroGeneratedUser fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + private java.lang.CharSequence name; + private java.lang.Integer favorite_number; + private java.lang.CharSequence favorite_color; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public AvroGeneratedUser() {} + + /** + * All-args constructor. + * @param name The new value for name + * @param favorite_number The new value for favorite_number + * @param favorite_color The new value for favorite_color + */ + public AvroGeneratedUser(java.lang.CharSequence name, java.lang.Integer favorite_number, java.lang.CharSequence favorite_color) { + this.name = name; + this.favorite_number = favorite_number; + this.favorite_color = favorite_color; + } + + public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return name; + case 1: return favorite_number; + case 2: return favorite_color; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: name = (java.lang.CharSequence)value$; break; + case 1: favorite_number = (java.lang.Integer)value$; break; + case 2: favorite_color = (java.lang.CharSequence)value$; break; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + /** + * Gets the value of the 'name' field. + * @return The value of the 'name' field. + */ + public java.lang.CharSequence getName() { + return name; + } + + + /** + * Sets the value of the 'name' field. + * @param value the value to set. + */ + public void setName(java.lang.CharSequence value) { + this.name = value; + } + + /** + * Gets the value of the 'favorite_number' field. + * @return The value of the 'favorite_number' field. + */ + public java.lang.Integer getFavoriteNumber() { + return favorite_number; + } + + + /** + * Sets the value of the 'favorite_number' field. + * @param value the value to set. + */ + public void setFavoriteNumber(java.lang.Integer value) { + this.favorite_number = value; + } + + /** + * Gets the value of the 'favorite_color' field. + * @return The value of the 'favorite_color' field. + */ + public java.lang.CharSequence getFavoriteColor() { + return favorite_color; + } + + + /** + * Sets the value of the 'favorite_color' field. + * @param value the value to set. + */ + public void setFavoriteColor(java.lang.CharSequence value) { + this.favorite_color = value; + } + + /** + * Creates a new AvroGeneratedUser RecordBuilder. + * @return A new AvroGeneratedUser RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder() { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); + } + + /** + * Creates a new AvroGeneratedUser RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new AvroGeneratedUser RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); + } + } + + /** + * Creates a new AvroGeneratedUser RecordBuilder by copying an existing AvroGeneratedUser instance. + * @param other The existing instance to copy. + * @return A new AvroGeneratedUser RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); + } + } + + /** + * RecordBuilder for AvroGeneratedUser instances. + */ + @org.apache.avro.specific.AvroGenerated + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private java.lang.CharSequence name; + private java.lang.Integer favorite_number; + private java.lang.CharSequence favorite_color; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { + super(other); + if (isValidValue(fields()[0], other.name)) { + this.name = data().deepCopy(fields()[0].schema(), other.name); + fieldSetFlags()[0] = other.fieldSetFlags()[0]; + } + if (isValidValue(fields()[1], other.favorite_number)) { + this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); + fieldSetFlags()[1] = other.fieldSetFlags()[1]; + } + if (isValidValue(fields()[2], other.favorite_color)) { + this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); + fieldSetFlags()[2] = other.fieldSetFlags()[2]; + } + } + + /** + * Creates a Builder by copying an existing AvroGeneratedUser instance + * @param other The existing instance to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { + super(SCHEMA$); + if (isValidValue(fields()[0], other.name)) { + this.name = data().deepCopy(fields()[0].schema(), other.name); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.favorite_number)) { + this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.favorite_color)) { + this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); + fieldSetFlags()[2] = true; + } + } + + /** + * Gets the value of the 'name' field. + * @return The value. + */ + public java.lang.CharSequence getName() { + return name; + } + + + /** + * Sets the value of the 'name' field. + * @param value The value of 'name'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setName(java.lang.CharSequence value) { + validate(fields()[0], value); + this.name = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'name' field has been set. + * @return True if the 'name' field has been set, false otherwise. + */ + public boolean hasName() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'name' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearName() { + name = null; + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'favorite_number' field. + * @return The value. + */ + public java.lang.Integer getFavoriteNumber() { + return favorite_number; + } + + + /** + * Sets the value of the 'favorite_number' field. + * @param value The value of 'favorite_number'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteNumber(java.lang.Integer value) { + validate(fields()[1], value); + this.favorite_number = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'favorite_number' field has been set. + * @return True if the 'favorite_number' field has been set, false otherwise. + */ + public boolean hasFavoriteNumber() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'favorite_number' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteNumber() { + favorite_number = null; + fieldSetFlags()[1] = false; + return this; + } + + /** + * Gets the value of the 'favorite_color' field. + * @return The value. + */ + public java.lang.CharSequence getFavoriteColor() { + return favorite_color; + } + + + /** + * Sets the value of the 'favorite_color' field. + * @param value The value of 'favorite_color'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteColor(java.lang.CharSequence value) { + validate(fields()[2], value); + this.favorite_color = value; + fieldSetFlags()[2] = true; + return this; + } + + /** + * Checks whether the 'favorite_color' field has been set. + * @return True if the 'favorite_color' field has been set, false otherwise. + */ + public boolean hasFavoriteColor() { + return fieldSetFlags()[2]; + } + + + /** + * Clears the value of the 'favorite_color' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteColor() { + favorite_color = null; + fieldSetFlags()[2] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public AvroGeneratedUser build() { + try { + AvroGeneratedUser record = new AvroGeneratedUser(); + record.name = fieldSetFlags()[0] ? this.name : (java.lang.CharSequence) defaultValue(fields()[0]); + record.favorite_number = fieldSetFlags()[1] ? this.favorite_number : (java.lang.Integer) defaultValue(fields()[1]); + record.favorite_color = fieldSetFlags()[2] ? this.favorite_color : (java.lang.CharSequence) defaultValue(fields()[2]); + return record; + } catch (org.apache.avro.AvroMissingFieldException e) { + throw e; + } catch (java.lang.Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + + @Override protected boolean hasCustomCoders() { return true; } + + @Override public void customEncode(org.apache.avro.io.Encoder out) + throws java.io.IOException + { + out.writeString(this.name); + + if (this.favorite_number == null) { + out.writeIndex(1); + out.writeNull(); + } else { + out.writeIndex(0); + out.writeInt(this.favorite_number); + } + + if (this.favorite_color == null) { + out.writeIndex(1); + out.writeNull(); + } else { + out.writeIndex(0); + out.writeString(this.favorite_color); + } + + } + + @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) + throws java.io.IOException + { + org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); + if (fieldOrder == null) { + this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); + + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_number = null; + } else { + this.favorite_number = in.readInt(); + } + + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_color = null; + } else { + this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); + } + + } else { + for (int i = 0; i < 3; i++) { + switch (fieldOrder[i].pos()) { + case 0: + this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); + break; + + case 1: + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_number = null; + } else { + this.favorite_number = in.readInt(); + } + break; + + case 2: + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_color = null; + } else { + this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); + } + break; + + default: + throw new java.io.IOException("Corrupt ResolvingDecoder."); + } + } + } + } +} + + + + + + + + + + diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java index 519ebd573979..c352e89410b9 100644 --- a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java +++ b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.extensions.avro.schemas; -import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.util.Utf8; import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; import org.apache.avro.message.SchemaStore; -import org.apache.avro.specific.SpecificData; - -import java.time.Instant; -import java.time.LocalDate; -import java.util.Map; @org.apache.avro.specific.AvroGenerated public class TestAvro extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { @@ -90,11 +88,11 @@ public static TestAvro fromByteBuffer( } private boolean bool_non_nullable; - private Integer int$; - private Long long$; - private Float float$; - private Double double$; - private CharSequence string; + private java.lang.Integer int$; + private java.lang.Long long$; + private java.lang.Float float$; + private java.lang.Double double$; + private java.lang.CharSequence string; private java.nio.ByteBuffer bytes; private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; private java.time.LocalDate date; @@ -102,7 +100,7 @@ public static TestAvro fromByteBuffer( private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; private java.util.List array; - private java.util.Map map; + private java.util.Map map; /** * Default constructor. Note that this does not initialize fields @@ -128,7 +126,7 @@ public TestAvro() {} * @param array The new value for array * @param map The new value for map */ - public TestAvro(Boolean bool_non_nullable, Integer int$, Long long$, Float float$, Double double$, CharSequence string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { + public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.CharSequence string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { this.bool_non_nullable = bool_non_nullable; this.int$ = int$; this.long$ = long$; @@ -145,45 +143,10 @@ public TestAvro(Boolean bool_non_nullable, Integer int$, Long long$, Float float this.map = map; } - /** - * Manually added a ompatible with Avro v1.8.2 API constructor - * - * @param bool_non_nullable - * @param int$ - * @param long$ - * @param float$ - * @param double$ - * @param string - * @param bytes - * @param fixed - * @param date - * @param timestampMillis - * @param TestEnum - * @param row - * @param array - * @param map - */ - public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.String string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, org.joda.time.LocalDate date, org.joda.time.DateTime timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { - this.bool_non_nullable = bool_non_nullable; - this.int$ = int$; - this.long$ = long$; - this.float$ = float$; - this.double$ = double$; - this.string = string; - this.bytes = bytes; - this.fixed = fixed; - this.date = LocalDate.of(date.getYear(), date.getMonthOfYear(), date.getDayOfMonth()); - this.timestampMillis = Instant.ofEpochMilli(timestampMillis.getMillis()); - this.TestEnum = TestEnum; - this.row = row; - this.array = array; - this.map = (Map)map; - } - - public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } public org.apache.avro.Schema getSchema() { return SCHEMA$; } // Used by DatumWriter. Applications should not call. - public Object get(int field$) { + public java.lang.Object get(int field$) { switch (field$) { case 0: return bool_non_nullable; case 1: return int$; @@ -229,14 +192,14 @@ public org.apache.avro.Conversion getConversion(int field) { // Used by DatumReader. Applications should not call. @SuppressWarnings(value="unchecked") - public void put(int field$, Object value$) { + public void put(int field$, java.lang.Object value$) { switch (field$) { - case 0: bool_non_nullable = (Boolean)value$; break; - case 1: int$ = (Integer)value$; break; - case 2: long$ = (Long)value$; break; - case 3: float$ = (Float)value$; break; - case 4: double$ = (Double)value$; break; - case 5: string = (CharSequence)value$; break; + case 0: bool_non_nullable = (java.lang.Boolean)value$; break; + case 1: int$ = (java.lang.Integer)value$; break; + case 2: long$ = (java.lang.Long)value$; break; + case 3: float$ = (java.lang.Float)value$; break; + case 4: double$ = (java.lang.Double)value$; break; + case 5: string = (java.lang.CharSequence)value$; break; case 6: bytes = (java.nio.ByteBuffer)value$; break; case 7: fixed = (org.apache.beam.sdk.extensions.avro.schemas.fixed4)value$; break; case 8: date = (java.time.LocalDate)value$; break; @@ -244,7 +207,7 @@ public void put(int field$, Object value$) { case 10: TestEnum = (org.apache.beam.sdk.extensions.avro.schemas.TestEnum)value$; break; case 11: row = (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested)value$; break; case 12: array = (java.util.List)value$; break; - case 13: map = (java.util.Map)value$; break; + case 13: map = (java.util.Map)value$; break; default: throw new IndexOutOfBoundsException("Invalid index: " + field$); } } @@ -270,7 +233,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'int$' field. * @return The value of the 'int$' field. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -279,7 +242,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'int$' field. * @param value the value to set. */ - public void setInt$(Integer value) { + public void setInt$(java.lang.Integer value) { this.int$ = value; } @@ -287,7 +250,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'long$' field. * @return The value of the 'long$' field. */ - public Long getLong$() { + public java.lang.Long getLong$() { return long$; } @@ -296,7 +259,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'long$' field. * @param value the value to set. */ - public void setLong$(Long value) { + public void setLong$(java.lang.Long value) { this.long$ = value; } @@ -304,7 +267,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'float$' field. * @return The value of the 'float$' field. */ - public Float getFloat$() { + public java.lang.Float getFloat$() { return float$; } @@ -313,7 +276,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'float$' field. * @param value the value to set. */ - public void setFloat$(Float value) { + public void setFloat$(java.lang.Float value) { this.float$ = value; } @@ -321,7 +284,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'double$' field. * @return The value of the 'double$' field. */ - public Double getDouble$() { + public java.lang.Double getDouble$() { return double$; } @@ -330,7 +293,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'double$' field. * @param value the value to set. */ - public void setDouble$(Double value) { + public void setDouble$(java.lang.Double value) { this.double$ = value; } @@ -338,7 +301,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'string' field. * @return The value of the 'string' field. */ - public CharSequence getString() { + public java.lang.CharSequence getString() { return string; } @@ -347,7 +310,7 @@ public CharSequence getString() { * Sets the value of the 'string' field. * @param value the value to set. */ - public void setString(CharSequence value) { + public void setString(java.lang.CharSequence value) { this.string = value; } @@ -474,7 +437,7 @@ public void setArray(java.util.List getMap() { + public java.util.Map getMap() { return map; } @@ -483,7 +446,7 @@ public java.util.Map value) { + public void setMap(java.util.Map value) { this.map = value; } @@ -529,11 +492,11 @@ public static class Builder extends org.apache.avro.specific.SpecificRecordBuild implements org.apache.avro.data.RecordBuilder { private boolean bool_non_nullable; - private Integer int$; - private Long long$; - private Float float$; - private Double double$; - private CharSequence string; + private java.lang.Integer int$; + private java.lang.Long long$; + private java.lang.Float float$; + private java.lang.Double double$; + private java.lang.CharSequence string; private java.nio.ByteBuffer bytes; private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; private java.time.LocalDate date; @@ -542,7 +505,7 @@ public static class Builder extends org.apache.avro.specific.SpecificRecordBuild private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder rowBuilder; private java.util.List array; - private java.util.Map map; + private java.util.Map map; /** Creates a new Builder */ private Builder() { @@ -724,7 +687,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNon * Gets the value of the 'int$' field. * @return The value. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -734,7 +697,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNon * @param value The value of 'int$'. * @return This builder. */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setInt$(Integer value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setInt$(java.lang.Integer value) { validate(fields()[1], value); this.int$ = value; fieldSetFlags()[1] = true; @@ -764,7 +727,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNon * Gets the value of the 'long$' field. * @return The value. */ - public Long getLong$() { + public java.lang.Long getLong$() { return long$; } @@ -774,7 +737,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNon * @param value The value of 'long$'. * @return This builder. */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setLong$(Long value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setLong$(java.lang.Long value) { validate(fields()[2], value); this.long$ = value; fieldSetFlags()[2] = true; @@ -804,7 +767,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNon * Gets the value of the 'float$' field. * @return The value. */ - public Float getFloat$() { + public java.lang.Float getFloat$() { return float$; } @@ -814,7 +777,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNon * @param value The value of 'float$'. * @return This builder. */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFloat$(Float value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFloat$(java.lang.Float value) { validate(fields()[3], value); this.float$ = value; fieldSetFlags()[3] = true; @@ -844,7 +807,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNon * Gets the value of the 'double$' field. * @return The value. */ - public Double getDouble$() { + public java.lang.Double getDouble$() { return double$; } @@ -854,7 +817,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNon * @param value The value of 'double$'. * @return This builder. */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDouble$(Double value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDouble$(java.lang.Double value) { validate(fields()[4], value); this.double$ = value; fieldSetFlags()[4] = true; @@ -884,7 +847,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNon * Gets the value of the 'string' field. * @return The value. */ - public CharSequence getString() { + public java.lang.CharSequence getString() { return string; } @@ -894,7 +857,7 @@ public CharSequence getString() { * @param value The value of 'string'. * @return This builder. */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setString(CharSequence value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setString(java.lang.CharSequence value) { validate(fields()[5], value); this.string = value; fieldSetFlags()[5] = true; @@ -1238,7 +1201,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearArray() * Gets the value of the 'map' field. * @return The value. */ - public java.util.Map getMap() { + public java.util.Map getMap() { return map; } @@ -1248,7 +1211,7 @@ public java.util.Map value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setMap(java.util.Map value) { validate(fields()[13], value); this.map = value; fieldSetFlags()[13] = true; @@ -1279,12 +1242,12 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearMap() { public TestAvro build() { try { TestAvro record = new TestAvro(); - record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); - record.long$ = fieldSetFlags()[2] ? this.long$ : (Long) defaultValue(fields()[2]); - record.float$ = fieldSetFlags()[3] ? this.float$ : (Float) defaultValue(fields()[3]); - record.double$ = fieldSetFlags()[4] ? this.double$ : (Double) defaultValue(fields()[4]); - record.string = fieldSetFlags()[5] ? this.string : (CharSequence) defaultValue(fields()[5]); + record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (java.lang.Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); + record.long$ = fieldSetFlags()[2] ? this.long$ : (java.lang.Long) defaultValue(fields()[2]); + record.float$ = fieldSetFlags()[3] ? this.float$ : (java.lang.Float) defaultValue(fields()[3]); + record.double$ = fieldSetFlags()[4] ? this.double$ : (java.lang.Double) defaultValue(fields()[4]); + record.string = fieldSetFlags()[5] ? this.string : (java.lang.CharSequence) defaultValue(fields()[5]); record.bytes = fieldSetFlags()[6] ? this.bytes : (java.nio.ByteBuffer) defaultValue(fields()[6]); record.fixed = fieldSetFlags()[7] ? this.fixed : (org.apache.beam.sdk.extensions.avro.schemas.fixed4) defaultValue(fields()[7]); record.date = fieldSetFlags()[8] ? this.date : (java.time.LocalDate) defaultValue(fields()[8]); @@ -1301,11 +1264,11 @@ public TestAvro build() { record.row = fieldSetFlags()[11] ? this.row : (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested) defaultValue(fields()[11]); } record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); - record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); + record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); return record; } catch (org.apache.avro.AvroMissingFieldException e) { throw e; - } catch (Exception e) { + } catch (java.lang.Exception e) { throw new org.apache.avro.AvroRuntimeException(e); } } diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java index cf1c2d7af3f0..5e952ca3d6fa 100644 --- a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java +++ b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java @@ -17,10 +17,12 @@ */ package org.apache.beam.sdk.extensions.avro.schemas; -import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.util.Utf8; import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; import org.apache.avro.message.SchemaStore; -import org.apache.avro.specific.SpecificData; @org.apache.avro.specific.AvroGenerated public class TestAvroNested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { @@ -82,7 +84,7 @@ public static TestAvroNested fromByteBuffer( } private boolean BOOL_NON_NULLABLE; - private Integer int$; + private java.lang.Integer int$; /** * Default constructor. Note that this does not initialize fields @@ -96,15 +98,15 @@ public TestAvroNested() {} * @param BOOL_NON_NULLABLE The new value for BOOL_NON_NULLABLE * @param int$ The new value for int */ - public TestAvroNested(Boolean BOOL_NON_NULLABLE, Integer int$) { + public TestAvroNested(java.lang.Boolean BOOL_NON_NULLABLE, java.lang.Integer int$) { this.BOOL_NON_NULLABLE = BOOL_NON_NULLABLE; this.int$ = int$; } - public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } public org.apache.avro.Schema getSchema() { return SCHEMA$; } // Used by DatumWriter. Applications should not call. - public Object get(int field$) { + public java.lang.Object get(int field$) { switch (field$) { case 0: return BOOL_NON_NULLABLE; case 1: return int$; @@ -114,10 +116,10 @@ public Object get(int field$) { // Used by DatumReader. Applications should not call. @SuppressWarnings(value="unchecked") - public void put(int field$, Object value$) { + public void put(int field$, java.lang.Object value$) { switch (field$) { - case 0: BOOL_NON_NULLABLE = (Boolean)value$; break; - case 1: int$ = (Integer)value$; break; + case 0: BOOL_NON_NULLABLE = (java.lang.Boolean)value$; break; + case 1: int$ = (java.lang.Integer)value$; break; default: throw new IndexOutOfBoundsException("Invalid index: " + field$); } } @@ -143,7 +145,7 @@ public void setBOOLNONNULLABLE(boolean value) { * Gets the value of the 'int$' field. * @return The value of the 'int$' field. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -152,7 +154,7 @@ public void setBOOLNONNULLABLE(boolean value) { * Sets the value of the 'int$' field. * @param value the value to set. */ - public void setInt$(Integer value) { + public void setInt$(java.lang.Integer value) { this.int$ = value; } @@ -198,7 +200,7 @@ public static class Builder extends org.apache.avro.specific.SpecificRecordBuild implements org.apache.avro.data.RecordBuilder { private boolean BOOL_NON_NULLABLE; - private Integer int$; + private java.lang.Integer int$; /** Creates a new Builder */ private Builder() { @@ -280,7 +282,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearB * Gets the value of the 'int$' field. * @return The value. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -290,7 +292,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearB * @param value The value of 'int$'. * @return This builder. */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setInt$(Integer value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setInt$(java.lang.Integer value) { validate(fields()[1], value); this.int$ = value; fieldSetFlags()[1] = true; @@ -321,12 +323,12 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearB public TestAvroNested build() { try { TestAvroNested record = new TestAvroNested(); - record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); + record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (java.lang.Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); return record; } catch (org.apache.avro.AvroMissingFieldException e) { throw e; - } catch (Exception e) { + } catch (java.lang.Exception e) { throw new org.apache.avro.AvroRuntimeException(e); } } diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java new file mode 100644 index 000000000000..4badde3f7b01 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java @@ -0,0 +1,523 @@ +/* + * 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.beam.sdk.extensions.avro.io; + +import org.apache.avro.generic.GenericArray; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.util.Utf8; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.SchemaStore; + +@org.apache.avro.specific.AvroGenerated +public class AvroGeneratedUser extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 3056733241446362297L; + + + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroGeneratedUser\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.io\",\"fields\":[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"favorite_number\",\"type\":[\"int\",\"null\"]},{\"name\":\"favorite_color\",\"type\":[\"string\",\"null\"]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static final SpecificData MODEL$ = new SpecificData(); + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder<>(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder<>(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageEncoder instance used by this class. + * @return the message encoder used by this class + */ + public static BinaryMessageEncoder getEncoder() { + return ENCODER; + } + + /** + * Return the BinaryMessageDecoder instance used by this class. + * @return the message decoder used by this class + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * 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 + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder<>(MODEL$, SCHEMA$, resolver); + } + + /** + * Serializes this AvroGeneratedUser to a ByteBuffer. + * @return a buffer holding the serialized data for this instance + * @throws java.io.IOException if this instance could not be serialized + */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** + * Deserializes a AvroGeneratedUser from a ByteBuffer. + * @param b a byte buffer holding serialized data for an instance of this class + * @return a AvroGeneratedUser instance decoded from the given buffer + * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class + */ + public static AvroGeneratedUser fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + private java.lang.CharSequence name; + private java.lang.Integer favorite_number; + private java.lang.CharSequence favorite_color; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public AvroGeneratedUser() {} + + /** + * All-args constructor. + * @param name The new value for name + * @param favorite_number The new value for favorite_number + * @param favorite_color The new value for favorite_color + */ + public AvroGeneratedUser(java.lang.CharSequence name, java.lang.Integer favorite_number, java.lang.CharSequence favorite_color) { + this.name = name; + this.favorite_number = favorite_number; + this.favorite_color = favorite_color; + } + + @Override + public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } + + @Override + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + + // Used by DatumWriter. Applications should not call. + @Override + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return name; + case 1: return favorite_number; + case 2: return favorite_color; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + // Used by DatumReader. Applications should not call. + @Override + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: name = (java.lang.CharSequence)value$; break; + case 1: favorite_number = (java.lang.Integer)value$; break; + case 2: favorite_color = (java.lang.CharSequence)value$; break; + default: throw new IndexOutOfBoundsException("Invalid index: " + field$); + } + } + + /** + * Gets the value of the 'name' field. + * @return The value of the 'name' field. + */ + public java.lang.CharSequence getName() { + return name; + } + + + /** + * Sets the value of the 'name' field. + * @param value the value to set. + */ + public void setName(java.lang.CharSequence value) { + this.name = value; + } + + /** + * Gets the value of the 'favorite_number' field. + * @return The value of the 'favorite_number' field. + */ + public java.lang.Integer getFavoriteNumber() { + return favorite_number; + } + + + /** + * Sets the value of the 'favorite_number' field. + * @param value the value to set. + */ + public void setFavoriteNumber(java.lang.Integer value) { + this.favorite_number = value; + } + + /** + * Gets the value of the 'favorite_color' field. + * @return The value of the 'favorite_color' field. + */ + public java.lang.CharSequence getFavoriteColor() { + return favorite_color; + } + + + /** + * Sets the value of the 'favorite_color' field. + * @param value the value to set. + */ + public void setFavoriteColor(java.lang.CharSequence value) { + this.favorite_color = value; + } + + /** + * Creates a new AvroGeneratedUser RecordBuilder. + * @return A new AvroGeneratedUser RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder() { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); + } + + /** + * Creates a new AvroGeneratedUser RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new AvroGeneratedUser RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); + } + } + + /** + * Creates a new AvroGeneratedUser RecordBuilder by copying an existing AvroGeneratedUser instance. + * @param other The existing instance to copy. + * @return A new AvroGeneratedUser RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); + } + } + + /** + * RecordBuilder for AvroGeneratedUser instances. + */ + @org.apache.avro.specific.AvroGenerated + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private java.lang.CharSequence name; + private java.lang.Integer favorite_number; + private java.lang.CharSequence favorite_color; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$, MODEL$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { + super(other); + if (isValidValue(fields()[0], other.name)) { + this.name = data().deepCopy(fields()[0].schema(), other.name); + fieldSetFlags()[0] = other.fieldSetFlags()[0]; + } + if (isValidValue(fields()[1], other.favorite_number)) { + this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); + fieldSetFlags()[1] = other.fieldSetFlags()[1]; + } + if (isValidValue(fields()[2], other.favorite_color)) { + this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); + fieldSetFlags()[2] = other.fieldSetFlags()[2]; + } + } + + /** + * Creates a Builder by copying an existing AvroGeneratedUser instance + * @param other The existing instance to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { + super(SCHEMA$, MODEL$); + if (isValidValue(fields()[0], other.name)) { + this.name = data().deepCopy(fields()[0].schema(), other.name); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.favorite_number)) { + this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.favorite_color)) { + this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); + fieldSetFlags()[2] = true; + } + } + + /** + * Gets the value of the 'name' field. + * @return The value. + */ + public java.lang.CharSequence getName() { + return name; + } + + + /** + * Sets the value of the 'name' field. + * @param value The value of 'name'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setName(java.lang.CharSequence value) { + validate(fields()[0], value); + this.name = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'name' field has been set. + * @return True if the 'name' field has been set, false otherwise. + */ + public boolean hasName() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'name' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearName() { + name = null; + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'favorite_number' field. + * @return The value. + */ + public java.lang.Integer getFavoriteNumber() { + return favorite_number; + } + + + /** + * Sets the value of the 'favorite_number' field. + * @param value The value of 'favorite_number'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteNumber(java.lang.Integer value) { + validate(fields()[1], value); + this.favorite_number = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'favorite_number' field has been set. + * @return True if the 'favorite_number' field has been set, false otherwise. + */ + public boolean hasFavoriteNumber() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'favorite_number' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteNumber() { + favorite_number = null; + fieldSetFlags()[1] = false; + return this; + } + + /** + * Gets the value of the 'favorite_color' field. + * @return The value. + */ + public java.lang.CharSequence getFavoriteColor() { + return favorite_color; + } + + + /** + * Sets the value of the 'favorite_color' field. + * @param value The value of 'favorite_color'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteColor(java.lang.CharSequence value) { + validate(fields()[2], value); + this.favorite_color = value; + fieldSetFlags()[2] = true; + return this; + } + + /** + * Checks whether the 'favorite_color' field has been set. + * @return True if the 'favorite_color' field has been set, false otherwise. + */ + public boolean hasFavoriteColor() { + return fieldSetFlags()[2]; + } + + + /** + * Clears the value of the 'favorite_color' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteColor() { + favorite_color = null; + fieldSetFlags()[2] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public AvroGeneratedUser build() { + try { + AvroGeneratedUser record = new AvroGeneratedUser(); + record.name = fieldSetFlags()[0] ? this.name : (java.lang.CharSequence) defaultValue(fields()[0]); + record.favorite_number = fieldSetFlags()[1] ? this.favorite_number : (java.lang.Integer) defaultValue(fields()[1]); + record.favorite_color = fieldSetFlags()[2] ? this.favorite_color : (java.lang.CharSequence) defaultValue(fields()[2]); + return record; + } catch (org.apache.avro.AvroMissingFieldException e) { + throw e; + } catch (java.lang.Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + + @Override protected boolean hasCustomCoders() { return true; } + + @Override public void customEncode(org.apache.avro.io.Encoder out) + throws java.io.IOException + { + out.writeString(this.name); + + if (this.favorite_number == null) { + out.writeIndex(1); + out.writeNull(); + } else { + out.writeIndex(0); + out.writeInt(this.favorite_number); + } + + if (this.favorite_color == null) { + out.writeIndex(1); + out.writeNull(); + } else { + out.writeIndex(0); + out.writeString(this.favorite_color); + } + + } + + @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) + throws java.io.IOException + { + org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); + if (fieldOrder == null) { + this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); + + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_number = null; + } else { + this.favorite_number = in.readInt(); + } + + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_color = null; + } else { + this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); + } + + } else { + for (int i = 0; i < 3; i++) { + switch (fieldOrder[i].pos()) { + case 0: + this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); + break; + + case 1: + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_number = null; + } else { + this.favorite_number = in.readInt(); + } + break; + + case 2: + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_color = null; + } else { + this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); + } + break; + + default: + throw new java.io.IOException("Corrupt ResolvingDecoder."); + } + } + } + } +} + + + + + + + + + + diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java index 926bfa9db9b4..f21d86c99bc1 100644 --- a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java +++ b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.extensions.avro.schemas; -import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.util.Utf8; import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; import org.apache.avro.message.SchemaStore; -import org.apache.avro.specific.SpecificData; - -import java.time.Instant; -import java.time.LocalDate; -import java.util.Map; @org.apache.avro.specific.AvroGenerated public class TestAvro extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { @@ -92,19 +90,19 @@ public static TestAvro fromByteBuffer( } private boolean bool_non_nullable; - private Integer int$; - private Long long$; - private Float float$; - private Double double$; - private CharSequence string; + private java.lang.Integer int$; + private java.lang.Long long$; + private java.lang.Float float$; + private java.lang.Double double$; + private java.lang.CharSequence string; private java.nio.ByteBuffer bytes; - private fixed4 fixed; + private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; private java.time.LocalDate date; private java.time.Instant timestampMillis; private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private TestAvroNested row; - private java.util.List array; - private java.util.Map map; + private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; + private java.util.List array; + private java.util.Map map; /** * Default constructor. Note that this does not initialize fields @@ -130,7 +128,7 @@ public TestAvro() {} * @param array The new value for array * @param map The new value for map */ - public TestAvro(Boolean bool_non_nullable, Integer int$, Long long$, Float float$, Double double$, CharSequence string, java.nio.ByteBuffer bytes, fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, TestAvroNested row, java.util.List array, java.util.Map map) { + public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.CharSequence string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { this.bool_non_nullable = bool_non_nullable; this.int$ = int$; this.long$ = long$; @@ -147,50 +145,15 @@ public TestAvro(Boolean bool_non_nullable, Integer int$, Long long$, Float float this.map = map; } - /** - * Manually added a ompatible with Avro v1.8.2 API constructor - * - * @param bool_non_nullable - * @param int$ - * @param long$ - * @param float$ - * @param double$ - * @param string - * @param bytes - * @param fixed - * @param date - * @param timestampMillis - * @param TestEnum - * @param row - * @param array - * @param map - */ - public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.String string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, org.joda.time.LocalDate date, org.joda.time.DateTime timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { - this.bool_non_nullable = bool_non_nullable; - this.int$ = int$; - this.long$ = long$; - this.float$ = float$; - this.double$ = double$; - this.string = string; - this.bytes = bytes; - this.fixed = fixed; - this.date = LocalDate.of(date.getYear(), date.getMonthOfYear(), date.getDayOfMonth()); - this.timestampMillis = Instant.ofEpochMilli(timestampMillis.getMillis()); - this.TestEnum = TestEnum; - this.row = row; - this.array = array; - this.map = (Map)map; - } - @Override - public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } @Override public org.apache.avro.Schema getSchema() { return SCHEMA$; } // Used by DatumWriter. Applications should not call. @Override - public Object get(int field$) { + public java.lang.Object get(int field$) { switch (field$) { case 0: return bool_non_nullable; case 1: return int$; @@ -237,22 +200,22 @@ public org.apache.avro.Conversion getConversion(int field) { // Used by DatumReader. Applications should not call. @Override @SuppressWarnings(value="unchecked") - public void put(int field$, Object value$) { + public void put(int field$, java.lang.Object value$) { switch (field$) { - case 0: bool_non_nullable = (Boolean)value$; break; - case 1: int$ = (Integer)value$; break; - case 2: long$ = (Long)value$; break; - case 3: float$ = (Float)value$; break; - case 4: double$ = (Double)value$; break; - case 5: string = (CharSequence)value$; break; + case 0: bool_non_nullable = (java.lang.Boolean)value$; break; + case 1: int$ = (java.lang.Integer)value$; break; + case 2: long$ = (java.lang.Long)value$; break; + case 3: float$ = (java.lang.Float)value$; break; + case 4: double$ = (java.lang.Double)value$; break; + case 5: string = (java.lang.CharSequence)value$; break; case 6: bytes = (java.nio.ByteBuffer)value$; break; - case 7: fixed = (fixed4)value$; break; + case 7: fixed = (org.apache.beam.sdk.extensions.avro.schemas.fixed4)value$; break; case 8: date = (java.time.LocalDate)value$; break; case 9: timestampMillis = (java.time.Instant)value$; break; case 10: TestEnum = (org.apache.beam.sdk.extensions.avro.schemas.TestEnum)value$; break; - case 11: row = (TestAvroNested)value$; break; - case 12: array = (java.util.List)value$; break; - case 13: map = (java.util.Map)value$; break; + case 11: row = (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested)value$; break; + case 12: array = (java.util.List)value$; break; + case 13: map = (java.util.Map)value$; break; default: throw new IndexOutOfBoundsException("Invalid index: " + field$); } } @@ -278,7 +241,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'int$' field. * @return The value of the 'int$' field. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -287,7 +250,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'int$' field. * @param value the value to set. */ - public void setInt$(Integer value) { + public void setInt$(java.lang.Integer value) { this.int$ = value; } @@ -295,7 +258,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'long$' field. * @return The value of the 'long$' field. */ - public Long getLong$() { + public java.lang.Long getLong$() { return long$; } @@ -304,7 +267,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'long$' field. * @param value the value to set. */ - public void setLong$(Long value) { + public void setLong$(java.lang.Long value) { this.long$ = value; } @@ -312,7 +275,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'float$' field. * @return The value of the 'float$' field. */ - public Float getFloat$() { + public java.lang.Float getFloat$() { return float$; } @@ -321,7 +284,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'float$' field. * @param value the value to set. */ - public void setFloat$(Float value) { + public void setFloat$(java.lang.Float value) { this.float$ = value; } @@ -329,7 +292,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'double$' field. * @return The value of the 'double$' field. */ - public Double getDouble$() { + public java.lang.Double getDouble$() { return double$; } @@ -338,7 +301,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'double$' field. * @param value the value to set. */ - public void setDouble$(Double value) { + public void setDouble$(java.lang.Double value) { this.double$ = value; } @@ -346,7 +309,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'string' field. * @return The value of the 'string' field. */ - public CharSequence getString() { + public java.lang.CharSequence getString() { return string; } @@ -355,7 +318,7 @@ public CharSequence getString() { * Sets the value of the 'string' field. * @param value the value to set. */ - public void setString(CharSequence value) { + public void setString(java.lang.CharSequence value) { this.string = value; } @@ -380,7 +343,7 @@ public void setBytes(java.nio.ByteBuffer value) { * Gets the value of the 'fixed' field. * @return The value of the 'fixed' field. */ - public fixed4 getFixed() { + public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { return fixed; } @@ -389,7 +352,7 @@ public fixed4 getFixed() { * Sets the value of the 'fixed' field. * @param value the value to set. */ - public void setFixed(fixed4 value) { + public void setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { this.fixed = value; } @@ -448,7 +411,7 @@ public void setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum val * Gets the value of the 'row' field. * @return The value of the 'row' field. */ - public TestAvroNested getRow() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { return row; } @@ -457,7 +420,7 @@ public TestAvroNested getRow() { * Sets the value of the 'row' field. * @param value the value to set. */ - public void setRow(TestAvroNested value) { + public void setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { this.row = value; } @@ -465,7 +428,7 @@ public void setRow(TestAvroNested value) { * Gets the value of the 'array' field. * @return The value of the 'array' field. */ - public java.util.List getArray() { + public java.util.List getArray() { return array; } @@ -474,7 +437,7 @@ public java.util.List getArray() { * Sets the value of the 'array' field. * @param value the value to set. */ - public void setArray(java.util.List value) { + public void setArray(java.util.List value) { this.array = value; } @@ -482,7 +445,7 @@ public void setArray(java.util.List value) { * Gets the value of the 'map' field. * @return The value of the 'map' field. */ - public java.util.Map getMap() { + public java.util.Map getMap() { return map; } @@ -491,7 +454,7 @@ public java.util.Map getMap() { * Sets the value of the 'map' field. * @param value the value to set. */ - public void setMap(java.util.Map value) { + public void setMap(java.util.Map value) { this.map = value; } @@ -499,8 +462,8 @@ public void setMap(java.util.Map value) { * Creates a new TestAvro RecordBuilder. * @return A new TestAvro RecordBuilder */ - public static Builder newBuilder() { - return new Builder(); + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder() { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); } /** @@ -508,11 +471,11 @@ public static Builder newBuilder() { * @param other The existing builder to copy. * @return A new TestAvro RecordBuilder */ - public static Builder newBuilder(Builder other) { + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { if (other == null) { - return new Builder(); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); } else { - return new Builder(other); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); } } @@ -521,11 +484,11 @@ public static Builder newBuilder(Builder other) { * @param other The existing instance to copy. * @return A new TestAvro RecordBuilder */ - public static Builder newBuilder(TestAvro other) { + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { if (other == null) { - return new Builder(); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); } else { - return new Builder(other); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); } } @@ -537,20 +500,20 @@ public static class Builder extends org.apache.avro.specific.SpecificRecordBuild implements org.apache.avro.data.RecordBuilder { private boolean bool_non_nullable; - private Integer int$; - private Long long$; - private Float float$; - private Double double$; - private CharSequence string; + private java.lang.Integer int$; + private java.lang.Long long$; + private java.lang.Float float$; + private java.lang.Double double$; + private java.lang.CharSequence string; private java.nio.ByteBuffer bytes; - private fixed4 fixed; + private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; private java.time.LocalDate date; private java.time.Instant timestampMillis; private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private TestAvroNested row; - private TestAvroNested.Builder rowBuilder; - private java.util.List array; - private java.util.Map map; + private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; + private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder rowBuilder; + private java.util.List array; + private java.util.Map map; /** Creates a new Builder */ private Builder() { @@ -561,7 +524,7 @@ private Builder() { * Creates a Builder by copying an existing Builder. * @param other The existing Builder to copy. */ - private Builder(Builder other) { + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { super(other); if (isValidValue(fields()[0], other.bool_non_nullable)) { this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); @@ -612,7 +575,7 @@ private Builder(Builder other) { fieldSetFlags()[11] = other.fieldSetFlags()[11]; } if (other.hasRowBuilder()) { - this.rowBuilder = TestAvroNested.newBuilder(other.getRowBuilder()); + this.rowBuilder = org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(other.getRowBuilder()); } if (isValidValue(fields()[12], other.array)) { this.array = data().deepCopy(fields()[12].schema(), other.array); @@ -628,7 +591,7 @@ private Builder(Builder other) { * Creates a Builder by copying an existing TestAvro instance * @param other The existing instance to copy. */ - private Builder(TestAvro other) { + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { super(SCHEMA$, MODEL$); if (isValidValue(fields()[0], other.bool_non_nullable)) { this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); @@ -703,7 +666,7 @@ public boolean getBoolNonNullable() { * @param value The value of 'bool_non_nullable'. * @return This builder. */ - public Builder setBoolNonNullable(boolean value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBoolNonNullable(boolean value) { validate(fields()[0], value); this.bool_non_nullable = value; fieldSetFlags()[0] = true; @@ -723,7 +686,7 @@ public boolean hasBoolNonNullable() { * Clears the value of the 'bool_non_nullable' field. * @return This builder. */ - public Builder clearBoolNonNullable() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNonNullable() { fieldSetFlags()[0] = false; return this; } @@ -732,7 +695,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'int$' field. * @return The value. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -742,7 +705,7 @@ public Builder clearBoolNonNullable() { * @param value The value of 'int$'. * @return This builder. */ - public Builder setInt$(Integer value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setInt$(java.lang.Integer value) { validate(fields()[1], value); this.int$ = value; fieldSetFlags()[1] = true; @@ -762,7 +725,7 @@ public Builder clearBoolNonNullable() { * Clears the value of the 'int$' field. * @return This builder. */ - public Builder clearInt$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearInt$() { int$ = null; fieldSetFlags()[1] = false; return this; @@ -772,7 +735,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'long$' field. * @return The value. */ - public Long getLong$() { + public java.lang.Long getLong$() { return long$; } @@ -782,7 +745,7 @@ public Builder clearBoolNonNullable() { * @param value The value of 'long$'. * @return This builder. */ - public Builder setLong$(Long value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setLong$(java.lang.Long value) { validate(fields()[2], value); this.long$ = value; fieldSetFlags()[2] = true; @@ -802,7 +765,7 @@ public Builder clearBoolNonNullable() { * Clears the value of the 'long$' field. * @return This builder. */ - public Builder clearLong$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearLong$() { long$ = null; fieldSetFlags()[2] = false; return this; @@ -812,7 +775,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'float$' field. * @return The value. */ - public Float getFloat$() { + public java.lang.Float getFloat$() { return float$; } @@ -822,7 +785,7 @@ public Builder clearBoolNonNullable() { * @param value The value of 'float$'. * @return This builder. */ - public Builder setFloat$(Float value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFloat$(java.lang.Float value) { validate(fields()[3], value); this.float$ = value; fieldSetFlags()[3] = true; @@ -842,7 +805,7 @@ public Builder clearBoolNonNullable() { * Clears the value of the 'float$' field. * @return This builder. */ - public Builder clearFloat$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFloat$() { float$ = null; fieldSetFlags()[3] = false; return this; @@ -852,7 +815,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'double$' field. * @return The value. */ - public Double getDouble$() { + public java.lang.Double getDouble$() { return double$; } @@ -862,7 +825,7 @@ public Builder clearBoolNonNullable() { * @param value The value of 'double$'. * @return This builder. */ - public Builder setDouble$(Double value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDouble$(java.lang.Double value) { validate(fields()[4], value); this.double$ = value; fieldSetFlags()[4] = true; @@ -882,7 +845,7 @@ public Builder clearBoolNonNullable() { * Clears the value of the 'double$' field. * @return This builder. */ - public Builder clearDouble$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDouble$() { double$ = null; fieldSetFlags()[4] = false; return this; @@ -892,7 +855,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'string' field. * @return The value. */ - public CharSequence getString() { + public java.lang.CharSequence getString() { return string; } @@ -902,7 +865,7 @@ public CharSequence getString() { * @param value The value of 'string'. * @return This builder. */ - public Builder setString(CharSequence value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setString(java.lang.CharSequence value) { validate(fields()[5], value); this.string = value; fieldSetFlags()[5] = true; @@ -922,7 +885,7 @@ public boolean hasString() { * Clears the value of the 'string' field. * @return This builder. */ - public Builder clearString() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearString() { string = null; fieldSetFlags()[5] = false; return this; @@ -942,7 +905,7 @@ public java.nio.ByteBuffer getBytes() { * @param value The value of 'bytes'. * @return This builder. */ - public Builder setBytes(java.nio.ByteBuffer value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBytes(java.nio.ByteBuffer value) { validate(fields()[6], value); this.bytes = value; fieldSetFlags()[6] = true; @@ -962,7 +925,7 @@ public boolean hasBytes() { * Clears the value of the 'bytes' field. * @return This builder. */ - public Builder clearBytes() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBytes() { bytes = null; fieldSetFlags()[6] = false; return this; @@ -972,7 +935,7 @@ public Builder clearBytes() { * Gets the value of the 'fixed' field. * @return The value. */ - public fixed4 getFixed() { + public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { return fixed; } @@ -982,7 +945,7 @@ public fixed4 getFixed() { * @param value The value of 'fixed'. * @return This builder. */ - public Builder setFixed(fixed4 value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { validate(fields()[7], value); this.fixed = value; fieldSetFlags()[7] = true; @@ -1002,7 +965,7 @@ public boolean hasFixed() { * Clears the value of the 'fixed' field. * @return This builder. */ - public Builder clearFixed() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFixed() { fixed = null; fieldSetFlags()[7] = false; return this; @@ -1022,7 +985,7 @@ public java.time.LocalDate getDate() { * @param value The value of 'date'. * @return This builder. */ - public Builder setDate(java.time.LocalDate value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDate(java.time.LocalDate value) { validate(fields()[8], value); this.date = value; fieldSetFlags()[8] = true; @@ -1042,7 +1005,7 @@ public boolean hasDate() { * Clears the value of the 'date' field. * @return This builder. */ - public Builder clearDate() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDate() { fieldSetFlags()[8] = false; return this; } @@ -1061,7 +1024,7 @@ public java.time.Instant getTimestampMillis() { * @param value The value of 'timestampMillis'. * @return This builder. */ - public Builder setTimestampMillis(java.time.Instant value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTimestampMillis(java.time.Instant value) { validate(fields()[9], value); this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); fieldSetFlags()[9] = true; @@ -1081,7 +1044,7 @@ public boolean hasTimestampMillis() { * Clears the value of the 'timestampMillis' field. * @return This builder. */ - public Builder clearTimestampMillis() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTimestampMillis() { fieldSetFlags()[9] = false; return this; } @@ -1100,7 +1063,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { * @param value The value of 'TestEnum'. * @return This builder. */ - public Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { validate(fields()[10], value); this.TestEnum = value; fieldSetFlags()[10] = true; @@ -1120,7 +1083,7 @@ public boolean hasTestEnum() { * Clears the value of the 'TestEnum' field. * @return This builder. */ - public Builder clearTestEnum() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTestEnum() { TestEnum = null; fieldSetFlags()[10] = false; return this; @@ -1130,7 +1093,7 @@ public Builder clearTestEnum() { * Gets the value of the 'row' field. * @return The value. */ - public TestAvroNested getRow() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { return row; } @@ -1140,7 +1103,7 @@ public TestAvroNested getRow() { * @param value The value of 'row'. * @return This builder. */ - public Builder setRow(TestAvroNested value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { validate(fields()[11], value); this.rowBuilder = null; this.row = value; @@ -1160,12 +1123,12 @@ public boolean hasRow() { * Gets the Builder instance for the 'row' field and creates one if it doesn't exist yet. * @return This builder. */ - public TestAvroNested.Builder getRowBuilder() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder getRowBuilder() { if (rowBuilder == null) { if (hasRow()) { - setRowBuilder(TestAvroNested.newBuilder(row)); + setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(row)); } else { - setRowBuilder(TestAvroNested.newBuilder()); + setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder()); } } return rowBuilder; @@ -1177,7 +1140,7 @@ public TestAvroNested.Builder getRowBuilder() { * @return This builder. */ - public Builder setRowBuilder(TestAvroNested.Builder value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder value) { clearRow(); rowBuilder = value; return this; @@ -1195,7 +1158,7 @@ public boolean hasRowBuilder() { * Clears the value of the 'row' field. * @return This builder. */ - public Builder clearRow() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearRow() { row = null; rowBuilder = null; fieldSetFlags()[11] = false; @@ -1206,7 +1169,7 @@ public Builder clearRow() { * Gets the value of the 'array' field. * @return The value. */ - public java.util.List getArray() { + public java.util.List getArray() { return array; } @@ -1216,7 +1179,7 @@ public java.util.List getArray() { * @param value The value of 'array'. * @return This builder. */ - public Builder setArray(java.util.List value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setArray(java.util.List value) { validate(fields()[12], value); this.array = value; fieldSetFlags()[12] = true; @@ -1236,7 +1199,7 @@ public boolean hasArray() { * Clears the value of the 'array' field. * @return This builder. */ - public Builder clearArray() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearArray() { array = null; fieldSetFlags()[12] = false; return this; @@ -1246,7 +1209,7 @@ public Builder clearArray() { * Gets the value of the 'map' field. * @return The value. */ - public java.util.Map getMap() { + public java.util.Map getMap() { return map; } @@ -1256,7 +1219,7 @@ public java.util.Map getMap() { * @param value The value of 'map'. * @return This builder. */ - public Builder setMap(java.util.Map value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setMap(java.util.Map value) { validate(fields()[13], value); this.map = value; fieldSetFlags()[13] = true; @@ -1276,7 +1239,7 @@ public boolean hasMap() { * Clears the value of the 'map' field. * @return This builder. */ - public Builder clearMap() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearMap() { map = null; fieldSetFlags()[13] = false; return this; @@ -1287,14 +1250,14 @@ public Builder clearMap() { public TestAvro build() { try { TestAvro record = new TestAvro(); - record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); - record.long$ = fieldSetFlags()[2] ? this.long$ : (Long) defaultValue(fields()[2]); - record.float$ = fieldSetFlags()[3] ? this.float$ : (Float) defaultValue(fields()[3]); - record.double$ = fieldSetFlags()[4] ? this.double$ : (Double) defaultValue(fields()[4]); - record.string = fieldSetFlags()[5] ? this.string : (CharSequence) defaultValue(fields()[5]); + record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (java.lang.Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); + record.long$ = fieldSetFlags()[2] ? this.long$ : (java.lang.Long) defaultValue(fields()[2]); + record.float$ = fieldSetFlags()[3] ? this.float$ : (java.lang.Float) defaultValue(fields()[3]); + record.double$ = fieldSetFlags()[4] ? this.double$ : (java.lang.Double) defaultValue(fields()[4]); + record.string = fieldSetFlags()[5] ? this.string : (java.lang.CharSequence) defaultValue(fields()[5]); record.bytes = fieldSetFlags()[6] ? this.bytes : (java.nio.ByteBuffer) defaultValue(fields()[6]); - record.fixed = fieldSetFlags()[7] ? this.fixed : (fixed4) defaultValue(fields()[7]); + record.fixed = fieldSetFlags()[7] ? this.fixed : (org.apache.beam.sdk.extensions.avro.schemas.fixed4) defaultValue(fields()[7]); record.date = fieldSetFlags()[8] ? this.date : (java.time.LocalDate) defaultValue(fields()[8]); record.timestampMillis = fieldSetFlags()[9] ? this.timestampMillis : (java.time.Instant) defaultValue(fields()[9]); record.TestEnum = fieldSetFlags()[10] ? this.TestEnum : (org.apache.beam.sdk.extensions.avro.schemas.TestEnum) defaultValue(fields()[10]); @@ -1306,14 +1269,14 @@ public TestAvro build() { throw e; } } else { - record.row = fieldSetFlags()[11] ? this.row : (TestAvroNested) defaultValue(fields()[11]); + record.row = fieldSetFlags()[11] ? this.row : (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested) defaultValue(fields()[11]); } - record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); - record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); + record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); + record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); return record; } catch (org.apache.avro.AvroMissingFieldException e) { throw e; - } catch (Exception e) { + } catch (java.lang.Exception e) { throw new org.apache.avro.AvroRuntimeException(e); } } diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java index cfbe2380cda8..0b250eee1597 100644 --- a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java +++ b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java @@ -17,10 +17,12 @@ */ package org.apache.beam.sdk.extensions.avro.schemas; -import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.util.Utf8; import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; import org.apache.avro.message.SchemaStore; -import org.apache.avro.specific.SpecificData; @org.apache.avro.specific.AvroGenerated public class TestAvroNested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { @@ -84,7 +86,7 @@ public static TestAvroNested fromByteBuffer( } private boolean BOOL_NON_NULLABLE; - private Integer int$; + private java.lang.Integer int$; /** * Default constructor. Note that this does not initialize fields @@ -98,20 +100,20 @@ public TestAvroNested() {} * @param BOOL_NON_NULLABLE The new value for BOOL_NON_NULLABLE * @param int$ The new value for int */ - public TestAvroNested(Boolean BOOL_NON_NULLABLE, Integer int$) { + public TestAvroNested(java.lang.Boolean BOOL_NON_NULLABLE, java.lang.Integer int$) { this.BOOL_NON_NULLABLE = BOOL_NON_NULLABLE; this.int$ = int$; } @Override - public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } @Override public org.apache.avro.Schema getSchema() { return SCHEMA$; } // Used by DatumWriter. Applications should not call. @Override - public Object get(int field$) { + public java.lang.Object get(int field$) { switch (field$) { case 0: return BOOL_NON_NULLABLE; case 1: return int$; @@ -122,10 +124,10 @@ public Object get(int field$) { // Used by DatumReader. Applications should not call. @Override @SuppressWarnings(value="unchecked") - public void put(int field$, Object value$) { + public void put(int field$, java.lang.Object value$) { switch (field$) { - case 0: BOOL_NON_NULLABLE = (Boolean)value$; break; - case 1: int$ = (Integer)value$; break; + case 0: BOOL_NON_NULLABLE = (java.lang.Boolean)value$; break; + case 1: int$ = (java.lang.Integer)value$; break; default: throw new IndexOutOfBoundsException("Invalid index: " + field$); } } @@ -151,7 +153,7 @@ public void setBOOLNONNULLABLE(boolean value) { * Gets the value of the 'int$' field. * @return The value of the 'int$' field. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -160,7 +162,7 @@ public void setBOOLNONNULLABLE(boolean value) { * Sets the value of the 'int$' field. * @param value the value to set. */ - public void setInt$(Integer value) { + public void setInt$(java.lang.Integer value) { this.int$ = value; } @@ -168,8 +170,8 @@ public void setBOOLNONNULLABLE(boolean value) { * Creates a new TestAvroNested RecordBuilder. * @return A new TestAvroNested RecordBuilder */ - public static Builder newBuilder() { - return new Builder(); + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder() { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); } /** @@ -177,11 +179,11 @@ public static Builder newBuilder() { * @param other The existing builder to copy. * @return A new TestAvroNested RecordBuilder */ - public static Builder newBuilder(Builder other) { + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { if (other == null) { - return new Builder(); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); } else { - return new Builder(other); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); } } @@ -190,11 +192,11 @@ public static Builder newBuilder(Builder other) { * @param other The existing instance to copy. * @return A new TestAvroNested RecordBuilder */ - public static Builder newBuilder(TestAvroNested other) { + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { if (other == null) { - return new Builder(); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); } else { - return new Builder(other); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); } } @@ -206,7 +208,7 @@ public static class Builder extends org.apache.avro.specific.SpecificRecordBuild implements org.apache.avro.data.RecordBuilder { private boolean BOOL_NON_NULLABLE; - private Integer int$; + private java.lang.Integer int$; /** Creates a new Builder */ private Builder() { @@ -217,7 +219,7 @@ private Builder() { * Creates a Builder by copying an existing Builder. * @param other The existing Builder to copy. */ - private Builder(Builder other) { + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { super(other); if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); @@ -233,7 +235,7 @@ private Builder(Builder other) { * Creates a Builder by copying an existing TestAvroNested instance * @param other The existing instance to copy. */ - private Builder(TestAvroNested other) { + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { super(SCHEMA$, MODEL$); if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); @@ -259,7 +261,7 @@ public boolean getBOOLNONNULLABLE() { * @param value The value of 'BOOL_NON_NULLABLE'. * @return This builder. */ - public Builder setBOOLNONNULLABLE(boolean value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setBOOLNONNULLABLE(boolean value) { validate(fields()[0], value); this.BOOL_NON_NULLABLE = value; fieldSetFlags()[0] = true; @@ -279,7 +281,7 @@ public boolean hasBOOLNONNULLABLE() { * Clears the value of the 'BOOL_NON_NULLABLE' field. * @return This builder. */ - public Builder clearBOOLNONNULLABLE() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearBOOLNONNULLABLE() { fieldSetFlags()[0] = false; return this; } @@ -288,7 +290,7 @@ public Builder clearBOOLNONNULLABLE() { * Gets the value of the 'int$' field. * @return The value. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -298,7 +300,7 @@ public Builder clearBOOLNONNULLABLE() { * @param value The value of 'int$'. * @return This builder. */ - public Builder setInt$(Integer value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setInt$(java.lang.Integer value) { validate(fields()[1], value); this.int$ = value; fieldSetFlags()[1] = true; @@ -318,7 +320,7 @@ public Builder clearBOOLNONNULLABLE() { * Clears the value of the 'int$' field. * @return This builder. */ - public Builder clearInt$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearInt$() { int$ = null; fieldSetFlags()[1] = false; return this; @@ -329,12 +331,12 @@ public Builder clearBOOLNONNULLABLE() { public TestAvroNested build() { try { TestAvroNested record = new TestAvroNested(); - record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); + record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (java.lang.Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); return record; } catch (org.apache.avro.AvroMissingFieldException e) { throw e; - } catch (Exception e) { + } catch (java.lang.Exception e) { throw new org.apache.avro.AvroRuntimeException(e); } } diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java new file mode 100644 index 000000000000..a91b86da2f29 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java @@ -0,0 +1,515 @@ +/* + * 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.beam.sdk.extensions.avro.io; + +import org.apache.avro.generic.GenericArray; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.util.Utf8; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.SchemaStore; + +@org.apache.avro.specific.AvroGenerated +public class AvroGeneratedUser extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 3056733241446362297L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroGeneratedUser\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.io\",\"fields\":[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"favorite_number\",\"type\":[\"int\",\"null\"]},{\"name\":\"favorite_color\",\"type\":[\"string\",\"null\"]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static SpecificData MODEL$ = new SpecificData(); + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageEncoder instance used by this class. + * @return the message encoder used by this class + */ + public static BinaryMessageEncoder getEncoder() { + return ENCODER; + } + + /** + * Return the BinaryMessageDecoder instance used by this class. + * @return the message decoder used by this class + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * 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 + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); + } + + /** + * Serializes this AvroGeneratedUser to a ByteBuffer. + * @return a buffer holding the serialized data for this instance + * @throws java.io.IOException if this instance could not be serialized + */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** + * Deserializes a AvroGeneratedUser from a ByteBuffer. + * @param b a byte buffer holding serialized data for an instance of this class + * @return a AvroGeneratedUser instance decoded from the given buffer + * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class + */ + public static AvroGeneratedUser fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + private java.lang.CharSequence name; + private java.lang.Integer favorite_number; + private java.lang.CharSequence favorite_color; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public AvroGeneratedUser() {} + + /** + * All-args constructor. + * @param name The new value for name + * @param favorite_number The new value for favorite_number + * @param favorite_color The new value for favorite_color + */ + public AvroGeneratedUser(java.lang.CharSequence name, java.lang.Integer favorite_number, java.lang.CharSequence favorite_color) { + this.name = name; + this.favorite_number = favorite_number; + this.favorite_color = favorite_color; + } + + public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return name; + case 1: return favorite_number; + case 2: return favorite_color; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: name = (java.lang.CharSequence)value$; break; + case 1: favorite_number = (java.lang.Integer)value$; break; + case 2: favorite_color = (java.lang.CharSequence)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'name' field. + * @return The value of the 'name' field. + */ + public java.lang.CharSequence getName() { + return name; + } + + + /** + * Sets the value of the 'name' field. + * @param value the value to set. + */ + public void setName(java.lang.CharSequence value) { + this.name = value; + } + + /** + * Gets the value of the 'favorite_number' field. + * @return The value of the 'favorite_number' field. + */ + public java.lang.Integer getFavoriteNumber() { + return favorite_number; + } + + + /** + * Sets the value of the 'favorite_number' field. + * @param value the value to set. + */ + public void setFavoriteNumber(java.lang.Integer value) { + this.favorite_number = value; + } + + /** + * Gets the value of the 'favorite_color' field. + * @return The value of the 'favorite_color' field. + */ + public java.lang.CharSequence getFavoriteColor() { + return favorite_color; + } + + + /** + * Sets the value of the 'favorite_color' field. + * @param value the value to set. + */ + public void setFavoriteColor(java.lang.CharSequence value) { + this.favorite_color = value; + } + + /** + * Creates a new AvroGeneratedUser RecordBuilder. + * @return A new AvroGeneratedUser RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder() { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); + } + + /** + * Creates a new AvroGeneratedUser RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new AvroGeneratedUser RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); + } + } + + /** + * Creates a new AvroGeneratedUser RecordBuilder by copying an existing AvroGeneratedUser instance. + * @param other The existing instance to copy. + * @return A new AvroGeneratedUser RecordBuilder + */ + public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { + if (other == null) { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); + } else { + return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); + } + } + + /** + * RecordBuilder for AvroGeneratedUser instances. + */ + @org.apache.avro.specific.AvroGenerated + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private java.lang.CharSequence name; + private java.lang.Integer favorite_number; + private java.lang.CharSequence favorite_color; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { + super(other); + if (isValidValue(fields()[0], other.name)) { + this.name = data().deepCopy(fields()[0].schema(), other.name); + fieldSetFlags()[0] = other.fieldSetFlags()[0]; + } + if (isValidValue(fields()[1], other.favorite_number)) { + this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); + fieldSetFlags()[1] = other.fieldSetFlags()[1]; + } + if (isValidValue(fields()[2], other.favorite_color)) { + this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); + fieldSetFlags()[2] = other.fieldSetFlags()[2]; + } + } + + /** + * Creates a Builder by copying an existing AvroGeneratedUser instance + * @param other The existing instance to copy. + */ + private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { + super(SCHEMA$); + if (isValidValue(fields()[0], other.name)) { + this.name = data().deepCopy(fields()[0].schema(), other.name); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.favorite_number)) { + this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.favorite_color)) { + this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); + fieldSetFlags()[2] = true; + } + } + + /** + * Gets the value of the 'name' field. + * @return The value. + */ + public java.lang.CharSequence getName() { + return name; + } + + + /** + * Sets the value of the 'name' field. + * @param value The value of 'name'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setName(java.lang.CharSequence value) { + validate(fields()[0], value); + this.name = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'name' field has been set. + * @return True if the 'name' field has been set, false otherwise. + */ + public boolean hasName() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'name' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearName() { + name = null; + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'favorite_number' field. + * @return The value. + */ + public java.lang.Integer getFavoriteNumber() { + return favorite_number; + } + + + /** + * Sets the value of the 'favorite_number' field. + * @param value The value of 'favorite_number'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteNumber(java.lang.Integer value) { + validate(fields()[1], value); + this.favorite_number = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'favorite_number' field has been set. + * @return True if the 'favorite_number' field has been set, false otherwise. + */ + public boolean hasFavoriteNumber() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'favorite_number' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteNumber() { + favorite_number = null; + fieldSetFlags()[1] = false; + return this; + } + + /** + * Gets the value of the 'favorite_color' field. + * @return The value. + */ + public java.lang.CharSequence getFavoriteColor() { + return favorite_color; + } + + + /** + * Sets the value of the 'favorite_color' field. + * @param value The value of 'favorite_color'. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteColor(java.lang.CharSequence value) { + validate(fields()[2], value); + this.favorite_color = value; + fieldSetFlags()[2] = true; + return this; + } + + /** + * Checks whether the 'favorite_color' field has been set. + * @return True if the 'favorite_color' field has been set, false otherwise. + */ + public boolean hasFavoriteColor() { + return fieldSetFlags()[2]; + } + + + /** + * Clears the value of the 'favorite_color' field. + * @return This builder. + */ + public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteColor() { + favorite_color = null; + fieldSetFlags()[2] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public AvroGeneratedUser build() { + try { + AvroGeneratedUser record = new AvroGeneratedUser(); + record.name = fieldSetFlags()[0] ? this.name : (java.lang.CharSequence) defaultValue(fields()[0]); + record.favorite_number = fieldSetFlags()[1] ? this.favorite_number : (java.lang.Integer) defaultValue(fields()[1]); + record.favorite_color = fieldSetFlags()[2] ? this.favorite_color : (java.lang.CharSequence) defaultValue(fields()[2]); + return record; + } catch (org.apache.avro.AvroMissingFieldException e) { + throw e; + } catch (java.lang.Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + + @Override protected boolean hasCustomCoders() { return true; } + + @Override public void customEncode(org.apache.avro.io.Encoder out) + throws java.io.IOException + { + out.writeString(this.name); + + if (this.favorite_number == null) { + out.writeIndex(1); + out.writeNull(); + } else { + out.writeIndex(0); + out.writeInt(this.favorite_number); + } + + if (this.favorite_color == null) { + out.writeIndex(1); + out.writeNull(); + } else { + out.writeIndex(0); + out.writeString(this.favorite_color); + } + + } + + @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) + throws java.io.IOException + { + org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); + if (fieldOrder == null) { + this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); + + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_number = null; + } else { + this.favorite_number = in.readInt(); + } + + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_color = null; + } else { + this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); + } + + } else { + for (int i = 0; i < 3; i++) { + switch (fieldOrder[i].pos()) { + case 0: + this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); + break; + + case 1: + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_number = null; + } else { + this.favorite_number = in.readInt(); + } + break; + + case 2: + if (in.readIndex() != 0) { + in.readNull(); + this.favorite_color = null; + } else { + this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); + } + break; + + default: + throw new java.io.IOException("Corrupt ResolvingDecoder."); + } + } + } + } +} + + + + + + + + + + diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java index 9493c7976c85..501fc6772ed3 100644 --- a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java +++ b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.extensions.avro.schemas; -import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.util.Utf8; import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; import org.apache.avro.message.SchemaStore; -import org.apache.avro.specific.SpecificData; - -import java.time.Instant; -import java.time.LocalDate; -import java.util.Map; @org.apache.avro.specific.AvroGenerated public class TestAvro extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { @@ -90,19 +88,19 @@ public static TestAvro fromByteBuffer( } private boolean bool_non_nullable; - private Integer int$; - private Long long$; - private Float float$; - private Double double$; - private CharSequence string; + private java.lang.Integer int$; + private java.lang.Long long$; + private java.lang.Float float$; + private java.lang.Double double$; + private java.lang.CharSequence string; private java.nio.ByteBuffer bytes; - private fixed4 fixed; + private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; private java.time.LocalDate date; private java.time.Instant timestampMillis; private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private TestAvroNested row; - private java.util.List array; - private java.util.Map map; + private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; + private java.util.List array; + private java.util.Map map; /** * Default constructor. Note that this does not initialize fields @@ -128,7 +126,7 @@ public TestAvro() {} * @param array The new value for array * @param map The new value for map */ - public TestAvro(Boolean bool_non_nullable, Integer int$, Long long$, Float float$, Double double$, CharSequence string, java.nio.ByteBuffer bytes, fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, TestAvroNested row, java.util.List array, java.util.Map map) { + public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.CharSequence string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { this.bool_non_nullable = bool_non_nullable; this.int$ = int$; this.long$ = long$; @@ -145,45 +143,10 @@ public TestAvro(Boolean bool_non_nullable, Integer int$, Long long$, Float float this.map = map; } - /** - * Manually added a ompatible with Avro v1.8.2 API constructor - * - * @param bool_non_nullable - * @param int$ - * @param long$ - * @param float$ - * @param double$ - * @param string - * @param bytes - * @param fixed - * @param date - * @param timestampMillis - * @param TestEnum - * @param row - * @param array - * @param map - */ - public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.String string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, org.joda.time.LocalDate date, org.joda.time.DateTime timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { - this.bool_non_nullable = bool_non_nullable; - this.int$ = int$; - this.long$ = long$; - this.float$ = float$; - this.double$ = double$; - this.string = string; - this.bytes = bytes; - this.fixed = fixed; - this.date = LocalDate.of(date.getYear(), date.getMonthOfYear(), date.getDayOfMonth()); - this.timestampMillis = Instant.ofEpochMilli(timestampMillis.getMillis()); - this.TestEnum = TestEnum; - this.row = row; - this.array = array; - this.map = (Map)map; - } - - public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } public org.apache.avro.Schema getSchema() { return SCHEMA$; } // Used by DatumWriter. Applications should not call. - public Object get(int field$) { + public java.lang.Object get(int field$) { switch (field$) { case 0: return bool_non_nullable; case 1: return int$; @@ -229,22 +192,22 @@ public org.apache.avro.Conversion getConversion(int field) { // Used by DatumReader. Applications should not call. @SuppressWarnings(value="unchecked") - public void put(int field$, Object value$) { + public void put(int field$, java.lang.Object value$) { switch (field$) { - case 0: bool_non_nullable = (Boolean)value$; break; - case 1: int$ = (Integer)value$; break; - case 2: long$ = (Long)value$; break; - case 3: float$ = (Float)value$; break; - case 4: double$ = (Double)value$; break; - case 5: string = (CharSequence)value$; break; + case 0: bool_non_nullable = (java.lang.Boolean)value$; break; + case 1: int$ = (java.lang.Integer)value$; break; + case 2: long$ = (java.lang.Long)value$; break; + case 3: float$ = (java.lang.Float)value$; break; + case 4: double$ = (java.lang.Double)value$; break; + case 5: string = (java.lang.CharSequence)value$; break; case 6: bytes = (java.nio.ByteBuffer)value$; break; - case 7: fixed = (fixed4)value$; break; + case 7: fixed = (org.apache.beam.sdk.extensions.avro.schemas.fixed4)value$; break; case 8: date = (java.time.LocalDate)value$; break; case 9: timestampMillis = (java.time.Instant)value$; break; case 10: TestEnum = (org.apache.beam.sdk.extensions.avro.schemas.TestEnum)value$; break; - case 11: row = (TestAvroNested)value$; break; - case 12: array = (java.util.List)value$; break; - case 13: map = (java.util.Map)value$; break; + case 11: row = (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested)value$; break; + case 12: array = (java.util.List)value$; break; + case 13: map = (java.util.Map)value$; break; default: throw new org.apache.avro.AvroRuntimeException("Bad index"); } } @@ -270,7 +233,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'int$' field. * @return The value of the 'int$' field. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -279,7 +242,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'int$' field. * @param value the value to set. */ - public void setInt$(Integer value) { + public void setInt$(java.lang.Integer value) { this.int$ = value; } @@ -287,7 +250,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'long$' field. * @return The value of the 'long$' field. */ - public Long getLong$() { + public java.lang.Long getLong$() { return long$; } @@ -296,7 +259,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'long$' field. * @param value the value to set. */ - public void setLong$(Long value) { + public void setLong$(java.lang.Long value) { this.long$ = value; } @@ -304,7 +267,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'float$' field. * @return The value of the 'float$' field. */ - public Float getFloat$() { + public java.lang.Float getFloat$() { return float$; } @@ -313,7 +276,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'float$' field. * @param value the value to set. */ - public void setFloat$(Float value) { + public void setFloat$(java.lang.Float value) { this.float$ = value; } @@ -321,7 +284,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'double$' field. * @return The value of the 'double$' field. */ - public Double getDouble$() { + public java.lang.Double getDouble$() { return double$; } @@ -330,7 +293,7 @@ public void setBoolNonNullable(boolean value) { * Sets the value of the 'double$' field. * @param value the value to set. */ - public void setDouble$(Double value) { + public void setDouble$(java.lang.Double value) { this.double$ = value; } @@ -338,7 +301,7 @@ public void setBoolNonNullable(boolean value) { * Gets the value of the 'string' field. * @return The value of the 'string' field. */ - public CharSequence getString() { + public java.lang.CharSequence getString() { return string; } @@ -347,7 +310,7 @@ public CharSequence getString() { * Sets the value of the 'string' field. * @param value the value to set. */ - public void setString(CharSequence value) { + public void setString(java.lang.CharSequence value) { this.string = value; } @@ -372,7 +335,7 @@ public void setBytes(java.nio.ByteBuffer value) { * Gets the value of the 'fixed' field. * @return The value of the 'fixed' field. */ - public fixed4 getFixed() { + public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { return fixed; } @@ -381,7 +344,7 @@ public fixed4 getFixed() { * Sets the value of the 'fixed' field. * @param value the value to set. */ - public void setFixed(fixed4 value) { + public void setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { this.fixed = value; } @@ -440,7 +403,7 @@ public void setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum val * Gets the value of the 'row' field. * @return The value of the 'row' field. */ - public TestAvroNested getRow() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { return row; } @@ -449,7 +412,7 @@ public TestAvroNested getRow() { * Sets the value of the 'row' field. * @param value the value to set. */ - public void setRow(TestAvroNested value) { + public void setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { this.row = value; } @@ -457,7 +420,7 @@ public void setRow(TestAvroNested value) { * Gets the value of the 'array' field. * @return The value of the 'array' field. */ - public java.util.List getArray() { + public java.util.List getArray() { return array; } @@ -466,7 +429,7 @@ public java.util.List getArray() { * Sets the value of the 'array' field. * @param value the value to set. */ - public void setArray(java.util.List value) { + public void setArray(java.util.List value) { this.array = value; } @@ -474,7 +437,7 @@ public void setArray(java.util.List value) { * Gets the value of the 'map' field. * @return The value of the 'map' field. */ - public java.util.Map getMap() { + public java.util.Map getMap() { return map; } @@ -483,7 +446,7 @@ public java.util.Map getMap() { * Sets the value of the 'map' field. * @param value the value to set. */ - public void setMap(java.util.Map value) { + public void setMap(java.util.Map value) { this.map = value; } @@ -491,8 +454,8 @@ public void setMap(java.util.Map value) { * Creates a new TestAvro RecordBuilder. * @return A new TestAvro RecordBuilder */ - public static Builder newBuilder() { - return new Builder(); + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder() { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); } /** @@ -500,11 +463,11 @@ public static Builder newBuilder() { * @param other The existing builder to copy. * @return A new TestAvro RecordBuilder */ - public static Builder newBuilder(Builder other) { + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { if (other == null) { - return new Builder(); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); } else { - return new Builder(other); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); } } @@ -513,11 +476,11 @@ public static Builder newBuilder(Builder other) { * @param other The existing instance to copy. * @return A new TestAvro RecordBuilder */ - public static Builder newBuilder(TestAvro other) { + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { if (other == null) { - return new Builder(); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); } else { - return new Builder(other); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); } } @@ -529,20 +492,20 @@ public static class Builder extends org.apache.avro.specific.SpecificRecordBuild implements org.apache.avro.data.RecordBuilder { private boolean bool_non_nullable; - private Integer int$; - private Long long$; - private Float float$; - private Double double$; - private CharSequence string; + private java.lang.Integer int$; + private java.lang.Long long$; + private java.lang.Float float$; + private java.lang.Double double$; + private java.lang.CharSequence string; private java.nio.ByteBuffer bytes; - private fixed4 fixed; + private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; private java.time.LocalDate date; private java.time.Instant timestampMillis; private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private TestAvroNested row; - private TestAvroNested.Builder rowBuilder; - private java.util.List array; - private java.util.Map map; + private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; + private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder rowBuilder; + private java.util.List array; + private java.util.Map map; /** Creates a new Builder */ private Builder() { @@ -553,7 +516,7 @@ private Builder() { * Creates a Builder by copying an existing Builder. * @param other The existing Builder to copy. */ - private Builder(Builder other) { + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { super(other); if (isValidValue(fields()[0], other.bool_non_nullable)) { this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); @@ -604,7 +567,7 @@ private Builder(Builder other) { fieldSetFlags()[11] = other.fieldSetFlags()[11]; } if (other.hasRowBuilder()) { - this.rowBuilder = TestAvroNested.newBuilder(other.getRowBuilder()); + this.rowBuilder = org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(other.getRowBuilder()); } if (isValidValue(fields()[12], other.array)) { this.array = data().deepCopy(fields()[12].schema(), other.array); @@ -620,7 +583,7 @@ private Builder(Builder other) { * Creates a Builder by copying an existing TestAvro instance * @param other The existing instance to copy. */ - private Builder(TestAvro other) { + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { super(SCHEMA$); if (isValidValue(fields()[0], other.bool_non_nullable)) { this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); @@ -695,7 +658,7 @@ public boolean getBoolNonNullable() { * @param value The value of 'bool_non_nullable'. * @return This builder. */ - public Builder setBoolNonNullable(boolean value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBoolNonNullable(boolean value) { validate(fields()[0], value); this.bool_non_nullable = value; fieldSetFlags()[0] = true; @@ -715,7 +678,7 @@ public boolean hasBoolNonNullable() { * Clears the value of the 'bool_non_nullable' field. * @return This builder. */ - public Builder clearBoolNonNullable() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNonNullable() { fieldSetFlags()[0] = false; return this; } @@ -724,7 +687,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'int$' field. * @return The value. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -734,7 +697,7 @@ public Builder clearBoolNonNullable() { * @param value The value of 'int$'. * @return This builder. */ - public Builder setInt$(Integer value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setInt$(java.lang.Integer value) { validate(fields()[1], value); this.int$ = value; fieldSetFlags()[1] = true; @@ -754,7 +717,7 @@ public Builder clearBoolNonNullable() { * Clears the value of the 'int$' field. * @return This builder. */ - public Builder clearInt$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearInt$() { int$ = null; fieldSetFlags()[1] = false; return this; @@ -764,7 +727,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'long$' field. * @return The value. */ - public Long getLong$() { + public java.lang.Long getLong$() { return long$; } @@ -774,7 +737,7 @@ public Builder clearBoolNonNullable() { * @param value The value of 'long$'. * @return This builder. */ - public Builder setLong$(Long value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setLong$(java.lang.Long value) { validate(fields()[2], value); this.long$ = value; fieldSetFlags()[2] = true; @@ -794,7 +757,7 @@ public Builder clearBoolNonNullable() { * Clears the value of the 'long$' field. * @return This builder. */ - public Builder clearLong$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearLong$() { long$ = null; fieldSetFlags()[2] = false; return this; @@ -804,7 +767,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'float$' field. * @return The value. */ - public Float getFloat$() { + public java.lang.Float getFloat$() { return float$; } @@ -814,7 +777,7 @@ public Builder clearBoolNonNullable() { * @param value The value of 'float$'. * @return This builder. */ - public Builder setFloat$(Float value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFloat$(java.lang.Float value) { validate(fields()[3], value); this.float$ = value; fieldSetFlags()[3] = true; @@ -834,7 +797,7 @@ public Builder clearBoolNonNullable() { * Clears the value of the 'float$' field. * @return This builder. */ - public Builder clearFloat$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFloat$() { float$ = null; fieldSetFlags()[3] = false; return this; @@ -844,7 +807,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'double$' field. * @return The value. */ - public Double getDouble$() { + public java.lang.Double getDouble$() { return double$; } @@ -854,7 +817,7 @@ public Builder clearBoolNonNullable() { * @param value The value of 'double$'. * @return This builder. */ - public Builder setDouble$(Double value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDouble$(java.lang.Double value) { validate(fields()[4], value); this.double$ = value; fieldSetFlags()[4] = true; @@ -874,7 +837,7 @@ public Builder clearBoolNonNullable() { * Clears the value of the 'double$' field. * @return This builder. */ - public Builder clearDouble$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDouble$() { double$ = null; fieldSetFlags()[4] = false; return this; @@ -884,7 +847,7 @@ public Builder clearBoolNonNullable() { * Gets the value of the 'string' field. * @return The value. */ - public CharSequence getString() { + public java.lang.CharSequence getString() { return string; } @@ -894,7 +857,7 @@ public CharSequence getString() { * @param value The value of 'string'. * @return This builder. */ - public Builder setString(CharSequence value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setString(java.lang.CharSequence value) { validate(fields()[5], value); this.string = value; fieldSetFlags()[5] = true; @@ -914,7 +877,7 @@ public boolean hasString() { * Clears the value of the 'string' field. * @return This builder. */ - public Builder clearString() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearString() { string = null; fieldSetFlags()[5] = false; return this; @@ -934,7 +897,7 @@ public java.nio.ByteBuffer getBytes() { * @param value The value of 'bytes'. * @return This builder. */ - public Builder setBytes(java.nio.ByteBuffer value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBytes(java.nio.ByteBuffer value) { validate(fields()[6], value); this.bytes = value; fieldSetFlags()[6] = true; @@ -954,7 +917,7 @@ public boolean hasBytes() { * Clears the value of the 'bytes' field. * @return This builder. */ - public Builder clearBytes() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBytes() { bytes = null; fieldSetFlags()[6] = false; return this; @@ -964,7 +927,7 @@ public Builder clearBytes() { * Gets the value of the 'fixed' field. * @return The value. */ - public fixed4 getFixed() { + public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { return fixed; } @@ -974,7 +937,7 @@ public fixed4 getFixed() { * @param value The value of 'fixed'. * @return This builder. */ - public Builder setFixed(fixed4 value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { validate(fields()[7], value); this.fixed = value; fieldSetFlags()[7] = true; @@ -994,7 +957,7 @@ public boolean hasFixed() { * Clears the value of the 'fixed' field. * @return This builder. */ - public Builder clearFixed() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFixed() { fixed = null; fieldSetFlags()[7] = false; return this; @@ -1014,7 +977,7 @@ public java.time.LocalDate getDate() { * @param value The value of 'date'. * @return This builder. */ - public Builder setDate(java.time.LocalDate value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDate(java.time.LocalDate value) { validate(fields()[8], value); this.date = value; fieldSetFlags()[8] = true; @@ -1034,7 +997,7 @@ public boolean hasDate() { * Clears the value of the 'date' field. * @return This builder. */ - public Builder clearDate() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDate() { fieldSetFlags()[8] = false; return this; } @@ -1053,7 +1016,7 @@ public java.time.Instant getTimestampMillis() { * @param value The value of 'timestampMillis'. * @return This builder. */ - public Builder setTimestampMillis(java.time.Instant value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTimestampMillis(java.time.Instant value) { validate(fields()[9], value); this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); fieldSetFlags()[9] = true; @@ -1073,7 +1036,7 @@ public boolean hasTimestampMillis() { * Clears the value of the 'timestampMillis' field. * @return This builder. */ - public Builder clearTimestampMillis() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTimestampMillis() { fieldSetFlags()[9] = false; return this; } @@ -1092,7 +1055,7 @@ public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { * @param value The value of 'TestEnum'. * @return This builder. */ - public Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { validate(fields()[10], value); this.TestEnum = value; fieldSetFlags()[10] = true; @@ -1112,7 +1075,7 @@ public boolean hasTestEnum() { * Clears the value of the 'TestEnum' field. * @return This builder. */ - public Builder clearTestEnum() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTestEnum() { TestEnum = null; fieldSetFlags()[10] = false; return this; @@ -1122,7 +1085,7 @@ public Builder clearTestEnum() { * Gets the value of the 'row' field. * @return The value. */ - public TestAvroNested getRow() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { return row; } @@ -1132,7 +1095,7 @@ public TestAvroNested getRow() { * @param value The value of 'row'. * @return This builder. */ - public Builder setRow(TestAvroNested value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { validate(fields()[11], value); this.rowBuilder = null; this.row = value; @@ -1152,12 +1115,12 @@ public boolean hasRow() { * Gets the Builder instance for the 'row' field and creates one if it doesn't exist yet. * @return This builder. */ - public TestAvroNested.Builder getRowBuilder() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder getRowBuilder() { if (rowBuilder == null) { if (hasRow()) { - setRowBuilder(TestAvroNested.newBuilder(row)); + setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(row)); } else { - setRowBuilder(TestAvroNested.newBuilder()); + setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder()); } } return rowBuilder; @@ -1168,7 +1131,7 @@ public TestAvroNested.Builder getRowBuilder() { * @param value The builder instance that must be set. * @return This builder. */ - public Builder setRowBuilder(TestAvroNested.Builder value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder value) { clearRow(); rowBuilder = value; return this; @@ -1186,7 +1149,7 @@ public boolean hasRowBuilder() { * Clears the value of the 'row' field. * @return This builder. */ - public Builder clearRow() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearRow() { row = null; rowBuilder = null; fieldSetFlags()[11] = false; @@ -1197,7 +1160,7 @@ public Builder clearRow() { * Gets the value of the 'array' field. * @return The value. */ - public java.util.List getArray() { + public java.util.List getArray() { return array; } @@ -1207,7 +1170,7 @@ public java.util.List getArray() { * @param value The value of 'array'. * @return This builder. */ - public Builder setArray(java.util.List value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setArray(java.util.List value) { validate(fields()[12], value); this.array = value; fieldSetFlags()[12] = true; @@ -1227,7 +1190,7 @@ public boolean hasArray() { * Clears the value of the 'array' field. * @return This builder. */ - public Builder clearArray() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearArray() { array = null; fieldSetFlags()[12] = false; return this; @@ -1237,7 +1200,7 @@ public Builder clearArray() { * Gets the value of the 'map' field. * @return The value. */ - public java.util.Map getMap() { + public java.util.Map getMap() { return map; } @@ -1247,7 +1210,7 @@ public java.util.Map getMap() { * @param value The value of 'map'. * @return This builder. */ - public Builder setMap(java.util.Map value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setMap(java.util.Map value) { validate(fields()[13], value); this.map = value; fieldSetFlags()[13] = true; @@ -1267,7 +1230,7 @@ public boolean hasMap() { * Clears the value of the 'map' field. * @return This builder. */ - public Builder clearMap() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearMap() { map = null; fieldSetFlags()[13] = false; return this; @@ -1278,14 +1241,14 @@ public Builder clearMap() { public TestAvro build() { try { TestAvro record = new TestAvro(); - record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); - record.long$ = fieldSetFlags()[2] ? this.long$ : (Long) defaultValue(fields()[2]); - record.float$ = fieldSetFlags()[3] ? this.float$ : (Float) defaultValue(fields()[3]); - record.double$ = fieldSetFlags()[4] ? this.double$ : (Double) defaultValue(fields()[4]); - record.string = fieldSetFlags()[5] ? this.string : (CharSequence) defaultValue(fields()[5]); + record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (java.lang.Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); + record.long$ = fieldSetFlags()[2] ? this.long$ : (java.lang.Long) defaultValue(fields()[2]); + record.float$ = fieldSetFlags()[3] ? this.float$ : (java.lang.Float) defaultValue(fields()[3]); + record.double$ = fieldSetFlags()[4] ? this.double$ : (java.lang.Double) defaultValue(fields()[4]); + record.string = fieldSetFlags()[5] ? this.string : (java.lang.CharSequence) defaultValue(fields()[5]); record.bytes = fieldSetFlags()[6] ? this.bytes : (java.nio.ByteBuffer) defaultValue(fields()[6]); - record.fixed = fieldSetFlags()[7] ? this.fixed : (fixed4) defaultValue(fields()[7]); + record.fixed = fieldSetFlags()[7] ? this.fixed : (org.apache.beam.sdk.extensions.avro.schemas.fixed4) defaultValue(fields()[7]); record.date = fieldSetFlags()[8] ? this.date : (java.time.LocalDate) defaultValue(fields()[8]); record.timestampMillis = fieldSetFlags()[9] ? this.timestampMillis : (java.time.Instant) defaultValue(fields()[9]); record.TestEnum = fieldSetFlags()[10] ? this.TestEnum : (org.apache.beam.sdk.extensions.avro.schemas.TestEnum) defaultValue(fields()[10]); @@ -1297,14 +1260,14 @@ public TestAvro build() { throw e; } } else { - record.row = fieldSetFlags()[11] ? this.row : (TestAvroNested) defaultValue(fields()[11]); + record.row = fieldSetFlags()[11] ? this.row : (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested) defaultValue(fields()[11]); } - record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); - record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); + record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); + record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); return record; } catch (org.apache.avro.AvroMissingFieldException e) { throw e; - } catch (Exception e) { + } catch (java.lang.Exception e) { throw new org.apache.avro.AvroRuntimeException(e); } } diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java index 179fe0c29736..4f5818ee51da 100644 --- a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java +++ b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java @@ -17,10 +17,12 @@ */ package org.apache.beam.sdk.extensions.avro.schemas; -import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.specific.SpecificData; +import org.apache.avro.util.Utf8; import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; import org.apache.avro.message.SchemaStore; -import org.apache.avro.specific.SpecificData; @org.apache.avro.specific.AvroGenerated public class TestAvroNested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { @@ -82,7 +84,7 @@ public static TestAvroNested fromByteBuffer( } private boolean BOOL_NON_NULLABLE; - private Integer int$; + private java.lang.Integer int$; /** * Default constructor. Note that this does not initialize fields @@ -96,15 +98,15 @@ public TestAvroNested() {} * @param BOOL_NON_NULLABLE The new value for BOOL_NON_NULLABLE * @param int$ The new value for int */ - public TestAvroNested(Boolean BOOL_NON_NULLABLE, Integer int$) { + public TestAvroNested(java.lang.Boolean BOOL_NON_NULLABLE, java.lang.Integer int$) { this.BOOL_NON_NULLABLE = BOOL_NON_NULLABLE; this.int$ = int$; } - public SpecificData getSpecificData() { return MODEL$; } + public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } public org.apache.avro.Schema getSchema() { return SCHEMA$; } // Used by DatumWriter. Applications should not call. - public Object get(int field$) { + public java.lang.Object get(int field$) { switch (field$) { case 0: return BOOL_NON_NULLABLE; case 1: return int$; @@ -114,10 +116,10 @@ public Object get(int field$) { // Used by DatumReader. Applications should not call. @SuppressWarnings(value="unchecked") - public void put(int field$, Object value$) { + public void put(int field$, java.lang.Object value$) { switch (field$) { - case 0: BOOL_NON_NULLABLE = (Boolean)value$; break; - case 1: int$ = (Integer)value$; break; + case 0: BOOL_NON_NULLABLE = (java.lang.Boolean)value$; break; + case 1: int$ = (java.lang.Integer)value$; break; default: throw new org.apache.avro.AvroRuntimeException("Bad index"); } } @@ -143,7 +145,7 @@ public void setBOOLNONNULLABLE(boolean value) { * Gets the value of the 'int$' field. * @return The value of the 'int$' field. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -152,7 +154,7 @@ public void setBOOLNONNULLABLE(boolean value) { * Sets the value of the 'int$' field. * @param value the value to set. */ - public void setInt$(Integer value) { + public void setInt$(java.lang.Integer value) { this.int$ = value; } @@ -160,8 +162,8 @@ public void setBOOLNONNULLABLE(boolean value) { * Creates a new TestAvroNested RecordBuilder. * @return A new TestAvroNested RecordBuilder */ - public static Builder newBuilder() { - return new Builder(); + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder() { + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); } /** @@ -169,11 +171,11 @@ public static Builder newBuilder() { * @param other The existing builder to copy. * @return A new TestAvroNested RecordBuilder */ - public static Builder newBuilder(Builder other) { + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { if (other == null) { - return new Builder(); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); } else { - return new Builder(other); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); } } @@ -182,11 +184,11 @@ public static Builder newBuilder(Builder other) { * @param other The existing instance to copy. * @return A new TestAvroNested RecordBuilder */ - public static Builder newBuilder(TestAvroNested other) { + public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { if (other == null) { - return new Builder(); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); } else { - return new Builder(other); + return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); } } @@ -198,7 +200,7 @@ public static class Builder extends org.apache.avro.specific.SpecificRecordBuild implements org.apache.avro.data.RecordBuilder { private boolean BOOL_NON_NULLABLE; - private Integer int$; + private java.lang.Integer int$; /** Creates a new Builder */ private Builder() { @@ -209,7 +211,7 @@ private Builder() { * Creates a Builder by copying an existing Builder. * @param other The existing Builder to copy. */ - private Builder(Builder other) { + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { super(other); if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); @@ -225,7 +227,7 @@ private Builder(Builder other) { * Creates a Builder by copying an existing TestAvroNested instance * @param other The existing instance to copy. */ - private Builder(TestAvroNested other) { + private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { super(SCHEMA$); if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); @@ -251,7 +253,7 @@ public boolean getBOOLNONNULLABLE() { * @param value The value of 'BOOL_NON_NULLABLE'. * @return This builder. */ - public Builder setBOOLNONNULLABLE(boolean value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setBOOLNONNULLABLE(boolean value) { validate(fields()[0], value); this.BOOL_NON_NULLABLE = value; fieldSetFlags()[0] = true; @@ -271,7 +273,7 @@ public boolean hasBOOLNONNULLABLE() { * Clears the value of the 'BOOL_NON_NULLABLE' field. * @return This builder. */ - public Builder clearBOOLNONNULLABLE() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearBOOLNONNULLABLE() { fieldSetFlags()[0] = false; return this; } @@ -280,7 +282,7 @@ public Builder clearBOOLNONNULLABLE() { * Gets the value of the 'int$' field. * @return The value. */ - public Integer getInt$() { + public java.lang.Integer getInt$() { return int$; } @@ -290,7 +292,7 @@ public Builder clearBOOLNONNULLABLE() { * @param value The value of 'int$'. * @return This builder. */ - public Builder setInt$(Integer value) { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setInt$(java.lang.Integer value) { validate(fields()[1], value); this.int$ = value; fieldSetFlags()[1] = true; @@ -310,7 +312,7 @@ public Builder clearBOOLNONNULLABLE() { * Clears the value of the 'int$' field. * @return This builder. */ - public Builder clearInt$() { + public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearInt$() { int$ = null; fieldSetFlags()[1] = false; return this; @@ -321,12 +323,12 @@ public Builder clearBOOLNONNULLABLE() { public TestAvroNested build() { try { TestAvroNested record = new TestAvroNested(); - record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (Integer) defaultValue(fields()[1]); + record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (java.lang.Boolean) defaultValue(fields()[0]); + record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); return record; } catch (org.apache.avro.AvroMissingFieldException e) { throw e; - } catch (Exception e) { + } catch (java.lang.Exception e) { throw new org.apache.avro.AvroRuntimeException(e); } } diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java index 7dfc8049eb33..ab3f4ec5398c 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java @@ -33,6 +33,7 @@ import java.io.ByteArrayOutputStream; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; @@ -55,12 +56,17 @@ import org.apache.avro.reflect.Stringable; import org.apache.avro.reflect.Union; import org.apache.avro.specific.SpecificData; +import org.apache.avro.specific.SpecificRecord; import org.apache.avro.util.Utf8; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.extensions.avro.schemas.TestAvro; +import org.apache.beam.sdk.extensions.avro.schemas.TestAvroFactory; +import org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested; +import org.apache.beam.sdk.extensions.avro.schemas.TestEnum; +import org.apache.beam.sdk.extensions.avro.schemas.fixed4; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.InterceptingUrlClassLoader; import org.apache.beam.sdk.testing.NeedsRunner; @@ -74,6 +80,8 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -81,6 +89,7 @@ import org.hamcrest.TypeSafeMatcher; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; +import org.joda.time.LocalDate; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -96,6 +105,24 @@ public class AvroCoderTest { new DateTime().withDate(1994, 10, 31).withZone(DateTimeZone.UTC); public static final DateTime DATETIME_B = new DateTime().withDate(1997, 4, 25).withZone(DateTimeZone.UTC); + private static final TestAvroNested AVRO_NESTED_SPECIFIC_RECORD = new TestAvroNested(true, 42); + private static final TestAvro AVRO_SPECIFIC_RECORD = + TestAvroFactory.newInstance( + true, + 43, + 44L, + 44.1f, + 44.2d, + "mystring", + ByteBuffer.wrap(new byte[] {1, 2, 3, 4}), + new fixed4(new byte[] {1, 2, 3, 4}), + new LocalDate(1979, 3, 14), + new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4), + TestEnum.abc, + AVRO_NESTED_SPECIFIC_RECORD, + ImmutableList.of(AVRO_NESTED_SPECIFIC_RECORD, AVRO_NESTED_SPECIFIC_RECORD), + ImmutableMap.of("k1", AVRO_NESTED_SPECIFIC_RECORD, "k2", AVRO_NESTED_SPECIFIC_RECORD)); + private static final String VERSION_AVRO = Schema.class.getPackage().getImplementationVersion(); @DefaultCoder(AvroCoder.class) private static class Pojo { @@ -301,6 +328,38 @@ public void testPojoEncoding() throws Exception { CoderProperties.coderDecodeEncodeEqual(coder, value); } + @Test + public void testSpecificRecordEncoding() throws Exception { + if (isBrokenMapComparison()) { + // Don't compare the map values because of AVRO-2943 + AVRO_SPECIFIC_RECORD.setMap(ImmutableMap.of()); + } + AvroCoder coder = + AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), false); + + assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); + CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); + } + + private boolean isBrokenMapComparison() { + return VERSION_AVRO.equals("1.9.2") + || VERSION_AVRO.equals("1.10.2") + || VERSION_AVRO.equals("1.11.1"); + } + + @Test + public void testReflectRecordEncoding() throws Exception { + AvroCoder coder = AvroCoder.of(TestAvro.class, true); + AvroCoder coderWithSchema = + AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), true); + + assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); + assertTrue(SpecificRecord.class.isAssignableFrom(coderWithSchema.getType())); + + CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); + CoderProperties.coderDecodeEncodeEqual(coderWithSchema, AVRO_SPECIFIC_RECORD); + } + @Test public void testDisableReflectionEncoding() { try { diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/SpecificRecordTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/SpecificRecordTest.java deleted file mode 100644 index 8063437b8465..000000000000 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/SpecificRecordTest.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.coders; - -import static org.junit.Assert.assertTrue; - -import java.nio.ByteBuffer; -import org.apache.avro.specific.SpecificRecord; -import org.apache.beam.sdk.extensions.avro.schemas.TestAvro; -import org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested; -import org.apache.beam.sdk.extensions.avro.schemas.TestEnum; -import org.apache.beam.sdk.extensions.avro.schemas.fixed4; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; -import org.joda.time.DateTime; -import org.joda.time.LocalDate; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class SpecificRecordTest { - private static final TestAvroNested AVRO_NESTED_SPECIFIC_RECORD = new TestAvroNested(true, 42); - - private static final TestAvro AVRO_SPECIFIC_RECORD = - new TestAvro( - true, - 43, - 44L, - 44.1f, - 44.2d, - "mystring", - ByteBuffer.wrap(new byte[] {1, 2, 3, 4}), - new fixed4(new byte[] {1, 2, 3, 4}), - new LocalDate(1979, 3, 14), - new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4), - TestEnum.abc, - AVRO_NESTED_SPECIFIC_RECORD, - ImmutableList.of(AVRO_NESTED_SPECIFIC_RECORD, AVRO_NESTED_SPECIFIC_RECORD), - ImmutableMap.of("k1", AVRO_NESTED_SPECIFIC_RECORD, "k2", AVRO_NESTED_SPECIFIC_RECORD)); - - @Test - public void testSpecificRecordEncoding() throws Exception { - AvroCoder coder = - AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), false); - - assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); - CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); - } - - @Test - public void testReflectRecordEncoding() throws Exception { - AvroCoder coder = AvroCoder.of(TestAvro.class, true); - AvroCoder coderWithSchema = - AvroCoder.of(TestAvro.class, AVRO_SPECIFIC_RECORD.getSchema(), true); - - assertTrue(SpecificRecord.class.isAssignableFrom(coder.getType())); - assertTrue(SpecificRecord.class.isAssignableFrom(coderWithSchema.getType())); - - CoderProperties.coderDecodeEncodeEqual(coder, AVRO_SPECIFIC_RECORD); - CoderProperties.coderDecodeEncodeEqual(coderWithSchema, AVRO_SPECIFIC_RECORD); - } -} diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUserFactory.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUserFactory.java new file mode 100644 index 000000000000..066b65c5d5e6 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUserFactory.java @@ -0,0 +1,49 @@ +/* + * 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.beam.sdk.extensions.avro.io; + +import java.lang.reflect.Constructor; +import org.apache.avro.Schema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Create a {@link AvroGeneratedUser} instance with different constructors. */ +public class AvroGeneratedUserFactory { + private static final Logger LOG = LoggerFactory.getLogger(AvroGeneratedUserFactory.class); + private static final String VERSION_AVRO = Schema.class.getPackage().getImplementationVersion(); + + public static AvroGeneratedUser newInstance( + String name, Integer favoriteNumber, String favoriteColor) { + + if (VERSION_AVRO.equals("1.8.2")) { + return new AvroGeneratedUser(name, favoriteNumber, favoriteColor); + } else { + try { + Constructor constructor; + constructor = + AvroGeneratedUser.class.getDeclaredConstructor( + CharSequence.class, Integer.class, CharSequence.class); + + return (AvroGeneratedUser) constructor.newInstance(name, favoriteNumber, favoriteColor); + } catch (ReflectiveOperationException e) { + LOG.error(String.format("Fail to create a AvroGeneratedUser instance: %s", e.getMessage())); + return new AvroGeneratedUser(); // return an empty instance to fail the tests + } + } + } +} diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java index 7456435948d3..158bf33cc875 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java @@ -367,9 +367,9 @@ private void testWriteThenReadGeneratedClass( List values = ImmutableList.of( - (T) new AvroGeneratedUser("Bob", 256, null), - (T) new AvroGeneratedUser("Alice", 128, null), - (T) new AvroGeneratedUser("Ted", null, "white")); + (T) AvroGeneratedUserFactory.newInstance("Bob", 256, null), + (T) AvroGeneratedUserFactory.newInstance("Alice", 128, null), + (T) AvroGeneratedUserFactory.newInstance("Ted", null, "white")); writePipeline .apply(Create.of(values)) diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java index f08c51b09711..9dfdbd499e1e 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java @@ -306,7 +306,7 @@ public String toString() { private static final LocalDate DATE = new LocalDate(1979, 3, 14); private static final TestAvroNested AVRO_NESTED_SPECIFIC_RECORD = new TestAvroNested(true, 42); private static final TestAvro AVRO_SPECIFIC_RECORD = - new TestAvro( + TestAvroFactory.newInstance( true, 43, 44L, diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java new file mode 100644 index 000000000000..3387f7bff8c5 --- /dev/null +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java @@ -0,0 +1,109 @@ +/* + * 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.beam.sdk.extensions.avro.schemas; + +import java.lang.reflect.Constructor; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import org.apache.avro.Schema; +import org.joda.time.DateTime; +import org.joda.time.LocalDate; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Create a {@link TestAvro} instance with different constructors. */ +public class TestAvroFactory { + private static final Logger LOG = LoggerFactory.getLogger(TestAvroFactory.class); + private static final String VERSION_AVRO = Schema.class.getPackage().getImplementationVersion(); + + public static TestAvro newInstance( + Boolean boolNonNullable, + Integer int$, + Long long$, + Float float$, + Double double$, + String string, + ByteBuffer bytes, + fixed4 fixed, + LocalDate date, + DateTime timestampMillis, + TestEnum testEnum, + TestAvroNested row, + List array, + Map map) { + + if (VERSION_AVRO.equals("1.8.2")) { + return new TestAvro( + boolNonNullable, + int$, + long$, + float$, + double$, + string, + bytes, + fixed, + date, + timestampMillis, + testEnum, + row, + array, + map); + } else { + try { + Constructor constructor; + constructor = + TestAvro.class.getDeclaredConstructor( + Boolean.class, + Integer.class, + Long.class, + Float.class, + Double.class, + CharSequence.class, + ByteBuffer.class, + fixed4.class, + java.time.LocalDate.class, + java.time.Instant.class, + TestEnum.class, + TestAvroNested.class, + java.util.List.class, + java.util.Map.class); + + return (TestAvro) + constructor.newInstance( + boolNonNullable, + int$, + long$, + float$, + double$, + string, + bytes, + fixed, + java.time.LocalDate.of(date.getYear(), date.getMonthOfYear(), date.getDayOfMonth()), + java.time.Instant.ofEpochMilli(timestampMillis.getMillis()), + testEnum, + row, + array, + map); + } catch (ReflectiveOperationException e) { + LOG.error(String.format("Fail to create a TestAvro instance: %s", e.getMessage())); + return new TestAvro(); // return an empty instance to fail the tests + } + } + } +} diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java index ac2ec14a2900..fa3d3343ac6e 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser; +import org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUserFactory; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; @@ -803,7 +804,7 @@ public void testAvroSchemaCoders() { assertTrue(records.hasSchema()); CoderProperties.coderSerializable(records.getCoder()); - AvroGeneratedUser user = new AvroGeneratedUser("foo", 42, "green"); + AvroGeneratedUser user = AvroGeneratedUserFactory.newInstance("foo", 42, "green"); PCollection users = pipeline.apply(Create.of(user).withCoder(AvroCoder.of(AvroGeneratedUser.class))); assertFalse(users.hasSchema()); From 055187a8712c2d97cbeb387ee6abda1013f7675c Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Thu, 13 Apr 2023 12:55:31 +0200 Subject: [PATCH 6/6] Generate Avro Pojos with avro-tools for different Avro versions --- .test-infra/jenkins/job_PreCommit_Java.groovy | 4 +- sdks/java/extensions/avro/build.gradle | 45 +- .../extensions/avro/io/AvroGeneratedUser.java | 515 ------- .../sdk/extensions/avro/schemas/TestAvro.java | 1305 ---------------- .../avro/schemas/TestAvroNested.java | 418 ------ .../sdk/extensions/avro/schemas/TestEnum.java | 25 - .../sdk/extensions/avro/schemas/fixed4.java | 56 - .../extensions/avro/io/AvroGeneratedUser.java | 523 ------- .../sdk/extensions/avro/schemas/TestAvro.java | 1313 ----------------- .../avro/schemas/TestAvroNested.java | 426 ------ .../sdk/extensions/avro/schemas/TestEnum.java | 27 - .../sdk/extensions/avro/schemas/fixed4.java | 56 - .../extensions/avro/io/AvroGeneratedUser.java | 515 ------- .../sdk/extensions/avro/schemas/TestAvro.java | 1304 ---------------- .../avro/schemas/TestAvroNested.java | 418 ------ .../sdk/extensions/avro/schemas/TestEnum.java | 25 - .../sdk/extensions/avro/schemas/fixed4.java | 56 - .../avro/schemas/TestAvroFactory.java | 24 +- 18 files changed, 45 insertions(+), 7010 deletions(-) delete mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java delete mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java delete mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java delete mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java delete mode 100644 sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java delete mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java delete mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java delete mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java delete mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java delete mode 100644 sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java delete mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java delete mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java delete mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java delete mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java delete mode 100644 sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java diff --git a/.test-infra/jenkins/job_PreCommit_Java.groovy b/.test-infra/jenkins/job_PreCommit_Java.groovy index e1fc2ee1080c..e3a79709563f 100644 --- a/.test-infra/jenkins/job_PreCommit_Java.groovy +++ b/.test-infra/jenkins/job_PreCommit_Java.groovy @@ -20,6 +20,7 @@ import PrecommitJobBuilder // exclude paths with their own PreCommit tasks def excludePaths = [ + 'extensions/avro', 'extensions/sql', 'io/amazon-web-services', 'io/amazon-web-services2', @@ -112,9 +113,6 @@ builder.build { execPattern('**/build/jacoco/*.exec') exclusionPattern('**/org/apache/beam/gradle/**,**/org/apache/beam/model/**,' + '**/org/apache/beam/runners/dataflow/worker/windmill/**,**/AutoValue_*,' + - '**/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.*,' + - '**/org/apache/beam/sdk/extensions/avro/schemas/Test*,' + - '**/org/apache/beam/sdk/extensions/avro/schemas/fixed4.*,' + excludePaths.collect {entry -> getModuleNameFromProject(entry) }.join(",") ) } } diff --git a/sdks/java/extensions/avro/build.gradle b/sdks/java/extensions/avro/build.gradle index 9ff56981b432..4aa765c08548 100644 --- a/sdks/java/extensions/avro/build.gradle +++ b/sdks/java/extensions/avro/build.gradle @@ -30,12 +30,12 @@ applyAvroNature() description = "Apache Beam :: SDKs :: Java :: Extensions :: Avro" def avroVersions = [ - '192': "1.9.2", + '192' : "1.9.2", '1102': "1.10.2", '1111': "1.11.1", ] -avroVersions.each{k,v -> configurations.create("avroVersion$k")} +avroVersions.each { k, v -> configurations.create("avroVersion$k") } // Exclude tests that need a runner test { @@ -48,14 +48,14 @@ test { dependencies { implementation library.java.byte_buddy implementation library.java.vendored_guava_26_0_jre - implementation (project(path: ":sdks:java:core", configuration: "shadow")) { + implementation(project(path: ":sdks:java:core", configuration: "shadow")) { // Exclude Avro dependencies from "core" since Avro support moved to this extension exclude group: "org.apache.avro", module: "avro" } implementation library.java.error_prone_annotations implementation library.java.avro implementation library.java.joda_time - testImplementation (project(path: ":sdks:java:core", configuration: "shadowTest")) { + testImplementation(project(path: ":sdks:java:core", configuration: "shadowTest")) { // Exclude Avro dependencies from "core" since Avro support moved to this extension exclude group: "org.apache.avro", module: "avro" } @@ -63,10 +63,13 @@ dependencies { testImplementation library.java.junit testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") testRuntimeOnly library.java.slf4j_jdk14 - avroVersions.each {"avroVersion$it.key" "org.apache.avro:avro:$it.value"} + avroVersions.each { + "avroVersion$it.key" "org.apache.avro:avro:$it.value" + "avroVersion$it.key" "org.apache.avro:avro-tools:$it.value" + } } -avroVersions.each {kv -> +avroVersions.each { kv -> configurations."avroVersion$kv.key" { resolutionStrategy { force "org.apache.avro:avro:$kv.value" @@ -76,7 +79,7 @@ avroVersions.each {kv -> sourceSets { "avro${kv.key}" { java { - srcDirs "src/test/avro${kv.key}" + srcDirs "build/generated/sources/avro${kv.key}/test/java" } compileClasspath = configurations."avroVersion$kv.key" + sourceSets.test.output + sourceSets.test.compileClasspath @@ -85,27 +88,43 @@ avroVersions.each {kv -> } "compileAvro${kv.key}Java" { - options.warnings = false checkerFramework { skipCheckerFramework = true } - checkstyle { - ignoreFailures = true - } } "spotbugsAvro${kv.key}" { ignoreFailures = true } + "generateAvro${kv.key}AvroJava" { + dependsOn "generateAvroClasses${kv.key}" + } + task "avroVersion${kv.key}Test"(type: Test) { group = "Verification" - description = "Runs Avro extension tests with Avro API $kv.value" + description = "Runs Avro extension tests with Avro version $kv.value" outputs.upToDateWhen { false } classpath = sourceSets."avro${kv.key}".runtimeClasspath include '**/*.class' exclude '**/AvroIOTest$NeedsRunnerTests$*.class' + + dependsOn "generateAvroClasses${kv.key}" + } + + task "generateAvroClasses${kv.key}"(type: JavaExec) { + group = "build" + description = "Generate Avro classes for Avro version $kv.value" + classpath = configurations."avroVersion$kv.key" + main = "org.apache.avro.tool.Main" + args = [ + "compile", + "schema", + "src/test/avro/org/apache/beam/sdk/extensions/avro/io/user.avsc", + "src/test/avro/org/apache/beam/sdk/extensions/avro/schemas/test.avsc", + "build/generated/sources/avro${kv.key}/test/java" + ] } } @@ -117,6 +136,6 @@ task avroVersionsTest { static def createTaskNames(Map prefixMap, String suffix) { return prefixMap.keySet().stream() - .map{version -> "avroVersion${version}${suffix}"} + .map { version -> "avroVersion${version}${suffix}" } .collect(Collectors.toList()) } diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java deleted file mode 100644 index 3920ca5374d4..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java +++ /dev/null @@ -1,515 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.io; - -import org.apache.avro.generic.GenericArray; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.util.Utf8; -import org.apache.avro.message.BinaryMessageEncoder; -import org.apache.avro.message.BinaryMessageDecoder; -import org.apache.avro.message.SchemaStore; - -@org.apache.avro.specific.AvroGenerated -public class AvroGeneratedUser extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - private static final long serialVersionUID = 3056733241446362297L; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroGeneratedUser\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.io\",\"fields\":[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"favorite_number\",\"type\":[\"int\",\"null\"]},{\"name\":\"favorite_color\",\"type\":[\"string\",\"null\"]}]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - private static SpecificData MODEL$ = new SpecificData(); - - private static final BinaryMessageEncoder ENCODER = - new BinaryMessageEncoder(MODEL$, SCHEMA$); - - private static final BinaryMessageDecoder DECODER = - new BinaryMessageDecoder(MODEL$, SCHEMA$); - - /** - * Return the BinaryMessageEncoder instance used by this class. - * @return the message encoder used by this class - */ - public static BinaryMessageEncoder getEncoder() { - return ENCODER; - } - - /** - * Return the BinaryMessageDecoder instance used by this class. - * @return the message decoder used by this class - */ - public static BinaryMessageDecoder getDecoder() { - return DECODER; - } - - /** - * 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 - */ - public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { - return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); - } - - /** - * Serializes this AvroGeneratedUser to a ByteBuffer. - * @return a buffer holding the serialized data for this instance - * @throws java.io.IOException if this instance could not be serialized - */ - public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { - return ENCODER.encode(this); - } - - /** - * Deserializes a AvroGeneratedUser from a ByteBuffer. - * @param b a byte buffer holding serialized data for an instance of this class - * @return a AvroGeneratedUser instance decoded from the given buffer - * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class - */ - public static AvroGeneratedUser fromByteBuffer( - java.nio.ByteBuffer b) throws java.io.IOException { - return DECODER.decode(b); - } - - private java.lang.CharSequence name; - private java.lang.Integer favorite_number; - private java.lang.CharSequence favorite_color; - - /** - * Default constructor. Note that this does not initialize fields - * to their default values from the schema. If that is desired then - * one should use newBuilder(). - */ - public AvroGeneratedUser() {} - - /** - * All-args constructor. - * @param name The new value for name - * @param favorite_number The new value for favorite_number - * @param favorite_color The new value for favorite_color - */ - public AvroGeneratedUser(java.lang.CharSequence name, java.lang.Integer favorite_number, java.lang.CharSequence favorite_color) { - this.name = name; - this.favorite_number = favorite_number; - this.favorite_color = favorite_color; - } - - public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - // Used by DatumWriter. Applications should not call. - public java.lang.Object get(int field$) { - switch (field$) { - case 0: return name; - case 1: return favorite_number; - case 2: return favorite_color; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - // Used by DatumReader. Applications should not call. - @SuppressWarnings(value="unchecked") - public void put(int field$, java.lang.Object value$) { - switch (field$) { - case 0: name = (java.lang.CharSequence)value$; break; - case 1: favorite_number = (java.lang.Integer)value$; break; - case 2: favorite_color = (java.lang.CharSequence)value$; break; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - /** - * Gets the value of the 'name' field. - * @return The value of the 'name' field. - */ - public java.lang.CharSequence getName() { - return name; - } - - - /** - * Sets the value of the 'name' field. - * @param value the value to set. - */ - public void setName(java.lang.CharSequence value) { - this.name = value; - } - - /** - * Gets the value of the 'favorite_number' field. - * @return The value of the 'favorite_number' field. - */ - public java.lang.Integer getFavoriteNumber() { - return favorite_number; - } - - - /** - * Sets the value of the 'favorite_number' field. - * @param value the value to set. - */ - public void setFavoriteNumber(java.lang.Integer value) { - this.favorite_number = value; - } - - /** - * Gets the value of the 'favorite_color' field. - * @return The value of the 'favorite_color' field. - */ - public java.lang.CharSequence getFavoriteColor() { - return favorite_color; - } - - - /** - * Sets the value of the 'favorite_color' field. - * @param value the value to set. - */ - public void setFavoriteColor(java.lang.CharSequence value) { - this.favorite_color = value; - } - - /** - * Creates a new AvroGeneratedUser RecordBuilder. - * @return A new AvroGeneratedUser RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder() { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); - } - - /** - * Creates a new AvroGeneratedUser RecordBuilder by copying an existing Builder. - * @param other The existing builder to copy. - * @return A new AvroGeneratedUser RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); - } - } - - /** - * Creates a new AvroGeneratedUser RecordBuilder by copying an existing AvroGeneratedUser instance. - * @param other The existing instance to copy. - * @return A new AvroGeneratedUser RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); - } - } - - /** - * RecordBuilder for AvroGeneratedUser instances. - */ - @org.apache.avro.specific.AvroGenerated - public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase - implements org.apache.avro.data.RecordBuilder { - - private java.lang.CharSequence name; - private java.lang.Integer favorite_number; - private java.lang.CharSequence favorite_color; - - /** Creates a new Builder */ - private Builder() { - super(SCHEMA$); - } - - /** - * Creates a Builder by copying an existing Builder. - * @param other The existing Builder to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { - super(other); - if (isValidValue(fields()[0], other.name)) { - this.name = data().deepCopy(fields()[0].schema(), other.name); - fieldSetFlags()[0] = other.fieldSetFlags()[0]; - } - if (isValidValue(fields()[1], other.favorite_number)) { - this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); - fieldSetFlags()[1] = other.fieldSetFlags()[1]; - } - if (isValidValue(fields()[2], other.favorite_color)) { - this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); - fieldSetFlags()[2] = other.fieldSetFlags()[2]; - } - } - - /** - * Creates a Builder by copying an existing AvroGeneratedUser instance - * @param other The existing instance to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { - super(SCHEMA$); - if (isValidValue(fields()[0], other.name)) { - this.name = data().deepCopy(fields()[0].schema(), other.name); - fieldSetFlags()[0] = true; - } - if (isValidValue(fields()[1], other.favorite_number)) { - this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); - fieldSetFlags()[1] = true; - } - if (isValidValue(fields()[2], other.favorite_color)) { - this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); - fieldSetFlags()[2] = true; - } - } - - /** - * Gets the value of the 'name' field. - * @return The value. - */ - public java.lang.CharSequence getName() { - return name; - } - - - /** - * Sets the value of the 'name' field. - * @param value The value of 'name'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setName(java.lang.CharSequence value) { - validate(fields()[0], value); - this.name = value; - fieldSetFlags()[0] = true; - return this; - } - - /** - * Checks whether the 'name' field has been set. - * @return True if the 'name' field has been set, false otherwise. - */ - public boolean hasName() { - return fieldSetFlags()[0]; - } - - - /** - * Clears the value of the 'name' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearName() { - name = null; - fieldSetFlags()[0] = false; - return this; - } - - /** - * Gets the value of the 'favorite_number' field. - * @return The value. - */ - public java.lang.Integer getFavoriteNumber() { - return favorite_number; - } - - - /** - * Sets the value of the 'favorite_number' field. - * @param value The value of 'favorite_number'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteNumber(java.lang.Integer value) { - validate(fields()[1], value); - this.favorite_number = value; - fieldSetFlags()[1] = true; - return this; - } - - /** - * Checks whether the 'favorite_number' field has been set. - * @return True if the 'favorite_number' field has been set, false otherwise. - */ - public boolean hasFavoriteNumber() { - return fieldSetFlags()[1]; - } - - - /** - * Clears the value of the 'favorite_number' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteNumber() { - favorite_number = null; - fieldSetFlags()[1] = false; - return this; - } - - /** - * Gets the value of the 'favorite_color' field. - * @return The value. - */ - public java.lang.CharSequence getFavoriteColor() { - return favorite_color; - } - - - /** - * Sets the value of the 'favorite_color' field. - * @param value The value of 'favorite_color'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteColor(java.lang.CharSequence value) { - validate(fields()[2], value); - this.favorite_color = value; - fieldSetFlags()[2] = true; - return this; - } - - /** - * Checks whether the 'favorite_color' field has been set. - * @return True if the 'favorite_color' field has been set, false otherwise. - */ - public boolean hasFavoriteColor() { - return fieldSetFlags()[2]; - } - - - /** - * Clears the value of the 'favorite_color' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteColor() { - favorite_color = null; - fieldSetFlags()[2] = false; - return this; - } - - @Override - @SuppressWarnings("unchecked") - public AvroGeneratedUser build() { - try { - AvroGeneratedUser record = new AvroGeneratedUser(); - record.name = fieldSetFlags()[0] ? this.name : (java.lang.CharSequence) defaultValue(fields()[0]); - record.favorite_number = fieldSetFlags()[1] ? this.favorite_number : (java.lang.Integer) defaultValue(fields()[1]); - record.favorite_color = fieldSetFlags()[2] ? this.favorite_color : (java.lang.CharSequence) defaultValue(fields()[2]); - return record; - } catch (org.apache.avro.AvroMissingFieldException e) { - throw e; - } catch (java.lang.Exception e) { - throw new org.apache.avro.AvroRuntimeException(e); - } - } - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumWriter - WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, SpecificData.getEncoder(out)); - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumReader - READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, SpecificData.getDecoder(in)); - } - - @Override protected boolean hasCustomCoders() { return true; } - - @Override public void customEncode(org.apache.avro.io.Encoder out) - throws java.io.IOException - { - out.writeString(this.name); - - if (this.favorite_number == null) { - out.writeIndex(1); - out.writeNull(); - } else { - out.writeIndex(0); - out.writeInt(this.favorite_number); - } - - if (this.favorite_color == null) { - out.writeIndex(1); - out.writeNull(); - } else { - out.writeIndex(0); - out.writeString(this.favorite_color); - } - - } - - @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) - throws java.io.IOException - { - org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); - if (fieldOrder == null) { - this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); - - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_number = null; - } else { - this.favorite_number = in.readInt(); - } - - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_color = null; - } else { - this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); - } - - } else { - for (int i = 0; i < 3; i++) { - switch (fieldOrder[i].pos()) { - case 0: - this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); - break; - - case 1: - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_number = null; - } else { - this.favorite_number = in.readInt(); - } - break; - - case 2: - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_color = null; - } else { - this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); - } - break; - - default: - throw new java.io.IOException("Corrupt ResolvingDecoder."); - } - } - } - } -} - - - - - - - - - - diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java deleted file mode 100644 index c352e89410b9..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java +++ /dev/null @@ -1,1305 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; - -import org.apache.avro.generic.GenericArray; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.util.Utf8; -import org.apache.avro.message.BinaryMessageEncoder; -import org.apache.avro.message.BinaryMessageDecoder; -import org.apache.avro.message.SchemaStore; - -@org.apache.avro.specific.AvroGenerated -public class TestAvro extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - private static final long serialVersionUID = 27902431178981259L; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvro\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"bool_non_nullable\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]},{\"name\":\"long\",\"type\":[\"long\",\"null\"]},{\"name\":\"float\",\"type\":[\"float\",\"null\"]},{\"name\":\"double\",\"type\":[\"double\",\"null\"]},{\"name\":\"string\",\"type\":[\"string\",\"null\"]},{\"name\":\"bytes\",\"type\":[\"bytes\",\"null\"]},{\"name\":\"fixed\",\"type\":{\"type\":\"fixed\",\"name\":\"fixed4\",\"size\":4}},{\"name\":\"date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},{\"name\":\"timestampMillis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}},{\"name\":\"TestEnum\",\"type\":{\"type\":\"enum\",\"name\":\"TestEnum\",\"symbols\":[\"abc\",\"cde\"]}},{\"name\":\"row\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"TestAvroNested\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}]},{\"name\":\"array\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\",\"TestAvroNested\"]}]},{\"name\":\"map\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"TestAvroNested\"]}]}]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - private static SpecificData MODEL$ = new SpecificData(); -static { - MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.DateConversion()); - MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.TimestampMillisConversion()); - } - - private static final BinaryMessageEncoder ENCODER = - new BinaryMessageEncoder(MODEL$, SCHEMA$); - - private static final BinaryMessageDecoder DECODER = - new BinaryMessageDecoder(MODEL$, SCHEMA$); - - /** - * Return the BinaryMessageEncoder instance used by this class. - * @return the message encoder used by this class - */ - public static BinaryMessageEncoder getEncoder() { - return ENCODER; - } - - /** - * Return the BinaryMessageDecoder instance used by this class. - * @return the message decoder used by this class - */ - public static BinaryMessageDecoder getDecoder() { - return DECODER; - } - - /** - * 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 - */ - public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { - return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); - } - - /** - * Serializes this TestAvro to a ByteBuffer. - * @return a buffer holding the serialized data for this instance - * @throws java.io.IOException if this instance could not be serialized - */ - public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { - return ENCODER.encode(this); - } - - /** - * Deserializes a TestAvro from a ByteBuffer. - * @param b a byte buffer holding serialized data for an instance of this class - * @return a TestAvro instance decoded from the given buffer - * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class - */ - public static TestAvro fromByteBuffer( - java.nio.ByteBuffer b) throws java.io.IOException { - return DECODER.decode(b); - } - - private boolean bool_non_nullable; - private java.lang.Integer int$; - private java.lang.Long long$; - private java.lang.Float float$; - private java.lang.Double double$; - private java.lang.CharSequence string; - private java.nio.ByteBuffer bytes; - private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; - private java.time.LocalDate date; - private java.time.Instant timestampMillis; - private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; - private java.util.List array; - private java.util.Map map; - - /** - * Default constructor. Note that this does not initialize fields - * to their default values from the schema. If that is desired then - * one should use newBuilder(). - */ - public TestAvro() {} - - /** - * All-args constructor. - * @param bool_non_nullable The new value for bool_non_nullable - * @param int$ The new value for int - * @param long$ The new value for long - * @param float$ The new value for float - * @param double$ The new value for double - * @param string The new value for string - * @param bytes The new value for bytes - * @param fixed The new value for fixed - * @param date The new value for date - * @param timestampMillis The new value for timestampMillis - * @param TestEnum The new value for TestEnum - * @param row The new value for row - * @param array The new value for array - * @param map The new value for map - */ - public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.CharSequence string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { - this.bool_non_nullable = bool_non_nullable; - this.int$ = int$; - this.long$ = long$; - this.float$ = float$; - this.double$ = double$; - this.string = string; - this.bytes = bytes; - this.fixed = fixed; - this.date = date; - this.timestampMillis = timestampMillis.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); - this.TestEnum = TestEnum; - this.row = row; - this.array = array; - this.map = map; - } - - public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - // Used by DatumWriter. Applications should not call. - public java.lang.Object get(int field$) { - switch (field$) { - case 0: return bool_non_nullable; - case 1: return int$; - case 2: return long$; - case 3: return float$; - case 4: return double$; - case 5: return string; - case 6: return bytes; - case 7: return fixed; - case 8: return date; - case 9: return timestampMillis; - case 10: return TestEnum; - case 11: return row; - case 12: return array; - case 13: return map; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - private static final org.apache.avro.Conversion[] conversions = - new org.apache.avro.Conversion[] { - null, - null, - null, - null, - null, - null, - null, - null, - new org.apache.avro.data.TimeConversions.DateConversion(), - new org.apache.avro.data.TimeConversions.TimestampMillisConversion(), - null, - null, - null, - null, - null - }; - - @Override - public org.apache.avro.Conversion getConversion(int field) { - return conversions[field]; - } - - // Used by DatumReader. Applications should not call. - @SuppressWarnings(value="unchecked") - public void put(int field$, java.lang.Object value$) { - switch (field$) { - case 0: bool_non_nullable = (java.lang.Boolean)value$; break; - case 1: int$ = (java.lang.Integer)value$; break; - case 2: long$ = (java.lang.Long)value$; break; - case 3: float$ = (java.lang.Float)value$; break; - case 4: double$ = (java.lang.Double)value$; break; - case 5: string = (java.lang.CharSequence)value$; break; - case 6: bytes = (java.nio.ByteBuffer)value$; break; - case 7: fixed = (org.apache.beam.sdk.extensions.avro.schemas.fixed4)value$; break; - case 8: date = (java.time.LocalDate)value$; break; - case 9: timestampMillis = (java.time.Instant)value$; break; - case 10: TestEnum = (org.apache.beam.sdk.extensions.avro.schemas.TestEnum)value$; break; - case 11: row = (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested)value$; break; - case 12: array = (java.util.List)value$; break; - case 13: map = (java.util.Map)value$; break; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - /** - * Gets the value of the 'bool_non_nullable' field. - * @return The value of the 'bool_non_nullable' field. - */ - public boolean getBoolNonNullable() { - return bool_non_nullable; - } - - - /** - * Sets the value of the 'bool_non_nullable' field. - * @param value the value to set. - */ - public void setBoolNonNullable(boolean value) { - this.bool_non_nullable = value; - } - - /** - * Gets the value of the 'int$' field. - * @return The value of the 'int$' field. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value the value to set. - */ - public void setInt$(java.lang.Integer value) { - this.int$ = value; - } - - /** - * Gets the value of the 'long$' field. - * @return The value of the 'long$' field. - */ - public java.lang.Long getLong$() { - return long$; - } - - - /** - * Sets the value of the 'long$' field. - * @param value the value to set. - */ - public void setLong$(java.lang.Long value) { - this.long$ = value; - } - - /** - * Gets the value of the 'float$' field. - * @return The value of the 'float$' field. - */ - public java.lang.Float getFloat$() { - return float$; - } - - - /** - * Sets the value of the 'float$' field. - * @param value the value to set. - */ - public void setFloat$(java.lang.Float value) { - this.float$ = value; - } - - /** - * Gets the value of the 'double$' field. - * @return The value of the 'double$' field. - */ - public java.lang.Double getDouble$() { - return double$; - } - - - /** - * Sets the value of the 'double$' field. - * @param value the value to set. - */ - public void setDouble$(java.lang.Double value) { - this.double$ = value; - } - - /** - * Gets the value of the 'string' field. - * @return The value of the 'string' field. - */ - public java.lang.CharSequence getString() { - return string; - } - - - /** - * Sets the value of the 'string' field. - * @param value the value to set. - */ - public void setString(java.lang.CharSequence value) { - this.string = value; - } - - /** - * Gets the value of the 'bytes' field. - * @return The value of the 'bytes' field. - */ - public java.nio.ByteBuffer getBytes() { - return bytes; - } - - - /** - * Sets the value of the 'bytes' field. - * @param value the value to set. - */ - public void setBytes(java.nio.ByteBuffer value) { - this.bytes = value; - } - - /** - * Gets the value of the 'fixed' field. - * @return The value of the 'fixed' field. - */ - public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { - return fixed; - } - - - /** - * Sets the value of the 'fixed' field. - * @param value the value to set. - */ - public void setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { - this.fixed = value; - } - - /** - * Gets the value of the 'date' field. - * @return The value of the 'date' field. - */ - public java.time.LocalDate getDate() { - return date; - } - - - /** - * Sets the value of the 'date' field. - * @param value the value to set. - */ - public void setDate(java.time.LocalDate value) { - this.date = value; - } - - /** - * Gets the value of the 'timestampMillis' field. - * @return The value of the 'timestampMillis' field. - */ - public java.time.Instant getTimestampMillis() { - return timestampMillis; - } - - - /** - * Sets the value of the 'timestampMillis' field. - * @param value the value to set. - */ - public void setTimestampMillis(java.time.Instant value) { - this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); - } - - /** - * Gets the value of the 'TestEnum' field. - * @return The value of the 'TestEnum' field. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { - return TestEnum; - } - - - /** - * Sets the value of the 'TestEnum' field. - * @param value the value to set. - */ - public void setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { - this.TestEnum = value; - } - - /** - * Gets the value of the 'row' field. - * @return The value of the 'row' field. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { - return row; - } - - - /** - * Sets the value of the 'row' field. - * @param value the value to set. - */ - public void setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { - this.row = value; - } - - /** - * Gets the value of the 'array' field. - * @return The value of the 'array' field. - */ - public java.util.List getArray() { - return array; - } - - - /** - * Sets the value of the 'array' field. - * @param value the value to set. - */ - public void setArray(java.util.List value) { - this.array = value; - } - - /** - * Gets the value of the 'map' field. - * @return The value of the 'map' field. - */ - public java.util.Map getMap() { - return map; - } - - - /** - * Sets the value of the 'map' field. - * @param value the value to set. - */ - public void setMap(java.util.Map value) { - this.map = value; - } - - /** - * Creates a new TestAvro RecordBuilder. - * @return A new TestAvro RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder() { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); - } - - /** - * Creates a new TestAvro RecordBuilder by copying an existing Builder. - * @param other The existing builder to copy. - * @return A new TestAvro RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); - } - } - - /** - * Creates a new TestAvro RecordBuilder by copying an existing TestAvro instance. - * @param other The existing instance to copy. - * @return A new TestAvro RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); - } - } - - /** - * RecordBuilder for TestAvro instances. - */ - @org.apache.avro.specific.AvroGenerated - public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase - implements org.apache.avro.data.RecordBuilder { - - private boolean bool_non_nullable; - private java.lang.Integer int$; - private java.lang.Long long$; - private java.lang.Float float$; - private java.lang.Double double$; - private java.lang.CharSequence string; - private java.nio.ByteBuffer bytes; - private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; - private java.time.LocalDate date; - private java.time.Instant timestampMillis; - private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; - private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder rowBuilder; - private java.util.List array; - private java.util.Map map; - - /** Creates a new Builder */ - private Builder() { - super(SCHEMA$); - } - - /** - * Creates a Builder by copying an existing Builder. - * @param other The existing Builder to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { - super(other); - if (isValidValue(fields()[0], other.bool_non_nullable)) { - this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); - fieldSetFlags()[0] = other.fieldSetFlags()[0]; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = other.fieldSetFlags()[1]; - } - if (isValidValue(fields()[2], other.long$)) { - this.long$ = data().deepCopy(fields()[2].schema(), other.long$); - fieldSetFlags()[2] = other.fieldSetFlags()[2]; - } - if (isValidValue(fields()[3], other.float$)) { - this.float$ = data().deepCopy(fields()[3].schema(), other.float$); - fieldSetFlags()[3] = other.fieldSetFlags()[3]; - } - if (isValidValue(fields()[4], other.double$)) { - this.double$ = data().deepCopy(fields()[4].schema(), other.double$); - fieldSetFlags()[4] = other.fieldSetFlags()[4]; - } - if (isValidValue(fields()[5], other.string)) { - this.string = data().deepCopy(fields()[5].schema(), other.string); - fieldSetFlags()[5] = other.fieldSetFlags()[5]; - } - if (isValidValue(fields()[6], other.bytes)) { - this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); - fieldSetFlags()[6] = other.fieldSetFlags()[6]; - } - if (isValidValue(fields()[7], other.fixed)) { - this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); - fieldSetFlags()[7] = other.fieldSetFlags()[7]; - } - if (isValidValue(fields()[8], other.date)) { - this.date = data().deepCopy(fields()[8].schema(), other.date); - fieldSetFlags()[8] = other.fieldSetFlags()[8]; - } - if (isValidValue(fields()[9], other.timestampMillis)) { - this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); - fieldSetFlags()[9] = other.fieldSetFlags()[9]; - } - if (isValidValue(fields()[10], other.TestEnum)) { - this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); - fieldSetFlags()[10] = other.fieldSetFlags()[10]; - } - if (isValidValue(fields()[11], other.row)) { - this.row = data().deepCopy(fields()[11].schema(), other.row); - fieldSetFlags()[11] = other.fieldSetFlags()[11]; - } - if (other.hasRowBuilder()) { - this.rowBuilder = org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(other.getRowBuilder()); - } - if (isValidValue(fields()[12], other.array)) { - this.array = data().deepCopy(fields()[12].schema(), other.array); - fieldSetFlags()[12] = other.fieldSetFlags()[12]; - } - if (isValidValue(fields()[13], other.map)) { - this.map = data().deepCopy(fields()[13].schema(), other.map); - fieldSetFlags()[13] = other.fieldSetFlags()[13]; - } - } - - /** - * Creates a Builder by copying an existing TestAvro instance - * @param other The existing instance to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { - super(SCHEMA$); - if (isValidValue(fields()[0], other.bool_non_nullable)) { - this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); - fieldSetFlags()[0] = true; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = true; - } - if (isValidValue(fields()[2], other.long$)) { - this.long$ = data().deepCopy(fields()[2].schema(), other.long$); - fieldSetFlags()[2] = true; - } - if (isValidValue(fields()[3], other.float$)) { - this.float$ = data().deepCopy(fields()[3].schema(), other.float$); - fieldSetFlags()[3] = true; - } - if (isValidValue(fields()[4], other.double$)) { - this.double$ = data().deepCopy(fields()[4].schema(), other.double$); - fieldSetFlags()[4] = true; - } - if (isValidValue(fields()[5], other.string)) { - this.string = data().deepCopy(fields()[5].schema(), other.string); - fieldSetFlags()[5] = true; - } - if (isValidValue(fields()[6], other.bytes)) { - this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); - fieldSetFlags()[6] = true; - } - if (isValidValue(fields()[7], other.fixed)) { - this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); - fieldSetFlags()[7] = true; - } - if (isValidValue(fields()[8], other.date)) { - this.date = data().deepCopy(fields()[8].schema(), other.date); - fieldSetFlags()[8] = true; - } - if (isValidValue(fields()[9], other.timestampMillis)) { - this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); - fieldSetFlags()[9] = true; - } - if (isValidValue(fields()[10], other.TestEnum)) { - this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); - fieldSetFlags()[10] = true; - } - if (isValidValue(fields()[11], other.row)) { - this.row = data().deepCopy(fields()[11].schema(), other.row); - fieldSetFlags()[11] = true; - } - this.rowBuilder = null; - if (isValidValue(fields()[12], other.array)) { - this.array = data().deepCopy(fields()[12].schema(), other.array); - fieldSetFlags()[12] = true; - } - if (isValidValue(fields()[13], other.map)) { - this.map = data().deepCopy(fields()[13].schema(), other.map); - fieldSetFlags()[13] = true; - } - } - - /** - * Gets the value of the 'bool_non_nullable' field. - * @return The value. - */ - public boolean getBoolNonNullable() { - return bool_non_nullable; - } - - - /** - * Sets the value of the 'bool_non_nullable' field. - * @param value The value of 'bool_non_nullable'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBoolNonNullable(boolean value) { - validate(fields()[0], value); - this.bool_non_nullable = value; - fieldSetFlags()[0] = true; - return this; - } - - /** - * Checks whether the 'bool_non_nullable' field has been set. - * @return True if the 'bool_non_nullable' field has been set, false otherwise. - */ - public boolean hasBoolNonNullable() { - return fieldSetFlags()[0]; - } - - - /** - * Clears the value of the 'bool_non_nullable' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNonNullable() { - fieldSetFlags()[0] = false; - return this; - } - - /** - * Gets the value of the 'int$' field. - * @return The value. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value The value of 'int$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setInt$(java.lang.Integer value) { - validate(fields()[1], value); - this.int$ = value; - fieldSetFlags()[1] = true; - return this; - } - - /** - * Checks whether the 'int$' field has been set. - * @return True if the 'int$' field has been set, false otherwise. - */ - public boolean hasInt$() { - return fieldSetFlags()[1]; - } - - - /** - * Clears the value of the 'int$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearInt$() { - int$ = null; - fieldSetFlags()[1] = false; - return this; - } - - /** - * Gets the value of the 'long$' field. - * @return The value. - */ - public java.lang.Long getLong$() { - return long$; - } - - - /** - * Sets the value of the 'long$' field. - * @param value The value of 'long$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setLong$(java.lang.Long value) { - validate(fields()[2], value); - this.long$ = value; - fieldSetFlags()[2] = true; - return this; - } - - /** - * Checks whether the 'long$' field has been set. - * @return True if the 'long$' field has been set, false otherwise. - */ - public boolean hasLong$() { - return fieldSetFlags()[2]; - } - - - /** - * Clears the value of the 'long$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearLong$() { - long$ = null; - fieldSetFlags()[2] = false; - return this; - } - - /** - * Gets the value of the 'float$' field. - * @return The value. - */ - public java.lang.Float getFloat$() { - return float$; - } - - - /** - * Sets the value of the 'float$' field. - * @param value The value of 'float$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFloat$(java.lang.Float value) { - validate(fields()[3], value); - this.float$ = value; - fieldSetFlags()[3] = true; - return this; - } - - /** - * Checks whether the 'float$' field has been set. - * @return True if the 'float$' field has been set, false otherwise. - */ - public boolean hasFloat$() { - return fieldSetFlags()[3]; - } - - - /** - * Clears the value of the 'float$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFloat$() { - float$ = null; - fieldSetFlags()[3] = false; - return this; - } - - /** - * Gets the value of the 'double$' field. - * @return The value. - */ - public java.lang.Double getDouble$() { - return double$; - } - - - /** - * Sets the value of the 'double$' field. - * @param value The value of 'double$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDouble$(java.lang.Double value) { - validate(fields()[4], value); - this.double$ = value; - fieldSetFlags()[4] = true; - return this; - } - - /** - * Checks whether the 'double$' field has been set. - * @return True if the 'double$' field has been set, false otherwise. - */ - public boolean hasDouble$() { - return fieldSetFlags()[4]; - } - - - /** - * Clears the value of the 'double$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDouble$() { - double$ = null; - fieldSetFlags()[4] = false; - return this; - } - - /** - * Gets the value of the 'string' field. - * @return The value. - */ - public java.lang.CharSequence getString() { - return string; - } - - - /** - * Sets the value of the 'string' field. - * @param value The value of 'string'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setString(java.lang.CharSequence value) { - validate(fields()[5], value); - this.string = value; - fieldSetFlags()[5] = true; - return this; - } - - /** - * Checks whether the 'string' field has been set. - * @return True if the 'string' field has been set, false otherwise. - */ - public boolean hasString() { - return fieldSetFlags()[5]; - } - - - /** - * Clears the value of the 'string' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearString() { - string = null; - fieldSetFlags()[5] = false; - return this; - } - - /** - * Gets the value of the 'bytes' field. - * @return The value. - */ - public java.nio.ByteBuffer getBytes() { - return bytes; - } - - - /** - * Sets the value of the 'bytes' field. - * @param value The value of 'bytes'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBytes(java.nio.ByteBuffer value) { - validate(fields()[6], value); - this.bytes = value; - fieldSetFlags()[6] = true; - return this; - } - - /** - * Checks whether the 'bytes' field has been set. - * @return True if the 'bytes' field has been set, false otherwise. - */ - public boolean hasBytes() { - return fieldSetFlags()[6]; - } - - - /** - * Clears the value of the 'bytes' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBytes() { - bytes = null; - fieldSetFlags()[6] = false; - return this; - } - - /** - * Gets the value of the 'fixed' field. - * @return The value. - */ - public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { - return fixed; - } - - - /** - * Sets the value of the 'fixed' field. - * @param value The value of 'fixed'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { - validate(fields()[7], value); - this.fixed = value; - fieldSetFlags()[7] = true; - return this; - } - - /** - * Checks whether the 'fixed' field has been set. - * @return True if the 'fixed' field has been set, false otherwise. - */ - public boolean hasFixed() { - return fieldSetFlags()[7]; - } - - - /** - * Clears the value of the 'fixed' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFixed() { - fixed = null; - fieldSetFlags()[7] = false; - return this; - } - - /** - * Gets the value of the 'date' field. - * @return The value. - */ - public java.time.LocalDate getDate() { - return date; - } - - - /** - * Sets the value of the 'date' field. - * @param value The value of 'date'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDate(java.time.LocalDate value) { - validate(fields()[8], value); - this.date = value; - fieldSetFlags()[8] = true; - return this; - } - - /** - * Checks whether the 'date' field has been set. - * @return True if the 'date' field has been set, false otherwise. - */ - public boolean hasDate() { - return fieldSetFlags()[8]; - } - - - /** - * Clears the value of the 'date' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDate() { - fieldSetFlags()[8] = false; - return this; - } - - /** - * Gets the value of the 'timestampMillis' field. - * @return The value. - */ - public java.time.Instant getTimestampMillis() { - return timestampMillis; - } - - - /** - * Sets the value of the 'timestampMillis' field. - * @param value The value of 'timestampMillis'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTimestampMillis(java.time.Instant value) { - validate(fields()[9], value); - this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); - fieldSetFlags()[9] = true; - return this; - } - - /** - * Checks whether the 'timestampMillis' field has been set. - * @return True if the 'timestampMillis' field has been set, false otherwise. - */ - public boolean hasTimestampMillis() { - return fieldSetFlags()[9]; - } - - - /** - * Clears the value of the 'timestampMillis' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTimestampMillis() { - fieldSetFlags()[9] = false; - return this; - } - - /** - * Gets the value of the 'TestEnum' field. - * @return The value. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { - return TestEnum; - } - - - /** - * Sets the value of the 'TestEnum' field. - * @param value The value of 'TestEnum'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { - validate(fields()[10], value); - this.TestEnum = value; - fieldSetFlags()[10] = true; - return this; - } - - /** - * Checks whether the 'TestEnum' field has been set. - * @return True if the 'TestEnum' field has been set, false otherwise. - */ - public boolean hasTestEnum() { - return fieldSetFlags()[10]; - } - - - /** - * Clears the value of the 'TestEnum' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTestEnum() { - TestEnum = null; - fieldSetFlags()[10] = false; - return this; - } - - /** - * Gets the value of the 'row' field. - * @return The value. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { - return row; - } - - - /** - * Sets the value of the 'row' field. - * @param value The value of 'row'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { - validate(fields()[11], value); - this.rowBuilder = null; - this.row = value; - fieldSetFlags()[11] = true; - return this; - } - - /** - * Checks whether the 'row' field has been set. - * @return True if the 'row' field has been set, false otherwise. - */ - public boolean hasRow() { - return fieldSetFlags()[11]; - } - - /** - * Gets the Builder instance for the 'row' field and creates one if it doesn't exist yet. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder getRowBuilder() { - if (rowBuilder == null) { - if (hasRow()) { - setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(row)); - } else { - setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder()); - } - } - return rowBuilder; - } - - /** - * Sets the Builder instance for the 'row' field - * @param value The builder instance that must be set. - * @return This builder. - */ - - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder value) { - clearRow(); - rowBuilder = value; - return this; - } - - /** - * Checks whether the 'row' field has an active Builder instance - * @return True if the 'row' field has an active Builder instance - */ - public boolean hasRowBuilder() { - return rowBuilder != null; - } - - /** - * Clears the value of the 'row' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearRow() { - row = null; - rowBuilder = null; - fieldSetFlags()[11] = false; - return this; - } - - /** - * Gets the value of the 'array' field. - * @return The value. - */ - public java.util.List getArray() { - return array; - } - - - /** - * Sets the value of the 'array' field. - * @param value The value of 'array'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setArray(java.util.List value) { - validate(fields()[12], value); - this.array = value; - fieldSetFlags()[12] = true; - return this; - } - - /** - * Checks whether the 'array' field has been set. - * @return True if the 'array' field has been set, false otherwise. - */ - public boolean hasArray() { - return fieldSetFlags()[12]; - } - - - /** - * Clears the value of the 'array' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearArray() { - array = null; - fieldSetFlags()[12] = false; - return this; - } - - /** - * Gets the value of the 'map' field. - * @return The value. - */ - public java.util.Map getMap() { - return map; - } - - - /** - * Sets the value of the 'map' field. - * @param value The value of 'map'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setMap(java.util.Map value) { - validate(fields()[13], value); - this.map = value; - fieldSetFlags()[13] = true; - return this; - } - - /** - * Checks whether the 'map' field has been set. - * @return True if the 'map' field has been set, false otherwise. - */ - public boolean hasMap() { - return fieldSetFlags()[13]; - } - - - /** - * Clears the value of the 'map' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearMap() { - map = null; - fieldSetFlags()[13] = false; - return this; - } - - @Override - @SuppressWarnings("unchecked") - public TestAvro build() { - try { - TestAvro record = new TestAvro(); - record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (java.lang.Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); - record.long$ = fieldSetFlags()[2] ? this.long$ : (java.lang.Long) defaultValue(fields()[2]); - record.float$ = fieldSetFlags()[3] ? this.float$ : (java.lang.Float) defaultValue(fields()[3]); - record.double$ = fieldSetFlags()[4] ? this.double$ : (java.lang.Double) defaultValue(fields()[4]); - record.string = fieldSetFlags()[5] ? this.string : (java.lang.CharSequence) defaultValue(fields()[5]); - record.bytes = fieldSetFlags()[6] ? this.bytes : (java.nio.ByteBuffer) defaultValue(fields()[6]); - record.fixed = fieldSetFlags()[7] ? this.fixed : (org.apache.beam.sdk.extensions.avro.schemas.fixed4) defaultValue(fields()[7]); - record.date = fieldSetFlags()[8] ? this.date : (java.time.LocalDate) defaultValue(fields()[8]); - record.timestampMillis = fieldSetFlags()[9] ? this.timestampMillis : (java.time.Instant) defaultValue(fields()[9]); - record.TestEnum = fieldSetFlags()[10] ? this.TestEnum : (org.apache.beam.sdk.extensions.avro.schemas.TestEnum) defaultValue(fields()[10]); - if (rowBuilder != null) { - try { - record.row = this.rowBuilder.build(); - } catch (org.apache.avro.AvroMissingFieldException e) { - e.addParentField(record.getSchema().getField("row")); - throw e; - } - } else { - record.row = fieldSetFlags()[11] ? this.row : (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested) defaultValue(fields()[11]); - } - record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); - record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); - return record; - } catch (org.apache.avro.AvroMissingFieldException e) { - throw e; - } catch (java.lang.Exception e) { - throw new org.apache.avro.AvroRuntimeException(e); - } - } - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumWriter - WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, SpecificData.getEncoder(out)); - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumReader - READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, SpecificData.getDecoder(in)); - } - -} - - - - - - - - - - diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java deleted file mode 100644 index 5e952ca3d6fa..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java +++ /dev/null @@ -1,418 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; - -import org.apache.avro.generic.GenericArray; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.util.Utf8; -import org.apache.avro.message.BinaryMessageEncoder; -import org.apache.avro.message.BinaryMessageDecoder; -import org.apache.avro.message.SchemaStore; - -@org.apache.avro.specific.AvroGenerated -public class TestAvroNested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - private static final long serialVersionUID = 4633138088036298925L; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvroNested\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - private static SpecificData MODEL$ = new SpecificData(); - - private static final BinaryMessageEncoder ENCODER = - new BinaryMessageEncoder(MODEL$, SCHEMA$); - - private static final BinaryMessageDecoder DECODER = - new BinaryMessageDecoder(MODEL$, SCHEMA$); - - /** - * Return the BinaryMessageEncoder instance used by this class. - * @return the message encoder used by this class - */ - public static BinaryMessageEncoder getEncoder() { - return ENCODER; - } - - /** - * Return the BinaryMessageDecoder instance used by this class. - * @return the message decoder used by this class - */ - public static BinaryMessageDecoder getDecoder() { - return DECODER; - } - - /** - * 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 - */ - public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { - return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); - } - - /** - * Serializes this TestAvroNested to a ByteBuffer. - * @return a buffer holding the serialized data for this instance - * @throws java.io.IOException if this instance could not be serialized - */ - public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { - return ENCODER.encode(this); - } - - /** - * Deserializes a TestAvroNested from a ByteBuffer. - * @param b a byte buffer holding serialized data for an instance of this class - * @return a TestAvroNested instance decoded from the given buffer - * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class - */ - public static TestAvroNested fromByteBuffer( - java.nio.ByteBuffer b) throws java.io.IOException { - return DECODER.decode(b); - } - - private boolean BOOL_NON_NULLABLE; - private java.lang.Integer int$; - - /** - * Default constructor. Note that this does not initialize fields - * to their default values from the schema. If that is desired then - * one should use newBuilder(). - */ - public TestAvroNested() {} - - /** - * All-args constructor. - * @param BOOL_NON_NULLABLE The new value for BOOL_NON_NULLABLE - * @param int$ The new value for int - */ - public TestAvroNested(java.lang.Boolean BOOL_NON_NULLABLE, java.lang.Integer int$) { - this.BOOL_NON_NULLABLE = BOOL_NON_NULLABLE; - this.int$ = int$; - } - - public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - // Used by DatumWriter. Applications should not call. - public java.lang.Object get(int field$) { - switch (field$) { - case 0: return BOOL_NON_NULLABLE; - case 1: return int$; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - // Used by DatumReader. Applications should not call. - @SuppressWarnings(value="unchecked") - public void put(int field$, java.lang.Object value$) { - switch (field$) { - case 0: BOOL_NON_NULLABLE = (java.lang.Boolean)value$; break; - case 1: int$ = (java.lang.Integer)value$; break; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - /** - * Gets the value of the 'BOOL_NON_NULLABLE' field. - * @return The value of the 'BOOL_NON_NULLABLE' field. - */ - public boolean getBOOLNONNULLABLE() { - return BOOL_NON_NULLABLE; - } - - - /** - * Sets the value of the 'BOOL_NON_NULLABLE' field. - * @param value the value to set. - */ - public void setBOOLNONNULLABLE(boolean value) { - this.BOOL_NON_NULLABLE = value; - } - - /** - * Gets the value of the 'int$' field. - * @return The value of the 'int$' field. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value the value to set. - */ - public void setInt$(java.lang.Integer value) { - this.int$ = value; - } - - /** - * Creates a new TestAvroNested RecordBuilder. - * @return A new TestAvroNested RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder() { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); - } - - /** - * Creates a new TestAvroNested RecordBuilder by copying an existing Builder. - * @param other The existing builder to copy. - * @return A new TestAvroNested RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); - } - } - - /** - * Creates a new TestAvroNested RecordBuilder by copying an existing TestAvroNested instance. - * @param other The existing instance to copy. - * @return A new TestAvroNested RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); - } - } - - /** - * RecordBuilder for TestAvroNested instances. - */ - @org.apache.avro.specific.AvroGenerated - public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase - implements org.apache.avro.data.RecordBuilder { - - private boolean BOOL_NON_NULLABLE; - private java.lang.Integer int$; - - /** Creates a new Builder */ - private Builder() { - super(SCHEMA$); - } - - /** - * Creates a Builder by copying an existing Builder. - * @param other The existing Builder to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { - super(other); - if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { - this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); - fieldSetFlags()[0] = other.fieldSetFlags()[0]; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = other.fieldSetFlags()[1]; - } - } - - /** - * Creates a Builder by copying an existing TestAvroNested instance - * @param other The existing instance to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { - super(SCHEMA$); - if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { - this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); - fieldSetFlags()[0] = true; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = true; - } - } - - /** - * Gets the value of the 'BOOL_NON_NULLABLE' field. - * @return The value. - */ - public boolean getBOOLNONNULLABLE() { - return BOOL_NON_NULLABLE; - } - - - /** - * Sets the value of the 'BOOL_NON_NULLABLE' field. - * @param value The value of 'BOOL_NON_NULLABLE'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setBOOLNONNULLABLE(boolean value) { - validate(fields()[0], value); - this.BOOL_NON_NULLABLE = value; - fieldSetFlags()[0] = true; - return this; - } - - /** - * Checks whether the 'BOOL_NON_NULLABLE' field has been set. - * @return True if the 'BOOL_NON_NULLABLE' field has been set, false otherwise. - */ - public boolean hasBOOLNONNULLABLE() { - return fieldSetFlags()[0]; - } - - - /** - * Clears the value of the 'BOOL_NON_NULLABLE' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearBOOLNONNULLABLE() { - fieldSetFlags()[0] = false; - return this; - } - - /** - * Gets the value of the 'int$' field. - * @return The value. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value The value of 'int$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setInt$(java.lang.Integer value) { - validate(fields()[1], value); - this.int$ = value; - fieldSetFlags()[1] = true; - return this; - } - - /** - * Checks whether the 'int$' field has been set. - * @return True if the 'int$' field has been set, false otherwise. - */ - public boolean hasInt$() { - return fieldSetFlags()[1]; - } - - - /** - * Clears the value of the 'int$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearInt$() { - int$ = null; - fieldSetFlags()[1] = false; - return this; - } - - @Override - @SuppressWarnings("unchecked") - public TestAvroNested build() { - try { - TestAvroNested record = new TestAvroNested(); - record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (java.lang.Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); - return record; - } catch (org.apache.avro.AvroMissingFieldException e) { - throw e; - } catch (java.lang.Exception e) { - throw new org.apache.avro.AvroRuntimeException(e); - } - } - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumWriter - WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, SpecificData.getEncoder(out)); - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumReader - READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, SpecificData.getDecoder(in)); - } - - @Override protected boolean hasCustomCoders() { return true; } - - @Override public void customEncode(org.apache.avro.io.Encoder out) - throws java.io.IOException - { - out.writeBoolean(this.BOOL_NON_NULLABLE); - - if (this.int$ == null) { - out.writeIndex(1); - out.writeNull(); - } else { - out.writeIndex(0); - out.writeInt(this.int$); - } - - } - - @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) - throws java.io.IOException - { - org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); - if (fieldOrder == null) { - this.BOOL_NON_NULLABLE = in.readBoolean(); - - if (in.readIndex() != 0) { - in.readNull(); - this.int$ = null; - } else { - this.int$ = in.readInt(); - } - - } else { - for (int i = 0; i < 2; i++) { - switch (fieldOrder[i].pos()) { - case 0: - this.BOOL_NON_NULLABLE = in.readBoolean(); - break; - - case 1: - if (in.readIndex() != 0) { - in.readNull(); - this.int$ = null; - } else { - this.int$ = in.readInt(); - } - break; - - default: - throw new java.io.IOException("Corrupt ResolvingDecoder."); - } - } - } - } -} - - - - - - - - - - diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java deleted file mode 100644 index 8d1e1d7dffe9..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; -@org.apache.avro.specific.AvroGenerated -public enum TestEnum implements org.apache.avro.generic.GenericEnumSymbol { - abc, cde ; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"enum\",\"name\":\"TestEnum\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"symbols\":[\"abc\",\"cde\"]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } -} diff --git a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java b/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java deleted file mode 100644 index 24c5be9ca18d..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1102/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; -@org.apache.avro.specific.FixedSize(4) -@org.apache.avro.specific.AvroGenerated -public class fixed4 extends org.apache.avro.specific.SpecificFixed { - private static final long serialVersionUID = -5646354132642432749L; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"fixed\",\"name\":\"fixed4\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"size\":4}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - - /** Creates a new fixed4 */ - public fixed4() { - super(); - } - - /** - * Creates a new fixed4 with the given bytes. - * @param bytes The bytes to create the new fixed4. - */ - public fixed4(byte[] bytes) { - super(bytes); - } - - private static final org.apache.avro.io.DatumWriter - WRITER$ = new org.apache.avro.specific.SpecificDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, org.apache.avro.specific.SpecificData.getEncoder(out)); - } - - private static final org.apache.avro.io.DatumReader - READER$ = new org.apache.avro.specific.SpecificDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, org.apache.avro.specific.SpecificData.getDecoder(in)); - } - -} diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java deleted file mode 100644 index 4badde3f7b01..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java +++ /dev/null @@ -1,523 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.io; - -import org.apache.avro.generic.GenericArray; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.util.Utf8; -import org.apache.avro.message.BinaryMessageEncoder; -import org.apache.avro.message.BinaryMessageDecoder; -import org.apache.avro.message.SchemaStore; - -@org.apache.avro.specific.AvroGenerated -public class AvroGeneratedUser extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - private static final long serialVersionUID = 3056733241446362297L; - - - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroGeneratedUser\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.io\",\"fields\":[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"favorite_number\",\"type\":[\"int\",\"null\"]},{\"name\":\"favorite_color\",\"type\":[\"string\",\"null\"]}]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - private static final SpecificData MODEL$ = new SpecificData(); - - private static final BinaryMessageEncoder ENCODER = - new BinaryMessageEncoder<>(MODEL$, SCHEMA$); - - private static final BinaryMessageDecoder DECODER = - new BinaryMessageDecoder<>(MODEL$, SCHEMA$); - - /** - * Return the BinaryMessageEncoder instance used by this class. - * @return the message encoder used by this class - */ - public static BinaryMessageEncoder getEncoder() { - return ENCODER; - } - - /** - * Return the BinaryMessageDecoder instance used by this class. - * @return the message decoder used by this class - */ - public static BinaryMessageDecoder getDecoder() { - return DECODER; - } - - /** - * 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 - */ - public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { - return new BinaryMessageDecoder<>(MODEL$, SCHEMA$, resolver); - } - - /** - * Serializes this AvroGeneratedUser to a ByteBuffer. - * @return a buffer holding the serialized data for this instance - * @throws java.io.IOException if this instance could not be serialized - */ - public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { - return ENCODER.encode(this); - } - - /** - * Deserializes a AvroGeneratedUser from a ByteBuffer. - * @param b a byte buffer holding serialized data for an instance of this class - * @return a AvroGeneratedUser instance decoded from the given buffer - * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class - */ - public static AvroGeneratedUser fromByteBuffer( - java.nio.ByteBuffer b) throws java.io.IOException { - return DECODER.decode(b); - } - - private java.lang.CharSequence name; - private java.lang.Integer favorite_number; - private java.lang.CharSequence favorite_color; - - /** - * Default constructor. Note that this does not initialize fields - * to their default values from the schema. If that is desired then - * one should use newBuilder(). - */ - public AvroGeneratedUser() {} - - /** - * All-args constructor. - * @param name The new value for name - * @param favorite_number The new value for favorite_number - * @param favorite_color The new value for favorite_color - */ - public AvroGeneratedUser(java.lang.CharSequence name, java.lang.Integer favorite_number, java.lang.CharSequence favorite_color) { - this.name = name; - this.favorite_number = favorite_number; - this.favorite_color = favorite_color; - } - - @Override - public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } - - @Override - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - - // Used by DatumWriter. Applications should not call. - @Override - public java.lang.Object get(int field$) { - switch (field$) { - case 0: return name; - case 1: return favorite_number; - case 2: return favorite_color; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - // Used by DatumReader. Applications should not call. - @Override - @SuppressWarnings(value="unchecked") - public void put(int field$, java.lang.Object value$) { - switch (field$) { - case 0: name = (java.lang.CharSequence)value$; break; - case 1: favorite_number = (java.lang.Integer)value$; break; - case 2: favorite_color = (java.lang.CharSequence)value$; break; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - /** - * Gets the value of the 'name' field. - * @return The value of the 'name' field. - */ - public java.lang.CharSequence getName() { - return name; - } - - - /** - * Sets the value of the 'name' field. - * @param value the value to set. - */ - public void setName(java.lang.CharSequence value) { - this.name = value; - } - - /** - * Gets the value of the 'favorite_number' field. - * @return The value of the 'favorite_number' field. - */ - public java.lang.Integer getFavoriteNumber() { - return favorite_number; - } - - - /** - * Sets the value of the 'favorite_number' field. - * @param value the value to set. - */ - public void setFavoriteNumber(java.lang.Integer value) { - this.favorite_number = value; - } - - /** - * Gets the value of the 'favorite_color' field. - * @return The value of the 'favorite_color' field. - */ - public java.lang.CharSequence getFavoriteColor() { - return favorite_color; - } - - - /** - * Sets the value of the 'favorite_color' field. - * @param value the value to set. - */ - public void setFavoriteColor(java.lang.CharSequence value) { - this.favorite_color = value; - } - - /** - * Creates a new AvroGeneratedUser RecordBuilder. - * @return A new AvroGeneratedUser RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder() { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); - } - - /** - * Creates a new AvroGeneratedUser RecordBuilder by copying an existing Builder. - * @param other The existing builder to copy. - * @return A new AvroGeneratedUser RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); - } - } - - /** - * Creates a new AvroGeneratedUser RecordBuilder by copying an existing AvroGeneratedUser instance. - * @param other The existing instance to copy. - * @return A new AvroGeneratedUser RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); - } - } - - /** - * RecordBuilder for AvroGeneratedUser instances. - */ - @org.apache.avro.specific.AvroGenerated - public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase - implements org.apache.avro.data.RecordBuilder { - - private java.lang.CharSequence name; - private java.lang.Integer favorite_number; - private java.lang.CharSequence favorite_color; - - /** Creates a new Builder */ - private Builder() { - super(SCHEMA$, MODEL$); - } - - /** - * Creates a Builder by copying an existing Builder. - * @param other The existing Builder to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { - super(other); - if (isValidValue(fields()[0], other.name)) { - this.name = data().deepCopy(fields()[0].schema(), other.name); - fieldSetFlags()[0] = other.fieldSetFlags()[0]; - } - if (isValidValue(fields()[1], other.favorite_number)) { - this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); - fieldSetFlags()[1] = other.fieldSetFlags()[1]; - } - if (isValidValue(fields()[2], other.favorite_color)) { - this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); - fieldSetFlags()[2] = other.fieldSetFlags()[2]; - } - } - - /** - * Creates a Builder by copying an existing AvroGeneratedUser instance - * @param other The existing instance to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { - super(SCHEMA$, MODEL$); - if (isValidValue(fields()[0], other.name)) { - this.name = data().deepCopy(fields()[0].schema(), other.name); - fieldSetFlags()[0] = true; - } - if (isValidValue(fields()[1], other.favorite_number)) { - this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); - fieldSetFlags()[1] = true; - } - if (isValidValue(fields()[2], other.favorite_color)) { - this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); - fieldSetFlags()[2] = true; - } - } - - /** - * Gets the value of the 'name' field. - * @return The value. - */ - public java.lang.CharSequence getName() { - return name; - } - - - /** - * Sets the value of the 'name' field. - * @param value The value of 'name'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setName(java.lang.CharSequence value) { - validate(fields()[0], value); - this.name = value; - fieldSetFlags()[0] = true; - return this; - } - - /** - * Checks whether the 'name' field has been set. - * @return True if the 'name' field has been set, false otherwise. - */ - public boolean hasName() { - return fieldSetFlags()[0]; - } - - - /** - * Clears the value of the 'name' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearName() { - name = null; - fieldSetFlags()[0] = false; - return this; - } - - /** - * Gets the value of the 'favorite_number' field. - * @return The value. - */ - public java.lang.Integer getFavoriteNumber() { - return favorite_number; - } - - - /** - * Sets the value of the 'favorite_number' field. - * @param value The value of 'favorite_number'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteNumber(java.lang.Integer value) { - validate(fields()[1], value); - this.favorite_number = value; - fieldSetFlags()[1] = true; - return this; - } - - /** - * Checks whether the 'favorite_number' field has been set. - * @return True if the 'favorite_number' field has been set, false otherwise. - */ - public boolean hasFavoriteNumber() { - return fieldSetFlags()[1]; - } - - - /** - * Clears the value of the 'favorite_number' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteNumber() { - favorite_number = null; - fieldSetFlags()[1] = false; - return this; - } - - /** - * Gets the value of the 'favorite_color' field. - * @return The value. - */ - public java.lang.CharSequence getFavoriteColor() { - return favorite_color; - } - - - /** - * Sets the value of the 'favorite_color' field. - * @param value The value of 'favorite_color'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteColor(java.lang.CharSequence value) { - validate(fields()[2], value); - this.favorite_color = value; - fieldSetFlags()[2] = true; - return this; - } - - /** - * Checks whether the 'favorite_color' field has been set. - * @return True if the 'favorite_color' field has been set, false otherwise. - */ - public boolean hasFavoriteColor() { - return fieldSetFlags()[2]; - } - - - /** - * Clears the value of the 'favorite_color' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteColor() { - favorite_color = null; - fieldSetFlags()[2] = false; - return this; - } - - @Override - @SuppressWarnings("unchecked") - public AvroGeneratedUser build() { - try { - AvroGeneratedUser record = new AvroGeneratedUser(); - record.name = fieldSetFlags()[0] ? this.name : (java.lang.CharSequence) defaultValue(fields()[0]); - record.favorite_number = fieldSetFlags()[1] ? this.favorite_number : (java.lang.Integer) defaultValue(fields()[1]); - record.favorite_color = fieldSetFlags()[2] ? this.favorite_color : (java.lang.CharSequence) defaultValue(fields()[2]); - return record; - } catch (org.apache.avro.AvroMissingFieldException e) { - throw e; - } catch (java.lang.Exception e) { - throw new org.apache.avro.AvroRuntimeException(e); - } - } - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumWriter - WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, SpecificData.getEncoder(out)); - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumReader - READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, SpecificData.getDecoder(in)); - } - - @Override protected boolean hasCustomCoders() { return true; } - - @Override public void customEncode(org.apache.avro.io.Encoder out) - throws java.io.IOException - { - out.writeString(this.name); - - if (this.favorite_number == null) { - out.writeIndex(1); - out.writeNull(); - } else { - out.writeIndex(0); - out.writeInt(this.favorite_number); - } - - if (this.favorite_color == null) { - out.writeIndex(1); - out.writeNull(); - } else { - out.writeIndex(0); - out.writeString(this.favorite_color); - } - - } - - @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) - throws java.io.IOException - { - org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); - if (fieldOrder == null) { - this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); - - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_number = null; - } else { - this.favorite_number = in.readInt(); - } - - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_color = null; - } else { - this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); - } - - } else { - for (int i = 0; i < 3; i++) { - switch (fieldOrder[i].pos()) { - case 0: - this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); - break; - - case 1: - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_number = null; - } else { - this.favorite_number = in.readInt(); - } - break; - - case 2: - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_color = null; - } else { - this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); - } - break; - - default: - throw new java.io.IOException("Corrupt ResolvingDecoder."); - } - } - } - } -} - - - - - - - - - - diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java deleted file mode 100644 index f21d86c99bc1..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java +++ /dev/null @@ -1,1313 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; - -import org.apache.avro.generic.GenericArray; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.util.Utf8; -import org.apache.avro.message.BinaryMessageEncoder; -import org.apache.avro.message.BinaryMessageDecoder; -import org.apache.avro.message.SchemaStore; - -@org.apache.avro.specific.AvroGenerated -public class TestAvro extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - private static final long serialVersionUID = 27902431178981259L; - - - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvro\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"bool_non_nullable\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]},{\"name\":\"long\",\"type\":[\"long\",\"null\"]},{\"name\":\"float\",\"type\":[\"float\",\"null\"]},{\"name\":\"double\",\"type\":[\"double\",\"null\"]},{\"name\":\"string\",\"type\":[\"string\",\"null\"]},{\"name\":\"bytes\",\"type\":[\"bytes\",\"null\"]},{\"name\":\"fixed\",\"type\":{\"type\":\"fixed\",\"name\":\"fixed4\",\"size\":4}},{\"name\":\"date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},{\"name\":\"timestampMillis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}},{\"name\":\"TestEnum\",\"type\":{\"type\":\"enum\",\"name\":\"TestEnum\",\"symbols\":[\"abc\",\"cde\"]}},{\"name\":\"row\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"TestAvroNested\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}]},{\"name\":\"array\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\",\"TestAvroNested\"]}]},{\"name\":\"map\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"TestAvroNested\"]}]}]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - private static final SpecificData MODEL$ = new SpecificData(); - static { - MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.DateConversion()); - MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.TimestampMillisConversion()); - } - - private static final BinaryMessageEncoder ENCODER = - new BinaryMessageEncoder<>(MODEL$, SCHEMA$); - - private static final BinaryMessageDecoder DECODER = - new BinaryMessageDecoder<>(MODEL$, SCHEMA$); - - /** - * Return the BinaryMessageEncoder instance used by this class. - * @return the message encoder used by this class - */ - public static BinaryMessageEncoder getEncoder() { - return ENCODER; - } - - /** - * Return the BinaryMessageDecoder instance used by this class. - * @return the message decoder used by this class - */ - public static BinaryMessageDecoder getDecoder() { - return DECODER; - } - - /** - * 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 - */ - public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { - return new BinaryMessageDecoder<>(MODEL$, SCHEMA$, resolver); - } - - /** - * Serializes this TestAvro to a ByteBuffer. - * @return a buffer holding the serialized data for this instance - * @throws java.io.IOException if this instance could not be serialized - */ - public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { - return ENCODER.encode(this); - } - - /** - * Deserializes a TestAvro from a ByteBuffer. - * @param b a byte buffer holding serialized data for an instance of this class - * @return a TestAvro instance decoded from the given buffer - * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class - */ - public static TestAvro fromByteBuffer( - java.nio.ByteBuffer b) throws java.io.IOException { - return DECODER.decode(b); - } - - private boolean bool_non_nullable; - private java.lang.Integer int$; - private java.lang.Long long$; - private java.lang.Float float$; - private java.lang.Double double$; - private java.lang.CharSequence string; - private java.nio.ByteBuffer bytes; - private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; - private java.time.LocalDate date; - private java.time.Instant timestampMillis; - private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; - private java.util.List array; - private java.util.Map map; - - /** - * Default constructor. Note that this does not initialize fields - * to their default values from the schema. If that is desired then - * one should use newBuilder(). - */ - public TestAvro() {} - - /** - * All-args constructor. - * @param bool_non_nullable The new value for bool_non_nullable - * @param int$ The new value for int - * @param long$ The new value for long - * @param float$ The new value for float - * @param double$ The new value for double - * @param string The new value for string - * @param bytes The new value for bytes - * @param fixed The new value for fixed - * @param date The new value for date - * @param timestampMillis The new value for timestampMillis - * @param TestEnum The new value for TestEnum - * @param row The new value for row - * @param array The new value for array - * @param map The new value for map - */ - public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.CharSequence string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { - this.bool_non_nullable = bool_non_nullable; - this.int$ = int$; - this.long$ = long$; - this.float$ = float$; - this.double$ = double$; - this.string = string; - this.bytes = bytes; - this.fixed = fixed; - this.date = date; - this.timestampMillis = timestampMillis.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); - this.TestEnum = TestEnum; - this.row = row; - this.array = array; - this.map = map; - } - - @Override - public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } - - @Override - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - - // Used by DatumWriter. Applications should not call. - @Override - public java.lang.Object get(int field$) { - switch (field$) { - case 0: return bool_non_nullable; - case 1: return int$; - case 2: return long$; - case 3: return float$; - case 4: return double$; - case 5: return string; - case 6: return bytes; - case 7: return fixed; - case 8: return date; - case 9: return timestampMillis; - case 10: return TestEnum; - case 11: return row; - case 12: return array; - case 13: return map; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - private static final org.apache.avro.Conversion[] conversions = - new org.apache.avro.Conversion[] { - null, - null, - null, - null, - null, - null, - null, - null, - new org.apache.avro.data.TimeConversions.DateConversion(), - new org.apache.avro.data.TimeConversions.TimestampMillisConversion(), - null, - null, - null, - null, - null - }; - - @Override - public org.apache.avro.Conversion getConversion(int field) { - return conversions[field]; - } - - // Used by DatumReader. Applications should not call. - @Override - @SuppressWarnings(value="unchecked") - public void put(int field$, java.lang.Object value$) { - switch (field$) { - case 0: bool_non_nullable = (java.lang.Boolean)value$; break; - case 1: int$ = (java.lang.Integer)value$; break; - case 2: long$ = (java.lang.Long)value$; break; - case 3: float$ = (java.lang.Float)value$; break; - case 4: double$ = (java.lang.Double)value$; break; - case 5: string = (java.lang.CharSequence)value$; break; - case 6: bytes = (java.nio.ByteBuffer)value$; break; - case 7: fixed = (org.apache.beam.sdk.extensions.avro.schemas.fixed4)value$; break; - case 8: date = (java.time.LocalDate)value$; break; - case 9: timestampMillis = (java.time.Instant)value$; break; - case 10: TestEnum = (org.apache.beam.sdk.extensions.avro.schemas.TestEnum)value$; break; - case 11: row = (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested)value$; break; - case 12: array = (java.util.List)value$; break; - case 13: map = (java.util.Map)value$; break; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - /** - * Gets the value of the 'bool_non_nullable' field. - * @return The value of the 'bool_non_nullable' field. - */ - public boolean getBoolNonNullable() { - return bool_non_nullable; - } - - - /** - * Sets the value of the 'bool_non_nullable' field. - * @param value the value to set. - */ - public void setBoolNonNullable(boolean value) { - this.bool_non_nullable = value; - } - - /** - * Gets the value of the 'int$' field. - * @return The value of the 'int$' field. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value the value to set. - */ - public void setInt$(java.lang.Integer value) { - this.int$ = value; - } - - /** - * Gets the value of the 'long$' field. - * @return The value of the 'long$' field. - */ - public java.lang.Long getLong$() { - return long$; - } - - - /** - * Sets the value of the 'long$' field. - * @param value the value to set. - */ - public void setLong$(java.lang.Long value) { - this.long$ = value; - } - - /** - * Gets the value of the 'float$' field. - * @return The value of the 'float$' field. - */ - public java.lang.Float getFloat$() { - return float$; - } - - - /** - * Sets the value of the 'float$' field. - * @param value the value to set. - */ - public void setFloat$(java.lang.Float value) { - this.float$ = value; - } - - /** - * Gets the value of the 'double$' field. - * @return The value of the 'double$' field. - */ - public java.lang.Double getDouble$() { - return double$; - } - - - /** - * Sets the value of the 'double$' field. - * @param value the value to set. - */ - public void setDouble$(java.lang.Double value) { - this.double$ = value; - } - - /** - * Gets the value of the 'string' field. - * @return The value of the 'string' field. - */ - public java.lang.CharSequence getString() { - return string; - } - - - /** - * Sets the value of the 'string' field. - * @param value the value to set. - */ - public void setString(java.lang.CharSequence value) { - this.string = value; - } - - /** - * Gets the value of the 'bytes' field. - * @return The value of the 'bytes' field. - */ - public java.nio.ByteBuffer getBytes() { - return bytes; - } - - - /** - * Sets the value of the 'bytes' field. - * @param value the value to set. - */ - public void setBytes(java.nio.ByteBuffer value) { - this.bytes = value; - } - - /** - * Gets the value of the 'fixed' field. - * @return The value of the 'fixed' field. - */ - public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { - return fixed; - } - - - /** - * Sets the value of the 'fixed' field. - * @param value the value to set. - */ - public void setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { - this.fixed = value; - } - - /** - * Gets the value of the 'date' field. - * @return The value of the 'date' field. - */ - public java.time.LocalDate getDate() { - return date; - } - - - /** - * Sets the value of the 'date' field. - * @param value the value to set. - */ - public void setDate(java.time.LocalDate value) { - this.date = value; - } - - /** - * Gets the value of the 'timestampMillis' field. - * @return The value of the 'timestampMillis' field. - */ - public java.time.Instant getTimestampMillis() { - return timestampMillis; - } - - - /** - * Sets the value of the 'timestampMillis' field. - * @param value the value to set. - */ - public void setTimestampMillis(java.time.Instant value) { - this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); - } - - /** - * Gets the value of the 'TestEnum' field. - * @return The value of the 'TestEnum' field. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { - return TestEnum; - } - - - /** - * Sets the value of the 'TestEnum' field. - * @param value the value to set. - */ - public void setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { - this.TestEnum = value; - } - - /** - * Gets the value of the 'row' field. - * @return The value of the 'row' field. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { - return row; - } - - - /** - * Sets the value of the 'row' field. - * @param value the value to set. - */ - public void setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { - this.row = value; - } - - /** - * Gets the value of the 'array' field. - * @return The value of the 'array' field. - */ - public java.util.List getArray() { - return array; - } - - - /** - * Sets the value of the 'array' field. - * @param value the value to set. - */ - public void setArray(java.util.List value) { - this.array = value; - } - - /** - * Gets the value of the 'map' field. - * @return The value of the 'map' field. - */ - public java.util.Map getMap() { - return map; - } - - - /** - * Sets the value of the 'map' field. - * @param value the value to set. - */ - public void setMap(java.util.Map value) { - this.map = value; - } - - /** - * Creates a new TestAvro RecordBuilder. - * @return A new TestAvro RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder() { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); - } - - /** - * Creates a new TestAvro RecordBuilder by copying an existing Builder. - * @param other The existing builder to copy. - * @return A new TestAvro RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); - } - } - - /** - * Creates a new TestAvro RecordBuilder by copying an existing TestAvro instance. - * @param other The existing instance to copy. - * @return A new TestAvro RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); - } - } - - /** - * RecordBuilder for TestAvro instances. - */ - @org.apache.avro.specific.AvroGenerated - public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase - implements org.apache.avro.data.RecordBuilder { - - private boolean bool_non_nullable; - private java.lang.Integer int$; - private java.lang.Long long$; - private java.lang.Float float$; - private java.lang.Double double$; - private java.lang.CharSequence string; - private java.nio.ByteBuffer bytes; - private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; - private java.time.LocalDate date; - private java.time.Instant timestampMillis; - private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; - private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder rowBuilder; - private java.util.List array; - private java.util.Map map; - - /** Creates a new Builder */ - private Builder() { - super(SCHEMA$, MODEL$); - } - - /** - * Creates a Builder by copying an existing Builder. - * @param other The existing Builder to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { - super(other); - if (isValidValue(fields()[0], other.bool_non_nullable)) { - this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); - fieldSetFlags()[0] = other.fieldSetFlags()[0]; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = other.fieldSetFlags()[1]; - } - if (isValidValue(fields()[2], other.long$)) { - this.long$ = data().deepCopy(fields()[2].schema(), other.long$); - fieldSetFlags()[2] = other.fieldSetFlags()[2]; - } - if (isValidValue(fields()[3], other.float$)) { - this.float$ = data().deepCopy(fields()[3].schema(), other.float$); - fieldSetFlags()[3] = other.fieldSetFlags()[3]; - } - if (isValidValue(fields()[4], other.double$)) { - this.double$ = data().deepCopy(fields()[4].schema(), other.double$); - fieldSetFlags()[4] = other.fieldSetFlags()[4]; - } - if (isValidValue(fields()[5], other.string)) { - this.string = data().deepCopy(fields()[5].schema(), other.string); - fieldSetFlags()[5] = other.fieldSetFlags()[5]; - } - if (isValidValue(fields()[6], other.bytes)) { - this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); - fieldSetFlags()[6] = other.fieldSetFlags()[6]; - } - if (isValidValue(fields()[7], other.fixed)) { - this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); - fieldSetFlags()[7] = other.fieldSetFlags()[7]; - } - if (isValidValue(fields()[8], other.date)) { - this.date = data().deepCopy(fields()[8].schema(), other.date); - fieldSetFlags()[8] = other.fieldSetFlags()[8]; - } - if (isValidValue(fields()[9], other.timestampMillis)) { - this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); - fieldSetFlags()[9] = other.fieldSetFlags()[9]; - } - if (isValidValue(fields()[10], other.TestEnum)) { - this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); - fieldSetFlags()[10] = other.fieldSetFlags()[10]; - } - if (isValidValue(fields()[11], other.row)) { - this.row = data().deepCopy(fields()[11].schema(), other.row); - fieldSetFlags()[11] = other.fieldSetFlags()[11]; - } - if (other.hasRowBuilder()) { - this.rowBuilder = org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(other.getRowBuilder()); - } - if (isValidValue(fields()[12], other.array)) { - this.array = data().deepCopy(fields()[12].schema(), other.array); - fieldSetFlags()[12] = other.fieldSetFlags()[12]; - } - if (isValidValue(fields()[13], other.map)) { - this.map = data().deepCopy(fields()[13].schema(), other.map); - fieldSetFlags()[13] = other.fieldSetFlags()[13]; - } - } - - /** - * Creates a Builder by copying an existing TestAvro instance - * @param other The existing instance to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { - super(SCHEMA$, MODEL$); - if (isValidValue(fields()[0], other.bool_non_nullable)) { - this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); - fieldSetFlags()[0] = true; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = true; - } - if (isValidValue(fields()[2], other.long$)) { - this.long$ = data().deepCopy(fields()[2].schema(), other.long$); - fieldSetFlags()[2] = true; - } - if (isValidValue(fields()[3], other.float$)) { - this.float$ = data().deepCopy(fields()[3].schema(), other.float$); - fieldSetFlags()[3] = true; - } - if (isValidValue(fields()[4], other.double$)) { - this.double$ = data().deepCopy(fields()[4].schema(), other.double$); - fieldSetFlags()[4] = true; - } - if (isValidValue(fields()[5], other.string)) { - this.string = data().deepCopy(fields()[5].schema(), other.string); - fieldSetFlags()[5] = true; - } - if (isValidValue(fields()[6], other.bytes)) { - this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); - fieldSetFlags()[6] = true; - } - if (isValidValue(fields()[7], other.fixed)) { - this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); - fieldSetFlags()[7] = true; - } - if (isValidValue(fields()[8], other.date)) { - this.date = data().deepCopy(fields()[8].schema(), other.date); - fieldSetFlags()[8] = true; - } - if (isValidValue(fields()[9], other.timestampMillis)) { - this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); - fieldSetFlags()[9] = true; - } - if (isValidValue(fields()[10], other.TestEnum)) { - this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); - fieldSetFlags()[10] = true; - } - if (isValidValue(fields()[11], other.row)) { - this.row = data().deepCopy(fields()[11].schema(), other.row); - fieldSetFlags()[11] = true; - } - this.rowBuilder = null; - if (isValidValue(fields()[12], other.array)) { - this.array = data().deepCopy(fields()[12].schema(), other.array); - fieldSetFlags()[12] = true; - } - if (isValidValue(fields()[13], other.map)) { - this.map = data().deepCopy(fields()[13].schema(), other.map); - fieldSetFlags()[13] = true; - } - } - - /** - * Gets the value of the 'bool_non_nullable' field. - * @return The value. - */ - public boolean getBoolNonNullable() { - return bool_non_nullable; - } - - - /** - * Sets the value of the 'bool_non_nullable' field. - * @param value The value of 'bool_non_nullable'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBoolNonNullable(boolean value) { - validate(fields()[0], value); - this.bool_non_nullable = value; - fieldSetFlags()[0] = true; - return this; - } - - /** - * Checks whether the 'bool_non_nullable' field has been set. - * @return True if the 'bool_non_nullable' field has been set, false otherwise. - */ - public boolean hasBoolNonNullable() { - return fieldSetFlags()[0]; - } - - - /** - * Clears the value of the 'bool_non_nullable' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNonNullable() { - fieldSetFlags()[0] = false; - return this; - } - - /** - * Gets the value of the 'int$' field. - * @return The value. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value The value of 'int$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setInt$(java.lang.Integer value) { - validate(fields()[1], value); - this.int$ = value; - fieldSetFlags()[1] = true; - return this; - } - - /** - * Checks whether the 'int$' field has been set. - * @return True if the 'int$' field has been set, false otherwise. - */ - public boolean hasInt$() { - return fieldSetFlags()[1]; - } - - - /** - * Clears the value of the 'int$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearInt$() { - int$ = null; - fieldSetFlags()[1] = false; - return this; - } - - /** - * Gets the value of the 'long$' field. - * @return The value. - */ - public java.lang.Long getLong$() { - return long$; - } - - - /** - * Sets the value of the 'long$' field. - * @param value The value of 'long$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setLong$(java.lang.Long value) { - validate(fields()[2], value); - this.long$ = value; - fieldSetFlags()[2] = true; - return this; - } - - /** - * Checks whether the 'long$' field has been set. - * @return True if the 'long$' field has been set, false otherwise. - */ - public boolean hasLong$() { - return fieldSetFlags()[2]; - } - - - /** - * Clears the value of the 'long$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearLong$() { - long$ = null; - fieldSetFlags()[2] = false; - return this; - } - - /** - * Gets the value of the 'float$' field. - * @return The value. - */ - public java.lang.Float getFloat$() { - return float$; - } - - - /** - * Sets the value of the 'float$' field. - * @param value The value of 'float$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFloat$(java.lang.Float value) { - validate(fields()[3], value); - this.float$ = value; - fieldSetFlags()[3] = true; - return this; - } - - /** - * Checks whether the 'float$' field has been set. - * @return True if the 'float$' field has been set, false otherwise. - */ - public boolean hasFloat$() { - return fieldSetFlags()[3]; - } - - - /** - * Clears the value of the 'float$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFloat$() { - float$ = null; - fieldSetFlags()[3] = false; - return this; - } - - /** - * Gets the value of the 'double$' field. - * @return The value. - */ - public java.lang.Double getDouble$() { - return double$; - } - - - /** - * Sets the value of the 'double$' field. - * @param value The value of 'double$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDouble$(java.lang.Double value) { - validate(fields()[4], value); - this.double$ = value; - fieldSetFlags()[4] = true; - return this; - } - - /** - * Checks whether the 'double$' field has been set. - * @return True if the 'double$' field has been set, false otherwise. - */ - public boolean hasDouble$() { - return fieldSetFlags()[4]; - } - - - /** - * Clears the value of the 'double$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDouble$() { - double$ = null; - fieldSetFlags()[4] = false; - return this; - } - - /** - * Gets the value of the 'string' field. - * @return The value. - */ - public java.lang.CharSequence getString() { - return string; - } - - - /** - * Sets the value of the 'string' field. - * @param value The value of 'string'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setString(java.lang.CharSequence value) { - validate(fields()[5], value); - this.string = value; - fieldSetFlags()[5] = true; - return this; - } - - /** - * Checks whether the 'string' field has been set. - * @return True if the 'string' field has been set, false otherwise. - */ - public boolean hasString() { - return fieldSetFlags()[5]; - } - - - /** - * Clears the value of the 'string' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearString() { - string = null; - fieldSetFlags()[5] = false; - return this; - } - - /** - * Gets the value of the 'bytes' field. - * @return The value. - */ - public java.nio.ByteBuffer getBytes() { - return bytes; - } - - - /** - * Sets the value of the 'bytes' field. - * @param value The value of 'bytes'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBytes(java.nio.ByteBuffer value) { - validate(fields()[6], value); - this.bytes = value; - fieldSetFlags()[6] = true; - return this; - } - - /** - * Checks whether the 'bytes' field has been set. - * @return True if the 'bytes' field has been set, false otherwise. - */ - public boolean hasBytes() { - return fieldSetFlags()[6]; - } - - - /** - * Clears the value of the 'bytes' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBytes() { - bytes = null; - fieldSetFlags()[6] = false; - return this; - } - - /** - * Gets the value of the 'fixed' field. - * @return The value. - */ - public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { - return fixed; - } - - - /** - * Sets the value of the 'fixed' field. - * @param value The value of 'fixed'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { - validate(fields()[7], value); - this.fixed = value; - fieldSetFlags()[7] = true; - return this; - } - - /** - * Checks whether the 'fixed' field has been set. - * @return True if the 'fixed' field has been set, false otherwise. - */ - public boolean hasFixed() { - return fieldSetFlags()[7]; - } - - - /** - * Clears the value of the 'fixed' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFixed() { - fixed = null; - fieldSetFlags()[7] = false; - return this; - } - - /** - * Gets the value of the 'date' field. - * @return The value. - */ - public java.time.LocalDate getDate() { - return date; - } - - - /** - * Sets the value of the 'date' field. - * @param value The value of 'date'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDate(java.time.LocalDate value) { - validate(fields()[8], value); - this.date = value; - fieldSetFlags()[8] = true; - return this; - } - - /** - * Checks whether the 'date' field has been set. - * @return True if the 'date' field has been set, false otherwise. - */ - public boolean hasDate() { - return fieldSetFlags()[8]; - } - - - /** - * Clears the value of the 'date' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDate() { - fieldSetFlags()[8] = false; - return this; - } - - /** - * Gets the value of the 'timestampMillis' field. - * @return The value. - */ - public java.time.Instant getTimestampMillis() { - return timestampMillis; - } - - - /** - * Sets the value of the 'timestampMillis' field. - * @param value The value of 'timestampMillis'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTimestampMillis(java.time.Instant value) { - validate(fields()[9], value); - this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); - fieldSetFlags()[9] = true; - return this; - } - - /** - * Checks whether the 'timestampMillis' field has been set. - * @return True if the 'timestampMillis' field has been set, false otherwise. - */ - public boolean hasTimestampMillis() { - return fieldSetFlags()[9]; - } - - - /** - * Clears the value of the 'timestampMillis' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTimestampMillis() { - fieldSetFlags()[9] = false; - return this; - } - - /** - * Gets the value of the 'TestEnum' field. - * @return The value. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { - return TestEnum; - } - - - /** - * Sets the value of the 'TestEnum' field. - * @param value The value of 'TestEnum'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { - validate(fields()[10], value); - this.TestEnum = value; - fieldSetFlags()[10] = true; - return this; - } - - /** - * Checks whether the 'TestEnum' field has been set. - * @return True if the 'TestEnum' field has been set, false otherwise. - */ - public boolean hasTestEnum() { - return fieldSetFlags()[10]; - } - - - /** - * Clears the value of the 'TestEnum' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTestEnum() { - TestEnum = null; - fieldSetFlags()[10] = false; - return this; - } - - /** - * Gets the value of the 'row' field. - * @return The value. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { - return row; - } - - - /** - * Sets the value of the 'row' field. - * @param value The value of 'row'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { - validate(fields()[11], value); - this.rowBuilder = null; - this.row = value; - fieldSetFlags()[11] = true; - return this; - } - - /** - * Checks whether the 'row' field has been set. - * @return True if the 'row' field has been set, false otherwise. - */ - public boolean hasRow() { - return fieldSetFlags()[11]; - } - - /** - * Gets the Builder instance for the 'row' field and creates one if it doesn't exist yet. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder getRowBuilder() { - if (rowBuilder == null) { - if (hasRow()) { - setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(row)); - } else { - setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder()); - } - } - return rowBuilder; - } - - /** - * Sets the Builder instance for the 'row' field - * @param value The builder instance that must be set. - * @return This builder. - */ - - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder value) { - clearRow(); - rowBuilder = value; - return this; - } - - /** - * Checks whether the 'row' field has an active Builder instance - * @return True if the 'row' field has an active Builder instance - */ - public boolean hasRowBuilder() { - return rowBuilder != null; - } - - /** - * Clears the value of the 'row' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearRow() { - row = null; - rowBuilder = null; - fieldSetFlags()[11] = false; - return this; - } - - /** - * Gets the value of the 'array' field. - * @return The value. - */ - public java.util.List getArray() { - return array; - } - - - /** - * Sets the value of the 'array' field. - * @param value The value of 'array'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setArray(java.util.List value) { - validate(fields()[12], value); - this.array = value; - fieldSetFlags()[12] = true; - return this; - } - - /** - * Checks whether the 'array' field has been set. - * @return True if the 'array' field has been set, false otherwise. - */ - public boolean hasArray() { - return fieldSetFlags()[12]; - } - - - /** - * Clears the value of the 'array' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearArray() { - array = null; - fieldSetFlags()[12] = false; - return this; - } - - /** - * Gets the value of the 'map' field. - * @return The value. - */ - public java.util.Map getMap() { - return map; - } - - - /** - * Sets the value of the 'map' field. - * @param value The value of 'map'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setMap(java.util.Map value) { - validate(fields()[13], value); - this.map = value; - fieldSetFlags()[13] = true; - return this; - } - - /** - * Checks whether the 'map' field has been set. - * @return True if the 'map' field has been set, false otherwise. - */ - public boolean hasMap() { - return fieldSetFlags()[13]; - } - - - /** - * Clears the value of the 'map' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearMap() { - map = null; - fieldSetFlags()[13] = false; - return this; - } - - @Override - @SuppressWarnings("unchecked") - public TestAvro build() { - try { - TestAvro record = new TestAvro(); - record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (java.lang.Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); - record.long$ = fieldSetFlags()[2] ? this.long$ : (java.lang.Long) defaultValue(fields()[2]); - record.float$ = fieldSetFlags()[3] ? this.float$ : (java.lang.Float) defaultValue(fields()[3]); - record.double$ = fieldSetFlags()[4] ? this.double$ : (java.lang.Double) defaultValue(fields()[4]); - record.string = fieldSetFlags()[5] ? this.string : (java.lang.CharSequence) defaultValue(fields()[5]); - record.bytes = fieldSetFlags()[6] ? this.bytes : (java.nio.ByteBuffer) defaultValue(fields()[6]); - record.fixed = fieldSetFlags()[7] ? this.fixed : (org.apache.beam.sdk.extensions.avro.schemas.fixed4) defaultValue(fields()[7]); - record.date = fieldSetFlags()[8] ? this.date : (java.time.LocalDate) defaultValue(fields()[8]); - record.timestampMillis = fieldSetFlags()[9] ? this.timestampMillis : (java.time.Instant) defaultValue(fields()[9]); - record.TestEnum = fieldSetFlags()[10] ? this.TestEnum : (org.apache.beam.sdk.extensions.avro.schemas.TestEnum) defaultValue(fields()[10]); - if (rowBuilder != null) { - try { - record.row = this.rowBuilder.build(); - } catch (org.apache.avro.AvroMissingFieldException e) { - e.addParentField(record.getSchema().getField("row")); - throw e; - } - } else { - record.row = fieldSetFlags()[11] ? this.row : (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested) defaultValue(fields()[11]); - } - record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); - record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); - return record; - } catch (org.apache.avro.AvroMissingFieldException e) { - throw e; - } catch (java.lang.Exception e) { - throw new org.apache.avro.AvroRuntimeException(e); - } - } - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumWriter - WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, SpecificData.getEncoder(out)); - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumReader - READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, SpecificData.getDecoder(in)); - } - -} - - - - - - - - - - diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java deleted file mode 100644 index 0b250eee1597..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java +++ /dev/null @@ -1,426 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; - -import org.apache.avro.generic.GenericArray; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.util.Utf8; -import org.apache.avro.message.BinaryMessageEncoder; -import org.apache.avro.message.BinaryMessageDecoder; -import org.apache.avro.message.SchemaStore; - -@org.apache.avro.specific.AvroGenerated -public class TestAvroNested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - private static final long serialVersionUID = 4633138088036298925L; - - - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvroNested\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - private static final SpecificData MODEL$ = new SpecificData(); - - private static final BinaryMessageEncoder ENCODER = - new BinaryMessageEncoder<>(MODEL$, SCHEMA$); - - private static final BinaryMessageDecoder DECODER = - new BinaryMessageDecoder<>(MODEL$, SCHEMA$); - - /** - * Return the BinaryMessageEncoder instance used by this class. - * @return the message encoder used by this class - */ - public static BinaryMessageEncoder getEncoder() { - return ENCODER; - } - - /** - * Return the BinaryMessageDecoder instance used by this class. - * @return the message decoder used by this class - */ - public static BinaryMessageDecoder getDecoder() { - return DECODER; - } - - /** - * 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 - */ - public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { - return new BinaryMessageDecoder<>(MODEL$, SCHEMA$, resolver); - } - - /** - * Serializes this TestAvroNested to a ByteBuffer. - * @return a buffer holding the serialized data for this instance - * @throws java.io.IOException if this instance could not be serialized - */ - public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { - return ENCODER.encode(this); - } - - /** - * Deserializes a TestAvroNested from a ByteBuffer. - * @param b a byte buffer holding serialized data for an instance of this class - * @return a TestAvroNested instance decoded from the given buffer - * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class - */ - public static TestAvroNested fromByteBuffer( - java.nio.ByteBuffer b) throws java.io.IOException { - return DECODER.decode(b); - } - - private boolean BOOL_NON_NULLABLE; - private java.lang.Integer int$; - - /** - * Default constructor. Note that this does not initialize fields - * to their default values from the schema. If that is desired then - * one should use newBuilder(). - */ - public TestAvroNested() {} - - /** - * All-args constructor. - * @param BOOL_NON_NULLABLE The new value for BOOL_NON_NULLABLE - * @param int$ The new value for int - */ - public TestAvroNested(java.lang.Boolean BOOL_NON_NULLABLE, java.lang.Integer int$) { - this.BOOL_NON_NULLABLE = BOOL_NON_NULLABLE; - this.int$ = int$; - } - - @Override - public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } - - @Override - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - - // Used by DatumWriter. Applications should not call. - @Override - public java.lang.Object get(int field$) { - switch (field$) { - case 0: return BOOL_NON_NULLABLE; - case 1: return int$; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - // Used by DatumReader. Applications should not call. - @Override - @SuppressWarnings(value="unchecked") - public void put(int field$, java.lang.Object value$) { - switch (field$) { - case 0: BOOL_NON_NULLABLE = (java.lang.Boolean)value$; break; - case 1: int$ = (java.lang.Integer)value$; break; - default: throw new IndexOutOfBoundsException("Invalid index: " + field$); - } - } - - /** - * Gets the value of the 'BOOL_NON_NULLABLE' field. - * @return The value of the 'BOOL_NON_NULLABLE' field. - */ - public boolean getBOOLNONNULLABLE() { - return BOOL_NON_NULLABLE; - } - - - /** - * Sets the value of the 'BOOL_NON_NULLABLE' field. - * @param value the value to set. - */ - public void setBOOLNONNULLABLE(boolean value) { - this.BOOL_NON_NULLABLE = value; - } - - /** - * Gets the value of the 'int$' field. - * @return The value of the 'int$' field. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value the value to set. - */ - public void setInt$(java.lang.Integer value) { - this.int$ = value; - } - - /** - * Creates a new TestAvroNested RecordBuilder. - * @return A new TestAvroNested RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder() { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); - } - - /** - * Creates a new TestAvroNested RecordBuilder by copying an existing Builder. - * @param other The existing builder to copy. - * @return A new TestAvroNested RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); - } - } - - /** - * Creates a new TestAvroNested RecordBuilder by copying an existing TestAvroNested instance. - * @param other The existing instance to copy. - * @return A new TestAvroNested RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); - } - } - - /** - * RecordBuilder for TestAvroNested instances. - */ - @org.apache.avro.specific.AvroGenerated - public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase - implements org.apache.avro.data.RecordBuilder { - - private boolean BOOL_NON_NULLABLE; - private java.lang.Integer int$; - - /** Creates a new Builder */ - private Builder() { - super(SCHEMA$, MODEL$); - } - - /** - * Creates a Builder by copying an existing Builder. - * @param other The existing Builder to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { - super(other); - if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { - this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); - fieldSetFlags()[0] = other.fieldSetFlags()[0]; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = other.fieldSetFlags()[1]; - } - } - - /** - * Creates a Builder by copying an existing TestAvroNested instance - * @param other The existing instance to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { - super(SCHEMA$, MODEL$); - if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { - this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); - fieldSetFlags()[0] = true; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = true; - } - } - - /** - * Gets the value of the 'BOOL_NON_NULLABLE' field. - * @return The value. - */ - public boolean getBOOLNONNULLABLE() { - return BOOL_NON_NULLABLE; - } - - - /** - * Sets the value of the 'BOOL_NON_NULLABLE' field. - * @param value The value of 'BOOL_NON_NULLABLE'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setBOOLNONNULLABLE(boolean value) { - validate(fields()[0], value); - this.BOOL_NON_NULLABLE = value; - fieldSetFlags()[0] = true; - return this; - } - - /** - * Checks whether the 'BOOL_NON_NULLABLE' field has been set. - * @return True if the 'BOOL_NON_NULLABLE' field has been set, false otherwise. - */ - public boolean hasBOOLNONNULLABLE() { - return fieldSetFlags()[0]; - } - - - /** - * Clears the value of the 'BOOL_NON_NULLABLE' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearBOOLNONNULLABLE() { - fieldSetFlags()[0] = false; - return this; - } - - /** - * Gets the value of the 'int$' field. - * @return The value. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value The value of 'int$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setInt$(java.lang.Integer value) { - validate(fields()[1], value); - this.int$ = value; - fieldSetFlags()[1] = true; - return this; - } - - /** - * Checks whether the 'int$' field has been set. - * @return True if the 'int$' field has been set, false otherwise. - */ - public boolean hasInt$() { - return fieldSetFlags()[1]; - } - - - /** - * Clears the value of the 'int$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearInt$() { - int$ = null; - fieldSetFlags()[1] = false; - return this; - } - - @Override - @SuppressWarnings("unchecked") - public TestAvroNested build() { - try { - TestAvroNested record = new TestAvroNested(); - record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (java.lang.Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); - return record; - } catch (org.apache.avro.AvroMissingFieldException e) { - throw e; - } catch (java.lang.Exception e) { - throw new org.apache.avro.AvroRuntimeException(e); - } - } - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumWriter - WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, SpecificData.getEncoder(out)); - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumReader - READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, SpecificData.getDecoder(in)); - } - - @Override protected boolean hasCustomCoders() { return true; } - - @Override public void customEncode(org.apache.avro.io.Encoder out) - throws java.io.IOException - { - out.writeBoolean(this.BOOL_NON_NULLABLE); - - if (this.int$ == null) { - out.writeIndex(1); - out.writeNull(); - } else { - out.writeIndex(0); - out.writeInt(this.int$); - } - - } - - @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) - throws java.io.IOException - { - org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); - if (fieldOrder == null) { - this.BOOL_NON_NULLABLE = in.readBoolean(); - - if (in.readIndex() != 0) { - in.readNull(); - this.int$ = null; - } else { - this.int$ = in.readInt(); - } - - } else { - for (int i = 0; i < 2; i++) { - switch (fieldOrder[i].pos()) { - case 0: - this.BOOL_NON_NULLABLE = in.readBoolean(); - break; - - case 1: - if (in.readIndex() != 0) { - in.readNull(); - this.int$ = null; - } else { - this.int$ = in.readInt(); - } - break; - - default: - throw new java.io.IOException("Corrupt ResolvingDecoder."); - } - } - } - } -} - - - - - - - - - - diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java deleted file mode 100644 index eb52356a472b..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; -@org.apache.avro.specific.AvroGenerated -public enum TestEnum implements org.apache.avro.generic.GenericEnumSymbol { - abc, cde ; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"enum\",\"name\":\"TestEnum\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"symbols\":[\"abc\",\"cde\"]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - @Override - public org.apache.avro.Schema getSchema() { return SCHEMA$; } -} diff --git a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java b/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java deleted file mode 100644 index 24c5be9ca18d..000000000000 --- a/sdks/java/extensions/avro/src/test/avro1111/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; -@org.apache.avro.specific.FixedSize(4) -@org.apache.avro.specific.AvroGenerated -public class fixed4 extends org.apache.avro.specific.SpecificFixed { - private static final long serialVersionUID = -5646354132642432749L; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"fixed\",\"name\":\"fixed4\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"size\":4}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - - /** Creates a new fixed4 */ - public fixed4() { - super(); - } - - /** - * Creates a new fixed4 with the given bytes. - * @param bytes The bytes to create the new fixed4. - */ - public fixed4(byte[] bytes) { - super(bytes); - } - - private static final org.apache.avro.io.DatumWriter - WRITER$ = new org.apache.avro.specific.SpecificDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, org.apache.avro.specific.SpecificData.getEncoder(out)); - } - - private static final org.apache.avro.io.DatumReader - READER$ = new org.apache.avro.specific.SpecificDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, org.apache.avro.specific.SpecificData.getDecoder(in)); - } - -} diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java deleted file mode 100644 index a91b86da2f29..000000000000 --- a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/io/AvroGeneratedUser.java +++ /dev/null @@ -1,515 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.io; - -import org.apache.avro.generic.GenericArray; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.util.Utf8; -import org.apache.avro.message.BinaryMessageEncoder; -import org.apache.avro.message.BinaryMessageDecoder; -import org.apache.avro.message.SchemaStore; - -@org.apache.avro.specific.AvroGenerated -public class AvroGeneratedUser extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - private static final long serialVersionUID = 3056733241446362297L; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroGeneratedUser\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.io\",\"fields\":[{\"name\":\"name\",\"type\":\"string\"},{\"name\":\"favorite_number\",\"type\":[\"int\",\"null\"]},{\"name\":\"favorite_color\",\"type\":[\"string\",\"null\"]}]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - private static SpecificData MODEL$ = new SpecificData(); - - private static final BinaryMessageEncoder ENCODER = - new BinaryMessageEncoder(MODEL$, SCHEMA$); - - private static final BinaryMessageDecoder DECODER = - new BinaryMessageDecoder(MODEL$, SCHEMA$); - - /** - * Return the BinaryMessageEncoder instance used by this class. - * @return the message encoder used by this class - */ - public static BinaryMessageEncoder getEncoder() { - return ENCODER; - } - - /** - * Return the BinaryMessageDecoder instance used by this class. - * @return the message decoder used by this class - */ - public static BinaryMessageDecoder getDecoder() { - return DECODER; - } - - /** - * 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 - */ - public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { - return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); - } - - /** - * Serializes this AvroGeneratedUser to a ByteBuffer. - * @return a buffer holding the serialized data for this instance - * @throws java.io.IOException if this instance could not be serialized - */ - public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { - return ENCODER.encode(this); - } - - /** - * Deserializes a AvroGeneratedUser from a ByteBuffer. - * @param b a byte buffer holding serialized data for an instance of this class - * @return a AvroGeneratedUser instance decoded from the given buffer - * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class - */ - public static AvroGeneratedUser fromByteBuffer( - java.nio.ByteBuffer b) throws java.io.IOException { - return DECODER.decode(b); - } - - private java.lang.CharSequence name; - private java.lang.Integer favorite_number; - private java.lang.CharSequence favorite_color; - - /** - * Default constructor. Note that this does not initialize fields - * to their default values from the schema. If that is desired then - * one should use newBuilder(). - */ - public AvroGeneratedUser() {} - - /** - * All-args constructor. - * @param name The new value for name - * @param favorite_number The new value for favorite_number - * @param favorite_color The new value for favorite_color - */ - public AvroGeneratedUser(java.lang.CharSequence name, java.lang.Integer favorite_number, java.lang.CharSequence favorite_color) { - this.name = name; - this.favorite_number = favorite_number; - this.favorite_color = favorite_color; - } - - public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - // Used by DatumWriter. Applications should not call. - public java.lang.Object get(int field$) { - switch (field$) { - case 0: return name; - case 1: return favorite_number; - case 2: return favorite_color; - default: throw new org.apache.avro.AvroRuntimeException("Bad index"); - } - } - - // Used by DatumReader. Applications should not call. - @SuppressWarnings(value="unchecked") - public void put(int field$, java.lang.Object value$) { - switch (field$) { - case 0: name = (java.lang.CharSequence)value$; break; - case 1: favorite_number = (java.lang.Integer)value$; break; - case 2: favorite_color = (java.lang.CharSequence)value$; break; - default: throw new org.apache.avro.AvroRuntimeException("Bad index"); - } - } - - /** - * Gets the value of the 'name' field. - * @return The value of the 'name' field. - */ - public java.lang.CharSequence getName() { - return name; - } - - - /** - * Sets the value of the 'name' field. - * @param value the value to set. - */ - public void setName(java.lang.CharSequence value) { - this.name = value; - } - - /** - * Gets the value of the 'favorite_number' field. - * @return The value of the 'favorite_number' field. - */ - public java.lang.Integer getFavoriteNumber() { - return favorite_number; - } - - - /** - * Sets the value of the 'favorite_number' field. - * @param value the value to set. - */ - public void setFavoriteNumber(java.lang.Integer value) { - this.favorite_number = value; - } - - /** - * Gets the value of the 'favorite_color' field. - * @return The value of the 'favorite_color' field. - */ - public java.lang.CharSequence getFavoriteColor() { - return favorite_color; - } - - - /** - * Sets the value of the 'favorite_color' field. - * @param value the value to set. - */ - public void setFavoriteColor(java.lang.CharSequence value) { - this.favorite_color = value; - } - - /** - * Creates a new AvroGeneratedUser RecordBuilder. - * @return A new AvroGeneratedUser RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder() { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); - } - - /** - * Creates a new AvroGeneratedUser RecordBuilder by copying an existing Builder. - * @param other The existing builder to copy. - * @return A new AvroGeneratedUser RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); - } - } - - /** - * Creates a new AvroGeneratedUser RecordBuilder by copying an existing AvroGeneratedUser instance. - * @param other The existing instance to copy. - * @return A new AvroGeneratedUser RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder newBuilder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder(other); - } - } - - /** - * RecordBuilder for AvroGeneratedUser instances. - */ - @org.apache.avro.specific.AvroGenerated - public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase - implements org.apache.avro.data.RecordBuilder { - - private java.lang.CharSequence name; - private java.lang.Integer favorite_number; - private java.lang.CharSequence favorite_color; - - /** Creates a new Builder */ - private Builder() { - super(SCHEMA$); - } - - /** - * Creates a Builder by copying an existing Builder. - * @param other The existing Builder to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder other) { - super(other); - if (isValidValue(fields()[0], other.name)) { - this.name = data().deepCopy(fields()[0].schema(), other.name); - fieldSetFlags()[0] = other.fieldSetFlags()[0]; - } - if (isValidValue(fields()[1], other.favorite_number)) { - this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); - fieldSetFlags()[1] = other.fieldSetFlags()[1]; - } - if (isValidValue(fields()[2], other.favorite_color)) { - this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); - fieldSetFlags()[2] = other.fieldSetFlags()[2]; - } - } - - /** - * Creates a Builder by copying an existing AvroGeneratedUser instance - * @param other The existing instance to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser other) { - super(SCHEMA$); - if (isValidValue(fields()[0], other.name)) { - this.name = data().deepCopy(fields()[0].schema(), other.name); - fieldSetFlags()[0] = true; - } - if (isValidValue(fields()[1], other.favorite_number)) { - this.favorite_number = data().deepCopy(fields()[1].schema(), other.favorite_number); - fieldSetFlags()[1] = true; - } - if (isValidValue(fields()[2], other.favorite_color)) { - this.favorite_color = data().deepCopy(fields()[2].schema(), other.favorite_color); - fieldSetFlags()[2] = true; - } - } - - /** - * Gets the value of the 'name' field. - * @return The value. - */ - public java.lang.CharSequence getName() { - return name; - } - - - /** - * Sets the value of the 'name' field. - * @param value The value of 'name'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setName(java.lang.CharSequence value) { - validate(fields()[0], value); - this.name = value; - fieldSetFlags()[0] = true; - return this; - } - - /** - * Checks whether the 'name' field has been set. - * @return True if the 'name' field has been set, false otherwise. - */ - public boolean hasName() { - return fieldSetFlags()[0]; - } - - - /** - * Clears the value of the 'name' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearName() { - name = null; - fieldSetFlags()[0] = false; - return this; - } - - /** - * Gets the value of the 'favorite_number' field. - * @return The value. - */ - public java.lang.Integer getFavoriteNumber() { - return favorite_number; - } - - - /** - * Sets the value of the 'favorite_number' field. - * @param value The value of 'favorite_number'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteNumber(java.lang.Integer value) { - validate(fields()[1], value); - this.favorite_number = value; - fieldSetFlags()[1] = true; - return this; - } - - /** - * Checks whether the 'favorite_number' field has been set. - * @return True if the 'favorite_number' field has been set, false otherwise. - */ - public boolean hasFavoriteNumber() { - return fieldSetFlags()[1]; - } - - - /** - * Clears the value of the 'favorite_number' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteNumber() { - favorite_number = null; - fieldSetFlags()[1] = false; - return this; - } - - /** - * Gets the value of the 'favorite_color' field. - * @return The value. - */ - public java.lang.CharSequence getFavoriteColor() { - return favorite_color; - } - - - /** - * Sets the value of the 'favorite_color' field. - * @param value The value of 'favorite_color'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder setFavoriteColor(java.lang.CharSequence value) { - validate(fields()[2], value); - this.favorite_color = value; - fieldSetFlags()[2] = true; - return this; - } - - /** - * Checks whether the 'favorite_color' field has been set. - * @return True if the 'favorite_color' field has been set, false otherwise. - */ - public boolean hasFavoriteColor() { - return fieldSetFlags()[2]; - } - - - /** - * Clears the value of the 'favorite_color' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.io.AvroGeneratedUser.Builder clearFavoriteColor() { - favorite_color = null; - fieldSetFlags()[2] = false; - return this; - } - - @Override - @SuppressWarnings("unchecked") - public AvroGeneratedUser build() { - try { - AvroGeneratedUser record = new AvroGeneratedUser(); - record.name = fieldSetFlags()[0] ? this.name : (java.lang.CharSequence) defaultValue(fields()[0]); - record.favorite_number = fieldSetFlags()[1] ? this.favorite_number : (java.lang.Integer) defaultValue(fields()[1]); - record.favorite_color = fieldSetFlags()[2] ? this.favorite_color : (java.lang.CharSequence) defaultValue(fields()[2]); - return record; - } catch (org.apache.avro.AvroMissingFieldException e) { - throw e; - } catch (java.lang.Exception e) { - throw new org.apache.avro.AvroRuntimeException(e); - } - } - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumWriter - WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, SpecificData.getEncoder(out)); - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumReader - READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, SpecificData.getDecoder(in)); - } - - @Override protected boolean hasCustomCoders() { return true; } - - @Override public void customEncode(org.apache.avro.io.Encoder out) - throws java.io.IOException - { - out.writeString(this.name); - - if (this.favorite_number == null) { - out.writeIndex(1); - out.writeNull(); - } else { - out.writeIndex(0); - out.writeInt(this.favorite_number); - } - - if (this.favorite_color == null) { - out.writeIndex(1); - out.writeNull(); - } else { - out.writeIndex(0); - out.writeString(this.favorite_color); - } - - } - - @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) - throws java.io.IOException - { - org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); - if (fieldOrder == null) { - this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); - - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_number = null; - } else { - this.favorite_number = in.readInt(); - } - - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_color = null; - } else { - this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); - } - - } else { - for (int i = 0; i < 3; i++) { - switch (fieldOrder[i].pos()) { - case 0: - this.name = in.readString(this.name instanceof Utf8 ? (Utf8)this.name : null); - break; - - case 1: - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_number = null; - } else { - this.favorite_number = in.readInt(); - } - break; - - case 2: - if (in.readIndex() != 0) { - in.readNull(); - this.favorite_color = null; - } else { - this.favorite_color = in.readString(this.favorite_color instanceof Utf8 ? (Utf8)this.favorite_color : null); - } - break; - - default: - throw new java.io.IOException("Corrupt ResolvingDecoder."); - } - } - } - } -} - - - - - - - - - - diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java deleted file mode 100644 index 501fc6772ed3..000000000000 --- a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvro.java +++ /dev/null @@ -1,1304 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; - -import org.apache.avro.generic.GenericArray; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.util.Utf8; -import org.apache.avro.message.BinaryMessageEncoder; -import org.apache.avro.message.BinaryMessageDecoder; -import org.apache.avro.message.SchemaStore; - -@org.apache.avro.specific.AvroGenerated -public class TestAvro extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - private static final long serialVersionUID = 27902431178981259L; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvro\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"bool_non_nullable\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]},{\"name\":\"long\",\"type\":[\"long\",\"null\"]},{\"name\":\"float\",\"type\":[\"float\",\"null\"]},{\"name\":\"double\",\"type\":[\"double\",\"null\"]},{\"name\":\"string\",\"type\":[\"string\",\"null\"]},{\"name\":\"bytes\",\"type\":[\"bytes\",\"null\"]},{\"name\":\"fixed\",\"type\":{\"type\":\"fixed\",\"name\":\"fixed4\",\"size\":4}},{\"name\":\"date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}},{\"name\":\"timestampMillis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}},{\"name\":\"TestEnum\",\"type\":{\"type\":\"enum\",\"name\":\"TestEnum\",\"symbols\":[\"abc\",\"cde\"]}},{\"name\":\"row\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"TestAvroNested\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}]},{\"name\":\"array\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\",\"TestAvroNested\"]}]},{\"name\":\"map\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"TestAvroNested\"]}]}]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - private static SpecificData MODEL$ = new SpecificData(); -static { - MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.DateConversion()); - MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.TimestampMillisConversion()); - } - - private static final BinaryMessageEncoder ENCODER = - new BinaryMessageEncoder(MODEL$, SCHEMA$); - - private static final BinaryMessageDecoder DECODER = - new BinaryMessageDecoder(MODEL$, SCHEMA$); - - /** - * Return the BinaryMessageEncoder instance used by this class. - * @return the message encoder used by this class - */ - public static BinaryMessageEncoder getEncoder() { - return ENCODER; - } - - /** - * Return the BinaryMessageDecoder instance used by this class. - * @return the message decoder used by this class - */ - public static BinaryMessageDecoder getDecoder() { - return DECODER; - } - - /** - * 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 - */ - public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { - return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); - } - - /** - * Serializes this TestAvro to a ByteBuffer. - * @return a buffer holding the serialized data for this instance - * @throws java.io.IOException if this instance could not be serialized - */ - public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { - return ENCODER.encode(this); - } - - /** - * Deserializes a TestAvro from a ByteBuffer. - * @param b a byte buffer holding serialized data for an instance of this class - * @return a TestAvro instance decoded from the given buffer - * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class - */ - public static TestAvro fromByteBuffer( - java.nio.ByteBuffer b) throws java.io.IOException { - return DECODER.decode(b); - } - - private boolean bool_non_nullable; - private java.lang.Integer int$; - private java.lang.Long long$; - private java.lang.Float float$; - private java.lang.Double double$; - private java.lang.CharSequence string; - private java.nio.ByteBuffer bytes; - private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; - private java.time.LocalDate date; - private java.time.Instant timestampMillis; - private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; - private java.util.List array; - private java.util.Map map; - - /** - * Default constructor. Note that this does not initialize fields - * to their default values from the schema. If that is desired then - * one should use newBuilder(). - */ - public TestAvro() {} - - /** - * All-args constructor. - * @param bool_non_nullable The new value for bool_non_nullable - * @param int$ The new value for int - * @param long$ The new value for long - * @param float$ The new value for float - * @param double$ The new value for double - * @param string The new value for string - * @param bytes The new value for bytes - * @param fixed The new value for fixed - * @param date The new value for date - * @param timestampMillis The new value for timestampMillis - * @param TestEnum The new value for TestEnum - * @param row The new value for row - * @param array The new value for array - * @param map The new value for map - */ - public TestAvro(java.lang.Boolean bool_non_nullable, java.lang.Integer int$, java.lang.Long long$, java.lang.Float float$, java.lang.Double double$, java.lang.CharSequence string, java.nio.ByteBuffer bytes, org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed, java.time.LocalDate date, java.time.Instant timestampMillis, org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum, org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row, java.util.List array, java.util.Map map) { - this.bool_non_nullable = bool_non_nullable; - this.int$ = int$; - this.long$ = long$; - this.float$ = float$; - this.double$ = double$; - this.string = string; - this.bytes = bytes; - this.fixed = fixed; - this.date = date; - this.timestampMillis = timestampMillis.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); - this.TestEnum = TestEnum; - this.row = row; - this.array = array; - this.map = map; - } - - public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - // Used by DatumWriter. Applications should not call. - public java.lang.Object get(int field$) { - switch (field$) { - case 0: return bool_non_nullable; - case 1: return int$; - case 2: return long$; - case 3: return float$; - case 4: return double$; - case 5: return string; - case 6: return bytes; - case 7: return fixed; - case 8: return date; - case 9: return timestampMillis; - case 10: return TestEnum; - case 11: return row; - case 12: return array; - case 13: return map; - default: throw new org.apache.avro.AvroRuntimeException("Bad index"); - } - } - - private static final org.apache.avro.Conversion[] conversions = - new org.apache.avro.Conversion[] { - null, - null, - null, - null, - null, - null, - null, - null, - new org.apache.avro.data.TimeConversions.DateConversion(), - new org.apache.avro.data.TimeConversions.TimestampMillisConversion(), - null, - null, - null, - null, - null - }; - - @Override - public org.apache.avro.Conversion getConversion(int field) { - return conversions[field]; - } - - // Used by DatumReader. Applications should not call. - @SuppressWarnings(value="unchecked") - public void put(int field$, java.lang.Object value$) { - switch (field$) { - case 0: bool_non_nullable = (java.lang.Boolean)value$; break; - case 1: int$ = (java.lang.Integer)value$; break; - case 2: long$ = (java.lang.Long)value$; break; - case 3: float$ = (java.lang.Float)value$; break; - case 4: double$ = (java.lang.Double)value$; break; - case 5: string = (java.lang.CharSequence)value$; break; - case 6: bytes = (java.nio.ByteBuffer)value$; break; - case 7: fixed = (org.apache.beam.sdk.extensions.avro.schemas.fixed4)value$; break; - case 8: date = (java.time.LocalDate)value$; break; - case 9: timestampMillis = (java.time.Instant)value$; break; - case 10: TestEnum = (org.apache.beam.sdk.extensions.avro.schemas.TestEnum)value$; break; - case 11: row = (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested)value$; break; - case 12: array = (java.util.List)value$; break; - case 13: map = (java.util.Map)value$; break; - default: throw new org.apache.avro.AvroRuntimeException("Bad index"); - } - } - - /** - * Gets the value of the 'bool_non_nullable' field. - * @return The value of the 'bool_non_nullable' field. - */ - public boolean getBoolNonNullable() { - return bool_non_nullable; - } - - - /** - * Sets the value of the 'bool_non_nullable' field. - * @param value the value to set. - */ - public void setBoolNonNullable(boolean value) { - this.bool_non_nullable = value; - } - - /** - * Gets the value of the 'int$' field. - * @return The value of the 'int$' field. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value the value to set. - */ - public void setInt$(java.lang.Integer value) { - this.int$ = value; - } - - /** - * Gets the value of the 'long$' field. - * @return The value of the 'long$' field. - */ - public java.lang.Long getLong$() { - return long$; - } - - - /** - * Sets the value of the 'long$' field. - * @param value the value to set. - */ - public void setLong$(java.lang.Long value) { - this.long$ = value; - } - - /** - * Gets the value of the 'float$' field. - * @return The value of the 'float$' field. - */ - public java.lang.Float getFloat$() { - return float$; - } - - - /** - * Sets the value of the 'float$' field. - * @param value the value to set. - */ - public void setFloat$(java.lang.Float value) { - this.float$ = value; - } - - /** - * Gets the value of the 'double$' field. - * @return The value of the 'double$' field. - */ - public java.lang.Double getDouble$() { - return double$; - } - - - /** - * Sets the value of the 'double$' field. - * @param value the value to set. - */ - public void setDouble$(java.lang.Double value) { - this.double$ = value; - } - - /** - * Gets the value of the 'string' field. - * @return The value of the 'string' field. - */ - public java.lang.CharSequence getString() { - return string; - } - - - /** - * Sets the value of the 'string' field. - * @param value the value to set. - */ - public void setString(java.lang.CharSequence value) { - this.string = value; - } - - /** - * Gets the value of the 'bytes' field. - * @return The value of the 'bytes' field. - */ - public java.nio.ByteBuffer getBytes() { - return bytes; - } - - - /** - * Sets the value of the 'bytes' field. - * @param value the value to set. - */ - public void setBytes(java.nio.ByteBuffer value) { - this.bytes = value; - } - - /** - * Gets the value of the 'fixed' field. - * @return The value of the 'fixed' field. - */ - public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { - return fixed; - } - - - /** - * Sets the value of the 'fixed' field. - * @param value the value to set. - */ - public void setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { - this.fixed = value; - } - - /** - * Gets the value of the 'date' field. - * @return The value of the 'date' field. - */ - public java.time.LocalDate getDate() { - return date; - } - - - /** - * Sets the value of the 'date' field. - * @param value the value to set. - */ - public void setDate(java.time.LocalDate value) { - this.date = value; - } - - /** - * Gets the value of the 'timestampMillis' field. - * @return The value of the 'timestampMillis' field. - */ - public java.time.Instant getTimestampMillis() { - return timestampMillis; - } - - - /** - * Sets the value of the 'timestampMillis' field. - * @param value the value to set. - */ - public void setTimestampMillis(java.time.Instant value) { - this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); - } - - /** - * Gets the value of the 'TestEnum' field. - * @return The value of the 'TestEnum' field. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { - return TestEnum; - } - - - /** - * Sets the value of the 'TestEnum' field. - * @param value the value to set. - */ - public void setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { - this.TestEnum = value; - } - - /** - * Gets the value of the 'row' field. - * @return The value of the 'row' field. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { - return row; - } - - - /** - * Sets the value of the 'row' field. - * @param value the value to set. - */ - public void setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { - this.row = value; - } - - /** - * Gets the value of the 'array' field. - * @return The value of the 'array' field. - */ - public java.util.List getArray() { - return array; - } - - - /** - * Sets the value of the 'array' field. - * @param value the value to set. - */ - public void setArray(java.util.List value) { - this.array = value; - } - - /** - * Gets the value of the 'map' field. - * @return The value of the 'map' field. - */ - public java.util.Map getMap() { - return map; - } - - - /** - * Sets the value of the 'map' field. - * @param value the value to set. - */ - public void setMap(java.util.Map value) { - this.map = value; - } - - /** - * Creates a new TestAvro RecordBuilder. - * @return A new TestAvro RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder() { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); - } - - /** - * Creates a new TestAvro RecordBuilder by copying an existing Builder. - * @param other The existing builder to copy. - * @return A new TestAvro RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); - } - } - - /** - * Creates a new TestAvro RecordBuilder by copying an existing TestAvro instance. - * @param other The existing instance to copy. - * @return A new TestAvro RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder(other); - } - } - - /** - * RecordBuilder for TestAvro instances. - */ - @org.apache.avro.specific.AvroGenerated - public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase - implements org.apache.avro.data.RecordBuilder { - - private boolean bool_non_nullable; - private java.lang.Integer int$; - private java.lang.Long long$; - private java.lang.Float float$; - private java.lang.Double double$; - private java.lang.CharSequence string; - private java.nio.ByteBuffer bytes; - private org.apache.beam.sdk.extensions.avro.schemas.fixed4 fixed; - private java.time.LocalDate date; - private java.time.Instant timestampMillis; - private org.apache.beam.sdk.extensions.avro.schemas.TestEnum TestEnum; - private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested row; - private org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder rowBuilder; - private java.util.List array; - private java.util.Map map; - - /** Creates a new Builder */ - private Builder() { - super(SCHEMA$); - } - - /** - * Creates a Builder by copying an existing Builder. - * @param other The existing Builder to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder other) { - super(other); - if (isValidValue(fields()[0], other.bool_non_nullable)) { - this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); - fieldSetFlags()[0] = other.fieldSetFlags()[0]; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = other.fieldSetFlags()[1]; - } - if (isValidValue(fields()[2], other.long$)) { - this.long$ = data().deepCopy(fields()[2].schema(), other.long$); - fieldSetFlags()[2] = other.fieldSetFlags()[2]; - } - if (isValidValue(fields()[3], other.float$)) { - this.float$ = data().deepCopy(fields()[3].schema(), other.float$); - fieldSetFlags()[3] = other.fieldSetFlags()[3]; - } - if (isValidValue(fields()[4], other.double$)) { - this.double$ = data().deepCopy(fields()[4].schema(), other.double$); - fieldSetFlags()[4] = other.fieldSetFlags()[4]; - } - if (isValidValue(fields()[5], other.string)) { - this.string = data().deepCopy(fields()[5].schema(), other.string); - fieldSetFlags()[5] = other.fieldSetFlags()[5]; - } - if (isValidValue(fields()[6], other.bytes)) { - this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); - fieldSetFlags()[6] = other.fieldSetFlags()[6]; - } - if (isValidValue(fields()[7], other.fixed)) { - this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); - fieldSetFlags()[7] = other.fieldSetFlags()[7]; - } - if (isValidValue(fields()[8], other.date)) { - this.date = data().deepCopy(fields()[8].schema(), other.date); - fieldSetFlags()[8] = other.fieldSetFlags()[8]; - } - if (isValidValue(fields()[9], other.timestampMillis)) { - this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); - fieldSetFlags()[9] = other.fieldSetFlags()[9]; - } - if (isValidValue(fields()[10], other.TestEnum)) { - this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); - fieldSetFlags()[10] = other.fieldSetFlags()[10]; - } - if (isValidValue(fields()[11], other.row)) { - this.row = data().deepCopy(fields()[11].schema(), other.row); - fieldSetFlags()[11] = other.fieldSetFlags()[11]; - } - if (other.hasRowBuilder()) { - this.rowBuilder = org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(other.getRowBuilder()); - } - if (isValidValue(fields()[12], other.array)) { - this.array = data().deepCopy(fields()[12].schema(), other.array); - fieldSetFlags()[12] = other.fieldSetFlags()[12]; - } - if (isValidValue(fields()[13], other.map)) { - this.map = data().deepCopy(fields()[13].schema(), other.map); - fieldSetFlags()[13] = other.fieldSetFlags()[13]; - } - } - - /** - * Creates a Builder by copying an existing TestAvro instance - * @param other The existing instance to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvro other) { - super(SCHEMA$); - if (isValidValue(fields()[0], other.bool_non_nullable)) { - this.bool_non_nullable = data().deepCopy(fields()[0].schema(), other.bool_non_nullable); - fieldSetFlags()[0] = true; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = true; - } - if (isValidValue(fields()[2], other.long$)) { - this.long$ = data().deepCopy(fields()[2].schema(), other.long$); - fieldSetFlags()[2] = true; - } - if (isValidValue(fields()[3], other.float$)) { - this.float$ = data().deepCopy(fields()[3].schema(), other.float$); - fieldSetFlags()[3] = true; - } - if (isValidValue(fields()[4], other.double$)) { - this.double$ = data().deepCopy(fields()[4].schema(), other.double$); - fieldSetFlags()[4] = true; - } - if (isValidValue(fields()[5], other.string)) { - this.string = data().deepCopy(fields()[5].schema(), other.string); - fieldSetFlags()[5] = true; - } - if (isValidValue(fields()[6], other.bytes)) { - this.bytes = data().deepCopy(fields()[6].schema(), other.bytes); - fieldSetFlags()[6] = true; - } - if (isValidValue(fields()[7], other.fixed)) { - this.fixed = data().deepCopy(fields()[7].schema(), other.fixed); - fieldSetFlags()[7] = true; - } - if (isValidValue(fields()[8], other.date)) { - this.date = data().deepCopy(fields()[8].schema(), other.date); - fieldSetFlags()[8] = true; - } - if (isValidValue(fields()[9], other.timestampMillis)) { - this.timestampMillis = data().deepCopy(fields()[9].schema(), other.timestampMillis); - fieldSetFlags()[9] = true; - } - if (isValidValue(fields()[10], other.TestEnum)) { - this.TestEnum = data().deepCopy(fields()[10].schema(), other.TestEnum); - fieldSetFlags()[10] = true; - } - if (isValidValue(fields()[11], other.row)) { - this.row = data().deepCopy(fields()[11].schema(), other.row); - fieldSetFlags()[11] = true; - } - this.rowBuilder = null; - if (isValidValue(fields()[12], other.array)) { - this.array = data().deepCopy(fields()[12].schema(), other.array); - fieldSetFlags()[12] = true; - } - if (isValidValue(fields()[13], other.map)) { - this.map = data().deepCopy(fields()[13].schema(), other.map); - fieldSetFlags()[13] = true; - } - } - - /** - * Gets the value of the 'bool_non_nullable' field. - * @return The value. - */ - public boolean getBoolNonNullable() { - return bool_non_nullable; - } - - - /** - * Sets the value of the 'bool_non_nullable' field. - * @param value The value of 'bool_non_nullable'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBoolNonNullable(boolean value) { - validate(fields()[0], value); - this.bool_non_nullable = value; - fieldSetFlags()[0] = true; - return this; - } - - /** - * Checks whether the 'bool_non_nullable' field has been set. - * @return True if the 'bool_non_nullable' field has been set, false otherwise. - */ - public boolean hasBoolNonNullable() { - return fieldSetFlags()[0]; - } - - - /** - * Clears the value of the 'bool_non_nullable' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBoolNonNullable() { - fieldSetFlags()[0] = false; - return this; - } - - /** - * Gets the value of the 'int$' field. - * @return The value. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value The value of 'int$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setInt$(java.lang.Integer value) { - validate(fields()[1], value); - this.int$ = value; - fieldSetFlags()[1] = true; - return this; - } - - /** - * Checks whether the 'int$' field has been set. - * @return True if the 'int$' field has been set, false otherwise. - */ - public boolean hasInt$() { - return fieldSetFlags()[1]; - } - - - /** - * Clears the value of the 'int$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearInt$() { - int$ = null; - fieldSetFlags()[1] = false; - return this; - } - - /** - * Gets the value of the 'long$' field. - * @return The value. - */ - public java.lang.Long getLong$() { - return long$; - } - - - /** - * Sets the value of the 'long$' field. - * @param value The value of 'long$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setLong$(java.lang.Long value) { - validate(fields()[2], value); - this.long$ = value; - fieldSetFlags()[2] = true; - return this; - } - - /** - * Checks whether the 'long$' field has been set. - * @return True if the 'long$' field has been set, false otherwise. - */ - public boolean hasLong$() { - return fieldSetFlags()[2]; - } - - - /** - * Clears the value of the 'long$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearLong$() { - long$ = null; - fieldSetFlags()[2] = false; - return this; - } - - /** - * Gets the value of the 'float$' field. - * @return The value. - */ - public java.lang.Float getFloat$() { - return float$; - } - - - /** - * Sets the value of the 'float$' field. - * @param value The value of 'float$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFloat$(java.lang.Float value) { - validate(fields()[3], value); - this.float$ = value; - fieldSetFlags()[3] = true; - return this; - } - - /** - * Checks whether the 'float$' field has been set. - * @return True if the 'float$' field has been set, false otherwise. - */ - public boolean hasFloat$() { - return fieldSetFlags()[3]; - } - - - /** - * Clears the value of the 'float$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFloat$() { - float$ = null; - fieldSetFlags()[3] = false; - return this; - } - - /** - * Gets the value of the 'double$' field. - * @return The value. - */ - public java.lang.Double getDouble$() { - return double$; - } - - - /** - * Sets the value of the 'double$' field. - * @param value The value of 'double$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDouble$(java.lang.Double value) { - validate(fields()[4], value); - this.double$ = value; - fieldSetFlags()[4] = true; - return this; - } - - /** - * Checks whether the 'double$' field has been set. - * @return True if the 'double$' field has been set, false otherwise. - */ - public boolean hasDouble$() { - return fieldSetFlags()[4]; - } - - - /** - * Clears the value of the 'double$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDouble$() { - double$ = null; - fieldSetFlags()[4] = false; - return this; - } - - /** - * Gets the value of the 'string' field. - * @return The value. - */ - public java.lang.CharSequence getString() { - return string; - } - - - /** - * Sets the value of the 'string' field. - * @param value The value of 'string'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setString(java.lang.CharSequence value) { - validate(fields()[5], value); - this.string = value; - fieldSetFlags()[5] = true; - return this; - } - - /** - * Checks whether the 'string' field has been set. - * @return True if the 'string' field has been set, false otherwise. - */ - public boolean hasString() { - return fieldSetFlags()[5]; - } - - - /** - * Clears the value of the 'string' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearString() { - string = null; - fieldSetFlags()[5] = false; - return this; - } - - /** - * Gets the value of the 'bytes' field. - * @return The value. - */ - public java.nio.ByteBuffer getBytes() { - return bytes; - } - - - /** - * Sets the value of the 'bytes' field. - * @param value The value of 'bytes'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setBytes(java.nio.ByteBuffer value) { - validate(fields()[6], value); - this.bytes = value; - fieldSetFlags()[6] = true; - return this; - } - - /** - * Checks whether the 'bytes' field has been set. - * @return True if the 'bytes' field has been set, false otherwise. - */ - public boolean hasBytes() { - return fieldSetFlags()[6]; - } - - - /** - * Clears the value of the 'bytes' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearBytes() { - bytes = null; - fieldSetFlags()[6] = false; - return this; - } - - /** - * Gets the value of the 'fixed' field. - * @return The value. - */ - public org.apache.beam.sdk.extensions.avro.schemas.fixed4 getFixed() { - return fixed; - } - - - /** - * Sets the value of the 'fixed' field. - * @param value The value of 'fixed'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setFixed(org.apache.beam.sdk.extensions.avro.schemas.fixed4 value) { - validate(fields()[7], value); - this.fixed = value; - fieldSetFlags()[7] = true; - return this; - } - - /** - * Checks whether the 'fixed' field has been set. - * @return True if the 'fixed' field has been set, false otherwise. - */ - public boolean hasFixed() { - return fieldSetFlags()[7]; - } - - - /** - * Clears the value of the 'fixed' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearFixed() { - fixed = null; - fieldSetFlags()[7] = false; - return this; - } - - /** - * Gets the value of the 'date' field. - * @return The value. - */ - public java.time.LocalDate getDate() { - return date; - } - - - /** - * Sets the value of the 'date' field. - * @param value The value of 'date'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setDate(java.time.LocalDate value) { - validate(fields()[8], value); - this.date = value; - fieldSetFlags()[8] = true; - return this; - } - - /** - * Checks whether the 'date' field has been set. - * @return True if the 'date' field has been set, false otherwise. - */ - public boolean hasDate() { - return fieldSetFlags()[8]; - } - - - /** - * Clears the value of the 'date' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearDate() { - fieldSetFlags()[8] = false; - return this; - } - - /** - * Gets the value of the 'timestampMillis' field. - * @return The value. - */ - public java.time.Instant getTimestampMillis() { - return timestampMillis; - } - - - /** - * Sets the value of the 'timestampMillis' field. - * @param value The value of 'timestampMillis'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTimestampMillis(java.time.Instant value) { - validate(fields()[9], value); - this.timestampMillis = value.truncatedTo(java.time.temporal.ChronoUnit.MILLIS); - fieldSetFlags()[9] = true; - return this; - } - - /** - * Checks whether the 'timestampMillis' field has been set. - * @return True if the 'timestampMillis' field has been set, false otherwise. - */ - public boolean hasTimestampMillis() { - return fieldSetFlags()[9]; - } - - - /** - * Clears the value of the 'timestampMillis' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTimestampMillis() { - fieldSetFlags()[9] = false; - return this; - } - - /** - * Gets the value of the 'TestEnum' field. - * @return The value. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestEnum getTestEnum() { - return TestEnum; - } - - - /** - * Sets the value of the 'TestEnum' field. - * @param value The value of 'TestEnum'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setTestEnum(org.apache.beam.sdk.extensions.avro.schemas.TestEnum value) { - validate(fields()[10], value); - this.TestEnum = value; - fieldSetFlags()[10] = true; - return this; - } - - /** - * Checks whether the 'TestEnum' field has been set. - * @return True if the 'TestEnum' field has been set, false otherwise. - */ - public boolean hasTestEnum() { - return fieldSetFlags()[10]; - } - - - /** - * Clears the value of the 'TestEnum' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearTestEnum() { - TestEnum = null; - fieldSetFlags()[10] = false; - return this; - } - - /** - * Gets the value of the 'row' field. - * @return The value. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested getRow() { - return row; - } - - - /** - * Sets the value of the 'row' field. - * @param value The value of 'row'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRow(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested value) { - validate(fields()[11], value); - this.rowBuilder = null; - this.row = value; - fieldSetFlags()[11] = true; - return this; - } - - /** - * Checks whether the 'row' field has been set. - * @return True if the 'row' field has been set, false otherwise. - */ - public boolean hasRow() { - return fieldSetFlags()[11]; - } - - /** - * Gets the Builder instance for the 'row' field and creates one if it doesn't exist yet. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder getRowBuilder() { - if (rowBuilder == null) { - if (hasRow()) { - setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder(row)); - } else { - setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.newBuilder()); - } - } - return rowBuilder; - } - - /** - * Sets the Builder instance for the 'row' field - * @param value The builder instance that must be set. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setRowBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder value) { - clearRow(); - rowBuilder = value; - return this; - } - - /** - * Checks whether the 'row' field has an active Builder instance - * @return True if the 'row' field has an active Builder instance - */ - public boolean hasRowBuilder() { - return rowBuilder != null; - } - - /** - * Clears the value of the 'row' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearRow() { - row = null; - rowBuilder = null; - fieldSetFlags()[11] = false; - return this; - } - - /** - * Gets the value of the 'array' field. - * @return The value. - */ - public java.util.List getArray() { - return array; - } - - - /** - * Sets the value of the 'array' field. - * @param value The value of 'array'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setArray(java.util.List value) { - validate(fields()[12], value); - this.array = value; - fieldSetFlags()[12] = true; - return this; - } - - /** - * Checks whether the 'array' field has been set. - * @return True if the 'array' field has been set, false otherwise. - */ - public boolean hasArray() { - return fieldSetFlags()[12]; - } - - - /** - * Clears the value of the 'array' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearArray() { - array = null; - fieldSetFlags()[12] = false; - return this; - } - - /** - * Gets the value of the 'map' field. - * @return The value. - */ - public java.util.Map getMap() { - return map; - } - - - /** - * Sets the value of the 'map' field. - * @param value The value of 'map'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder setMap(java.util.Map value) { - validate(fields()[13], value); - this.map = value; - fieldSetFlags()[13] = true; - return this; - } - - /** - * Checks whether the 'map' field has been set. - * @return True if the 'map' field has been set, false otherwise. - */ - public boolean hasMap() { - return fieldSetFlags()[13]; - } - - - /** - * Clears the value of the 'map' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvro.Builder clearMap() { - map = null; - fieldSetFlags()[13] = false; - return this; - } - - @Override - @SuppressWarnings("unchecked") - public TestAvro build() { - try { - TestAvro record = new TestAvro(); - record.bool_non_nullable = fieldSetFlags()[0] ? this.bool_non_nullable : (java.lang.Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); - record.long$ = fieldSetFlags()[2] ? this.long$ : (java.lang.Long) defaultValue(fields()[2]); - record.float$ = fieldSetFlags()[3] ? this.float$ : (java.lang.Float) defaultValue(fields()[3]); - record.double$ = fieldSetFlags()[4] ? this.double$ : (java.lang.Double) defaultValue(fields()[4]); - record.string = fieldSetFlags()[5] ? this.string : (java.lang.CharSequence) defaultValue(fields()[5]); - record.bytes = fieldSetFlags()[6] ? this.bytes : (java.nio.ByteBuffer) defaultValue(fields()[6]); - record.fixed = fieldSetFlags()[7] ? this.fixed : (org.apache.beam.sdk.extensions.avro.schemas.fixed4) defaultValue(fields()[7]); - record.date = fieldSetFlags()[8] ? this.date : (java.time.LocalDate) defaultValue(fields()[8]); - record.timestampMillis = fieldSetFlags()[9] ? this.timestampMillis : (java.time.Instant) defaultValue(fields()[9]); - record.TestEnum = fieldSetFlags()[10] ? this.TestEnum : (org.apache.beam.sdk.extensions.avro.schemas.TestEnum) defaultValue(fields()[10]); - if (rowBuilder != null) { - try { - record.row = this.rowBuilder.build(); - } catch (org.apache.avro.AvroMissingFieldException e) { - e.addParentField(record.getSchema().getField("row")); - throw e; - } - } else { - record.row = fieldSetFlags()[11] ? this.row : (org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested) defaultValue(fields()[11]); - } - record.array = fieldSetFlags()[12] ? this.array : (java.util.List) defaultValue(fields()[12]); - record.map = fieldSetFlags()[13] ? this.map : (java.util.Map) defaultValue(fields()[13]); - return record; - } catch (org.apache.avro.AvroMissingFieldException e) { - throw e; - } catch (java.lang.Exception e) { - throw new org.apache.avro.AvroRuntimeException(e); - } - } - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumWriter - WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, SpecificData.getEncoder(out)); - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumReader - READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, SpecificData.getDecoder(in)); - } - -} - - - - - - - - - - diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java deleted file mode 100644 index 4f5818ee51da..000000000000 --- a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestAvroNested.java +++ /dev/null @@ -1,418 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; - -import org.apache.avro.generic.GenericArray; -import org.apache.avro.specific.SpecificData; -import org.apache.avro.util.Utf8; -import org.apache.avro.message.BinaryMessageEncoder; -import org.apache.avro.message.BinaryMessageDecoder; -import org.apache.avro.message.SchemaStore; - -@org.apache.avro.specific.AvroGenerated -public class TestAvroNested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - private static final long serialVersionUID = 4633138088036298925L; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestAvroNested\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"fields\":[{\"name\":\"BOOL_NON_NULLABLE\",\"type\":\"boolean\"},{\"name\":\"int\",\"type\":[\"int\",\"null\"]}]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - - private static SpecificData MODEL$ = new SpecificData(); - - private static final BinaryMessageEncoder ENCODER = - new BinaryMessageEncoder(MODEL$, SCHEMA$); - - private static final BinaryMessageDecoder DECODER = - new BinaryMessageDecoder(MODEL$, SCHEMA$); - - /** - * Return the BinaryMessageEncoder instance used by this class. - * @return the message encoder used by this class - */ - public static BinaryMessageEncoder getEncoder() { - return ENCODER; - } - - /** - * Return the BinaryMessageDecoder instance used by this class. - * @return the message decoder used by this class - */ - public static BinaryMessageDecoder getDecoder() { - return DECODER; - } - - /** - * 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 - */ - public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { - return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); - } - - /** - * Serializes this TestAvroNested to a ByteBuffer. - * @return a buffer holding the serialized data for this instance - * @throws java.io.IOException if this instance could not be serialized - */ - public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { - return ENCODER.encode(this); - } - - /** - * Deserializes a TestAvroNested from a ByteBuffer. - * @param b a byte buffer holding serialized data for an instance of this class - * @return a TestAvroNested instance decoded from the given buffer - * @throws java.io.IOException if the given bytes could not be deserialized into an instance of this class - */ - public static TestAvroNested fromByteBuffer( - java.nio.ByteBuffer b) throws java.io.IOException { - return DECODER.decode(b); - } - - private boolean BOOL_NON_NULLABLE; - private java.lang.Integer int$; - - /** - * Default constructor. Note that this does not initialize fields - * to their default values from the schema. If that is desired then - * one should use newBuilder(). - */ - public TestAvroNested() {} - - /** - * All-args constructor. - * @param BOOL_NON_NULLABLE The new value for BOOL_NON_NULLABLE - * @param int$ The new value for int - */ - public TestAvroNested(java.lang.Boolean BOOL_NON_NULLABLE, java.lang.Integer int$) { - this.BOOL_NON_NULLABLE = BOOL_NON_NULLABLE; - this.int$ = int$; - } - - public org.apache.avro.specific.SpecificData getSpecificData() { return MODEL$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - // Used by DatumWriter. Applications should not call. - public java.lang.Object get(int field$) { - switch (field$) { - case 0: return BOOL_NON_NULLABLE; - case 1: return int$; - default: throw new org.apache.avro.AvroRuntimeException("Bad index"); - } - } - - // Used by DatumReader. Applications should not call. - @SuppressWarnings(value="unchecked") - public void put(int field$, java.lang.Object value$) { - switch (field$) { - case 0: BOOL_NON_NULLABLE = (java.lang.Boolean)value$; break; - case 1: int$ = (java.lang.Integer)value$; break; - default: throw new org.apache.avro.AvroRuntimeException("Bad index"); - } - } - - /** - * Gets the value of the 'BOOL_NON_NULLABLE' field. - * @return The value of the 'BOOL_NON_NULLABLE' field. - */ - public boolean getBOOLNONNULLABLE() { - return BOOL_NON_NULLABLE; - } - - - /** - * Sets the value of the 'BOOL_NON_NULLABLE' field. - * @param value the value to set. - */ - public void setBOOLNONNULLABLE(boolean value) { - this.BOOL_NON_NULLABLE = value; - } - - /** - * Gets the value of the 'int$' field. - * @return The value of the 'int$' field. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value the value to set. - */ - public void setInt$(java.lang.Integer value) { - this.int$ = value; - } - - /** - * Creates a new TestAvroNested RecordBuilder. - * @return A new TestAvroNested RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder() { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); - } - - /** - * Creates a new TestAvroNested RecordBuilder by copying an existing Builder. - * @param other The existing builder to copy. - * @return A new TestAvroNested RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); - } - } - - /** - * Creates a new TestAvroNested RecordBuilder by copying an existing TestAvroNested instance. - * @param other The existing instance to copy. - * @return A new TestAvroNested RecordBuilder - */ - public static org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder newBuilder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { - if (other == null) { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(); - } else { - return new org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder(other); - } - } - - /** - * RecordBuilder for TestAvroNested instances. - */ - @org.apache.avro.specific.AvroGenerated - public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase - implements org.apache.avro.data.RecordBuilder { - - private boolean BOOL_NON_NULLABLE; - private java.lang.Integer int$; - - /** Creates a new Builder */ - private Builder() { - super(SCHEMA$); - } - - /** - * Creates a Builder by copying an existing Builder. - * @param other The existing Builder to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder other) { - super(other); - if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { - this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); - fieldSetFlags()[0] = other.fieldSetFlags()[0]; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = other.fieldSetFlags()[1]; - } - } - - /** - * Creates a Builder by copying an existing TestAvroNested instance - * @param other The existing instance to copy. - */ - private Builder(org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested other) { - super(SCHEMA$); - if (isValidValue(fields()[0], other.BOOL_NON_NULLABLE)) { - this.BOOL_NON_NULLABLE = data().deepCopy(fields()[0].schema(), other.BOOL_NON_NULLABLE); - fieldSetFlags()[0] = true; - } - if (isValidValue(fields()[1], other.int$)) { - this.int$ = data().deepCopy(fields()[1].schema(), other.int$); - fieldSetFlags()[1] = true; - } - } - - /** - * Gets the value of the 'BOOL_NON_NULLABLE' field. - * @return The value. - */ - public boolean getBOOLNONNULLABLE() { - return BOOL_NON_NULLABLE; - } - - - /** - * Sets the value of the 'BOOL_NON_NULLABLE' field. - * @param value The value of 'BOOL_NON_NULLABLE'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setBOOLNONNULLABLE(boolean value) { - validate(fields()[0], value); - this.BOOL_NON_NULLABLE = value; - fieldSetFlags()[0] = true; - return this; - } - - /** - * Checks whether the 'BOOL_NON_NULLABLE' field has been set. - * @return True if the 'BOOL_NON_NULLABLE' field has been set, false otherwise. - */ - public boolean hasBOOLNONNULLABLE() { - return fieldSetFlags()[0]; - } - - - /** - * Clears the value of the 'BOOL_NON_NULLABLE' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearBOOLNONNULLABLE() { - fieldSetFlags()[0] = false; - return this; - } - - /** - * Gets the value of the 'int$' field. - * @return The value. - */ - public java.lang.Integer getInt$() { - return int$; - } - - - /** - * Sets the value of the 'int$' field. - * @param value The value of 'int$'. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder setInt$(java.lang.Integer value) { - validate(fields()[1], value); - this.int$ = value; - fieldSetFlags()[1] = true; - return this; - } - - /** - * Checks whether the 'int$' field has been set. - * @return True if the 'int$' field has been set, false otherwise. - */ - public boolean hasInt$() { - return fieldSetFlags()[1]; - } - - - /** - * Clears the value of the 'int$' field. - * @return This builder. - */ - public org.apache.beam.sdk.extensions.avro.schemas.TestAvroNested.Builder clearInt$() { - int$ = null; - fieldSetFlags()[1] = false; - return this; - } - - @Override - @SuppressWarnings("unchecked") - public TestAvroNested build() { - try { - TestAvroNested record = new TestAvroNested(); - record.BOOL_NON_NULLABLE = fieldSetFlags()[0] ? this.BOOL_NON_NULLABLE : (java.lang.Boolean) defaultValue(fields()[0]); - record.int$ = fieldSetFlags()[1] ? this.int$ : (java.lang.Integer) defaultValue(fields()[1]); - return record; - } catch (org.apache.avro.AvroMissingFieldException e) { - throw e; - } catch (java.lang.Exception e) { - throw new org.apache.avro.AvroRuntimeException(e); - } - } - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumWriter - WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, SpecificData.getEncoder(out)); - } - - @SuppressWarnings("unchecked") - private static final org.apache.avro.io.DatumReader - READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, SpecificData.getDecoder(in)); - } - - @Override protected boolean hasCustomCoders() { return true; } - - @Override public void customEncode(org.apache.avro.io.Encoder out) - throws java.io.IOException - { - out.writeBoolean(this.BOOL_NON_NULLABLE); - - if (this.int$ == null) { - out.writeIndex(1); - out.writeNull(); - } else { - out.writeIndex(0); - out.writeInt(this.int$); - } - - } - - @Override public void customDecode(org.apache.avro.io.ResolvingDecoder in) - throws java.io.IOException - { - org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff(); - if (fieldOrder == null) { - this.BOOL_NON_NULLABLE = in.readBoolean(); - - if (in.readIndex() != 0) { - in.readNull(); - this.int$ = null; - } else { - this.int$ = in.readInt(); - } - - } else { - for (int i = 0; i < 2; i++) { - switch (fieldOrder[i].pos()) { - case 0: - this.BOOL_NON_NULLABLE = in.readBoolean(); - break; - - case 1: - if (in.readIndex() != 0) { - in.readNull(); - this.int$ = null; - } else { - this.int$ = in.readInt(); - } - break; - - default: - throw new java.io.IOException("Corrupt ResolvingDecoder."); - } - } - } - } -} - - - - - - - - - - diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java deleted file mode 100644 index 8d1e1d7dffe9..000000000000 --- a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/TestEnum.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; -@org.apache.avro.specific.AvroGenerated -public enum TestEnum implements org.apache.avro.generic.GenericEnumSymbol { - abc, cde ; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"enum\",\"name\":\"TestEnum\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"symbols\":[\"abc\",\"cde\"]}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } -} diff --git a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java b/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java deleted file mode 100644 index 24c5be9ca18d..000000000000 --- a/sdks/java/extensions/avro/src/test/avro192/org/apache/beam/sdk/extensions/avro/schemas/fixed4.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.beam.sdk.extensions.avro.schemas; -@org.apache.avro.specific.FixedSize(4) -@org.apache.avro.specific.AvroGenerated -public class fixed4 extends org.apache.avro.specific.SpecificFixed { - private static final long serialVersionUID = -5646354132642432749L; - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"fixed\",\"name\":\"fixed4\",\"namespace\":\"org.apache.beam.sdk.extensions.avro.schemas\",\"size\":4}"); - public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - public org.apache.avro.Schema getSchema() { return SCHEMA$; } - - /** Creates a new fixed4 */ - public fixed4() { - super(); - } - - /** - * Creates a new fixed4 with the given bytes. - * @param bytes The bytes to create the new fixed4. - */ - public fixed4(byte[] bytes) { - super(bytes); - } - - private static final org.apache.avro.io.DatumWriter - WRITER$ = new org.apache.avro.specific.SpecificDatumWriter(SCHEMA$); - - @Override public void writeExternal(java.io.ObjectOutput out) - throws java.io.IOException { - WRITER$.write(this, org.apache.avro.specific.SpecificData.getEncoder(out)); - } - - private static final org.apache.avro.io.DatumReader - READER$ = new org.apache.avro.specific.SpecificDatumReader(SCHEMA$); - - @Override public void readExternal(java.io.ObjectInput in) - throws java.io.IOException { - READER$.read(this, org.apache.avro.specific.SpecificData.getDecoder(in)); - } - -} diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java index 3387f7bff8c5..dd549f7f2435 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/TestAvroFactory.java @@ -34,10 +34,10 @@ public class TestAvroFactory { public static TestAvro newInstance( Boolean boolNonNullable, - Integer int$, - Long long$, - Float float$, - Double double$, + Integer integer, + Long aLong, + Float aFloat, + Double aDouble, String string, ByteBuffer bytes, fixed4 fixed, @@ -51,10 +51,10 @@ public static TestAvro newInstance( if (VERSION_AVRO.equals("1.8.2")) { return new TestAvro( boolNonNullable, - int$, - long$, - float$, - double$, + integer, + aLong, + aFloat, + aDouble, string, bytes, fixed, @@ -87,10 +87,10 @@ public static TestAvro newInstance( return (TestAvro) constructor.newInstance( boolNonNullable, - int$, - long$, - float$, - double$, + integer, + aLong, + aFloat, + aDouble, string, bytes, fixed,