-
Notifications
You must be signed in to change notification settings - Fork 4.5k
make FieldValueTypeInformation creators take a TypeDescriptor parameter #32081
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
ce83352
d2e3478
ce5ff31
70be685
6ef957a
0e7018d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
|
|
||
| import java.io.Serializable; | ||
| import org.apache.beam.sdk.annotations.Internal; | ||
| import org.checkerframework.checker.nullness.qual.NonNull; | ||
| import org.checkerframework.checker.nullness.qual.Nullable; | ||
|
|
||
| /** | ||
|
|
@@ -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 { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why do you have ObjectT extends Object?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. so apparently there's a difference (I don't understand why it would be so, though) between a with a signature as above, we can only write: or alternatively: The checkerframework will make sure that we propagate the non-nullness of the parameter. nor will this: but it will work if I change the signature of in other words, the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yeah, at least not when we're talking type parameters, for example: gives me |
||
| @Nullable | ||
| ValueT get(ObjectT object); | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,6 +27,7 @@ | |
| import java.util.Arrays; | ||
| import java.util.Collections; | ||
| import java.util.Map; | ||
| import java.util.Optional; | ||
| import java.util.stream.Stream; | ||
| import org.apache.beam.sdk.schemas.annotations.SchemaCaseFormat; | ||
| import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; | ||
|
|
@@ -40,10 +41,6 @@ | |
|
|
||
| /** Represents type information for a Java type that will be used to infer a Schema type. */ | ||
| @AutoValue | ||
| @SuppressWarnings({ | ||
| "nullness", // TODO(https://github.com/apache/beam/issues/20497) | ||
| "rawtypes" | ||
| }) | ||
| public abstract class FieldValueTypeInformation implements Serializable { | ||
| /** Optionally returns the field index. */ | ||
| public abstract @Nullable Integer getNumber(); | ||
|
|
@@ -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); | ||
| } | ||
|
||
|
|
||
| public static FieldValueTypeInformation forField( | ||
| @Nullable TypeDescriptor<?> typeDescriptor, Field field, int index) { | ||
| TypeDescriptor<?> type = | ||
| Optional.ofNullable(typeDescriptor) | ||
| .map(td -> (TypeDescriptor) td.resolveType(field.getGenericType())) | ||
| // fall back to previous behavior | ||
| .orElseGet(() -> TypeDescriptor.of(field.getGenericType())); | ||
| return new AutoValue_FieldValueTypeInformation.Builder() | ||
| .setName(getNameOverride(field.getName(), field)) | ||
| .setNumber(getNumberOverride(index, field)) | ||
| .setNullable(hasNullableAnnotation(field)) | ||
| .setType(type) | ||
| .setRawType(type.getRawType()) | ||
| .setField(field) | ||
| .setElementType(getIterableComponentType(field)) | ||
| .setMapKeyType(getMapKeyType(field)) | ||
| .setMapValueType(getMapValueType(field)) | ||
| .setElementType(getIterableComponentType(type)) | ||
| .setMapKeyType(getMapKeyType(type)) | ||
| .setMapValueType(getMapValueType(type)) | ||
| .setOneOfTypes(Collections.emptyMap()) | ||
| .setDescription(getFieldDescription(field)) | ||
| .build(); | ||
|
|
@@ -185,6 +191,11 @@ public static <T extends AnnotatedElement & Member> String getNameOverride( | |
| } | ||
|
|
||
| public static FieldValueTypeInformation forGetter(Method method, int index) { | ||
| return forGetter(null, method, index); | ||
| } | ||
|
|
||
| public static FieldValueTypeInformation forGetter( | ||
| @Nullable TypeDescriptor<?> typeDescriptor, Method method, int index) { | ||
| String name; | ||
| if (method.getName().startsWith("get")) { | ||
| name = ReflectUtils.stripPrefix(method.getName(), "get"); | ||
|
|
@@ -194,7 +205,12 @@ public static FieldValueTypeInformation forGetter(Method method, int index) { | |
| throw new RuntimeException("Getter has wrong prefix " + method.getName()); | ||
| } | ||
|
|
||
| TypeDescriptor<?> type = TypeDescriptor.of(method.getGenericReturnType()); | ||
| TypeDescriptor<?> type = | ||
| Optional.ofNullable(typeDescriptor) | ||
| .map(td -> (TypeDescriptor) td.resolveType(method.getGenericReturnType())) | ||
| // fall back to previous behavior | ||
| .orElseGet(() -> TypeDescriptor.of(method.getGenericReturnType())); | ||
|
|
||
| boolean nullable = hasNullableReturnType(method); | ||
| return new AutoValue_FieldValueTypeInformation.Builder() | ||
| .setName(getNameOverride(name, method)) | ||
|
|
@@ -253,18 +269,32 @@ private static boolean isNullableAnnotation(Annotation annotation) { | |
| } | ||
|
|
||
| public static FieldValueTypeInformation forSetter(Method method) { | ||
| return forSetter(method, "set"); | ||
| return forSetter(null, method); | ||
| } | ||
|
|
||
| public static FieldValueTypeInformation forSetter(Method method, String setterPrefix) { | ||
| return forSetter(null, method, setterPrefix); | ||
| } | ||
|
|
||
| public static FieldValueTypeInformation forSetter( | ||
| @Nullable TypeDescriptor<?> typeDescriptor, Method method) { | ||
| return forSetter(typeDescriptor, method, "set"); | ||
| } | ||
|
|
||
| public static FieldValueTypeInformation forSetter( | ||
| @Nullable TypeDescriptor<?> typeDescriptor, Method method, String setterPrefix) { | ||
| String name; | ||
| if (method.getName().startsWith(setterPrefix)) { | ||
| name = ReflectUtils.stripPrefix(method.getName(), setterPrefix); | ||
| } else { | ||
| throw new RuntimeException("Setter has wrong prefix " + method.getName()); | ||
| } | ||
|
|
||
| TypeDescriptor<?> type = TypeDescriptor.of(method.getGenericParameterTypes()[0]); | ||
| TypeDescriptor<?> type = | ||
| Optional.ofNullable(typeDescriptor) | ||
| .map(td -> (TypeDescriptor) td.resolveType(method.getGenericParameterTypes()[0])) | ||
| // fall back to previous behavior | ||
| .orElseGet(() -> TypeDescriptor.of(method.getGenericParameterTypes()[0])); | ||
| boolean nullable = hasSingleNullableParameter(method); | ||
| return new AutoValue_FieldValueTypeInformation.Builder() | ||
| .setName(name) | ||
|
|
@@ -283,10 +313,6 @@ public FieldValueTypeInformation withName(String name) { | |
| return toBuilder().setName(name).build(); | ||
| } | ||
|
|
||
| private static FieldValueTypeInformation getIterableComponentType(Field field) { | ||
| return getIterableComponentType(TypeDescriptor.of(field.getGenericType())); | ||
| } | ||
|
|
||
| static @Nullable FieldValueTypeInformation getIterableComponentType(TypeDescriptor<?> valueType) { | ||
| // TODO: Figure out nullable elements. | ||
| TypeDescriptor<?> componentType = ReflectUtils.getIterableComponentType(valueType); | ||
|
|
@@ -306,34 +332,23 @@ private static FieldValueTypeInformation getIterableComponentType(Field field) { | |
| .build(); | ||
| } | ||
|
|
||
| // If the Field is a map type, returns the key type, otherwise returns a null reference. | ||
|
|
||
| private static @Nullable FieldValueTypeInformation getMapKeyType(Field field) { | ||
| return getMapKeyType(TypeDescriptor.of(field.getGenericType())); | ||
| } | ||
|
|
||
| // If the type is a map type, returns the key type, otherwise returns a null reference. | ||
| private static @Nullable FieldValueTypeInformation getMapKeyType( | ||
| TypeDescriptor<?> typeDescriptor) { | ||
| return getMapType(typeDescriptor, 0); | ||
| } | ||
|
|
||
| // If the Field is a map type, returns the value type, otherwise returns a null reference. | ||
|
|
||
| private static @Nullable FieldValueTypeInformation getMapValueType(Field field) { | ||
| return getMapType(TypeDescriptor.of(field.getGenericType()), 1); | ||
| } | ||
|
|
||
| // If the type is a map type, returns the value type, otherwise returns a null reference. | ||
| private static @Nullable FieldValueTypeInformation getMapValueType( | ||
| TypeDescriptor<?> typeDescriptor) { | ||
| return getMapType(typeDescriptor, 1); | ||
| } | ||
|
|
||
| // If the Field is a map type, returns the key or value type (0 is key type, 1 is value). | ||
| // Otherwise returns a null reference. | ||
| @SuppressWarnings("unchecked") | ||
| private static @Nullable FieldValueTypeInformation getMapType( | ||
| TypeDescriptor<?> valueType, int index) { | ||
| TypeDescriptor mapType = ReflectUtils.getMapType(valueType, index); | ||
| TypeDescriptor<?> mapType = ReflectUtils.getMapType(valueType, index); | ||
| if (mapType == null) { | ||
| return null; | ||
| } | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what are these new tags for?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
so in
RowValueGettersFactorythere's this code:since we're in the constructor, the
thisparameter is still not fully initialized, so by using this annotation we can temporarily (for the duration of the initalization of the CachingFactory object) allow assigning theinnerFactoryfield a value which itself is under initialization (or indeed whose initialization status is unknown, as the signature of the CachingFactory constructor tells us)see also https://checkerframework.org/manual/#circular-initialization