Skip to content

Commit

Permalink
[BEAM-790] Validate PipelineOptions Default annotation.
Browse files Browse the repository at this point in the history
  • Loading branch information
peihe committed Oct 21, 2016
1 parent 4c90582 commit 0e3d981
Show file tree
Hide file tree
Showing 3 changed files with 246 additions and 57 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@
import java.util.TreeMap;
import java.util.TreeSet;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import org.apache.beam.sdk.options.Validation.Required;
import org.apache.beam.sdk.runners.PipelineRunner;
import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
Expand Down Expand Up @@ -1057,56 +1058,87 @@ private static void validateMethodAnnotations(
methodNameToAllMethodMap.put(method, method);
}

List<InconsistentlyIgnoredGetters> incompletelyIgnoredGetters = new ArrayList<>();
List<IgnoredSetter> ignoredSetters = new ArrayList<>();
// Verify that there is no getter with a mixed @JsonIgnore annotation.
validateGettersNoMixedAnnotations(
methodNameToAllMethodMap, descriptors, AnnotationPredicate.JSON_IGNORE);

// Verify that there is no getter with a mixed @Default annotation.
validateGettersNoMixedAnnotations(
methodNameToAllMethodMap, descriptors, AnnotationPredicate.DEFAULT_VALUE);

// Verify that no setter has @JsonIgnore.
validateSettersNoAnnotations(
methodNameToAllMethodMap, descriptors, AnnotationPredicate.JSON_IGNORE);

// Verify that no setter has @Default.
validateSettersNoAnnotations(
methodNameToAllMethodMap, descriptors, AnnotationPredicate.DEFAULT_VALUE);
}

