Skip to content

Commit dd556b2

Browse files
authored
Merge pull request #32081: make FieldValueTypeInformation creators take a TypeDescriptor parameter
1 parent 2a27cc6 commit dd556b2

File tree

28 files changed

+827
-531
lines changed

28 files changed

+827
-531
lines changed

sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java

Lines changed: 10 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@
1919

2020
import java.lang.reflect.Method;
2121
import java.lang.reflect.Modifier;
22-
import java.util.Comparator;
2322
import java.util.List;
2423
import java.util.stream.Collectors;
2524
import org.apache.beam.sdk.schemas.annotations.SchemaIgnore;
@@ -32,13 +31,10 @@
3231
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
3332
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
3433
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
34+
import org.checkerframework.checker.nullness.qual.NonNull;
3535
import org.checkerframework.checker.nullness.qual.Nullable;
3636

3737
/** A {@link SchemaProvider} for AutoValue classes. */
38-
@SuppressWarnings({
39-
"nullness", // TODO(https://github.com/apache/beam/issues/20497)
40-
"rawtypes"
41-
})
4238
public class AutoValueSchema extends GetterBasedSchemaProviderV2 {
4339
/** {@link FieldValueTypeSupplier} that's based on AutoValue getters. */
4440
@VisibleForTesting
@@ -49,7 +45,11 @@ public static class AbstractGetterTypeSupplier implements FieldValueTypeSupplier
4945
public List<FieldValueTypeInformation> get(TypeDescriptor<?> typeDescriptor) {
5046

5147
// If the generated class is passed in, we want to look at the base class to find the getters.
52-
TypeDescriptor<?> targetTypeDescriptor = AutoValueUtils.getBaseAutoValueClass(typeDescriptor);
48+
TypeDescriptor<?> targetTypeDescriptor =
49+
Preconditions.checkNotNull(
50+
AutoValueUtils.getBaseAutoValueClass(typeDescriptor),
51+
"unable to determine base AutoValue class for type {}",
52+
typeDescriptor);
5353

5454
List<Method> methods =
5555
ReflectUtils.getMethods(targetTypeDescriptor.getRawType()).stream()
@@ -62,9 +62,9 @@ public List<FieldValueTypeInformation> get(TypeDescriptor<?> typeDescriptor) {
6262
.collect(Collectors.toList());
6363
List<FieldValueTypeInformation> types = Lists.newArrayListWithCapacity(methods.size());
6464
for (int i = 0; i < methods.size(); ++i) {
65-
types.add(FieldValueTypeInformation.forGetter(methods.get(i), i));
65+
types.add(FieldValueTypeInformation.forGetter(typeDescriptor, methods.get(i), i));
6666
}
67-
types.sort(Comparator.comparing(FieldValueTypeInformation::getNumber));
67+
types.sort(JavaBeanUtils.comparingNullFirst(FieldValueTypeInformation::getNumber));
6868
validateFieldNumbers(types);
6969
return types;
7070
}
@@ -89,8 +89,8 @@ private static void validateFieldNumbers(List<FieldValueTypeInformation> types)
8989
}
9090

9191
@Override
92-
public List<FieldValueGetter> fieldValueGetters(
93-
TypeDescriptor<?> targetTypeDescriptor, Schema schema) {
92+
public <T> List<FieldValueGetter<@NonNull T, Object>> fieldValueGetters(
93+
TypeDescriptor<T> targetTypeDescriptor, Schema schema) {
9494
return JavaBeanUtils.getGetters(
9595
targetTypeDescriptor,
9696
schema,

sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/CachingFactory.java

Lines changed: 13 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,9 @@
2020
import java.util.Objects;
2121
import java.util.concurrent.ConcurrentHashMap;
2222
import org.apache.beam.sdk.values.TypeDescriptor;
23+
import org.checkerframework.checker.initialization.qual.NotOnlyInitialized;
24+
import org.checkerframework.checker.initialization.qual.UnknownInitialization;
25+
import org.checkerframework.checker.nullness.qual.NonNull;
2326
import org.checkerframework.checker.nullness.qual.Nullable;
2427

2528
/**
@@ -32,24 +35,25 @@
3235
* significant for larger schemas) on each lookup. This wrapper caches the value returned by the
3336
* inner factory, so the schema comparison only need happen on the first lookup.
3437
*/
35-
@SuppressWarnings({
36-
"nullness", // TODO(https://github.com/apache/beam/issues/20497)
37-
"rawtypes"
38-
})
39-
public class CachingFactory<CreatedT> implements Factory<CreatedT> {
38+
public class CachingFactory<CreatedT extends @NonNull Object> implements Factory<CreatedT> {
4039
private transient @Nullable ConcurrentHashMap<TypeDescriptor<?>, CreatedT> cache = null;
4140

42-
private final Factory<CreatedT> innerFactory;
41+
private final @NotOnlyInitialized Factory<CreatedT> innerFactory;
4342

44-
public CachingFactory(Factory<CreatedT> innerFactory) {
43+
public CachingFactory(@UnknownInitialization Factory<CreatedT> innerFactory) {
4544
this.innerFactory = innerFactory;
4645
}
4746

48-
@Override
49-
public CreatedT create(TypeDescriptor<?> typeDescriptor, Schema schema) {
47+
private ConcurrentHashMap<TypeDescriptor<?>, CreatedT> getCache() {
5048
if (cache == null) {
5149
cache = new ConcurrentHashMap<>();
5250
}
51+
return cache;
52+
}
53+
54+
@Override
55+
public CreatedT create(TypeDescriptor<?> typeDescriptor, Schema schema) {
56+
ConcurrentHashMap<TypeDescriptor<?>, CreatedT> cache = getCache();
5357
CreatedT cached = cache.get(typeDescriptor);
5458
if (cached != null) {
5559
return cached;

sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueGetter.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919

2020
import java.io.Serializable;
2121
import org.apache.beam.sdk.annotations.Internal;
22+
import org.checkerframework.checker.nullness.qual.NonNull;
2223
import org.checkerframework.checker.nullness.qual.Nullable;
2324

2425
/**
@@ -29,7 +30,7 @@
2930
* <p>Implementations of this interface are generated at runtime to map object fields to Row fields.
3031
*/
3132
@Internal
32-
public interface FieldValueGetter<ObjectT, ValueT> extends Serializable {
33+
public interface FieldValueGetter<ObjectT extends @NonNull Object, ValueT> extends Serializable {
3334
@Nullable
3435
ValueT get(ObjectT object);
3536

sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java

Lines changed: 43 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,9 @@
2727
import java.util.Arrays;
2828
import java.util.Collections;
2929
import java.util.Map;
30+
import java.util.Optional;
3031
import java.util.stream.Stream;
32+
import org.apache.beam.sdk.annotations.Internal;
3133
import org.apache.beam.sdk.schemas.annotations.SchemaCaseFormat;
3234
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
3335
import org.apache.beam.sdk.schemas.annotations.SchemaFieldName;
@@ -40,10 +42,7 @@
4042

4143
/** Represents type information for a Java type that will be used to infer a Schema type. */
4244
@AutoValue
43-
@SuppressWarnings({
44-
"nullness", // TODO(https://github.com/apache/beam/issues/20497)
45-
"rawtypes"
46-
})
45+
@Internal
4746
public abstract class FieldValueTypeInformation implements Serializable {
4847
/** Optionally returns the field index. */
4948
public abstract @Nullable Integer getNumber();
@@ -125,18 +124,23 @@ public static FieldValueTypeInformation forOneOf(
125124
.build();
126125
}
127126

128-
public static FieldValueTypeInformation forField(Field field, int index) {
129-
TypeDescriptor<?> type = TypeDescriptor.of(field.getGenericType());
127+
public static FieldValueTypeInformation forField(
128+
@Nullable TypeDescriptor<?> typeDescriptor, Field field, int index) {
129+
TypeDescriptor<?> type =
130+
Optional.ofNullable(typeDescriptor)
131+
.map(td -> (TypeDescriptor) td.resolveType(field.getGenericType()))
132+
// fall back to previous behavior
133+
.orElseGet(() -> TypeDescriptor.of(field.getGenericType()));
130134
return new AutoValue_FieldValueTypeInformation.Builder()
131135
.setName(getNameOverride(field.getName(), field))
132136
.setNumber(getNumberOverride(index, field))
133137
.setNullable(hasNullableAnnotation(field))
134138
.setType(type)
135139
.setRawType(type.getRawType())
136140
.setField(field)
137-
.setElementType(getIterableComponentType(field))
138-
.setMapKeyType(getMapKeyType(field))
139-
.setMapValueType(getMapValueType(field))
141+
.setElementType(getIterableComponentType(type))
142+
.setMapKeyType(getMapKeyType(type))
143+
.setMapValueType(getMapValueType(type))
140144
.setOneOfTypes(Collections.emptyMap())
141145
.setDescription(getFieldDescription(field))
142146
.build();
@@ -185,6 +189,11 @@ public static <T extends AnnotatedElement & Member> String getNameOverride(
185189
}
186190

187191
public static FieldValueTypeInformation forGetter(Method method, int index) {
192+
return forGetter(null, method, index);
193+
}
194+
195+
public static FieldValueTypeInformation forGetter(
196+
@Nullable TypeDescriptor<?> typeDescriptor, Method method, int index) {
188197
String name;
189198
if (method.getName().startsWith("get")) {
190199
name = ReflectUtils.stripPrefix(method.getName(), "get");
@@ -194,7 +203,12 @@ public static FieldValueTypeInformation forGetter(Method method, int index) {
194203
throw new RuntimeException("Getter has wrong prefix " + method.getName());
195204
}
196205

197-
TypeDescriptor<?> type = TypeDescriptor.of(method.getGenericReturnType());
206+
TypeDescriptor<?> type =
207+
Optional.ofNullable(typeDescriptor)
208+
.map(td -> (TypeDescriptor) td.resolveType(method.getGenericReturnType()))
209+
// fall back to previous behavior
210+
.orElseGet(() -> TypeDescriptor.of(method.getGenericReturnType()));
211+
198212
boolean nullable = hasNullableReturnType(method);
199213
return new AutoValue_FieldValueTypeInformation.Builder()
200214
.setName(getNameOverride(name, method))
@@ -253,18 +267,32 @@ private static boolean isNullableAnnotation(Annotation annotation) {
253267
}
254268

255269
public static FieldValueTypeInformation forSetter(Method method) {
256-
return forSetter(method, "set");
270+
return forSetter(null, method);
257271
}
258272

259273
public static FieldValueTypeInformation forSetter(Method method, String setterPrefix) {
274+
return forSetter(null, method, setterPrefix);
275+
}
276+
277+
public static FieldValueTypeInformation forSetter(
278+
@Nullable TypeDescriptor<?> typeDescriptor, Method method) {
279+
return forSetter(typeDescriptor, method, "set");
280+
}
281+
282+
public static FieldValueTypeInformation forSetter(
283+
@Nullable TypeDescriptor<?> typeDescriptor, Method method, String setterPrefix) {
260284
String name;
261285
if (method.getName().startsWith(setterPrefix)) {
262286
name = ReflectUtils.stripPrefix(method.getName(), setterPrefix);
263287
} else {
264288
throw new RuntimeException("Setter has wrong prefix " + method.getName());
265289
}
266290

267-
TypeDescriptor<?> type = TypeDescriptor.of(method.getGenericParameterTypes()[0]);
291+
TypeDescriptor<?> type =
292+
Optional.ofNullable(typeDescriptor)
293+
.map(td -> (TypeDescriptor) td.resolveType(method.getGenericParameterTypes()[0]))
294+
// fall back to previous behavior
295+
.orElseGet(() -> TypeDescriptor.of(method.getGenericParameterTypes()[0]));
268296
boolean nullable = hasSingleNullableParameter(method);
269297
return new AutoValue_FieldValueTypeInformation.Builder()
270298
.setName(name)
@@ -283,10 +311,6 @@ public FieldValueTypeInformation withName(String name) {
283311
return toBuilder().setName(name).build();
284312
}
285313

286-
private static FieldValueTypeInformation getIterableComponentType(Field field) {
287-
return getIterableComponentType(TypeDescriptor.of(field.getGenericType()));
288-
}
289-
290314
static @Nullable FieldValueTypeInformation getIterableComponentType(TypeDescriptor<?> valueType) {
291315
// TODO: Figure out nullable elements.
292316
TypeDescriptor<?> componentType = ReflectUtils.getIterableComponentType(valueType);
@@ -306,34 +330,23 @@ private static FieldValueTypeInformation getIterableComponentType(Field field) {
306330
.build();
307331
}
308332

309-
// If the Field is a map type, returns the key type, otherwise returns a null reference.
310-
311-
private static @Nullable FieldValueTypeInformation getMapKeyType(Field field) {
312-
return getMapKeyType(TypeDescriptor.of(field.getGenericType()));
313-
}
314-
333+
// If the type is a map type, returns the key type, otherwise returns a null reference.
315334
private static @Nullable FieldValueTypeInformation getMapKeyType(
316335
TypeDescriptor<?> typeDescriptor) {
317336
return getMapType(typeDescriptor, 0);
318337
}
319338

320-
// If the Field is a map type, returns the value type, otherwise returns a null reference.
321-
322-
private static @Nullable FieldValueTypeInformation getMapValueType(Field field) {
323-
return getMapType(TypeDescriptor.of(field.getGenericType()), 1);
324-
}
325-
339+
// If the type is a map type, returns the value type, otherwise returns a null reference.
326340
private static @Nullable FieldValueTypeInformation getMapValueType(
327341
TypeDescriptor<?> typeDescriptor) {
328342
return getMapType(typeDescriptor, 1);
329343
}
330344

331345
// If the Field is a map type, returns the key or value type (0 is key type, 1 is value).
332346
// Otherwise returns a null reference.
333-
@SuppressWarnings("unchecked")
334347
private static @Nullable FieldValueTypeInformation getMapType(
335348
TypeDescriptor<?> valueType, int index) {
336-
TypeDescriptor mapType = ReflectUtils.getMapType(valueType, index);
349+
TypeDescriptor<?> mapType = ReflectUtils.getMapType(valueType, index);
337350
if (mapType == null) {
338351
return null;
339352
}

0 commit comments

Comments
 (0)