Skip to content

Commit b369dfa

Browse files
authored
make use of generic type info in *Utils classes (#36581)
2 parents a799d2b + 050369f commit b369dfa

File tree

5 files changed

+90
-231
lines changed

5 files changed

+90
-231
lines changed

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

Lines changed: 7 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.beam.sdk.schemas.utils;
1919

2020
import static org.apache.beam.sdk.util.ByteBuddyUtils.getClassLoadingStrategy;
21-
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
2221

2322
import java.lang.reflect.Constructor;
2423
import java.lang.reflect.InvocationTargetException;
@@ -63,9 +62,9 @@
6362
import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.InjectPackageStrategy;
6463
import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversion;
6564
import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversionsFactory;
65+
import org.apache.beam.sdk.util.Preconditions;
6666
import org.apache.beam.sdk.util.common.ReflectHelpers;
6767
import org.apache.beam.sdk.values.TypeDescriptor;
68-
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
6968
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
7069
import org.checkerframework.checker.nullness.qual.Nullable;
7170

@@ -189,7 +188,7 @@ private static boolean matchConstructor(
189188
Collectors.toMap(
190189
f ->
191190
ReflectUtils.stripGetterPrefix(
192-
Preconditions.checkNotNull(
191+
Preconditions.checkArgumentNotNull(
193192
f.getMethod(), JavaBeanUtils.GETTER_WITH_NULL_METHOD_ERROR)
194193
.getName()),
195194
Function.identity()));
@@ -249,7 +248,7 @@ private static boolean matchConstructor(
249248
for (FieldValueTypeInformation type : schemaTypes) {
250249
String autoValueFieldName =
251250
ReflectUtils.stripGetterPrefix(
252-
Preconditions.checkNotNull(
251+
Preconditions.checkArgumentNotNull(
253252
type.getMethod(), JavaBeanUtils.GETTER_WITH_NULL_METHOD_ERROR)
254253
.getName());
255254

@@ -347,11 +346,10 @@ public ByteCodeAppender appender(final Target implementationTarget) {
347346

348347
TypeConversion<Type> convertType = typeConversionsFactory.createTypeConversion(true);
349348
for (int i = 0; i < setters.size(); ++i) {
350-
Method setterMethod = checkNotNull(setters.get(i).getMethod());
351-
Parameter parameter = setterMethod.getParameters()[0];
349+
FieldValueTypeInformation setterType = setters.get(i);
350+
Method setterMethod = Preconditions.checkStateNotNull(setterType.getMethod());
352351
ForLoadedType convertedType =
353-
new ForLoadedType(
354-
(Class) convertType.convert(TypeDescriptor.of(parameter.getParameterizedType())));
352+
new ForLoadedType((Class) convertType.convert(setterType.getType()));
355353

356354
StackManipulation readParameter =
357355
new StackManipulation.Compound(
@@ -366,7 +364,7 @@ public ByteCodeAppender appender(final Target implementationTarget) {
366364
Duplication.SINGLE,
367365
typeConversionsFactory
368366
.createSetterConversions(readParameter)
369-
.convert(TypeDescriptor.of(parameter.getType())),
367+
.convert(setterType.getType()),
370368
MethodInvocation.invoke(new ForLoadedMethod(setterMethod)),
371369
Removal.SINGLE);
372370
}

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

Lines changed: 33 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -78,11 +78,11 @@
7878
import org.apache.beam.sdk.schemas.FieldValueHaver;
7979
import org.apache.beam.sdk.schemas.FieldValueSetter;
8080
import org.apache.beam.sdk.schemas.FieldValueTypeInformation;
81+
import org.apache.beam.sdk.util.Preconditions;
8182
import org.apache.beam.sdk.util.common.ReflectHelpers;
8283
import org.apache.beam.sdk.values.TypeDescriptor;
8384
import org.apache.beam.sdk.values.TypeParameter;
8485
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function;
85-
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
8686
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Verify;
8787
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Collections2;
8888
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
@@ -275,7 +275,7 @@ public TypeConversion<StackManipulation> createSetterConversions(StackManipulati
275275
public abstract static class TypeConversion<T> {
276276
public T convert(TypeDescriptor<?> typeDescriptor) {
277277
if (typeDescriptor.isArray()
278-
&& !Preconditions.checkNotNull(typeDescriptor.getComponentType())
278+
&& !Preconditions.checkArgumentNotNull(typeDescriptor.getComponentType())
279279
.getRawType()
280280
.equals(byte.class)) {
281281
// Byte arrays are special, so leave those alone.
@@ -363,31 +363,31 @@ protected ConvertType(boolean returnRawTypes) {
363363
@Override
364364
protected Type convertArray(TypeDescriptor<?> type) {
365365
TypeDescriptor<?> ret =
366-
createCollectionType(Preconditions.checkNotNull(type.getComponentType()));
366+
createCollectionType(Preconditions.checkArgumentNotNull(type.getComponentType()));
367367
return returnRawTypes ? ret.getRawType() : ret.getType();
368368
}
369369

370370
@Override
371371
protected Type convertCollection(TypeDescriptor<?> type) {
372372
TypeDescriptor<?> ret =
373373
createCollectionType(
374-
Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type)));
374+
Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type)));
375375
return returnRawTypes ? ret.getRawType() : ret.getType();
376376
}
377377

378378
@Override
379379
protected Type convertList(TypeDescriptor<?> type) {
380380
TypeDescriptor<?> ret =
381381
createCollectionType(
382-
Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type)));
382+
Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type)));
383383
return returnRawTypes ? ret.getRawType() : ret.getType();
384384
}
385385

386386
@Override
387387
protected Type convertIterable(TypeDescriptor<?> type) {
388388
TypeDescriptor<?> ret =
389389
createIterableType(
390-
Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type)));
390+
Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type)));
391391
return returnRawTypes ? ret.getRawType() : ret.getType();
392392
}
393393

