[ 
https://issues.apache.org/jira/browse/BEAM-5061?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16669552#comment-16669552
 ] 

Kenneth Knowles commented on BEAM-5061:
---------------------------------------

My (shallow) understanding is that a bytebuddy upgrade is needed, and it has 
some new conventions to support new and tighter aspects of Java's access model.

> Invisible parameter type exception in JDK 10
> --------------------------------------------
>
>                 Key: BEAM-5061
>                 URL: https://issues.apache.org/jira/browse/BEAM-5061
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-java-core
>    Affects Versions: 2.5.0
>            Reporter: Mike Pedersen
>            Priority: Major
>
> When using JDK 10, using a ParDo after a CoGroupByKey seems to create the 
> following exception when executed on local runner:
> {noformat}
> Exception in thread "main" 
> org.apache.beam.repackaged.beam_runners_direct_java.com.google.common.util.concurrent.UncheckedExecutionException:
>  java.lang.IllegalStateException: Invisible parameter type of Main$1 arg0 for 
> public Main$1$DoFnInvoker(Main$1)
>     at 
> org.apache.beam.repackaged.beam_runners_direct_java.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2214)
>     at 
> org.apache.beam.repackaged.beam_runners_direct_java.com.google.common.cache.LocalCache.get(LocalCache.java:4053)
>     at 
> org.apache.beam.repackaged.beam_runners_direct_java.com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4057)
>     ...
> Caused by: java.lang.IllegalStateException: Invisible parameter type of 
> Main$1 arg0 for public Main$1$DoFnInvoker(Main$1)
>     at 
> org.apache.beam.repackaged.beam_sdks_java_core.net.bytebuddy.dynamic.scaffold.InstrumentedType$Default.validated(InstrumentedType.java:925)
>     at 
> org.apache.beam.repackaged.beam_sdks_java_core.net.bytebuddy.dynamic.scaffold.MethodRegistry$Default.prepare(MethodRegistry.java:465)
>     at 
> org.apache.beam.repackaged.beam_sdks_java_core.net.bytebuddy.dynamic.scaffold.subclass.SubclassDynamicTypeBuilder.make(SubclassDynamicTypeBuilder.java:170)
>     ...
> {noformat}
> This error disappears completely when using JDK 8. Here is a minimal example 
> to reproduce it:
> {code:java}
> import org.apache.beam.sdk.Pipeline;
> import org.apache.beam.sdk.options.PipelineOptions;
> import org.apache.beam.sdk.options.PipelineOptionsFactory;
> import org.apache.beam.sdk.transforms.Create;
> import org.apache.beam.sdk.transforms.DoFn;
> import org.apache.beam.sdk.transforms.ParDo;
> import org.apache.beam.sdk.transforms.join.CoGbkResult;
> import org.apache.beam.sdk.transforms.join.CoGroupByKey;
> import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
> import org.apache.beam.sdk.values.KV;
> import org.apache.beam.sdk.values.PCollection;
> import org.apache.beam.sdk.values.TupleTag;
> import java.util.Arrays;
> import java.util.List;
> public class Main {
>     public static void main(String[] args) {
>         PipelineOptions options = PipelineOptionsFactory.create();
>         Pipeline p = Pipeline.create(options);
>         final TupleTag<String> emailsTag = new TupleTag<>();
>         final TupleTag<String> phonesTag = new TupleTag<>();
>         final List<KV<String, String>> emailsList =
>                 Arrays.asList(
>                         KV.of("amy", "a...@example.com"),
>                         KV.of("carl", "c...@example.com"),
>                         KV.of("julia", "ju...@example.com"),
>                         KV.of("carl", "c...@email.com"));
>         final List<KV<String, String>> phonesList =
>                 Arrays.asList(
>                         KV.of("amy", "111-222-3333"),
>                         KV.of("james", "222-333-4444"),
>                         KV.of("amy", "333-444-5555"),
>                         KV.of("carl", "444-555-6666"));
>         PCollection<KV<String, String>> emails = p.apply("CreateEmails", 
> Create.of(emailsList));
>         PCollection<KV<String, String>> phones = p.apply("CreatePhones", 
> Create.of(phonesList));
>         PCollection<KV<String, CoGbkResult>> results =
>                 KeyedPCollectionTuple.of(emailsTag, emails)
>                         .and(phonesTag, phones)
>                         .apply(CoGroupByKey.<String>create());
>         PCollection<String> contactLines =
>                 results.apply(
>                         ParDo.of(
>                                 new DoFn<KV<String, CoGbkResult>, String>() {
>                                     @ProcessElement
>                                     public void processElement(ProcessContext 
> c) {
>                                         KV<String, CoGbkResult> e = 
> c.element();
>                                         String name = e.getKey();
>                                         Iterable<String> emailsIter = 
> e.getValue().getAll(emailsTag);
>                                         Iterable<String> phonesIter = 
> e.getValue().getAll(phonesTag);
>                                         String formattedResult = "";
>                                         c.output(formattedResult);
>                                     }
>                                 }));
>         p.run().waitUntilFinish();
>     }
> }{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to