Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

make FieldValueTypeInformation creators take a TypeDescriptor parameter #32081

Merged
merged 6 commits into from
Oct 30, 2024
Merged
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 @@ -19,7 +19,6 @@

import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.beam.sdk.schemas.annotations.SchemaIgnore;
Expand All @@ -32,13 +31,10 @@
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
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.NonNull;
import org.checkerframework.checker.nullness.qual.Nullable;

/** A {@link SchemaProvider} for AutoValue classes. */
@SuppressWarnings({
"nullness", // TODO(https://github.com/apache/beam/issues/20497)
"rawtypes"
})
public class AutoValueSchema extends GetterBasedSchemaProviderV2 {
/** {@link FieldValueTypeSupplier} that's based on AutoValue getters. */
@VisibleForTesting
Expand All @@ -49,7 +45,11 @@ public static class AbstractGetterTypeSupplier implements FieldValueTypeSupplier
public List<FieldValueTypeInformation> get(TypeDescriptor<?> typeDescriptor) {

// If the generated class is passed in, we want to look at the base class to find the getters.
TypeDescriptor<?> targetTypeDescriptor = AutoValueUtils.getBaseAutoValueClass(typeDescriptor);
TypeDescriptor<?> targetTypeDescriptor =
Preconditions.checkNotNull(
AutoValueUtils.getBaseAutoValueClass(typeDescriptor),
"unable to determine base AutoValue class for type {}",
typeDescriptor);

List<Method> methods =
ReflectUtils.getMethods(targetTypeDescriptor.getRawType()).stream()
Expand All @@ -62,9 +62,9 @@ public List<FieldValueTypeInformation> get(TypeDescriptor<?> typeDescriptor) {
.collect(Collectors.toList());
List<FieldValueTypeInformation> types = Lists.newArrayListWithCapacity(methods.size());
for (int i = 0; i < methods.size(); ++i) {
types.add(FieldValueTypeInformation.forGetter(methods.get(i), i));
types.add(FieldValueTypeInformation.forGetter(typeDescriptor, methods.get(i), i));
}
types.sort(Comparator.comparing(FieldValueTypeInformation::getNumber));
types.sort(JavaBeanUtils.comparingNullFirst(FieldValueTypeInformation::getNumber));
validateFieldNumbers(types);
return types;
}
Expand All @@ -89,8 +89,8 @@ private static void validateFieldNumbers(List<FieldValueTypeInformation> types)
}

@Override
public List<FieldValueGetter> fieldValueGetters(
TypeDescriptor<?> targetTypeDescriptor, Schema schema) {
public <T> List<FieldValueGetter<@NonNull T, Object>> fieldValueGetters(
TypeDescriptor<T> targetTypeDescriptor, Schema schema) {
return JavaBeanUtils.getGetters(
targetTypeDescriptor,
schema,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,9 @@
import java.util.Objects;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.checkerframework.checker.initialization.qual.NotOnlyInitialized;
import org.checkerframework.checker.initialization.qual.UnknownInitialization;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.checkerframework.checker.nullness.qual.Nullable;

/**
Expand All @@ -32,24 +35,25 @@
* significant for larger schemas) on each lookup. This wrapper caches the value returned by the
* inner factory, so the schema comparison only need happen on the first lookup.
*/
@SuppressWarnings({
"nullness", // TODO(https://github.com/apache/beam/issues/20497)
"rawtypes"
})
public class CachingFactory<CreatedT> implements Factory<CreatedT> {
public class CachingFactory<CreatedT extends @NonNull Object> implements Factory<CreatedT> {
private transient @Nullable ConcurrentHashMap<TypeDescriptor<?>, CreatedT> cache = null;

private final Factory<CreatedT> innerFactory;
private final @NotOnlyInitialized Factory<CreatedT> innerFactory;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what are these new tags for?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so in RowValueGettersFactory there's this code:

    RowValueGettersFactory(Factory<List<FieldValueGetter<T, Object>>> gettersFactory) {
      this.gettersFactory = gettersFactory;
      this.cachingGettersFactory = new CachingFactory<>(this);
    }

since we're in the constructor, the this parameter is still not fully initialized, so by using this annotation we can temporarily (for the duration of the initalization of the CachingFactory object) allow assigning the innerFactory field a value which itself is under initialization (or indeed whose initialization status is unknown, as the signature of the CachingFactory constructor tells us)

see also https://checkerframework.org/manual/#circular-initialization


public CachingFactory(Factory<CreatedT> innerFactory) {
public CachingFactory(@UnknownInitialization Factory<CreatedT> innerFactory) {
this.innerFactory = innerFactory;
}

@Override
public CreatedT create(TypeDescriptor<?> typeDescriptor, Schema schema) {
private ConcurrentHashMap<TypeDescriptor<?>, CreatedT> getCache() {
if (cache == null) {
cache = new ConcurrentHashMap<>();
}
return cache;
}

@Override
public CreatedT create(TypeDescriptor<?> typeDescriptor, Schema schema) {
ConcurrentHashMap<TypeDescriptor<?>, CreatedT> cache = getCache();
CreatedT cached = cache.get(typeDescriptor);
if (cached != null) {
return cached;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import java.io.Serializable;
import org.apache.beam.sdk.annotations.Internal;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.checkerframework.checker.nullness.qual.Nullable;

/**
Expand All @@ -29,7 +30,7 @@
* <p>Implementations of this interface are generated at runtime to map object fields to Row fields.
*/
@Internal
public interface FieldValueGetter<ObjectT, ValueT> extends Serializable {
public interface FieldValueGetter<ObjectT extends @NonNull Object, ValueT> extends Serializable {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do you have ObjectT extends Object?

Copy link
Contributor Author

@tilgalas tilgalas Oct 25, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so apparently there's a difference (I don't understand why it would be so, though) between a @NonNull T and T extends @NonNull Object - here's an example, consider:

  public interface I<T extends @NonNull Object> {
      T getT();
  }

with a signature as above, we can only write:

  public static <T extends @NonNull Object> void f(I<T> i) {
    System.out.println(i.getT().getClass());
  }

or alternatively:

  public static <T> void f(I<@NonNull T> i) {
    System.out.println(i.getT().getClass());
  }

The checkerframework will make sure that we propagate the non-nullness of the parameter.
This won't work for example:

  public static <@Nullable T> void f(I<T> i) {
    T t = i.getT();
    if (t != null) {
      System.out.println(t.getClass());
    }
  }

nor will this:

  public static <T> void f(I<@Nullable T> i) {
    T t = i.getT();
    if (t != null) {
      System.out.println(t.getClass());
    }
  }

but it will work if I change the signature of I to:

  public interface I<@NonNull T> {
      T getT();
  }

in other words, the @NonNull T annotation on the interface is not really enforced anywhere, it's more like a suggestion, and so it's not really expressing the concept of the interface only works with @NonNull types

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Generally in Beam I've mostly seen us using @nullable, not @nonnull. I believe that NonNull is assumed unless @nullable is specified. Are you not finding that to be the case?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah, at least not when we're talking type parameters, for example:

  public interface I<T> {
      T getT();
  }

  public static <T> void f(I<T> i) {
    System.out.println(i.getT().getClass());
  }

gives me

error: [dereference.of.nullable] dereference of possibly-null reference i.getT()
  System.out.println(i.getT().getClass());
                           ^

@Nullable
ValueT get(ObjectT object);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,9 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Stream;
import org.apache.beam.sdk.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;
Expand All @@ -40,10 +42,7 @@

/** Represents type information for a Java type that will be used to infer a Schema type. */
@AutoValue
@SuppressWarnings({
"nullness", // TODO(https://github.com/apache/beam/issues/20497)
"rawtypes"
})
@Internal
public abstract class FieldValueTypeInformation implements Serializable {
/** Optionally returns the field index. */
public abstract @Nullable Integer getNumber();
Expand Down Expand Up @@ -125,18 +124,23 @@ public static FieldValueTypeInformation forOneOf(
.build();
}

public static FieldValueTypeInformation forField(Field field, int index) {
TypeDescriptor<?> type = TypeDescriptor.of(field.getGenericType());
public static FieldValueTypeInformation forField(
@Nullable TypeDescriptor<?> typeDescriptor, Field field, int index) {
TypeDescriptor<?> type =
Optional.ofNullable(typeDescriptor)
.map(td -> (TypeDescriptor) td.resolveType(field.getGenericType()))
// fall back to previous behavior
.orElseGet(() -> TypeDescriptor.of(field.getGenericType()));
return new AutoValue_FieldValueTypeInformation.Builder()
.setName(getNameOverride(field.getName(), field))
.setNumber(getNumberOverride(index, field))
.setNullable(hasNullableAnnotation(field))
.setType(type)
.setRawType(type.getRawType())
.setField(field)
.setElementType(getIterableComponentType(field))
.setMapKeyType(getMapKeyType(field))
.setMapValueType(getMapValueType(field))
.setElementType(getIterableComponentType(type))
.setMapKeyType(getMapKeyType(type))
.setMapValueType(getMapValueType(type))
.setOneOfTypes(Collections.emptyMap())
.setDescription(getFieldDescription(field))
.build();
Expand Down Expand Up @@ -185,6 +189,11 @@ public static <T extends AnnotatedElement & Member> String getNameOverride(
}

public static FieldValueTypeInformation forGetter(Method method, int index) {
return forGetter(null, method, index);
}

public static FieldValueTypeInformation forGetter(
@Nullable TypeDescriptor<?> typeDescriptor, Method method, int index) {
String name;
if (method.getName().startsWith("get")) {
name = ReflectUtils.stripPrefix(method.getName(), "get");
Expand All @@ -194,7 +203,12 @@ public static FieldValueTypeInformation forGetter(Method method, int index) {
throw new RuntimeException("Getter has wrong prefix " + method.getName());
}

TypeDescriptor<?> type = TypeDescriptor.of(method.getGenericReturnType());
TypeDescriptor<?> type =
Optional.ofNullable(typeDescriptor)
.map(td -> (TypeDescriptor) td.resolveType(method.getGenericReturnType()))
// fall back to previous behavior
.orElseGet(() -> TypeDescriptor.of(method.getGenericReturnType()));

boolean nullable = hasNullableReturnType(method);
return new AutoValue_FieldValueTypeInformation.Builder()
.setName(getNameOverride(name, method))
Expand Down Expand Up @@ -253,18 +267,32 @@ private static boolean isNullableAnnotation(Annotation annotation) {
}

public static FieldValueTypeInformation forSetter(Method method) {
return forSetter(method, "set");
return forSetter(null, method);
}

public static FieldValueTypeInformation forSetter(Method method, String setterPrefix) {
return forSetter(null, method, setterPrefix);
}

public static FieldValueTypeInformation forSetter(
@Nullable TypeDescriptor<?> typeDescriptor, Method method) {
return forSetter(typeDescriptor, method, "set");
}

public static FieldValueTypeInformation forSetter(
@Nullable TypeDescriptor<?> typeDescriptor, Method method, String setterPrefix) {
String name;
if (method.getName().startsWith(setterPrefix)) {
name = ReflectUtils.stripPrefix(method.getName(), setterPrefix);
} else {
throw new RuntimeException("Setter has wrong prefix " + method.getName());
}

TypeDescriptor<?> type = TypeDescriptor.of(method.getGenericParameterTypes()[0]);
TypeDescriptor<?> type =
Optional.ofNullable(typeDescriptor)
.map(td -> (TypeDescriptor) td.resolveType(method.getGenericParameterTypes()[0]))
// fall back to previous behavior
.orElseGet(() -> TypeDescriptor.of(method.getGenericParameterTypes()[0]));
boolean nullable = hasSingleNullableParameter(method);
return new AutoValue_FieldValueTypeInformation.Builder()
.setName(name)
Expand All @@ -283,10 +311,6 @@ public FieldValueTypeInformation withName(String name) {
return toBuilder().setName(name).build();
}

private static FieldValueTypeInformation getIterableComponentType(Field field) {
return getIterableComponentType(TypeDescriptor.of(field.getGenericType()));
}

static @Nullable FieldValueTypeInformation getIterableComponentType(TypeDescriptor<?> valueType) {
// TODO: Figure out nullable elements.
TypeDescriptor<?> componentType = ReflectUtils.getIterableComponentType(valueType);
Expand All @@ -306,34 +330,23 @@ private static FieldValueTypeInformation getIterableComponentType(Field field) {
.build();
}

// If the Field is a map type, returns the key type, otherwise returns a null reference.

private static @Nullable FieldValueTypeInformation getMapKeyType(Field field) {
return getMapKeyType(TypeDescriptor.of(field.getGenericType()));
}

// If the type is a map type, returns the key type, otherwise returns a null reference.
private static @Nullable FieldValueTypeInformation getMapKeyType(
TypeDescriptor<?> typeDescriptor) {
return getMapType(typeDescriptor, 0);
}

// If the Field is a map type, returns the value type, otherwise returns a null reference.

private static @Nullable FieldValueTypeInformation getMapValueType(Field field) {
return getMapType(TypeDescriptor.of(field.getGenericType()), 1);
}

// If the type is a map type, returns the value type, otherwise returns a null reference.
private static @Nullable FieldValueTypeInformation getMapValueType(
TypeDescriptor<?> typeDescriptor) {
return getMapType(typeDescriptor, 1);
}

// If the Field is a map type, returns the key or value type (0 is key type, 1 is value).
// Otherwise returns a null reference.
@SuppressWarnings("unchecked")
private static @Nullable FieldValueTypeInformation getMapType(
TypeDescriptor<?> valueType, int index) {
TypeDescriptor mapType = ReflectUtils.getMapType(valueType, index);
TypeDescriptor<?> mapType = ReflectUtils.getMapType(valueType, index);
if (mapType == null) {
return null;
}
Expand Down
Loading
Loading