Hi Reuvan,

Here's the complete stack trace:

Exception in thread "main" java.lang.IllegalArgumentException: Type of @Element 
must match the DoFn typeCreate.Values/Read(CreateSource).out [PCollection]
        at 
org.apache.beam.sdk.transforms.ParDo.getDoFnSchemaInformation(ParDo.java:601)
        at 
org.apache.beam.repackaged.direct_java.runners.core.construction.ParDoTranslation.translateParDo(ParDoTranslation.java:190)
        at 
org.apache.beam.repackaged.direct_java.runners.core.construction.ParDoTranslation$ParDoTranslator.translate(ParDoTranslation.java:128)
        at 
org.apache.beam.repackaged.direct_java.runners.core.construction.PTransformTranslation.toProto(PTransformTranslation.java:225)
        at 
org.apache.beam.repackaged.direct_java.runners.core.construction.ParDoTranslation.getParDoPayload(ParDoTranslation.java:689)
        at 
org.apache.beam.repackaged.direct_java.runners.core.construction.ParDoTranslation.isSplittable(ParDoTranslation.java:704)
        at 
org.apache.beam.repackaged.direct_java.runners.core.construction.PTransformMatchers$6.matches(PTransformMatchers.java:269)
        at 
org.apache.beam.sdk.Pipeline$2.visitPrimitiveTransform(Pipeline.java:282)
        at 
org.apache.beam.sdk.runners.TransformHierarchy$Node.visit(TransformHierarchy.java:665)
        at 
org.apache.beam.sdk.runners.TransformHierarchy$Node.visit(TransformHierarchy.java:657)
        at 
org.apache.beam.sdk.runners.TransformHierarchy$Node.visit(TransformHierarchy.java:657)
        at 
org.apache.beam.sdk.runners.TransformHierarchy$Node.access$600(TransformHierarchy.java:317)
        at 
org.apache.beam.sdk.runners.TransformHierarchy.visit(TransformHierarchy.java:251)
        at org.apache.beam.sdk.Pipeline.traverseTopologically(Pipeline.java:460)
        at org.apache.beam.sdk.Pipeline.replace(Pipeline.java:260)
        at org.apache.beam.sdk.Pipeline.replaceAll(Pipeline.java:210)
        at 
org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:170)
        at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:67)
        at org.apache.beam.sdk.Pipeline.run(Pipeline.java:315)
        at org.apache.beam.sdk.Pipeline.run(Pipeline.java:301)

As mentioned earlier, I don't know if this is what should be expected (or if 
it's something worth addressing) within Beam or if the preferred approach would 
be to simply always rely on the use of the ProcessContext if you aren't 
natively writing your Beam applications in Kotlin.

On 2020/05/27 20:30:43, Reuven Lax <re...@google.com> wrote: 
> It could also be that Kotlin is defeating Beam's type analysis, if it
> changes type-parameter ordering for example. It may also be that the
> TypeToken framework we use for analyzing Java types isn't working properly
> on these Kotlin types.
> 
> On Wed, May 27, 2020 at 1:27 PM Reuven Lax <re...@google.com> wrote:
> 
> > Do you have the full stack trace from that exception?
> >
> > On Wed, May 27, 2020 at 1:13 PM Rion Williams <rionmons...@gmail.com>
> > wrote:
> >
> >> Correct, Kotlin uses an Int type as opposed to Java’s integer, however in
> >> this case I had assumed that since the PCollection being constructed and
> >> used by the DoFn both use the same Kotlin Int type that it would be able to
> >> bind properly (even when explicitly typing the Create to use the Kotlin
> >> type).
> >>
> >> When doing the same thing with Kotlin Strings, the @Element attribute
> >> works as expected, so I don’t know if this is an issue purely related to
> >> underlying type conversions with numeric Kotlin types and what’s the best
> >> way to handle this? I know using the ProcessContext works just as you’d
> >> expect, however for simple transforms the @Element approach can be a bit
> >> easier to grok.
> >>
> >> On May 27, 2020, at 3:01 PM, Reuven Lax <re...@google.com> wrote:
> >>
> >> 
> >> I'm assuming that Kotlin has its own type for Int, which is not the same
> >> as Java's Integer type.
> >>
> >> On Fri, May 22, 2020 at 8:19 AM Rion Williams <rionmons...@gmail.com>
> >> wrote:
> >>
> >>> Hi all,
> >>>
> >>> I was writing a very simple transform in Kotlin as follows that takes in
> >>> a series of integers and applies a simply DoFn against them:
> >>>
> >>> pipeline
> >>>             .apply(Create.of(1, 2, 3))
> >>>             .apply(ParDo.of(object: DoFn<Int, Int>(){
> >>>                     @ProcessElement
> >>>                     fun processElement(@Element element: Int){
> >>>                         // Omitted for brevity
> >>>                     }
> >>>                 })
> >>>             )
> >>>
> >>> The issue seems to arise when we use the `@Element` attribute on the
> >>> element which fails with the following error:
> >>>
> >>> Exception in thread "main" java.lang.IllegalArgumentException: Type of
> >>> @Element must match the DoFn typeCreate.Values/Read(CreateSource).out
> >>> [PCollection]
> >>>
> >>> Basically, it seems that the use of the `@Element` attribute isn't able
> >>> to properly decode or recognize the Kotlin `Int`, however if we adjust the
> >>> DoFn to instead use the ProcessContext argument, which is able to resolve
> >>> the element via `context.element()`.
> >>>
> >>> Is there anything that seems wrong here? I'd imagine that this should
> >>> just "work", or maybe there's some specific configuration that I might be
> >>> missing as this is the first Kotlin issue that I've encountered when
> >>> interacting with Beam.
> >>>
> >>> I'll be happy to provide any additional details for this if needed.
> >>>
> >>> Thanks,
> >>>
> >>> Rion
> >>>
> >>>
> 

Reply via email to