@@ -426,12 +426,20 @@ protected Type convertDefault(TypeDescriptor<?> type) {
426426
return returnRawTypes ? type.getRawType() : type.getType();
427427
}
428428

429+
public static TypeDescriptor<?> primitiveToWrapper(TypeDescriptor<?> typeDescriptor) {
430+
Class<?> cls = typeDescriptor.getRawType();
431+
if (cls.isPrimitive()) {
432+
return TypeDescriptor.of(ClassUtils.primitiveToWrapper(cls));
433+
} else {
434+
return typeDescriptor;
435+
}
436+
}
437+
429438
@SuppressWarnings("unchecked")
430439
private <ElementT> TypeDescriptor<Collection<ElementT>> createCollectionType(
431440
TypeDescriptor<?> componentType) {
432441
TypeDescriptor<ElementT> wrappedComponentType =
433-
(TypeDescriptor<ElementT>)
434-
TypeDescriptor.of(ClassUtils.primitiveToWrapper(componentType.getRawType()));
442+
(TypeDescriptor<ElementT>) primitiveToWrapper(componentType);
435443
return new TypeDescriptor<Collection<ElementT>>() {}.where(
436444
new TypeParameter<ElementT>() {}, wrappedComponentType);
437445
}
@@ -440,8 +448,7 @@ private <ElementT> TypeDescriptor<Collection<ElementT>> createCollectionType(
440448
private <ElementT> TypeDescriptor<Iterable<ElementT>> createIterableType(
441449
TypeDescriptor<?> componentType) {
442450
TypeDescriptor<ElementT> wrappedComponentType =
443-
(TypeDescriptor<ElementT>)
444-
TypeDescriptor.of(ClassUtils.primitiveToWrapper(componentType.getRawType()));
451+
(TypeDescriptor<ElementT>) primitiveToWrapper(componentType);
445452
return new TypeDescriptor<Iterable<ElementT>>() {}.where(
446453
new TypeParameter<ElementT>() {}, wrappedComponentType);
447454
}
@@ -670,12 +677,12 @@ protected StackManipulation convertArray(TypeDescriptor<?> type) {
670677
// return isComponentTypePrimitive ? Arrays.asList(ArrayUtils.toObject(value))
671678
// : Arrays.asList(value);
672679

673-
TypeDescriptor<?> componentType = Preconditions.checkNotNull(type.getComponentType());
680+
TypeDescriptor<?> componentType = Preconditions.checkArgumentNotNull(type.getComponentType());
674681
ForLoadedType loadedArrayType = new ForLoadedType(type.getRawType());
675682
StackManipulation readArrayValue = readValue;
676683
// Row always expects to get an Iterable back for array types. Wrap this array into a
677684
// List using Arrays.asList before returning.
678-
if (Preconditions.checkNotNull(loadedArrayType.getComponentType()).isPrimitive()) {
685+
if (Preconditions.checkArgumentNotNull(loadedArrayType.getComponentType()).isPrimitive()) {
679686
// Arrays.asList doesn't take primitive arrays, so convert first using ArrayUtils.toObject.
680687
readArrayValue =
681688
new Compound(
@@ -723,7 +730,7 @@ protected StackManipulation convertArray(TypeDescriptor<?> type) {
723730
@Override
724731
protected StackManipulation convertIterable(TypeDescriptor<?> type) {
725732
TypeDescriptor<?> componentType =
726-
Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type));
733+
Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type));
727734
Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType);
728735

729736
final TypeDescriptor<?> finalComponentType = ReflectUtils.boxIfPrimitive(componentType);
@@ -744,7 +751,7 @@ protected StackManipulation convertIterable(TypeDescriptor<?> type) {
744751
@Override
745752
protected StackManipulation convertCollection(TypeDescriptor<?> type) {
746753
TypeDescriptor<?> componentType =
747-
Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type));
754+
Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type));
748755
Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType);
749756
final TypeDescriptor<?> finalComponentType = ReflectUtils.boxIfPrimitive(componentType);
750757
if (!finalComponentType.hasUnresolvedParameters()) {
@@ -764,7 +771,7 @@ protected StackManipulation convertCollection(TypeDescriptor<?> type) {
764771
@Override
765772
protected StackManipulation convertList(TypeDescriptor<?> type) {
766773
TypeDescriptor<?> componentType =
767-
Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type));
774+
Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type));
768775
Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType);
769776
final TypeDescriptor<?> finalComponentType = ReflectUtils.boxIfPrimitive(componentType);
770777
if (!finalComponentType.hasUnresolvedParameters()) {
@@ -1017,7 +1024,7 @@ protected StackManipulation convertArray(TypeDescriptor<?> type) {
10171024
.build()
10181025
.asErasure();
10191026

1020-
TypeDescriptor<?> componentType = Preconditions.checkNotNull(type.getComponentType());
1027+
TypeDescriptor<?> componentType = Preconditions.checkArgumentNotNull(type.getComponentType());
10211028
Type rowElementType = getFactory().createTypeConversion(false).convert(componentType);
10221029
final TypeDescriptor<?> arrayElementType = ReflectUtils.boxIfPrimitive(componentType);
10231030
StackManipulation readTransformedValue = readValue;
@@ -1076,7 +1083,7 @@ protected StackManipulation convertArray(TypeDescriptor<?> type) {
10761083
@Override
10771084
protected StackManipulation convertIterable(TypeDescriptor<?> type) {
10781085
final TypeDescriptor<?> iterableElementType =
1079-
Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type));
1086+
Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type));
10801087
Type rowElementType = getFactory().createTypeConversion(false).convert(iterableElementType);
10811088
if (!iterableElementType.hasUnresolvedParameters()) {
10821089
ForLoadedType conversionFunction =
@@ -1096,7 +1103,7 @@ protected StackManipulation convertIterable(TypeDescriptor<?> type) {
10961103
@Override
10971104
protected StackManipulation convertCollection(TypeDescriptor<?> type) {
10981105
final TypeDescriptor<?> collectionElementType =
1099-
Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type));
1106+
Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type));
11001107
Type rowElementType = getFactory().createTypeConversion(false).convert(collectionElementType);
11011108

