> To clarify, it seems like BYTES type is already supported right

BYTES is, but BYTE (a single byte) is not.  To be honest, until this thread
I didn't realize that there were schema types that weren't supported across
languages, it doesn't seem to be called out anywhere other than the
protobuf definition that only a subset are.  I just assumed them not being
implemented was just a bug.

On Tue, Oct 12, 2021 at 7:36 PM Chamikara Jayalath <[email protected]>
wrote:

>
>
> On Tue, Oct 12, 2021 at 3:33 PM Robert Burke <[email protected]> wrote:
>
>> I think for Go, it's mostly needs a change here:
>>
>>
>> https://github.com/apache/beam/blob/4b11efdf96ea4a471e078ec49906c40ef033aafb/sdks/go/pkg/beam/core/graph/coder/row.go#L187
>>
>> Specifically, returning false when i >= len(nils) (along with the actual
>> bit check for when i < len(nils)).
>>
>>
>>
>> On Tue, Oct 12, 2021, 3:19 PM Robert Bradshaw <[email protected]>
>> wrote:
>>
>>> On Tue, Oct 12, 2021 at 2:04 PM Brian Hulette <[email protected]>
>>> wrote:
>>> >
>>> > I wrote that spec, my intention was to document what the Java
>>> implementation was doing. I'm sorry my poor description has gotten us in
>>> this situation. I agree with Reuven that we should avoid changing the Java
>>> implementation for a few reasons:
>>> > - As Reuven pointed out, we support update compatibility for row coder
>>> in Java. This isn't true for Python, Go (is it?)
>>>
>>> We also support update for Python, but if we're going to have to
>>> change one of them (and it's certainly a bug that they don't agree)
>>> Java has a much larger streaming install base.
>>>
>>> Fortunately, this should be backwards compatible: coders need not
>>> reject trailing zeros, but can also be tolerant of them not being
>>> present. Should be an easy fix (any takers?).
>>>
>>> > - As Steve pointed out, it's more efficient.
>>> > - Personally, my intention from the outset was to match the Java
>>> implementation. I'd prefer to fix my mistake.
>>> >
>>> > I can also provide a few pointers for the original questions:
>>> > - Regarding Python support for additional types, see BEAM-7996 [1]
>>> > - Regarding cross-language testing, see tests for beam:coder:row:v1 in
>>> standard_coders.yaml [2]
>>> >
>>> > Brian
>>> >
>>> > [1] https://issues.apache.org/jira/browse/BEAM-7996
>>> > [2]
>>> https://github.com/apache/beam/blob/master/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
>>> >
>>> >
>>> > On Tue, Oct 12, 2021 at 1:55 PM Steve Niemitz <[email protected]>
>>> wrote:
>>> >>
>>> >> imo supporting stripping trailing 0 bytes from the bitset encoding is
>>> the way to go.  It's both backwards compatible with existing serialized
>>> rows, as well as more space efficient.
>>> >>
>>> >> On Tue, Oct 12, 2021 at 4:52 PM Reuven Lax <[email protected]> wrote:
>>> >>>
>>> >>> Of course. But if we write a spec that doesn't agree with the
>>> existing Java coders, that is also futile. We can't easily change Java
>>> coders due to update compatibility concerns, and we definitely need
>>> portability to work with Java!
>>> >>>
>>> >>> On Tue, Oct 12, 2021 at 1:47 PM Robert Burke <[email protected]>
>>> wrote:
>>> >>>>
>>> >>>> If it's not in the spec it's not Beam,  because any alternative is
>>> Anti Portability ;)
>>> >>>>
>>> >>>> On Tue, Oct 12, 2021, 1:45 PM Reuven Lax <[email protected]> wrote:
>>> >>>>>
>>> >>>>> Row just uses the existing Java BitSetCoder, which predates the
>>> writing of that spec :)
>>> >>>>>
>>> >>>>> On Tue, Oct 12, 2021 at 1:42 PM Robert Burke <[email protected]>
>>> wrote:
>>> >>>>>>
>>> >>>>>> The null fields bitset encoder is defines in the pipeline runner
>>> proto here:
>>> https://github.com/apache/beam/blob/4b11efdf96ea4a471e078ec49906c40ef033aafb/model/pipeline/src/main/proto/beam_runner_api.proto#L976
>>> >>>>>>
>>> >>>>>> Per my reading of the spec, the bit set must include the ceiling
>>> of num_fields/8 bytes, as it doesn't say "trailing bytes for non-nil in
>>> fields may be dropped". However it might be interpreted as that by the that
>>> an empty byte array indicating no nils.  This is what go implements in the
>>> coder.WriteRowHeader and coder.ReadRowHeader functions.
>>> >>>>>>
>>> >>>>>> But that strikes me as a special case for fully populated rows,
>>> not a natural extension of a poorly phrased general rule.
>>> >>>>>>
>>> >>>>>> On Tue, Oct 12, 2021, 1:31 PM Reuven Lax <[email protected]>
>>> wrote:
>>> >>>>>>>
>>> >>>>>>> Do you think that BitSetCoder is incorrect?
>>> >>>>>>>
>>> >>>>>>> On Tue, Oct 12, 2021 at 1:27 PM Steve Niemitz <
>>> [email protected]> wrote:
>>> >>>>>>>>
>>> >>>>>>>> Yeah I believe they're all bugs/missing features in the python
>>> implementation.  The nullable BitSet one is arguably a bug in the java
>>> implementation, but since there's no low-level spec on how Rows are
>>> actually encoded it's hard to say who's right.  I think Go might have the
>>> same bug there, in which case that's two languages doing it "wrong" and one
>>> doing it "right". :P
>>> >>>>>>>>
>>> >>>>>>>> On Tue, Oct 12, 2021 at 4:20 PM Reuven Lax <[email protected]>
>>> wrote:
>>> >>>>>>>>>
>>> >>>>>>>>> These are bugs in Python, correct?
>>> >>>>>>>>>
>>> >>>>>>>>> On Tue, Oct 12, 2021 at 1:18 PM Steve Niemitz <
>>> [email protected]> wrote:
>>> >>>>>>>>>>
>>> >>>>>>>>>> It seems like there's a good amount of incompatibility
>>> between java and python wrt beam Rows.  For example the following are
>>> unsupported in python (that I've noticed so far)
>>> >>>>>>>>>> - BYTE
>>>
>>
> To clarify, it seems like BYTES type is already supported right ? This was
> added here: https://github.com/apache/beam/pull/12324
> BYTES type is also a standard coder:
> https://github.com/apache/beam/blob/1ce290bab031192c22f643cac92bd6470788798d/model/pipeline/src/main/proto/beam_runner_api.proto#L787
> Regarding null values, I think, in addition to fixing the incompatibility,
> we also need to make NullableCoder a standard coder to support x-lang
> PCollections that contain null values (also came up in
> https://issues.apache.org/jira/browse/BEAM-10529)
>
> Are there other types that are missing in Python RowCoder support that are
> not mentioned in https://issues.apache.org/jira/browse/BEAM-7996 ? It
> will be good to know what types we would want to prioritize supporting to
> improve usability of RowCoder for x-lang.
>
> Thanks,
> Cham
>
>
>
>> >>>>>>>>>> - INT16
>>> >>>>>>>>>> - OneOf
>>> >>>>>>>>>>
>>> >>>>>>>>>> Additionally, it seems like nullable fields don't really work
>>> correctly, the java BitSetCoder won't encoding trailing empty bytes in the
>>> BitSet, but the python side is expecting every num_fields / 8 bytes to be
>>> present. [1]
>>> >>>>>>>>>>
>>> >>>>>>>>>> Certainly these are bugs, but in general it seems to point to
>>> a lack of integration testing for xlang interop in general.  I plan on
>>> submitting PRs to fix the bugs above (or at least some of them), are there
>>> tests I can change to better exercise these paths?
>>> >>>>>>>>>>
>>> >>>>>>>>>> [1]
>>> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/coders/row_coder.py#L198
>>>
>>

Reply via email to