mosche commented on code in PR #26320:
URL: https://github.com/apache/beam/pull/26320#discussion_r1205583401
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java:
##########
@@ -116,56 +106,40 @@
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
-public class AvroCoder<T> extends CustomCoder<T> {
+public abstract class AvroCoder<T> extends CustomCoder<T> {
/**
* Returns an {@code AvroCoder} instance for the provided element type.
*
* @param <T> the element type
*/
public static <T> AvroCoder<T> of(TypeDescriptor<T> type) {
- return of(type, true);
- }
-
- /**
- * Returns an {@code AvroCoder} instance for the provided element type,
respecting whether to use
- * Avro's Reflect* or Specific* suite for encoding and decoding.
- *
- * @param <T> the element type
- */
- public static <T> AvroCoder<T> of(TypeDescriptor<T> type, boolean
useReflectApi) {
@SuppressWarnings("unchecked")
Class<T> clazz = (Class<T>) type.getRawType();
- return of(clazz, useReflectApi);
- }
-
- /**
- * Returns an {@code AvroCoder} instance for the provided element class.
- *
- * @param <T> the element type
- */
- public static <T> AvroCoder<T> of(Class<T> clazz) {
- return of(clazz, true);
+ return of(clazz);
}
/**
* Returns an {@code AvroGenericCoder} instance for the Avro schema. The
implicit type is
* GenericRecord.
*/
- public static AvroGenericCoder of(Schema schema) {
+ public static AvroCoder<GenericRecord> of(Schema schema) {
return AvroGenericCoder.of(schema);
}
/**
- * Returns an {@code AvroCoder} instance for the given class, respecting
whether to use Avro's
- * Reflect* or Specific* suite for encoding and decoding.
+ * Returns an {@code AvroCoder} instance for the provided element class.
*
* @param <T> the element type
*/
- public static <T> AvroCoder<T> of(Class<T> type, boolean useReflectApi) {
Review Comment:
Same as above, please keep this for backwards compatibility.
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java:
##########
@@ -116,56 +106,40 @@
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
-public class AvroCoder<T> extends CustomCoder<T> {
+public abstract class AvroCoder<T> extends CustomCoder<T> {
/**
* Returns an {@code AvroCoder} instance for the provided element type.
*
* @param <T> the element type
*/
public static <T> AvroCoder<T> of(TypeDescriptor<T> type) {
- return of(type, true);
- }
-
- /**
- * Returns an {@code AvroCoder} instance for the provided element type,
respecting whether to use
- * Avro's Reflect* or Specific* suite for encoding and decoding.
- *
- * @param <T> the element type
- */
- public static <T> AvroCoder<T> of(TypeDescriptor<T> type, boolean
useReflectApi) {
Review Comment:
Please keep the API backwards compatible. If there's reasons not to use this
factory method any longer, please deprecate it and explain what to use instead.
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java:
##########
@@ -116,56 +106,40 @@
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
-public class AvroCoder<T> extends CustomCoder<T> {
+public abstract class AvroCoder<T> extends CustomCoder<T> {
/**
* Returns an {@code AvroCoder} instance for the provided element type.
*
* @param <T> the element type
*/
public static <T> AvroCoder<T> of(TypeDescriptor<T> type) {
- return of(type, true);
- }
-
- /**
- * Returns an {@code AvroCoder} instance for the provided element type,
respecting whether to use
- * Avro's Reflect* or Specific* suite for encoding and decoding.
- *
- * @param <T> the element type
- */
- public static <T> AvroCoder<T> of(TypeDescriptor<T> type, boolean
useReflectApi) {
@SuppressWarnings("unchecked")
Class<T> clazz = (Class<T>) type.getRawType();
- return of(clazz, useReflectApi);
- }
-
- /**
- * Returns an {@code AvroCoder} instance for the provided element class.
- *
- * @param <T> the element type
- */
- public static <T> AvroCoder<T> of(Class<T> clazz) {
- return of(clazz, true);
+ return of(clazz);
}
/**
* Returns an {@code AvroGenericCoder} instance for the Avro schema. The
implicit type is
* GenericRecord.
*/
- public static AvroGenericCoder of(Schema schema) {
+ public static AvroCoder<GenericRecord> of(Schema schema) {
Review Comment:
Please keep the return type unchanged to maintain backwards compatibility.
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java:
##########
@@ -177,17 +151,13 @@ public static <T> AvroCoder<T> of(Class<T> type, boolean
useReflectApi) {
* @param <T> the element type
*/
public static <T> AvroCoder<T> of(Class<T> type, Schema schema) {
- return of(type, schema, true);
- }
-
- /**
- * Returns an {@code AvroCoder} instance for the given class and schema,
respecting whether to use
- * Avro's Reflect* or Specific* suite for encoding and decoding.
- *
- * @param <T> the element type
- */
- public static <T> AvroCoder<T> of(Class<T> type, Schema schema, boolean
useReflectApi) {
Review Comment:
Same as above, please keep this for backwards compatibility.
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java:
##########
@@ -365,10 +296,6 @@ public Class<T> getType() {
return type;
}
- public boolean useReflectApi() {
Review Comment:
Please deprecate instead
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java:
##########
@@ -768,53 +695,22 @@ private static Field getField(Class<?> originalClazz,
String name) {
}
}
+ /**
+ * @return {@code true} if the two {@link AvroCoder} instances have the same
class, type and
+ * schema.
+ */
@Override
public boolean equals(@Nullable Object other) {
Review Comment:
Please also test `datumFactory` here (and implement their equals/hashCode
accordingly), same applies to hashCode of course.
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java:
##########
@@ -768,53 +695,22 @@ private static Field getField(Class<?> originalClazz,
String name) {
}
}
+ /**
+ * @return {@code true} if the two {@link AvroCoder} instances have the same
class, type and
+ * schema.
+ */
@Override
public boolean equals(@Nullable Object other) {
- if (other == this) {
- return true;
- }
- if (!(other instanceof AvroCoder)) {
+ if (other == null || this.getClass() != other.getClass()) {
return false;
}
AvroCoder<?> that = (AvroCoder<?>) other;
return Objects.equals(this.schemaSupplier.get(), that.schemaSupplier.get())
- && Objects.equals(this.typeDescriptor, that.typeDescriptor)
- && this.useReflectApi == that.useReflectApi;
+ && Objects.equals(this.typeDescriptor, that.typeDescriptor);
}
@Override
public int hashCode() {
- return Objects.hash(schemaSupplier.get(), typeDescriptor, useReflectApi);
- }
-
- /**
- * Conversion for DateTime.
- *
- * <p>This is a copy from Avro 1.8's TimestampConversion, which is renamed
in Avro 1.9. Defining
- * own copy gives flexibility for Beam Java SDK to work with Avro 1.8 and
1.9 at runtime.
- *
- * @see <a href="https://issues.apache.org/jira/browse/BEAM-9144">BEAM-9144:
Beam's own Avro
- * TimeConversion class in beam-sdk-java-core</a>
- */
- public static class JodaTimestampConversion extends Conversion<DateTime> {
Review Comment:
This is critical as it breaks users using Joda types in their Pojos. To keep
the behavior consistent this logical conversion has to be added to the reflect
coder, if not to all three Avro coder / datum factory types (still wondering
myself).
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoder.java:
##########
@@ -306,41 +256,27 @@ public ReflectData get() {
private final EmptyOnDeserializationThreadLocal<DatumWriter<T>> writer;
private final EmptyOnDeserializationThreadLocal<DatumReader<T>> reader;
- // Lazily re-instantiated after deserialization
- private final Supplier<ReflectData> reflectData;
-
- protected AvroCoder(Class<T> type, Schema schema) {
- this(type, schema, false);
- }
-
- protected AvroCoder(Class<T> type, Schema schema, boolean useReflectApi) {
+ protected AvroCoder(Class<T> type, AvroDatumFactory<T> datumFactory, Schema
schema) {
Review Comment:
The old constructor was accessible by subclasses, so it would be better not
to remove it:
```java
protected AvroCoder(Class<T> type, Schema schema, boolean useReflectApi) {
this(type, useReflectApi ? AvroDatumFactory.ReflectDatumFactory.of(type)
: AvroDatumFactory.SpecificDatumFactory.of(type), schema);
}
```
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/coders/AvroReflectCoder.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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 org.apache.avro.Schema;
+import org.apache.avro.reflect.Nullable;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.reflect.Union;
+import org.apache.beam.sdk.extensions.avro.io.AvroDatumFactory;
+
+/**
+ * AvroCoder specialisation for avro classes using Java reflection.
+ *
+ * <p>Only concrete classes with a no-argument constructor can be mapped to
Avro records. All
+ * inherited fields that are not static or transient are included. Fields are
not permitted to be
+ * null unless annotated by {@link Nullable} or a {@link Union} schema
containing {@code "null"}.
+ */
+public class AvroReflectCoder<T> extends AvroCoder<T> {
Review Comment:
Why subclassing AvroCoder if the only difference is datum factory? I'd
suggest to better keep the old API (using the `useReflectApi` flag) to not
increase the public surface for no obvious gain. Same for AvroSpecificCoder...
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]