Uploaded image for project: 'Beam'
  1. Beam
  2. BEAM-8669

Default methods not allowed in PipelineOptions

Details

    • Bug
    • Status: Open
    • P3
    • Resolution: Unresolved
    • 2.16.0
    • None
    • beam-model
    • None

    Description

      If I create a class that extends PipelineOptions and contains a default method, the PipelineOptionsFactory will throw an exception because all non-static, non-synthetic and non-known methods need to have a getter and a setter.

      For example, these PipelineOptions

      public interface MyOptions extends PipelineOptions {
        void setValue(String s);
        String getValue();
      
         default List<String> getValues() {
           return Arrays.asList(getValue().split(","));
         }
      }

      will throw an exception in org.apache.beam.sdk.options.PipelineOptionsFactory.java:

      private static void validateMethodsAreEitherBeanMethodOrKnownMethod(
          Class<? extends PipelineOptions> iface,
          Class<? extends PipelineOptions> klass,
          List<PropertyDescriptor> descriptors) {
      
      ...
      // Verify that no additional methods are on an interface that aren't a bean property.
      // Because methods can have multiple declarations, we do a name-based comparison
      // here to prevent false positives.
      SortedSet<Method> unknownMethods = new TreeSet<>(MethodComparator.INSTANCE);
      unknownMethods.addAll(
          Sets.filter(
              Sets.difference(Sets.newHashSet(iface.getMethods()), knownMethods),
              Predicates.and(
                  NOT_SYNTHETIC_PREDICATE,
                  input -> !knownMethodsNames.contains(input.getName()),
                  NOT_STATIC_PREDICATE)));
      checkArgument(
          unknownMethods.isEmpty(),
          "Methods %s on [%s] do not conform to being bean properties.",
          FluentIterable.from(unknownMethods).transform(ReflectHelpers.METHOD_FORMATTER),
          iface.getName());
      }

      Having a NOT_DEFAULT_PREDICATE in addition to the other predicates would allow 

      private static final Predicate<Method> NOT_DEFAULT_PREDICATE = input -> !input.isDefault();
      

      Seems like it would do the trick.

      Attachments

        Activity

          People

            Unassigned Unassigned
            chrisstockton chris stockton
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated: