Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -347,11 +347,10 @@ public ByteCodeAppender appender(final Target implementationTarget) {

TypeConversion<Type> 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 = checkNotNull(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(
Expand All @@ -366,7 +365,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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 <ElementT> TypeDescriptor<Collection<ElementT>> createCollectionType(
TypeDescriptor<?> componentType) {
TypeDescriptor<ElementT> wrappedComponentType =
(TypeDescriptor<ElementT>)
TypeDescriptor.of(ClassUtils.primitiveToWrapper(componentType.getRawType()));
(TypeDescriptor<ElementT>) primitiveToWrapper(componentType);
return new TypeDescriptor<Collection<ElementT>>() {}.where(
new TypeParameter<ElementT>() {}, wrappedComponentType);
}
Expand All @@ -440,8 +448,7 @@ private <ElementT> TypeDescriptor<Collection<ElementT>> createCollectionType(
private <ElementT> TypeDescriptor<Iterable<ElementT>> createIterableType(
TypeDescriptor<?> componentType) {
TypeDescriptor<ElementT> wrappedComponentType =
(TypeDescriptor<ElementT>)
TypeDescriptor.of(ClassUtils.primitiveToWrapper(componentType.getRawType()));
(TypeDescriptor<ElementT>) primitiveToWrapper(componentType);
return new TypeDescriptor<Iterable<ElementT>>() {}.where(
new TypeParameter<ElementT>() {}, wrappedComponentType);
}
Expand Down Expand Up @@ -1510,10 +1517,10 @@ public ByteCodeAppender appender(final Target implementationTarget) {
// Push all creator parameters on the stack.
TypeConversion<Type> convertType = typeConversionsFactory.createTypeConversion(true);
for (int i = 0; i < parameters.size(); i++) {
Parameter parameter = parameters.get(i);
FieldValueTypeInformation fieldType =
fields.get(Preconditions.checkNotNull(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.
Expand All @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -439,6 +441,13 @@ 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.checkNotNull(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 =
Expand All @@ -449,7 +458,8 @@ public ByteCodeAppender appender(final Target implementationTarget) {
MethodInvocation.invoke(
new ForLoadedMethod(
Preconditions.checkNotNull(
typeInformation.getMethod(), GETTER_WITH_NULL_METHOD_ERROR))));
typeInformation.getMethod(), GETTER_WITH_NULL_METHOD_ERROR))),
cast);

StackManipulation stackManipulation =
new StackManipulation.Compound(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -151,18 +152,13 @@ private static <T> SchemaUserTypeCreator createSetFieldCreator(
Schema schema,
List<FieldValueTypeInformation> types,
TypeConversionsFactory typeConversionsFactory) {
// Get the list of class fields ordered by schema.
List<Field> fields =
types.stream()
.map(type -> Preconditions.checkNotNull(type.getField()))
.collect(Collectors.toList());
try {
DynamicType.Builder<SchemaUserTypeCreator> 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))
Expand Down Expand Up @@ -305,11 +301,8 @@ public static <T> 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))
Expand All @@ -331,107 +324,25 @@ public static <T> SchemaUserTypeCreator createStaticCreator(
private static <ObjectT, ValueT>
DynamicType.Builder<FieldValueGetter<@NonNull ObjectT, ValueT>> implementGetterMethods(
DynamicType.Builder<FieldValueGetter<@NonNull ObjectT, ValueT>> 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<TypeDescriptorWithSchema<?>, List<FieldValueSetter<?, ?>>>
CACHED_SETTERS = Maps.newConcurrentMap();

public static <T> List<FieldValueSetter<@NonNull T, Object>> getSetters(
TypeDescriptor<T> 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<FieldValueTypeInformation> 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}.
*
* <pre><code>
* class Setter implements {@literal FieldValueSetter<POJO, FieldType>} {
* {@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);
* }
* }
* </code></pre>
*/
@SuppressWarnings("unchecked")
private static <ObjectT, ValueT> FieldValueSetter<ObjectT, ValueT> createSetter(
FieldValueTypeInformation typeInformation, TypeConversionsFactory typeConversionsFactory) {
Field field = Preconditions.checkNotNull(typeInformation.getField());
DynamicType.Builder<FieldValueSetter<ObjectT, ValueT>> 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 <ObjectT, ValueT>
DynamicType.Builder<FieldValueSetter<ObjectT, ValueT>> implementSetterMethods(
DynamicType.Builder<FieldValueSetter<ObjectT, ValueT>> 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;
}

Expand All @@ -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);
Expand Down Expand Up @@ -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<Field> fields;
private final List<FieldValueTypeInformation> typeInformations;
private final Class<?> pojoClass;
private final TypeConversionsFactory typeConversionsFactory;

SetFieldCreateInstruction(
List<Field> fields, Class<?> pojoClass, TypeConversionsFactory typeConversionsFactory) {
this.fields = fields;
List<FieldValueTypeInformation> typeInformations,
Class<?> pojoClass,
TypeConversionsFactory typeConversionsFactory) {
this.typeInformations = typeInformations;
this.pojoClass = pojoClass;
this.typeConversionsFactory = typeConversionsFactory;
}
Expand Down Expand Up @@ -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<Type> 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 =
Expand All @@ -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);
Expand Down
Loading
Loading