I think my original message made it sound like what I thought was confusing
was how `Any` works. The scenario that I actually think is confusing is *if
a user registers a coder for a data type, this preference will get ignored
in non-obvious situations and can (and in my scenario, has) result in
non-obvious downstream issues.*

On Fri, Jan 5, 2024 at 12:05 PM Robert Bradshaw via dev <dev@beam.apache.org>
wrote:

> On Fri, Jan 5, 2024 at 7:38 AM Joey Tran <joey.t...@schrodinger.com>
> wrote:
>
>> I've been working with a few data types that are in practice
>> unpicklable and I've run into a couple issues stemming from the `Any` type
>> hint, which when used, will result in the PickleCoder getting used even if
>> there's a coder in the coder registry that matches the data element.
>>
>
> This is likely because we don't know the data type at the time we choose
> the coder.
>
>
>> This was pretty unexpected to me and can result in pretty cryptic
>> downstream issues. In the best case, you get an error at pickling time [1],
>> and in the worse case, the pickling "succeeds" (since many objects can get
>> (de)pickeld without obvious error) but then results in downstream issues
>> (e.g. some data doesn't survive depickling).
>>
>
> It shouldn't be the case that an object depickles successfully but
> incorrectly; sounds like a bug in some custom pickling code.
>
You don't need custom pickling code for this to happen. For a contrived
example, you could imagine some class that caches some state specific to a
local system and saves it to a private local variable. If you pickle one of
these and then unpickle it on a different system, it would've been
unpickled successfully but would be in a bad state.

Rather than mucking around with custom pickling, someone might want to just
implement a coder for their special class instead.


> One example case of the latter is if you flatten a few pcollections
>> including a pcollection generated by `beam.Create([])` (the inferred output
>> type an empty create becomes Any)
>>
>
> Can you add a type hint to the Create?
>
Yeah this fixes the issue, it's just not obvious (or at least to me) that
(1) beam.Create([]) will have an output type of Any (often times the
parameter to beam.Create will be some local variable which makes it less
obvious) and that (2) in this particular case, _how_ downstream
pcollections get decoded will be slightly different. In the worse case, the
issue won't even result in an error at decoding time (as mentioned before),
so then you have to backtrack from some possibly unrelated sounding
traceback.


>
>
>> Would it make sense to introduce a new fallback coder that takes
>> precedence over the `PickleCoder` that encodes both the data type (by just
>> pickling it) and the data encoded using the registry-found coder?
>>
>
> This is essentially re-implementing pickle :)
>
Pickle doesn't use coders from the coder registry which I think is the key
distinction here



> This would have some space ramifications for storing the data type for
>> every element. Of course this coder would only kick in _if_ the data type
>> was found in the registry, otherwise we'd proceed to the picklecoder like
>> we do currently
>>
>
> I do not think we'd want to introduce this as the default--that'd likely
> make common cases much more expensive. IIRC you can manually override the
> fallback coder with one of your own choosing. Alternatively, you could look
> at using copyreg for your problematic types.
>
>
Ah you can indeed override the fallback coder. Okay I'll just do that for
our use of Beam.

For sake of discussion though, I think it'd be a small-ish cost incurred
once per data type in the collection. The first time we run into a data
type, we see if it's the registry and if it is, use that coder, otherwise
cache the result (`types_not_in_registry: set`). All data types not in the
registry could then just be fast tracked to the picklecoder as before.


> [1] https://github.com/apache/beam/issues/29908 (Issue arises from
>> ReshuffleFromKey using `Any` as a pcollection type
>>
>

Reply via email to