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

Udi Meiri commented on BEAM-6352:
---------------------------------

I can recreate this in 2.9.0, using the Java quickstart instructions and 
modifying ./src/main/java/org/apache/beam/examples/WordCount.java:

```
import org.apache.beam.sdk.transforms.Watch;
import org.joda.time.Duration;
import org.apache.beam.sdk.io.FileIO;
```

```
  static void runWordCount(WordCountOptions options) {
    Pipeline.create(options)
        .apply(
            FileIO.match()
                .filepattern(options.getInputFile())
                .continuously(Duration.standardMinutes(1), 
Watch.Growth.never()));
  }
```

```
$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount     
 -Dexec.args="--inputFile=pom.xml --output=counts" -Pdirect-runner
...
java.lang.IllegalArgumentException: 
org.apache.beam.sdk.transforms.Watch$WatchGrowthFn, @ProcessElement 
process(ProcessContext, GrowthTracker): Has tracker type 
Watch.GrowthTracker<OutputT, KeyT, TerminationStateT>, but the DoFn's tracker 
type must be of type RestrictionTracker.
    at 
org.apache.beam.sdk.transforms.reflect.DoFnSignatures$ErrorReporter.throwIllegalArgument
 (DoFnSignatures.java:1507)
    at 
org.apache.beam.sdk.transforms.reflect.DoFnSignatures$ErrorReporter.checkArgument
 (DoFnSignatures.java:1512)
    at 
org.apache.beam.sdk.transforms.reflect.DoFnSignatures.verifySplittableMethods 
(DoFnSignatures.java:593)
    at org.apache.beam.sdk.transforms.reflect.DoFnSignatures.parseSignature 
(DoFnSignatures.java:472)
    at 
org.apache.beam.sdk.transforms.reflect.DoFnSignatures.lambda$getSignature$0 
(DoFnSignatures.java:140)
    at java.util.HashMap.computeIfAbsent (HashMap.java:1128)
    at org.apache.beam.sdk.transforms.reflect.DoFnSignatures.getSignature 
(DoFnSignatures.java:140)
    at org.apache.beam.sdk.transforms.ParDo.validate (ParDo.java:546)
    at org.apache.beam.sdk.transforms.ParDo.of (ParDo.java:393)
    at org.apache.beam.sdk.transforms.Watch$Growth.expand (Watch.java:689)
    at org.apache.beam.sdk.transforms.Watch$Growth.expand (Watch.java:157)
    at org.apache.beam.sdk.Pipeline.applyInternal (Pipeline.java:537)
    at org.apache.beam.sdk.Pipeline.applyTransform (Pipeline.java:488)
    at org.apache.beam.sdk.values.PCollection.apply (PCollection.java:370)
    at org.apache.beam.sdk.io.FileIO$MatchAll.expand (FileIO.java:614)
    at org.apache.beam.sdk.io.FileIO$MatchAll.expand (FileIO.java:572)
    at org.apache.beam.sdk.Pipeline.applyInternal (Pipeline.java:537)
    at org.apache.beam.sdk.Pipeline.applyTransform (Pipeline.java:488)
    at org.apache.beam.sdk.values.PCollection.apply (PCollection.java:370)
    at org.apache.beam.sdk.io.FileIO$Match.expand (FileIO.java:567)
    at org.apache.beam.sdk.io.FileIO$Match.expand (FileIO.java:514)
    at org.apache.beam.sdk.Pipeline.applyInternal (Pipeline.java:537)
    at org.apache.beam.sdk.Pipeline.applyTransform (Pipeline.java:471)
    at org.apache.beam.sdk.values.PBegin.apply (PBegin.java:44)
    at org.apache.beam.sdk.Pipeline.apply (Pipeline.java:167)
    at org.apache.beam.examples.WordCount.runWordCount (WordCount.java:178)
    at org.apache.beam.examples.WordCount.main (WordCount.java:188)
    at sun.reflect.NativeMethodAccessorImpl.invoke0 (Native Method)
    at sun.reflect.NativeMethodAccessorImpl.invoke 
(NativeMethodAccessorImpl.java:62)
    at sun.reflect.DelegatingMethodAccessorImpl.invoke 
(DelegatingMethodAccessorImpl.java:43)
    at java.lang.reflect.Method.invoke (Method.java:498)
    at org.codehaus.mojo.exec.ExecJavaMojo$1.run (ExecJavaMojo.java:282)
    at java.lang.Thread.run (Thread.java:748)
```

> Watch PTransform is broken
> --------------------------
>
>                 Key: BEAM-6352
>                 URL: https://issues.apache.org/jira/browse/BEAM-6352
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>            Reporter: Gleb Kanterov
>            Assignee: Kenneth Knowles
>            Priority: Major
>
> List of affected tests:
> org.apache.beam.sdk.transforms.WatchTest > 
> testSinglePollMultipleInputsWithSideInput FAILED
> org.apache.beam.sdk.transforms.WatchTest > testMultiplePollsWithKeyExtractor 
> FAILED
> org.apache.beam.sdk.transforms.WatchTest > testSinglePollMultipleInputs FAILED
> org.apache.beam.sdk.transforms.WatchTest > 
> testMultiplePollsWithTerminationDueToTerminationCondition FAILED
> org.apache.beam.sdk.transforms.WatchTest > testMultiplePollsWithManyResults 
> FAILED
> org.apache.beam.sdk.transforms.WatchTest > testSinglePollWithManyResults 
> FAILED
> org.apache.beam.sdk.transforms.WatchTest > 
> testMultiplePollsStopAfterTimeSinceNewOutput 
> org.apache.beam.sdk.transforms.WatchTest > 
> testMultiplePollsWithTerminationBecauseOutputIsFinal FAILED
> org.apache.beam.sdk.io.AvroIOTest$NeedsRunnerTests > 
> testContinuouslyWriteAndReadMultipleFilepatterns[0: true] FAILED
> org.apache.beam.sdk.io.AvroIOTest$NeedsRunnerTests > 
> testContinuouslyWriteAndReadMultipleFilepatterns[1: false] FAILED
> org.apache.beam.sdk.io.FileIOTest > testMatchWatchForNewFiles FAILED
> org.apache.beam.sdk.io.TextIOReadTest$BasicIOTest > testReadWatchForNewFiles 
> FAILED
> {code}
> java.lang.IllegalArgumentException: 
> org.apache.beam.sdk.transforms.Watch$WatchGrowthFn, @ProcessElement 
> process(ProcessContext, GrowthTracker): Has tracker type 
> Watch.GrowthTracker<OutputT, KeyT, TerminationStateT>, but the DoFn's tracker 
> type must be of type RestrictionTracker.
> {code}
> Relevant pull requests:
> - https://github.com/apache/beam/pull/6467
> - https://github.com/apache/beam/pull/7374
> Now tests are marked with @Ignore referencing this JIRA issue



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

Reply via email to