On Mon, Apr 8, 2019 at 1:57 AM Robert Bradshaw <rober...@google.com> wrote:

> On Sat, Apr 6, 2019 at 12:08 AM Kenneth Knowles <k...@apache.org> wrote:
> >
> >
> >
> > On Fri, Apr 5, 2019 at 2:24 PM Robert Bradshaw <rober...@google.com>
> wrote:
> >>
> >> On Fri, Apr 5, 2019 at 6:24 PM Kenneth Knowles <k...@apache.org> wrote:
> >> >
> >> > Nested and unnested contexts are two different encodings. Can we just
> give them different URNs? We can even just express the length-prefixed
> UTF-8 as a composition of the length-prefix URN and the UTF-8 URN.
> >>
> >> It's not that simple, especially when it comes to composite encodings.
> >> E.g. for some coders, nested(C) == unnested(C), for some coders
> >> nested(C) == lenth_prefix(unnested(C)), and for other coders it's
> >> something else altogether (e.g. when creating a kv coder, the first
> >> component must use nested context, and the second inherits the nested
> >> vs. unnested context). When creating TupleCoder(A, B) one doesn't want
> >> to forcibly use LenthPrefixCoder(A) and LengthPrefixCoder(B), nor does
> >> one want to force LengthPrefixCoder(TupleCoder(A, B)) because A and B
> >> may themselves be large and incrementally written (e.g.
> >> IterableCoder). Using distinct URNs doesn't work well if the runner is
> >> free to compose and decompose tuple, iterable, etc. coders that it
> >> doesn't understand.
> >>
> >> Until we stop using Coders for IO (a worthy but probably lofty goal)
> >> we will continue to need the unnested context (lest we expect and
> >> produce length-prefixed coders in text files, as bigtable keys, etc.).
> >> On the other hand, almost all internal use is nested (due to sending
> >> elements around as part of element streams). The other place we cross
> >> over is LengthPrefixCoder that encodes its values using the unnested
> >> context prefixed by the unnested encoding length.
> >>
> >> Perhaps a step in the right direction would be to consistently use the
> >> unnested context everywhere but IOs (meaning when we talked about
> >> coders from the FnAPI perspective, they're *always* in the nested
> >> context, and hence always have the one and only encoding defined by
> >> that URN, including when wrapped by a length prefix coder (which would
> >> sometimes result in double length prefixing, but I think that's a
> >> price worth paying (or we could do something more clever like make
> >> length-prefix an (explicit) modifier on a coder rather than a new
> >> coder itself that would default to length prefixing (or some of the
> >> other delimiting schemes we've come up with) but allow the component
> >> coder to offer alternative length-prefix-compatible encodings))). IOs
> >> could be updated to take Parsers and Formatters (or whatever we call
> >> them) with the Coders in the constructors left as syntactic sugar
> >> until we could remove them in 3.0. As Luke says, we have a chance to
> >> fix our coders for portable pipelines now.
> >>
> >> In the (very) short term, we're stuck with a nested and unnested
> >> version of StringUtf8, just as we have for bytes, lest we change the
> >> meaning of (or disallow some of) TupleCoder[StrUtf8Coder, ...],
> >> LengthPrefixCoder[StrUtf8Coder], and using StringUtf8Coder for IO.
> >
> >
> > First, let's note that "nested" and "outer" are a misnomer. The
> distinction is whether it is the last thing encoded in the stream. In a
> KvCoder<KeyCoder, ValueCoder> the ValueCoder is actually encoded in the
> "outer" context though the value is nested. No doubt a good amount of
> confusion comes from the initial and continued use of this terminology.
>
> +1. I think of these as "self-delimiting" vs. "externally-delimited."
>
> > So, all that said, it is a simple fact that UTF-8 and length-prefixed
> UTF-8 are two different encodings. Encodings are the fundamental concept
> here and coders encapsulate two encodings, with some subtle and
> inconsistently-applied rules about when to use which encoding. I think we
> should still give them distinct URNs unless impossible. You've outlined
> some steps to clarify the situation.
>
> Currently, we have a one-to-two relationship between Coders and
> encodings, and a one-to-one relationship between URNs and encoders.
>
> To make the first one-to-one, we would either have to make
> StringUtf8Coder unsuitable for TextIO (letting it always prefix its
> contents with a length) or unsuitable for the key part of a KV, the
> element of an iterable, etc. (where the length is required).
>

Definitely in favor of dropping TextIO from the conversation. Just need to
think about compatibility, but I would save that for after the idealized
end goal is defined, and hack it somehow. to the one-to-one vs one-to-two,
can we have:

 - SDK coder -> proto: be smart about knowing what the context is and uses
the appropriate URN (this is when building the graph)
 - proto -> SDK: instantiate a variant of the coder that is fixed to one
context (this is when you get a process bundle instruction from the runner)

I think this is more-or-less the idea you mean by...

Alternatively we could give Coders the ability to return the
> nested/unnested version of themselves, but this also gets messy
> because it depends on the ultimate outer context which we don't always
> have at hand (and leads to surprises, e.g. asking for the key coder of
> a KV coder may not return the same coder that was used to construct
> it).
>

... and I don't think it would be too messy, because fixing the context
would happen around submission time, yes? Basically, it keep "Java Coder"
as a vestigial concept that compiles away, much like PValue/PInput/POutput.


> On the other hand, breaking the one-to-one relationship of Coders and
> URNs is also undesirable, because it forces a choice when serializing
> to protos (where one may not always know all the contexts it's used
> in)


When would you not know the context? I posit that toProto should always
know the context.


> and makes the round-trip through protos non-idempotent (if the URN
> gets decoded back into a Coder that does not have this dual-defined
> encoding).


This doesn't seem like such a problem. Of course, I would say that, as I am
proposing to do just this :-)

