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 7bff2450b853..78808fdc10c8 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 @@ -18,7 +18,6 @@ package org.apache.beam.sdk.schemas.utils; import static org.apache.beam.sdk.util.ByteBuddyUtils.getClassLoadingStrategy; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; @@ -63,9 +62,9 @@ import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.InjectPackageStrategy; import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversion; import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversionsFactory; +import org.apache.beam.sdk.util.Preconditions; 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.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; @@ -189,7 +188,7 @@ private static boolean matchConstructor( Collectors.toMap( f -> ReflectUtils.stripGetterPrefix( - Preconditions.checkNotNull( + Preconditions.checkArgumentNotNull( f.getMethod(), JavaBeanUtils.GETTER_WITH_NULL_METHOD_ERROR) .getName()), Function.identity())); @@ -249,7 +248,7 @@ private static boolean matchConstructor( for (FieldValueTypeInformation type : schemaTypes) { String autoValueFieldName = ReflectUtils.stripGetterPrefix( - Preconditions.checkNotNull( + Preconditions.checkArgumentNotNull( type.getMethod(), JavaBeanUtils.GETTER_WITH_NULL_METHOD_ERROR) .getName()); @@ -347,11 +346,10 @@ public ByteCodeAppender appender(final Target implementationTarget) { TypeConversion convertType = typeConversionsFactory.createTypeConversion(true); for (int i = 0; i < setters.size(); ++i) { - Method setterMethod = checkNotNull(setters.get(i).getMethod()); - Parameter parameter = setterMethod.getParameters()[0]; + FieldValueTypeInformation setterType = setters.get(i); + Method setterMethod = Preconditions.checkStateNotNull(setterType.getMethod()); ForLoadedType convertedType = - new ForLoadedType( - (Class) convertType.convert(TypeDescriptor.of(parameter.getParameterizedType()))); + new ForLoadedType((Class) convertType.convert(setterType.getType())); StackManipulation readParameter = new StackManipulation.Compound( @@ -366,7 +364,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { Duplication.SINGLE, typeConversionsFactory .createSetterConversions(readParameter) - .convert(TypeDescriptor.of(parameter.getType())), + .convert(setterType.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 e99459ddc60a..8bc6c99ca5c6 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 @@ -78,11 +78,11 @@ import org.apache.beam.sdk.schemas.FieldValueHaver; import org.apache.beam.sdk.schemas.FieldValueSetter; import org.apache.beam.sdk.schemas.FieldValueTypeInformation; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeParameter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Verify; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Collections2; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -275,7 +275,7 @@ public TypeConversion createSetterConversions(StackManipulati public abstract static class TypeConversion { public T convert(TypeDescriptor typeDescriptor) { if (typeDescriptor.isArray() - && !Preconditions.checkNotNull(typeDescriptor.getComponentType()) + && !Preconditions.checkArgumentNotNull(typeDescriptor.getComponentType()) .getRawType() .equals(byte.class)) { // Byte arrays are special, so leave those alone. @@ -363,7 +363,7 @@ protected ConvertType(boolean returnRawTypes) { @Override protected Type convertArray(TypeDescriptor type) { TypeDescriptor ret = - createCollectionType(Preconditions.checkNotNull(type.getComponentType())); + createCollectionType(Preconditions.checkArgumentNotNull(type.getComponentType())); return returnRawTypes ? ret.getRawType() : ret.getType(); } @@ -371,7 +371,7 @@ protected Type convertArray(TypeDescriptor type) { protected Type convertCollection(TypeDescriptor type) { TypeDescriptor ret = createCollectionType( - Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type))); + Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type))); return returnRawTypes ? ret.getRawType() : ret.getType(); } @@ -379,7 +379,7 @@ protected Type convertCollection(TypeDescriptor type) { protected Type convertList(TypeDescriptor type) { TypeDescriptor ret = createCollectionType( - Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type))); + Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type))); return returnRawTypes ? ret.getRawType() : ret.getType(); } @@ -387,7 +387,7 @@ protected Type convertList(TypeDescriptor type) { protected Type convertIterable(TypeDescriptor type) { TypeDescriptor ret = createIterableType( - Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type))); + Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type))); return returnRawTypes ? ret.getRawType() : ret.getType(); } @@ -426,12 +426,20 @@ protected Type convertDefault(TypeDescriptor type) { return returnRawTypes ? type.getRawType() : type.getType(); } + public static TypeDescriptor primitiveToWrapper(TypeDescriptor typeDescriptor) { + Class cls = typeDescriptor.getRawType(); + if (cls.isPrimitive()) { + return TypeDescriptor.of(ClassUtils.primitiveToWrapper(cls)); + } else { + return typeDescriptor; + } + } + @SuppressWarnings("unchecked") private TypeDescriptor> createCollectionType( TypeDescriptor componentType) { TypeDescriptor wrappedComponentType = - (TypeDescriptor) - TypeDescriptor.of(ClassUtils.primitiveToWrapper(componentType.getRawType())); + (TypeDescriptor) primitiveToWrapper(componentType); return new TypeDescriptor>() {}.where( new TypeParameter() {}, wrappedComponentType); } @@ -440,8 +448,7 @@ private TypeDescriptor> createCollectionType( private TypeDescriptor> createIterableType( TypeDescriptor componentType) { TypeDescriptor wrappedComponentType = - (TypeDescriptor) - TypeDescriptor.of(ClassUtils.primitiveToWrapper(componentType.getRawType())); + (TypeDescriptor) primitiveToWrapper(componentType); return new TypeDescriptor>() {}.where( new TypeParameter() {}, wrappedComponentType); } @@ -670,12 +677,12 @@ protected StackManipulation convertArray(TypeDescriptor type) { // return isComponentTypePrimitive ? Arrays.asList(ArrayUtils.toObject(value)) // : Arrays.asList(value); - TypeDescriptor componentType = Preconditions.checkNotNull(type.getComponentType()); + TypeDescriptor componentType = Preconditions.checkArgumentNotNull(type.getComponentType()); ForLoadedType loadedArrayType = new ForLoadedType(type.getRawType()); StackManipulation readArrayValue = readValue; // Row always expects to get an Iterable back for array types. Wrap this array into a // List using Arrays.asList before returning. - if (Preconditions.checkNotNull(loadedArrayType.getComponentType()).isPrimitive()) { + if (Preconditions.checkArgumentNotNull(loadedArrayType.getComponentType()).isPrimitive()) { // Arrays.asList doesn't take primitive arrays, so convert first using ArrayUtils.toObject. readArrayValue = new Compound( @@ -723,7 +730,7 @@ protected StackManipulation convertArray(TypeDescriptor type) { @Override protected StackManipulation convertIterable(TypeDescriptor type) { TypeDescriptor componentType = - Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type)); + Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type)); Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType); final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType); @@ -744,7 +751,7 @@ protected StackManipulation convertIterable(TypeDescriptor type) { @Override protected StackManipulation convertCollection(TypeDescriptor type) { TypeDescriptor componentType = - Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type)); + Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type)); Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType); final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType); if (!finalComponentType.hasUnresolvedParameters()) { @@ -764,7 +771,7 @@ protected StackManipulation convertCollection(TypeDescriptor type) { @Override protected StackManipulation convertList(TypeDescriptor type) { TypeDescriptor componentType = - Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type)); + Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type)); Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType); final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType); if (!finalComponentType.hasUnresolvedParameters()) { @@ -1017,7 +1024,7 @@ protected StackManipulation convertArray(TypeDescriptor type) { .build() .asErasure(); - TypeDescriptor componentType = Preconditions.checkNotNull(type.getComponentType()); + TypeDescriptor componentType = Preconditions.checkArgumentNotNull(type.getComponentType()); Type rowElementType = getFactory().createTypeConversion(false).convert(componentType); final TypeDescriptor arrayElementType = ReflectUtils.boxIfPrimitive(componentType); StackManipulation readTransformedValue = readValue; @@ -1076,7 +1083,7 @@ protected StackManipulation convertArray(TypeDescriptor type) { @Override protected StackManipulation convertIterable(TypeDescriptor type) { final TypeDescriptor iterableElementType = - Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type)); + Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type)); Type rowElementType = getFactory().createTypeConversion(false).convert(iterableElementType); if (!iterableElementType.hasUnresolvedParameters()) { ForLoadedType conversionFunction = @@ -1096,7 +1103,7 @@ protected StackManipulation convertIterable(TypeDescriptor type) { @Override protected StackManipulation convertCollection(TypeDescriptor type) { final TypeDescriptor collectionElementType = - Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type)); + Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type)); Type rowElementType = getFactory().createTypeConversion(false).convert(collectionElementType); if (!collectionElementType.hasUnresolvedParameters()) { @@ -1117,7 +1124,7 @@ protected StackManipulation convertCollection(TypeDescriptor type) { @Override protected StackManipulation convertList(TypeDescriptor type) { final TypeDescriptor collectionElementType = - Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type)); + Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type)); Type rowElementType = getFactory().createTypeConversion(false).convert(collectionElementType); StackManipulation readTrasformedValue = readValue; @@ -1147,9 +1154,9 @@ protected StackManipulation convertList(TypeDescriptor type) { @Override protected StackManipulation convertMap(TypeDescriptor type) { final TypeDescriptor keyElementType = - Preconditions.checkNotNull(ReflectUtils.getMapType(type, 0)); + Preconditions.checkArgumentNotNull(ReflectUtils.getMapType(type, 0)); final TypeDescriptor valueElementType = - Preconditions.checkNotNull(ReflectUtils.getMapType(type, 1)); + Preconditions.checkArgumentNotNull(ReflectUtils.getMapType(type, 1)); Type rowKeyType = getFactory().createTypeConversion(false).convert(keyElementType); Type rowValueType = getFactory().createTypeConversion(false).convert(valueElementType); @@ -1510,17 +1517,17 @@ public ByteCodeAppender appender(final Target implementationTarget) { // Push all creator parameters on the stack. TypeConversion convertType = typeConversionsFactory.createTypeConversion(true); for (int i = 0; i < parameters.size(); i++) { - Parameter parameter = parameters.get(i); + FieldValueTypeInformation fieldType = + fields.get(Preconditions.checkStateNotNull(fieldMapping.get(i))); ForLoadedType convertedType = - new ForLoadedType( - (Class) convertType.convert(TypeDescriptor.of(parameter.getType()))); + new ForLoadedType((Class) convertType.convert(fieldType.getType())); // The instruction to read the parameter. Use the fieldMapping to reorder parameters as // necessary. StackManipulation readParameter = new StackManipulation.Compound( MethodVariableAccess.REFERENCE.loadFrom(1), - IntegerConstant.forValue(Preconditions.checkNotNull(fieldMapping.get(i))), + IntegerConstant.forValue(Preconditions.checkStateNotNull(fieldMapping.get(i))), ArrayAccess.REFERENCE.load(), TypeCasting.to(convertedType)); stackManipulation = @@ -1528,7 +1535,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { stackManipulation, typeConversionsFactory .createSetterConversions(readParameter) - .convert(TypeDescriptor.of(parameter.getParameterizedType()))); + .convert(fieldType.getType())); } stackManipulation = new StackManipulation.Compound( 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 32b4ef97b70e..10f465787216 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 @@ -31,6 +31,7 @@ import net.bytebuddy.ByteBuddy; import net.bytebuddy.asm.AsmVisitorWrapper; import net.bytebuddy.description.method.MethodDescription.ForLoadedMethod; +import net.bytebuddy.description.type.TypeDescription; import net.bytebuddy.dynamic.DynamicType; import net.bytebuddy.dynamic.scaffold.InstrumentedType; import net.bytebuddy.implementation.FixedValue; @@ -39,6 +40,7 @@ import net.bytebuddy.implementation.bytecode.ByteCodeAppender.Size; import net.bytebuddy.implementation.bytecode.Removal; import net.bytebuddy.implementation.bytecode.StackManipulation; +import net.bytebuddy.implementation.bytecode.assign.TypeCasting; import net.bytebuddy.implementation.bytecode.member.MethodInvocation; import net.bytebuddy.implementation.bytecode.member.MethodReturn; import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; @@ -55,9 +57,9 @@ import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.StaticFactoryMethodInstruction; import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversionsFactory; import org.apache.beam.sdk.schemas.utils.ReflectUtils.TypeDescriptorWithSchema; +import org.apache.beam.sdk.util.Preconditions; 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.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; @@ -101,7 +103,8 @@ public static void validateJavaBean( for (FieldValueTypeInformation type : getters) { FieldValueTypeInformation setterType = setterMap.get(type.getName()); - Method m = Preconditions.checkNotNull(type.getMethod(), GETTER_WITH_NULL_METHOD_ERROR); + Method m = + Preconditions.checkArgumentNotNull(type.getMethod(), GETTER_WITH_NULL_METHOD_ERROR); if (setterType == null) { throw new RuntimeException( String.format( @@ -172,7 +175,8 @@ FieldValueGetter createGetter( FieldValueTypeInformation typeInformation, TypeConversionsFactory typeConversionsFactory) { final Method m = - Preconditions.checkNotNull(typeInformation.getMethod(), GETTER_WITH_NULL_METHOD_ERROR); + Preconditions.checkArgumentNotNull( + typeInformation.getMethod(), GETTER_WITH_NULL_METHOD_ERROR); DynamicType.Builder> builder = ByteBuddyUtils.subclassGetterInterface( BYTE_BUDDY, @@ -239,7 +243,8 @@ public static List getSetters( public static FieldValueSetter createSetter( FieldValueTypeInformation typeInformation, TypeConversionsFactory typeConversionsFactory) { final Method m = - Preconditions.checkNotNull(typeInformation.getMethod(), SETTER_WITH_NULL_METHOD_ERROR); + Preconditions.checkArgumentNotNull( + typeInformation.getMethod(), SETTER_WITH_NULL_METHOD_ERROR); DynamicType.Builder> builder = ByteBuddyUtils.subclassSetterInterface( BYTE_BUDDY, @@ -439,6 +444,14 @@ public ByteCodeAppender appender(final Target implementationTarget) { return (methodVisitor, implementationContext, instrumentedMethod) -> { // this + method parameters. int numLocals = 1 + instrumentedMethod.getParameters().size(); + StackManipulation cast = + typeInformation + .getRawType() + .isAssignableFrom( + Preconditions.checkStateNotNull(typeInformation.getMethod()) + .getReturnType()) + ? StackManipulation.Trivial.INSTANCE + : TypeCasting.to(TypeDescription.ForLoadedType.of(typeInformation.getRawType())); // StackManipulation that will read the value from the class field. StackManipulation readValue = @@ -448,8 +461,9 @@ public ByteCodeAppender appender(final Target implementationTarget) { // Invoke the getter MethodInvocation.invoke( new ForLoadedMethod( - Preconditions.checkNotNull( - typeInformation.getMethod(), GETTER_WITH_NULL_METHOD_ERROR)))); + Preconditions.checkStateNotNull( + typeInformation.getMethod(), GETTER_WITH_NULL_METHOD_ERROR))), + cast); StackManipulation stackManipulation = new StackManipulation.Compound( @@ -492,7 +506,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { StackManipulation readField = MethodVariableAccess.REFERENCE.loadFrom(2); Method method = - Preconditions.checkNotNull( + Preconditions.checkStateNotNull( fieldValueTypeInformation.getMethod(), SETTER_WITH_NULL_METHOD_ERROR); boolean setterMethodReturnsVoid = method.getReturnType().equals(Void.TYPE); // Read the object onto the stack. 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 8e33d321a1c6..3aac12a9169b 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 @@ -30,6 +30,7 @@ import net.bytebuddy.ByteBuddy; import net.bytebuddy.asm.AsmVisitorWrapper; import net.bytebuddy.description.field.FieldDescription.ForLoadedField; +import net.bytebuddy.description.type.TypeDescription; import net.bytebuddy.description.type.TypeDescription.ForLoadedType; import net.bytebuddy.dynamic.DynamicType; import net.bytebuddy.dynamic.scaffold.InstrumentedType; @@ -151,18 +152,13 @@ private static SchemaUserTypeCreator createSetFieldCreator( Schema schema, List types, TypeConversionsFactory typeConversionsFactory) { - // Get the list of class fields ordered by schema. - List fields = - types.stream() - .map(type -> Preconditions.checkNotNull(type.getField())) - .collect(Collectors.toList()); try { DynamicType.Builder builder = BYTE_BUDDY .with(new InjectPackageStrategy(clazz)) .subclass(SchemaUserTypeCreator.class) .method(ElementMatchers.named("create")) - .intercept(new SetFieldCreateInstruction(fields, clazz, typeConversionsFactory)); + .intercept(new SetFieldCreateInstruction(types, clazz, typeConversionsFactory)); return builder .visit(new AsmVisitorWrapper.ForDeclaredMethods().writerFlags(ClassWriter.COMPUTE_FRAMES)) @@ -305,11 +301,8 @@ public static SchemaUserTypeCreator createStaticCreator( ByteBuddyUtils.subclassGetterInterface( BYTE_BUDDY, field.getDeclaringClass(), - typeConversionsFactory - .createTypeConversion(false) - .convert(TypeDescriptor.of(field.getType()))); - builder = - implementGetterMethods(builder, field, typeInformation.getName(), typeConversionsFactory); + typeConversionsFactory.createTypeConversion(false).convert(typeInformation.getType())); + builder = implementGetterMethods(builder, typeInformation, typeConversionsFactory); try { return builder .visit(new AsmVisitorWrapper.ForDeclaredMethods().writerFlags(ClassWriter.COMPUTE_FRAMES)) @@ -331,107 +324,25 @@ public static SchemaUserTypeCreator createStaticCreator( private static DynamicType.Builder> implementGetterMethods( DynamicType.Builder> builder, - Field field, - String name, + FieldValueTypeInformation typeInformation, TypeConversionsFactory typeConversionsFactory) { return builder .visit(new AsmVisitorWrapper.ForDeclaredMethods().writerFlags(ClassWriter.COMPUTE_FRAMES)) .method(ElementMatchers.named("name")) - .intercept(FixedValue.reference(name)) + .intercept(FixedValue.reference(typeInformation.getName())) .method(ElementMatchers.named("get")) - .intercept(new ReadFieldInstruction(field, typeConversionsFactory)); - } - - // The list of setters for a class is cached, so we only create the classes the first time - // getSetters is called. - private static final Map, List>> - CACHED_SETTERS = Maps.newConcurrentMap(); - - public static List> getSetters( - TypeDescriptor typeDescriptor, - Schema schema, - FieldValueTypeSupplier fieldValueTypeSupplier, - TypeConversionsFactory typeConversionsFactory) { - // Return the setters, ordered by their position in the schema. - return (List) - CACHED_SETTERS.computeIfAbsent( - TypeDescriptorWithSchema.create(typeDescriptor, schema), - c -> { - List types = - fieldValueTypeSupplier.get(typeDescriptor, schema); - return types.stream() - .map(t -> createSetter(t, typeConversionsFactory)) - .collect(Collectors.toList()); - }); - } - - /** - * Generate the following {@link FieldValueSetter} class for the {@link Field}. - * - *

-   *   class Setter implements {@literal FieldValueSetter} {
-   *     {@literal @}Override public String name() { return field.getName(); }
-   *     {@literal @}Override public Class type() { return field.getType(); }
-   *     {@literal @}Override public Type elementType() { return elementType; }
-   *     {@literal @}Override public Type mapKeyType() { return mapKeyType; }
-   *     {@literal @}Override public Type mapValueType() { return mapValueType; }
-   *     {@literal @}Override public void set(POJO pojo, FieldType value) {
-   *        pojo.field = convert(value);
-   *      }
-   *   }
-   * 
- */ - @SuppressWarnings("unchecked") - private static FieldValueSetter createSetter( - FieldValueTypeInformation typeInformation, TypeConversionsFactory typeConversionsFactory) { - Field field = Preconditions.checkNotNull(typeInformation.getField()); - DynamicType.Builder> builder = - ByteBuddyUtils.subclassSetterInterface( - BYTE_BUDDY, - field.getDeclaringClass(), - typeConversionsFactory - .createTypeConversion(false) - .convert(TypeDescriptor.of(field.getType()))); - builder = implementSetterMethods(builder, field, typeConversionsFactory); - try { - return builder - .visit(new AsmVisitorWrapper.ForDeclaredMethods().writerFlags(ClassWriter.COMPUTE_FRAMES)) - .make() - .load( - ReflectHelpers.findClassLoader(field.getDeclaringClass().getClassLoader()), - getClassLoadingStrategy(field.getDeclaringClass())) - .getLoaded() - .getDeclaredConstructor() - .newInstance(); - } catch (InstantiationException - | IllegalAccessException - | NoSuchMethodException - | InvocationTargetException e) { - throw new RuntimeException("Unable to generate a getter for field '" + field + "'.", e); - } - } - - private static - DynamicType.Builder> implementSetterMethods( - DynamicType.Builder> builder, - Field field, - TypeConversionsFactory typeConversionsFactory) { - return builder - .visit(new AsmVisitorWrapper.ForDeclaredMethods().writerFlags(ClassWriter.COMPUTE_FRAMES)) - .method(ElementMatchers.named("name")) - .intercept(FixedValue.reference(field.getName())) - .method(ElementMatchers.named("set")) - .intercept(new SetFieldInstruction(field, typeConversionsFactory)); + .intercept(new ReadFieldInstruction(typeInformation, typeConversionsFactory)); } // Implements a method to read a public field out of an object. static class ReadFieldInstruction implements Implementation { // Field that will be read. - private final Field field; + private final FieldValueTypeInformation typeInformation; private final TypeConversionsFactory typeConversionsFactory; - ReadFieldInstruction(Field field, TypeConversionsFactory typeConversionsFactory) { - this.field = field; + ReadFieldInstruction( + FieldValueTypeInformation typeInformation, TypeConversionsFactory typeConversionsFactory) { + this.typeInformation = typeInformation; this.typeConversionsFactory = typeConversionsFactory; } @@ -446,19 +357,25 @@ public ByteCodeAppender appender(final Target implementationTarget) { // this + method parameters. int numLocals = 1 + instrumentedMethod.getParameters().size(); + StackManipulation cast = + typeInformation.getRawType().isAssignableFrom(typeInformation.getField().getType()) + ? StackManipulation.Trivial.INSTANCE + : TypeCasting.to(TypeDescription.ForLoadedType.of(typeInformation.getRawType())); + // StackManipulation that will read the value from the class field. StackManipulation readValue = new StackManipulation.Compound( // Method param is offset 1 (offset 0 is the this parameter). MethodVariableAccess.REFERENCE.loadFrom(1), // Read the field from the object. - FieldAccess.forField(new ForLoadedField(field)).read()); + FieldAccess.forField(new ForLoadedField(typeInformation.getField())).read(), + cast); StackManipulation stackManipulation = new StackManipulation.Compound( typeConversionsFactory .createGetterConversions(readValue) - .convert(TypeDescriptor.of(field.getGenericType())), + .convert(typeInformation.getType()), MethodReturn.REFERENCE); StackManipulation.Size size = stackManipulation.apply(methodVisitor, implementationContext); @@ -513,13 +430,15 @@ public ByteCodeAppender appender(final Target implementationTarget) { // Implements a method to construct an object. static class SetFieldCreateInstruction implements Implementation { - private final List fields; + private final List typeInformations; private final Class pojoClass; private final TypeConversionsFactory typeConversionsFactory; SetFieldCreateInstruction( - List fields, Class pojoClass, TypeConversionsFactory typeConversionsFactory) { - this.fields = fields; + List typeInformations, + Class pojoClass, + TypeConversionsFactory typeConversionsFactory) { + this.typeInformations = typeInformations; this.pojoClass = pojoClass; this.typeConversionsFactory = typeConversionsFactory; } @@ -551,11 +470,12 @@ public ByteCodeAppender appender(final Target implementationTarget) { // The types in the POJO might be the types returned by Beam's Row class, // so we have to convert the types used by Beam's Row class. TypeConversion convertType = typeConversionsFactory.createTypeConversion(true); - for (int i = 0; i < fields.size(); ++i) { - Field field = fields.get(i); + for (int i = 0; i < typeInformations.size(); ++i) { + FieldValueTypeInformation typeInformation = typeInformations.get(i); + Field field = typeInformation.getField(); ForLoadedType convertedType = - new ForLoadedType((Class) convertType.convert(TypeDescriptor.of(field.getType()))); + new ForLoadedType((Class) convertType.convert(typeInformation.getType())); // The instruction to read the parameter. StackManipulation readParameter = @@ -572,7 +492,7 @@ public ByteCodeAppender appender(final Target implementationTarget) { // Do any conversions necessary. typeConversionsFactory .createSetterConversions(readParameter) - .convert(TypeDescriptor.of(field.getType())), + .convert(typeInformation.getType()), // Now update the field. FieldAccess.forField(new ForLoadedField(field)).write()); stackManipulation = new StackManipulation.Compound(stackManipulation, updateField); 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 378cdc06805f..6b9fbcd30a27 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 @@ -23,7 +23,6 @@ import static org.apache.beam.sdk.schemas.utils.TestPOJOs.NESTED_POJO_SCHEMA; import static org.apache.beam.sdk.schemas.utils.TestPOJOs.NESTED_POJO_WITH_SIMPLE_POJO_SCHEMA; import static org.apache.beam.sdk.schemas.utils.TestPOJOs.POJO_WITH_BOXED_FIELDS_SCHEMA; -import static org.apache.beam.sdk.schemas.utils.TestPOJOs.POJO_WITH_BYTE_ARRAY_SCHEMA; import static org.apache.beam.sdk.schemas.utils.TestPOJOs.PRIMITIVE_ARRAY_POJO_SCHEMA; import static org.apache.beam.sdk.schemas.utils.TestPOJOs.PRIMITIVE_MAP_POJO_SCHEMA; import static org.apache.beam.sdk.schemas.utils.TestPOJOs.SIMPLE_POJO_SCHEMA; @@ -37,7 +36,6 @@ import java.nio.charset.StandardCharsets; import java.util.List; import org.apache.beam.sdk.schemas.FieldValueGetter; -import org.apache.beam.sdk.schemas.FieldValueSetter; import org.apache.beam.sdk.schemas.JavaFieldSchema.JavaFieldTypeSupplier; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.DefaultTypeConversionsFactory; @@ -46,7 +44,6 @@ import org.apache.beam.sdk.schemas.utils.TestPOJOs.NestedMapPOJO; import org.apache.beam.sdk.schemas.utils.TestPOJOs.NestedPOJO; import org.apache.beam.sdk.schemas.utils.TestPOJOs.POJOWithBoxedFields; -import org.apache.beam.sdk.schemas.utils.TestPOJOs.POJOWithByteArray; import org.apache.beam.sdk.schemas.utils.TestPOJOs.POJOWithNullables; import org.apache.beam.sdk.schemas.utils.TestPOJOs.PrimitiveArrayPOJO; import org.apache.beam.sdk.schemas.utils.TestPOJOs.PrimitiveMapPOJO; @@ -182,44 +179,6 @@ public void testGeneratedSimpleGetters() { assertEquals("stringBuilder", getters.get(11).get(simplePojo)); } - @Test - public void testGeneratedSimpleSetters() { - SimplePOJO simplePojo = new SimplePOJO(); - List> setters = - POJOUtils.getSetters( - new TypeDescriptor() {}, - SIMPLE_POJO_SCHEMA, - JavaFieldTypeSupplier.INSTANCE, - new DefaultTypeConversionsFactory()); - assertEquals(12, setters.size()); - - setters.get(0).set(simplePojo, "field1"); - setters.get(1).set(simplePojo, (byte) 41); - setters.get(2).set(simplePojo, (short) 42); - setters.get(3).set(simplePojo, (int) 43); - setters.get(4).set(simplePojo, (long) 44); - setters.get(5).set(simplePojo, true); - setters.get(6).set(simplePojo, DATE.toInstant()); - setters.get(7).set(simplePojo, INSTANT); - setters.get(8).set(simplePojo, BYTE_ARRAY); - setters.get(9).set(simplePojo, BYTE_BUFFER.array()); - setters.get(10).set(simplePojo, new BigDecimal(42)); - setters.get(11).set(simplePojo, "stringBuilder"); - - assertEquals("field1", simplePojo.str); - assertEquals((byte) 41, simplePojo.aByte); - assertEquals((short) 42, simplePojo.aShort); - assertEquals((int) 43, simplePojo.anInt); - assertEquals((long) 44, simplePojo.aLong); - assertTrue(simplePojo.aBoolean); - assertEquals(DATE, simplePojo.dateTime); - assertEquals(INSTANT, simplePojo.instant); - assertArrayEquals("Unexpected bytes", BYTE_ARRAY, simplePojo.bytes); - assertEquals(BYTE_BUFFER, simplePojo.byteBuffer); - assertEquals(new BigDecimal(42), simplePojo.bigDecimal); - assertEquals("stringBuilder", simplePojo.stringBuilder.toString()); - } - @Test public void testGeneratedSimpleBoxedGetters() { POJOWithBoxedFields pojo = new POJOWithBoxedFields((byte) 41, (short) 42, 43, 44L, true); @@ -236,43 +195,4 @@ public void testGeneratedSimpleBoxedGetters() { assertEquals((long) 44, getters.get(3).get(pojo)); assertTrue((Boolean) getters.get(4).get(pojo)); } - - @Test - public void testGeneratedSimpleBoxedSetters() { - POJOWithBoxedFields pojo = new POJOWithBoxedFields(); - List> setters = - POJOUtils.getSetters( - new TypeDescriptor() {}, - POJO_WITH_BOXED_FIELDS_SCHEMA, - JavaFieldTypeSupplier.INSTANCE, - new DefaultTypeConversionsFactory()); - - setters.get(0).set(pojo, (byte) 41); - setters.get(1).set(pojo, (short) 42); - setters.get(2).set(pojo, (int) 43); - setters.get(3).set(pojo, (long) 44); - setters.get(4).set(pojo, true); - - assertEquals((byte) 41, pojo.aByte.byteValue()); - assertEquals((short) 42, pojo.aShort.shortValue()); - assertEquals((int) 43, pojo.anInt.intValue()); - assertEquals((long) 44, pojo.aLong.longValue()); - assertTrue(pojo.aBoolean.booleanValue()); - } - - @Test - public void testGeneratedByteBufferSetters() { - POJOWithByteArray pojo = new POJOWithByteArray(); - List> setters = - POJOUtils.getSetters( - new TypeDescriptor() {}, - POJO_WITH_BYTE_ARRAY_SCHEMA, - JavaFieldTypeSupplier.INSTANCE, - new DefaultTypeConversionsFactory()); - setters.get(0).set(pojo, BYTE_ARRAY); - setters.get(1).set(pojo, BYTE_BUFFER.array()); - - assertArrayEquals("not equal", BYTE_ARRAY, pojo.bytes1); - assertEquals(BYTE_BUFFER, pojo.bytes2); - } }