On Mon, Apr 8, 2019 at 4:03 PM Robert Bradshaw <rober...@google.com> wrote:
> This email is already very long, but in summary I think the right > answer is to just get rid of Outer altogether (except possibly for > IOs, which we'd only preserve for legacy reasons until 3.0). > > - Robert > I had forgotten that compatibility from legacy to portable pipelines is not a concern. So, can this be made into a plan? Would it start from this point: - Java classes have to exist as-is from a user's point of view, for compatibility - Portable pipelines should include only self-delimiting encodings (tiny primitives do not require length prefix, large iterables get special treatment) - When creating a Coder from a portable proto, instantiate one that is fixed to the Inner context I would skip having known relationship between a coder and a self-delimiting variant. 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, >