Also, if a runner knows about special pairs of URNs that
> represent the nested/unnested encodings of the same type, and rules
> like knowing constraints on components of KVs, etc., this seems an
> even worse situation than we're already in.
>

Perhaps even worse than this would be to enshrine the idea of
self-delimiting/non-self-delimiting coders as a flag in the proto, so
runners know to flip it. That is really bad, making the model inherit
accidental complexity from the Java SDK. But is this flag always just a
length prefix, in practice? If so, then the smarts to add to toProto would
cause the self-delimiting variant of UTF-8 coder to be a composite
LengthPrefix(UTF-8) coder. Then the model doesn't change and the runner
does not need to be aware of special pairs of URNs.


> > I think the most meaningful issue is runners manipulating coders. But
> the runner should be able to instruct the SDK (where the coder actually
> executes) about which encoding to use. I need to think through some
> examples of where the SDK tells the runner the encoding of something and
> those where the runner fabricates steps and instructs the SDK what encoding
> to use for elements. GroupByKey, GetKeys, etc, are examples where the
> context will change.
>
> Combiner lifting and side inputs are good candidates as well.
>

GBK example:

 - Input: KvCoder[Outer]<KeyCoder[Inner], ValueCoder[Outer]>
 - Output: KvCoder[Outer]<KeyCoder[Inner],
IterableCoder[Outer]<ValueCoder[Inner]>>

In this case, it is the SDK providing both, so it can easily make the
ValueCoder[Inner] a LengthPrefix(ValueCoder[Outer]). If the ValueCoder is
already length-prefixed there is a cost. That could be detected and elided
I think; might need additional methods but not if the ValueCoder toProto
already encoded in a readable way.

GBKVaiGBKO example, where the runner chooses a particular implementation
strategy. Assuming the client language is not Java so inner/outer switching
cannot be done unless explicitly represented in the proto.

 - Input: KvCoder[Outer]<KeyCoder[Inner], ValueCoder[Outer]>
 - Intermediate: KvCoder[Outer]<KeyCoder[Inner],
IterableCoder[Outer]<WindowedValueCoder[Outer]<ValueCoder[Inner]>>>

Since the input ValueCoder is always Outer, it is easy to add a
length-prefix without knowing what it is. So from these examples, I
hypothesize that the hard case is when there is a user coder in Inner
encoding and the runner wants to turn it into Outer encoding, but the Inner
encoding does not explicitly indicate that it is a LengthPrefix(Outer)
encoding. I believe that this is not a problem - the runner can just leave
it as the Inner encoding since every Inner encoding is a safe Outer
encoding (but not vice-versa). I don't know of a real example, but if there
were a runner-inserted GetKeys then the KeyCoder[Inner] might *want* to
change to KeyCoder[Outer] but that wouldn't be a requirement.

For side inputs, PCollection<ValueCoder[Outer]> the runner may want to put
a bunch of elements in some data structure that requires self-delimiting.
Same logic, it is always easy to move from Outer to Inner.

Kenn


