RyanSkraba commented on code in PR #23395: URL: https://github.com/apache/flink/pull/23395#discussion_r1368636472
########## docs/content/docs/connectors/table/formats/avro.md: ########## @@ -80,6 +80,14 @@ Format Options <td>String</td> <td>Specify what format to use, here should be <code>'avro'</code>.</td> </tr> + <tr> + <td><h5>avro.encoding</h5></td> + <td>optional</td> + <td>yes</td> + <td>binary</td> + <td>String</td> + <td>Serialization encoding to use. The valid enumerations are: binary, json. <a href="https://avro.apache.org/docs/1.11.1/specification/#encodings">(reference)</a></td> Review Comment: This is a bit irritating, since the current version is 1.11.3, but there's an outstanding issue preventing the Avro website from being published :/ I suggest using current/ since the valid encodings are unlikely to change in the near future. ########## flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataDeserializationSchema.java: ########## @@ -66,7 +67,25 @@ public class AvroRowDataDeserializationSchema implements DeserializationSchema<R */ public AvroRowDataDeserializationSchema(RowType rowType, TypeInformation<RowData> typeInfo) { this( - AvroDeserializationSchema.forGeneric(AvroSchemaConverter.convertToSchema(rowType)), + AvroDeserializationSchema.forGeneric( Review Comment: This is extremely minor -- but you might want to call the other constructor(s) from this one: ``` this(rowType, typeInfo, AvroEncoding.BINARY); ``` This might just be a personal preference to avoid wondering if there's any _other_ difference between the two constructors while reading the code. Also applies to `AvroRowDataSerializationSchema`. ########## flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowDataDeSerializationSchemaTest.java: ########## @@ -222,12 +234,14 @@ void testSerializeDeserializeBasedOnNestedSchema() throws Exception { AvroRowDataSerializationSchema serializationSchema = new AvroRowDataSerializationSchema( rowType, - AvroSerializationSchema.forGeneric(nullableOuterSchema), + AvroSerializationSchema.forGeneric( Review Comment: With the public BINARY-default methods maintained, you might want to revert these. This test doesn't work with JSON encoding? ########## docs/content/docs/connectors/table/formats/avro.md: ########## @@ -80,6 +80,14 @@ Format Options <td>String</td> <td>Specify what format to use, here should be <code>'avro'</code>.</td> </tr> + <tr> + <td><h5>avro.encoding</h5></td> + <td>optional</td> + <td>yes</td> + <td>binary</td> + <td>String</td> + <td>Serialization encoding to use. The valid enumerations are: binary, json. <a href="https://avro.apache.org/docs/1.11.1/specification/#encodings">(reference)</a></td> Review Comment: ```suggestion <td>Serialization encoding to use. The valid enumerations are: binary, json. <a href="https://avro.apache.org/docs/current/specification/#encodings">(reference)</a></td> ``` ########## flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroSerializationSchema.java: ########## @@ -54,8 +55,8 @@ public class AvroSerializationSchema<T> implements SerializationSchema<T> { * @return serialized record in form of byte array */ public static <T extends SpecificRecord> AvroSerializationSchema<T> forSpecific( Review Comment: Some more public API. Also applies to `forGeneric` in this class. ########## flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroDeserializationSchemaTest.java: ########## @@ -37,46 +38,56 @@ class AvroDeserializationSchemaTest { private static final Address address = TestDataGenerator.generateRandomAddress(new Random()); + private static final AvroEncoding[] ENCODINGS = {AvroEncoding.BINARY, AvroEncoding.JSON}; + @Test void testNullRecord() throws Exception { Review Comment: Just a suggestion -- almost all of these tests could be run as JUnit parameterized tests. For example: ``` @ParameterizedTest @EnumSource(AvroEncoding.class) void testNullRecord(AvroEncoding encoding) throws Exception { DeserializationSchema<Address> deserializer = AvroDeserializationSchema.forSpecific(Address.class, encoding); Address deserializedAddress = deserializer.deserialize(null); assertThat(deserializedAddress).isNull(); } ``` This changes little when everything is working, but it adds additional context to the unit test when *one* of the AvroEncoding types fails. -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org