On Fri, Jan 28, 2022 at 9:05 AM Kellen Dye <[email protected]> wrote:
> I manually unwrapped a bunch of the code while debugging and synchronizing > on ctxt worked in that case; I haven't built new beam artifacts. I can > submit a PR (effectively just adding a synchronized block), but am/was > unsure if beam maintainers had some other insight into the jackson-beam > interface. I am just a dilettante when it comes to these internals. > > The multithreaded access is happening because DESERIALIZATION_CONTEXT is > reused and (I think) incorrectly accessed in the beam code in a way that > isn't according to the jackson guarantees. In the short term synchronizing > on ctxt should be enough, but encouraging jackson to make this class thread > safe would also be useful. > I think you nailed it. It is a Beam bug. Making DESERIALIZATION_CONTEXT a ThreadLocal seems correct if it is performance sensitive, or just a locally allocated object if not. It looks like the usage in deserializeNode [1] would be on a hot path. Kind of weird since the same DESERIALIZATION_CONTEXT is used to build the jsonDeserializer (which is cached, so not on a hot path [2]) and then *also* passed to its deserialize method. The inconsistency between it being a construction-time constant vs "actually doing work" time constant makes me think something is awry. But anyhow I'd make it a ThreadLocal based on my quick read and your analysis. Entirely possible that I am misreading things. Kenn [1] https://github.com/apache/beam/blob/b52762bf150cacceb0fdeb1f0dc85cbea6e6f39c/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java#L1795 [2] https://github.com/apache/beam/blob/b52762bf150cacceb0fdeb1f0dc85cbea6e6f39c/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java#L1770 > > We're seeing it specifically because we have large numbers of pipelines > with multiple test suites (which by default run in parallel in scalatest). > These construct TestPipeline instances which causes the > serialization-deserialization roundtrip. > > Cheers, > Kellen > > > > > On Fri, Jan 28, 2022 at 11:46 AM Kenneth Knowles <[email protected]> wrote: > >> If I understand correctly, the documentation does imply that such >> synchronization at [2] should not be necessary and that [4] and [5] are >> correct. Of course it would be better if the docs said _how_ this guarantee >> is provided, or perhaps more than that it is a matter of restating it as a >> requirement for any consumer of a DeserializationContext. >> >> That said, if there's a bug causing the DeserializationContext to be >> accessed concurrently, then working around it with additional >> synchronization seems reasonable since a ValueProvider should be >> deserialized infrequently (not on the critical data path). Do you know how >> the multithreaded access is occurring? >> >> Kenn >> >> On Fri, Jan 28, 2022 at 7:30 AM Kerry Donny-Clark <[email protected]> >> wrote: >> >>> Hi Kellen, thanks for figuring this out. Have you implemented >>> synchronizing on ctxt, or do you need someone else to try that? >>> >>> On Fri, Jan 28, 2022 at 12:36 AM Kellen Dye <[email protected]> wrote: >>> >>>> We (Spotify) are experiencing flaky tests on beam 2.35.0 as a result of >>>> NullPointerExceptions during pipeline construction. >>>> >>>> Stacktrace [1]. Root cause appears to be multiple threads >>>> accessing DeserializationContext from >>>> ValueProvider.Deserializer.deserialize [2] which, according to the javadocs >>>> "is guaranteed to only be used from single-threaded context" [3] >>>> >>>> In the context internals, the mutable _currentType is set [4] then >>>> accessed a few lines later [5], but in a multithreaded situation the writes >>>> clobber each other and a call to next() can fail with a NPE. >>>> >>>> I think synchronizing on ctxt in [2] should be sufficient to avoid this >>>> issue, but I'm not sure if the NPEs are due to beam misuse or something on >>>> the jackson side that should be changed. >>>> >>>> Failing (scala) test here, scalatest runs tests in parallel: >>>> >>>> https://gist.github.com/kellen/124185463c16a66167a7fa704147c510#file-fail-scala >>>> >>>> Test stacktrace: >>>> >>>> https://gist.github.com/kellen/124185463c16a66167a7fa704147c510#file-test_stacktrace-txt >>>> >>>> Cheers, >>>> Kellen >>>> >>>> [1] >>>> https://gist.github.com/kellen/124185463c16a66167a7fa704147c510#file-stacktrace-txt >>>> [2] >>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java#L359 >>>> [3] >>>> https://github.com/joansmith/jackson-databind/blob/master/src/main/java/com/fasterxml/jackson/databind/DeserializationContext.java#L36 >>>> [4] >>>> https://github.com/joansmith/jackson-databind/blob/master/src/main/java/com/fasterxml/jackson/databind/DeserializationContext.java#L652 >>>> [5] >>>> https://github.com/joansmith/jackson-databind/blob/master/src/main/java/com/fasterxml/jackson/databind/DeserializationContext.java#L656 >>>> >>>> >>>> > > -- > > Kellen Dye > > Senior Data Engineer, flatmap > > Slack: @kellend | Phone: (646) 951 3457 <(646)%20951-3457> > > This e-mail (including any attachments) may contain information that is > confidential and/or privileged. It is intended only for the recipient(s). > If you have reason to believe that you are not the intended recipient of > this e-mail, please contact the sender immediately and delete the e-mail > from your computer. > >