11021109
if (!collectionElementType.hasUnresolvedParameters()) {
@@ -1117,7 +1124,7 @@ protected StackManipulation convertCollection(TypeDescriptor<?> type) {
11171124
@Override
11181125
protected StackManipulation convertList(TypeDescriptor<?> type) {
11191126
final TypeDescriptor<?> collectionElementType =
1120-
Preconditions.checkNotNull(ReflectUtils.getIterableComponentType(type));
1127+
Preconditions.checkArgumentNotNull(ReflectUtils.getIterableComponentType(type));
11211128
Type rowElementType = getFactory().createTypeConversion(false).convert(collectionElementType);
11221129

11231130
StackManipulation readTrasformedValue = readValue;
@@ -1147,9 +1154,9 @@ protected StackManipulation convertList(TypeDescriptor<?> type) {
11471154
@Override
11481155
protected StackManipulation convertMap(TypeDescriptor<?> type) {
11491156
final TypeDescriptor<?> keyElementType =
1150-
Preconditions.checkNotNull(ReflectUtils.getMapType(type, 0));
1157+
Preconditions.checkArgumentNotNull(ReflectUtils.getMapType(type, 0));
11511158
final TypeDescriptor<?> valueElementType =
1152-
Preconditions.checkNotNull(ReflectUtils.getMapType(type, 1));
1159+
Preconditions.checkArgumentNotNull(ReflectUtils.getMapType(type, 1));
11531160
Type rowKeyType = getFactory().createTypeConversion(false).convert(keyElementType);
11541161
Type rowValueType = getFactory().createTypeConversion(false).convert(valueElementType);
11551162

@@ -1510,25 +1517,25 @@ public ByteCodeAppender appender(final Target implementationTarget) {
15101517
// Push all creator parameters on the stack.
15111518
TypeConversion<Type> convertType = typeConversionsFactory.createTypeConversion(true);
15121519
for (int i = 0; i < parameters.size(); i++) {
1513-
Parameter parameter = parameters.get(i);
1520+
FieldValueTypeInformation fieldType =
1521+
fields.get(Preconditions.checkStateNotNull(fieldMapping.get(i)));
15141522
ForLoadedType convertedType =
1515-
new ForLoadedType(
1516-
(Class) convertType.convert(TypeDescriptor.of(parameter.getType())));
1523+
new ForLoadedType((Class) convertType.convert(fieldType.getType()));
15171524

15181525
// The instruction to read the parameter. Use the fieldMapping to reorder parameters as
15191526
// necessary.
15201527
StackManipulation readParameter =
15211528
new StackManipulation.Compound(
15221529
MethodVariableAccess.REFERENCE.loadFrom(1),
1523-
IntegerConstant.forValue(Preconditions.checkNotNull(fieldMapping.get(i))),
1530+
IntegerConstant.forValue(Preconditions.checkStateNotNull(fieldMapping.get(i))),
15241531
ArrayAccess.REFERENCE.load(),
15251532
TypeCasting.to(convertedType));
15261533
stackManipulation =
15271534
new StackManipulation.Compound(
15281535
stackManipulation,
15291536
typeConversionsFactory
15301537
.createSetterConversions(readParameter)
1531-
.convert(TypeDescriptor.of(parameter.getParameterizedType())));
1538+
.convert(fieldType.getType()));
15321539
}
15331540
stackManipulation =
15341541
new StackManipulation.Compound(

0 commit comments

Comments
 (0)