reuvenlax commented on code in PR #32081:
URL: https://github.com/apache/beam/pull/32081#discussion_r1811391760
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java:
##########
@@ -126,17 +123,26 @@ public static FieldValueTypeInformation forOneOf(
}
public static FieldValueTypeInformation forField(Field field, int index) {
- TypeDescriptor<?> type = TypeDescriptor.of(field.getGenericType());
+ return forField(null, field, index);
+ }
Review Comment:
This class wasn't really meant to be public. I would add an @Internal tag to
the class, and not bother keeping this backwards-compatible method
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/GetterBasedSchemaProvider.java:
##########
@@ -112,22 +111,28 @@ public SchemaUserTypeCreator schemaTypeCreator(
return schemaTypeCreator(targetTypeDescriptor.getRawType(), schema);
}
- private class ToRowWithValueGetters<T> implements SerializableFunction<T,
Row> {
+ private class ToRowWithValueGetters<T extends @NonNull Object>
+ implements SerializableFunction<T, Row> {
private final Schema schema;
- private final Factory<List<FieldValueGetter>> getterFactory;
+ private final Factory<List<FieldValueGetter<?, ?>>> getterFactory;
public ToRowWithValueGetters(Schema schema) {
this.schema = schema;
// Since we know that this factory is always called from inside the
lambda with the same
// schema, return a caching factory that caches the first value seen for
each class. This
// prevents having to lookup the getter list each time createGetters is
called.
this.getterFactory =
-
RowValueGettersFactory.of(GetterBasedSchemaProvider.this::fieldValueGetters);
+ RowValueGettersFactory.of(
+ (Factory<List<FieldValueGetter<?, ?>>>)
+ (typeDescriptor, schema1) ->
+ (List)
+ GetterBasedSchemaProvider.this.fieldValueGetters(
+ typeDescriptor, schema1));
}
@Override
public Row apply(T input) {
- return Row.withSchema(schema).withFieldValueGetters(getterFactory,
input);
+ return Row.withSchema(schema).withFieldValueGetters((Factory)
getterFactory, input);
Review Comment:
why do you need this cast?
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java:
##########
@@ -42,45 +42,43 @@
* the appropriate fields from the POJO.
*/
@SuppressWarnings("rawtypes")
-public class RowWithGetters extends Row {
- private final Object getterTarget;
- private final List<FieldValueGetter> getters;
+public class RowWithGetters<T extends @NonNull Object> extends Row {
+ private final T getterTarget;
+ private final List<FieldValueGetter<T, ?>> getters;
private @Nullable Map<Integer, @Nullable Object> cache = null;
RowWithGetters(
- Schema schema, Factory<List<FieldValueGetter>> getterFactory, Object
getterTarget) {
+ Schema schema, Factory<List<FieldValueGetter<T, ?>>> getterFactory, T
getterTarget) {
super(schema);
this.getterTarget = getterTarget;
this.getters =
getterFactory.create(TypeDescriptor.of(getterTarget.getClass()), schema);
}
@Override
@SuppressWarnings({"TypeParameterUnusedInFormals", "unchecked"})
- public <T> @Nullable T getValue(int fieldIdx) {
+ public @Nullable Object getValue(int fieldIdx) {
Review Comment:
why remove type parameter?
##########
sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java:
##########
@@ -486,10 +518,17 @@ public static Schema toBeamSchema(org.apache.avro.Schema
schema) {
return builder.build();
}
+ @EnsuresNonNullIf(
+ expression = {"#1"},
+ result = false)
+ private static boolean isNullOrEmpty(@Nullable String str) {
+ return str == null || str.isEmpty();
+ }
Review Comment:
what is wrong with Strings.isNullOrEmpty?
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java:
##########
@@ -836,10 +838,10 @@ public int nextFieldId() {
}
@Internal
- public Row withFieldValueGetters(
- Factory<List<FieldValueGetter>> fieldValueGetterFactory, Object
getterTarget) {
+ public <T extends @NonNull Object> Row withFieldValueGetters(
Review Comment:
unclear on this bound as well
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/CachingFactory.java:
##########
@@ -39,9 +41,9 @@
public class CachingFactory<CreatedT> implements Factory<CreatedT> {
private transient @Nullable ConcurrentHashMap<TypeDescriptor<?>, CreatedT>
cache = null;
- private final Factory<CreatedT> innerFactory;
+ private final @NotOnlyInitialized Factory<CreatedT> innerFactory;
Review Comment:
what are these new tags for?
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueGetter.java:
##########
@@ -29,7 +30,7 @@
* <p>Implementations of this interface are generated at runtime to map object
fields to Row fields.
*/
@Internal
-public interface FieldValueGetter<ObjectT, ValueT> extends Serializable {
+public interface FieldValueGetter<ObjectT extends @NonNull Object, ValueT>
extends Serializable {
Review Comment:
why do you have ObjectT extends Object?
--
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]