Upon further investigation, this tests always writes to ./target/wordcountresult-00000-of-00002 and ./target/wordcountresult-00001-of-00002. So after a successful test run, any further run without a `clean` will spuriously succeed. I was running via IntelliJ so did not do the ritual `mvn clean` workaround. So reproduction appears to be easy and we could fix the test (if we don't remove it) to use a fresh temp dir.
This seems to point to a bug in waitUntilFinish() and/or Apex if the topology is shut down before this ParDo is run. This is a ParDo with trivial bounded input but with side inputs. So I would guess the bug is either in watermark tracking / readiness of the side input or just how PushbackSideInputDoFnRunner is used. On Wed, Jul 5, 2017 at 12:23 PM, Reuven Lax <re...@google.com.invalid> wrote: > I've done a bit more debugging with logging. It appears that the finalize > ParDo is never being invoked in this Apex test (or at least the LOG.info in > that ParDo never runs). This ParDo is run on a constant element (code > snippet below), so it should always run. > > PCollection<Void> singletonCollection = p.apply(Create.of((Void) null)); > singletonCollection > .apply("Finalize", ParDo.of(new DoFn<Void, Integer>() { > @ProcessElement > public void processElement(ProcessContext c) throws Exception { > LOG.info("Finalizing write operation {}.", writeOperation); > > > On Wed, Jul 5, 2017 at 11:22 AM, Kenneth Knowles <k...@google.com.invalid> > wrote: > > > Data-dependent file destinations is a pretty great feature. We also have > > another change to make to this @Experimental feature, and it would be > nice > > to get them both into 2.1.0 if we can unblock this quickly. > > > > I just tried this too, and failed to reproduce it. But Jenkins and Reuven > > both have a reliable repro. > > > > Questionss: > > > > - Any ideas about how these configurations differ? > > - Does this actually affect users? > > - Once we have another test that catches this issue, can we delete this > > test? > > > > Every other test passes, including the actual example WordCountIT. Since > > the PR doesn't change primitives, it also seems like it is an existing > > issue. And the test seems redundant with our other testing but won't get > as > > much maintenance attention. I don't want to stop catching whatever this > > issue is, though. > > > > Kenn > > > > On Wed, Jul 5, 2017 at 10:31 AM, Reuven Lax <re...@google.com.invalid> > > wrote: > > > > > Hi Thomas, > > > > > > This only happens with https://github.com/apache/beam/pull/3356. > > > > > > Reuven > > > > > > On Mon, Jul 3, 2017 at 6:11 AM, Thomas Weise <t...@apache.org> wrote: > > > > > > > Hi Reuven, > > > > > > > > I'm not able to reproduce the issue locally. I was hoping to see > which > > > > thread is attempting to emit the results. In Apex, only the operator > > > thread > > > > can emit the results, any other thread that is launched by the > operator > > > > cannot. I'm not aware of ParDo managing separate threads though and > > > assume > > > > this must be a race. If you still have the log, can you send it to > me? > > > > > > > > Thanks, > > > > Thomas > > > > > > > > > > > > > > > > On Sat, Jul 1, 2017 at 5:51 AM, Reuven Lax <re...@google.com.invalid > > > > > > wrote: > > > > > > > > > pr/3356 fails in the Apex WordCountTest. The failed test is here > > > > > <https://builds.apache.org/job/beam_PreCommit_Java_ > > > > > MavenInstall/12829/org.apache.beam$beam-runners-apex/ > > > > > testReport/org.apache.beam.runners.apex.examples/WordCountTest/ > > > > > testWordCountExample/> > > > > > : > > > > > > > > > > Upon debugging, it looks like this is likely a problem in the Apex > > > runner > > > > > itself. A ParDo calls output(), and that triggers an exception > thrown > > > > from > > > > > inside the Apex runner. The Apex runner calls emit on a > > > DefaultOutputPort > > > > > (ApexParDoOperator.java:275), and that throws an exception inside > of > > > > > verifyOperatorThread(). > > > > > > > > > > I'm going to ignore this failure for now as it seems unrelated to > my > > > PR, > > > > > but does someone want to take a look? > > > > > > > > > > Reuven > > > > > > > > > > > > > > >