/**
* Validates that getters don't have mixed annotation.
*/
private static void validateGettersNoMixedAnnotations(
SortedSetMultimap<Method, Method> methodNameToAllMethodMap,
List<PropertyDescriptor> descriptors,
AnnotationPredicate annotationPredicate) {
List<InconsistentlyAnnotatedGetters> incompletelyAnnotatedGetters = new ArrayList<>();
for (PropertyDescriptor descriptor : descriptors) {
if (descriptor.getReadMethod() == null
|| descriptor.getWriteMethod() == null
|| IGNORED_METHODS.contains(descriptor.getReadMethod())
|| IGNORED_METHODS.contains(descriptor.getWriteMethod())) {
|| IGNORED_METHODS.contains(descriptor.getReadMethod())) {
continue;
}
// Verify that there is no getter with a mixed @JsonIgnore annotation and verify
// that no setter has @JsonIgnore.
SortedSet<Method> getters = methodNameToAllMethodMap.get(descriptor.getReadMethod());
SortedSet<Method> gettersWithJsonIgnore = Sets.filter(getters, JsonIgnorePredicate.INSTANCE);
SortedSet<Method> gettersWithTheAnnotation = Sets.filter(getters, annotationPredicate);

Iterable<String> getterClassNames = FluentIterable.from(getters)
.transform(MethodToDeclaringClassFunction.INSTANCE)
.transform(ReflectHelpers.CLASS_NAME);
Iterable<String> gettersWithJsonIgnoreClassNames = FluentIterable.from(gettersWithJsonIgnore)
Iterable<String> gettersWithTheAnnotationClassNames = FluentIterable.from(gettersWithTheAnnotation)
.transform(MethodToDeclaringClassFunction.INSTANCE)
.transform(ReflectHelpers.CLASS_NAME);

if (!(gettersWithJsonIgnore.isEmpty() || getters.size() == gettersWithJsonIgnore.size())) {
InconsistentlyIgnoredGetters err = new InconsistentlyIgnoredGetters();
if (!(gettersWithTheAnnotation.isEmpty() || getters.size() == gettersWithTheAnnotation.size())) {
InconsistentlyAnnotatedGetters err = new InconsistentlyAnnotatedGetters();
err.descriptor = descriptor;
err.getterClassNames = getterClassNames;
err.gettersWithJsonIgnoreClassNames = gettersWithJsonIgnoreClassNames;
incompletelyIgnoredGetters.add(err);
err.gettersWithTheAnnotationClassNames = gettersWithTheAnnotationClassNames;
incompletelyAnnotatedGetters.add(err);
}
if (!incompletelyIgnoredGetters.isEmpty()) {
}
throwForGettersWithInconsistentAnnotation(
incompletelyAnnotatedGetters, annotationPredicate.annotationClass);
}

/**
* Validates that setters don't have the given annotation.
*/
private static void validateSettersNoAnnotations(
SortedSetMultimap<Method, Method> methodNameToAllMethodMap,
List<PropertyDescriptor> descriptors,
AnnotationPredicate annotationPredicate) {
List<AnnotatedSetter> annotatedSetters = new ArrayList<>();
for (PropertyDescriptor descriptor : descriptors) {
if (descriptor.getWriteMethod() == null
|| IGNORED_METHODS.contains(descriptor.getWriteMethod())) {
continue;
}

SortedSet<Method> settersWithJsonIgnore =
SortedSet<Method> settersWithTheAnnotation =
Sets.filter(methodNameToAllMethodMap.get(descriptor.getWriteMethod()),
JsonIgnorePredicate.INSTANCE);
annotationPredicate);

Iterable<String> settersWithJsonIgnoreClassNames = FluentIterable.from(settersWithJsonIgnore)
Iterable<String> settersWithTheAnnotationClassNames = FluentIterable.from(settersWithTheAnnotation)
.transform(MethodToDeclaringClassFunction.INSTANCE)
.transform(ReflectHelpers.CLASS_NAME);

if (!settersWithJsonIgnore.isEmpty()) {
IgnoredSetter ignored = new IgnoredSetter();
ignored.descriptor = descriptor;
ignored.settersWithJsonIgnoreClassNames = settersWithJsonIgnoreClassNames;
ignoredSetters.add(ignored);
if (!settersWithTheAnnotation.isEmpty()) {
AnnotatedSetter annotated = new AnnotatedSetter();
annotated.descriptor = descriptor;
annotated.settersWithTheAnnotationClassNames = settersWithTheAnnotationClassNames;
annotatedSetters.add(annotated);
}
}
throwForGettersWithInconsistentJsonIgnore(incompletelyIgnoredGetters);
throwForSettersWithJsonIgnore(ignoredSetters);
throwForSettersWithTheAnnotation(annotatedSetters, annotationPredicate.annotationClass);
}

/**
Expand Down Expand Up @@ -1221,53 +1253,62 @@ private static void throwForMultipleDefinitions(
}
}

private static class InconsistentlyIgnoredGetters {
private static class InconsistentlyAnnotatedGetters {
PropertyDescriptor descriptor;
Iterable<String> getterClassNames;
Iterable<String> gettersWithJsonIgnoreClassNames;
Iterable<String> gettersWithTheAnnotationClassNames;
}

private static void throwForGettersWithInconsistentJsonIgnore(
List<InconsistentlyIgnoredGetters> getters) {
private static void throwForGettersWithInconsistentAnnotation(
List<InconsistentlyAnnotatedGetters> getters,
Class<? extends Annotation> annotationClass) {
if (getters.size() == 1) {
InconsistentlyIgnoredGetters getter = getters.get(0);
InconsistentlyAnnotatedGetters getter = getters.get(0);
throw new IllegalArgumentException(String.format(
"Expected getter for property [%s] to be marked with @JsonIgnore on all %s, "
"Expected getter for property [%s] to be marked with @%s on all %s, "
+ "found only on %s",
getter.descriptor.getName(), getter.getterClassNames,
getter.gettersWithJsonIgnoreClassNames));
getter.descriptor.getName(),
annotationClass.getSimpleName(),
getter.getterClassNames,
getter.gettersWithTheAnnotationClassNames));
} else if (getters.size() > 1) {
StringBuilder errorBuilder =
new StringBuilder("Property getters are inconsistently marked with @JsonIgnore:");
for (InconsistentlyIgnoredGetters getter : getters) {
StringBuilder errorBuilder =new StringBuilder(String.format(
"Property getters are inconsistently marked with @%s:", annotationClass.getSimpleName()));
for (InconsistentlyAnnotatedGetters getter : getters) {
errorBuilder.append(
String.format("%n - Expected for property [%s] to be marked on all %s, "
+ "found only on %s",
getter.descriptor.getName(), getter.getterClassNames,
getter.gettersWithJsonIgnoreClassNames));
getter.gettersWithTheAnnotationClassNames));
}
throw new IllegalArgumentException(errorBuilder.toString());
}
}

private static class IgnoredSetter {
private static class AnnotatedSetter {
PropertyDescriptor descriptor;
Iterable<String> settersWithJsonIgnoreClassNames;
Iterable<String> settersWithTheAnnotationClassNames;
}

private static void throwForSettersWithJsonIgnore(List<IgnoredSetter> setters) {
private static void throwForSettersWithTheAnnotation(
List<AnnotatedSetter> setters,
Class<? extends Annotation> annotationClass) {
if (setters.size() == 1) {
IgnoredSetter setter = setters.get(0);
throw new IllegalArgumentException(
String.format("Expected setter for property [%s] to not be marked with @JsonIgnore on %s",
setter.descriptor.getName(), setter.settersWithJsonIgnoreClassNames));
AnnotatedSetter setter = setters.get(0);
throw new IllegalArgumentException(String.format(
"Expected setter for property [%s] to not be marked with @%s on %s",
setter.descriptor.getName(),
annotationClass.getSimpleName(),
setter.settersWithTheAnnotationClassNames));
} else if (setters.size() > 1) {
StringBuilder builder = new StringBuilder("Found setters marked with @JsonIgnore:");
for (IgnoredSetter setter : setters) {
builder.append(
String.format("%n - Setter for property [%s] should not be marked with @JsonIgnore "
+ "on %s",
setter.descriptor.getName(), setter.settersWithJsonIgnoreClassNames));
StringBuilder builder = new StringBuilder(
String.format("Found setters marked with @%s:", annotationClass.getSimpleName()));
for (AnnotatedSetter setter : setters) {
builder.append(String.format(
"%n - Setter for property [%s] should not be marked with @%s on %s",
setter.descriptor.getName(),
annotationClass.getSimpleName(),
setter.settersWithTheAnnotationClassNames));
}
throw new IllegalArgumentException(builder.toString());
}
Expand Down Expand Up @@ -1353,14 +1394,44 @@ public Class<?> apply(Method input) {
}

/**
* A {@link Predicate} that returns true if the method is annotated with
* {@link JsonIgnore @JsonIgnore}.
* A {@link Predicate} that returns true if the method is annotated with {@code annotationClass}.
*/
static class JsonIgnorePredicate implements Predicate<Method> {
static final JsonIgnorePredicate INSTANCE = new JsonIgnorePredicate();
static class AnnotationPredicate implements Predicate<Method> {
static final AnnotationPredicate JSON_IGNORE = new AnnotationPredicate(
new Predicate<Method>() {
@Override
public boolean apply(Method input) {
return input.isAnnotationPresent(JsonIgnore.class);
}
},
JsonIgnore.class);

static final AnnotationPredicate DEFAULT_VALUE = new AnnotationPredicate(
new Predicate<Method>() {
@Override
public boolean apply(Method input) {
for (Class<?> klass : Default.class.getDeclaredClasses()) {
if (klass.isAnnotation()
&& input.isAnnotationPresent((Class<? extends Annotation>) klass)) {
return true;
}
}
return false;
}
},
Default.class);

private final Predicate<Method> predicate;
private final Class<? extends Annotation> annotationClass;

AnnotationPredicate(Predicate<Method> predicate, Class<? extends Annotation> annotationClass) {
this.predicate = predicate;
this.annotationClass = annotationClass;
}

@Override
public boolean apply(Method input) {
return input.isAnnotationPresent(JsonIgnore.class);
return predicate.apply(input);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@
import java.util.concurrent.ThreadLocalRandom;
import javax.annotation.Nullable;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.beam.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate;
import org.apache.beam.sdk.options.PipelineOptionsFactory.AnnotationPredicate;
import org.apache.beam.sdk.options.PipelineOptionsFactory.Registration;
import org.apache.beam.sdk.options.ValueProvider.RuntimeValueProvider;
import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
Expand Down Expand Up @@ -635,7 +635,7 @@ private void removeIgnoredOptions(
// Find all the method names that are annotated with JSON ignore.
Set<String> jsonIgnoreMethodNames = FluentIterable.from(
ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces))
.filter(JsonIgnorePredicate.INSTANCE).transform(new Function<Method, String>() {
.filter(AnnotationPredicate.JSON_IGNORE).transform(new Function<Method, String>() {
@Override
public String apply(Method input) {
return input.getName();
Expand Down
Loading

0 comments on commit 0e3d981

Please sign in to comment.