> >> > On Fri, Apr 5, 2019 at 12:38 AM Robert Bradshaw <rober...@google.com>
> wrote:
> >> >>
> >> >> On Fri, Apr 5, 2019 at 12:50 AM Heejong Lee <heej...@google.com>
> wrote:
> >> >> >
> >> >> > Robert, does nested/unnested context work properly for Java?
> >> >>
> >> >> I believe so. It is similar to the bytes coder, that prefixes vs. not
> >> >> based on the context.
> >> >>
> >> >> > I can see that the Context is fixed to NESTED[1] and the encode
> method with the Context parameter is marked as deprecated[2].
> >> >> >
> >> >> > [1]:
> https://github.com/apache/beam/blob/0868e7544fd1e96db67ff5b9e70a67802c0f0c8e/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java#L68
> >> >> > [2]:
> https://github.com/apache/beam/blob/0868e7544fd1e96db67ff5b9e70a67802c0f0c8e/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L132
> >> >>
> >> >> That doesn't mean it's unused, e.g.
> >> >>
> >> >>
> https://github.com/apache/beam/blob/release-2.12.0/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java#L160
> >> >>
> https://github.com/apache/beam/blob/release-2.12.0/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/LengthPrefixCoder.java#L64
> >> >>
> >> >> (and I'm sure there's others).
> >> >>
> >> >> > On Thu, Apr 4, 2019 at 3:25 PM Robert Bradshaw <
> rober...@google.com> wrote:
> >> >> >>
> >> >> >> I don't know why there are two separate copies of
> >> >> >> standard_coders.yaml--originally there was just one (though it did
> >> >> >> live in the Python directory). I'm guessing a copy was made rather
> >> >> >> than just pointing both to the new location, but that completely
> >> >> >> defeats the point. I can't seem to access JIRA right now; could
> >> >> >> someone file an issue to resolve this?
> >> >> >>
> >> >> >> I also think the spec should be next to the definition of the URN,
> >> >> >> that's one of the reason the URNs were originally in a markdown
> file
> >> >> >> (to encourage good documentation, literate programming style).
> Many
> >> >> >> coders already have their specs there.
> >> >> >>
> >> >> >> Regarding backwards compatibility, we can't change existing
> coders,
> >> >> >> and making new coders won't help with inference ('cause changing
> that
> >> >> >> would also be backwards incompatible). Fortunately, I think we're
> >> >> >> already doing the consistent thing here: In both Python and Java
> the
> >> >> >> raw UTF-8 encoded bytes are encoded when used in an *unnested*
> context
> >> >> >> and the length-prefixed UTF-8 encoded bytes are used when the
> coder is
> >> >> >> used in a *nested* context.
> >> >> >>
> >> >> >> I'd really like to see the whole nested/unnested context go away,
> but
> >> >> >> that'll probably require Beam 3.0; it causes way more confusion
> than
> >> >> >> the couple of bytes it saves in a couple of places.
> >> >> >>
> >> >> >> - Robert
> >> >> >>
> >> >> >> On Thu, Apr 4, 2019 at 10:55 PM Robert Burke <rob...@frantil.com>
> wrote:
> >> >> >> >
> >> >> >> > My 2cents is that the "Textual description" should be part of
> the documentation of the URNs on the Proto messages, since that's the
> common place. I've added a short description for the varints for example,
> and we already have lenghthier format & protocol descriptions there for
> iterables and similar.
> >> >> >> >
> >> >> >> > The proto [1] *can be* the spec if we want it to be.
> >> >> >> >
> >> >> >> > [1]:
> https://github.com/apache/beam/blob/069fc3de95bd96f34c363308ad9ba988ab58502d/model/pipeline/src/main/proto/beam_runner_api.proto#L557
> >> >> >> >
> >> >> >> > On Thu, 4 Apr 2019 at 13:51, Kenneth Knowles <k...@apache.org>
> wrote:
> >> >> >> >>
> >> >> >> >>
> >> >> >> >>
> >> >> >> >> On Thu, Apr 4, 2019 at 1:49 PM Robert Burke <
> rob...@frantil.com> wrote:
> >> >> >> >>>
> >> >> >> >>> We should probably move the "java" version of the yaml file
> [1] to a common location rather than deep in the java hierarchy, or copying
> it for Go and Python, but that can be a separate task. It's probably
> non-trivial since it looks like it's part of a java resources structure.
> >> >> >> >>
> >> >> >> >>
> >> >> >> >> Seems like /model is a good place for this if we don't want to
> invent a new language-independent hierarchy.
> >> >> >> >>
> >> >> >> >> Kenn
> >> >> >> >>
> >> >> >> >>
> >> >> >> >>>
> >> >> >> >>> Luke, the Go SDK doesn't currently do this validation, but it
> shouldn't be difficult, given pointers to the Java and Python variants of
> the tests to crib from [2]. Care would need to be taken so that Beam Go SDK
> users (such as they are) aren't forced to run them, and not have the yaml
> file to read. I'd suggest putting it with the integration tests [3].
> >> >> >> >>>
> >> >> >> >>> I've filed a JIRA (BEAM-7009) for tracking this Go SDK side
> work. [4]
> >> >> >> >>>
> >> >> >> >>> 1:
> https://github.com/apache/beam/blob/master/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
> >> >> >> >>> 2:
> https://github.com/apache/beam/search?q=standard_coders.yaml&unscoped_q=standard_coders.yaml
> >> >> >> >>> 3: https://github.com/apache/beam/tree/master/sdks/go/test
> >> >> >> >>> 4: https://issues.apache.org/jira/browse/BEAM-7009
> >> >> >> >>>
> >> >> >> >>>
> >> >> >> >>> On Thu, 4 Apr 2019 at 13:28, Lukasz Cwik <lc...@google.com>
> wrote:
> >> >> >> >>>>
> >> >> >> >>>>
> >> >> >> >>>>
> >> >> >> >>>> On Thu, Apr 4, 2019 at 1:15 PM Chamikara Jayalath <
> chamik...@google.com> wrote:
> >> >> >> >>>>>
> >> >> >> >>>>>
> >> >> >> >>>>>
> >> >> >> >>>>> On Thu, Apr 4, 2019 at 12:15 PM Lukasz Cwik <
> lc...@google.com> wrote:
> >> >> >> >>>>>>
> >> >> >> >>>>>> standard_coders.yaml[1] is where we are currently defining
> these formats.
> >> >> >> >>>>>> Unfortunately the Python SDK has its own copy[2].
> >> >> >> >>>>>
> >> >> >> >>>>>
> >> >> >> >>>>> Ah great. Thanks for the pointer. Any idea why there's  a
> separate copy for Python ? I didn't see a significant difference in
> definitions looking at few random coders there but I might have missed
> something. If there's no reason to maintain two, we should probably unify.
> >> >> >> >>>>> Also, seems like we haven't added the definition for UTF-8
> coder yet.
> >> >> >> >>>>>
> >> >> >> >>>>
> >> >> >> >>>> Not certain as well. I did notice the timer coder definition
> didn't exist in the Python copy.
> >> >> >> >>>>
> >> >> >> >>>>>>
> >> >> >> >>>>>>
> >> >> >> >>>>>> Here is an example PR[3] that adds the
> "beam:coder:double:v1" as tests to the Java and Python SDKs to ensure
> interoperability.
> >> >> >> >>>>>>
> >> >> >> >>>>>> Robert Burke, does the Go SDK have a test where it uses
> standard_coders.yaml and runs compatibility tests?
> >> >> >> >>>>>>
> >> >> >> >>>>>> Chamikara, creating new coder classes is a pain since the
> type -> coder mapping per SDK language would select the non-well known type
> if we added a new one to a language. If we swapped the default type->coder
> mapping, this would still break update for pipelines forcing users to
> update their code to select the non-well known type. If we don't change the
> default type->coder mapping, the well known coder will gain little usage. I
> think we should fix the Python coder to use the same encoding as Java for
> UTF-8 strings before there are too many Python SDK users.
> >> >> >> >>>>>
> >> >> >> >>>>>
> >> >> >> >>>>> I was thinking that may be we should just change the
> default UTF-8 coder for Fn API path which is experimental. Updating Python
> to do what's done for Java is fine if we agree that encoding used for Java
> should be the standard.
> >> >> >> >>>>>
> >> >> >> >>>>
> >> >> >> >>>> That is a good idea to use the Fn API experiment to control
> which gets selected.
> >> >> >> >>>>
> >> >> >> >>>>>>
> >> >> >> >>>>>>
> >> >> >> >>>>>> 1:
> https://github.com/apache/beam/blob/master/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
> >> >> >> >>>>>> 2:
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/testing/data/standard_coders.yaml
> >> >> >> >>>>>> 3: https://github.com/apache/beam/pull/8205
> >> >> >> >>>>>>
> >> >> >> >>>>>> On Thu, Apr 4, 2019 at 11:50 AM Chamikara Jayalath <
> chamik...@google.com> wrote:
> >> >> >> >>>>>>>
> >> >> >> >>>>>>>
> >> >> >> >>>>>>>
> >> >> >> >>>>>>> On Thu, Apr 4, 2019 at 11:29 AM Robert Bradshaw <
> rober...@google.com> wrote:
> >> >> >> >>>>>>>>
> >> >> >> >>>>>>>> A URN defines the encoding.
> >> >> >> >>>>>>>>
> >> >> >> >>>>>>>> There are (unfortunately) *two* encodings defined for a
> Coder (defined
> >> >> >> >>>>>>>> by a URN), the nested and the unnested one. IIRC, in
> both Java and
> >> >> >> >>>>>>>> Python, the nested one prefixes with a var-int length,
> and the
> >> >> >> >>>>>>>> unnested one does not.
> >> >> >> >>>>>>>
> >> >> >> >>>>>>>
> >> >> >> >>>>>>> Could you clarify where we define the exact encoding ? I
> only see a URN for UTF-8 [1] while if you look at the implementations Java
> includes length in the encoding [1] while Python [1] does not.
> >> >> >> >>>>>>>
> >> >> >> >>>>>>> [1]
> https://github.com/apache/beam/blob/069fc3de95bd96f34c363308ad9ba988ab58502d/model/pipeline/src/main/proto/beam_runner_api.proto#L563
> >> >> >> >>>>>>> [2]
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java#L50
> >> >> >> >>>>>>> [3]
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/coders/coders.py#L321
> >> >> >> >>>>>>>
> >> >> >> >>>>>>>
> >> >> >> >>>>>>>>
> >> >> >> >>>>>>>>
> >> >> >> >>>>>>>> We should define the spec clearly and have
> cross-language tests.
> >> >> >> >>>>>>>
> >> >> >> >>>>>>>
> >> >> >> >>>>>>> +1
> >> >> >> >>>>>>>
> >> >> >> >>>>>>> Regarding backwards compatibility, I agree that we should
> probably not update existing coder classes. Probably we should just
> standardize the correct encoding (may be as a comment near corresponding
> URN in the beam_runner_api.proto ?) and create new coder classes as needed.
> >> >> >> >>>>>>>
> >> >> >> >>>>>>>>
> >> >> >> >>>>>>>>
> >> >> >> >>>>>>>> On Thu, Apr 4, 2019 at 8:13 PM Pablo Estrada <
> pabl...@google.com> wrote:
> >> >> >> >>>>>>>> >
> >> >> >> >>>>>>>> > Could this be a backwards-incompatible change that
> would break pipelines from upgrading? If they have data in-flight in
> between operators, and we change the coder, they would break?
> >> >> >> >>>>>>>> > I know very little about coders, but since nobody has
> mentioned it, I wanted to make sure we have it in mind.
> >> >> >> >>>>>>>> > -P.
> >> >> >> >>>>>>>> >
> >> >> >> >>>>>>>> > On Wed, Apr 3, 2019 at 8:33 PM Kenneth Knowles <
> k...@apache.org> wrote:
> >> >> >> >>>>>>>> >>
> >> >> >> >>>>>>>> >> Agree that a coder URN defines the encoding. I see
> that string UTF-8 was added to the proto enum, but it needs a written spec
> of the encoding. Ideally some test data that different languages can use to
> drive compliance testing.
> >> >> >> >>>>>>>> >>
> >> >> >> >>>>>>>> >> Kenn
> >> >> >> >>>>>>>> >>
> >> >> >> >>>>>>>> >> On Wed, Apr 3, 2019 at 6:21 PM Robert Burke <
> rob...@frantil.com> wrote:
> >> >> >> >>>>>>>> >>>
> >> >> >> >>>>>>>> >>> String UTF8 was recently added as a "standard coder
> " URN in the protos, but I don't think that developed beyond Java, so
> adding it to Python would be reasonable in my opinion.
> >> >> >> >>>>>>>> >>>
> >> >> >> >>>>>>>> >>> The Go SDK handles Strings as "custom coders"
> presently which for Go are always length prefixed (and reported to the
> Runner as LP+CustomCoder). It would be straight forward to add the correct
> handling for strings, as Go natively treats strings as UTF8.
> >> >> >> >>>>>>>> >>>
> >> >> >> >>>>>>>> >>>
> >> >> >> >>>>>>>> >>> On Wed, Apr 3, 2019, 5:03 PM Heejong Lee <
> heej...@google.com> wrote:
> >> >> >> >>>>>>>> >>>>
> >> >> >> >>>>>>>> >>>> Hi all,
> >> >> >> >>>>>>>> >>>>
> >> >> >> >>>>>>>> >>>> It looks like UTF-8 String Coder in Java and Python
> SDKs uses different encoding schemes. StringUtf8Coder in Java SDK puts the
> varint length of the input string before actual data bytes however
> StrUtf8Coder in Python SDK directly encodes the input string to bytes
> value. For the last few weeks, I've been testing and fixing cross-language
> IO transforms and this discrepancy is a major blocker for me. IMO, we
> should unify the encoding schemes of UTF8 strings across the different SDKs
> and make it a standard coder. Any thoughts?
> >> >> >> >>>>>>>> >>>>
> >> >> >> >>>>>>>> >>>> Thanks,
>

Reply via email to