Filed JIRA: https://issues.apache.org/jira/browse/BEAM-1448

On Thu, Feb 9, 2017 at 8:48 PM Kenneth Knowles <k...@google.com.invalid>
wrote:

> +1 totally agree that this is hardly documented and worth making clear.
> File a JIRA?
>
> Your write-up is nice, though I have a little to add: The context doesn't
> even really indicate whether it is nested or not; we chose the wrong name.
> It actually indicates whether this is the _last_ object in the stream, so
> the method `isWholeStream()` is much better named. For example, the value
> in KvCoder [1].
>
> We should also emphasize the use of CoderProperties as the best way to test
> a coder. It will test it in multiple contexts, including when there are
> extra bytes, if I recall correctly.
>
> File a JIRA?
>
> Kenn
>
> [1]
>
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java#L89
>
> On Thu, Feb 9, 2017 at 10:12 AM, Aviem Zur <aviem...@gmail.com> wrote:
>
> > Hi,
> >
> > I think improvements can be made to the documentation of `encode` and
> > `decode` methods in `Coder`.
> >
> > A coder may be used to encode/decode several objects using a single
> stream,
> > you cannot assume that the stream the coder encodes to/decodes from only
> > contains bytes representing a single object. For example, when the coder
> is
> > used in an `IterableCoder`, for example in `GroupByKey`.
> >
> > When implementing a coder this needs to be taken into account.
> >
> > The `context` argument in `encode` and `decode` methods provides the
> > necessary information.
> >
> > The existing documentation for these methods does not seem to cover this.
> > If users are not aware of this when implementing these methods it can
> cause
> > errors or skewed results.
> >
> > See:
> > https://github.com/apache/beam/blob/master/sdks/java/
> > core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L126
> > and:
> > https://github.com/apache/beam/blob/master/sdks/java/
> > core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L137
> >
> > This is partially addressed in the documentation of the static `Context`
> > values:
> > `OUTER`:
> > https://github.com/apache/beam/blob/master/sdks/java/
> > core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L72
> > and `NESTED`:
> > https://github.com/apache/beam/blob/master/sdks/java/
> > core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L79
> >
> > However, I think that the documentation of `encode` and `decode` should
> > explain this concept clearly, to avoid confusing users implementing
> coders.
> >
>

Reply via email to