diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index bbdc3a3910ef8..3f63c0c9975f2 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 3 + "modification": 2 } diff --git a/.github/trigger_files/IO_Iceberg_Unit_Tests.json b/.github/trigger_files/IO_Iceberg_Unit_Tests.json index b26833333238c..e3d6056a5de96 100644 --- a/.github/trigger_files/IO_Iceberg_Unit_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Unit_Tests.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 2 + "modification": 1 } diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json index b26833333238c..c537844dc84ad 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Dataflow.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 2 + "modification": 3 } diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json index e3d6056a5de96..b26833333238c 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 2 } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java index c369eefeb65c1..5ccfe39b92af1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java @@ -19,10 +19,8 @@ import java.lang.reflect.Method; import java.lang.reflect.Modifier; -import java.lang.reflect.Type; import java.util.Comparator; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import org.apache.beam.sdk.schemas.annotations.SchemaIgnore; import org.apache.beam.sdk.schemas.utils.AutoValueUtils; @@ -63,9 +61,8 @@ public List get(TypeDescriptor typeDescriptor) { .filter(m -> !m.isAnnotationPresent(SchemaIgnore.class)) .collect(Collectors.toList()); List types = Lists.newArrayListWithCapacity(methods.size()); - Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); for (int i = 0; i < methods.size(); ++i) { - types.add(FieldValueTypeInformation.forGetter(methods.get(i), i, boundTypes)); + types.add(FieldValueTypeInformation.forGetter(methods.get(i), i)); } types.sort(Comparator.comparing(FieldValueTypeInformation::getNumber)); validateFieldNumbers(types); @@ -146,8 +143,7 @@ public SchemaUserTypeCreator schemaTypeCreator( @Override public @Nullable Schema schemaFor(TypeDescriptor typeDescriptor) { - Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); return JavaBeanUtils.schemaFromJavaBeanClass( - typeDescriptor, AbstractGetterTypeSupplier.INSTANCE, boundTypes); + typeDescriptor, AbstractGetterTypeSupplier.INSTANCE); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java index 64687e6d33814..750709192c08b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java @@ -24,12 +24,10 @@ import java.lang.reflect.Field; import java.lang.reflect.Member; import java.lang.reflect.Method; -import java.lang.reflect.Type; import java.util.Arrays; import java.util.Collections; import java.util.Map; import java.util.stream.Stream; -import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.annotations.SchemaCaseFormat; import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.annotations.SchemaFieldName; @@ -46,7 +44,6 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) -@Internal public abstract class FieldValueTypeInformation implements Serializable { /** Optionally returns the field index. */ public abstract @Nullable Integer getNumber(); @@ -128,10 +125,8 @@ public static FieldValueTypeInformation forOneOf( .build(); } - public static FieldValueTypeInformation forField( - Field field, int index, Map boundTypes) { - TypeDescriptor type = - TypeDescriptor.of(ReflectUtils.resolveType(field.getGenericType(), boundTypes)); + public static FieldValueTypeInformation forField(Field field, int index) { + TypeDescriptor type = TypeDescriptor.of(field.getGenericType()); return new AutoValue_FieldValueTypeInformation.Builder() .setName(getNameOverride(field.getName(), field)) .setNumber(getNumberOverride(index, field)) @@ -139,9 +134,9 @@ public static FieldValueTypeInformation forField( .setType(type) .setRawType(type.getRawType()) .setField(field) - .setElementType(getIterableComponentType(field, boundTypes)) - .setMapKeyType(getMapKeyType(field, boundTypes)) - .setMapValueType(getMapValueType(field, boundTypes)) + .setElementType(getIterableComponentType(field)) + .setMapKeyType(getMapKeyType(field)) + .setMapValueType(getMapValueType(field)) .setOneOfTypes(Collections.emptyMap()) .setDescription(getFieldDescription(field)) .build(); @@ -189,8 +184,7 @@ public static String getNameOverride( return fieldDescription.value(); } - public static FieldValueTypeInformation forGetter( - Method method, int index, Map boundTypes) { + public static FieldValueTypeInformation forGetter(Method method, int index) { String name; if (method.getName().startsWith("get")) { name = ReflectUtils.stripPrefix(method.getName(), "get"); @@ -200,8 +194,7 @@ public static FieldValueTypeInformation forGetter( throw new RuntimeException("Getter has wrong prefix " + method.getName()); } - TypeDescriptor type = - TypeDescriptor.of(ReflectUtils.resolveType(method.getGenericReturnType(), boundTypes)); + TypeDescriptor type = TypeDescriptor.of(method.getGenericReturnType()); boolean nullable = hasNullableReturnType(method); return new AutoValue_FieldValueTypeInformation.Builder() .setName(getNameOverride(name, method)) @@ -210,9 +203,9 @@ public static FieldValueTypeInformation forGetter( .setType(type) .setRawType(type.getRawType()) .setMethod(method) - .setElementType(getIterableComponentType(type, boundTypes)) - .setMapKeyType(getMapKeyType(type, boundTypes)) - .setMapValueType(getMapValueType(type, boundTypes)) + .setElementType(getIterableComponentType(type)) + .setMapKeyType(getMapKeyType(type)) + .setMapValueType(getMapValueType(type)) .setOneOfTypes(Collections.emptyMap()) .setDescription(getFieldDescription(method)) .build(); @@ -259,13 +252,11 @@ private static boolean isNullableAnnotation(Annotation annotation) { return annotation.annotationType().getSimpleName().equals("Nullable"); } - public static FieldValueTypeInformation forSetter( - Method method, Map boundParameters) { - return forSetter(method, "set", boundParameters); + public static FieldValueTypeInformation forSetter(Method method) { + return forSetter(method, "set"); } - public static FieldValueTypeInformation forSetter( - Method method, String setterPrefix, Map boundTypes) { + public static FieldValueTypeInformation forSetter(Method method, String setterPrefix) { String name; if (method.getName().startsWith(setterPrefix)) { name = ReflectUtils.stripPrefix(method.getName(), setterPrefix); @@ -273,9 +264,7 @@ public static FieldValueTypeInformation forSetter( throw new RuntimeException("Setter has wrong prefix " + method.getName()); } - TypeDescriptor type = - TypeDescriptor.of( - ReflectUtils.resolveType(method.getGenericParameterTypes()[0], boundTypes)); + TypeDescriptor type = TypeDescriptor.of(method.getGenericParameterTypes()[0]); boolean nullable = hasSingleNullableParameter(method); return new AutoValue_FieldValueTypeInformation.Builder() .setName(name) @@ -283,9 +272,9 @@ public static FieldValueTypeInformation forSetter( .setType(type) .setRawType(type.getRawType()) .setMethod(method) - .setElementType(getIterableComponentType(type, boundTypes)) - .setMapKeyType(getMapKeyType(type, boundTypes)) - .setMapValueType(getMapValueType(type, boundTypes)) + .setElementType(getIterableComponentType(type)) + .setMapKeyType(getMapKeyType(type)) + .setMapValueType(getMapValueType(type)) .setOneOfTypes(Collections.emptyMap()) .build(); } @@ -294,15 +283,13 @@ public FieldValueTypeInformation withName(String name) { return toBuilder().setName(name).build(); } - private static FieldValueTypeInformation getIterableComponentType( - Field field, Map boundTypes) { - return getIterableComponentType(TypeDescriptor.of(field.getGenericType()), boundTypes); + private static FieldValueTypeInformation getIterableComponentType(Field field) { + return getIterableComponentType(TypeDescriptor.of(field.getGenericType())); } - static @Nullable FieldValueTypeInformation getIterableComponentType( - TypeDescriptor valueType, Map boundTypes) { + static @Nullable FieldValueTypeInformation getIterableComponentType(TypeDescriptor valueType) { // TODO: Figure out nullable elements. - TypeDescriptor componentType = ReflectUtils.getIterableComponentType(valueType, boundTypes); + TypeDescriptor componentType = ReflectUtils.getIterableComponentType(valueType); if (componentType == null) { return null; } @@ -312,43 +299,41 @@ private static FieldValueTypeInformation getIterableComponentType( .setNullable(false) .setType(componentType) .setRawType(componentType.getRawType()) - .setElementType(getIterableComponentType(componentType, boundTypes)) - .setMapKeyType(getMapKeyType(componentType, boundTypes)) - .setMapValueType(getMapValueType(componentType, boundTypes)) + .setElementType(getIterableComponentType(componentType)) + .setMapKeyType(getMapKeyType(componentType)) + .setMapValueType(getMapValueType(componentType)) .setOneOfTypes(Collections.emptyMap()) .build(); } // If the Field is a map type, returns the key type, otherwise returns a null reference. - private static @Nullable FieldValueTypeInformation getMapKeyType( - Field field, Map boundTypes) { - return getMapKeyType(TypeDescriptor.of(field.getGenericType()), boundTypes); + private static @Nullable FieldValueTypeInformation getMapKeyType(Field field) { + return getMapKeyType(TypeDescriptor.of(field.getGenericType())); } private static @Nullable FieldValueTypeInformation getMapKeyType( - TypeDescriptor typeDescriptor, Map boundTypes) { - return getMapType(typeDescriptor, 0, boundTypes); + 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, Map boundTypes) { - return getMapType(TypeDescriptor.of(field.getGenericType()), 1, boundTypes); + private static @Nullable FieldValueTypeInformation getMapValueType(Field field) { + return getMapType(TypeDescriptor.of(field.getGenericType()), 1); } private static @Nullable FieldValueTypeInformation getMapValueType( - TypeDescriptor typeDescriptor, Map boundTypes) { - return getMapType(typeDescriptor, 1, boundTypes); + 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, Map boundTypes) { - TypeDescriptor mapType = ReflectUtils.getMapType(valueType, index, boundTypes); + TypeDescriptor valueType, int index) { + TypeDescriptor mapType = ReflectUtils.getMapType(valueType, index); if (mapType == null) { return null; } @@ -357,9 +342,9 @@ private static FieldValueTypeInformation getIterableComponentType( .setNullable(false) .setType(mapType) .setRawType(mapType.getRawType()) - .setElementType(getIterableComponentType(mapType, boundTypes)) - .setMapKeyType(getMapKeyType(mapType, boundTypes)) - .setMapValueType(getMapValueType(mapType, boundTypes)) + .setElementType(getIterableComponentType(mapType)) + .setMapKeyType(getMapKeyType(mapType)) + .setMapValueType(getMapValueType(mapType)) .setOneOfTypes(Collections.emptyMap()) .build(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java index ad71576670bf8..a9cf01c520579 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java @@ -19,10 +19,8 @@ import java.lang.reflect.Constructor; import java.lang.reflect.Method; -import java.lang.reflect.Type; import java.util.Comparator; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import org.apache.beam.sdk.schemas.annotations.SchemaCaseFormat; import org.apache.beam.sdk.schemas.annotations.SchemaFieldName; @@ -69,9 +67,8 @@ public List get(TypeDescriptor typeDescriptor) { .filter(m -> !m.isAnnotationPresent(SchemaIgnore.class)) .collect(Collectors.toList()); List types = Lists.newArrayListWithCapacity(methods.size()); - Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); for (int i = 0; i < methods.size(); ++i) { - types.add(FieldValueTypeInformation.forGetter(methods.get(i), i, boundTypes)); + types.add(FieldValueTypeInformation.forGetter(methods.get(i), i)); } types.sort(Comparator.comparing(FieldValueTypeInformation::getNumber)); validateFieldNumbers(types); @@ -114,11 +111,10 @@ public static class SetterTypeSupplier implements FieldValueTypeSupplier { @Override public List get(TypeDescriptor typeDescriptor) { - Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); return ReflectUtils.getMethods(typeDescriptor.getRawType()).stream() .filter(ReflectUtils::isSetter) .filter(m -> !m.isAnnotationPresent(SchemaIgnore.class)) - .map(m -> FieldValueTypeInformation.forSetter(m, boundTypes)) + .map(FieldValueTypeInformation::forSetter) .map( t -> { if (t.getMethod().getAnnotation(SchemaFieldNumber.class) != null) { @@ -160,10 +156,8 @@ public boolean equals(@Nullable Object obj) { @Override public Schema schemaFor(TypeDescriptor typeDescriptor) { - Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); Schema schema = - JavaBeanUtils.schemaFromJavaBeanClass( - typeDescriptor, GetterTypeSupplier.INSTANCE, boundTypes); + JavaBeanUtils.schemaFromJavaBeanClass(typeDescriptor, GetterTypeSupplier.INSTANCE); // If there are no creator methods, then validate that we have setters for every field. // Otherwise, we will have no way of creating instances of the class. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java index da0f59c8ee969..21f07c47b47f8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java @@ -21,10 +21,8 @@ import java.lang.reflect.Field; import java.lang.reflect.Method; import java.lang.reflect.Modifier; -import java.lang.reflect.Type; import java.util.Comparator; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -64,11 +62,9 @@ public List get(TypeDescriptor typeDescriptor) { ReflectUtils.getFields(typeDescriptor.getRawType()).stream() .filter(m -> !m.isAnnotationPresent(SchemaIgnore.class)) .collect(Collectors.toList()); - List types = Lists.newArrayListWithCapacity(fields.size()); - Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); for (int i = 0; i < fields.size(); ++i) { - types.add(FieldValueTypeInformation.forField(fields.get(i), i, boundTypes)); + types.add(FieldValueTypeInformation.forField(fields.get(i), i)); } types.sort(Comparator.comparing(FieldValueTypeInformation::getNumber)); validateFieldNumbers(types); @@ -115,9 +111,7 @@ private static void validateFieldNumbers(List types) @Override public Schema schemaFor(TypeDescriptor typeDescriptor) { - Map boundTypes = ReflectUtils.getAllBoundTypes(typeDescriptor); - return POJOUtils.schemaFromPojoClass( - typeDescriptor, JavaFieldTypeSupplier.INSTANCE, boundTypes); + return POJOUtils.schemaFromPojoClass(typeDescriptor, JavaFieldTypeSupplier.INSTANCE); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaProvider.java index b7e3cdf60c18f..37b4952e529c8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaProvider.java @@ -38,7 +38,8 @@ public interface SchemaProvider extends Serializable { * Given a type, return a function that converts that type to a {@link Row} object If no schema * exists, returns null. */ - @Nullable SerializableFunction toRowFunction(TypeDescriptor typeDescriptor); + @Nullable + SerializableFunction toRowFunction(TypeDescriptor typeDescriptor); /** * Given a type, returns a function that converts from a {@link Row} object to that type. If no diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java index 5d8b7aab61937..679a1fcf54fc2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java @@ -76,12 +76,13 @@ void registerProvider(TypeDescriptor typeDescriptor, SchemaProvider schemaProvid providers.put(typeDescriptor, schemaProvider); } - private @Nullable SchemaProvider schemaProviderFor(TypeDescriptor typeDescriptor) { + @Override + public @Nullable Schema schemaFor(TypeDescriptor typeDescriptor) { TypeDescriptor type = typeDescriptor; do { SchemaProvider schemaProvider = providers.get(type); if (schemaProvider != null) { - return schemaProvider; + return schemaProvider.schemaFor(type); } Class superClass = type.getRawType().getSuperclass(); if (superClass == null || superClass.equals(Object.class)) { @@ -91,24 +92,38 @@ void registerProvider(TypeDescriptor typeDescriptor, SchemaProvider schemaProvid } while (true); } - @Override - public @Nullable Schema schemaFor(TypeDescriptor typeDescriptor) { - @Nullable SchemaProvider schemaProvider = schemaProviderFor(typeDescriptor); - return schemaProvider != null ? schemaProvider.schemaFor(typeDescriptor) : null; - } - @Override public @Nullable SerializableFunction toRowFunction( TypeDescriptor typeDescriptor) { - @Nullable SchemaProvider schemaProvider = schemaProviderFor(typeDescriptor); - return schemaProvider != null ? schemaProvider.toRowFunction(typeDescriptor) : null; + TypeDescriptor type = typeDescriptor; + do { + SchemaProvider schemaProvider = providers.get(type); + if (schemaProvider != null) { + return (SerializableFunction) schemaProvider.toRowFunction(type); + } + Class superClass = type.getRawType().getSuperclass(); + if (superClass == null || superClass.equals(Object.class)) { + return null; + } + type = TypeDescriptor.of(superClass); + } while (true); } @Override public @Nullable SerializableFunction fromRowFunction( TypeDescriptor typeDescriptor) { - @Nullable SchemaProvider schemaProvider = schemaProviderFor(typeDescriptor); - return schemaProvider != null ? schemaProvider.fromRowFunction(typeDescriptor) : null; + TypeDescriptor type = typeDescriptor; + do { + SchemaProvider schemaProvider = providers.get(type); + if (schemaProvider != null) { + return (SerializableFunction) schemaProvider.fromRowFunction(type); + } + Class superClass = type.getRawType().getSuperclass(); + if (superClass == null || superClass.equals(Object.class)) { + return null; + } + type = TypeDescriptor.of(superClass); + } while (true); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java index c3a71bbb454b5..54e2a595fa71c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/JavaRowUdf.java @@ -160,8 +160,7 @@ public FunctionAndType(Type outputType, Function function) { public FunctionAndType(TypeDescriptor outputType, Function function) { this( - StaticSchemaInference.fieldFromType( - outputType, new EmptyFieldValueTypeSupplier(), Collections.emptyMap()), + StaticSchemaInference.fieldFromType(outputType, new EmptyFieldValueTypeSupplier()), function); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java index 74e97bad4f0f1..d7fddd8abfed2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java @@ -53,7 +53,6 @@ import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; import net.bytebuddy.jar.asm.ClassWriter; import net.bytebuddy.matcher.ElementMatchers; -import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.FieldValueTypeInformation; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaUserTypeCreator; @@ -64,7 +63,6 @@ import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.checkerframework.checker.nullness.qual.Nullable; /** Utilities for managing AutoValue schemas. */ @@ -72,7 +70,6 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) -@Internal public class AutoValueUtils { public static TypeDescriptor getBaseAutoValueClass(TypeDescriptor typeDescriptor) { // AutoValue extensions may be nested @@ -164,7 +161,7 @@ private static boolean matchConstructor( // Verify that constructor parameters match (name and type) the inferred schema. for (Parameter parameter : constructor.getParameters()) { FieldValueTypeInformation type = typeMap.get(parameter.getName()); - if (type == null || !type.getRawType().equals(parameter.getType())) { + if (type == null || type.getRawType() != parameter.getType()) { valid = false; break; } @@ -181,7 +178,7 @@ private static boolean matchConstructor( } name = name.substring(0, name.length() - 1); FieldValueTypeInformation type = typeMap.get(name); - if (type == null || !type.getRawType().equals(parameter.getType())) { + if (type == null || type.getRawType() != parameter.getType()) { return false; } } @@ -199,12 +196,11 @@ private static boolean matchConstructor( return null; } - Map boundTypes = ReflectUtils.getAllBoundTypes(TypeDescriptor.of(builderClass)); - Map setterTypes = Maps.newHashMap(); - ReflectUtils.getMethods(builderClass).stream() - .filter(ReflectUtils::isSetter) - .map(m -> FieldValueTypeInformation.forSetter(m, boundTypes)) - .forEach(fv -> setterTypes.putIfAbsent(fv.getName(), fv)); + Map setterTypes = + ReflectUtils.getMethods(builderClass).stream() + .filter(ReflectUtils::isSetter) + .map(FieldValueTypeInformation::forSetter) + .collect(Collectors.toMap(FieldValueTypeInformation::getName, Function.identity())); List setterMethods = Lists.newArrayList(); // The builder methods to call in order. @@ -325,7 +321,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { Duplication.SINGLE, typeConversionsFactory .createSetterConversions(readParameter) - .convert(TypeDescriptor.of(parameter.getParameterizedType())), + .convert(TypeDescriptor.of(parameter.getType())), MethodInvocation.invoke(new ForLoadedMethod(setterMethod)), Removal.SINGLE); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java index 65adc33a1bab5..c2b33c2d2315b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java @@ -344,22 +344,19 @@ protected Type convertArray(TypeDescriptor type) { @Override protected Type convertCollection(TypeDescriptor type) { - TypeDescriptor ret = - createCollectionType(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); + TypeDescriptor ret = createCollectionType(ReflectUtils.getIterableComponentType(type)); return returnRawTypes ? ret.getRawType() : ret.getType(); } @Override protected Type convertList(TypeDescriptor type) { - TypeDescriptor ret = - createCollectionType(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); + TypeDescriptor ret = createCollectionType(ReflectUtils.getIterableComponentType(type)); return returnRawTypes ? ret.getRawType() : ret.getType(); } @Override protected Type convertIterable(TypeDescriptor type) { - TypeDescriptor ret = - createIterableType(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); + TypeDescriptor ret = createIterableType(ReflectUtils.getIterableComponentType(type)); return returnRawTypes ? ret.getRawType() : ret.getType(); } @@ -690,8 +687,7 @@ protected StackManipulation convertArray(TypeDescriptor type) { @Override protected StackManipulation convertIterable(TypeDescriptor type) { - TypeDescriptor componentType = - ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); + TypeDescriptor componentType = ReflectUtils.getIterableComponentType(type); Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType); final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType); @@ -711,8 +707,7 @@ protected StackManipulation convertIterable(TypeDescriptor type) { @Override protected StackManipulation convertCollection(TypeDescriptor type) { - TypeDescriptor componentType = - ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); + TypeDescriptor componentType = ReflectUtils.getIterableComponentType(type); Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType); final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType); if (!finalComponentType.hasUnresolvedParameters()) { @@ -731,8 +726,7 @@ protected StackManipulation convertCollection(TypeDescriptor type) { @Override protected StackManipulation convertList(TypeDescriptor type) { - TypeDescriptor componentType = - ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); + TypeDescriptor componentType = ReflectUtils.getIterableComponentType(type); Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType); final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType); if (!finalComponentType.hasUnresolvedParameters()) { @@ -751,8 +745,8 @@ protected StackManipulation convertList(TypeDescriptor type) { @Override protected StackManipulation convertMap(TypeDescriptor type) { - final TypeDescriptor keyType = ReflectUtils.getMapType(type, 0, Collections.emptyMap()); - final TypeDescriptor valueType = ReflectUtils.getMapType(type, 1, Collections.emptyMap()); + final TypeDescriptor keyType = ReflectUtils.getMapType(type, 0); + final TypeDescriptor valueType = ReflectUtils.getMapType(type, 1); Type convertedKeyType = getFactory().createTypeConversion(true).convert(keyType); Type convertedValueType = getFactory().createTypeConversion(true).convert(valueType); @@ -1044,9 +1038,8 @@ protected StackManipulation convertIterable(TypeDescriptor type) { Type rowElementType = getFactory() .createTypeConversion(false) - .convert(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); - final TypeDescriptor iterableElementType = - ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); + .convert(ReflectUtils.getIterableComponentType(type)); + final TypeDescriptor iterableElementType = ReflectUtils.getIterableComponentType(type); if (!iterableElementType.hasUnresolvedParameters()) { ForLoadedType conversionFunction = new ForLoadedType( @@ -1067,9 +1060,8 @@ protected StackManipulation convertCollection(TypeDescriptor type) { Type rowElementType = getFactory() .createTypeConversion(false) - .convert(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); - final TypeDescriptor collectionElementType = - ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); + .convert(ReflectUtils.getIterableComponentType(type)); + final TypeDescriptor collectionElementType = ReflectUtils.getIterableComponentType(type); if (!collectionElementType.hasUnresolvedParameters()) { ForLoadedType conversionFunction = @@ -1091,9 +1083,8 @@ protected StackManipulation convertList(TypeDescriptor type) { Type rowElementType = getFactory() .createTypeConversion(false) - .convert(ReflectUtils.getIterableComponentType(type, Collections.emptyMap())); - final TypeDescriptor collectionElementType = - ReflectUtils.getIterableComponentType(type, Collections.emptyMap()); + .convert(ReflectUtils.getIterableComponentType(type)); + final TypeDescriptor collectionElementType = ReflectUtils.getIterableComponentType(type); StackManipulation readTrasformedValue = readValue; if (!collectionElementType.hasUnresolvedParameters()) { @@ -1122,17 +1113,11 @@ protected StackManipulation convertList(TypeDescriptor type) { @Override protected StackManipulation convertMap(TypeDescriptor type) { Type rowKeyType = - getFactory() - .createTypeConversion(false) - .convert(ReflectUtils.getMapType(type, 0, Collections.emptyMap())); - final TypeDescriptor keyElementType = - ReflectUtils.getMapType(type, 0, Collections.emptyMap()); + getFactory().createTypeConversion(false).convert(ReflectUtils.getMapType(type, 0)); + final TypeDescriptor keyElementType = ReflectUtils.getMapType(type, 0); Type rowValueType = - getFactory() - .createTypeConversion(false) - .convert(ReflectUtils.getMapType(type, 1, Collections.emptyMap())); - final TypeDescriptor valueElementType = - ReflectUtils.getMapType(type, 1, Collections.emptyMap()); + getFactory().createTypeConversion(false).convert(ReflectUtils.getMapType(type, 1)); + final TypeDescriptor valueElementType = ReflectUtils.getMapType(type, 1); StackManipulation readTrasformedValue = readValue; if (!keyElementType.hasUnresolvedParameters() @@ -1490,7 +1475,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { Parameter parameter = parameters.get(i); ForLoadedType convertedType = new ForLoadedType( - (Class) convertType.convert(TypeDescriptor.of(parameter.getParameterizedType()))); + (Class) convertType.convert(TypeDescriptor.of(parameter.getType()))); // The instruction to read the parameter. Use the fieldMapping to reorder parameters as // necessary. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java index e98a0b9495cf1..7f2403035d97f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java @@ -22,7 +22,6 @@ import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Type; -import java.util.Collections; import java.util.ServiceLoader; import net.bytebuddy.ByteBuddy; import net.bytebuddy.asm.AsmVisitorWrapper; @@ -37,7 +36,6 @@ import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; import net.bytebuddy.jar.asm.ClassWriter; import net.bytebuddy.matcher.ElementMatchers; -import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.JavaFieldSchema.JavaFieldTypeSupplier; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; @@ -58,7 +56,6 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) -@Internal public class ConvertHelpers { private static class SchemaInformationProviders { private static final ServiceLoader INSTANCE = @@ -151,8 +148,7 @@ public static SerializableFunction getConvertPrimitive( TypeDescriptor outputTypeDescriptor, TypeConversionsFactory typeConversionsFactory) { FieldType expectedFieldType = - StaticSchemaInference.fieldFromType( - outputTypeDescriptor, JavaFieldTypeSupplier.INSTANCE, Collections.emptyMap()); + StaticSchemaInference.fieldFromType(outputTypeDescriptor, JavaFieldTypeSupplier.INSTANCE); if (!expectedFieldType.equals(fieldType)) { throw new IllegalArgumentException( "Element argument type " diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java index 83f6b5c928d8e..911f79f6eeedb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java @@ -22,7 +22,6 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.lang.reflect.Type; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -43,7 +42,6 @@ import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; import net.bytebuddy.jar.asm.ClassWriter; import net.bytebuddy.matcher.ElementMatchers; -import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.FieldValueGetter; import org.apache.beam.sdk.schemas.FieldValueSetter; import org.apache.beam.sdk.schemas.FieldValueTypeInformation; @@ -63,15 +61,11 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) -@Internal public class JavaBeanUtils { /** Create a {@link Schema} for a Java Bean class. */ public static Schema schemaFromJavaBeanClass( - TypeDescriptor typeDescriptor, - FieldValueTypeSupplier fieldValueTypeSupplier, - Map boundTypes) { - return StaticSchemaInference.schemaFromClass( - typeDescriptor, fieldValueTypeSupplier, boundTypes); + TypeDescriptor typeDescriptor, FieldValueTypeSupplier fieldValueTypeSupplier) { + return StaticSchemaInference.schemaFromClass(typeDescriptor, fieldValueTypeSupplier); } private static final String CONSTRUCTOR_HELP_STRING = diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java index 1e60c9312cb3e..571b9c6909009 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java @@ -49,7 +49,6 @@ import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; import net.bytebuddy.jar.asm.ClassWriter; import net.bytebuddy.matcher.ElementMatchers; -import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.FieldValueGetter; import org.apache.beam.sdk.schemas.FieldValueSetter; import org.apache.beam.sdk.schemas.FieldValueTypeInformation; @@ -71,15 +70,11 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) }) -@Internal public class POJOUtils { public static Schema schemaFromPojoClass( - TypeDescriptor typeDescriptor, - FieldValueTypeSupplier fieldValueTypeSupplier, - Map boundTypes) { - return StaticSchemaInference.schemaFromClass( - typeDescriptor, fieldValueTypeSupplier, boundTypes); + TypeDescriptor typeDescriptor, FieldValueTypeSupplier fieldValueTypeSupplier) { + return StaticSchemaInference.schemaFromClass(typeDescriptor, fieldValueTypeSupplier); } // Static ByteBuddy instance used by all helpers. @@ -306,7 +301,7 @@ public static SchemaUserTypeCreator createStaticCreator( field.getDeclaringClass(), typeConversionsFactory .createTypeConversion(false) - .convert(TypeDescriptor.of(field.getGenericType()))); + .convert(TypeDescriptor.of(field.getType()))); builder = implementGetterMethods(builder, field, typeInformation.getName(), typeConversionsFactory); try { @@ -388,7 +383,7 @@ private static FieldValueSetter createSetter( field.getDeclaringClass(), typeConversionsFactory .createTypeConversion(false) - .convert(TypeDescriptor.of(field.getGenericType()))); + .convert(TypeDescriptor.of(field.getType()))); builder = implementSetterMethods(builder, field, typeConversionsFactory); try { return builder @@ -496,7 +491,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { // Do any conversions necessary. typeConversionsFactory .createSetterConversions(readField) - .convert(TypeDescriptor.of(field.getGenericType())), + .convert(TypeDescriptor.of(field.getType())), // Now update the field and return void. FieldAccess.forField(new ForLoadedField(field)).write(), MethodReturn.VOID); @@ -551,8 +546,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { Field field = fields.get(i); ForLoadedType convertedType = - new ForLoadedType( - (Class) convertType.convert(TypeDescriptor.of(field.getGenericType()))); + new ForLoadedType((Class) convertType.convert(TypeDescriptor.of(field.getType()))); // The instruction to read the parameter. StackManipulation readParameter = @@ -569,7 +563,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { // Do any conversions necessary. typeConversionsFactory .createSetterConversions(readParameter) - .convert(TypeDescriptor.of(field.getGenericType())), + .convert(TypeDescriptor.of(field.getType())), // Now update the field. FieldAccess.forField(new ForLoadedField(field)).write()); stackManipulation = new StackManipulation.Compound(stackManipulation, updateField); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java index 32cfa56891930..4349a04c28adf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java @@ -26,17 +26,16 @@ import java.lang.reflect.Modifier; import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; -import java.lang.reflect.TypeVariable; import java.security.InvalidParameterException; import java.util.Arrays; import java.util.Collection; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.annotations.SchemaCreate; -import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; @@ -89,23 +88,14 @@ public static List getMethods(Class clazz) { return DECLARED_METHODS.computeIfAbsent( clazz, c -> { - List methods = Lists.newArrayList(); - do { - if (c.getPackage() != null && c.getPackage().getName().startsWith("java.")) { - break; // skip java built-in classes - } - Arrays.stream(c.getDeclaredMethods()) - .filter( - m -> - !m.isBridge()) // Covariant overloads insert bridge functions, which we must - // ignore. - .filter(m -> !Modifier.isPrivate(m.getModifiers())) - .filter(m -> !Modifier.isProtected(m.getModifiers())) - .filter(m -> !Modifier.isStatic(m.getModifiers())) - .forEach(methods::add); - c = c.getSuperclass(); - } while (c != null); - return methods; + return Arrays.stream(c.getDeclaredMethods()) + .filter( + m -> !m.isBridge()) // Covariant overloads insert bridge functions, which we must + // ignore. + .filter(m -> !Modifier.isPrivate(m.getModifiers())) + .filter(m -> !Modifier.isProtected(m.getModifiers())) + .filter(m -> !Modifier.isStatic(m.getModifiers())) + .collect(Collectors.toList()); }); } @@ -211,8 +201,7 @@ public static String stripSetterPrefix(String method) { } /** For an array T[] or a subclass of Iterable, return a TypeDescriptor describing T. */ - public static @Nullable TypeDescriptor getIterableComponentType( - TypeDescriptor valueType, Map boundTypes) { + public static @Nullable TypeDescriptor getIterableComponentType(TypeDescriptor valueType) { TypeDescriptor componentType = null; if (valueType.isArray()) { Type component = valueType.getComponentType().getType(); @@ -226,7 +215,7 @@ public static String stripSetterPrefix(String method) { ParameterizedType ptype = (ParameterizedType) collection.getType(); java.lang.reflect.Type[] params = ptype.getActualTypeArguments(); checkArgument(params.length == 1); - componentType = TypeDescriptor.of(resolveType(params[0], boundTypes)); + componentType = TypeDescriptor.of(params[0]); } else { throw new RuntimeException("Collection parameter is not parameterized!"); } @@ -234,15 +223,14 @@ public static String stripSetterPrefix(String method) { return componentType; } - public static TypeDescriptor getMapType( - TypeDescriptor valueType, int index, Map boundTypes) { + public static TypeDescriptor getMapType(TypeDescriptor valueType, int index) { TypeDescriptor mapType = null; if (valueType.isSubtypeOf(TypeDescriptor.of(Map.class))) { TypeDescriptor> map = valueType.getSupertype(Map.class); if (map.getType() instanceof ParameterizedType) { ParameterizedType ptype = (ParameterizedType) map.getType(); java.lang.reflect.Type[] params = ptype.getActualTypeArguments(); - mapType = TypeDescriptor.of(resolveType(params[index], boundTypes)); + mapType = TypeDescriptor.of(params[index]); } else { throw new RuntimeException("Map type is not parameterized! " + map); } @@ -255,49 +243,4 @@ public static TypeDescriptor boxIfPrimitive(TypeDescriptor typeDescriptor) { ? TypeDescriptor.of(Primitives.wrap(typeDescriptor.getRawType())) : typeDescriptor; } - - /** - * If this (or a base class)is a paremeterized type, return a map of all TypeVariable->Type - * bindings. This allows us to resolve types in any contained fields or methods. - */ - public static Map getAllBoundTypes(TypeDescriptor typeDescriptor) { - Map boundParameters = Maps.newHashMap(); - TypeDescriptor currentType = typeDescriptor; - do { - if (currentType.getType() instanceof ParameterizedType) { - ParameterizedType parameterizedType = (ParameterizedType) currentType.getType(); - TypeVariable[] typeVariables = currentType.getRawType().getTypeParameters(); - Type[] typeArguments = parameterizedType.getActualTypeArguments(); - ; - if (typeArguments.length != typeVariables.length) { - throw new RuntimeException("Unmatching arguments lengths in type " + typeDescriptor); - } - for (int i = 0; i < typeVariables.length; ++i) { - boundParameters.put(typeVariables[i], typeArguments[i]); - } - } - Type superClass = currentType.getRawType().getGenericSuperclass(); - if (superClass == null || superClass.equals(Object.class)) { - break; - } - currentType = TypeDescriptor.of(superClass); - } while (true); - return boundParameters; - } - - public static Type resolveType(Type type, Map boundTypes) { - TypeDescriptor typeDescriptor = TypeDescriptor.of(type); - if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Iterable.class)) - || typeDescriptor.isSubtypeOf(TypeDescriptor.of(Map.class))) { - // Don't resolve these as we special case map and interable. - return type; - } - - if (type instanceof TypeVariable) { - TypeVariable typeVariable = (TypeVariable) type; - return Preconditions.checkArgumentNotNull(boundTypes.get(typeVariable)); - } else { - return type; - } - } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java index 275bc41be53da..196ee6f865934 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java @@ -19,7 +19,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import java.lang.reflect.Type; +import java.lang.reflect.ParameterizedType; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.Arrays; @@ -29,12 +29,10 @@ import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.schemas.FieldValueTypeInformation; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType; -import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.ReadableInstant; @@ -44,7 +42,6 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20497) "rawtypes" }) -@Internal public class StaticSchemaInference { public static List sortBySchema( List types, Schema schema) { @@ -88,17 +85,14 @@ enum MethodType { * public getter methods, or special annotations on the class. */ public static Schema schemaFromClass( - TypeDescriptor typeDescriptor, - FieldValueTypeSupplier fieldValueTypeSupplier, - Map boundTypes) { - return schemaFromClass(typeDescriptor, fieldValueTypeSupplier, new HashMap<>(), boundTypes); + TypeDescriptor typeDescriptor, FieldValueTypeSupplier fieldValueTypeSupplier) { + return schemaFromClass(typeDescriptor, fieldValueTypeSupplier, new HashMap<>()); } private static Schema schemaFromClass( TypeDescriptor typeDescriptor, FieldValueTypeSupplier fieldValueTypeSupplier, - Map, Schema> alreadyVisitedSchemas, - Map boundTypes) { + Map, Schema> alreadyVisitedSchemas) { if (alreadyVisitedSchemas.containsKey(typeDescriptor)) { Schema existingSchema = alreadyVisitedSchemas.get(typeDescriptor); if (existingSchema == null) { @@ -112,7 +106,7 @@ private static Schema schemaFromClass( Schema.Builder builder = Schema.builder(); for (FieldValueTypeInformation type : fieldValueTypeSupplier.get(typeDescriptor)) { Schema.FieldType fieldType = - fieldFromType(type.getType(), fieldValueTypeSupplier, alreadyVisitedSchemas, boundTypes); + fieldFromType(type.getType(), fieldValueTypeSupplier, alreadyVisitedSchemas); Schema.Field f = type.isNullable() ? Schema.Field.nullable(type.getName(), fieldType) @@ -129,18 +123,15 @@ private static Schema schemaFromClass( /** Map a Java field type to a Beam Schema FieldType. */ public static Schema.FieldType fieldFromType( - TypeDescriptor type, - FieldValueTypeSupplier fieldValueTypeSupplier, - Map boundTypes) { - return fieldFromType(type, fieldValueTypeSupplier, new HashMap<>(), boundTypes); + TypeDescriptor type, FieldValueTypeSupplier fieldValueTypeSupplier) { + return fieldFromType(type, fieldValueTypeSupplier, new HashMap<>()); } // TODO(https://github.com/apache/beam/issues/21567): support type inference for logical types private static Schema.FieldType fieldFromType( TypeDescriptor type, FieldValueTypeSupplier fieldValueTypeSupplier, - Map, Schema> alreadyVisitedSchemas, - Map boundTypes) { + Map, Schema> alreadyVisitedSchemas) { FieldType primitiveType = PRIMITIVE_TYPES.get(type.getRawType()); if (primitiveType != null) { return primitiveType; @@ -161,25 +152,27 @@ private static Schema.FieldType fieldFromType( } else { // Otherwise this is an array type. return FieldType.array( - fieldFromType(component, fieldValueTypeSupplier, alreadyVisitedSchemas, boundTypes)); + fieldFromType(component, fieldValueTypeSupplier, alreadyVisitedSchemas)); } } else if (type.isSubtypeOf(TypeDescriptor.of(Map.class))) { - FieldType keyType = - fieldFromType( - ReflectUtils.getMapType(type, 0, boundTypes), - fieldValueTypeSupplier, - alreadyVisitedSchemas, - boundTypes); - FieldType valueType = - fieldFromType( - ReflectUtils.getMapType(type, 1, boundTypes), - fieldValueTypeSupplier, - alreadyVisitedSchemas, - boundTypes); - checkArgument( - keyType.getTypeName().isPrimitiveType(), - "Only primitive types can be map keys. type: " + keyType.getTypeName()); - return FieldType.map(keyType, valueType); + TypeDescriptor> map = type.getSupertype(Map.class); + if (map.getType() instanceof ParameterizedType) { + ParameterizedType ptype = (ParameterizedType) map.getType(); + java.lang.reflect.Type[] params = ptype.getActualTypeArguments(); + checkArgument(params.length == 2); + FieldType keyType = + fieldFromType( + TypeDescriptor.of(params[0]), fieldValueTypeSupplier, alreadyVisitedSchemas); + FieldType valueType = + fieldFromType( + TypeDescriptor.of(params[1]), fieldValueTypeSupplier, alreadyVisitedSchemas); + checkArgument( + keyType.getTypeName().isPrimitiveType(), + "Only primitive types can be map keys. type: " + keyType.getTypeName()); + return FieldType.map(keyType, valueType); + } else { + throw new RuntimeException("Cannot infer schema from unparameterized map."); + } } else if (type.isSubtypeOf(TypeDescriptor.of(CharSequence.class))) { return FieldType.STRING; } else if (type.isSubtypeOf(TypeDescriptor.of(ReadableInstant.class))) { @@ -187,22 +180,26 @@ private static Schema.FieldType fieldFromType( } else if (type.isSubtypeOf(TypeDescriptor.of(ByteBuffer.class))) { return FieldType.BYTES; } else if (type.isSubtypeOf(TypeDescriptor.of(Iterable.class))) { - FieldType elementType = - fieldFromType( - Preconditions.checkArgumentNotNull( - ReflectUtils.getIterableComponentType(type, boundTypes)), - fieldValueTypeSupplier, - alreadyVisitedSchemas, - boundTypes); - // TODO: should this be AbstractCollection? - if (type.isSubtypeOf(TypeDescriptor.of(Collection.class))) { - return FieldType.array(elementType); + TypeDescriptor> iterable = type.getSupertype(Iterable.class); + if (iterable.getType() instanceof ParameterizedType) { + ParameterizedType ptype = (ParameterizedType) iterable.getType(); + java.lang.reflect.Type[] params = ptype.getActualTypeArguments(); + checkArgument(params.length == 1); + // TODO: should this be AbstractCollection? + if (type.isSubtypeOf(TypeDescriptor.of(Collection.class))) { + return FieldType.array( + fieldFromType( + TypeDescriptor.of(params[0]), fieldValueTypeSupplier, alreadyVisitedSchemas)); + } else { + return FieldType.iterable( + fieldFromType( + TypeDescriptor.of(params[0]), fieldValueTypeSupplier, alreadyVisitedSchemas)); + } } else { - return FieldType.iterable(elementType); + throw new RuntimeException("Cannot infer schema from unparameterized collection."); } } else { - return FieldType.row( - schemaFromClass(type, fieldValueTypeSupplier, alreadyVisitedSchemas, boundTypes)); + return FieldType.row(schemaFromClass(type, fieldValueTypeSupplier, alreadyVisitedSchemas)); } } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java index 49fd2bfe2259c..d0ee623dea7cf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java @@ -28,7 +28,6 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.util.Map; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; @@ -40,7 +39,6 @@ import org.apache.beam.sdk.schemas.utils.SchemaTestUtils; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CaseFormat; import org.joda.time.DateTime; import org.joda.time.Instant; @@ -888,151 +886,4 @@ public void testSchema_SchemaFieldDescription() throws NoSuchSchemaException { assertEquals(FIELD_DESCRIPTION_SCHEMA.getField("lng"), schema.getField("lng")); assertEquals(FIELD_DESCRIPTION_SCHEMA.getField("str"), schema.getField("str")); } - - @AutoValue - @DefaultSchema(AutoValueSchema.class) - abstract static class ParameterizedAutoValue { - abstract W getValue1(); - - abstract T getValue2(); - - abstract V getValue3(); - - abstract X getValue4(); - } - - @Test - public void testAutoValueWithTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor> typeDescriptor = - new TypeDescriptor>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_SCHEMA) - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @DefaultSchema(AutoValueSchema.class) - abstract static class ParameterizedAutoValueSubclass - extends ParameterizedAutoValue { - abstract T getValue5(); - } - - @Test - public void testAutoValueWithInheritedTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor> typeDescriptor = - new TypeDescriptor>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_SCHEMA) - .addInt16Field("value5") - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @AutoValue - @DefaultSchema(AutoValueSchema.class) - abstract static class NestedParameterizedCollectionAutoValue { - abstract Iterable getNested(); - - abstract Map getMap(); - } - - @Test - public void testAutoValueWithNestedCollectionTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor< - NestedParameterizedCollectionAutoValue< - ParameterizedAutoValue, String>> - typeDescriptor = - new TypeDescriptor< - NestedParameterizedCollectionAutoValue< - ParameterizedAutoValue, String>>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedInnerSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_SCHEMA) - .build(); - final Schema expectedSchema = - Schema.builder() - .addIterableField("nested", FieldType.row(expectedInnerSchema)) - .addMapField("map", FieldType.STRING, FieldType.row(expectedInnerSchema)) - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @Test - public void testAutoValueWithDoublyNestedCollectionTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor< - NestedParameterizedCollectionAutoValue< - Iterable>, String>> - typeDescriptor = - new TypeDescriptor< - NestedParameterizedCollectionAutoValue< - Iterable>, - String>>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedInnerSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_SCHEMA) - .build(); - final Schema expectedSchema = - Schema.builder() - .addIterableField("nested", FieldType.iterable(FieldType.row(expectedInnerSchema))) - .addMapField( - "map", FieldType.STRING, FieldType.iterable(FieldType.row(expectedInnerSchema))) - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @AutoValue - @DefaultSchema(AutoValueSchema.class) - abstract static class NestedParameterizedAutoValue { - abstract T getNested(); - } - - @Test - public void testAutoValueWithNestedTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor< - NestedParameterizedAutoValue< - ParameterizedAutoValue>> - typeDescriptor = - new TypeDescriptor< - NestedParameterizedAutoValue< - ParameterizedAutoValue>>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedInnerSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_SCHEMA) - .build(); - final Schema expectedSchema = - Schema.builder().addRowField("nested", expectedInnerSchema).build(); - assertTrue(expectedSchema.equivalent(schema)); - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java index 2252c3aef0dba..5313feb5c6c0b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java @@ -68,7 +68,6 @@ import org.apache.beam.sdk.schemas.utils.TestJavaBeans.SimpleBeanWithAnnotations; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -626,127 +625,4 @@ public void testSetterConstructionWithRenamedFields() throws NoSuchSchemaExcepti assertEquals( registry.getFromRowFunction(BeanWithCaseFormat.class).apply(row), beanWithCaseFormat); } - - @Test - public void testBeanWithTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor> - typeDescriptor = - new TypeDescriptor< - TestJavaBeans.SimpleParameterizedBean>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_BEAN_SCHEMA) - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @Test - public void testBeanWithInheritedTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor> typeDescriptor = - new TypeDescriptor>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_BEAN_SCHEMA) - .addInt16Field("value5") - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @Test - public void testBeanWithNestedCollectionTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor< - TestJavaBeans.NestedParameterizedCollectionBean< - TestJavaBeans.SimpleParameterizedBean, String>> - typeDescriptor = - new TypeDescriptor< - TestJavaBeans.NestedParameterizedCollectionBean< - TestJavaBeans.SimpleParameterizedBean, - String>>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedInnerSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_BEAN_SCHEMA) - .build(); - final Schema expectedSchema = - Schema.builder() - .addIterableField("nested", Schema.FieldType.row(expectedInnerSchema)) - .addMapField("map", Schema.FieldType.STRING, Schema.FieldType.row(expectedInnerSchema)) - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @Test - public void testBeanWithDoublyNestedCollectionTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor< - TestJavaBeans.NestedParameterizedCollectionBean< - Iterable>, - String>> - typeDescriptor = - new TypeDescriptor< - TestJavaBeans.NestedParameterizedCollectionBean< - Iterable< - TestJavaBeans.SimpleParameterizedBean>, - String>>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedInnerSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_BEAN_SCHEMA) - .build(); - final Schema expectedSchema = - Schema.builder() - .addIterableField( - "nested", Schema.FieldType.iterable(Schema.FieldType.row(expectedInnerSchema))) - .addMapField( - "map", - Schema.FieldType.STRING, - Schema.FieldType.iterable(Schema.FieldType.row(expectedInnerSchema))) - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @Test - public void testBeanWithNestedTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor< - TestJavaBeans.NestedParameterizedBean< - TestJavaBeans.SimpleParameterizedBean>> - typeDescriptor = - new TypeDescriptor< - TestJavaBeans.NestedParameterizedBean< - TestJavaBeans.SimpleParameterizedBean>>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedInnerSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_BEAN_SCHEMA) - .build(); - final Schema expectedSchema = - Schema.builder().addRowField("nested", expectedInnerSchema).build(); - assertTrue(expectedSchema.equivalent(schema)); - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java index 70bc3030924be..11bef79b26f73 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java @@ -76,7 +76,6 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -782,123 +781,4 @@ public void testCircularNestedPOJOThrows() throws NoSuchSchemaException { thrown.getMessage(), containsString("TestPOJOs$FirstCircularNestedPOJO")); } - - @Test - public void testPojoWithTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor> - typeDescriptor = - new TypeDescriptor< - TestPOJOs.SimpleParameterizedPOJO>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_POJO_SCHEMA) - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @Test - public void testPojoWithInheritedTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor> typeDescriptor = - new TypeDescriptor>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_POJO_SCHEMA) - .addInt16Field("value5") - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @Test - public void testPojoWithNestedCollectionTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor< - TestPOJOs.NestedParameterizedCollectionPOJO< - TestPOJOs.SimpleParameterizedPOJO, String>> - typeDescriptor = - new TypeDescriptor< - TestPOJOs.NestedParameterizedCollectionPOJO< - TestPOJOs.SimpleParameterizedPOJO, - String>>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedInnerSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_POJO_SCHEMA) - .build(); - final Schema expectedSchema = - Schema.builder() - .addIterableField("nested", FieldType.row(expectedInnerSchema)) - .addMapField("map", FieldType.STRING, FieldType.row(expectedInnerSchema)) - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @Test - public void testPojoWithDoublyNestedCollectionTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor< - TestPOJOs.NestedParameterizedCollectionPOJO< - Iterable>, - String>> - typeDescriptor = - new TypeDescriptor< - TestPOJOs.NestedParameterizedCollectionPOJO< - Iterable>, - String>>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedInnerSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_POJO_SCHEMA) - .build(); - final Schema expectedSchema = - Schema.builder() - .addIterableField("nested", FieldType.iterable(FieldType.row(expectedInnerSchema))) - .addMapField( - "map", FieldType.STRING, FieldType.iterable(FieldType.row(expectedInnerSchema))) - .build(); - assertTrue(expectedSchema.equivalent(schema)); - } - - @Test - public void testPojoWithNestedTypeParameter() throws NoSuchSchemaException { - SchemaRegistry registry = SchemaRegistry.createDefault(); - TypeDescriptor< - TestPOJOs.NestedParameterizedPOJO< - TestPOJOs.SimpleParameterizedPOJO>> - typeDescriptor = - new TypeDescriptor< - TestPOJOs.NestedParameterizedPOJO< - TestPOJOs.SimpleParameterizedPOJO>>() {}; - Schema schema = registry.getSchema(typeDescriptor); - - final Schema expectedInnerSchema = - Schema.builder() - .addBooleanField("value1") - .addStringField("value2") - .addInt64Field("value3") - .addRowField("value4", SIMPLE_POJO_SCHEMA) - .build(); - final Schema expectedSchema = - Schema.builder().addRowField("nested", expectedInnerSchema).build(); - assertTrue(expectedSchema.equivalent(schema)); - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java index e0a45c2c82fea..021e39b848491 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java @@ -34,7 +34,6 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.util.Collections; import java.util.List; import org.apache.beam.sdk.schemas.FieldValueGetter; import org.apache.beam.sdk.schemas.FieldValueSetter; @@ -66,9 +65,7 @@ public class JavaBeanUtilsTest { public void testNullable() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, - GetterTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); assertTrue(schema.getField("str").getType().getNullable()); assertFalse(schema.getField("anInt").getType().getNullable()); } @@ -77,9 +74,7 @@ public void testNullable() { public void testSimpleBean() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, - GetterTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(SIMPLE_BEAN_SCHEMA, schema); } @@ -87,9 +82,7 @@ public void testSimpleBean() { public void testNestedBean() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, - GetterTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(NESTED_BEAN_SCHEMA, schema); } @@ -97,9 +90,7 @@ public void testNestedBean() { public void testPrimitiveArray() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, - GetterTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(PRIMITIVE_ARRAY_BEAN_SCHEMA, schema); } @@ -107,9 +98,7 @@ public void testPrimitiveArray() { public void testNestedArray() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, - GetterTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(NESTED_ARRAY_BEAN_SCHEMA, schema); } @@ -117,9 +106,7 @@ public void testNestedArray() { public void testNestedCollection() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, - GetterTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(NESTED_COLLECTION_BEAN_SCHEMA, schema); } @@ -127,9 +114,7 @@ public void testNestedCollection() { public void testPrimitiveMap() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, - GetterTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(PRIMITIVE_MAP_BEAN_SCHEMA, schema); } @@ -137,9 +122,7 @@ public void testPrimitiveMap() { public void testNestedMap() { Schema schema = JavaBeanUtils.schemaFromJavaBeanClass( - new TypeDescriptor() {}, - GetterTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, GetterTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(NESTED_MAP_BEAN_SCHEMA, schema); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java index 46c098dddaeb7..723353ed8d153 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java @@ -35,7 +35,6 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.util.Collections; import java.util.List; import org.apache.beam.sdk.schemas.FieldValueGetter; import org.apache.beam.sdk.schemas.FieldValueSetter; @@ -72,9 +71,7 @@ public class POJOUtilsTest { public void testNullables() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); assertTrue(schema.getField("str").getType().getNullable()); assertFalse(schema.getField("anInt").getType().getNullable()); } @@ -83,9 +80,7 @@ public void testNullables() { public void testSimplePOJO() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); assertEquals(SIMPLE_POJO_SCHEMA, schema); } @@ -93,9 +88,7 @@ public void testSimplePOJO() { public void testNestedPOJO() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(NESTED_POJO_SCHEMA, schema); } @@ -104,8 +97,7 @@ public void testNestedPOJOWithSimplePOJO() { Schema schema = POJOUtils.schemaFromPojoClass( new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap()); + JavaFieldTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(NESTED_POJO_WITH_SIMPLE_POJO_SCHEMA, schema); } @@ -113,9 +105,7 @@ public void testNestedPOJOWithSimplePOJO() { public void testPrimitiveArray() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(PRIMITIVE_ARRAY_POJO_SCHEMA, schema); } @@ -123,9 +113,7 @@ public void testPrimitiveArray() { public void testNestedArray() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(NESTED_ARRAY_POJO_SCHEMA, schema); } @@ -133,9 +121,7 @@ public void testNestedArray() { public void testNestedCollection() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(NESTED_COLLECTION_POJO_SCHEMA, schema); } @@ -143,9 +129,7 @@ public void testNestedCollection() { public void testPrimitiveMap() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(PRIMITIVE_MAP_POJO_SCHEMA, schema); } @@ -153,9 +137,7 @@ public void testPrimitiveMap() { public void testNestedMap() { Schema schema = POJOUtils.schemaFromPojoClass( - new TypeDescriptor() {}, - JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap()); + new TypeDescriptor() {}, JavaFieldTypeSupplier.INSTANCE); SchemaTestUtils.assertSchemaEquivalent(NESTED_MAP_POJO_SCHEMA, schema); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java index cbc9761449718..b5ad6f989d9e4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java @@ -1397,95 +1397,4 @@ public void setValue(@Nullable Float value) { Schema.Field.nullable("value", FieldType.FLOAT) .withDescription("This value is the value stored in the object as a float.")) .build(); - - @DefaultSchema(JavaBeanSchema.class) - public static class SimpleParameterizedBean { - @Nullable private W value1; - @Nullable private T value2; - @Nullable private V value3; - @Nullable private X value4; - - public W getValue1() { - return value1; - } - - public void setValue1(W value1) { - this.value1 = value1; - } - - public T getValue2() { - return value2; - } - - public void setValue2(T value2) { - this.value2 = value2; - } - - public V getValue3() { - return value3; - } - - public void setValue3(V value3) { - this.value3 = value3; - } - - public X getValue4() { - return value4; - } - - public void setValue4(X value4) { - this.value4 = value4; - } - } - - @DefaultSchema(JavaBeanSchema.class) - public static class SimpleParameterizedBeanSubclass - extends SimpleParameterizedBean { - @Nullable private T value5; - - public SimpleParameterizedBeanSubclass() {} - - public T getValue5() { - return value5; - } - - public void setValue5(T value5) { - this.value5 = value5; - } - } - - @DefaultSchema(JavaBeanSchema.class) - public static class NestedParameterizedCollectionBean { - private Iterable nested; - private Map map; - - public Iterable getNested() { - return nested; - } - - public Map getMap() { - return map; - } - - public void setNested(Iterable nested) { - this.nested = nested; - } - - public void setMap(Map map) { - this.map = map; - } - } - - @DefaultSchema(JavaBeanSchema.class) - public static class NestedParameterizedBean { - private T nested; - - public T getNested() { - return nested; - } - - public void setNested(T nested) { - this.nested = nested; - } - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java index ce7409365d09f..789de02adee89 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java @@ -495,125 +495,6 @@ public int hashCode() { .addStringField("stringBuilder") .build(); - @DefaultSchema(JavaFieldSchema.class) - public static class SimpleParameterizedPOJO { - public W value1; - public T value2; - public V value3; - public X value4; - - public SimpleParameterizedPOJO() {} - - public SimpleParameterizedPOJO(W value1, T value2, V value3, X value4) { - this.value1 = value1; - this.value2 = value2; - this.value3 = value3; - this.value4 = value4; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof SimpleParameterizedPOJO)) { - return false; - } - SimpleParameterizedPOJO that = (SimpleParameterizedPOJO) o; - return Objects.equals(value1, that.value1) - && Objects.equals(value2, that.value2) - && Objects.equals(value3, that.value3) - && Objects.equals(value4, that.value4); - } - - @Override - public int hashCode() { - return Objects.hash(value1, value2, value3, value4); - } - } - - @DefaultSchema(JavaFieldSchema.class) - public static class SimpleParameterizedPOJOSubclass - extends SimpleParameterizedPOJO { - public T value5; - - public SimpleParameterizedPOJOSubclass() {} - - public SimpleParameterizedPOJOSubclass(T value5) { - this.value5 = value5; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof SimpleParameterizedPOJOSubclass)) { - return false; - } - SimpleParameterizedPOJOSubclass that = (SimpleParameterizedPOJOSubclass) o; - return Objects.equals(value5, that.value5); - } - - @Override - public int hashCode() { - return Objects.hash(value4); - } - } - - @DefaultSchema(JavaFieldSchema.class) - public static class NestedParameterizedCollectionPOJO { - public Iterable nested; - public Map map; - - public NestedParameterizedCollectionPOJO(Iterable nested, Map map) { - this.nested = nested; - this.map = map; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof NestedParameterizedCollectionPOJO)) { - return false; - } - NestedParameterizedCollectionPOJO that = (NestedParameterizedCollectionPOJO) o; - return Objects.equals(nested, that.nested) && Objects.equals(map, that.map); - } - - @Override - public int hashCode() { - return Objects.hash(nested, map); - } - } - - @DefaultSchema(JavaFieldSchema.class) - public static class NestedParameterizedPOJO { - public T nested; - - public NestedParameterizedPOJO(T nested) { - this.nested = nested; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof NestedParameterizedPOJO)) { - return false; - } - NestedParameterizedPOJO that = (NestedParameterizedPOJO) o; - return Objects.equals(nested, that.nested); - } - - @Override - public int hashCode() { - return Objects.hash(nested); - } - } /** A POJO containing a nested class. * */ @DefaultSchema(JavaFieldSchema.class) public static class NestedPOJO { @@ -1006,7 +887,7 @@ public boolean equals(@Nullable Object o) { if (this == o) { return true; } - if (!(o instanceof PojoWithIterable)) { + if (!(o instanceof PojoWithNestedArray)) { return false; } PojoWithIterable that = (PojoWithIterable) o; diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroByteBuddyUtils.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroByteBuddyUtils.java index 1a530a3f6ca58..0a82663c1771c 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroByteBuddyUtils.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroByteBuddyUtils.java @@ -78,8 +78,8 @@ private static SchemaUserTypeCreator createCreator(Class clazz, Schema sc // Generate a method call to create and invoke the SpecificRecord's constructor. . MethodCall construct = MethodCall.construct(baseConstructor); - for (int i = 0; i < baseConstructor.getGenericParameterTypes().length; ++i) { - Type baseType = baseConstructor.getGenericParameterTypes()[i]; + for (int i = 0; i < baseConstructor.getParameterTypes().length; ++i) { + Class baseType = baseConstructor.getParameterTypes()[i]; construct = construct.with(readAndConvertParameter(baseType, i), baseType); } @@ -110,7 +110,7 @@ private static SchemaUserTypeCreator createCreator(Class clazz, Schema sc } private static StackManipulation readAndConvertParameter( - Type constructorParameterType, int index) { + Class constructorParameterType, int index) { TypeConversionsFactory typeConversionsFactory = new AvroUtils.AvroTypeConversionFactory(); // The types in the AVRO-generated constructor might be the types returned by Beam's Row class, diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java index 1324d254e44ea..1b1c459693074 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java @@ -814,9 +814,6 @@ public List get(TypeDescriptor typeDescriptor) { @Override public List get(TypeDescriptor typeDescriptor, Schema schema) { - Map boundTypes = - ReflectUtils.getAllBoundTypes(typeDescriptor); - Map mapping = getMapping(schema); List methods = ReflectUtils.getMethods(typeDescriptor.getRawType()); List types = Lists.newArrayList(); @@ -824,7 +821,7 @@ public List get(TypeDescriptor typeDescriptor, Sch Method method = methods.get(i); if (ReflectUtils.isGetter(method)) { FieldValueTypeInformation fieldValueTypeInformation = - FieldValueTypeInformation.forGetter(method, i, boundTypes); + FieldValueTypeInformation.forGetter(method, i); String name = mapping.get(fieldValueTypeInformation.getName()); if (name != null) { types.add(fieldValueTypeInformation.withName(name)); @@ -868,16 +865,13 @@ private Map getMapping(Schema schema) { private static final class AvroPojoFieldValueTypeSupplier implements FieldValueTypeSupplier { @Override public List get(TypeDescriptor typeDescriptor) { - Map boundTypes = - ReflectUtils.getAllBoundTypes(typeDescriptor); List classFields = ReflectUtils.getFields(typeDescriptor.getRawType()); Map types = Maps.newHashMap(); for (int i = 0; i < classFields.size(); ++i) { java.lang.reflect.Field f = classFields.get(i); if (!f.isAnnotationPresent(AvroIgnore.class)) { - FieldValueTypeInformation typeInformation = - FieldValueTypeInformation.forField(f, i, boundTypes); + FieldValueTypeInformation typeInformation = FieldValueTypeInformation.forField(f, i); AvroName avroname = f.getAnnotation(AvroName.class); if (avroname != null) { typeInformation = typeInformation.withName(avroname.value()); diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java index fcfc40403b43d..d159e9de44a8f 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java @@ -39,7 +39,6 @@ import java.lang.reflect.Modifier; import java.lang.reflect.Type; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -1046,8 +1045,7 @@ FieldValueSetter getProtoFieldValueSetter( } else { Method method = getProtoSetter(methods, field.getName(), field.getType()); return JavaBeanUtils.createSetter( - FieldValueTypeInformation.forSetter( - method, protoSetterPrefix(field.getType()), Collections.emptyMap()), + FieldValueTypeInformation.forSetter(method, protoSetterPrefix(field.getType())), new ProtoTypeConversionsFactory()); } } diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java index 4b8d51abdea6b..faf3ad407af5d 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java @@ -23,7 +23,6 @@ import com.google.protobuf.DynamicMessage; import com.google.protobuf.Message; import java.lang.reflect.Method; -import java.util.Collections; import java.util.List; import java.util.Map; import org.apache.beam.sdk.extensions.protobuf.ProtoByteBuddyUtils.ProtoTypeConversionsFactory; @@ -73,8 +72,7 @@ public List get(TypeDescriptor typeDescriptor, Sch Method method = getProtoGetter(methods, oneOfField.getName(), oneOfField.getType()); oneOfTypes.put( oneOfField.getName(), - FieldValueTypeInformation.forGetter(method, i, Collections.emptyMap()) - .withName(field.getName())); + FieldValueTypeInformation.forGetter(method, i).withName(field.getName())); } // Add an entry that encapsulates information about all possible getters. types.add( @@ -84,9 +82,7 @@ public List get(TypeDescriptor typeDescriptor, Sch } else { // This is a simple field. Add the getter. Method method = getProtoGetter(methods, field.getName(), field.getType()); - types.add( - FieldValueTypeInformation.forGetter(method, i, Collections.emptyMap()) - .withName(field.getName())); + types.add(FieldValueTypeInformation.forGetter(method, i).withName(field.getName())); } } return types; diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java index 64f600903d87b..d5f1745a9a2ce 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java @@ -25,7 +25,6 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -390,8 +389,7 @@ private Schema generateSchemaDirectly( fieldName, StaticSchemaInference.fieldFromType( TypeDescriptor.of(field.getClass()), - JavaFieldSchema.JavaFieldTypeSupplier.INSTANCE, - Collections.emptyMap())); + JavaFieldSchema.JavaFieldTypeSupplier.INSTANCE)); } counter++; diff --git a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftSchema.java b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftSchema.java index 73b3709da8323..5f4e195f227fb 100644 --- a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftSchema.java +++ b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftSchema.java @@ -242,11 +242,10 @@ private FieldValueTypeInformation fieldValueTypeInfo(Class type, String field if (factoryMethods.size() > 1) { throw new IllegalStateException("Overloaded factory methods: " + factoryMethods); } - return FieldValueTypeInformation.forSetter(factoryMethods.get(0), "", Collections.emptyMap()); + return FieldValueTypeInformation.forSetter(factoryMethods.get(0), ""); } else { try { - return FieldValueTypeInformation.forField( - type.getDeclaredField(fieldName), 0, Collections.emptyMap()); + return FieldValueTypeInformation.forField(type.getDeclaredField(fieldName), 0); } catch (NoSuchFieldException e) { throw new IllegalArgumentException(e); }