[
https://issues.apache.org/jira/browse/BEAM-6352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16742565#comment-16742565
]
Boyuan Zhang commented on BEAM-6352:
------------------------------------
Researching a little bit. The root cause is in Luke's
PR([https://github.com/apache/beam/pull/6467]), there is a checkArguement
changes:
{code}
processElementErrors.checkArgument(processElement.trackerT().getRawType().equals(RestrictionTracker.class),
"Has tracker type %s, but the DoFn's tracker type must be of type
RestrictionTracker.",
formatType(processElement.trackerT()));
{code}
which enforces a DoFn use RestrictionTracker in processElement rather than a
subtype of RestrictionTracker.
But in the WatchGrowthFn, the usage of tracker is not only limited to
RestrictionTracker,
[example|https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java#L727].
There are 2 possible solutions from my knowlagde:
1. Expose the exact tracker from
[RestrictionTrackerObserver|https://github.com/apache/beam/blob/master/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java#L42]
2. In the checkArguement section, make WatchGrowthTracker as a special case.
I'm not super familiar with Watch.java so I cannot tell which is the right way
to go.
> 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
> Affects Versions: 2.9.0
> Reporter: Gleb Kanterov
> Assignee: Boyuan Zhang
> Priority: Blocker
> Fix For: 2.10.0
>
>
> 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)