We read every piece of feedback, and take your input very seriously.
To see all available qualifiers, see our documentation.
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
The lifecycle of the DoFn is not respected in case of exception in any of the lifecycle methods after setup.
Imported from Jira BEAM-3245. Original Jira may contain additional context. Reported by: iemejia.
The text was updated successfully, but these errors were encountered:
Found that tearDown is never called in Dataflow runner v1 or v2, even pipeline finished normally
Just use a simple DoFn:
pipeline .apply(GenerateSequence.from(0).to(1000)) .apply(ParDo.of(new SomeDoFn()));
where SomeDoFn is
static class SomeDoFn extends DoFn<Long, Long> { protected transient Integer id; @Setup public void setup() { Integer idOld = id; id = ThreadLocalRandom.current().nextInt(0, 1000); LOG.info("@Setup {}, {} at {}", idOld, id, Integer.toHexString(System.identityHashCode(this))); } @StartBundle public void startBundle() { LOG.info("@StartBundle {} at {}", id, Integer.toHexString(System.identityHashCode(this))); } @ProcessElement public void process(@Element Long input, OutputReceiver<Long> receiver) { // LOG.info("@ProcessElement for {} at bundle {}", input, id); receiver.output(input); } @FinishBundle public void finishBundle() throws Exception { LOG.info("@FinishBundle {} at {}", id, Integer.toHexString(System.identityHashCode(this))); } @Teardown public void teardown() { LOG.info("@Teardown {} at {}", id, Integer.toHexString(System.identityHashCode(this))); } @Override protected void finalize() throws Throwable { LOG.info("finalize"); } }
seen log in setup, start/finishBundle, but not teardown.
Sorry, something went wrong.
Checked that for Dataflow legacy runner, teardown is called after a DoFn throw error in processElement or finishBundle, but not Dataflow runner v2.
update: runner v2 case partly fixed by #32522
No branches or pull requests
The lifecycle of the DoFn is not respected in case of exception in any of the lifecycle methods after setup.
Imported from Jira BEAM-3245. Original Jira may contain additional context.
Reported by: iemejia.
The text was updated successfully, but these errors were encountered: