You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Jan Lukavský <je...@seznam.cz> on 2021/09/01 06:47:47 UTC

Re: Unexpected in TestStream in Portable Mode

Sorry, I needed some time to let that sink in. :-)

I think I understand why (and how) this will work for TestStream, still 
have a couple of notes, though:

  a) the problem of type compatibility arises with the primitive Read as 
well, though we can solve it with different expansion for TestStream, 
that solution is not applicable to Read, because it has different contract

  b) the same exact problem problem will arise every time we inline any 
transform that would otherwise be run in an environment

  c) extracting bytes from TestStream actually bypasses "model coders" 
on the side of runner, should we do that in other cases as well?

The TestStream problem is a special case, I think a more generic 
solution would be better. Two questions:

  1) Could we create a mapping in runners-core-construction-java that 
would take Pipeline proto, and PCollectionId and create a mapping 
function from "sdk coder space" to "runner coder space"? That could be 
optimized to identity if the coder of the PCollection consists of model 
coders only. In that case the sdk coder and runner coder are identical, 
producing the same types. This mapping function could be reused by both 
portable TestStream, inlined primitive Read and any other future inlined 
transform.

  2) Why does runner need to understand the types it processes and does 
not work with raw bytes all the times, including model coders, the same 
way as is the proposed solution for TestStream, but for all transforms? 
The first step to every executable stage would then be to decode the 
input from raw bytes and only then process it. What is the benefit of 
runner understanding _some_ of the coders? It would be required for a 
runner to understand coders for root transforms (Impulse? any other?), 
but are there any other places where this is necessary?

On 9/1/21 1:37 AM, Robert Bradshaw wrote:
> On Tue, Aug 31, 2021 at 4:31 PM Jan Lukavský <je...@seznam.cz> wrote:
>> If I can depict that:
>>
>> TestStream0 -> TestStream1 (DoFn), that is
>>
>> TestStream0 -> TestStream1 (executable stage) -> consumer 2 (executable
>> stage)
> Yes. Expanding it out more,
>
> TestStream0 -> pc1 -> TestStream1 -> pc2 -> Consumer2 -> ...
>
> Where pc1 is a PCollection<bytes> and pc2 is a PCollection<T>.
>
>> In this scenario TestStream0 can produce byte[], and TestStream1 can
>> consume them, the problem is that if the coder of TestStream0
> I assume you mean TestStream1, as TestStream0 has no coder.
>
>> is some
>> (recursively) known model coder (e.g. KvCoder), then consumer 2 will not
>> be able to decode that data.
> We know that, by construction, TestStream1 and Consumer2 both are
> executed in an environment that understands pc2's Coder<T>. The
> difference here is that the runner need not understand Coder<T> as it
> could inject length prefixing on pc2 if necessary.
>
>
>> On 9/1/21 1:24 AM, Robert Bradshaw wrote:
>>> On Tue, Aug 31, 2021 at 4:12 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>> Fortunately, there's no value in having the primitive TestStream
>>>>> produce elements of arbitrary type (vs. sources, where the read is
>>>>> inherently typed depending on the source and particular library used
>>>>> for reading it).
>>>> I disagree with this one - the consumer(s) of elements from TestStream
>>>> are (should be treated as) unknown to the runner, might be inlined,
>>>> might be x-lang and whether or not will pass the boundary of executable
>>>> stage, all these questions are important. The coder (and if it is model
>>>> coder or not, and what are the sub-coders) must be precisely known to
>>>> all parties that cooperate on the computation, because otherwise these
>>>> parties might not agree on the binary representation.
>>> What I'm saying is that there's little value in a  primitive
>>> TestStream that can produce all sorts of T, vs. a composite TestStream
>>> consisting of a TestStream0 primitive producing only bytes, followed
>>> by a TestStream1 that decodes those bytes into T. This sidesteps all
>>> issues of coder compatibility, as bytes is well-known and TestStream1
>>> is just a DoFn.
>>>
>>>> On 9/1/21 12:59 AM, Robert Bradshaw wrote:
>>>>> Due to issues with Flink executing Reads as SDFs, there is an
>>>>> alternative mode where Read is treated as a primitive and executed
>>>>> directly in the runner. This, of course, requires that the Read's UDF
>>>>> be implemented in the same language as the Runner (specifically, in
>>>>> this case, Java) and their interpretation of what Coders should or
>>>>> should not be wrapped agree (lest other complications arise).
>>>>>
>>>>> It's true one can view TestStream as a source, but it's a bit more
>>>>> than that as it has the ability to wait for quiessence before emitting
>>>>> more elements/watermark updates to give stronger guarantees on
>>>>> determinism, which generally requires deeper runner integration. In
>>>>> addition, TestStream is not associated with a UDF or specific SDK the
>>>>> way Sources are which is where the problem arises: TestStream gives
>>>>> the encoded form of the elements but the runner may need to produce a
>>>>> different encoded form of the elements (say, one with length
>>>>> prefixing) which is not possible without knowing the Coder.
>>>>> Fortunately, there's no value in having the primitive TestStream
>>>>> produce elements of arbitrary type (vs. sources, where the read is
>>>>> inherently typed depending on the source and particular library used
>>>>> for reading it).
>>>>>
>>>>>
>>>>> On Tue, Aug 31, 2021 at 3:41 PM Ke Wu <ke...@gmail.com> wrote:
>>>>>> Read does not have translation in portability, so the implementation is that it needs to be primitive transform explicitly implemented by the runner. The encoding/decoding has to happen in the runner.
>>>>>>
>>>>>>
>>>>>> Could you help me understand this a bit more? IIRC, Read is NOT being translated in portable mode exactly means it is a composite transform instead of primitive because all primitive transforms are required to be translated. In addition, Read is a composite transform of Impulse, which produces dummy bytes [1] to trigger subsequent ParDo/ExecutableStage, where decoding the actual source happens [2]
>>>>>>
>>>>>> There seems to be no role of the SDK harness with regard to the TestStream, because the elements are already encoded by the submitting SDK. The coders must match nevertheless, because you can have Events of type KV<KV<WindowedValue<Integer, Object>>> and what will and what will not get length-prefixed depends on which parts exactly are "known" (model) coders and which are not. Encoding the whole value as single byte array will not work for the consuming SDK harness, which will see that there should be nested KvCoders instead.
>>>>>>
>>>>>>
>>>>>> I don’t think I fully understand what you say here. TestStream is currently a primitive transform, therefore there is no role of SDK harness. This is what the proposal to change, to make TestStream a composite transform with a primitive transform and subsequent ParDo to decode to the desired format.
>>>>>>
>>>>>>
>>>>>> [1] https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Impulse.java#L39
>>>>>> [2] https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L149
>>>>>>
>>>>>> On Aug 31, 2021, at 3:21 PM, Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>
>>>>>> On 9/1/21 12:13 AM, Ke Wu wrote:
>>>>>>
>>>>>> Hi Jan,
>>>>>>
>>>>>> Here is my understanding,
>>>>>>
>>>>>> Runner is being brought up by job server driver, which is up and running before the job submission, i.e. it is job agnostic. Therefore, the runner it brought up does not have any SDK coder available and artifact staging only happens for SDK workers.
>>>>>>
>>>>>> You are right that Read and TestStream are sources, however the one thing that distinguish them is that Read transform is a composite transform and the decoding happens in ParDo/ExecutableStage, i.e. on SDK worker.
>>>>>>
>>>>>> Read does not have translation in portability, so the implementation is that it needs to be primitive transform explicitly implemented by the runner. The encoding/decoding has to happen in the runner.
>>>>>>
>>>>>>
>>>>>> The proposal here is also to make the public facing TestStream transform a composite transform instead of primitive now, so that the decoding would occur on the SDK worker side where SDK coder is available, and the primitive that powers TestStream, which will be directly translated by runner to always produce raw bytes, and these raw bytes will be decoded on the SDK worker side.
>>>>>>
>>>>>> There seems to be no role of the SDK harness with regard to the TestStream, because the elements are already encoded by the submitting SDK. The coders must match nevertheless, because you can have Events of type KV<KV<WindowedValue<Integer, Object>>> and what will and what will not get length-prefixed depends on which parts exactly are "known" (model) coders and which are not. Encoding the whole value as single byte array will not work for the consuming SDK harness, which will see that there should be nested KvCoders instead.
>>>>>>
>>>>>>
>>>>>> Best,
>>>>>> Ke
>>>>>>
>>>>>> On Aug 31, 2021, at 2:56 PM, Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>
>>>>>> Sorry if I'm missing something obvious, but I don't quite see the difference between Read and TestStream regarding the discussed issue with coders. Couple of thoughts:
>>>>>>
>>>>>>     a) both Read and TestStream are _sources_ - they produce elements that are consumed by downstream transforms
>>>>>>
>>>>>>     b) the coder of a particular PCollection is defined by the Pipeline proto - it is the (client side) SDK that owns the Pipeline and that defines all the coders
>>>>>>
>>>>>>     c) runners must adhere to these coders, because otherwise there is risk of coder mismatch, most probably on edges like x-lang transforms or inlined transforms
>>>>>>
>>>>>> I tried the approach of encoding the output of Read into byte array as well, but that turns out to have the problem that once there is a (partially) known coder in play, this does not work, because the consuming transform (executable stage) expects to see the wire coder - that is not simply byte array, because the type of elements might be for instance KV<K, V>, where KvCoder is one of ModelCoders. That does not encode using LengthPrefixCoder and as such will be incompatible with LengthPrefixCoder(ByteArrayCoder). The TestStream needs to know the coder of elements, because that defines where exactly must or must not be inserted length-prefixing. The logic in LengthPrefixUnknownCoders [1] is recursive for ModelCoders.
>>>>>>
>>>>>> [1] https://github.com/apache/beam/blob/ff70e740a2155592dfcb302ff6303cc19660a268/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>
>>>>>> On 8/31/21 11:29 PM, Ke Wu wrote:
>>>>>>
>>>>>> Awesome! Thank you Luke and Robert.
>>>>>>
>>>>>> Also created https://issues.apache.org/jira/browse/BEAM-12828 to track unit test conversion. I could take it after I updated Samza runner to support TestStream in portable mode.
>>>>>>
>>>>>> On Aug 31, 2021, at 2:05 PM, Robert Bradshaw <ro...@google.com> wrote:
>>>>>>
>>>>>> Created https://issues.apache.org/jira/browse/BEAM-12827 to track this.
>>>>>>
>>>>>> +1 to converting tests to just use longs for better coverage for now.
>>>>>>
>>>>>> Also, yes, this is very similar to the issues encountered by Reads,
>>>>>> but the solution is a bit simpler as there's no need for the
>>>>>> TestStream primitive to interact with the decoded version of the
>>>>>> elements (unlike Reads, where the sources often give elements in
>>>>>> un-encoded form) and no user code to run.
>>>>>>
>>>>>> - Robert
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Tue, Aug 31, 2021 at 11:00 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>
>>>>>>
>>>>>> This looks (and likely has the same cause) similar to what I have experienced when making primitive Read supported by Flink. The final solution would be to make SDK coders known to the runner of the same SDK (already present in various different threads). But until then, the solution seems to be something like [1]. The root cause is that the executable stage expects its input to be encoded by the SDK harness, and that part is missing when the transform is inlined (like Read in my case, or TestStream in your case). The intoWireTypes method simulates precisely this part - it encodes the PCollection via coder defined in the SDK harness and then decodes it by coder defined by the runner (which match on binary level, but produce different types).
>>>>>>
>>>>>> Jan
>>>>>>
>>>>>> [1] https://github.com/apache/beam/blob/dd7945f9f259a2989f9396d1d7a8dcb122711a52/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java#L657
>>>>>>
>>>>>> On 8/31/21 7:27 PM, Luke Cwik wrote:
>>>>>>
>>>>>> I originally wasn't for making it a composite because it changes the "graph" structure but the more I thought about it the more I like it.
>>>>>>
>>>>>> On Tue, Aug 31, 2021 at 10:06 AM Robert Bradshaw <ro...@google.com> wrote:
>>>>>>
>>>>>>
>>>>>> On Tue, Aug 31, 2021 at 9:18 AM Luke Cwik <lc...@google.com> wrote:
>>>>>>
>>>>>>
>>>>>> On Mon, Aug 30, 2021 at 7:07 PM Ke Wu <ke...@gmail.com> wrote:
>>>>>>
>>>>>>
>>>>>> Hello everyone,
>>>>>>
>>>>>> This is Ke. I am working on enable TestStream support for Samza Runner in portable mode and discovers something unexpected.
>>>>>>
>>>>>> In my implementation for Samza Runner, couple of tests are failing with errors like
>>>>>>
>>>>>>
>>>>>> java.lang.ClassCastException: java.lang.Integer cannot be cast to [B
>>>>>>
>>>>>> I noticed these tests have the same symptom on Flink Runner as well, which are currently excluded:
>>>>>>
>>>>>> https://issues.apache.org/jira/browse/BEAM-12048
>>>>>> https://issues.apache.org/jira/browse/BEAM-12050
>>>>>>
>>>>>>
>>>>>> After some more digging, I realized that it is because the combination of following facts:
>>>>>>
>>>>>> TestStream is a primitive transform, therefore, Runners are supposed to translate directly, the most intuitive implementation for each runner to do is to parse the payload to decode TestStream.Event [1] on the Runner process to be handed over to subsequent stages.
>>>>>> When TestStream used with Integers, i.e. VarIntCoder to initialize, since VarIntCoder is NOT a registered ModelCoder [2], it will be treated as custom coder during conversion to protobuf pipeline [3] and will be replaced with byte array coder [4] when runner sends data to SDK worker.
>>>>>> Therefore an error occurs because the decoded TestStream.Event has Integer as its value but the remote input receiver is expecting byte array, causing java.lang.ClassCastException: java.lang.Integer cannot be cast to [B
>>>>>>
>>>>>>
>>>>>> In addition, I tried to update all these failed tests to use Long instead of Integer, and all tests will pass since VarLongCoder is a known coder. I do understand that runner process does not have user artifacts staged so it can only use coders in  beam model when communicating with SDK worker process.
>>>>>>
>>>>>> Couple of questions on this:
>>>>>>
>>>>>> 1. Is it expected that VarIntegerCoder is not a known coder?
>>>>>>
>>>>>>
>>>>>>
>>>>>> Yes since no one has worked to make it a well known coder.
>>>>>>
>>>>>>
>>>>>> The notion of "integer" vs. "long" is also language-specific detail as
>>>>>> well, so not sure it makes sense as a well-known coder.
>>>>>>
>>>>>> It can be made a well known coder and this would solve the immediate problem but not the long term issue of portable TestStream not supporting arbitrary types.
>>>>>>
>>>>>>
>>>>>> +1. Rather than making coder a property of TestStream, I would be in
>>>>>> favor of the TestStream primitive always producing bytes (basically,
>>>>>> by definition), and providing a composite that consists of this
>>>>>> followed by a decoding to give us a typed TestStream.
>>>>>>
>>>>>>
>>>>>> 2. Is TestStream always supposed to be translated the payload as raw bytes in order that runner process can always send it to SDK worker with the default byte array coder and asks SDK worker to decode accordingly?
>>>>>>
>>>>>>
>>>>>>
>>>>>> Having the runner treat it always as bytes and not T is likely the best solution but isn't necessary.
>>>>>>
>>>>>> 3. If Yes to 2), then does it mean, TestStream needs to be translated in a completely different way in portable mode from classic mode since in classic mode, translator can directly translates the payload to its final format.
>>>>>>
>>>>>>
>>>>>> There are a few ways to fix the current implementation to work for all types. One way would be if we required the encoded_element to be the "nested" encoding and then ensured that the runner uses a WindowedValue<ByteArrayCoder in outer context> and the SDK used WindowedValue<T> (note that this isn't WindowedValue<LengthPrefix<T>>) for the wire coders. This is quite annoying cause the runner inserts length prefixing in a lot of places (effectively every time it sees an unknown type) so we would need to special case this and propagate this correction through any runner native transforms (e.g. GBK) until the SDK consumes it.
>>>>>>
>>>>>> Another way would be to ensure that the SDK always uses LengthPrefix<T> as the PCollection encoding and the encoded_element format. This would mean that the runner can translate it to a T if it so chooses and won't have the annoying special case propagation logic. This leaks the length prefixing into the SDK at graph construction time which is not what it was meant for.
>>>>>>
>>>>>> Swapping to use an existing well known type is by far the easiest approach as you had discovered and won't impact the correctness of the tests.
>>>>>>
>>>>>>
>>>>>> Best,
>>>>>> Ke
>>>>>>
>>>>>>
>>>>>> [1] https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java#L52
>>>>>> [2] https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java#L65
>>>>>> [3] https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L99
>>>>>> [4] https://github.com/apache/beam/blob/master/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L93
>>>>>>
>>>>>>
>>>>>>
>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
JIRAs at https://issues.apache.org/jira/browse/BEAM-12934,
https://issues.apache.org/jira/browse/BEAM-12933.

PR for Python at https://github.com/apache/beam/pull/15550

On Wed, Sep 22, 2021 at 8:55 AM Robert Bradshaw <ro...@google.com> wrote:

> That is correct. Technically one only needs to take the intersection of
> coders for environments that could possibly be consumers, but in my Python
> fix I simply take the intersection of coders over all environments.
>
> On Wed, Sep 22, 2021 at 7:57 AM Jan Lukavský <je...@seznam.cz> wrote:
>
>> Hi,
>>
>> picking this once again, as it seems to me, that the problem is even more
>> complex than simply consulting _one_ (producer) SDK for capabilities.
>> Because of cross-language, it is possible, that the SDK that produces the
>> data can have a different set of standard coders, than the SDK that will
>> consume the data. Moreover, it is even possible, that there will be
>> multiple consumers of the same PCollection written in different SDKs, which
>> would mean, that we need to take the intersection of standard coders of
>> producer and all consumers to be able to deduce the final wire coder. Is
>> this correct? Do we have a tracking Jira for fixing the Java and Python
>> runner-core?
>>
>>  Jan
>> On 9/9/21 7:29 PM, Robert Bradshaw wrote:
>>
>> You are right, Java does not implement this correctly. It should be
>> querying the capabilities section of the environment proto. (For java
>> environments, this is populated from ModelCoders, e.g.
>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386
>> )
>>
>> Looks like Python doesn't do any better:
>> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459
>>
>>
>>
>> On Thu, Sep 9, 2021 at 10:13 AM Jan Lukavský <je...@seznam.cz> wrote:
>>
>>> This makes a *lot* of sense. But it seems to me, that this is not the
>>> way Java-based runners - that use runners-core-construction-java module -
>>> handle it. If I interpret it correctly, then the set of ModelCoders is
>>> hard-coded [1] and essentially required to be known by all SDKs [2].
>>>
>>> There seems to be no negotiation between what SDK harness knows and what
>>> the runner knows. The runner might be able to define the wire coder for the
>>> SDK (via the ProcessBundleDescriptor), but the SDK (for Java runners) seems
>>> not to be able to play any role in this [3]. Therefore I think that if an
>>> SDK does not know the set of Java ModelCoders, then the runner and the SDK
>>> might not agree on the binary encoding of the elements (BTW, which is why
>>> Java Coders cannot be part of ModelCoders and I finally understand why I
>>> had such troubles adding it there - it cannot be there).
>>>
>>> Is it possible we are missing some part of this runner-to-sdk coder
>>> negotiation in runners-core-contruction-java?
>>>
>>> [1]
>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75
>>>
>>> [2]
>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62
>>>
>>> [3]
>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104
>>> On 9/9/21 12:18 AM, Robert Bradshaw wrote:
>>>
>>> The whole notion of an absolute set of known coders is a misnomer. It
>>> would require all Runners and SDKs to be updated synchronously for every
>>> new coder they might want to share.
>>>
>>> Instead, what we have are
>>>
>>> * Standard Coders which have well-defined, language-agnostic
>>> representations and encodings, which may be used for interoperability and
>>> efficiency, and
>>> * Required Coders which are the minimum needed to execute the pipeline.
>>>
>>> The latter consists only of bytes (for impulse), kv and iterable (for
>>> GBK), windowed value (for windowing information) and length prefix (to be
>>> able to handle anything else).
>>>
>>>
>>> On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <ro...@frantil.com> wrote:
>>>
>>>> Is the claim that the Standard bytes and String_utf8 coders are not
>>>> "known coders"?
>>>>
>>>>  What's the point of the standard coders if they are not the canonical
>>>> "known coders" that can generally be expected to be known by runners/other
>>>> SDKs?
>>>>
>>>>
>>>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>>>>
>>>> The Go SDK rather heavily biases towards using the standard coders for
>>>> their closes language equivalents rather than going into override/custom
>>>> specified soup. (It's not possible to globally override the coders for the
>>>> '[]byte' and 'string' types, nor is there often reason to.)
>>>>
>>>> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <jr...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Hey all,
>>>>>>
>>>>>> Just catching up on the thread since I did the TestStream Go SDK
>>>>>> implementation. The discussion about length prefixing behavior for known
>>>>>> vs. unknown coders is interesting, since we ran into strings and byte
>>>>>> slices getting extra length prefixes attached to them by Flink despite
>>>>>> being known coders.
>>>>>>
>>>>>
>>>>> Known to who?
>>>>>
>>>>>
>>>>>> Based on what's been said, that isn't expected behavior, right?
>>>>>>
>>>>>
>>>>> No, it's not.
>>>>>
>>>>> I would check to make sure the Go SDK is respecting the Coder (length
>>>>> prefixed or not) that's set on the channel, rather than guessing at what it
>>>>> expects it to be based on the Go type.
>>>>>
>>>>>
>>>>>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>
>>>>>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>>>>>
>>>>>>> Historically the DataflowRunner has been much more careful about not
>>>>>>> breaking update, since this is a frequent operation by Dataflow users. I
>>>>>>> think we've been less careful aboutt other runners, but as we see clearly
>>>>>>> here Fllnk users do care about this as well, so we should probably test
>>>>>>> upgrade compatibility for Flink.
>>>>>>>
>>>>>>> One strategy that Dataflow uses is to avoid embedding the Java
>>>>>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>>>>>> breakage (as we see with the issue you llnked to). Possibly similar
>>>>>>> strategies should be investigated for Fllink.
>>>>>>>
>>>>>>> +1, that would be great!
>>>>>>>
>>>>>>>
>>>>>>> Reuven
>>>>>>>
>>>>>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>>
>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>> iterable)
>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>> > changing coders is hard (due to no way to update the encoding for
>>>>>>>> a
>>>>>>>> > streaming pipeline).
>>>>>>>> This is unrelated, but - regarding changing coders due to concerns
>>>>>>>> about
>>>>>>>> pipeline upgrades, we break this quite often, at least for some
>>>>>>>> runners.
>>>>>>>> Most recently [1].
>>>>>>>>
>>>>>>>> > It is currently the latter for runners using this code (which not
>>>>>>>> all
>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>> This is interesting and unexpected for me. How do runners decide
>>>>>>>> about
>>>>>>>> how they encode elements between SDK harness and the runner? How do
>>>>>>>> they
>>>>>>>> inform the SDK harness about this decision? My impression was that
>>>>>>>> this
>>>>>>>> is well-defined at the model level. If not, then we have the reason
>>>>>>>> for
>>>>>>>> misunderstanding in this conversation. :-)
>>>>>>>>
>>>>>>>>   Jan
>>>>>>>>
>>>>>>>> [1]
>>>>>>>>
>>>>>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>>>>>
>>>>>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>>>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>>>>>> >>
>>>>>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>> >>>> Hi Robert,
>>>>>>>> >>>>
>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>> reasons,
>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>> delimiting)
>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>> encoded bytes,
>>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>>> SDK. The
>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>>>> for an
>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>> >>>>>
>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>>>> already
>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>> complicate the
>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>> sending the
>>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>>> solved
>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>> known by the
>>>>>>>> >>>>> runner).)
>>>>>>>> >>>> It is hard for me to argue with "historical reasons". But -
>>>>>>>> the "nested"
>>>>>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>>>>>> runner-coder
>>>>>>>> >>>> spaces.
>>>>>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>>>>>> >> Hm, do you mean the Context passed to the encode/decode method?
>>>>>>>> [1] That
>>>>>>>> >> seems to be deprecated, I assume that most coders use the default
>>>>>>>> >> implementation and simply ignore the Context?
>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>> iterable)
>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>> > changing coders is hard (due to no way to update the encoding for
>>>>>>>> a
>>>>>>>> > streaming pipeline).
>>>>>>>> >
>>>>>>>> >> Even if not - whether or
>>>>>>>> >> not the elements are encoded using NESTED Context or UNNESTED
>>>>>>>> Context
>>>>>>>> >> should be part of the contract of TestStream, right? Most likely
>>>>>>>> it is
>>>>>>>> >> the UNNESTED one, if I understand correctly what that does.
>>>>>>>> Under what
>>>>>>>> >> conditions is the deprecated encode/decode method used?
>>>>>>>> > Yes, it's the UNNESTED one.
>>>>>>>> >
>>>>>>>> >> [1]
>>>>>>>> >>
>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>>>>>> >>
>>>>>>>> >>>> The runner's responsibility is not to go from "<nested
>>>>>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary
>>>>>>>> coder.
>>>>>>>> >>>> That is really impossible. But a coder is a function, right?
>>>>>>>> Function
>>>>>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>>>>>> provides a
>>>>>>>> >>>> set of elements, and these elements are the "universe". For
>>>>>>>> those
>>>>>>>> >>>> elements it also provides the encoded form, which can be
>>>>>>>> interpreted as
>>>>>>>> >>>> the definition of the coder.
>>>>>>>> >>> The problem here is that there is not "the encoded form" for a
>>>>>>>> Coder
>>>>>>>> >>> but two encoded forms, and we have the wrong one. Things could
>>>>>>>> be made
>>>>>>>> >>> to work if we had the other.
>>>>>>>> >> Which two encoded forms do you refer to? Elements encoded by
>>>>>>>> both the
>>>>>>>> >> SDK-coder and runner-coder (and I ignore the Context here once
>>>>>>>> again)
>>>>>>>> >> have the same binary representation (which they must have,
>>>>>>>> otherwise it
>>>>>>>> >> would be impossible to decode elements coming from the runner to
>>>>>>>> the
>>>>>>>> >> SDK-harness or vice-versa).
>>>>>>>> >>>> Therefore - technically (and formally) -
>>>>>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>>>>>> regardless of
>>>>>>>> >>>> the language the runner is written in.
>>>>>>>> >>>>
>>>>>>>> >>>> To move  this discussion forward, I think we should look for
>>>>>>>> answers to
>>>>>>>> >>>> the following questions:
>>>>>>>> >>>>
>>>>>>>> >>>>     a) do we have any clues that show, that the proposed "in
>>>>>>>> runner"
>>>>>>>> >>>> solution will not work?
>>>>>>>> >>> OK, thinking about it some more, in the TestStream, we can use
>>>>>>>> the
>>>>>>>> >>> happy coincidence that
>>>>>>>> >>>
>>>>>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>>>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>>>>>> >>> nested=False)
>>>>>>>> >>>
>>>>>>>> >>> (where || denotes concatenation) and the fact that we have
>>>>>>>> >>>
>>>>>>>> >>>       C.encode(x, nested=False)
>>>>>>>> >>>
>>>>>>>> >>> in hand.
>>>>>>>> >>>
>>>>>>>> >>> A possible fix here for the OP's question is that when
>>>>>>>> rehydrating the
>>>>>>>> >>> TestStream transform it must behave differently according to
>>>>>>>> the coder
>>>>>>>> >>> used in the subsequent channel (e.g. for known coders, it
>>>>>>>> decodes the
>>>>>>>> >>> elements and emits them directly, but for unknown coders, it
>>>>>>>> prefixes
>>>>>>>> >>> them with their length and emits byte strings. It gets more
>>>>>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>>>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>>>>>> unknown-coder))
>>>>>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different
>>>>>>>> encodings
>>>>>>>> >>> (and the latter, which is the default, requires transcoding the
>>>>>>>> bytes
>>>>>>>> >>> to inject the length in the middle which is found by decoding
>>>>>>>> the
>>>>>>>> >>> first component). As well as getting more complex, this really
>>>>>>>> seems
>>>>>>>> >>> to violate the spirit of separation of concerns.
>>>>>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>>>>>> >> KV<LP<unknown>, known>? From my understanding it is always the
>>>>>>>> latter,
>>>>>>>> >> because of [2].
>>>>>>>> >>
>>>>>>>> >> [2]
>>>>>>>> >>
>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>>> > It is currently the latter for runners using this code (which not
>>>>>>>> all
>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>> >
>>>>>>>> >>>>     b) do we think, that it will not be robust enough to
>>>>>>>> incorporate the
>>>>>>>> >>>> other use-cases (line generic transform inlining, taking into
>>>>>>>> account
>>>>>>>> >>>> that this applies only to runners that are written in the same
>>>>>>>> language
>>>>>>>> >>>> as the submitting SDK, because otherwise, there is nothing to
>>>>>>>> inline)?
>>>>>>>> >>> Being in the same language is not a prerequisite to "inlining,"
>>>>>>>> e.g.
>>>>>>>> >>> the PubSub source on Dataflow is recognized as such and not
>>>>>>>> executed
>>>>>>>> >>> as SDK code but natively.
>>>>>>>> >> Agree, that is actually exactly what happens with the
>>>>>>>> TestStream. The
>>>>>>>> >> transform need not be in the same language, as long as it is
>>>>>>>> completely
>>>>>>>> >> understood by the runner, including the SDK-coder (either
>>>>>>>> explicitly -
>>>>>>>> >> which might be due to the PCollection coder being composed of
>>>>>>>> well-known
>>>>>>>> >> coders only, or implicitly like in the case of TestStream, where
>>>>>>>> the
>>>>>>>> >> elements are encoded using the SDK coder.
>>>>>>>> >>> It is more likely that inlining occurs in the same language if
>>>>>>>> there
>>>>>>>> >>> are UDFs involved.
>>>>>>>> >>>
>>>>>>>> >>>> I'm convinced, that the TestStream-decode expansion solution
>>>>>>>> is an
>>>>>>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>>>>>>> bothering
>>>>>>>> >>>> this mailing list with my emails on this. :-)
>>>>>>>> >>>>
>>>>>>>> >>>> WDYT?
>>>>>>>> >>> While not a solution to the general problem, I think the
>>>>>>>> >>> TestStream-only-does-bytes simplifies its definition (primitives
>>>>>>>> >>> should have as simple/easy to implement definitions as
>>>>>>>> possible) and
>>>>>>>> >>> brings it closer to the other root we have: Impulse. (We could
>>>>>>>> go a
>>>>>>>> >>> step further and rather than emitting encoded elements, with
>>>>>>>> the data
>>>>>>>> >>> in the proto itself, it emits sequence numbers, and a
>>>>>>>> subsequent ParDo
>>>>>>>> >>> maps those to concrete elements (e.g. via an in-memory map),
>>>>>>>> but that
>>>>>>>> >>> further step doesn't buy much...)
>>>>>>>> >>>
>>>>>>>> >>> Only runners that want to do inlining would have to take on the
>>>>>>>> >>> complexity of a fully generic solution.
>>>>>>>> >> I think that if the simplification brings something, we can do
>>>>>>>> that, but
>>>>>>>> >> I'd like to understand why we cannot (or should not) use the
>>>>>>>> generic
>>>>>>>> >> solution. I think it definitely *should* be possible to use a
>>>>>>>> generic
>>>>>>>> >> solution, because otherwise the solution would not be generic.
>>>>>>>> And it
>>>>>>>> >> would imply, that we are unable to do generic transform
>>>>>>>> inlining, which
>>>>>>>> >> I would find really strange. That would immediately mean, that
>>>>>>>> we are
>>>>>>>> >> unable to construct classical runner as a special case of the
>>>>>>>> portable
>>>>>>>> >> one, which would be bad I think.
>>>>>>>> >>
>>>>>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>>>>>> SDK-coder,
>>>>>>>> >> or does this go through the LengthPrefixUnknownCoders logic? If
>>>>>>>> not,
>>>>>>>> >> then the problem would be there, because that means, that the
>>>>>>>> SDK-coder
>>>>>>>> >> cannot be (implicitly) defined in the runner. If the elements
>>>>>>>> would be
>>>>>>>> >> encoded using LP, then it would be possible to decode them using
>>>>>>>> >> runner-coder and the problem should be solved, or am I still
>>>>>>>> missing
>>>>>>>> >> some key parts?
>>>>>>>> > Yes, the problem is precisely that there are (unspecified)
>>>>>>>> constraints
>>>>>>>> > on the coder used by the TestStreamPayload. Just requiring that
>>>>>>>> it be
>>>>>>>> > length prefixed is not enough, you have to make constraints on
>>>>>>>> > sometimes pushing down the length prefixing if it's a composite
>>>>>>>> (like
>>>>>>>> > a KV) that depend on what the runner is expected to support in
>>>>>>>> terms
>>>>>>>> > of composites and/or the choices it chooses for the channel (and
>>>>>>>> the
>>>>>>>> > runner, not knowing the coder, can't transcode between these
>>>>>>>> choices).
>>>>>>>> >
>>>>>>>> > The simpler solution is to constrain this coder to just be byte[]
>>>>>>>> > rather than let it be a little bit flexible (but not wholly
>>>>>>>> flexible).
>>>>>>>> >
>>>>>>>> > As for a fully generic solution, I think the issue encountered
>>>>>>>> with
>>>>>>>> > inlining Read vs. TestStream are related to this, but not really
>>>>>>>> the
>>>>>>>> > same. With TestStream one has an encoded representation of the
>>>>>>>> > elements provided by the SDK that the Runner and has no SDK
>>>>>>>> > representation/execution whereas with the Reads one has unencoded
>>>>>>>> > elements in hand and a Coder that is understood by both (so long
>>>>>>>> as
>>>>>>>> > the channel can be negotiated correctly). FWIW, I think the proper
>>>>>>>> > solution to inlining a Read (or other Transform that would
>>>>>>>> typically
>>>>>>>> > be executed in the SDK) is to treat it as a special environment
>>>>>>>> (where
>>>>>>>> > we know logically it can work) and then elide, as possible, the
>>>>>>>> > various encodings, grpc calls, etc. that are unneeded as
>>>>>>>> everything is
>>>>>>>> > in process.
>>>>>>>> >
>>>>>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>> >>>>>>>> Hi,
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>> I had some more time thinking about this and I'll try to
>>>>>>>> recap that.
>>>>>>>> >>>>>>>> First some invariants:
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       a) each PCollection<T> has actually two coders - an
>>>>>>>> _SDK coder_ and a
>>>>>>>> >>>>>>>> _runner coder_. These coders have the property, that each
>>>>>>>> one can
>>>>>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not
>>>>>>>> true, the
>>>>>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in
>>>>>>>> general).
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>>>>>> environment, the
>>>>>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>>>>>> SDK-harness and
>>>>>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       c) under specific circumstances, the encode-decode
>>>>>>>> step can be
>>>>>>>> >>>>>>>> optimized out, that is the case where the SDK coder and
>>>>>>>> all its
>>>>>>>> >>>>>>>> subcoders are all well-known to the runner (in the
>>>>>>>> present, that means
>>>>>>>> >>>>>>>> that all the parts present in the model coders set). The
>>>>>>>> reason for that
>>>>>>>> >>>>>>>> is that in this specific situation
>>>>>>>> runner_decode(sdk_encode(X)) = X.
>>>>>>>> >>>>>>>> This property is essential.
>>>>>>>> >>>>>>> However, in general, X can only pass from the SDK to the
>>>>>>>> runner (or
>>>>>>>> >>>>>>> vice versa) in encoded form.
>>>>>>>> >>>>>> In general yes, but we are (mostly) talking transform
>>>>>>>> inlining here, so
>>>>>>>> >>>>>> it that particular situation, the elements might be passed
>>>>>>>> in decoded form.
>>>>>>>> >>>>>>>>       d) from b) immediately follows, that when a
>>>>>>>> PTransform does not run in
>>>>>>>> >>>>>>>> an environment (and this might be due to the transform
>>>>>>>> being runner
>>>>>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>>>>>>>> elements have
>>>>>>>> >>>>>>>> to be encoded by SDK coder, immediately following decode
>>>>>>>> by runner
>>>>>>>> >>>>>>>> coder. That (surprisingly) applies even to situations when
>>>>>>>> runner is
>>>>>>>> >>>>>>>> implemented using different language than the client SDK,
>>>>>>>> because it
>>>>>>>> >>>>>>>> implies that the type of produced elements must be one of
>>>>>>>> types encoded
>>>>>>>> >>>>>>>> using model coders (well-known to the runner, otherwise
>>>>>>>> the SDK will not
>>>>>>>> >>>>>>>> be able to consume it). But - due to property c) - this
>>>>>>>> means that this
>>>>>>>> >>>>>>>> encode-decode step can be optimized out. This does not
>>>>>>>> mean that it is
>>>>>>>> >>>>>>>> not (logically) present, though. This is exactly the case
>>>>>>>> of native
>>>>>>>> >>>>>>>> Impulse transform.
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>> Now, from that we can conclude that on the boundary
>>>>>>>> between executable
>>>>>>>> >>>>>>>> stages, or between runner (inlined) transform and
>>>>>>>> executable stage, each
>>>>>>>> >>>>>>>> PCollection has to be encoded using SDK coder and
>>>>>>>> immediately decoded by
>>>>>>>> >>>>>>>> runner coder, *unless this can be optimized out* by
>>>>>>>> property c).
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>> This gives us two options where to implement this
>>>>>>>> encode/decode step:
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       1) completely inside runner with the possibility to
>>>>>>>> optimize the
>>>>>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       2) partly in the runner and partly in the SDK - that
>>>>>>>> is we encode
>>>>>>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass
>>>>>>>> those to the
>>>>>>>> >>>>>>>> SDK harness and apply a custom decode step there. This
>>>>>>>> works because SDK
>>>>>>>> >>>>>>>> coder encoded elements are in byte[], and that is
>>>>>>>> well-known coder type.
>>>>>>>> >>>>>>>> We again only leverage property c) and optimize the SDK
>>>>>>>> coder encode,
>>>>>>>> >>>>>>>> runner decode step out.
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream
>>>>>>>> producing byte[] and
>>>>>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>>>>>>>> inlined
>>>>>>>> >>>>>>>> transform, the elements are produced directly in runner
>>>>>>>> (the SDK coder
>>>>>>>> >>>>>>>> is not known to the runner, but that does not matter,
>>>>>>>> because the
>>>>>>>> >>>>>>>> elements are already encoded by client).
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>      From the above it seems to me, that option 1) should
>>>>>>>> be preferred, because:
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       i) it is generic, applicable to all inlined
>>>>>>>> transforms, any sources
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       ii) it is consistent with how things logically work
>>>>>>>> underneath
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       iii) it offers better room for optimization - option
>>>>>>>> 2) might result
>>>>>>>> >>>>>>>> in cases when the elements are passed from the runner to
>>>>>>>> the SDK-harness
>>>>>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>>>>>> immediately
>>>>>>>> >>>>>>>> encoding back using SDK-coder and returned back to the
>>>>>>>> runner. This
>>>>>>>> >>>>>>>> would be the case when TestStream would be directly
>>>>>>>> consumed by inlined
>>>>>>>> >>>>>>>> (or external) transform.
>>>>>>>> >>>>>>> (1) is not possible if the Coder in question is not known
>>>>>>>> to the
>>>>>>>> >>>>>>> Runner, which is why I proposed (2).
>>>>>>>> >>>>>> There is no particular need for the coder to be known. If
>>>>>>>> transform is
>>>>>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded
>>>>>>>> form of the
>>>>>>>> >>>>>> elements. That holds true if:
>>>>>>>> >>>>>>
>>>>>>>> >>>>>>      a) either the SDK coder is known, or
>>>>>>>> >>>>>>
>>>>>>>> >>>>>>      b) encoded form of the produced elements is known in
>>>>>>>> advance
>>>>>>>> >>>>>>
>>>>>>>> >>>>>> For TestStream it is the case b). For inlined primitive Read
>>>>>>>> (or any
>>>>>>>> >>>>>> other transform which executes code) it is a).
>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>> reasons,
>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>> delimiting)
>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>> encoded bytes,
>>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>>> SDK. The
>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>>>> for an
>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>> >>>>>
>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>>>> already
>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>> complicate the
>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>> sending the
>>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>>> solved
>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>> known by the
>>>>>>>> >>>>> runner).)
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>> --
>>>>>>
>>>>>>
>>>>>> Jack McCluskey
>>>>>> SWE - DataPLS PLAT/ Beam Go
>>>>>> RDU
>>>>>> jrmccluskey@gmail.com
>>>>>>
>>>>>>
>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
That is correct. Technically one only needs to take the intersection of
coders for environments that could possibly be consumers, but in my Python
fix I simply take the intersection of coders over all environments.

On Wed, Sep 22, 2021 at 7:57 AM Jan Lukavský <je...@seznam.cz> wrote:

> Hi,
>
> picking this once again, as it seems to me, that the problem is even more
> complex than simply consulting _one_ (producer) SDK for capabilities.
> Because of cross-language, it is possible, that the SDK that produces the
> data can have a different set of standard coders, than the SDK that will
> consume the data. Moreover, it is even possible, that there will be
> multiple consumers of the same PCollection written in different SDKs, which
> would mean, that we need to take the intersection of standard coders of
> producer and all consumers to be able to deduce the final wire coder. Is
> this correct? Do we have a tracking Jira for fixing the Java and Python
> runner-core?
>
>  Jan
> On 9/9/21 7:29 PM, Robert Bradshaw wrote:
>
> You are right, Java does not implement this correctly. It should be
> querying the capabilities section of the environment proto. (For java
> environments, this is populated from ModelCoders, e.g.
> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386
> )
>
> Looks like Python doesn't do any better:
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459
>
>
>
> On Thu, Sep 9, 2021 at 10:13 AM Jan Lukavský <je...@seznam.cz> wrote:
>
>> This makes a *lot* of sense. But it seems to me, that this is not the way
>> Java-based runners - that use runners-core-construction-java module -
>> handle it. If I interpret it correctly, then the set of ModelCoders is
>> hard-coded [1] and essentially required to be known by all SDKs [2].
>>
>> There seems to be no negotiation between what SDK harness knows and what
>> the runner knows. The runner might be able to define the wire coder for the
>> SDK (via the ProcessBundleDescriptor), but the SDK (for Java runners) seems
>> not to be able to play any role in this [3]. Therefore I think that if an
>> SDK does not know the set of Java ModelCoders, then the runner and the SDK
>> might not agree on the binary encoding of the elements (BTW, which is why
>> Java Coders cannot be part of ModelCoders and I finally understand why I
>> had such troubles adding it there - it cannot be there).
>>
>> Is it possible we are missing some part of this runner-to-sdk coder
>> negotiation in runners-core-contruction-java?
>>
>> [1]
>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75
>>
>> [2]
>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62
>>
>> [3]
>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104
>> On 9/9/21 12:18 AM, Robert Bradshaw wrote:
>>
>> The whole notion of an absolute set of known coders is a misnomer. It
>> would require all Runners and SDKs to be updated synchronously for every
>> new coder they might want to share.
>>
>> Instead, what we have are
>>
>> * Standard Coders which have well-defined, language-agnostic
>> representations and encodings, which may be used for interoperability and
>> efficiency, and
>> * Required Coders which are the minimum needed to execute the pipeline.
>>
>> The latter consists only of bytes (for impulse), kv and iterable (for
>> GBK), windowed value (for windowing information) and length prefix (to be
>> able to handle anything else).
>>
>>
>> On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <ro...@frantil.com> wrote:
>>
>>> Is the claim that the Standard bytes and String_utf8 coders are not
>>> "known coders"?
>>>
>>>  What's the point of the standard coders if they are not the canonical
>>> "known coders" that can generally be expected to be known by runners/other
>>> SDKs?
>>>
>>>
>>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>>>
>>> The Go SDK rather heavily biases towards using the standard coders for
>>> their closes language equivalents rather than going into override/custom
>>> specified soup. (It's not possible to globally override the coders for the
>>> '[]byte' and 'string' types, nor is there often reason to.)
>>>
>>> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <jr...@google.com>
>>>> wrote:
>>>>
>>>>> Hey all,
>>>>>
>>>>> Just catching up on the thread since I did the TestStream Go SDK
>>>>> implementation. The discussion about length prefixing behavior for known
>>>>> vs. unknown coders is interesting, since we ran into strings and byte
>>>>> slices getting extra length prefixes attached to them by Flink despite
>>>>> being known coders.
>>>>>
>>>>
>>>> Known to who?
>>>>
>>>>
>>>>> Based on what's been said, that isn't expected behavior, right?
>>>>>
>>>>
>>>> No, it's not.
>>>>
>>>> I would check to make sure the Go SDK is respecting the Coder (length
>>>> prefixed or not) that's set on the channel, rather than guessing at what it
>>>> expects it to be based on the Go type.
>>>>
>>>>
>>>>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>
>>>>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>>>>
>>>>>> Historically the DataflowRunner has been much more careful about not
>>>>>> breaking update, since this is a frequent operation by Dataflow users. I
>>>>>> think we've been less careful aboutt other runners, but as we see clearly
>>>>>> here Fllnk users do care about this as well, so we should probably test
>>>>>> upgrade compatibility for Flink.
>>>>>>
>>>>>> One strategy that Dataflow uses is to avoid embedding the Java
>>>>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>>>>> breakage (as we see with the issue you llnked to). Possibly similar
>>>>>> strategies should be investigated for Fllink.
>>>>>>
>>>>>> +1, that would be great!
>>>>>>
>>>>>>
>>>>>> Reuven
>>>>>>
>>>>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>
>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>> iterable)
>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>> > changing coders is hard (due to no way to update the encoding for a
>>>>>>> > streaming pipeline).
>>>>>>> This is unrelated, but - regarding changing coders due to concerns
>>>>>>> about
>>>>>>> pipeline upgrades, we break this quite often, at least for some
>>>>>>> runners.
>>>>>>> Most recently [1].
>>>>>>>
>>>>>>> > It is currently the latter for runners using this code (which not
>>>>>>> all
>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>> This is interesting and unexpected for me. How do runners decide
>>>>>>> about
>>>>>>> how they encode elements between SDK harness and the runner? How do
>>>>>>> they
>>>>>>> inform the SDK harness about this decision? My impression was that
>>>>>>> this
>>>>>>> is well-defined at the model level. If not, then we have the reason
>>>>>>> for
>>>>>>> misunderstanding in this conversation. :-)
>>>>>>>
>>>>>>>   Jan
>>>>>>>
>>>>>>> [1]
>>>>>>>
>>>>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>>>>
>>>>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz>
>>>>>>> wrote:
>>>>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>>>>> >>
>>>>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>>>>> wrote:
>>>>>>> >>>> Hi Robert,
>>>>>>> >>>>
>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>> reasons,
>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>> delimiting)
>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>> encoded bytes,
>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>> SDK. The
>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>>> for an
>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>> >>>>>
>>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>>> already
>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>> complicate the
>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>> sending the
>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>> solved
>>>>>>> >>>>> differently for inlined reads as the coder can always be known
>>>>>>> by the
>>>>>>> >>>>> runner).)
>>>>>>> >>>> It is hard for me to argue with "historical reasons". But - the
>>>>>>> "nested"
>>>>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>>>>> runner-coder
>>>>>>> >>>> spaces.
>>>>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>>>>> >> Hm, do you mean the Context passed to the encode/decode method?
>>>>>>> [1] That
>>>>>>> >> seems to be deprecated, I assume that most coders use the default
>>>>>>> >> implementation and simply ignore the Context?
>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>> iterable)
>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>> > changing coders is hard (due to no way to update the encoding for a
>>>>>>> > streaming pipeline).
>>>>>>> >
>>>>>>> >> Even if not - whether or
>>>>>>> >> not the elements are encoded using NESTED Context or UNNESTED
>>>>>>> Context
>>>>>>> >> should be part of the contract of TestStream, right? Most likely
>>>>>>> it is
>>>>>>> >> the UNNESTED one, if I understand correctly what that does. Under
>>>>>>> what
>>>>>>> >> conditions is the deprecated encode/decode method used?
>>>>>>> > Yes, it's the UNNESTED one.
>>>>>>> >
>>>>>>> >> [1]
>>>>>>> >>
>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>>>>> >>
>>>>>>> >>>> The runner's responsibility is not to go from "<nested
>>>>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary
>>>>>>> coder.
>>>>>>> >>>> That is really impossible. But a coder is a function, right?
>>>>>>> Function
>>>>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>>>>> provides a
>>>>>>> >>>> set of elements, and these elements are the "universe". For
>>>>>>> those
>>>>>>> >>>> elements it also provides the encoded form, which can be
>>>>>>> interpreted as
>>>>>>> >>>> the definition of the coder.
>>>>>>> >>> The problem here is that there is not "the encoded form" for a
>>>>>>> Coder
>>>>>>> >>> but two encoded forms, and we have the wrong one. Things could
>>>>>>> be made
>>>>>>> >>> to work if we had the other.
>>>>>>> >> Which two encoded forms do you refer to? Elements encoded by both
>>>>>>> the
>>>>>>> >> SDK-coder and runner-coder (and I ignore the Context here once
>>>>>>> again)
>>>>>>> >> have the same binary representation (which they must have,
>>>>>>> otherwise it
>>>>>>> >> would be impossible to decode elements coming from the runner to
>>>>>>> the
>>>>>>> >> SDK-harness or vice-versa).
>>>>>>> >>>> Therefore - technically (and formally) -
>>>>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>>>>> regardless of
>>>>>>> >>>> the language the runner is written in.
>>>>>>> >>>>
>>>>>>> >>>> To move  this discussion forward, I think we should look for
>>>>>>> answers to
>>>>>>> >>>> the following questions:
>>>>>>> >>>>
>>>>>>> >>>>     a) do we have any clues that show, that the proposed "in
>>>>>>> runner"
>>>>>>> >>>> solution will not work?
>>>>>>> >>> OK, thinking about it some more, in the TestStream, we can use
>>>>>>> the
>>>>>>> >>> happy coincidence that
>>>>>>> >>>
>>>>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>>>>> >>> nested=False)
>>>>>>> >>>
>>>>>>> >>> (where || denotes concatenation) and the fact that we have
>>>>>>> >>>
>>>>>>> >>>       C.encode(x, nested=False)
>>>>>>> >>>
>>>>>>> >>> in hand.
>>>>>>> >>>
>>>>>>> >>> A possible fix here for the OP's question is that when
>>>>>>> rehydrating the
>>>>>>> >>> TestStream transform it must behave differently according to the
>>>>>>> coder
>>>>>>> >>> used in the subsequent channel (e.g. for known coders, it
>>>>>>> decodes the
>>>>>>> >>> elements and emits them directly, but for unknown coders, it
>>>>>>> prefixes
>>>>>>> >>> them with their length and emits byte strings. It gets more
>>>>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>>>>> unknown-coder))
>>>>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different
>>>>>>> encodings
>>>>>>> >>> (and the latter, which is the default, requires transcoding the
>>>>>>> bytes
>>>>>>> >>> to inject the length in the middle which is found by decoding the
>>>>>>> >>> first component). As well as getting more complex, this really
>>>>>>> seems
>>>>>>> >>> to violate the spirit of separation of concerns.
>>>>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>>>>> >> KV<LP<unknown>, known>? From my understanding it is always the
>>>>>>> latter,
>>>>>>> >> because of [2].
>>>>>>> >>
>>>>>>> >> [2]
>>>>>>> >>
>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>> > It is currently the latter for runners using this code (which not
>>>>>>> all
>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>> >
>>>>>>> >>>>     b) do we think, that it will not be robust enough to
>>>>>>> incorporate the
>>>>>>> >>>> other use-cases (line generic transform inlining, taking into
>>>>>>> account
>>>>>>> >>>> that this applies only to runners that are written in the same
>>>>>>> language
>>>>>>> >>>> as the submitting SDK, because otherwise, there is nothing to
>>>>>>> inline)?
>>>>>>> >>> Being in the same language is not a prerequisite to "inlining,"
>>>>>>> e.g.
>>>>>>> >>> the PubSub source on Dataflow is recognized as such and not
>>>>>>> executed
>>>>>>> >>> as SDK code but natively.
>>>>>>> >> Agree, that is actually exactly what happens with the TestStream.
>>>>>>> The
>>>>>>> >> transform need not be in the same language, as long as it is
>>>>>>> completely
>>>>>>> >> understood by the runner, including the SDK-coder (either
>>>>>>> explicitly -
>>>>>>> >> which might be due to the PCollection coder being composed of
>>>>>>> well-known
>>>>>>> >> coders only, or implicitly like in the case of TestStream, where
>>>>>>> the
>>>>>>> >> elements are encoded using the SDK coder.
>>>>>>> >>> It is more likely that inlining occurs in the same language if
>>>>>>> there
>>>>>>> >>> are UDFs involved.
>>>>>>> >>>
>>>>>>> >>>> I'm convinced, that the TestStream-decode expansion solution is
>>>>>>> an
>>>>>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>>>>>> bothering
>>>>>>> >>>> this mailing list with my emails on this. :-)
>>>>>>> >>>>
>>>>>>> >>>> WDYT?
>>>>>>> >>> While not a solution to the general problem, I think the
>>>>>>> >>> TestStream-only-does-bytes simplifies its definition (primitives
>>>>>>> >>> should have as simple/easy to implement definitions as possible)
>>>>>>> and
>>>>>>> >>> brings it closer to the other root we have: Impulse. (We could
>>>>>>> go a
>>>>>>> >>> step further and rather than emitting encoded elements, with the
>>>>>>> data
>>>>>>> >>> in the proto itself, it emits sequence numbers, and a subsequent
>>>>>>> ParDo
>>>>>>> >>> maps those to concrete elements (e.g. via an in-memory map), but
>>>>>>> that
>>>>>>> >>> further step doesn't buy much...)
>>>>>>> >>>
>>>>>>> >>> Only runners that want to do inlining would have to take on the
>>>>>>> >>> complexity of a fully generic solution.
>>>>>>> >> I think that if the simplification brings something, we can do
>>>>>>> that, but
>>>>>>> >> I'd like to understand why we cannot (or should not) use the
>>>>>>> generic
>>>>>>> >> solution. I think it definitely *should* be possible to use a
>>>>>>> generic
>>>>>>> >> solution, because otherwise the solution would not be generic.
>>>>>>> And it
>>>>>>> >> would imply, that we are unable to do generic transform inlining,
>>>>>>> which
>>>>>>> >> I would find really strange. That would immediately mean, that we
>>>>>>> are
>>>>>>> >> unable to construct classical runner as a special case of the
>>>>>>> portable
>>>>>>> >> one, which would be bad I think.
>>>>>>> >>
>>>>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>>>>> SDK-coder,
>>>>>>> >> or does this go through the LengthPrefixUnknownCoders logic? If
>>>>>>> not,
>>>>>>> >> then the problem would be there, because that means, that the
>>>>>>> SDK-coder
>>>>>>> >> cannot be (implicitly) defined in the runner. If the elements
>>>>>>> would be
>>>>>>> >> encoded using LP, then it would be possible to decode them using
>>>>>>> >> runner-coder and the problem should be solved, or am I still
>>>>>>> missing
>>>>>>> >> some key parts?
>>>>>>> > Yes, the problem is precisely that there are (unspecified)
>>>>>>> constraints
>>>>>>> > on the coder used by the TestStreamPayload. Just requiring that it
>>>>>>> be
>>>>>>> > length prefixed is not enough, you have to make constraints on
>>>>>>> > sometimes pushing down the length prefixing if it's a composite
>>>>>>> (like
>>>>>>> > a KV) that depend on what the runner is expected to support in
>>>>>>> terms
>>>>>>> > of composites and/or the choices it chooses for the channel (and
>>>>>>> the
>>>>>>> > runner, not knowing the coder, can't transcode between these
>>>>>>> choices).
>>>>>>> >
>>>>>>> > The simpler solution is to constrain this coder to just be byte[]
>>>>>>> > rather than let it be a little bit flexible (but not wholly
>>>>>>> flexible).
>>>>>>> >
>>>>>>> > As for a fully generic solution, I think the issue encountered with
>>>>>>> > inlining Read vs. TestStream are related to this, but not really
>>>>>>> the
>>>>>>> > same. With TestStream one has an encoded representation of the
>>>>>>> > elements provided by the SDK that the Runner and has no SDK
>>>>>>> > representation/execution whereas with the Reads one has unencoded
>>>>>>> > elements in hand and a Coder that is understood by both (so long as
>>>>>>> > the channel can be negotiated correctly). FWIW, I think the proper
>>>>>>> > solution to inlining a Read (or other Transform that would
>>>>>>> typically
>>>>>>> > be executed in the SDK) is to treat it as a special environment
>>>>>>> (where
>>>>>>> > we know logically it can work) and then elide, as possible, the
>>>>>>> > various encodings, grpc calls, etc. that are unneeded as
>>>>>>> everything is
>>>>>>> > in process.
>>>>>>> >
>>>>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>>>>>> wrote:
>>>>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
>>>>>>> wrote:
>>>>>>> >>>>>>>> Hi,
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>> I had some more time thinking about this and I'll try to
>>>>>>> recap that.
>>>>>>> >>>>>>>> First some invariants:
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       a) each PCollection<T> has actually two coders - an
>>>>>>> _SDK coder_ and a
>>>>>>> >>>>>>>> _runner coder_. These coders have the property, that each
>>>>>>> one can
>>>>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not
>>>>>>> true, the
>>>>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in
>>>>>>> general).
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>>>>> environment, the
>>>>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>>>>> SDK-harness and
>>>>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       c) under specific circumstances, the encode-decode
>>>>>>> step can be
>>>>>>> >>>>>>>> optimized out, that is the case where the SDK coder and all
>>>>>>> its
>>>>>>> >>>>>>>> subcoders are all well-known to the runner (in the present,
>>>>>>> that means
>>>>>>> >>>>>>>> that all the parts present in the model coders set). The
>>>>>>> reason for that
>>>>>>> >>>>>>>> is that in this specific situation
>>>>>>> runner_decode(sdk_encode(X)) = X.
>>>>>>> >>>>>>>> This property is essential.
>>>>>>> >>>>>>> However, in general, X can only pass from the SDK to the
>>>>>>> runner (or
>>>>>>> >>>>>>> vice versa) in encoded form.
>>>>>>> >>>>>> In general yes, but we are (mostly) talking transform
>>>>>>> inlining here, so
>>>>>>> >>>>>> it that particular situation, the elements might be passed in
>>>>>>> decoded form.
>>>>>>> >>>>>>>>       d) from b) immediately follows, that when a
>>>>>>> PTransform does not run in
>>>>>>> >>>>>>>> an environment (and this might be due to the transform
>>>>>>> being runner
>>>>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>>>>>>> elements have
>>>>>>> >>>>>>>> to be encoded by SDK coder, immediately following decode by
>>>>>>> runner
>>>>>>> >>>>>>>> coder. That (surprisingly) applies even to situations when
>>>>>>> runner is
>>>>>>> >>>>>>>> implemented using different language than the client SDK,
>>>>>>> because it
>>>>>>> >>>>>>>> implies that the type of produced elements must be one of
>>>>>>> types encoded
>>>>>>> >>>>>>>> using model coders (well-known to the runner, otherwise the
>>>>>>> SDK will not
>>>>>>> >>>>>>>> be able to consume it). But - due to property c) - this
>>>>>>> means that this
>>>>>>> >>>>>>>> encode-decode step can be optimized out. This does not mean
>>>>>>> that it is
>>>>>>> >>>>>>>> not (logically) present, though. This is exactly the case
>>>>>>> of native
>>>>>>> >>>>>>>> Impulse transform.
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>> Now, from that we can conclude that on the boundary between
>>>>>>> executable
>>>>>>> >>>>>>>> stages, or between runner (inlined) transform and
>>>>>>> executable stage, each
>>>>>>> >>>>>>>> PCollection has to be encoded using SDK coder and
>>>>>>> immediately decoded by
>>>>>>> >>>>>>>> runner coder, *unless this can be optimized out* by
>>>>>>> property c).
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>> This gives us two options where to implement this
>>>>>>> encode/decode step:
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       1) completely inside runner with the possibility to
>>>>>>> optimize the
>>>>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       2) partly in the runner and partly in the SDK - that
>>>>>>> is we encode
>>>>>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass
>>>>>>> those to the
>>>>>>> >>>>>>>> SDK harness and apply a custom decode step there. This
>>>>>>> works because SDK
>>>>>>> >>>>>>>> coder encoded elements are in byte[], and that is
>>>>>>> well-known coder type.
>>>>>>> >>>>>>>> We again only leverage property c) and optimize the SDK
>>>>>>> coder encode,
>>>>>>> >>>>>>>> runner decode step out.
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream
>>>>>>> producing byte[] and
>>>>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>>>>>>> inlined
>>>>>>> >>>>>>>> transform, the elements are produced directly in runner
>>>>>>> (the SDK coder
>>>>>>> >>>>>>>> is not known to the runner, but that does not matter,
>>>>>>> because the
>>>>>>> >>>>>>>> elements are already encoded by client).
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>      From the above it seems to me, that option 1) should
>>>>>>> be preferred, because:
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       i) it is generic, applicable to all inlined
>>>>>>> transforms, any sources
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       ii) it is consistent with how things logically work
>>>>>>> underneath
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       iii) it offers better room for optimization - option
>>>>>>> 2) might result
>>>>>>> >>>>>>>> in cases when the elements are passed from the runner to
>>>>>>> the SDK-harness
>>>>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>>>>> immediately
>>>>>>> >>>>>>>> encoding back using SDK-coder and returned back to the
>>>>>>> runner. This
>>>>>>> >>>>>>>> would be the case when TestStream would be directly
>>>>>>> consumed by inlined
>>>>>>> >>>>>>>> (or external) transform.
>>>>>>> >>>>>>> (1) is not possible if the Coder in question is not known to
>>>>>>> the
>>>>>>> >>>>>>> Runner, which is why I proposed (2).
>>>>>>> >>>>>> There is no particular need for the coder to be known. If
>>>>>>> transform is
>>>>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded form
>>>>>>> of the
>>>>>>> >>>>>> elements. That holds true if:
>>>>>>> >>>>>>
>>>>>>> >>>>>>      a) either the SDK coder is known, or
>>>>>>> >>>>>>
>>>>>>> >>>>>>      b) encoded form of the produced elements is known in
>>>>>>> advance
>>>>>>> >>>>>>
>>>>>>> >>>>>> For TestStream it is the case b). For inlined primitive Read
>>>>>>> (or any
>>>>>>> >>>>>> other transform which executes code) it is a).
>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>> reasons,
>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>> delimiting)
>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>> encoded bytes,
>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>> SDK. The
>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>>> for an
>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>> >>>>>
>>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>>> already
>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>> complicate the
>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>> sending the
>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>> solved
>>>>>>> >>>>> differently for inlined reads as the coder can always be known
>>>>>>> by the
>>>>>>> >>>>> runner).)
>>>>>>>
>>>>>>
>>>>>
>>>>> --
>>>>>
>>>>>
>>>>> Jack McCluskey
>>>>> SWE - DataPLS PLAT/ Beam Go
>>>>> RDU
>>>>> jrmccluskey@gmail.com
>>>>>
>>>>>
>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
Hi,

picking this once again, as it seems to me, that the problem is even 
more complex than simply consulting _one_ (producer) SDK for 
capabilities. Because of cross-language, it is possible, that the SDK 
that produces the data can have a different set of standard coders, than 
the SDK that will consume the data. Moreover, it is even possible, that 
there will be multiple consumers of the same PCollection written in 
different SDKs, which would mean, that we need to take the intersection 
of standard coders of producer and all consumers to be able to deduce 
the final wire coder. Is this correct? Do we have a tracking Jira for 
fixing the Java and Python runner-core?

  Jan

On 9/9/21 7:29 PM, Robert Bradshaw wrote:
> You are right, Java does not implement this correctly. It should be 
> querying the capabilities section of the environment proto. (For java 
> environments, this is populated from ModelCoders, e.g. 
> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386 
> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386> 
> )
>
> Looks like Python doesn't do any better: 
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459 
> <https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459>
>
>
>
> On Thu, Sep 9, 2021 at 10:13 AM Jan Lukavský <je.ik@seznam.cz 
> <ma...@seznam.cz>> wrote:
>
>     This makes a *lot* of sense. But it seems to me, that this is not
>     the way Java-based runners - that use
>     runners-core-construction-java module - handle it. If I interpret
>     it correctly, then the set of ModelCoders is hard-coded [1] and
>     essentially required to be known by all SDKs [2].
>
>     There seems to be no negotiation between what SDK harness knows
>     and what the runner knows. The runner might be able to define the
>     wire coder for the SDK (via the ProcessBundleDescriptor), but the
>     SDK (for Java runners) seems not to be able to play any role in
>     this [3]. Therefore I think that if an SDK does not know the set
>     of Java ModelCoders, then the runner and the SDK might not agree
>     on the binary encoding of the elements (BTW, which is why Java
>     Coders cannot be part of ModelCoders and I finally understand why
>     I had such troubles adding it there - it cannot be there).
>
>     Is it possible we are missing some part of this runner-to-sdk
>     coder negotiation in runners-core-contruction-java?
>
>     [1]
>     https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75
>     <https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75>
>
>     [2]
>     https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62
>     <https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62>
>
>     [3]
>     https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104
>     <https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104>
>
>     On 9/9/21 12:18 AM, Robert Bradshaw wrote:
>>     The whole notion of an absolute set of known coders is a
>>     misnomer. It would require all Runners and SDKs to be updated
>>     synchronously for every new coder they might want to share.
>>
>>     Instead, what we have are
>>
>>     * Standard Coders which have well-defined, language-agnostic
>>     representations and encodings, which may be used for
>>     interoperability and efficiency, and
>>     * Required Coders which are the minimum needed to execute the
>>     pipeline.
>>
>>     The latter consists only of bytes (for impulse), kv and iterable
>>     (for GBK), windowed value (for windowing information) and
>>     length prefix (to be able to handle anything else).
>>
>>
>>     On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <robert@frantil.com
>>     <ma...@frantil.com>> wrote:
>>
>>         Is the claim that the Standard bytes and String_utf8 coders
>>         are not "known coders"?
>>
>>          What's the point of the standard coders if they are not the
>>         canonical "known coders" that can generally be expected to be
>>         known by runners/other SDKs?
>>
>>         https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>>         <https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790>
>>
>>         The Go SDK rather heavily biases towards using the standard
>>         coders for their closes language equivalents rather than
>>         going into override/custom specified soup. (It's not possible
>>         to globally override the coders for the '[]byte' and 'string'
>>         types, nor is there often reason to.)
>>
>>         On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw
>>         <robertwb@google.com <ma...@google.com>> wrote:
>>
>>             On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey
>>             <jrmccluskey@google.com <ma...@google.com>>
>>             wrote:
>>
>>                 Hey all,
>>
>>                 Just catching up on the thread since I did the
>>                 TestStream Go SDK implementation. The discussion
>>                 about length prefixing behavior for known vs. unknown
>>                 coders is interesting, since we ran into strings and
>>                 byte slices getting extra length prefixes attached to
>>                 them by Flink despite being known coders.
>>
>>
>>             Known to who?
>>
>>                 Based on what's been said, that isn't expected
>>                 behavior, right?
>>
>>
>>             No, it's not.
>>
>>             I would check to make sure the Go SDK is respecting the
>>             Coder (length prefixed or not) that's set on the channel,
>>             rather than guessing at what it expects it to be based on
>>             the Go type.
>>
>>                 On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský
>>                 <je.ik@seznam.cz <ma...@seznam.cz>> wrote:
>>
>>                     On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>                     Historically the DataflowRunner has been much
>>>                     more careful about not breaking update, since
>>>                     this is a frequent operation by Dataflow users.
>>>                     I think we've been less careful aboutt other
>>>                     runners, but as we see clearly here Fllnk users
>>>                     do care about this as well, so we should
>>>                     probably test upgrade compatibility for Flink.
>>>
>>>                     One strategy that Dataflow uses is to avoid
>>>                     embedding the Java serialized form of a Coder in
>>>                     the graph, as this is a much higher risk of
>>>                     breakage (as we see with the issue you llnked
>>>                     to). Possibly similar strategies should be
>>>                     investigated for Fllink.
>>                     +1, that would be great!
>>>
>>>                     Reuven
>>>
>>>                     On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský
>>>                     <je.ik@seznam.cz <ma...@seznam.cz>> wrote:
>>>
>>>                         > Unfortunately the most basic coders (e.g.
>>>                         bytes, string, kv, iterable)
>>>                         > care about Context because they predated
>>>                         this deprecation, and
>>>                         > changing coders is hard (due to no way to
>>>                         update the encoding for a
>>>                         > streaming pipeline).
>>>                         This is unrelated, but - regarding changing
>>>                         coders due to concerns about
>>>                         pipeline upgrades, we break this quite
>>>                         often, at least for some runners.
>>>                         Most recently [1].
>>>
>>>                         > It is currently the latter for runners
>>>                         using this code (which not all
>>>                         > do, e.g. the ULR and Dataflow runners). I
>>>                         don't think we want to
>>>                         > ossify this decision as part of the spec.
>>>                         (Note that even what's
>>>                         > "known" and "unknown" can change from
>>>                         runner to runner.)
>>>                         This is interesting and unexpected for me.
>>>                         How do runners decide about
>>>                         how they encode elements between SDK harness
>>>                         and the runner? How do they
>>>                         inform the SDK harness about this decision?
>>>                         My impression was that this
>>>                         is well-defined at the model level. If not,
>>>                         then we have the reason for
>>>                         misunderstanding in this conversation. :-)
>>>
>>>                           Jan
>>>
>>>                         [1]
>>>                         https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>                         <https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E>
>>>
>>>                         On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>                         > On Sat, Sep 4, 2021 at 6:52 AM Jan
>>>                         Lukavský <je.ik@seznam.cz
>>>                         <ma...@seznam.cz>> wrote:
>>>                         >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>                         >>
>>>                         >>> On Fri, Sep 3, 2021 at 11:42 AM Jan
>>>                         Lukavský<je.ik@seznam.cz
>>>                         <ma...@seznam.cz>> wrote:
>>>                         >>>> Hi Robert,
>>>                         >>>>
>>>                         >>>>> There's another hitch here for
>>>                         TestStream. For historical reasons,
>>>                         >>>>> coders actually represent two
>>>                         encodings: nested (aka self delimiting)
>>>                         >>>>> and unnested. TestStream elements are
>>>                         given as unnested encoded bytes,
>>>                         >>>>> but the nested encoding is required
>>>                         for sending data to the SDK. The
>>>                         >>>>> runner can't go from <nested encoding>
>>>                         to <unnested encoding> for an
>>>                         >>>>> arbitrary unknown coder.
>>>                         >>>>>
>>>                         >>>>> (Even if it weren't for this
>>>                         complication, to be able to send already
>>>                         >>>>> encoded bytes of an unknown coder to
>>>                         the SDK will also complicate the
>>>                         >>>>> logic in choosing the coder to be used
>>>                         for the channel and sending the
>>>                         >>>>> data, which is some of what you're
>>>                         running into (but can be solved
>>>                         >>>>> differently for inlined reads as the
>>>                         coder can always be known by the
>>>                         >>>>> runner).)
>>>                         >>>> It is hard for me to argue with
>>>                         "historical reasons". But - the "nested"
>>>                         >>>> and "unnested" coders look very similar
>>>                         to SDK-coder and runner-coder
>>>                         >>>> spaces.
>>>                         >>> Unfortunately, they're actually
>>>                         orthogonal to that.
>>>                         >> Hm, do you mean the Context passed to the
>>>                         encode/decode method? [1] That
>>>                         >> seems to be deprecated, I assume that
>>>                         most coders use the default
>>>                         >> implementation and simply ignore the Context?
>>>                         > Unfortunately the most basic coders (e.g.
>>>                         bytes, string, kv, iterable)
>>>                         > care about Context because they predated
>>>                         this deprecation, and
>>>                         > changing coders is hard (due to no way to
>>>                         update the encoding for a
>>>                         > streaming pipeline).
>>>                         >
>>>                         >> Even if not - whether or
>>>                         >> not the elements are encoded using NESTED
>>>                         Context or UNNESTED Context
>>>                         >> should be part of the contract of
>>>                         TestStream, right? Most likely it is
>>>                         >> the UNNESTED one, if I understand
>>>                         correctly what that does. Under what
>>>                         >> conditions is the deprecated
>>>                         encode/decode method used?
>>>                         > Yes, it's the UNNESTED one.
>>>                         >
>>>                         >> [1]
>>>                         >>
>>>                         https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>                         <https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134>
>>>                         >>
>>>                         >>>> The runner's responsibility is not to
>>>                         go from "<nested
>>>                         >>>> encoding>" (SDK coder) to "<unnested
>>>                         encoding>" for arbitrary coder.
>>>                         >>>> That is really impossible. But a coder
>>>                         is a function, right? Function
>>>                         >>>> maps from universe A to universe B (in
>>>                         general). TestStream provides a
>>>                         >>>> set of elements, and these elements are
>>>                         the "universe". For those
>>>                         >>>> elements it also provides the encoded
>>>                         form, which can be interpreted as
>>>                         >>>> the definition of the coder.
>>>                         >>> The problem here is that there is not
>>>                         "the encoded form" for a Coder
>>>                         >>> but two encoded forms, and we have the
>>>                         wrong one. Things could be made
>>>                         >>> to work if we had the other.
>>>                         >> Which two encoded forms do you refer to?
>>>                         Elements encoded by both the
>>>                         >> SDK-coder and runner-coder (and I ignore
>>>                         the Context here once again)
>>>                         >> have the same binary representation
>>>                         (which they must have, otherwise it
>>>                         >> would be impossible to decode elements
>>>                         coming from the runner to the
>>>                         >> SDK-harness or vice-versa).
>>>                         >>>> Therefore - technically (and formally) -
>>>                         >>>> the SDK coder for the TestStream is
>>>                         known to the runner, regardless of
>>>                         >>>> the language the runner is written in.
>>>                         >>>>
>>>                         >>>> To move this discussion forward, I
>>>                         think we should look for answers to
>>>                         >>>> the following questions:
>>>                         >>>>
>>>                         >>>>     a) do we have any clues that show,
>>>                         that the proposed "in runner"
>>>                         >>>> solution will not work?
>>>                         >>> OK, thinking about it some more, in the
>>>                         TestStream, we can use the
>>>                         >>> happy coincidence that
>>>                         >>>
>>>                         >>>  LengthPrefixed(C).encode(x, nested=True) ==
>>>                         >>> VarLong.encode(len(C.encode(x,
>>>                         nested=False))) || C.encode(x,
>>>                         >>> nested=False)
>>>                         >>>
>>>                         >>> (where || denotes concatenation) and the
>>>                         fact that we have
>>>                         >>>
>>>                         >>>  C.encode(x, nested=False)
>>>                         >>>
>>>                         >>> in hand.
>>>                         >>>
>>>                         >>> A possible fix here for the OP's
>>>                         question is that when rehydrating the
>>>                         >>> TestStream transform it must behave
>>>                         differently according to the coder
>>>                         >>> used in the subsequent channel (e.g. for
>>>                         known coders, it decodes the
>>>                         >>> elements and emits them directly, but
>>>                         for unknown coders, it prefixes
>>>                         >>> them with their length and emits byte
>>>                         strings. It gets more
>>>                         >>> complicated for nested coders, e.g. for
>>>                         a KV<known-coder,
>>>                         >>> unknown-coder> the channel might be
>>>                         LP(KV<known-coder, unknown-coder))
>>>                         >>> or KV<known-coder, LP(unknown-coder))
>>>                         which have different encodings
>>>                         >>> (and the latter, which is the default,
>>>                         requires transcoding the bytes
>>>                         >>> to inject the length in the middle which
>>>                         is found by decoding the
>>>                         >>> first component). As well as getting
>>>                         more complex, this really seems
>>>                         >>> to violate the spirit of separation of
>>>                         concerns.
>>>                         >> How do we make the decision if the
>>>                         channel is LP<KV<..>> or
>>>                         >> KV<LP<unknown>, known>? From my
>>>                         understanding it is always the latter,
>>>                         >> because of [2].
>>>                         >>
>>>                         >> [2]
>>>                         >>
>>>                         https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>                         <https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48>
>>>                         > It is currently the latter for runners
>>>                         using this code (which not all
>>>                         > do, e.g. the ULR and Dataflow runners). I
>>>                         don't think we want to
>>>                         > ossify this decision as part of the spec.
>>>                         (Note that even what's
>>>                         > "known" and "unknown" can change from
>>>                         runner to runner.)
>>>                         >
>>>                         >>>>     b) do we think, that it will not be
>>>                         robust enough to incorporate the
>>>                         >>>> other use-cases (line generic transform
>>>                         inlining, taking into account
>>>                         >>>> that this applies only to runners that
>>>                         are written in the same language
>>>                         >>>> as the submitting SDK, because
>>>                         otherwise, there is nothing to inline)?
>>>                         >>> Being in the same language is not a
>>>                         prerequisite to "inlining," e.g.
>>>                         >>> the PubSub source on Dataflow is
>>>                         recognized as such and not executed
>>>                         >>> as SDK code but natively.
>>>                         >> Agree, that is actually exactly what
>>>                         happens with the TestStream. The
>>>                         >> transform need not be in the same
>>>                         language, as long as it is completely
>>>                         >> understood by the runner, including the
>>>                         SDK-coder (either explicitly -
>>>                         >> which might be due to the PCollection
>>>                         coder being composed of well-known
>>>                         >> coders only, or implicitly like in the
>>>                         case of TestStream, where the
>>>                         >> elements are encoded using the SDK coder.
>>>                         >>> It is more likely that inlining occurs
>>>                         in the same language if there
>>>                         >>> are UDFs involved.
>>>                         >>>
>>>                         >>>> I'm convinced, that the
>>>                         TestStream-decode expansion solution is an
>>>                         >>>> ad-hoc solution to a generic problem,
>>>                         which is why I'm still bothering
>>>                         >>>> this mailing list with my emails on
>>>                         this. :-)
>>>                         >>>>
>>>                         >>>> WDYT?
>>>                         >>> While not a solution to the general
>>>                         problem, I think the
>>>                         >>> TestStream-only-does-bytes simplifies
>>>                         its definition (primitives
>>>                         >>> should have as simple/easy to implement
>>>                         definitions as possible) and
>>>                         >>> brings it closer to the other root we
>>>                         have: Impulse. (We could go a
>>>                         >>> step further and rather than emitting
>>>                         encoded elements, with the data
>>>                         >>> in the proto itself, it emits sequence
>>>                         numbers, and a subsequent ParDo
>>>                         >>> maps those to concrete elements (e.g.
>>>                         via an in-memory map), but that
>>>                         >>> further step doesn't buy much...)
>>>                         >>>
>>>                         >>> Only runners that want to do inlining
>>>                         would have to take on the
>>>                         >>> complexity of a fully generic solution.
>>>                         >> I think that if the simplification brings
>>>                         something, we can do that, but
>>>                         >> I'd like to understand why we cannot (or
>>>                         should not) use the generic
>>>                         >> solution. I think it definitely *should*
>>>                         be possible to use a generic
>>>                         >> solution, because otherwise the solution
>>>                         would not be generic. And it
>>>                         >> would imply, that we are unable to do
>>>                         generic transform inlining, which
>>>                         >> I would find really strange. That would
>>>                         immediately mean, that we are
>>>                         >> unable to construct classical runner as a
>>>                         special case of the portable
>>>                         >> one, which would be bad I think.
>>>                         >>
>>>                         >> The elements in the TestStreamPayload are
>>>                         encoded with pure SDK-coder,
>>>                         >> or does this go through the
>>>                         LengthPrefixUnknownCoders logic? If not,
>>>                         >> then the problem would be there, because
>>>                         that means, that the SDK-coder
>>>                         >> cannot be (implicitly) defined in the
>>>                         runner. If the elements would be
>>>                         >> encoded using LP, then it would be
>>>                         possible to decode them using
>>>                         >> runner-coder and the problem should be
>>>                         solved, or am I still missing
>>>                         >> some key parts?
>>>                         > Yes, the problem is precisely that there
>>>                         are (unspecified) constraints
>>>                         > on the coder used by the
>>>                         TestStreamPayload. Just requiring that it be
>>>                         > length prefixed is not enough, you have to
>>>                         make constraints on
>>>                         > sometimes pushing down the length
>>>                         prefixing if it's a composite (like
>>>                         > a KV) that depend on what the runner is
>>>                         expected to support in terms
>>>                         > of composites and/or the choices it
>>>                         chooses for the channel (and the
>>>                         > runner, not knowing the coder, can't
>>>                         transcode between these choices).
>>>                         >
>>>                         > The simpler solution is to constrain this
>>>                         coder to just be byte[]
>>>                         > rather than let it be a little bit
>>>                         flexible (but not wholly flexible).
>>>                         >
>>>                         > As for a fully generic solution, I think
>>>                         the issue encountered with
>>>                         > inlining Read vs. TestStream are related
>>>                         to this, but not really the
>>>                         > same. With TestStream one has an encoded
>>>                         representation of the
>>>                         > elements provided by the SDK that the
>>>                         Runner and has no SDK
>>>                         > representation/execution whereas with the
>>>                         Reads one has unencoded
>>>                         > elements in hand and a Coder that is
>>>                         understood by both (so long as
>>>                         > the channel can be negotiated correctly).
>>>                         FWIW, I think the proper
>>>                         > solution to inlining a Read (or other
>>>                         Transform that would typically
>>>                         > be executed in the SDK) is to treat it as
>>>                         a special environment (where
>>>                         > we know logically it can work) and then
>>>                         elide, as possible, the
>>>                         > various encodings, grpc calls, etc. that
>>>                         are unneeded as everything is
>>>                         > in process.
>>>                         >
>>>                         >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>                         >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan
>>>                         Lukavský<je.ik@seznam.cz
>>>                         <ma...@seznam.cz>> wrote:
>>>                         >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>                         >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan
>>>                         Lukavský<je.ik@seznam.cz
>>>                         <ma...@seznam.cz>> wrote:
>>>                         >>>>>>>> Hi,
>>>                         >>>>>>>>
>>>                         >>>>>>>> I had some more time thinking about
>>>                         this and I'll try to recap that.
>>>                         >>>>>>>> First some invariants:
>>>                         >>>>>>>>
>>>                         >>>>>>>>       a) each PCollection<T> has
>>>                         actually two coders - an _SDK coder_ and a
>>>                         >>>>>>>> _runner coder_. These coders have
>>>                         the property, that each one can
>>>                         >>>>>>>> _decode_ what the other encoded,
>>>                         but the opposite is not true, the
>>>                         >>>>>>>> coders cannot _encode_ what the
>>>                         other _decoded_ (in general).
>>>                         >>>>>>>>
>>>                         >>>>>>>>       b) when is a PCollection<T>
>>>                         computed inside an environment, the
>>>                         >>>>>>>> elements are encoded using SDK
>>>                         coder on the side of SDK-harness and
>>>                         >>>>>>>> decoded using runner coder after
>>>                         receiving in the runner
>>>                         >>>>>>>>
>>>                         >>>>>>>>       c) under specific
>>>                         circumstances, the encode-decode step can be
>>>                         >>>>>>>> optimized out, that is the case
>>>                         where the SDK coder and all its
>>>                         >>>>>>>> subcoders are all well-known to the
>>>                         runner (in the present, that means
>>>                         >>>>>>>> that all the parts present in the
>>>                         model coders set). The reason for that
>>>                         >>>>>>>> is that in this specific situation
>>>                         runner_decode(sdk_encode(X)) = X.
>>>                         >>>>>>>> This property is essential.
>>>                         >>>>>>> However, in general, X can only pass
>>>                         from the SDK to the runner (or
>>>                         >>>>>>> vice versa) in encoded form.
>>>                         >>>>>> In general yes, but we are (mostly)
>>>                         talking transform inlining here, so
>>>                         >>>>>> it that particular situation, the
>>>                         elements might be passed in decoded form.
>>>                         >>>>>>>>       d) from b) immediately
>>>                         follows, that when a PTransform does not run in
>>>                         >>>>>>>> an environment (and this might be
>>>                         due to the transform being runner
>>>                         >>>>>>>> native, inlined, source (e.g.
>>>                         Impulse or TestStream)) the elements have
>>>                         >>>>>>>> to be encoded by SDK coder,
>>>                         immediately following decode by runner
>>>                         >>>>>>>> coder. That (surprisingly) applies
>>>                         even to situations when runner is
>>>                         >>>>>>>> implemented using different
>>>                         language than the client SDK, because it
>>>                         >>>>>>>> implies that the type of produced
>>>                         elements must be one of types encoded
>>>                         >>>>>>>> using model coders (well-known to
>>>                         the runner, otherwise the SDK will not
>>>                         >>>>>>>> be able to consume it). But - due
>>>                         to property c) - this means that this
>>>                         >>>>>>>> encode-decode step can be optimized
>>>                         out. This does not mean that it is
>>>                         >>>>>>>> not (logically) present, though.
>>>                         This is exactly the case of native
>>>                         >>>>>>>> Impulse transform.
>>>                         >>>>>>>>
>>>                         >>>>>>>> Now, from that we can conclude that
>>>                         on the boundary between executable
>>>                         >>>>>>>> stages, or between runner (inlined)
>>>                         transform and executable stage, each
>>>                         >>>>>>>> PCollection has to be encoded using
>>>                         SDK coder and immediately decoded by
>>>                         >>>>>>>> runner coder, *unless this can be
>>>                         optimized out* by property c).
>>>                         >>>>>>>>
>>>                         >>>>>>>> This gives us two options where to
>>>                         implement this encode/decode step:
>>>                         >>>>>>>>
>>>                         >>>>>>>>       1) completely inside runner
>>>                         with the possibility to optimize the
>>>                         >>>>>>>> encode/decode step by identity
>>>                         under right circumstances
>>>                         >>>>>>>>
>>>                         >>>>>>>>       2) partly in the runner and
>>>                         partly in the SDK - that is we encode
>>>                         >>>>>>>> elements of PCollection using SDK
>>>                         coder into bytes, pass those to the
>>>                         >>>>>>>> SDK harness and apply a custom
>>>                         decode step there. This works because SDK
>>>                         >>>>>>>> coder encoded elements are in
>>>                         byte[], and that is well-known coder type.
>>>                         >>>>>>>> We again only leverage property c)
>>>                         and optimize the SDK coder encode,
>>>                         >>>>>>>> runner decode step out.
>>>                         >>>>>>>>
>>>                         >>>>>>>> The option 2) is exactly the
>>>                         proposal of TestStream producing byte[] and
>>>                         >>>>>>>> decoding inside SDK-harness, the
>>>                         TestStream is actually inlined
>>>                         >>>>>>>> transform, the elements are
>>>                         produced directly in runner (the SDK coder
>>>                         >>>>>>>> is not known to the runner, but
>>>                         that does not matter, because the
>>>                         >>>>>>>> elements are already encoded by
>>>                         client).
>>>                         >>>>>>>>
>>>                         >>>>>>>>      From the above it seems to me,
>>>                         that option 1) should be preferred, because:
>>>                         >>>>>>>>
>>>                         >>>>>>>>       i) it is generic, applicable
>>>                         to all inlined transforms, any sources
>>>                         >>>>>>>>
>>>                         >>>>>>>>       ii) it is consistent with how
>>>                         things logically work underneath
>>>                         >>>>>>>>
>>>                         >>>>>>>>       iii) it offers better room
>>>                         for optimization - option 2) might result
>>>                         >>>>>>>> in cases when the elements are
>>>                         passed from the runner to the SDK-harness
>>>                         >>>>>>>> only for the sake of the decoding
>>>                         from SDK coder and immediately
>>>                         >>>>>>>> encoding back using SDK-coder and
>>>                         returned back to the runner. This
>>>                         >>>>>>>> would be the case when TestStream
>>>                         would be directly consumed by inlined
>>>                         >>>>>>>> (or external) transform.
>>>                         >>>>>>> (1) is not possible if the Coder in
>>>                         question is not known to the
>>>                         >>>>>>> Runner, which is why I proposed (2).
>>>                         >>>>>> There is no particular need for the
>>>                         coder to be known. If transform is
>>>                         >>>>>> to be inlined, what *has* to be known
>>>                         is the SDK-encoded form of the
>>>                         >>>>>> elements. That holds true if:
>>>                         >>>>>>
>>>                         >>>>>>   a) either the SDK coder is known, or
>>>                         >>>>>>
>>>                         >>>>>>   b) encoded form of the produced
>>>                         elements is known in advance
>>>                         >>>>>>
>>>                         >>>>>> For TestStream it is the case b). For
>>>                         inlined primitive Read (or any
>>>                         >>>>>> other transform which executes code)
>>>                         it is a).
>>>                         >>>>> There's another hitch here for
>>>                         TestStream. For historical reasons,
>>>                         >>>>> coders actually represent two
>>>                         encodings: nested (aka self delimiting)
>>>                         >>>>> and unnested. TestStream elements are
>>>                         given as unnested encoded bytes,
>>>                         >>>>> but the nested encoding is required
>>>                         for sending data to the SDK. The
>>>                         >>>>> runner can't go from <nested encoding>
>>>                         to <unnested encoding> for an
>>>                         >>>>> arbitrary unknown coder.
>>>                         >>>>>
>>>                         >>>>> (Even if it weren't for this
>>>                         complication, to be able to send already
>>>                         >>>>> encoded bytes of an unknown coder to
>>>                         the SDK will also complicate the
>>>                         >>>>> logic in choosing the coder to be used
>>>                         for the channel and sending the
>>>                         >>>>> data, which is some of what you're
>>>                         running into (but can be solved
>>>                         >>>>> differently for inlined reads as the
>>>                         coder can always be known by the
>>>                         >>>>> runner).)
>>>
>>
>>
>>                 -- 
>>
>>                 	
>>
>>                 Jack McCluskey
>>                 SWE - DataPLS PLAT/ Beam Go
>>                 RDU
>>                 jrmccluskey@gmail.com <ma...@gmail.com>
>>
>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Burke <ro...@frantil.com>.
https://issues.apache.org/jira/browse/BEAM-12866

I suppose we could switch between the existing to be deprecated mode and
the new mode by omitting the coder parameter. Runners would either
understand what it means, fail with an unknown coder, or for newer runners,
say that the mode is unsupported.


On Thu, Sep 9, 2021, 1:23 PM Robert Bradshaw <ro...@google.com> wrote:

> Well, a runner is *always* free to LP the encoded bytes, regardless of
> whether the SDK understands them. But it may need to LP the components
> rather than the whole thing. And it may not be able to do that if some of
> the components are unknown to the runner. And of course this decision may
> not be very local (e.g. it depends on the consumers, of which there may be
> more than one).
>
> The solution is to deprecate the coder-parameterized TestStream (it only
> and always produces bytes) and implement the typed TestStream as a
> composite. https://issues.apache.org/jira/browse/BEAM-2352322
>
> On Thu, Sep 9, 2021 at 1:13 PM Robert Burke <ro...@frantil.com> wrote:
>
>> IIUC that means that for TestStream a runner should:
>> * Look at the coder being used in TestStream
>> * Look at the list of SDK coder capabilities.
>> * If the test stream coder is built of known coder capabilities, then it
>> should not try to LP the provided encoded bytes, but instead simply window
>> wrap them?
>>
>> Is that right?
>>
>> On Thu, Sep 9, 2021, 12:58 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> Go doesn't have a (portable, cross-language) runner. It's fine for SDKs
>>> to enumerate the coders they understand. Runners, however, should respect
>>> what SDKs declare.
>>>
>>> On Thu, Sep 9, 2021 at 12:54 PM Jack McCluskey <jr...@google.com>
>>> wrote:
>>>
>>>> Go also doesn't do this, instead maintaining a list of known standard
>>>> coders and using that as a list of capabilities:
>>>> https://github.com/apache/beam/blob/4bc870806099f03265e7dfb48b142f00cee42f47/sdks/go/pkg/beam/core/runtime/graphx/coder.go#L59
>>>>
>>>> On Thu, Sep 9, 2021 at 1:29 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> You are right, Java does not implement this correctly. It should be
>>>>> querying the capabilities section of the environment proto. (For java
>>>>> environments, this is populated from ModelCoders, e.g.
>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386
>>>>> )
>>>>>
>>>>> Looks like Python doesn't do any better:
>>>>> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459
>>>>>
>>>>>
>>>>>
>>>>> On Thu, Sep 9, 2021 at 10:13 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>
>>>>>> This makes a *lot* of sense. But it seems to me, that this is not the
>>>>>> way Java-based runners - that use runners-core-construction-java module -
>>>>>> handle it. If I interpret it correctly, then the set of ModelCoders is
>>>>>> hard-coded [1] and essentially required to be known by all SDKs [2].
>>>>>>
>>>>>> There seems to be no negotiation between what SDK harness knows and
>>>>>> what the runner knows. The runner might be able to define the wire coder
>>>>>> for the SDK (via the ProcessBundleDescriptor), but the SDK (for Java
>>>>>> runners) seems not to be able to play any role in this [3]. Therefore I
>>>>>> think that if an SDK does not know the set of Java ModelCoders, then the
>>>>>> runner and the SDK might not agree on the binary encoding of the elements
>>>>>> (BTW, which is why Java Coders cannot be part of ModelCoders and I finally
>>>>>> understand why I had such troubles adding it there - it cannot be there).
>>>>>>
>>>>>> Is it possible we are missing some part of this runner-to-sdk coder
>>>>>> negotiation in runners-core-contruction-java?
>>>>>>
>>>>>> [1]
>>>>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75
>>>>>>
>>>>>> [2]
>>>>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62
>>>>>>
>>>>>> [3]
>>>>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104
>>>>>> On 9/9/21 12:18 AM, Robert Bradshaw wrote:
>>>>>>
>>>>>> The whole notion of an absolute set of known coders is a misnomer. It
>>>>>> would require all Runners and SDKs to be updated synchronously for every
>>>>>> new coder they might want to share.
>>>>>>
>>>>>> Instead, what we have are
>>>>>>
>>>>>> * Standard Coders which have well-defined, language-agnostic
>>>>>> representations and encodings, which may be used for interoperability and
>>>>>> efficiency, and
>>>>>> * Required Coders which are the minimum needed to execute the
>>>>>> pipeline.
>>>>>>
>>>>>> The latter consists only of bytes (for impulse), kv and iterable (for
>>>>>> GBK), windowed value (for windowing information) and length prefix (to be
>>>>>> able to handle anything else).
>>>>>>
>>>>>>
>>>>>> On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <ro...@frantil.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Is the claim that the Standard bytes and String_utf8 coders are not
>>>>>>> "known coders"?
>>>>>>>
>>>>>>>  What's the point of the standard coders if they are not the
>>>>>>> canonical "known coders" that can generally be expected to be known by
>>>>>>> runners/other SDKs?
>>>>>>>
>>>>>>>
>>>>>>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>>>>>>>
>>>>>>> The Go SDK rather heavily biases towards using the standard coders
>>>>>>> for their closes language equivalents rather than going into
>>>>>>> override/custom specified soup. (It's not possible to globally override the
>>>>>>> coders for the '[]byte' and 'string' types, nor is there often reason to.)
>>>>>>>
>>>>>>> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <
>>>>>>>> jrmccluskey@google.com> wrote:
>>>>>>>>
>>>>>>>>> Hey all,
>>>>>>>>>
>>>>>>>>> Just catching up on the thread since I did the TestStream Go SDK
>>>>>>>>> implementation. The discussion about length prefixing behavior for known
>>>>>>>>> vs. unknown coders is interesting, since we ran into strings and byte
>>>>>>>>> slices getting extra length prefixes attached to them by Flink despite
>>>>>>>>> being known coders.
>>>>>>>>>
>>>>>>>>
>>>>>>>> Known to who?
>>>>>>>>
>>>>>>>>
>>>>>>>>> Based on what's been said, that isn't expected behavior, right?
>>>>>>>>>
>>>>>>>>
>>>>>>>> No, it's not.
>>>>>>>>
>>>>>>>> I would check to make sure the Go SDK is respecting the Coder
>>>>>>>> (length prefixed or not) that's set on the channel, rather than guessing at
>>>>>>>> what it expects it to be based on the Go type.
>>>>>>>>
>>>>>>>>
>>>>>>>>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>>>>>>>>
>>>>>>>>>> Historically the DataflowRunner has been much more careful about
>>>>>>>>>> not breaking update, since this is a frequent operation by Dataflow users.
>>>>>>>>>> I think we've been less careful aboutt other runners, but as we see clearly
>>>>>>>>>> here Fllnk users do care about this as well, so we should probably test
>>>>>>>>>> upgrade compatibility for Flink.
>>>>>>>>>>
>>>>>>>>>> One strategy that Dataflow uses is to avoid embedding the Java
>>>>>>>>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>>>>>>>>> breakage (as we see with the issue you llnked to). Possibly similar
>>>>>>>>>> strategies should be investigated for Fllink.
>>>>>>>>>>
>>>>>>>>>> +1, that would be great!
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Reuven
>>>>>>>>>>
>>>>>>>>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>>>>> iterable)
>>>>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>>>>> > changing coders is hard (due to no way to update the encoding
>>>>>>>>>>> for a
>>>>>>>>>>> > streaming pipeline).
>>>>>>>>>>> This is unrelated, but - regarding changing coders due to
>>>>>>>>>>> concerns about
>>>>>>>>>>> pipeline upgrades, we break this quite often, at least for some
>>>>>>>>>>> runners.
>>>>>>>>>>> Most recently [1].
>>>>>>>>>>>
>>>>>>>>>>> > It is currently the latter for runners using this code (which
>>>>>>>>>>> not all
>>>>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want
>>>>>>>>>>> to
>>>>>>>>>>> > ossify this decision as part of the spec. (Note that even
>>>>>>>>>>> what's
>>>>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>>>>> This is interesting and unexpected for me. How do runners decide
>>>>>>>>>>> about
>>>>>>>>>>> how they encode elements between SDK harness and the runner? How
>>>>>>>>>>> do they
>>>>>>>>>>> inform the SDK harness about this decision? My impression was
>>>>>>>>>>> that this
>>>>>>>>>>> is well-defined at the model level. If not, then we have the
>>>>>>>>>>> reason for
>>>>>>>>>>> misunderstanding in this conversation. :-)
>>>>>>>>>>>
>>>>>>>>>>>   Jan
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>>
>>>>>>>>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>>>>>>>>
>>>>>>>>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>>>>>>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz>
>>>>>>>>>>> wrote:
>>>>>>>>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>>>>>>>>> >>
>>>>>>>>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>>>>> wrote:
>>>>>>>>>>> >>>> Hi Robert,
>>>>>>>>>>> >>>>
>>>>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>>>>> reasons,
>>>>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>>>>> delimiting)
>>>>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>>>>> encoded bytes,
>>>>>>>>>>> >>>>> but the nested encoding is required for sending data to
>>>>>>>>>>> the SDK. The
>>>>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested
>>>>>>>>>>> encoding> for an
>>>>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>>>>> >>>>>
>>>>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to
>>>>>>>>>>> send already
>>>>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>>>>> complicate the
>>>>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>>>>> sending the
>>>>>>>>>>> >>>>> data, which is some of what you're running into (but can
>>>>>>>>>>> be solved
>>>>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>>>>> known by the
>>>>>>>>>>> >>>>> runner).)
>>>>>>>>>>> >>>> It is hard for me to argue with "historical reasons". But -
>>>>>>>>>>> the "nested"
>>>>>>>>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>>>>>>>>> runner-coder
>>>>>>>>>>> >>>> spaces.
>>>>>>>>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>>>>>>>>> >> Hm, do you mean the Context passed to the encode/decode
>>>>>>>>>>> method? [1] That
>>>>>>>>>>> >> seems to be deprecated, I assume that most coders use the
>>>>>>>>>>> default
>>>>>>>>>>> >> implementation and simply ignore the Context?
>>>>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>>>>> iterable)
>>>>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>>>>> > changing coders is hard (due to no way to update the encoding
>>>>>>>>>>> for a
>>>>>>>>>>> > streaming pipeline).
>>>>>>>>>>> >
>>>>>>>>>>> >> Even if not - whether or
>>>>>>>>>>> >> not the elements are encoded using NESTED Context or UNNESTED
>>>>>>>>>>> Context
>>>>>>>>>>> >> should be part of the contract of TestStream, right? Most
>>>>>>>>>>> likely it is
>>>>>>>>>>> >> the UNNESTED one, if I understand correctly what that does.
>>>>>>>>>>> Under what
>>>>>>>>>>> >> conditions is the deprecated encode/decode method used?
>>>>>>>>>>> > Yes, it's the UNNESTED one.
>>>>>>>>>>> >
>>>>>>>>>>> >> [1]
>>>>>>>>>>> >>
>>>>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>>>>>>>>> >>
>>>>>>>>>>> >>>> The runner's responsibility is not to go from "<nested
>>>>>>>>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for
>>>>>>>>>>> arbitrary coder.
>>>>>>>>>>> >>>> That is really impossible. But a coder is a function,
>>>>>>>>>>> right? Function
>>>>>>>>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>>>>>>>>> provides a
>>>>>>>>>>> >>>> set of elements, and these elements are the "universe". For
>>>>>>>>>>> those
>>>>>>>>>>> >>>> elements it also provides the encoded form, which can be
>>>>>>>>>>> interpreted as
>>>>>>>>>>> >>>> the definition of the coder.
>>>>>>>>>>> >>> The problem here is that there is not "the encoded form" for
>>>>>>>>>>> a Coder
>>>>>>>>>>> >>> but two encoded forms, and we have the wrong one. Things
>>>>>>>>>>> could be made
>>>>>>>>>>> >>> to work if we had the other.
>>>>>>>>>>> >> Which two encoded forms do you refer to? Elements encoded by
>>>>>>>>>>> both the
>>>>>>>>>>> >> SDK-coder and runner-coder (and I ignore the Context here
>>>>>>>>>>> once again)
>>>>>>>>>>> >> have the same binary representation (which they must have,
>>>>>>>>>>> otherwise it
>>>>>>>>>>> >> would be impossible to decode elements coming from the runner
>>>>>>>>>>> to the
>>>>>>>>>>> >> SDK-harness or vice-versa).
>>>>>>>>>>> >>>> Therefore - technically (and formally) -
>>>>>>>>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>>>>>>>>> regardless of
>>>>>>>>>>> >>>> the language the runner is written in.
>>>>>>>>>>> >>>>
>>>>>>>>>>> >>>> To move  this discussion forward, I think we should look
>>>>>>>>>>> for answers to
>>>>>>>>>>> >>>> the following questions:
>>>>>>>>>>> >>>>
>>>>>>>>>>> >>>>     a) do we have any clues that show, that the proposed
>>>>>>>>>>> "in runner"
>>>>>>>>>>> >>>> solution will not work?
>>>>>>>>>>> >>> OK, thinking about it some more, in the TestStream, we can
>>>>>>>>>>> use the
>>>>>>>>>>> >>> happy coincidence that
>>>>>>>>>>> >>>
>>>>>>>>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>>>>>>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>>>>>>>>> >>> nested=False)
>>>>>>>>>>> >>>
>>>>>>>>>>> >>> (where || denotes concatenation) and the fact that we have
>>>>>>>>>>> >>>
>>>>>>>>>>> >>>       C.encode(x, nested=False)
>>>>>>>>>>> >>>
>>>>>>>>>>> >>> in hand.
>>>>>>>>>>> >>>
>>>>>>>>>>> >>> A possible fix here for the OP's question is that when
>>>>>>>>>>> rehydrating the
>>>>>>>>>>> >>> TestStream transform it must behave differently according to
>>>>>>>>>>> the coder
>>>>>>>>>>> >>> used in the subsequent channel (e.g. for known coders, it
>>>>>>>>>>> decodes the
>>>>>>>>>>> >>> elements and emits them directly, but for unknown coders, it
>>>>>>>>>>> prefixes
>>>>>>>>>>> >>> them with their length and emits byte strings. It gets more
>>>>>>>>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>>>>>>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>>>>>>>>> unknown-coder))
>>>>>>>>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different
>>>>>>>>>>> encodings
>>>>>>>>>>> >>> (and the latter, which is the default, requires transcoding
>>>>>>>>>>> the bytes
>>>>>>>>>>> >>> to inject the length in the middle which is found by
>>>>>>>>>>> decoding the
>>>>>>>>>>> >>> first component). As well as getting more complex, this
>>>>>>>>>>> really seems
>>>>>>>>>>> >>> to violate the spirit of separation of concerns.
>>>>>>>>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>>>>>>>>> >> KV<LP<unknown>, known>? From my understanding it is always
>>>>>>>>>>> the latter,
>>>>>>>>>>> >> because of [2].
>>>>>>>>>>> >>
>>>>>>>>>>> >> [2]
>>>>>>>>>>> >>
>>>>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>>>>>> > It is currently the latter for runners using this code (which
>>>>>>>>>>> not all
>>>>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want
>>>>>>>>>>> to
>>>>>>>>>>> > ossify this decision as part of the spec. (Note that even
>>>>>>>>>>> what's
>>>>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>>>>> >
>>>>>>>>>>> >>>>     b) do we think, that it will not be robust enough to
>>>>>>>>>>> incorporate the
>>>>>>>>>>> >>>> other use-cases (line generic transform inlining, taking
>>>>>>>>>>> into account
>>>>>>>>>>> >>>> that this applies only to runners that are written in the
>>>>>>>>>>> same language
>>>>>>>>>>> >>>> as the submitting SDK, because otherwise, there is nothing
>>>>>>>>>>> to inline)?
>>>>>>>>>>> >>> Being in the same language is not a prerequisite to
>>>>>>>>>>> "inlining," e.g.
>>>>>>>>>>> >>> the PubSub source on Dataflow is recognized as such and not
>>>>>>>>>>> executed
>>>>>>>>>>> >>> as SDK code but natively.
>>>>>>>>>>> >> Agree, that is actually exactly what happens with the
>>>>>>>>>>> TestStream. The
>>>>>>>>>>> >> transform need not be in the same language, as long as it is
>>>>>>>>>>> completely
>>>>>>>>>>> >> understood by the runner, including the SDK-coder (either
>>>>>>>>>>> explicitly -
>>>>>>>>>>> >> which might be due to the PCollection coder being composed of
>>>>>>>>>>> well-known
>>>>>>>>>>> >> coders only, or implicitly like in the case of TestStream,
>>>>>>>>>>> where the
>>>>>>>>>>> >> elements are encoded using the SDK coder.
>>>>>>>>>>> >>> It is more likely that inlining occurs in the same language
>>>>>>>>>>> if there
>>>>>>>>>>> >>> are UDFs involved.
>>>>>>>>>>> >>>
>>>>>>>>>>> >>>> I'm convinced, that the TestStream-decode expansion
>>>>>>>>>>> solution is an
>>>>>>>>>>> >>>> ad-hoc solution to a generic problem, which is why I'm
>>>>>>>>>>> still bothering
>>>>>>>>>>> >>>> this mailing list with my emails on this. :-)
>>>>>>>>>>> >>>>
>>>>>>>>>>> >>>> WDYT?
>>>>>>>>>>> >>> While not a solution to the general problem, I think the
>>>>>>>>>>> >>> TestStream-only-does-bytes simplifies its definition
>>>>>>>>>>> (primitives
>>>>>>>>>>> >>> should have as simple/easy to implement definitions as
>>>>>>>>>>> possible) and
>>>>>>>>>>> >>> brings it closer to the other root we have: Impulse. (We
>>>>>>>>>>> could go a
>>>>>>>>>>> >>> step further and rather than emitting encoded elements, with
>>>>>>>>>>> the data
>>>>>>>>>>> >>> in the proto itself, it emits sequence numbers, and a
>>>>>>>>>>> subsequent ParDo
>>>>>>>>>>> >>> maps those to concrete elements (e.g. via an in-memory map),
>>>>>>>>>>> but that
>>>>>>>>>>> >>> further step doesn't buy much...)
>>>>>>>>>>> >>>
>>>>>>>>>>> >>> Only runners that want to do inlining would have to take on
>>>>>>>>>>> the
>>>>>>>>>>> >>> complexity of a fully generic solution.
>>>>>>>>>>> >> I think that if the simplification brings something, we can
>>>>>>>>>>> do that, but
>>>>>>>>>>> >> I'd like to understand why we cannot (or should not) use the
>>>>>>>>>>> generic
>>>>>>>>>>> >> solution. I think it definitely *should* be possible to use a
>>>>>>>>>>> generic
>>>>>>>>>>> >> solution, because otherwise the solution would not be
>>>>>>>>>>> generic. And it
>>>>>>>>>>> >> would imply, that we are unable to do generic transform
>>>>>>>>>>> inlining, which
>>>>>>>>>>> >> I would find really strange. That would immediately mean,
>>>>>>>>>>> that we are
>>>>>>>>>>> >> unable to construct classical runner as a special case of the
>>>>>>>>>>> portable
>>>>>>>>>>> >> one, which would be bad I think.
>>>>>>>>>>> >>
>>>>>>>>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>>>>>>>>> SDK-coder,
>>>>>>>>>>> >> or does this go through the LengthPrefixUnknownCoders logic?
>>>>>>>>>>> If not,
>>>>>>>>>>> >> then the problem would be there, because that means, that the
>>>>>>>>>>> SDK-coder
>>>>>>>>>>> >> cannot be (implicitly) defined in the runner. If the elements
>>>>>>>>>>> would be
>>>>>>>>>>> >> encoded using LP, then it would be possible to decode them
>>>>>>>>>>> using
>>>>>>>>>>> >> runner-coder and the problem should be solved, or am I still
>>>>>>>>>>> missing
>>>>>>>>>>> >> some key parts?
>>>>>>>>>>> > Yes, the problem is precisely that there are (unspecified)
>>>>>>>>>>> constraints
>>>>>>>>>>> > on the coder used by the TestStreamPayload. Just requiring
>>>>>>>>>>> that it be
>>>>>>>>>>> > length prefixed is not enough, you have to make constraints on
>>>>>>>>>>> > sometimes pushing down the length prefixing if it's a
>>>>>>>>>>> composite (like
>>>>>>>>>>> > a KV) that depend on what the runner is expected to support in
>>>>>>>>>>> terms
>>>>>>>>>>> > of composites and/or the choices it chooses for the channel
>>>>>>>>>>> (and the
>>>>>>>>>>> > runner, not knowing the coder, can't transcode between these
>>>>>>>>>>> choices).
>>>>>>>>>>> >
>>>>>>>>>>> > The simpler solution is to constrain this coder to just be
>>>>>>>>>>> byte[]
>>>>>>>>>>> > rather than let it be a little bit flexible (but not wholly
>>>>>>>>>>> flexible).
>>>>>>>>>>> >
>>>>>>>>>>> > As for a fully generic solution, I think the issue encountered
>>>>>>>>>>> with
>>>>>>>>>>> > inlining Read vs. TestStream are related to this, but not
>>>>>>>>>>> really the
>>>>>>>>>>> > same. With TestStream one has an encoded representation of the
>>>>>>>>>>> > elements provided by the SDK that the Runner and has no SDK
>>>>>>>>>>> > representation/execution whereas with the Reads one has
>>>>>>>>>>> unencoded
>>>>>>>>>>> > elements in hand and a Coder that is understood by both (so
>>>>>>>>>>> long as
>>>>>>>>>>> > the channel can be negotiated correctly). FWIW, I think the
>>>>>>>>>>> proper
>>>>>>>>>>> > solution to inlining a Read (or other Transform that would
>>>>>>>>>>> typically
>>>>>>>>>>> > be executed in the SDK) is to treat it as a special
>>>>>>>>>>> environment (where
>>>>>>>>>>> > we know logically it can work) and then elide, as possible, the
>>>>>>>>>>> > various encodings, grpc calls, etc. that are unneeded as
>>>>>>>>>>> everything is
>>>>>>>>>>> > in process.
>>>>>>>>>>> >
>>>>>>>>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>>>>>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<
>>>>>>>>>>> je.ik@seznam.cz>  wrote:
>>>>>>>>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>>>>>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<
>>>>>>>>>>> je.ik@seznam.cz>  wrote:
>>>>>>>>>>> >>>>>>>> Hi,
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>> I had some more time thinking about this and I'll try
>>>>>>>>>>> to recap that.
>>>>>>>>>>> >>>>>>>> First some invariants:
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>>       a) each PCollection<T> has actually two coders -
>>>>>>>>>>> an _SDK coder_ and a
>>>>>>>>>>> >>>>>>>> _runner coder_. These coders have the property, that
>>>>>>>>>>> each one can
>>>>>>>>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is
>>>>>>>>>>> not true, the
>>>>>>>>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in
>>>>>>>>>>> general).
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>>>>>>>>> environment, the
>>>>>>>>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>>>>>>>>> SDK-harness and
>>>>>>>>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>>       c) under specific circumstances, the
>>>>>>>>>>> encode-decode step can be
>>>>>>>>>>> >>>>>>>> optimized out, that is the case where the SDK coder and
>>>>>>>>>>> all its
>>>>>>>>>>> >>>>>>>> subcoders are all well-known to the runner (in the
>>>>>>>>>>> present, that means
>>>>>>>>>>> >>>>>>>> that all the parts present in the model coders set).
>>>>>>>>>>> The reason for that
>>>>>>>>>>> >>>>>>>> is that in this specific situation
>>>>>>>>>>> runner_decode(sdk_encode(X)) = X.
>>>>>>>>>>> >>>>>>>> This property is essential.
>>>>>>>>>>> >>>>>>> However, in general, X can only pass from the SDK to the
>>>>>>>>>>> runner (or
>>>>>>>>>>> >>>>>>> vice versa) in encoded form.
>>>>>>>>>>> >>>>>> In general yes, but we are (mostly) talking transform
>>>>>>>>>>> inlining here, so
>>>>>>>>>>> >>>>>> it that particular situation, the elements might be
>>>>>>>>>>> passed in decoded form.
>>>>>>>>>>> >>>>>>>>       d) from b) immediately follows, that when a
>>>>>>>>>>> PTransform does not run in
>>>>>>>>>>> >>>>>>>> an environment (and this might be due to the transform
>>>>>>>>>>> being runner
>>>>>>>>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream))
>>>>>>>>>>> the elements have
>>>>>>>>>>> >>>>>>>> to be encoded by SDK coder, immediately following
>>>>>>>>>>> decode by runner
>>>>>>>>>>> >>>>>>>> coder. That (surprisingly) applies even to situations
>>>>>>>>>>> when runner is
>>>>>>>>>>> >>>>>>>> implemented using different language than the client
>>>>>>>>>>> SDK, because it
>>>>>>>>>>> >>>>>>>> implies that the type of produced elements must be one
>>>>>>>>>>> of types encoded
>>>>>>>>>>> >>>>>>>> using model coders (well-known to the runner, otherwise
>>>>>>>>>>> the SDK will not
>>>>>>>>>>> >>>>>>>> be able to consume it). But - due to property c) - this
>>>>>>>>>>> means that this
>>>>>>>>>>> >>>>>>>> encode-decode step can be optimized out. This does not
>>>>>>>>>>> mean that it is
>>>>>>>>>>> >>>>>>>> not (logically) present, though. This is exactly the
>>>>>>>>>>> case of native
>>>>>>>>>>> >>>>>>>> Impulse transform.
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>> Now, from that we can conclude that on the boundary
>>>>>>>>>>> between executable
>>>>>>>>>>> >>>>>>>> stages, or between runner (inlined) transform and
>>>>>>>>>>> executable stage, each
>>>>>>>>>>> >>>>>>>> PCollection has to be encoded using SDK coder and
>>>>>>>>>>> immediately decoded by
>>>>>>>>>>> >>>>>>>> runner coder, *unless this can be optimized out* by
>>>>>>>>>>> property c).
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>> This gives us two options where to implement this
>>>>>>>>>>> encode/decode step:
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>>       1) completely inside runner with the possibility
>>>>>>>>>>> to optimize the
>>>>>>>>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>>       2) partly in the runner and partly in the SDK -
>>>>>>>>>>> that is we encode
>>>>>>>>>>> >>>>>>>> elements of PCollection using SDK coder into bytes,
>>>>>>>>>>> pass those to the
>>>>>>>>>>> >>>>>>>> SDK harness and apply a custom decode step there. This
>>>>>>>>>>> works because SDK
>>>>>>>>>>> >>>>>>>> coder encoded elements are in byte[], and that is
>>>>>>>>>>> well-known coder type.
>>>>>>>>>>> >>>>>>>> We again only leverage property c) and optimize the SDK
>>>>>>>>>>> coder encode,
>>>>>>>>>>> >>>>>>>> runner decode step out.
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream
>>>>>>>>>>> producing byte[] and
>>>>>>>>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>>>>>>>>>>> inlined
>>>>>>>>>>> >>>>>>>> transform, the elements are produced directly in runner
>>>>>>>>>>> (the SDK coder
>>>>>>>>>>> >>>>>>>> is not known to the runner, but that does not matter,
>>>>>>>>>>> because the
>>>>>>>>>>> >>>>>>>> elements are already encoded by client).
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>>      From the above it seems to me, that option 1)
>>>>>>>>>>> should be preferred, because:
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>>       i) it is generic, applicable to all inlined
>>>>>>>>>>> transforms, any sources
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>>       ii) it is consistent with how things logically
>>>>>>>>>>> work underneath
>>>>>>>>>>> >>>>>>>>
>>>>>>>>>>> >>>>>>>>       iii) it offers better room for optimization -
>>>>>>>>>>> option 2) might result
>>>>>>>>>>> >>>>>>>> in cases when the elements are passed from the runner
>>>>>>>>>>> to the SDK-harness
>>>>>>>>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>>>>>>>>> immediately
>>>>>>>>>>> >>>>>>>> encoding back using SDK-coder and returned back to the
>>>>>>>>>>> runner. This
>>>>>>>>>>> >>>>>>>> would be the case when TestStream would be directly
>>>>>>>>>>> consumed by inlined
>>>>>>>>>>> >>>>>>>> (or external) transform.
>>>>>>>>>>> >>>>>>> (1) is not possible if the Coder in question is not
>>>>>>>>>>> known to the
>>>>>>>>>>> >>>>>>> Runner, which is why I proposed (2).
>>>>>>>>>>> >>>>>> There is no particular need for the coder to be known. If
>>>>>>>>>>> transform is
>>>>>>>>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded
>>>>>>>>>>> form of the
>>>>>>>>>>> >>>>>> elements. That holds true if:
>>>>>>>>>>> >>>>>>
>>>>>>>>>>> >>>>>>      a) either the SDK coder is known, or
>>>>>>>>>>> >>>>>>
>>>>>>>>>>> >>>>>>      b) encoded form of the produced elements is known in
>>>>>>>>>>> advance
>>>>>>>>>>> >>>>>>
>>>>>>>>>>> >>>>>> For TestStream it is the case b). For inlined primitive
>>>>>>>>>>> Read (or any
>>>>>>>>>>> >>>>>> other transform which executes code) it is a).
>>>>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>>>>> reasons,
>>>>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>>>>> delimiting)
>>>>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>>>>> encoded bytes,
>>>>>>>>>>> >>>>> but the nested encoding is required for sending data to
>>>>>>>>>>> the SDK. The
>>>>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested
>>>>>>>>>>> encoding> for an
>>>>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>>>>> >>>>>
>>>>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to
>>>>>>>>>>> send already
>>>>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>>>>> complicate the
>>>>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>>>>> sending the
>>>>>>>>>>> >>>>> data, which is some of what you're running into (but can
>>>>>>>>>>> be solved
>>>>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>>>>> known by the
>>>>>>>>>>> >>>>> runner).)
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Jack McCluskey
>>>>>>>>> SWE - DataPLS PLAT/ Beam Go
>>>>>>>>> RDU
>>>>>>>>> jrmccluskey@gmail.com
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
Well, a runner is *always* free to LP the encoded bytes, regardless of
whether the SDK understands them. But it may need to LP the components
rather than the whole thing. And it may not be able to do that if some of
the components are unknown to the runner. And of course this decision may
not be very local (e.g. it depends on the consumers, of which there may be
more than one).

The solution is to deprecate the coder-parameterized TestStream (it only
and always produces bytes) and implement the typed TestStream as a
composite. https://issues.apache.org/jira/browse/BEAM-2352322

On Thu, Sep 9, 2021 at 1:13 PM Robert Burke <ro...@frantil.com> wrote:

> IIUC that means that for TestStream a runner should:
> * Look at the coder being used in TestStream
> * Look at the list of SDK coder capabilities.
> * If the test stream coder is built of known coder capabilities, then it
> should not try to LP the provided encoded bytes, but instead simply window
> wrap them?
>
> Is that right?
>
> On Thu, Sep 9, 2021, 12:58 PM Robert Bradshaw <ro...@google.com> wrote:
>
>> Go doesn't have a (portable, cross-language) runner. It's fine for SDKs
>> to enumerate the coders they understand. Runners, however, should respect
>> what SDKs declare.
>>
>> On Thu, Sep 9, 2021 at 12:54 PM Jack McCluskey <jr...@google.com>
>> wrote:
>>
>>> Go also doesn't do this, instead maintaining a list of known standard
>>> coders and using that as a list of capabilities:
>>> https://github.com/apache/beam/blob/4bc870806099f03265e7dfb48b142f00cee42f47/sdks/go/pkg/beam/core/runtime/graphx/coder.go#L59
>>>
>>> On Thu, Sep 9, 2021 at 1:29 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> You are right, Java does not implement this correctly. It should be
>>>> querying the capabilities section of the environment proto. (For java
>>>> environments, this is populated from ModelCoders, e.g.
>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386
>>>> )
>>>>
>>>> Looks like Python doesn't do any better:
>>>> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459
>>>>
>>>>
>>>>
>>>> On Thu, Sep 9, 2021 at 10:13 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>
>>>>> This makes a *lot* of sense. But it seems to me, that this is not the
>>>>> way Java-based runners - that use runners-core-construction-java module -
>>>>> handle it. If I interpret it correctly, then the set of ModelCoders is
>>>>> hard-coded [1] and essentially required to be known by all SDKs [2].
>>>>>
>>>>> There seems to be no negotiation between what SDK harness knows and
>>>>> what the runner knows. The runner might be able to define the wire coder
>>>>> for the SDK (via the ProcessBundleDescriptor), but the SDK (for Java
>>>>> runners) seems not to be able to play any role in this [3]. Therefore I
>>>>> think that if an SDK does not know the set of Java ModelCoders, then the
>>>>> runner and the SDK might not agree on the binary encoding of the elements
>>>>> (BTW, which is why Java Coders cannot be part of ModelCoders and I finally
>>>>> understand why I had such troubles adding it there - it cannot be there).
>>>>>
>>>>> Is it possible we are missing some part of this runner-to-sdk coder
>>>>> negotiation in runners-core-contruction-java?
>>>>>
>>>>> [1]
>>>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75
>>>>>
>>>>> [2]
>>>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62
>>>>>
>>>>> [3]
>>>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104
>>>>> On 9/9/21 12:18 AM, Robert Bradshaw wrote:
>>>>>
>>>>> The whole notion of an absolute set of known coders is a misnomer. It
>>>>> would require all Runners and SDKs to be updated synchronously for every
>>>>> new coder they might want to share.
>>>>>
>>>>> Instead, what we have are
>>>>>
>>>>> * Standard Coders which have well-defined, language-agnostic
>>>>> representations and encodings, which may be used for interoperability and
>>>>> efficiency, and
>>>>> * Required Coders which are the minimum needed to execute the
>>>>> pipeline.
>>>>>
>>>>> The latter consists only of bytes (for impulse), kv and iterable (for
>>>>> GBK), windowed value (for windowing information) and length prefix (to be
>>>>> able to handle anything else).
>>>>>
>>>>>
>>>>> On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <ro...@frantil.com>
>>>>> wrote:
>>>>>
>>>>>> Is the claim that the Standard bytes and String_utf8 coders are not
>>>>>> "known coders"?
>>>>>>
>>>>>>  What's the point of the standard coders if they are not the
>>>>>> canonical "known coders" that can generally be expected to be known by
>>>>>> runners/other SDKs?
>>>>>>
>>>>>>
>>>>>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>>>>>>
>>>>>> The Go SDK rather heavily biases towards using the standard coders
>>>>>> for their closes language equivalents rather than going into
>>>>>> override/custom specified soup. (It's not possible to globally override the
>>>>>> coders for the '[]byte' and 'string' types, nor is there often reason to.)
>>>>>>
>>>>>> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <
>>>>>>> jrmccluskey@google.com> wrote:
>>>>>>>
>>>>>>>> Hey all,
>>>>>>>>
>>>>>>>> Just catching up on the thread since I did the TestStream Go SDK
>>>>>>>> implementation. The discussion about length prefixing behavior for known
>>>>>>>> vs. unknown coders is interesting, since we ran into strings and byte
>>>>>>>> slices getting extra length prefixes attached to them by Flink despite
>>>>>>>> being known coders.
>>>>>>>>
>>>>>>>
>>>>>>> Known to who?
>>>>>>>
>>>>>>>
>>>>>>>> Based on what's been said, that isn't expected behavior, right?
>>>>>>>>
>>>>>>>
>>>>>>> No, it's not.
>>>>>>>
>>>>>>> I would check to make sure the Go SDK is respecting the Coder
>>>>>>> (length prefixed or not) that's set on the channel, rather than guessing at
>>>>>>> what it expects it to be based on the Go type.
>>>>>>>
>>>>>>>
>>>>>>>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>>>>>>>
>>>>>>>>> Historically the DataflowRunner has been much more careful about
>>>>>>>>> not breaking update, since this is a frequent operation by Dataflow users.
>>>>>>>>> I think we've been less careful aboutt other runners, but as we see clearly
>>>>>>>>> here Fllnk users do care about this as well, so we should probably test
>>>>>>>>> upgrade compatibility for Flink.
>>>>>>>>>
>>>>>>>>> One strategy that Dataflow uses is to avoid embedding the Java
>>>>>>>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>>>>>>>> breakage (as we see with the issue you llnked to). Possibly similar
>>>>>>>>> strategies should be investigated for Fllink.
>>>>>>>>>
>>>>>>>>> +1, that would be great!
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Reuven
>>>>>>>>>
>>>>>>>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>>>> iterable)
>>>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>>>> > changing coders is hard (due to no way to update the encoding
>>>>>>>>>> for a
>>>>>>>>>> > streaming pipeline).
>>>>>>>>>> This is unrelated, but - regarding changing coders due to
>>>>>>>>>> concerns about
>>>>>>>>>> pipeline upgrades, we break this quite often, at least for some
>>>>>>>>>> runners.
>>>>>>>>>> Most recently [1].
>>>>>>>>>>
>>>>>>>>>> > It is currently the latter for runners using this code (which
>>>>>>>>>> not all
>>>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>>>> This is interesting and unexpected for me. How do runners decide
>>>>>>>>>> about
>>>>>>>>>> how they encode elements between SDK harness and the runner? How
>>>>>>>>>> do they
>>>>>>>>>> inform the SDK harness about this decision? My impression was
>>>>>>>>>> that this
>>>>>>>>>> is well-defined at the model level. If not, then we have the
>>>>>>>>>> reason for
>>>>>>>>>> misunderstanding in this conversation. :-)
>>>>>>>>>>
>>>>>>>>>>   Jan
>>>>>>>>>>
>>>>>>>>>> [1]
>>>>>>>>>>
>>>>>>>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>>>>>>>
>>>>>>>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>>>>>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz>
>>>>>>>>>> wrote:
>>>>>>>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>>>>>>>> >>
>>>>>>>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>>>> wrote:
>>>>>>>>>> >>>> Hi Robert,
>>>>>>>>>> >>>>
>>>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>>>> reasons,
>>>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>>>> delimiting)
>>>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>>>> encoded bytes,
>>>>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>>>>> SDK. The
>>>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested
>>>>>>>>>> encoding> for an
>>>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>>>> >>>>>
>>>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to
>>>>>>>>>> send already
>>>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>>>> complicate the
>>>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>>>> sending the
>>>>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>>>>> solved
>>>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>>>> known by the
>>>>>>>>>> >>>>> runner).)
>>>>>>>>>> >>>> It is hard for me to argue with "historical reasons". But -
>>>>>>>>>> the "nested"
>>>>>>>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>>>>>>>> runner-coder
>>>>>>>>>> >>>> spaces.
>>>>>>>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>>>>>>>> >> Hm, do you mean the Context passed to the encode/decode
>>>>>>>>>> method? [1] That
>>>>>>>>>> >> seems to be deprecated, I assume that most coders use the
>>>>>>>>>> default
>>>>>>>>>> >> implementation and simply ignore the Context?
>>>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>>>> iterable)
>>>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>>>> > changing coders is hard (due to no way to update the encoding
>>>>>>>>>> for a
>>>>>>>>>> > streaming pipeline).
>>>>>>>>>> >
>>>>>>>>>> >> Even if not - whether or
>>>>>>>>>> >> not the elements are encoded using NESTED Context or UNNESTED
>>>>>>>>>> Context
>>>>>>>>>> >> should be part of the contract of TestStream, right? Most
>>>>>>>>>> likely it is
>>>>>>>>>> >> the UNNESTED one, if I understand correctly what that does.
>>>>>>>>>> Under what
>>>>>>>>>> >> conditions is the deprecated encode/decode method used?
>>>>>>>>>> > Yes, it's the UNNESTED one.
>>>>>>>>>> >
>>>>>>>>>> >> [1]
>>>>>>>>>> >>
>>>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>>>>>>>> >>
>>>>>>>>>> >>>> The runner's responsibility is not to go from "<nested
>>>>>>>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for
>>>>>>>>>> arbitrary coder.
>>>>>>>>>> >>>> That is really impossible. But a coder is a function, right?
>>>>>>>>>> Function
>>>>>>>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>>>>>>>> provides a
>>>>>>>>>> >>>> set of elements, and these elements are the "universe". For
>>>>>>>>>> those
>>>>>>>>>> >>>> elements it also provides the encoded form, which can be
>>>>>>>>>> interpreted as
>>>>>>>>>> >>>> the definition of the coder.
>>>>>>>>>> >>> The problem here is that there is not "the encoded form" for
>>>>>>>>>> a Coder
>>>>>>>>>> >>> but two encoded forms, and we have the wrong one. Things
>>>>>>>>>> could be made
>>>>>>>>>> >>> to work if we had the other.
>>>>>>>>>> >> Which two encoded forms do you refer to? Elements encoded by
>>>>>>>>>> both the
>>>>>>>>>> >> SDK-coder and runner-coder (and I ignore the Context here once
>>>>>>>>>> again)
>>>>>>>>>> >> have the same binary representation (which they must have,
>>>>>>>>>> otherwise it
>>>>>>>>>> >> would be impossible to decode elements coming from the runner
>>>>>>>>>> to the
>>>>>>>>>> >> SDK-harness or vice-versa).
>>>>>>>>>> >>>> Therefore - technically (and formally) -
>>>>>>>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>>>>>>>> regardless of
>>>>>>>>>> >>>> the language the runner is written in.
>>>>>>>>>> >>>>
>>>>>>>>>> >>>> To move  this discussion forward, I think we should look for
>>>>>>>>>> answers to
>>>>>>>>>> >>>> the following questions:
>>>>>>>>>> >>>>
>>>>>>>>>> >>>>     a) do we have any clues that show, that the proposed "in
>>>>>>>>>> runner"
>>>>>>>>>> >>>> solution will not work?
>>>>>>>>>> >>> OK, thinking about it some more, in the TestStream, we can
>>>>>>>>>> use the
>>>>>>>>>> >>> happy coincidence that
>>>>>>>>>> >>>
>>>>>>>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>>>>>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>>>>>>>> >>> nested=False)
>>>>>>>>>> >>>
>>>>>>>>>> >>> (where || denotes concatenation) and the fact that we have
>>>>>>>>>> >>>
>>>>>>>>>> >>>       C.encode(x, nested=False)
>>>>>>>>>> >>>
>>>>>>>>>> >>> in hand.
>>>>>>>>>> >>>
>>>>>>>>>> >>> A possible fix here for the OP's question is that when
>>>>>>>>>> rehydrating the
>>>>>>>>>> >>> TestStream transform it must behave differently according to
>>>>>>>>>> the coder
>>>>>>>>>> >>> used in the subsequent channel (e.g. for known coders, it
>>>>>>>>>> decodes the
>>>>>>>>>> >>> elements and emits them directly, but for unknown coders, it
>>>>>>>>>> prefixes
>>>>>>>>>> >>> them with their length and emits byte strings. It gets more
>>>>>>>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>>>>>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>>>>>>>> unknown-coder))
>>>>>>>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different
>>>>>>>>>> encodings
>>>>>>>>>> >>> (and the latter, which is the default, requires transcoding
>>>>>>>>>> the bytes
>>>>>>>>>> >>> to inject the length in the middle which is found by decoding
>>>>>>>>>> the
>>>>>>>>>> >>> first component). As well as getting more complex, this
>>>>>>>>>> really seems
>>>>>>>>>> >>> to violate the spirit of separation of concerns.
>>>>>>>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>>>>>>>> >> KV<LP<unknown>, known>? From my understanding it is always the
>>>>>>>>>> latter,
>>>>>>>>>> >> because of [2].
>>>>>>>>>> >>
>>>>>>>>>> >> [2]
>>>>>>>>>> >>
>>>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>>>>> > It is currently the latter for runners using this code (which
>>>>>>>>>> not all
>>>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>>>> >
>>>>>>>>>> >>>>     b) do we think, that it will not be robust enough to
>>>>>>>>>> incorporate the
>>>>>>>>>> >>>> other use-cases (line generic transform inlining, taking
>>>>>>>>>> into account
>>>>>>>>>> >>>> that this applies only to runners that are written in the
>>>>>>>>>> same language
>>>>>>>>>> >>>> as the submitting SDK, because otherwise, there is nothing
>>>>>>>>>> to inline)?
>>>>>>>>>> >>> Being in the same language is not a prerequisite to
>>>>>>>>>> "inlining," e.g.
>>>>>>>>>> >>> the PubSub source on Dataflow is recognized as such and not
>>>>>>>>>> executed
>>>>>>>>>> >>> as SDK code but natively.
>>>>>>>>>> >> Agree, that is actually exactly what happens with the
>>>>>>>>>> TestStream. The
>>>>>>>>>> >> transform need not be in the same language, as long as it is
>>>>>>>>>> completely
>>>>>>>>>> >> understood by the runner, including the SDK-coder (either
>>>>>>>>>> explicitly -
>>>>>>>>>> >> which might be due to the PCollection coder being composed of
>>>>>>>>>> well-known
>>>>>>>>>> >> coders only, or implicitly like in the case of TestStream,
>>>>>>>>>> where the
>>>>>>>>>> >> elements are encoded using the SDK coder.
>>>>>>>>>> >>> It is more likely that inlining occurs in the same language
>>>>>>>>>> if there
>>>>>>>>>> >>> are UDFs involved.
>>>>>>>>>> >>>
>>>>>>>>>> >>>> I'm convinced, that the TestStream-decode expansion solution
>>>>>>>>>> is an
>>>>>>>>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>>>>>>>>> bothering
>>>>>>>>>> >>>> this mailing list with my emails on this. :-)
>>>>>>>>>> >>>>
>>>>>>>>>> >>>> WDYT?
>>>>>>>>>> >>> While not a solution to the general problem, I think the
>>>>>>>>>> >>> TestStream-only-does-bytes simplifies its definition
>>>>>>>>>> (primitives
>>>>>>>>>> >>> should have as simple/easy to implement definitions as
>>>>>>>>>> possible) and
>>>>>>>>>> >>> brings it closer to the other root we have: Impulse. (We
>>>>>>>>>> could go a
>>>>>>>>>> >>> step further and rather than emitting encoded elements, with
>>>>>>>>>> the data
>>>>>>>>>> >>> in the proto itself, it emits sequence numbers, and a
>>>>>>>>>> subsequent ParDo
>>>>>>>>>> >>> maps those to concrete elements (e.g. via an in-memory map),
>>>>>>>>>> but that
>>>>>>>>>> >>> further step doesn't buy much...)
>>>>>>>>>> >>>
>>>>>>>>>> >>> Only runners that want to do inlining would have to take on
>>>>>>>>>> the
>>>>>>>>>> >>> complexity of a fully generic solution.
>>>>>>>>>> >> I think that if the simplification brings something, we can do
>>>>>>>>>> that, but
>>>>>>>>>> >> I'd like to understand why we cannot (or should not) use the
>>>>>>>>>> generic
>>>>>>>>>> >> solution. I think it definitely *should* be possible to use a
>>>>>>>>>> generic
>>>>>>>>>> >> solution, because otherwise the solution would not be generic.
>>>>>>>>>> And it
>>>>>>>>>> >> would imply, that we are unable to do generic transform
>>>>>>>>>> inlining, which
>>>>>>>>>> >> I would find really strange. That would immediately mean, that
>>>>>>>>>> we are
>>>>>>>>>> >> unable to construct classical runner as a special case of the
>>>>>>>>>> portable
>>>>>>>>>> >> one, which would be bad I think.
>>>>>>>>>> >>
>>>>>>>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>>>>>>>> SDK-coder,
>>>>>>>>>> >> or does this go through the LengthPrefixUnknownCoders logic?
>>>>>>>>>> If not,
>>>>>>>>>> >> then the problem would be there, because that means, that the
>>>>>>>>>> SDK-coder
>>>>>>>>>> >> cannot be (implicitly) defined in the runner. If the elements
>>>>>>>>>> would be
>>>>>>>>>> >> encoded using LP, then it would be possible to decode them
>>>>>>>>>> using
>>>>>>>>>> >> runner-coder and the problem should be solved, or am I still
>>>>>>>>>> missing
>>>>>>>>>> >> some key parts?
>>>>>>>>>> > Yes, the problem is precisely that there are (unspecified)
>>>>>>>>>> constraints
>>>>>>>>>> > on the coder used by the TestStreamPayload. Just requiring that
>>>>>>>>>> it be
>>>>>>>>>> > length prefixed is not enough, you have to make constraints on
>>>>>>>>>> > sometimes pushing down the length prefixing if it's a composite
>>>>>>>>>> (like
>>>>>>>>>> > a KV) that depend on what the runner is expected to support in
>>>>>>>>>> terms
>>>>>>>>>> > of composites and/or the choices it chooses for the channel
>>>>>>>>>> (and the
>>>>>>>>>> > runner, not knowing the coder, can't transcode between these
>>>>>>>>>> choices).
>>>>>>>>>> >
>>>>>>>>>> > The simpler solution is to constrain this coder to just be
>>>>>>>>>> byte[]
>>>>>>>>>> > rather than let it be a little bit flexible (but not wholly
>>>>>>>>>> flexible).
>>>>>>>>>> >
>>>>>>>>>> > As for a fully generic solution, I think the issue encountered
>>>>>>>>>> with
>>>>>>>>>> > inlining Read vs. TestStream are related to this, but not
>>>>>>>>>> really the
>>>>>>>>>> > same. With TestStream one has an encoded representation of the
>>>>>>>>>> > elements provided by the SDK that the Runner and has no SDK
>>>>>>>>>> > representation/execution whereas with the Reads one has
>>>>>>>>>> unencoded
>>>>>>>>>> > elements in hand and a Coder that is understood by both (so
>>>>>>>>>> long as
>>>>>>>>>> > the channel can be negotiated correctly). FWIW, I think the
>>>>>>>>>> proper
>>>>>>>>>> > solution to inlining a Read (or other Transform that would
>>>>>>>>>> typically
>>>>>>>>>> > be executed in the SDK) is to treat it as a special environment
>>>>>>>>>> (where
>>>>>>>>>> > we know logically it can work) and then elide, as possible, the
>>>>>>>>>> > various encodings, grpc calls, etc. that are unneeded as
>>>>>>>>>> everything is
>>>>>>>>>> > in process.
>>>>>>>>>> >
>>>>>>>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>>>>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>>>> wrote:
>>>>>>>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>>>>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<
>>>>>>>>>> je.ik@seznam.cz>  wrote:
>>>>>>>>>> >>>>>>>> Hi,
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>> I had some more time thinking about this and I'll try to
>>>>>>>>>> recap that.
>>>>>>>>>> >>>>>>>> First some invariants:
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>>       a) each PCollection<T> has actually two coders -
>>>>>>>>>> an _SDK coder_ and a
>>>>>>>>>> >>>>>>>> _runner coder_. These coders have the property, that
>>>>>>>>>> each one can
>>>>>>>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not
>>>>>>>>>> true, the
>>>>>>>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in
>>>>>>>>>> general).
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>>>>>>>> environment, the
>>>>>>>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>>>>>>>> SDK-harness and
>>>>>>>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>>       c) under specific circumstances, the encode-decode
>>>>>>>>>> step can be
>>>>>>>>>> >>>>>>>> optimized out, that is the case where the SDK coder and
>>>>>>>>>> all its
>>>>>>>>>> >>>>>>>> subcoders are all well-known to the runner (in the
>>>>>>>>>> present, that means
>>>>>>>>>> >>>>>>>> that all the parts present in the model coders set). The
>>>>>>>>>> reason for that
>>>>>>>>>> >>>>>>>> is that in this specific situation
>>>>>>>>>> runner_decode(sdk_encode(X)) = X.
>>>>>>>>>> >>>>>>>> This property is essential.
>>>>>>>>>> >>>>>>> However, in general, X can only pass from the SDK to the
>>>>>>>>>> runner (or
>>>>>>>>>> >>>>>>> vice versa) in encoded form.
>>>>>>>>>> >>>>>> In general yes, but we are (mostly) talking transform
>>>>>>>>>> inlining here, so
>>>>>>>>>> >>>>>> it that particular situation, the elements might be passed
>>>>>>>>>> in decoded form.
>>>>>>>>>> >>>>>>>>       d) from b) immediately follows, that when a
>>>>>>>>>> PTransform does not run in
>>>>>>>>>> >>>>>>>> an environment (and this might be due to the transform
>>>>>>>>>> being runner
>>>>>>>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream))
>>>>>>>>>> the elements have
>>>>>>>>>> >>>>>>>> to be encoded by SDK coder, immediately following decode
>>>>>>>>>> by runner
>>>>>>>>>> >>>>>>>> coder. That (surprisingly) applies even to situations
>>>>>>>>>> when runner is
>>>>>>>>>> >>>>>>>> implemented using different language than the client
>>>>>>>>>> SDK, because it
>>>>>>>>>> >>>>>>>> implies that the type of produced elements must be one
>>>>>>>>>> of types encoded
>>>>>>>>>> >>>>>>>> using model coders (well-known to the runner, otherwise
>>>>>>>>>> the SDK will not
>>>>>>>>>> >>>>>>>> be able to consume it). But - due to property c) - this
>>>>>>>>>> means that this
>>>>>>>>>> >>>>>>>> encode-decode step can be optimized out. This does not
>>>>>>>>>> mean that it is
>>>>>>>>>> >>>>>>>> not (logically) present, though. This is exactly the
>>>>>>>>>> case of native
>>>>>>>>>> >>>>>>>> Impulse transform.
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>> Now, from that we can conclude that on the boundary
>>>>>>>>>> between executable
>>>>>>>>>> >>>>>>>> stages, or between runner (inlined) transform and
>>>>>>>>>> executable stage, each
>>>>>>>>>> >>>>>>>> PCollection has to be encoded using SDK coder and
>>>>>>>>>> immediately decoded by
>>>>>>>>>> >>>>>>>> runner coder, *unless this can be optimized out* by
>>>>>>>>>> property c).
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>> This gives us two options where to implement this
>>>>>>>>>> encode/decode step:
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>>       1) completely inside runner with the possibility
>>>>>>>>>> to optimize the
>>>>>>>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>>       2) partly in the runner and partly in the SDK -
>>>>>>>>>> that is we encode
>>>>>>>>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass
>>>>>>>>>> those to the
>>>>>>>>>> >>>>>>>> SDK harness and apply a custom decode step there. This
>>>>>>>>>> works because SDK
>>>>>>>>>> >>>>>>>> coder encoded elements are in byte[], and that is
>>>>>>>>>> well-known coder type.
>>>>>>>>>> >>>>>>>> We again only leverage property c) and optimize the SDK
>>>>>>>>>> coder encode,
>>>>>>>>>> >>>>>>>> runner decode step out.
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream
>>>>>>>>>> producing byte[] and
>>>>>>>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>>>>>>>>>> inlined
>>>>>>>>>> >>>>>>>> transform, the elements are produced directly in runner
>>>>>>>>>> (the SDK coder
>>>>>>>>>> >>>>>>>> is not known to the runner, but that does not matter,
>>>>>>>>>> because the
>>>>>>>>>> >>>>>>>> elements are already encoded by client).
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>>      From the above it seems to me, that option 1)
>>>>>>>>>> should be preferred, because:
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>>       i) it is generic, applicable to all inlined
>>>>>>>>>> transforms, any sources
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>>       ii) it is consistent with how things logically
>>>>>>>>>> work underneath
>>>>>>>>>> >>>>>>>>
>>>>>>>>>> >>>>>>>>       iii) it offers better room for optimization -
>>>>>>>>>> option 2) might result
>>>>>>>>>> >>>>>>>> in cases when the elements are passed from the runner to
>>>>>>>>>> the SDK-harness
>>>>>>>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>>>>>>>> immediately
>>>>>>>>>> >>>>>>>> encoding back using SDK-coder and returned back to the
>>>>>>>>>> runner. This
>>>>>>>>>> >>>>>>>> would be the case when TestStream would be directly
>>>>>>>>>> consumed by inlined
>>>>>>>>>> >>>>>>>> (or external) transform.
>>>>>>>>>> >>>>>>> (1) is not possible if the Coder in question is not known
>>>>>>>>>> to the
>>>>>>>>>> >>>>>>> Runner, which is why I proposed (2).
>>>>>>>>>> >>>>>> There is no particular need for the coder to be known. If
>>>>>>>>>> transform is
>>>>>>>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded
>>>>>>>>>> form of the
>>>>>>>>>> >>>>>> elements. That holds true if:
>>>>>>>>>> >>>>>>
>>>>>>>>>> >>>>>>      a) either the SDK coder is known, or
>>>>>>>>>> >>>>>>
>>>>>>>>>> >>>>>>      b) encoded form of the produced elements is known in
>>>>>>>>>> advance
>>>>>>>>>> >>>>>>
>>>>>>>>>> >>>>>> For TestStream it is the case b). For inlined primitive
>>>>>>>>>> Read (or any
>>>>>>>>>> >>>>>> other transform which executes code) it is a).
>>>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>>>> reasons,
>>>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>>>> delimiting)
>>>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>>>> encoded bytes,
>>>>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>>>>> SDK. The
>>>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested
>>>>>>>>>> encoding> for an
>>>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>>>> >>>>>
>>>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to
>>>>>>>>>> send already
>>>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>>>> complicate the
>>>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>>>> sending the
>>>>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>>>>> solved
>>>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>>>> known by the
>>>>>>>>>> >>>>> runner).)
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>>
>>>>>>>>
>>>>>>>> Jack McCluskey
>>>>>>>> SWE - DataPLS PLAT/ Beam Go
>>>>>>>> RDU
>>>>>>>> jrmccluskey@gmail.com
>>>>>>>>
>>>>>>>>
>>>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Burke <ro...@frantil.com>.
IIUC that means that for TestStream a runner should:
* Look at the coder being used in TestStream
* Look at the list of SDK coder capabilities.
* If the test stream coder is built of known coder capabilities, then it
should not try to LP the provided encoded bytes, but instead simply window
wrap them?

Is that right?

On Thu, Sep 9, 2021, 12:58 PM Robert Bradshaw <ro...@google.com> wrote:

> Go doesn't have a (portable, cross-language) runner. It's fine for SDKs to
> enumerate the coders they understand. Runners, however, should respect what
> SDKs declare.
>
> On Thu, Sep 9, 2021 at 12:54 PM Jack McCluskey <jr...@google.com>
> wrote:
>
>> Go also doesn't do this, instead maintaining a list of known standard
>> coders and using that as a list of capabilities:
>> https://github.com/apache/beam/blob/4bc870806099f03265e7dfb48b142f00cee42f47/sdks/go/pkg/beam/core/runtime/graphx/coder.go#L59
>>
>> On Thu, Sep 9, 2021 at 1:29 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> You are right, Java does not implement this correctly. It should be
>>> querying the capabilities section of the environment proto. (For java
>>> environments, this is populated from ModelCoders, e.g.
>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386
>>> )
>>>
>>> Looks like Python doesn't do any better:
>>> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459
>>>
>>>
>>>
>>> On Thu, Sep 9, 2021 at 10:13 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>
>>>> This makes a *lot* of sense. But it seems to me, that this is not the
>>>> way Java-based runners - that use runners-core-construction-java module -
>>>> handle it. If I interpret it correctly, then the set of ModelCoders is
>>>> hard-coded [1] and essentially required to be known by all SDKs [2].
>>>>
>>>> There seems to be no negotiation between what SDK harness knows and
>>>> what the runner knows. The runner might be able to define the wire coder
>>>> for the SDK (via the ProcessBundleDescriptor), but the SDK (for Java
>>>> runners) seems not to be able to play any role in this [3]. Therefore I
>>>> think that if an SDK does not know the set of Java ModelCoders, then the
>>>> runner and the SDK might not agree on the binary encoding of the elements
>>>> (BTW, which is why Java Coders cannot be part of ModelCoders and I finally
>>>> understand why I had such troubles adding it there - it cannot be there).
>>>>
>>>> Is it possible we are missing some part of this runner-to-sdk coder
>>>> negotiation in runners-core-contruction-java?
>>>>
>>>> [1]
>>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75
>>>>
>>>> [2]
>>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62
>>>>
>>>> [3]
>>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104
>>>> On 9/9/21 12:18 AM, Robert Bradshaw wrote:
>>>>
>>>> The whole notion of an absolute set of known coders is a misnomer. It
>>>> would require all Runners and SDKs to be updated synchronously for every
>>>> new coder they might want to share.
>>>>
>>>> Instead, what we have are
>>>>
>>>> * Standard Coders which have well-defined, language-agnostic
>>>> representations and encodings, which may be used for interoperability and
>>>> efficiency, and
>>>> * Required Coders which are the minimum needed to execute the pipeline.
>>>>
>>>> The latter consists only of bytes (for impulse), kv and iterable (for
>>>> GBK), windowed value (for windowing information) and length prefix (to be
>>>> able to handle anything else).
>>>>
>>>>
>>>> On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <ro...@frantil.com> wrote:
>>>>
>>>>> Is the claim that the Standard bytes and String_utf8 coders are not
>>>>> "known coders"?
>>>>>
>>>>>  What's the point of the standard coders if they are not the canonical
>>>>> "known coders" that can generally be expected to be known by runners/other
>>>>> SDKs?
>>>>>
>>>>>
>>>>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>>>>>
>>>>> The Go SDK rather heavily biases towards using the standard coders for
>>>>> their closes language equivalents rather than going into override/custom
>>>>> specified soup. (It's not possible to globally override the coders for the
>>>>> '[]byte' and 'string' types, nor is there often reason to.)
>>>>>
>>>>> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <jr...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hey all,
>>>>>>>
>>>>>>> Just catching up on the thread since I did the TestStream Go SDK
>>>>>>> implementation. The discussion about length prefixing behavior for known
>>>>>>> vs. unknown coders is interesting, since we ran into strings and byte
>>>>>>> slices getting extra length prefixes attached to them by Flink despite
>>>>>>> being known coders.
>>>>>>>
>>>>>>
>>>>>> Known to who?
>>>>>>
>>>>>>
>>>>>>> Based on what's been said, that isn't expected behavior, right?
>>>>>>>
>>>>>>
>>>>>> No, it's not.
>>>>>>
>>>>>> I would check to make sure the Go SDK is respecting the Coder (length
>>>>>> prefixed or not) that's set on the channel, rather than guessing at what it
>>>>>> expects it to be based on the Go type.
>>>>>>
>>>>>>
>>>>>>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>>
>>>>>>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>>>>>>
>>>>>>>> Historically the DataflowRunner has been much more careful about
>>>>>>>> not breaking update, since this is a frequent operation by Dataflow users.
>>>>>>>> I think we've been less careful aboutt other runners, but as we see clearly
>>>>>>>> here Fllnk users do care about this as well, so we should probably test
>>>>>>>> upgrade compatibility for Flink.
>>>>>>>>
>>>>>>>> One strategy that Dataflow uses is to avoid embedding the Java
>>>>>>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>>>>>>> breakage (as we see with the issue you llnked to). Possibly similar
>>>>>>>> strategies should be investigated for Fllink.
>>>>>>>>
>>>>>>>> +1, that would be great!
>>>>>>>>
>>>>>>>>
>>>>>>>> Reuven
>>>>>>>>
>>>>>>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>>> iterable)
>>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>>> > changing coders is hard (due to no way to update the encoding
>>>>>>>>> for a
>>>>>>>>> > streaming pipeline).
>>>>>>>>> This is unrelated, but - regarding changing coders due to concerns
>>>>>>>>> about
>>>>>>>>> pipeline upgrades, we break this quite often, at least for some
>>>>>>>>> runners.
>>>>>>>>> Most recently [1].
>>>>>>>>>
>>>>>>>>> > It is currently the latter for runners using this code (which
>>>>>>>>> not all
>>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>>> This is interesting and unexpected for me. How do runners decide
>>>>>>>>> about
>>>>>>>>> how they encode elements between SDK harness and the runner? How
>>>>>>>>> do they
>>>>>>>>> inform the SDK harness about this decision? My impression was that
>>>>>>>>> this
>>>>>>>>> is well-defined at the model level. If not, then we have the
>>>>>>>>> reason for
>>>>>>>>> misunderstanding in this conversation. :-)
>>>>>>>>>
>>>>>>>>>   Jan
>>>>>>>>>
>>>>>>>>> [1]
>>>>>>>>>
>>>>>>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>>>>>>
>>>>>>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>>>>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz>
>>>>>>>>> wrote:
>>>>>>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>>>>>>> >>
>>>>>>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>>> wrote:
>>>>>>>>> >>>> Hi Robert,
>>>>>>>>> >>>>
>>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>>> reasons,
>>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>>> delimiting)
>>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>>> encoded bytes,
>>>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>>>> SDK. The
>>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested
>>>>>>>>> encoding> for an
>>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>>> >>>>>
>>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to
>>>>>>>>> send already
>>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>>> complicate the
>>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>>> sending the
>>>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>>>> solved
>>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>>> known by the
>>>>>>>>> >>>>> runner).)
>>>>>>>>> >>>> It is hard for me to argue with "historical reasons". But -
>>>>>>>>> the "nested"
>>>>>>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>>>>>>> runner-coder
>>>>>>>>> >>>> spaces.
>>>>>>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>>>>>>> >> Hm, do you mean the Context passed to the encode/decode method?
>>>>>>>>> [1] That
>>>>>>>>> >> seems to be deprecated, I assume that most coders use the
>>>>>>>>> default
>>>>>>>>> >> implementation and simply ignore the Context?
>>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>>> iterable)
>>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>>> > changing coders is hard (due to no way to update the encoding
>>>>>>>>> for a
>>>>>>>>> > streaming pipeline).
>>>>>>>>> >
>>>>>>>>> >> Even if not - whether or
>>>>>>>>> >> not the elements are encoded using NESTED Context or UNNESTED
>>>>>>>>> Context
>>>>>>>>> >> should be part of the contract of TestStream, right? Most
>>>>>>>>> likely it is
>>>>>>>>> >> the UNNESTED one, if I understand correctly what that does.
>>>>>>>>> Under what
>>>>>>>>> >> conditions is the deprecated encode/decode method used?
>>>>>>>>> > Yes, it's the UNNESTED one.
>>>>>>>>> >
>>>>>>>>> >> [1]
>>>>>>>>> >>
>>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>>>>>>> >>
>>>>>>>>> >>>> The runner's responsibility is not to go from "<nested
>>>>>>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary
>>>>>>>>> coder.
>>>>>>>>> >>>> That is really impossible. But a coder is a function, right?
>>>>>>>>> Function
>>>>>>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>>>>>>> provides a
>>>>>>>>> >>>> set of elements, and these elements are the "universe". For
>>>>>>>>> those
>>>>>>>>> >>>> elements it also provides the encoded form, which can be
>>>>>>>>> interpreted as
>>>>>>>>> >>>> the definition of the coder.
>>>>>>>>> >>> The problem here is that there is not "the encoded form" for a
>>>>>>>>> Coder
>>>>>>>>> >>> but two encoded forms, and we have the wrong one. Things could
>>>>>>>>> be made
>>>>>>>>> >>> to work if we had the other.
>>>>>>>>> >> Which two encoded forms do you refer to? Elements encoded by
>>>>>>>>> both the
>>>>>>>>> >> SDK-coder and runner-coder (and I ignore the Context here once
>>>>>>>>> again)
>>>>>>>>> >> have the same binary representation (which they must have,
>>>>>>>>> otherwise it
>>>>>>>>> >> would be impossible to decode elements coming from the runner
>>>>>>>>> to the
>>>>>>>>> >> SDK-harness or vice-versa).
>>>>>>>>> >>>> Therefore - technically (and formally) -
>>>>>>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>>>>>>> regardless of
>>>>>>>>> >>>> the language the runner is written in.
>>>>>>>>> >>>>
>>>>>>>>> >>>> To move  this discussion forward, I think we should look for
>>>>>>>>> answers to
>>>>>>>>> >>>> the following questions:
>>>>>>>>> >>>>
>>>>>>>>> >>>>     a) do we have any clues that show, that the proposed "in
>>>>>>>>> runner"
>>>>>>>>> >>>> solution will not work?
>>>>>>>>> >>> OK, thinking about it some more, in the TestStream, we can use
>>>>>>>>> the
>>>>>>>>> >>> happy coincidence that
>>>>>>>>> >>>
>>>>>>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>>>>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>>>>>>> >>> nested=False)
>>>>>>>>> >>>
>>>>>>>>> >>> (where || denotes concatenation) and the fact that we have
>>>>>>>>> >>>
>>>>>>>>> >>>       C.encode(x, nested=False)
>>>>>>>>> >>>
>>>>>>>>> >>> in hand.
>>>>>>>>> >>>
>>>>>>>>> >>> A possible fix here for the OP's question is that when
>>>>>>>>> rehydrating the
>>>>>>>>> >>> TestStream transform it must behave differently according to
>>>>>>>>> the coder
>>>>>>>>> >>> used in the subsequent channel (e.g. for known coders, it
>>>>>>>>> decodes the
>>>>>>>>> >>> elements and emits them directly, but for unknown coders, it
>>>>>>>>> prefixes
>>>>>>>>> >>> them with their length and emits byte strings. It gets more
>>>>>>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>>>>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>>>>>>> unknown-coder))
>>>>>>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different
>>>>>>>>> encodings
>>>>>>>>> >>> (and the latter, which is the default, requires transcoding
>>>>>>>>> the bytes
>>>>>>>>> >>> to inject the length in the middle which is found by decoding
>>>>>>>>> the
>>>>>>>>> >>> first component). As well as getting more complex, this really
>>>>>>>>> seems
>>>>>>>>> >>> to violate the spirit of separation of concerns.
>>>>>>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>>>>>>> >> KV<LP<unknown>, known>? From my understanding it is always the
>>>>>>>>> latter,
>>>>>>>>> >> because of [2].
>>>>>>>>> >>
>>>>>>>>> >> [2]
>>>>>>>>> >>
>>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>>>> > It is currently the latter for runners using this code (which
>>>>>>>>> not all
>>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>>> >
>>>>>>>>> >>>>     b) do we think, that it will not be robust enough to
>>>>>>>>> incorporate the
>>>>>>>>> >>>> other use-cases (line generic transform inlining, taking into
>>>>>>>>> account
>>>>>>>>> >>>> that this applies only to runners that are written in the
>>>>>>>>> same language
>>>>>>>>> >>>> as the submitting SDK, because otherwise, there is nothing to
>>>>>>>>> inline)?
>>>>>>>>> >>> Being in the same language is not a prerequisite to
>>>>>>>>> "inlining," e.g.
>>>>>>>>> >>> the PubSub source on Dataflow is recognized as such and not
>>>>>>>>> executed
>>>>>>>>> >>> as SDK code but natively.
>>>>>>>>> >> Agree, that is actually exactly what happens with the
>>>>>>>>> TestStream. The
>>>>>>>>> >> transform need not be in the same language, as long as it is
>>>>>>>>> completely
>>>>>>>>> >> understood by the runner, including the SDK-coder (either
>>>>>>>>> explicitly -
>>>>>>>>> >> which might be due to the PCollection coder being composed of
>>>>>>>>> well-known
>>>>>>>>> >> coders only, or implicitly like in the case of TestStream,
>>>>>>>>> where the
>>>>>>>>> >> elements are encoded using the SDK coder.
>>>>>>>>> >>> It is more likely that inlining occurs in the same language if
>>>>>>>>> there
>>>>>>>>> >>> are UDFs involved.
>>>>>>>>> >>>
>>>>>>>>> >>>> I'm convinced, that the TestStream-decode expansion solution
>>>>>>>>> is an
>>>>>>>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>>>>>>>> bothering
>>>>>>>>> >>>> this mailing list with my emails on this. :-)
>>>>>>>>> >>>>
>>>>>>>>> >>>> WDYT?
>>>>>>>>> >>> While not a solution to the general problem, I think the
>>>>>>>>> >>> TestStream-only-does-bytes simplifies its definition
>>>>>>>>> (primitives
>>>>>>>>> >>> should have as simple/easy to implement definitions as
>>>>>>>>> possible) and
>>>>>>>>> >>> brings it closer to the other root we have: Impulse. (We could
>>>>>>>>> go a
>>>>>>>>> >>> step further and rather than emitting encoded elements, with
>>>>>>>>> the data
>>>>>>>>> >>> in the proto itself, it emits sequence numbers, and a
>>>>>>>>> subsequent ParDo
>>>>>>>>> >>> maps those to concrete elements (e.g. via an in-memory map),
>>>>>>>>> but that
>>>>>>>>> >>> further step doesn't buy much...)
>>>>>>>>> >>>
>>>>>>>>> >>> Only runners that want to do inlining would have to take on the
>>>>>>>>> >>> complexity of a fully generic solution.
>>>>>>>>> >> I think that if the simplification brings something, we can do
>>>>>>>>> that, but
>>>>>>>>> >> I'd like to understand why we cannot (or should not) use the
>>>>>>>>> generic
>>>>>>>>> >> solution. I think it definitely *should* be possible to use a
>>>>>>>>> generic
>>>>>>>>> >> solution, because otherwise the solution would not be generic.
>>>>>>>>> And it
>>>>>>>>> >> would imply, that we are unable to do generic transform
>>>>>>>>> inlining, which
>>>>>>>>> >> I would find really strange. That would immediately mean, that
>>>>>>>>> we are
>>>>>>>>> >> unable to construct classical runner as a special case of the
>>>>>>>>> portable
>>>>>>>>> >> one, which would be bad I think.
>>>>>>>>> >>
>>>>>>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>>>>>>> SDK-coder,
>>>>>>>>> >> or does this go through the LengthPrefixUnknownCoders logic? If
>>>>>>>>> not,
>>>>>>>>> >> then the problem would be there, because that means, that the
>>>>>>>>> SDK-coder
>>>>>>>>> >> cannot be (implicitly) defined in the runner. If the elements
>>>>>>>>> would be
>>>>>>>>> >> encoded using LP, then it would be possible to decode them using
>>>>>>>>> >> runner-coder and the problem should be solved, or am I still
>>>>>>>>> missing
>>>>>>>>> >> some key parts?
>>>>>>>>> > Yes, the problem is precisely that there are (unspecified)
>>>>>>>>> constraints
>>>>>>>>> > on the coder used by the TestStreamPayload. Just requiring that
>>>>>>>>> it be
>>>>>>>>> > length prefixed is not enough, you have to make constraints on
>>>>>>>>> > sometimes pushing down the length prefixing if it's a composite
>>>>>>>>> (like
>>>>>>>>> > a KV) that depend on what the runner is expected to support in
>>>>>>>>> terms
>>>>>>>>> > of composites and/or the choices it chooses for the channel (and
>>>>>>>>> the
>>>>>>>>> > runner, not knowing the coder, can't transcode between these
>>>>>>>>> choices).
>>>>>>>>> >
>>>>>>>>> > The simpler solution is to constrain this coder to just be byte[]
>>>>>>>>> > rather than let it be a little bit flexible (but not wholly
>>>>>>>>> flexible).
>>>>>>>>> >
>>>>>>>>> > As for a fully generic solution, I think the issue encountered
>>>>>>>>> with
>>>>>>>>> > inlining Read vs. TestStream are related to this, but not really
>>>>>>>>> the
>>>>>>>>> > same. With TestStream one has an encoded representation of the
>>>>>>>>> > elements provided by the SDK that the Runner and has no SDK
>>>>>>>>> > representation/execution whereas with the Reads one has unencoded
>>>>>>>>> > elements in hand and a Coder that is understood by both (so long
>>>>>>>>> as
>>>>>>>>> > the channel can be negotiated correctly). FWIW, I think the
>>>>>>>>> proper
>>>>>>>>> > solution to inlining a Read (or other Transform that would
>>>>>>>>> typically
>>>>>>>>> > be executed in the SDK) is to treat it as a special environment
>>>>>>>>> (where
>>>>>>>>> > we know logically it can work) and then elide, as possible, the
>>>>>>>>> > various encodings, grpc calls, etc. that are unneeded as
>>>>>>>>> everything is
>>>>>>>>> > in process.
>>>>>>>>> >
>>>>>>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>>>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>>> wrote:
>>>>>>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>>>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<
>>>>>>>>> je.ik@seznam.cz>  wrote:
>>>>>>>>> >>>>>>>> Hi,
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>> I had some more time thinking about this and I'll try to
>>>>>>>>> recap that.
>>>>>>>>> >>>>>>>> First some invariants:
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>>       a) each PCollection<T> has actually two coders - an
>>>>>>>>> _SDK coder_ and a
>>>>>>>>> >>>>>>>> _runner coder_. These coders have the property, that each
>>>>>>>>> one can
>>>>>>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not
>>>>>>>>> true, the
>>>>>>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in
>>>>>>>>> general).
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>>>>>>> environment, the
>>>>>>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>>>>>>> SDK-harness and
>>>>>>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>>       c) under specific circumstances, the encode-decode
>>>>>>>>> step can be
>>>>>>>>> >>>>>>>> optimized out, that is the case where the SDK coder and
>>>>>>>>> all its
>>>>>>>>> >>>>>>>> subcoders are all well-known to the runner (in the
>>>>>>>>> present, that means
>>>>>>>>> >>>>>>>> that all the parts present in the model coders set). The
>>>>>>>>> reason for that
>>>>>>>>> >>>>>>>> is that in this specific situation
>>>>>>>>> runner_decode(sdk_encode(X)) = X.
>>>>>>>>> >>>>>>>> This property is essential.
>>>>>>>>> >>>>>>> However, in general, X can only pass from the SDK to the
>>>>>>>>> runner (or
>>>>>>>>> >>>>>>> vice versa) in encoded form.
>>>>>>>>> >>>>>> In general yes, but we are (mostly) talking transform
>>>>>>>>> inlining here, so
>>>>>>>>> >>>>>> it that particular situation, the elements might be passed
>>>>>>>>> in decoded form.
>>>>>>>>> >>>>>>>>       d) from b) immediately follows, that when a
>>>>>>>>> PTransform does not run in
>>>>>>>>> >>>>>>>> an environment (and this might be due to the transform
>>>>>>>>> being runner
>>>>>>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>>>>>>>>> elements have
>>>>>>>>> >>>>>>>> to be encoded by SDK coder, immediately following decode
>>>>>>>>> by runner
>>>>>>>>> >>>>>>>> coder. That (surprisingly) applies even to situations
>>>>>>>>> when runner is
>>>>>>>>> >>>>>>>> implemented using different language than the client SDK,
>>>>>>>>> because it
>>>>>>>>> >>>>>>>> implies that the type of produced elements must be one of
>>>>>>>>> types encoded
>>>>>>>>> >>>>>>>> using model coders (well-known to the runner, otherwise
>>>>>>>>> the SDK will not
>>>>>>>>> >>>>>>>> be able to consume it). But - due to property c) - this
>>>>>>>>> means that this
>>>>>>>>> >>>>>>>> encode-decode step can be optimized out. This does not
>>>>>>>>> mean that it is
>>>>>>>>> >>>>>>>> not (logically) present, though. This is exactly the case
>>>>>>>>> of native
>>>>>>>>> >>>>>>>> Impulse transform.
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>> Now, from that we can conclude that on the boundary
>>>>>>>>> between executable
>>>>>>>>> >>>>>>>> stages, or between runner (inlined) transform and
>>>>>>>>> executable stage, each
>>>>>>>>> >>>>>>>> PCollection has to be encoded using SDK coder and
>>>>>>>>> immediately decoded by
>>>>>>>>> >>>>>>>> runner coder, *unless this can be optimized out* by
>>>>>>>>> property c).
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>> This gives us two options where to implement this
>>>>>>>>> encode/decode step:
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>>       1) completely inside runner with the possibility to
>>>>>>>>> optimize the
>>>>>>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>>       2) partly in the runner and partly in the SDK -
>>>>>>>>> that is we encode
>>>>>>>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass
>>>>>>>>> those to the
>>>>>>>>> >>>>>>>> SDK harness and apply a custom decode step there. This
>>>>>>>>> works because SDK
>>>>>>>>> >>>>>>>> coder encoded elements are in byte[], and that is
>>>>>>>>> well-known coder type.
>>>>>>>>> >>>>>>>> We again only leverage property c) and optimize the SDK
>>>>>>>>> coder encode,
>>>>>>>>> >>>>>>>> runner decode step out.
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream
>>>>>>>>> producing byte[] and
>>>>>>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>>>>>>>>> inlined
>>>>>>>>> >>>>>>>> transform, the elements are produced directly in runner
>>>>>>>>> (the SDK coder
>>>>>>>>> >>>>>>>> is not known to the runner, but that does not matter,
>>>>>>>>> because the
>>>>>>>>> >>>>>>>> elements are already encoded by client).
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>>      From the above it seems to me, that option 1) should
>>>>>>>>> be preferred, because:
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>>       i) it is generic, applicable to all inlined
>>>>>>>>> transforms, any sources
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>>       ii) it is consistent with how things logically work
>>>>>>>>> underneath
>>>>>>>>> >>>>>>>>
>>>>>>>>> >>>>>>>>       iii) it offers better room for optimization -
>>>>>>>>> option 2) might result
>>>>>>>>> >>>>>>>> in cases when the elements are passed from the runner to
>>>>>>>>> the SDK-harness
>>>>>>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>>>>>>> immediately
>>>>>>>>> >>>>>>>> encoding back using SDK-coder and returned back to the
>>>>>>>>> runner. This
>>>>>>>>> >>>>>>>> would be the case when TestStream would be directly
>>>>>>>>> consumed by inlined
>>>>>>>>> >>>>>>>> (or external) transform.
>>>>>>>>> >>>>>>> (1) is not possible if the Coder in question is not known
>>>>>>>>> to the
>>>>>>>>> >>>>>>> Runner, which is why I proposed (2).
>>>>>>>>> >>>>>> There is no particular need for the coder to be known. If
>>>>>>>>> transform is
>>>>>>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded
>>>>>>>>> form of the
>>>>>>>>> >>>>>> elements. That holds true if:
>>>>>>>>> >>>>>>
>>>>>>>>> >>>>>>      a) either the SDK coder is known, or
>>>>>>>>> >>>>>>
>>>>>>>>> >>>>>>      b) encoded form of the produced elements is known in
>>>>>>>>> advance
>>>>>>>>> >>>>>>
>>>>>>>>> >>>>>> For TestStream it is the case b). For inlined primitive
>>>>>>>>> Read (or any
>>>>>>>>> >>>>>> other transform which executes code) it is a).
>>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>>> reasons,
>>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>>> delimiting)
>>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>>> encoded bytes,
>>>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>>>> SDK. The
>>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested
>>>>>>>>> encoding> for an
>>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>>> >>>>>
>>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to
>>>>>>>>> send already
>>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>>> complicate the
>>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>>> sending the
>>>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>>>> solved
>>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>>> known by the
>>>>>>>>> >>>>> runner).)
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>>
>>>>>>>
>>>>>>> Jack McCluskey
>>>>>>> SWE - DataPLS PLAT/ Beam Go
>>>>>>> RDU
>>>>>>> jrmccluskey@gmail.com
>>>>>>>
>>>>>>>
>>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
Go doesn't have a (portable, cross-language) runner. It's fine for SDKs to
enumerate the coders they understand. Runners, however, should respect what
SDKs declare.

On Thu, Sep 9, 2021 at 12:54 PM Jack McCluskey <jr...@google.com>
wrote:

> Go also doesn't do this, instead maintaining a list of known standard
> coders and using that as a list of capabilities:
> https://github.com/apache/beam/blob/4bc870806099f03265e7dfb48b142f00cee42f47/sdks/go/pkg/beam/core/runtime/graphx/coder.go#L59
>
> On Thu, Sep 9, 2021 at 1:29 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> You are right, Java does not implement this correctly. It should be
>> querying the capabilities section of the environment proto. (For java
>> environments, this is populated from ModelCoders, e.g.
>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386
>> )
>>
>> Looks like Python doesn't do any better:
>> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459
>>
>>
>>
>> On Thu, Sep 9, 2021 at 10:13 AM Jan Lukavský <je...@seznam.cz> wrote:
>>
>>> This makes a *lot* of sense. But it seems to me, that this is not the
>>> way Java-based runners - that use runners-core-construction-java module -
>>> handle it. If I interpret it correctly, then the set of ModelCoders is
>>> hard-coded [1] and essentially required to be known by all SDKs [2].
>>>
>>> There seems to be no negotiation between what SDK harness knows and what
>>> the runner knows. The runner might be able to define the wire coder for the
>>> SDK (via the ProcessBundleDescriptor), but the SDK (for Java runners) seems
>>> not to be able to play any role in this [3]. Therefore I think that if an
>>> SDK does not know the set of Java ModelCoders, then the runner and the SDK
>>> might not agree on the binary encoding of the elements (BTW, which is why
>>> Java Coders cannot be part of ModelCoders and I finally understand why I
>>> had such troubles adding it there - it cannot be there).
>>>
>>> Is it possible we are missing some part of this runner-to-sdk coder
>>> negotiation in runners-core-contruction-java?
>>>
>>> [1]
>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75
>>>
>>> [2]
>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62
>>>
>>> [3]
>>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104
>>> On 9/9/21 12:18 AM, Robert Bradshaw wrote:
>>>
>>> The whole notion of an absolute set of known coders is a misnomer. It
>>> would require all Runners and SDKs to be updated synchronously for every
>>> new coder they might want to share.
>>>
>>> Instead, what we have are
>>>
>>> * Standard Coders which have well-defined, language-agnostic
>>> representations and encodings, which may be used for interoperability and
>>> efficiency, and
>>> * Required Coders which are the minimum needed to execute the pipeline.
>>>
>>> The latter consists only of bytes (for impulse), kv and iterable (for
>>> GBK), windowed value (for windowing information) and length prefix (to be
>>> able to handle anything else).
>>>
>>>
>>> On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <ro...@frantil.com> wrote:
>>>
>>>> Is the claim that the Standard bytes and String_utf8 coders are not
>>>> "known coders"?
>>>>
>>>>  What's the point of the standard coders if they are not the canonical
>>>> "known coders" that can generally be expected to be known by runners/other
>>>> SDKs?
>>>>
>>>>
>>>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>>>>
>>>> The Go SDK rather heavily biases towards using the standard coders for
>>>> their closes language equivalents rather than going into override/custom
>>>> specified soup. (It's not possible to globally override the coders for the
>>>> '[]byte' and 'string' types, nor is there often reason to.)
>>>>
>>>> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <jr...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Hey all,
>>>>>>
>>>>>> Just catching up on the thread since I did the TestStream Go SDK
>>>>>> implementation. The discussion about length prefixing behavior for known
>>>>>> vs. unknown coders is interesting, since we ran into strings and byte
>>>>>> slices getting extra length prefixes attached to them by Flink despite
>>>>>> being known coders.
>>>>>>
>>>>>
>>>>> Known to who?
>>>>>
>>>>>
>>>>>> Based on what's been said, that isn't expected behavior, right?
>>>>>>
>>>>>
>>>>> No, it's not.
>>>>>
>>>>> I would check to make sure the Go SDK is respecting the Coder (length
>>>>> prefixed or not) that's set on the channel, rather than guessing at what it
>>>>> expects it to be based on the Go type.
>>>>>
>>>>>
>>>>>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>
>>>>>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>>>>>
>>>>>>> Historically the DataflowRunner has been much more careful about not
>>>>>>> breaking update, since this is a frequent operation by Dataflow users. I
>>>>>>> think we've been less careful aboutt other runners, but as we see clearly
>>>>>>> here Fllnk users do care about this as well, so we should probably test
>>>>>>> upgrade compatibility for Flink.
>>>>>>>
>>>>>>> One strategy that Dataflow uses is to avoid embedding the Java
>>>>>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>>>>>> breakage (as we see with the issue you llnked to). Possibly similar
>>>>>>> strategies should be investigated for Fllink.
>>>>>>>
>>>>>>> +1, that would be great!
>>>>>>>
>>>>>>>
>>>>>>> Reuven
>>>>>>>
>>>>>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>>
>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>> iterable)
>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>> > changing coders is hard (due to no way to update the encoding for
>>>>>>>> a
>>>>>>>> > streaming pipeline).
>>>>>>>> This is unrelated, but - regarding changing coders due to concerns
>>>>>>>> about
>>>>>>>> pipeline upgrades, we break this quite often, at least for some
>>>>>>>> runners.
>>>>>>>> Most recently [1].
>>>>>>>>
>>>>>>>> > It is currently the latter for runners using this code (which not
>>>>>>>> all
>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>> This is interesting and unexpected for me. How do runners decide
>>>>>>>> about
>>>>>>>> how they encode elements between SDK harness and the runner? How do
>>>>>>>> they
>>>>>>>> inform the SDK harness about this decision? My impression was that
>>>>>>>> this
>>>>>>>> is well-defined at the model level. If not, then we have the reason
>>>>>>>> for
>>>>>>>> misunderstanding in this conversation. :-)
>>>>>>>>
>>>>>>>>   Jan
>>>>>>>>
>>>>>>>> [1]
>>>>>>>>
>>>>>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>>>>>
>>>>>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>>>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>>>>>> >>
>>>>>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>> >>>> Hi Robert,
>>>>>>>> >>>>
>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>> reasons,
>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>> delimiting)
>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>> encoded bytes,
>>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>>> SDK. The
>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>>>> for an
>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>> >>>>>
>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>>>> already
>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>> complicate the
>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>> sending the
>>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>>> solved
>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>> known by the
>>>>>>>> >>>>> runner).)
>>>>>>>> >>>> It is hard for me to argue with "historical reasons". But -
>>>>>>>> the "nested"
>>>>>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>>>>>> runner-coder
>>>>>>>> >>>> spaces.
>>>>>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>>>>>> >> Hm, do you mean the Context passed to the encode/decode method?
>>>>>>>> [1] That
>>>>>>>> >> seems to be deprecated, I assume that most coders use the default
>>>>>>>> >> implementation and simply ignore the Context?
>>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>>> iterable)
>>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>>> > changing coders is hard (due to no way to update the encoding for
>>>>>>>> a
>>>>>>>> > streaming pipeline).
>>>>>>>> >
>>>>>>>> >> Even if not - whether or
>>>>>>>> >> not the elements are encoded using NESTED Context or UNNESTED
>>>>>>>> Context
>>>>>>>> >> should be part of the contract of TestStream, right? Most likely
>>>>>>>> it is
>>>>>>>> >> the UNNESTED one, if I understand correctly what that does.
>>>>>>>> Under what
>>>>>>>> >> conditions is the deprecated encode/decode method used?
>>>>>>>> > Yes, it's the UNNESTED one.
>>>>>>>> >
>>>>>>>> >> [1]
>>>>>>>> >>
>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>>>>>> >>
>>>>>>>> >>>> The runner's responsibility is not to go from "<nested
>>>>>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary
>>>>>>>> coder.
>>>>>>>> >>>> That is really impossible. But a coder is a function, right?
>>>>>>>> Function
>>>>>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>>>>>> provides a
>>>>>>>> >>>> set of elements, and these elements are the "universe". For
>>>>>>>> those
>>>>>>>> >>>> elements it also provides the encoded form, which can be
>>>>>>>> interpreted as
>>>>>>>> >>>> the definition of the coder.
>>>>>>>> >>> The problem here is that there is not "the encoded form" for a
>>>>>>>> Coder
>>>>>>>> >>> but two encoded forms, and we have the wrong one. Things could
>>>>>>>> be made
>>>>>>>> >>> to work if we had the other.
>>>>>>>> >> Which two encoded forms do you refer to? Elements encoded by
>>>>>>>> both the
>>>>>>>> >> SDK-coder and runner-coder (and I ignore the Context here once
>>>>>>>> again)
>>>>>>>> >> have the same binary representation (which they must have,
>>>>>>>> otherwise it
>>>>>>>> >> would be impossible to decode elements coming from the runner to
>>>>>>>> the
>>>>>>>> >> SDK-harness or vice-versa).
>>>>>>>> >>>> Therefore - technically (and formally) -
>>>>>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>>>>>> regardless of
>>>>>>>> >>>> the language the runner is written in.
>>>>>>>> >>>>
>>>>>>>> >>>> To move  this discussion forward, I think we should look for
>>>>>>>> answers to
>>>>>>>> >>>> the following questions:
>>>>>>>> >>>>
>>>>>>>> >>>>     a) do we have any clues that show, that the proposed "in
>>>>>>>> runner"
>>>>>>>> >>>> solution will not work?
>>>>>>>> >>> OK, thinking about it some more, in the TestStream, we can use
>>>>>>>> the
>>>>>>>> >>> happy coincidence that
>>>>>>>> >>>
>>>>>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>>>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>>>>>> >>> nested=False)
>>>>>>>> >>>
>>>>>>>> >>> (where || denotes concatenation) and the fact that we have
>>>>>>>> >>>
>>>>>>>> >>>       C.encode(x, nested=False)
>>>>>>>> >>>
>>>>>>>> >>> in hand.
>>>>>>>> >>>
>>>>>>>> >>> A possible fix here for the OP's question is that when
>>>>>>>> rehydrating the
>>>>>>>> >>> TestStream transform it must behave differently according to
>>>>>>>> the coder
>>>>>>>> >>> used in the subsequent channel (e.g. for known coders, it
>>>>>>>> decodes the
>>>>>>>> >>> elements and emits them directly, but for unknown coders, it
>>>>>>>> prefixes
>>>>>>>> >>> them with their length and emits byte strings. It gets more
>>>>>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>>>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>>>>>> unknown-coder))
>>>>>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different
>>>>>>>> encodings
>>>>>>>> >>> (and the latter, which is the default, requires transcoding the
>>>>>>>> bytes
>>>>>>>> >>> to inject the length in the middle which is found by decoding
>>>>>>>> the
>>>>>>>> >>> first component). As well as getting more complex, this really
>>>>>>>> seems
>>>>>>>> >>> to violate the spirit of separation of concerns.
>>>>>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>>>>>> >> KV<LP<unknown>, known>? From my understanding it is always the
>>>>>>>> latter,
>>>>>>>> >> because of [2].
>>>>>>>> >>
>>>>>>>> >> [2]
>>>>>>>> >>
>>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>>> > It is currently the latter for runners using this code (which not
>>>>>>>> all
>>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>>> >
>>>>>>>> >>>>     b) do we think, that it will not be robust enough to
>>>>>>>> incorporate the
>>>>>>>> >>>> other use-cases (line generic transform inlining, taking into
>>>>>>>> account
>>>>>>>> >>>> that this applies only to runners that are written in the same
>>>>>>>> language
>>>>>>>> >>>> as the submitting SDK, because otherwise, there is nothing to
>>>>>>>> inline)?
>>>>>>>> >>> Being in the same language is not a prerequisite to "inlining,"
>>>>>>>> e.g.
>>>>>>>> >>> the PubSub source on Dataflow is recognized as such and not
>>>>>>>> executed
>>>>>>>> >>> as SDK code but natively.
>>>>>>>> >> Agree, that is actually exactly what happens with the
>>>>>>>> TestStream. The
>>>>>>>> >> transform need not be in the same language, as long as it is
>>>>>>>> completely
>>>>>>>> >> understood by the runner, including the SDK-coder (either
>>>>>>>> explicitly -
>>>>>>>> >> which might be due to the PCollection coder being composed of
>>>>>>>> well-known
>>>>>>>> >> coders only, or implicitly like in the case of TestStream, where
>>>>>>>> the
>>>>>>>> >> elements are encoded using the SDK coder.
>>>>>>>> >>> It is more likely that inlining occurs in the same language if
>>>>>>>> there
>>>>>>>> >>> are UDFs involved.
>>>>>>>> >>>
>>>>>>>> >>>> I'm convinced, that the TestStream-decode expansion solution
>>>>>>>> is an
>>>>>>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>>>>>>> bothering
>>>>>>>> >>>> this mailing list with my emails on this. :-)
>>>>>>>> >>>>
>>>>>>>> >>>> WDYT?
>>>>>>>> >>> While not a solution to the general problem, I think the
>>>>>>>> >>> TestStream-only-does-bytes simplifies its definition (primitives
>>>>>>>> >>> should have as simple/easy to implement definitions as
>>>>>>>> possible) and
>>>>>>>> >>> brings it closer to the other root we have: Impulse. (We could
>>>>>>>> go a
>>>>>>>> >>> step further and rather than emitting encoded elements, with
>>>>>>>> the data
>>>>>>>> >>> in the proto itself, it emits sequence numbers, and a
>>>>>>>> subsequent ParDo
>>>>>>>> >>> maps those to concrete elements (e.g. via an in-memory map),
>>>>>>>> but that
>>>>>>>> >>> further step doesn't buy much...)
>>>>>>>> >>>
>>>>>>>> >>> Only runners that want to do inlining would have to take on the
>>>>>>>> >>> complexity of a fully generic solution.
>>>>>>>> >> I think that if the simplification brings something, we can do
>>>>>>>> that, but
>>>>>>>> >> I'd like to understand why we cannot (or should not) use the
>>>>>>>> generic
>>>>>>>> >> solution. I think it definitely *should* be possible to use a
>>>>>>>> generic
>>>>>>>> >> solution, because otherwise the solution would not be generic.
>>>>>>>> And it
>>>>>>>> >> would imply, that we are unable to do generic transform
>>>>>>>> inlining, which
>>>>>>>> >> I would find really strange. That would immediately mean, that
>>>>>>>> we are
>>>>>>>> >> unable to construct classical runner as a special case of the
>>>>>>>> portable
>>>>>>>> >> one, which would be bad I think.
>>>>>>>> >>
>>>>>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>>>>>> SDK-coder,
>>>>>>>> >> or does this go through the LengthPrefixUnknownCoders logic? If
>>>>>>>> not,
>>>>>>>> >> then the problem would be there, because that means, that the
>>>>>>>> SDK-coder
>>>>>>>> >> cannot be (implicitly) defined in the runner. If the elements
>>>>>>>> would be
>>>>>>>> >> encoded using LP, then it would be possible to decode them using
>>>>>>>> >> runner-coder and the problem should be solved, or am I still
>>>>>>>> missing
>>>>>>>> >> some key parts?
>>>>>>>> > Yes, the problem is precisely that there are (unspecified)
>>>>>>>> constraints
>>>>>>>> > on the coder used by the TestStreamPayload. Just requiring that
>>>>>>>> it be
>>>>>>>> > length prefixed is not enough, you have to make constraints on
>>>>>>>> > sometimes pushing down the length prefixing if it's a composite
>>>>>>>> (like
>>>>>>>> > a KV) that depend on what the runner is expected to support in
>>>>>>>> terms
>>>>>>>> > of composites and/or the choices it chooses for the channel (and
>>>>>>>> the
>>>>>>>> > runner, not knowing the coder, can't transcode between these
>>>>>>>> choices).
>>>>>>>> >
>>>>>>>> > The simpler solution is to constrain this coder to just be byte[]
>>>>>>>> > rather than let it be a little bit flexible (but not wholly
>>>>>>>> flexible).
>>>>>>>> >
>>>>>>>> > As for a fully generic solution, I think the issue encountered
>>>>>>>> with
>>>>>>>> > inlining Read vs. TestStream are related to this, but not really
>>>>>>>> the
>>>>>>>> > same. With TestStream one has an encoded representation of the
>>>>>>>> > elements provided by the SDK that the Runner and has no SDK
>>>>>>>> > representation/execution whereas with the Reads one has unencoded
>>>>>>>> > elements in hand and a Coder that is understood by both (so long
>>>>>>>> as
>>>>>>>> > the channel can be negotiated correctly). FWIW, I think the proper
>>>>>>>> > solution to inlining a Read (or other Transform that would
>>>>>>>> typically
>>>>>>>> > be executed in the SDK) is to treat it as a special environment
>>>>>>>> (where
>>>>>>>> > we know logically it can work) and then elide, as possible, the
>>>>>>>> > various encodings, grpc calls, etc. that are unneeded as
>>>>>>>> everything is
>>>>>>>> > in process.
>>>>>>>> >
>>>>>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>> >>>>>>>> Hi,
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>> I had some more time thinking about this and I'll try to
>>>>>>>> recap that.
>>>>>>>> >>>>>>>> First some invariants:
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       a) each PCollection<T> has actually two coders - an
>>>>>>>> _SDK coder_ and a
>>>>>>>> >>>>>>>> _runner coder_. These coders have the property, that each
>>>>>>>> one can
>>>>>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not
>>>>>>>> true, the
>>>>>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in
>>>>>>>> general).
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>>>>>> environment, the
>>>>>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>>>>>> SDK-harness and
>>>>>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       c) under specific circumstances, the encode-decode
>>>>>>>> step can be
>>>>>>>> >>>>>>>> optimized out, that is the case where the SDK coder and
>>>>>>>> all its
>>>>>>>> >>>>>>>> subcoders are all well-known to the runner (in the
>>>>>>>> present, that means
>>>>>>>> >>>>>>>> that all the parts present in the model coders set). The
>>>>>>>> reason for that
>>>>>>>> >>>>>>>> is that in this specific situation
>>>>>>>> runner_decode(sdk_encode(X)) = X.
>>>>>>>> >>>>>>>> This property is essential.
>>>>>>>> >>>>>>> However, in general, X can only pass from the SDK to the
>>>>>>>> runner (or
>>>>>>>> >>>>>>> vice versa) in encoded form.
>>>>>>>> >>>>>> In general yes, but we are (mostly) talking transform
>>>>>>>> inlining here, so
>>>>>>>> >>>>>> it that particular situation, the elements might be passed
>>>>>>>> in decoded form.
>>>>>>>> >>>>>>>>       d) from b) immediately follows, that when a
>>>>>>>> PTransform does not run in
>>>>>>>> >>>>>>>> an environment (and this might be due to the transform
>>>>>>>> being runner
>>>>>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>>>>>>>> elements have
>>>>>>>> >>>>>>>> to be encoded by SDK coder, immediately following decode
>>>>>>>> by runner
>>>>>>>> >>>>>>>> coder. That (surprisingly) applies even to situations when
>>>>>>>> runner is
>>>>>>>> >>>>>>>> implemented using different language than the client SDK,
>>>>>>>> because it
>>>>>>>> >>>>>>>> implies that the type of produced elements must be one of
>>>>>>>> types encoded
>>>>>>>> >>>>>>>> using model coders (well-known to the runner, otherwise
>>>>>>>> the SDK will not
>>>>>>>> >>>>>>>> be able to consume it). But - due to property c) - this
>>>>>>>> means that this
>>>>>>>> >>>>>>>> encode-decode step can be optimized out. This does not
>>>>>>>> mean that it is
>>>>>>>> >>>>>>>> not (logically) present, though. This is exactly the case
>>>>>>>> of native
>>>>>>>> >>>>>>>> Impulse transform.
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>> Now, from that we can conclude that on the boundary
>>>>>>>> between executable
>>>>>>>> >>>>>>>> stages, or between runner (inlined) transform and
>>>>>>>> executable stage, each
>>>>>>>> >>>>>>>> PCollection has to be encoded using SDK coder and
>>>>>>>> immediately decoded by
>>>>>>>> >>>>>>>> runner coder, *unless this can be optimized out* by
>>>>>>>> property c).
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>> This gives us two options where to implement this
>>>>>>>> encode/decode step:
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       1) completely inside runner with the possibility to
>>>>>>>> optimize the
>>>>>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       2) partly in the runner and partly in the SDK - that
>>>>>>>> is we encode
>>>>>>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass
>>>>>>>> those to the
>>>>>>>> >>>>>>>> SDK harness and apply a custom decode step there. This
>>>>>>>> works because SDK
>>>>>>>> >>>>>>>> coder encoded elements are in byte[], and that is
>>>>>>>> well-known coder type.
>>>>>>>> >>>>>>>> We again only leverage property c) and optimize the SDK
>>>>>>>> coder encode,
>>>>>>>> >>>>>>>> runner decode step out.
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream
>>>>>>>> producing byte[] and
>>>>>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>>>>>>>> inlined
>>>>>>>> >>>>>>>> transform, the elements are produced directly in runner
>>>>>>>> (the SDK coder
>>>>>>>> >>>>>>>> is not known to the runner, but that does not matter,
>>>>>>>> because the
>>>>>>>> >>>>>>>> elements are already encoded by client).
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>      From the above it seems to me, that option 1) should
>>>>>>>> be preferred, because:
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       i) it is generic, applicable to all inlined
>>>>>>>> transforms, any sources
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       ii) it is consistent with how things logically work
>>>>>>>> underneath
>>>>>>>> >>>>>>>>
>>>>>>>> >>>>>>>>       iii) it offers better room for optimization - option
>>>>>>>> 2) might result
>>>>>>>> >>>>>>>> in cases when the elements are passed from the runner to
>>>>>>>> the SDK-harness
>>>>>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>>>>>> immediately
>>>>>>>> >>>>>>>> encoding back using SDK-coder and returned back to the
>>>>>>>> runner. This
>>>>>>>> >>>>>>>> would be the case when TestStream would be directly
>>>>>>>> consumed by inlined
>>>>>>>> >>>>>>>> (or external) transform.
>>>>>>>> >>>>>>> (1) is not possible if the Coder in question is not known
>>>>>>>> to the
>>>>>>>> >>>>>>> Runner, which is why I proposed (2).
>>>>>>>> >>>>>> There is no particular need for the coder to be known. If
>>>>>>>> transform is
>>>>>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded
>>>>>>>> form of the
>>>>>>>> >>>>>> elements. That holds true if:
>>>>>>>> >>>>>>
>>>>>>>> >>>>>>      a) either the SDK coder is known, or
>>>>>>>> >>>>>>
>>>>>>>> >>>>>>      b) encoded form of the produced elements is known in
>>>>>>>> advance
>>>>>>>> >>>>>>
>>>>>>>> >>>>>> For TestStream it is the case b). For inlined primitive Read
>>>>>>>> (or any
>>>>>>>> >>>>>> other transform which executes code) it is a).
>>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>>> reasons,
>>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>>> delimiting)
>>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>>> encoded bytes,
>>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>>> SDK. The
>>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>>>> for an
>>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>>> >>>>>
>>>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>>>> already
>>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>>> complicate the
>>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>>> sending the
>>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>>> solved
>>>>>>>> >>>>> differently for inlined reads as the coder can always be
>>>>>>>> known by the
>>>>>>>> >>>>> runner).)
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>> --
>>>>>>
>>>>>>
>>>>>> Jack McCluskey
>>>>>> SWE - DataPLS PLAT/ Beam Go
>>>>>> RDU
>>>>>> jrmccluskey@gmail.com
>>>>>>
>>>>>>
>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Jack McCluskey <jr...@google.com>.
Go also doesn't do this, instead maintaining a list of known standard
coders and using that as a list of capabilities:
https://github.com/apache/beam/blob/4bc870806099f03265e7dfb48b142f00cee42f47/sdks/go/pkg/beam/core/runtime/graphx/coder.go#L59

On Thu, Sep 9, 2021 at 1:29 PM Robert Bradshaw <ro...@google.com> wrote:

> You are right, Java does not implement this correctly. It should be
> querying the capabilities section of the environment proto. (For java
> environments, this is populated from ModelCoders, e.g.
> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386
> )
>
> Looks like Python doesn't do any better:
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459
>
>
>
> On Thu, Sep 9, 2021 at 10:13 AM Jan Lukavský <je...@seznam.cz> wrote:
>
>> This makes a *lot* of sense. But it seems to me, that this is not the way
>> Java-based runners - that use runners-core-construction-java module -
>> handle it. If I interpret it correctly, then the set of ModelCoders is
>> hard-coded [1] and essentially required to be known by all SDKs [2].
>>
>> There seems to be no negotiation between what SDK harness knows and what
>> the runner knows. The runner might be able to define the wire coder for the
>> SDK (via the ProcessBundleDescriptor), but the SDK (for Java runners) seems
>> not to be able to play any role in this [3]. Therefore I think that if an
>> SDK does not know the set of Java ModelCoders, then the runner and the SDK
>> might not agree on the binary encoding of the elements (BTW, which is why
>> Java Coders cannot be part of ModelCoders and I finally understand why I
>> had such troubles adding it there - it cannot be there).
>>
>> Is it possible we are missing some part of this runner-to-sdk coder
>> negotiation in runners-core-contruction-java?
>>
>> [1]
>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75
>>
>> [2]
>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62
>>
>> [3]
>> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104
>> On 9/9/21 12:18 AM, Robert Bradshaw wrote:
>>
>> The whole notion of an absolute set of known coders is a misnomer. It
>> would require all Runners and SDKs to be updated synchronously for every
>> new coder they might want to share.
>>
>> Instead, what we have are
>>
>> * Standard Coders which have well-defined, language-agnostic
>> representations and encodings, which may be used for interoperability and
>> efficiency, and
>> * Required Coders which are the minimum needed to execute the pipeline.
>>
>> The latter consists only of bytes (for impulse), kv and iterable (for
>> GBK), windowed value (for windowing information) and length prefix (to be
>> able to handle anything else).
>>
>>
>> On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <ro...@frantil.com> wrote:
>>
>>> Is the claim that the Standard bytes and String_utf8 coders are not
>>> "known coders"?
>>>
>>>  What's the point of the standard coders if they are not the canonical
>>> "known coders" that can generally be expected to be known by runners/other
>>> SDKs?
>>>
>>>
>>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>>>
>>> The Go SDK rather heavily biases towards using the standard coders for
>>> their closes language equivalents rather than going into override/custom
>>> specified soup. (It's not possible to globally override the coders for the
>>> '[]byte' and 'string' types, nor is there often reason to.)
>>>
>>> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <jr...@google.com>
>>>> wrote:
>>>>
>>>>> Hey all,
>>>>>
>>>>> Just catching up on the thread since I did the TestStream Go SDK
>>>>> implementation. The discussion about length prefixing behavior for known
>>>>> vs. unknown coders is interesting, since we ran into strings and byte
>>>>> slices getting extra length prefixes attached to them by Flink despite
>>>>> being known coders.
>>>>>
>>>>
>>>> Known to who?
>>>>
>>>>
>>>>> Based on what's been said, that isn't expected behavior, right?
>>>>>
>>>>
>>>> No, it's not.
>>>>
>>>> I would check to make sure the Go SDK is respecting the Coder (length
>>>> prefixed or not) that's set on the channel, rather than guessing at what it
>>>> expects it to be based on the Go type.
>>>>
>>>>
>>>>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>
>>>>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>>>>
>>>>>> Historically the DataflowRunner has been much more careful about not
>>>>>> breaking update, since this is a frequent operation by Dataflow users. I
>>>>>> think we've been less careful aboutt other runners, but as we see clearly
>>>>>> here Fllnk users do care about this as well, so we should probably test
>>>>>> upgrade compatibility for Flink.
>>>>>>
>>>>>> One strategy that Dataflow uses is to avoid embedding the Java
>>>>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>>>>> breakage (as we see with the issue you llnked to). Possibly similar
>>>>>> strategies should be investigated for Fllink.
>>>>>>
>>>>>> +1, that would be great!
>>>>>>
>>>>>>
>>>>>> Reuven
>>>>>>
>>>>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>
>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>> iterable)
>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>> > changing coders is hard (due to no way to update the encoding for a
>>>>>>> > streaming pipeline).
>>>>>>> This is unrelated, but - regarding changing coders due to concerns
>>>>>>> about
>>>>>>> pipeline upgrades, we break this quite often, at least for some
>>>>>>> runners.
>>>>>>> Most recently [1].
>>>>>>>
>>>>>>> > It is currently the latter for runners using this code (which not
>>>>>>> all
>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>> This is interesting and unexpected for me. How do runners decide
>>>>>>> about
>>>>>>> how they encode elements between SDK harness and the runner? How do
>>>>>>> they
>>>>>>> inform the SDK harness about this decision? My impression was that
>>>>>>> this
>>>>>>> is well-defined at the model level. If not, then we have the reason
>>>>>>> for
>>>>>>> misunderstanding in this conversation. :-)
>>>>>>>
>>>>>>>   Jan
>>>>>>>
>>>>>>> [1]
>>>>>>>
>>>>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>>>>
>>>>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz>
>>>>>>> wrote:
>>>>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>>>>> >>
>>>>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>>>>> wrote:
>>>>>>> >>>> Hi Robert,
>>>>>>> >>>>
>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>> reasons,
>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>> delimiting)
>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>> encoded bytes,
>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>> SDK. The
>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>>> for an
>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>> >>>>>
>>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>>> already
>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>> complicate the
>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>> sending the
>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>> solved
>>>>>>> >>>>> differently for inlined reads as the coder can always be known
>>>>>>> by the
>>>>>>> >>>>> runner).)
>>>>>>> >>>> It is hard for me to argue with "historical reasons". But - the
>>>>>>> "nested"
>>>>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>>>>> runner-coder
>>>>>>> >>>> spaces.
>>>>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>>>>> >> Hm, do you mean the Context passed to the encode/decode method?
>>>>>>> [1] That
>>>>>>> >> seems to be deprecated, I assume that most coders use the default
>>>>>>> >> implementation and simply ignore the Context?
>>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>>> iterable)
>>>>>>> > care about Context because they predated this deprecation, and
>>>>>>> > changing coders is hard (due to no way to update the encoding for a
>>>>>>> > streaming pipeline).
>>>>>>> >
>>>>>>> >> Even if not - whether or
>>>>>>> >> not the elements are encoded using NESTED Context or UNNESTED
>>>>>>> Context
>>>>>>> >> should be part of the contract of TestStream, right? Most likely
>>>>>>> it is
>>>>>>> >> the UNNESTED one, if I understand correctly what that does. Under
>>>>>>> what
>>>>>>> >> conditions is the deprecated encode/decode method used?
>>>>>>> > Yes, it's the UNNESTED one.
>>>>>>> >
>>>>>>> >> [1]
>>>>>>> >>
>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>>>>> >>
>>>>>>> >>>> The runner's responsibility is not to go from "<nested
>>>>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary
>>>>>>> coder.
>>>>>>> >>>> That is really impossible. But a coder is a function, right?
>>>>>>> Function
>>>>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>>>>> provides a
>>>>>>> >>>> set of elements, and these elements are the "universe". For
>>>>>>> those
>>>>>>> >>>> elements it also provides the encoded form, which can be
>>>>>>> interpreted as
>>>>>>> >>>> the definition of the coder.
>>>>>>> >>> The problem here is that there is not "the encoded form" for a
>>>>>>> Coder
>>>>>>> >>> but two encoded forms, and we have the wrong one. Things could
>>>>>>> be made
>>>>>>> >>> to work if we had the other.
>>>>>>> >> Which two encoded forms do you refer to? Elements encoded by both
>>>>>>> the
>>>>>>> >> SDK-coder and runner-coder (and I ignore the Context here once
>>>>>>> again)
>>>>>>> >> have the same binary representation (which they must have,
>>>>>>> otherwise it
>>>>>>> >> would be impossible to decode elements coming from the runner to
>>>>>>> the
>>>>>>> >> SDK-harness or vice-versa).
>>>>>>> >>>> Therefore - technically (and formally) -
>>>>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>>>>> regardless of
>>>>>>> >>>> the language the runner is written in.
>>>>>>> >>>>
>>>>>>> >>>> To move  this discussion forward, I think we should look for
>>>>>>> answers to
>>>>>>> >>>> the following questions:
>>>>>>> >>>>
>>>>>>> >>>>     a) do we have any clues that show, that the proposed "in
>>>>>>> runner"
>>>>>>> >>>> solution will not work?
>>>>>>> >>> OK, thinking about it some more, in the TestStream, we can use
>>>>>>> the
>>>>>>> >>> happy coincidence that
>>>>>>> >>>
>>>>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>>>>> >>> nested=False)
>>>>>>> >>>
>>>>>>> >>> (where || denotes concatenation) and the fact that we have
>>>>>>> >>>
>>>>>>> >>>       C.encode(x, nested=False)
>>>>>>> >>>
>>>>>>> >>> in hand.
>>>>>>> >>>
>>>>>>> >>> A possible fix here for the OP's question is that when
>>>>>>> rehydrating the
>>>>>>> >>> TestStream transform it must behave differently according to the
>>>>>>> coder
>>>>>>> >>> used in the subsequent channel (e.g. for known coders, it
>>>>>>> decodes the
>>>>>>> >>> elements and emits them directly, but for unknown coders, it
>>>>>>> prefixes
>>>>>>> >>> them with their length and emits byte strings. It gets more
>>>>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>>>>> unknown-coder))
>>>>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different
>>>>>>> encodings
>>>>>>> >>> (and the latter, which is the default, requires transcoding the
>>>>>>> bytes
>>>>>>> >>> to inject the length in the middle which is found by decoding the
>>>>>>> >>> first component). As well as getting more complex, this really
>>>>>>> seems
>>>>>>> >>> to violate the spirit of separation of concerns.
>>>>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>>>>> >> KV<LP<unknown>, known>? From my understanding it is always the
>>>>>>> latter,
>>>>>>> >> because of [2].
>>>>>>> >>
>>>>>>> >> [2]
>>>>>>> >>
>>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>> > It is currently the latter for runners using this code (which not
>>>>>>> all
>>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>>> >
>>>>>>> >>>>     b) do we think, that it will not be robust enough to
>>>>>>> incorporate the
>>>>>>> >>>> other use-cases (line generic transform inlining, taking into
>>>>>>> account
>>>>>>> >>>> that this applies only to runners that are written in the same
>>>>>>> language
>>>>>>> >>>> as the submitting SDK, because otherwise, there is nothing to
>>>>>>> inline)?
>>>>>>> >>> Being in the same language is not a prerequisite to "inlining,"
>>>>>>> e.g.
>>>>>>> >>> the PubSub source on Dataflow is recognized as such and not
>>>>>>> executed
>>>>>>> >>> as SDK code but natively.
>>>>>>> >> Agree, that is actually exactly what happens with the TestStream.
>>>>>>> The
>>>>>>> >> transform need not be in the same language, as long as it is
>>>>>>> completely
>>>>>>> >> understood by the runner, including the SDK-coder (either
>>>>>>> explicitly -
>>>>>>> >> which might be due to the PCollection coder being composed of
>>>>>>> well-known
>>>>>>> >> coders only, or implicitly like in the case of TestStream, where
>>>>>>> the
>>>>>>> >> elements are encoded using the SDK coder.
>>>>>>> >>> It is more likely that inlining occurs in the same language if
>>>>>>> there
>>>>>>> >>> are UDFs involved.
>>>>>>> >>>
>>>>>>> >>>> I'm convinced, that the TestStream-decode expansion solution is
>>>>>>> an
>>>>>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>>>>>> bothering
>>>>>>> >>>> this mailing list with my emails on this. :-)
>>>>>>> >>>>
>>>>>>> >>>> WDYT?
>>>>>>> >>> While not a solution to the general problem, I think the
>>>>>>> >>> TestStream-only-does-bytes simplifies its definition (primitives
>>>>>>> >>> should have as simple/easy to implement definitions as possible)
>>>>>>> and
>>>>>>> >>> brings it closer to the other root we have: Impulse. (We could
>>>>>>> go a
>>>>>>> >>> step further and rather than emitting encoded elements, with the
>>>>>>> data
>>>>>>> >>> in the proto itself, it emits sequence numbers, and a subsequent
>>>>>>> ParDo
>>>>>>> >>> maps those to concrete elements (e.g. via an in-memory map), but
>>>>>>> that
>>>>>>> >>> further step doesn't buy much...)
>>>>>>> >>>
>>>>>>> >>> Only runners that want to do inlining would have to take on the
>>>>>>> >>> complexity of a fully generic solution.
>>>>>>> >> I think that if the simplification brings something, we can do
>>>>>>> that, but
>>>>>>> >> I'd like to understand why we cannot (or should not) use the
>>>>>>> generic
>>>>>>> >> solution. I think it definitely *should* be possible to use a
>>>>>>> generic
>>>>>>> >> solution, because otherwise the solution would not be generic.
>>>>>>> And it
>>>>>>> >> would imply, that we are unable to do generic transform inlining,
>>>>>>> which
>>>>>>> >> I would find really strange. That would immediately mean, that we
>>>>>>> are
>>>>>>> >> unable to construct classical runner as a special case of the
>>>>>>> portable
>>>>>>> >> one, which would be bad I think.
>>>>>>> >>
>>>>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>>>>> SDK-coder,
>>>>>>> >> or does this go through the LengthPrefixUnknownCoders logic? If
>>>>>>> not,
>>>>>>> >> then the problem would be there, because that means, that the
>>>>>>> SDK-coder
>>>>>>> >> cannot be (implicitly) defined in the runner. If the elements
>>>>>>> would be
>>>>>>> >> encoded using LP, then it would be possible to decode them using
>>>>>>> >> runner-coder and the problem should be solved, or am I still
>>>>>>> missing
>>>>>>> >> some key parts?
>>>>>>> > Yes, the problem is precisely that there are (unspecified)
>>>>>>> constraints
>>>>>>> > on the coder used by the TestStreamPayload. Just requiring that it
>>>>>>> be
>>>>>>> > length prefixed is not enough, you have to make constraints on
>>>>>>> > sometimes pushing down the length prefixing if it's a composite
>>>>>>> (like
>>>>>>> > a KV) that depend on what the runner is expected to support in
>>>>>>> terms
>>>>>>> > of composites and/or the choices it chooses for the channel (and
>>>>>>> the
>>>>>>> > runner, not knowing the coder, can't transcode between these
>>>>>>> choices).
>>>>>>> >
>>>>>>> > The simpler solution is to constrain this coder to just be byte[]
>>>>>>> > rather than let it be a little bit flexible (but not wholly
>>>>>>> flexible).
>>>>>>> >
>>>>>>> > As for a fully generic solution, I think the issue encountered with
>>>>>>> > inlining Read vs. TestStream are related to this, but not really
>>>>>>> the
>>>>>>> > same. With TestStream one has an encoded representation of the
>>>>>>> > elements provided by the SDK that the Runner and has no SDK
>>>>>>> > representation/execution whereas with the Reads one has unencoded
>>>>>>> > elements in hand and a Coder that is understood by both (so long as
>>>>>>> > the channel can be negotiated correctly). FWIW, I think the proper
>>>>>>> > solution to inlining a Read (or other Transform that would
>>>>>>> typically
>>>>>>> > be executed in the SDK) is to treat it as a special environment
>>>>>>> (where
>>>>>>> > we know logically it can work) and then elide, as possible, the
>>>>>>> > various encodings, grpc calls, etc. that are unneeded as
>>>>>>> everything is
>>>>>>> > in process.
>>>>>>> >
>>>>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>>>>>> wrote:
>>>>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
>>>>>>> wrote:
>>>>>>> >>>>>>>> Hi,
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>> I had some more time thinking about this and I'll try to
>>>>>>> recap that.
>>>>>>> >>>>>>>> First some invariants:
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       a) each PCollection<T> has actually two coders - an
>>>>>>> _SDK coder_ and a
>>>>>>> >>>>>>>> _runner coder_. These coders have the property, that each
>>>>>>> one can
>>>>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not
>>>>>>> true, the
>>>>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in
>>>>>>> general).
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>>>>> environment, the
>>>>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>>>>> SDK-harness and
>>>>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       c) under specific circumstances, the encode-decode
>>>>>>> step can be
>>>>>>> >>>>>>>> optimized out, that is the case where the SDK coder and all
>>>>>>> its
>>>>>>> >>>>>>>> subcoders are all well-known to the runner (in the present,
>>>>>>> that means
>>>>>>> >>>>>>>> that all the parts present in the model coders set). The
>>>>>>> reason for that
>>>>>>> >>>>>>>> is that in this specific situation
>>>>>>> runner_decode(sdk_encode(X)) = X.
>>>>>>> >>>>>>>> This property is essential.
>>>>>>> >>>>>>> However, in general, X can only pass from the SDK to the
>>>>>>> runner (or
>>>>>>> >>>>>>> vice versa) in encoded form.
>>>>>>> >>>>>> In general yes, but we are (mostly) talking transform
>>>>>>> inlining here, so
>>>>>>> >>>>>> it that particular situation, the elements might be passed in
>>>>>>> decoded form.
>>>>>>> >>>>>>>>       d) from b) immediately follows, that when a
>>>>>>> PTransform does not run in
>>>>>>> >>>>>>>> an environment (and this might be due to the transform
>>>>>>> being runner
>>>>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>>>>>>> elements have
>>>>>>> >>>>>>>> to be encoded by SDK coder, immediately following decode by
>>>>>>> runner
>>>>>>> >>>>>>>> coder. That (surprisingly) applies even to situations when
>>>>>>> runner is
>>>>>>> >>>>>>>> implemented using different language than the client SDK,
>>>>>>> because it
>>>>>>> >>>>>>>> implies that the type of produced elements must be one of
>>>>>>> types encoded
>>>>>>> >>>>>>>> using model coders (well-known to the runner, otherwise the
>>>>>>> SDK will not
>>>>>>> >>>>>>>> be able to consume it). But - due to property c) - this
>>>>>>> means that this
>>>>>>> >>>>>>>> encode-decode step can be optimized out. This does not mean
>>>>>>> that it is
>>>>>>> >>>>>>>> not (logically) present, though. This is exactly the case
>>>>>>> of native
>>>>>>> >>>>>>>> Impulse transform.
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>> Now, from that we can conclude that on the boundary between
>>>>>>> executable
>>>>>>> >>>>>>>> stages, or between runner (inlined) transform and
>>>>>>> executable stage, each
>>>>>>> >>>>>>>> PCollection has to be encoded using SDK coder and
>>>>>>> immediately decoded by
>>>>>>> >>>>>>>> runner coder, *unless this can be optimized out* by
>>>>>>> property c).
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>> This gives us two options where to implement this
>>>>>>> encode/decode step:
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       1) completely inside runner with the possibility to
>>>>>>> optimize the
>>>>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       2) partly in the runner and partly in the SDK - that
>>>>>>> is we encode
>>>>>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass
>>>>>>> those to the
>>>>>>> >>>>>>>> SDK harness and apply a custom decode step there. This
>>>>>>> works because SDK
>>>>>>> >>>>>>>> coder encoded elements are in byte[], and that is
>>>>>>> well-known coder type.
>>>>>>> >>>>>>>> We again only leverage property c) and optimize the SDK
>>>>>>> coder encode,
>>>>>>> >>>>>>>> runner decode step out.
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream
>>>>>>> producing byte[] and
>>>>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>>>>>>> inlined
>>>>>>> >>>>>>>> transform, the elements are produced directly in runner
>>>>>>> (the SDK coder
>>>>>>> >>>>>>>> is not known to the runner, but that does not matter,
>>>>>>> because the
>>>>>>> >>>>>>>> elements are already encoded by client).
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>      From the above it seems to me, that option 1) should
>>>>>>> be preferred, because:
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       i) it is generic, applicable to all inlined
>>>>>>> transforms, any sources
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       ii) it is consistent with how things logically work
>>>>>>> underneath
>>>>>>> >>>>>>>>
>>>>>>> >>>>>>>>       iii) it offers better room for optimization - option
>>>>>>> 2) might result
>>>>>>> >>>>>>>> in cases when the elements are passed from the runner to
>>>>>>> the SDK-harness
>>>>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>>>>> immediately
>>>>>>> >>>>>>>> encoding back using SDK-coder and returned back to the
>>>>>>> runner. This
>>>>>>> >>>>>>>> would be the case when TestStream would be directly
>>>>>>> consumed by inlined
>>>>>>> >>>>>>>> (or external) transform.
>>>>>>> >>>>>>> (1) is not possible if the Coder in question is not known to
>>>>>>> the
>>>>>>> >>>>>>> Runner, which is why I proposed (2).
>>>>>>> >>>>>> There is no particular need for the coder to be known. If
>>>>>>> transform is
>>>>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded form
>>>>>>> of the
>>>>>>> >>>>>> elements. That holds true if:
>>>>>>> >>>>>>
>>>>>>> >>>>>>      a) either the SDK coder is known, or
>>>>>>> >>>>>>
>>>>>>> >>>>>>      b) encoded form of the produced elements is known in
>>>>>>> advance
>>>>>>> >>>>>>
>>>>>>> >>>>>> For TestStream it is the case b). For inlined primitive Read
>>>>>>> (or any
>>>>>>> >>>>>> other transform which executes code) it is a).
>>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>>> reasons,
>>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>>> delimiting)
>>>>>>> >>>>> and unnested. TestStream elements are given as unnested
>>>>>>> encoded bytes,
>>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>>> SDK. The
>>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>>> for an
>>>>>>> >>>>> arbitrary unknown coder.
>>>>>>> >>>>>
>>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>>> already
>>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>>> complicate the
>>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>>> sending the
>>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>>> solved
>>>>>>> >>>>> differently for inlined reads as the coder can always be known
>>>>>>> by the
>>>>>>> >>>>> runner).)
>>>>>>>
>>>>>>
>>>>>
>>>>> --
>>>>>
>>>>>
>>>>> Jack McCluskey
>>>>> SWE - DataPLS PLAT/ Beam Go
>>>>> RDU
>>>>> jrmccluskey@gmail.com
>>>>>
>>>>>
>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
You are right, Java does not implement this correctly. It should be
querying the capabilities section of the environment proto. (For java
environments, this is populated from ModelCoders, e.g.
https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java#L386
)

Looks like Python doesn't do any better:
https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L459



On Thu, Sep 9, 2021 at 10:13 AM Jan Lukavský <je...@seznam.cz> wrote:

> This makes a *lot* of sense. But it seems to me, that this is not the way
> Java-based runners - that use runners-core-construction-java module -
> handle it. If I interpret it correctly, then the set of ModelCoders is
> hard-coded [1] and essentially required to be known by all SDKs [2].
>
> There seems to be no negotiation between what SDK harness knows and what
> the runner knows. The runner might be able to define the wire coder for the
> SDK (via the ProcessBundleDescriptor), but the SDK (for Java runners) seems
> not to be able to play any role in this [3]. Therefore I think that if an
> SDK does not know the set of Java ModelCoders, then the runner and the SDK
> might not agree on the binary encoding of the elements (BTW, which is why
> Java Coders cannot be part of ModelCoders and I finally understand why I
> had such troubles adding it there - it cannot be there).
>
> Is it possible we are missing some part of this runner-to-sdk coder
> negotiation in runners-core-contruction-java?
>
> [1]
> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75
>
> [2]
> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62
>
> [3]
> https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104
> On 9/9/21 12:18 AM, Robert Bradshaw wrote:
>
> The whole notion of an absolute set of known coders is a misnomer. It
> would require all Runners and SDKs to be updated synchronously for every
> new coder they might want to share.
>
> Instead, what we have are
>
> * Standard Coders which have well-defined, language-agnostic
> representations and encodings, which may be used for interoperability and
> efficiency, and
> * Required Coders which are the minimum needed to execute the pipeline.
>
> The latter consists only of bytes (for impulse), kv and iterable (for
> GBK), windowed value (for windowing information) and length prefix (to be
> able to handle anything else).
>
>
> On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <ro...@frantil.com> wrote:
>
>> Is the claim that the Standard bytes and String_utf8 coders are not
>> "known coders"?
>>
>>  What's the point of the standard coders if they are not the canonical
>> "known coders" that can generally be expected to be known by runners/other
>> SDKs?
>>
>>
>> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>>
>> The Go SDK rather heavily biases towards using the standard coders for
>> their closes language equivalents rather than going into override/custom
>> specified soup. (It's not possible to globally override the coders for the
>> '[]byte' and 'string' types, nor is there often reason to.)
>>
>> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com> wrote:
>>
>>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <jr...@google.com>
>>> wrote:
>>>
>>>> Hey all,
>>>>
>>>> Just catching up on the thread since I did the TestStream Go SDK
>>>> implementation. The discussion about length prefixing behavior for known
>>>> vs. unknown coders is interesting, since we ran into strings and byte
>>>> slices getting extra length prefixes attached to them by Flink despite
>>>> being known coders.
>>>>
>>>
>>> Known to who?
>>>
>>>
>>>> Based on what's been said, that isn't expected behavior, right?
>>>>
>>>
>>> No, it's not.
>>>
>>> I would check to make sure the Go SDK is respecting the Coder (length
>>> prefixed or not) that's set on the channel, rather than guessing at what it
>>> expects it to be based on the Go type.
>>>
>>>
>>>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>
>>>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>>>
>>>>> Historically the DataflowRunner has been much more careful about not
>>>>> breaking update, since this is a frequent operation by Dataflow users. I
>>>>> think we've been less careful aboutt other runners, but as we see clearly
>>>>> here Fllnk users do care about this as well, so we should probably test
>>>>> upgrade compatibility for Flink.
>>>>>
>>>>> One strategy that Dataflow uses is to avoid embedding the Java
>>>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>>>> breakage (as we see with the issue you llnked to). Possibly similar
>>>>> strategies should be investigated for Fllink.
>>>>>
>>>>> +1, that would be great!
>>>>>
>>>>>
>>>>> Reuven
>>>>>
>>>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>
>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>> iterable)
>>>>>> > care about Context because they predated this deprecation, and
>>>>>> > changing coders is hard (due to no way to update the encoding for a
>>>>>> > streaming pipeline).
>>>>>> This is unrelated, but - regarding changing coders due to concerns
>>>>>> about
>>>>>> pipeline upgrades, we break this quite often, at least for some
>>>>>> runners.
>>>>>> Most recently [1].
>>>>>>
>>>>>> > It is currently the latter for runners using this code (which not
>>>>>> all
>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>> This is interesting and unexpected for me. How do runners decide
>>>>>> about
>>>>>> how they encode elements between SDK harness and the runner? How do
>>>>>> they
>>>>>> inform the SDK harness about this decision? My impression was that
>>>>>> this
>>>>>> is well-defined at the model level. If not, then we have the reason
>>>>>> for
>>>>>> misunderstanding in this conversation. :-)
>>>>>>
>>>>>>   Jan
>>>>>>
>>>>>> [1]
>>>>>>
>>>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>>>
>>>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz>
>>>>>> wrote:
>>>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>>>> >>
>>>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>>>> wrote:
>>>>>> >>>> Hi Robert,
>>>>>> >>>>
>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>> reasons,
>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>> delimiting)
>>>>>> >>>>> and unnested. TestStream elements are given as unnested encoded
>>>>>> bytes,
>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>> SDK. The
>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>> for an
>>>>>> >>>>> arbitrary unknown coder.
>>>>>> >>>>>
>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>> already
>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>> complicate the
>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>> sending the
>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>> solved
>>>>>> >>>>> differently for inlined reads as the coder can always be known
>>>>>> by the
>>>>>> >>>>> runner).)
>>>>>> >>>> It is hard for me to argue with "historical reasons". But - the
>>>>>> "nested"
>>>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>>>> runner-coder
>>>>>> >>>> spaces.
>>>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>>>> >> Hm, do you mean the Context passed to the encode/decode method?
>>>>>> [1] That
>>>>>> >> seems to be deprecated, I assume that most coders use the default
>>>>>> >> implementation and simply ignore the Context?
>>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>>> iterable)
>>>>>> > care about Context because they predated this deprecation, and
>>>>>> > changing coders is hard (due to no way to update the encoding for a
>>>>>> > streaming pipeline).
>>>>>> >
>>>>>> >> Even if not - whether or
>>>>>> >> not the elements are encoded using NESTED Context or UNNESTED
>>>>>> Context
>>>>>> >> should be part of the contract of TestStream, right? Most likely
>>>>>> it is
>>>>>> >> the UNNESTED one, if I understand correctly what that does. Under
>>>>>> what
>>>>>> >> conditions is the deprecated encode/decode method used?
>>>>>> > Yes, it's the UNNESTED one.
>>>>>> >
>>>>>> >> [1]
>>>>>> >>
>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>>>> >>
>>>>>> >>>> The runner's responsibility is not to go from "<nested
>>>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary
>>>>>> coder.
>>>>>> >>>> That is really impossible. But a coder is a function, right?
>>>>>> Function
>>>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>>>> provides a
>>>>>> >>>> set of elements, and these elements are the "universe". For those
>>>>>> >>>> elements it also provides the encoded form, which can be
>>>>>> interpreted as
>>>>>> >>>> the definition of the coder.
>>>>>> >>> The problem here is that there is not "the encoded form" for a
>>>>>> Coder
>>>>>> >>> but two encoded forms, and we have the wrong one. Things could be
>>>>>> made
>>>>>> >>> to work if we had the other.
>>>>>> >> Which two encoded forms do you refer to? Elements encoded by both
>>>>>> the
>>>>>> >> SDK-coder and runner-coder (and I ignore the Context here once
>>>>>> again)
>>>>>> >> have the same binary representation (which they must have,
>>>>>> otherwise it
>>>>>> >> would be impossible to decode elements coming from the runner to
>>>>>> the
>>>>>> >> SDK-harness or vice-versa).
>>>>>> >>>> Therefore - technically (and formally) -
>>>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>>>> regardless of
>>>>>> >>>> the language the runner is written in.
>>>>>> >>>>
>>>>>> >>>> To move  this discussion forward, I think we should look for
>>>>>> answers to
>>>>>> >>>> the following questions:
>>>>>> >>>>
>>>>>> >>>>     a) do we have any clues that show, that the proposed "in
>>>>>> runner"
>>>>>> >>>> solution will not work?
>>>>>> >>> OK, thinking about it some more, in the TestStream, we can use the
>>>>>> >>> happy coincidence that
>>>>>> >>>
>>>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>>>> >>> nested=False)
>>>>>> >>>
>>>>>> >>> (where || denotes concatenation) and the fact that we have
>>>>>> >>>
>>>>>> >>>       C.encode(x, nested=False)
>>>>>> >>>
>>>>>> >>> in hand.
>>>>>> >>>
>>>>>> >>> A possible fix here for the OP's question is that when
>>>>>> rehydrating the
>>>>>> >>> TestStream transform it must behave differently according to the
>>>>>> coder
>>>>>> >>> used in the subsequent channel (e.g. for known coders, it decodes
>>>>>> the
>>>>>> >>> elements and emits them directly, but for unknown coders, it
>>>>>> prefixes
>>>>>> >>> them with their length and emits byte strings. It gets more
>>>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>>>> unknown-coder))
>>>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different
>>>>>> encodings
>>>>>> >>> (and the latter, which is the default, requires transcoding the
>>>>>> bytes
>>>>>> >>> to inject the length in the middle which is found by decoding the
>>>>>> >>> first component). As well as getting more complex, this really
>>>>>> seems
>>>>>> >>> to violate the spirit of separation of concerns.
>>>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>>>> >> KV<LP<unknown>, known>? From my understanding it is always the
>>>>>> latter,
>>>>>> >> because of [2].
>>>>>> >>
>>>>>> >> [2]
>>>>>> >>
>>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>> > It is currently the latter for runners using this code (which not
>>>>>> all
>>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>>> >
>>>>>> >>>>     b) do we think, that it will not be robust enough to
>>>>>> incorporate the
>>>>>> >>>> other use-cases (line generic transform inlining, taking into
>>>>>> account
>>>>>> >>>> that this applies only to runners that are written in the same
>>>>>> language
>>>>>> >>>> as the submitting SDK, because otherwise, there is nothing to
>>>>>> inline)?
>>>>>> >>> Being in the same language is not a prerequisite to "inlining,"
>>>>>> e.g.
>>>>>> >>> the PubSub source on Dataflow is recognized as such and not
>>>>>> executed
>>>>>> >>> as SDK code but natively.
>>>>>> >> Agree, that is actually exactly what happens with the TestStream.
>>>>>> The
>>>>>> >> transform need not be in the same language, as long as it is
>>>>>> completely
>>>>>> >> understood by the runner, including the SDK-coder (either
>>>>>> explicitly -
>>>>>> >> which might be due to the PCollection coder being composed of
>>>>>> well-known
>>>>>> >> coders only, or implicitly like in the case of TestStream, where
>>>>>> the
>>>>>> >> elements are encoded using the SDK coder.
>>>>>> >>> It is more likely that inlining occurs in the same language if
>>>>>> there
>>>>>> >>> are UDFs involved.
>>>>>> >>>
>>>>>> >>>> I'm convinced, that the TestStream-decode expansion solution is
>>>>>> an
>>>>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>>>>> bothering
>>>>>> >>>> this mailing list with my emails on this. :-)
>>>>>> >>>>
>>>>>> >>>> WDYT?
>>>>>> >>> While not a solution to the general problem, I think the
>>>>>> >>> TestStream-only-does-bytes simplifies its definition (primitives
>>>>>> >>> should have as simple/easy to implement definitions as possible)
>>>>>> and
>>>>>> >>> brings it closer to the other root we have: Impulse. (We could go
>>>>>> a
>>>>>> >>> step further and rather than emitting encoded elements, with the
>>>>>> data
>>>>>> >>> in the proto itself, it emits sequence numbers, and a subsequent
>>>>>> ParDo
>>>>>> >>> maps those to concrete elements (e.g. via an in-memory map), but
>>>>>> that
>>>>>> >>> further step doesn't buy much...)
>>>>>> >>>
>>>>>> >>> Only runners that want to do inlining would have to take on the
>>>>>> >>> complexity of a fully generic solution.
>>>>>> >> I think that if the simplification brings something, we can do
>>>>>> that, but
>>>>>> >> I'd like to understand why we cannot (or should not) use the
>>>>>> generic
>>>>>> >> solution. I think it definitely *should* be possible to use a
>>>>>> generic
>>>>>> >> solution, because otherwise the solution would not be generic. And
>>>>>> it
>>>>>> >> would imply, that we are unable to do generic transform inlining,
>>>>>> which
>>>>>> >> I would find really strange. That would immediately mean, that we
>>>>>> are
>>>>>> >> unable to construct classical runner as a special case of the
>>>>>> portable
>>>>>> >> one, which would be bad I think.
>>>>>> >>
>>>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>>>> SDK-coder,
>>>>>> >> or does this go through the LengthPrefixUnknownCoders logic? If
>>>>>> not,
>>>>>> >> then the problem would be there, because that means, that the
>>>>>> SDK-coder
>>>>>> >> cannot be (implicitly) defined in the runner. If the elements
>>>>>> would be
>>>>>> >> encoded using LP, then it would be possible to decode them using
>>>>>> >> runner-coder and the problem should be solved, or am I still
>>>>>> missing
>>>>>> >> some key parts?
>>>>>> > Yes, the problem is precisely that there are (unspecified)
>>>>>> constraints
>>>>>> > on the coder used by the TestStreamPayload. Just requiring that it
>>>>>> be
>>>>>> > length prefixed is not enough, you have to make constraints on
>>>>>> > sometimes pushing down the length prefixing if it's a composite
>>>>>> (like
>>>>>> > a KV) that depend on what the runner is expected to support in terms
>>>>>> > of composites and/or the choices it chooses for the channel (and the
>>>>>> > runner, not knowing the coder, can't transcode between these
>>>>>> choices).
>>>>>> >
>>>>>> > The simpler solution is to constrain this coder to just be byte[]
>>>>>> > rather than let it be a little bit flexible (but not wholly
>>>>>> flexible).
>>>>>> >
>>>>>> > As for a fully generic solution, I think the issue encountered with
>>>>>> > inlining Read vs. TestStream are related to this, but not really the
>>>>>> > same. With TestStream one has an encoded representation of the
>>>>>> > elements provided by the SDK that the Runner and has no SDK
>>>>>> > representation/execution whereas with the Reads one has unencoded
>>>>>> > elements in hand and a Coder that is understood by both (so long as
>>>>>> > the channel can be negotiated correctly). FWIW, I think the proper
>>>>>> > solution to inlining a Read (or other Transform that would typically
>>>>>> > be executed in the SDK) is to treat it as a special environment
>>>>>> (where
>>>>>> > we know logically it can work) and then elide, as possible, the
>>>>>> > various encodings, grpc calls, etc. that are unneeded as everything
>>>>>> is
>>>>>> > in process.
>>>>>> >
>>>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>>>>> wrote:
>>>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
>>>>>> wrote:
>>>>>> >>>>>>>> Hi,
>>>>>> >>>>>>>>
>>>>>> >>>>>>>> I had some more time thinking about this and I'll try to
>>>>>> recap that.
>>>>>> >>>>>>>> First some invariants:
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>       a) each PCollection<T> has actually two coders - an
>>>>>> _SDK coder_ and a
>>>>>> >>>>>>>> _runner coder_. These coders have the property, that each
>>>>>> one can
>>>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not
>>>>>> true, the
>>>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in general).
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>>>> environment, the
>>>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>>>> SDK-harness and
>>>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>       c) under specific circumstances, the encode-decode
>>>>>> step can be
>>>>>> >>>>>>>> optimized out, that is the case where the SDK coder and all
>>>>>> its
>>>>>> >>>>>>>> subcoders are all well-known to the runner (in the present,
>>>>>> that means
>>>>>> >>>>>>>> that all the parts present in the model coders set). The
>>>>>> reason for that
>>>>>> >>>>>>>> is that in this specific situation
>>>>>> runner_decode(sdk_encode(X)) = X.
>>>>>> >>>>>>>> This property is essential.
>>>>>> >>>>>>> However, in general, X can only pass from the SDK to the
>>>>>> runner (or
>>>>>> >>>>>>> vice versa) in encoded form.
>>>>>> >>>>>> In general yes, but we are (mostly) talking transform inlining
>>>>>> here, so
>>>>>> >>>>>> it that particular situation, the elements might be passed in
>>>>>> decoded form.
>>>>>> >>>>>>>>       d) from b) immediately follows, that when a PTransform
>>>>>> does not run in
>>>>>> >>>>>>>> an environment (and this might be due to the transform being
>>>>>> runner
>>>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>>>>>> elements have
>>>>>> >>>>>>>> to be encoded by SDK coder, immediately following decode by
>>>>>> runner
>>>>>> >>>>>>>> coder. That (surprisingly) applies even to situations when
>>>>>> runner is
>>>>>> >>>>>>>> implemented using different language than the client SDK,
>>>>>> because it
>>>>>> >>>>>>>> implies that the type of produced elements must be one of
>>>>>> types encoded
>>>>>> >>>>>>>> using model coders (well-known to the runner, otherwise the
>>>>>> SDK will not
>>>>>> >>>>>>>> be able to consume it). But - due to property c) - this
>>>>>> means that this
>>>>>> >>>>>>>> encode-decode step can be optimized out. This does not mean
>>>>>> that it is
>>>>>> >>>>>>>> not (logically) present, though. This is exactly the case of
>>>>>> native
>>>>>> >>>>>>>> Impulse transform.
>>>>>> >>>>>>>>
>>>>>> >>>>>>>> Now, from that we can conclude that on the boundary between
>>>>>> executable
>>>>>> >>>>>>>> stages, or between runner (inlined) transform and executable
>>>>>> stage, each
>>>>>> >>>>>>>> PCollection has to be encoded using SDK coder and
>>>>>> immediately decoded by
>>>>>> >>>>>>>> runner coder, *unless this can be optimized out* by property
>>>>>> c).
>>>>>> >>>>>>>>
>>>>>> >>>>>>>> This gives us two options where to implement this
>>>>>> encode/decode step:
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>       1) completely inside runner with the possibility to
>>>>>> optimize the
>>>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>       2) partly in the runner and partly in the SDK - that
>>>>>> is we encode
>>>>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass
>>>>>> those to the
>>>>>> >>>>>>>> SDK harness and apply a custom decode step there. This works
>>>>>> because SDK
>>>>>> >>>>>>>> coder encoded elements are in byte[], and that is well-known
>>>>>> coder type.
>>>>>> >>>>>>>> We again only leverage property c) and optimize the SDK
>>>>>> coder encode,
>>>>>> >>>>>>>> runner decode step out.
>>>>>> >>>>>>>>
>>>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream
>>>>>> producing byte[] and
>>>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>>>>>> inlined
>>>>>> >>>>>>>> transform, the elements are produced directly in runner (the
>>>>>> SDK coder
>>>>>> >>>>>>>> is not known to the runner, but that does not matter,
>>>>>> because the
>>>>>> >>>>>>>> elements are already encoded by client).
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>      From the above it seems to me, that option 1) should be
>>>>>> preferred, because:
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>       i) it is generic, applicable to all inlined
>>>>>> transforms, any sources
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>       ii) it is consistent with how things logically work
>>>>>> underneath
>>>>>> >>>>>>>>
>>>>>> >>>>>>>>       iii) it offers better room for optimization - option
>>>>>> 2) might result
>>>>>> >>>>>>>> in cases when the elements are passed from the runner to the
>>>>>> SDK-harness
>>>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>>>> immediately
>>>>>> >>>>>>>> encoding back using SDK-coder and returned back to the
>>>>>> runner. This
>>>>>> >>>>>>>> would be the case when TestStream would be directly consumed
>>>>>> by inlined
>>>>>> >>>>>>>> (or external) transform.
>>>>>> >>>>>>> (1) is not possible if the Coder in question is not known to
>>>>>> the
>>>>>> >>>>>>> Runner, which is why I proposed (2).
>>>>>> >>>>>> There is no particular need for the coder to be known. If
>>>>>> transform is
>>>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded form
>>>>>> of the
>>>>>> >>>>>> elements. That holds true if:
>>>>>> >>>>>>
>>>>>> >>>>>>      a) either the SDK coder is known, or
>>>>>> >>>>>>
>>>>>> >>>>>>      b) encoded form of the produced elements is known in
>>>>>> advance
>>>>>> >>>>>>
>>>>>> >>>>>> For TestStream it is the case b). For inlined primitive Read
>>>>>> (or any
>>>>>> >>>>>> other transform which executes code) it is a).
>>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>>> reasons,
>>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>>> delimiting)
>>>>>> >>>>> and unnested. TestStream elements are given as unnested encoded
>>>>>> bytes,
>>>>>> >>>>> but the nested encoding is required for sending data to the
>>>>>> SDK. The
>>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>>> for an
>>>>>> >>>>> arbitrary unknown coder.
>>>>>> >>>>>
>>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>>> already
>>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>>> complicate the
>>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>>> sending the
>>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>>> solved
>>>>>> >>>>> differently for inlined reads as the coder can always be known
>>>>>> by the
>>>>>> >>>>> runner).)
>>>>>>
>>>>>
>>>>
>>>> --
>>>>
>>>>
>>>> Jack McCluskey
>>>> SWE - DataPLS PLAT/ Beam Go
>>>> RDU
>>>> jrmccluskey@gmail.com
>>>>
>>>>
>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
This makes a *lot* of sense. But it seems to me, that this is not the 
way Java-based runners - that use runners-core-construction-java module 
- handle it. If I interpret it correctly, then the set of ModelCoders is 
hard-coded [1] and essentially required to be known by all SDKs [2].

There seems to be no negotiation between what SDK harness knows and what 
the runner knows. The runner might be able to define the wire coder for 
the SDK (via the ProcessBundleDescriptor), but the SDK (for Java 
runners) seems not to be able to play any role in this [3]. Therefore I 
think that if an SDK does not know the set of Java ModelCoders, then the 
runner and the SDK might not agree on the binary encoding of the 
elements (BTW, which is why Java Coders cannot be part of ModelCoders 
and I finally understand why I had such troubles adding it there - it 
cannot be there).

Is it possible we are missing some part of this runner-to-sdk coder 
negotiation in runners-core-contruction-java?

[1] 
https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java#L75

[2] 
https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L62

[3] 
https://github.com/apache/beam/blob/a871a494a9935093ab5f42d88d584d32aa233b8a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L104

On 9/9/21 12:18 AM, Robert Bradshaw wrote:
> The whole notion of an absolute set of known coders is a misnomer. It 
> would require all Runners and SDKs to be updated synchronously for 
> every new coder they might want to share.
>
> Instead, what we have are
>
> * Standard Coders which have well-defined, language-agnostic 
> representations and encodings, which may be used for interoperability 
> and efficiency, and
> * Required Coders which are the minimum needed to execute the pipeline.
>
> The latter consists only of bytes (for impulse), kv and iterable (for 
> GBK), windowed value (for windowing information) and length prefix (to 
> be able to handle anything else).
>
>
> On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <robert@frantil.com 
> <ma...@frantil.com>> wrote:
>
>     Is the claim that the Standard bytes and String_utf8 coders are
>     not "known coders"?
>
>      What's the point of the standard coders if they are not the
>     canonical "known coders" that can generally be expected to be
>     known by runners/other SDKs?
>
>     https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>     <https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790>
>
>     The Go SDK rather heavily biases towards using the standard coders
>     for their closes language equivalents rather than going into
>     override/custom specified soup. (It's not possible to globally
>     override the coders for the '[]byte' and 'string' types, nor is
>     there often reason to.)
>
>     On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <robertwb@google.com
>     <ma...@google.com>> wrote:
>
>         On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey
>         <jrmccluskey@google.com <ma...@google.com>> wrote:
>
>             Hey all,
>
>             Just catching up on the thread since I did the TestStream
>             Go SDK implementation. The discussion about length
>             prefixing behavior for known vs. unknown coders is
>             interesting, since we ran into strings and byte slices
>             getting extra length prefixes attached to them by Flink
>             despite being known coders.
>
>
>         Known to who?
>
>             Based on what's been said, that isn't expected behavior,
>             right?
>
>
>         No, it's not.
>
>         I would check to make sure the Go SDK is respecting the Coder
>         (length prefixed or not) that's set on the channel, rather
>         than guessing at what it expects it to be based on the Go type.
>
>             On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský
>             <je.ik@seznam.cz <ma...@seznam.cz>> wrote:
>
>                 On 9/7/21 6:02 PM, Reuven Lax wrote:
>>                 Historically the DataflowRunner has been much more
>>                 careful about not breaking update, since this is a
>>                 frequent operation by Dataflow users. I think we've
>>                 been less careful aboutt other runners, but as we see
>>                 clearly here Fllnk users do care about this as well,
>>                 so we should probably test upgrade compatibility for
>>                 Flink.
>>
>>                 One strategy that Dataflow uses is to avoid embedding
>>                 the Java serialized form of a Coder in the graph, as
>>                 this is a much higher risk of breakage (as we see
>>                 with the issue you llnked to). Possibly similar
>>                 strategies should be investigated for Fllink.
>                 +1, that would be great!
>>
>>                 Reuven
>>
>>                 On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský
>>                 <je.ik@seznam.cz <ma...@seznam.cz>> wrote:
>>
>>                     > Unfortunately the most basic coders (e.g.
>>                     bytes, string, kv, iterable)
>>                     > care about Context because they predated this
>>                     deprecation, and
>>                     > changing coders is hard (due to no way to
>>                     update the encoding for a
>>                     > streaming pipeline).
>>                     This is unrelated, but - regarding changing
>>                     coders due to concerns about
>>                     pipeline upgrades, we break this quite often, at
>>                     least for some runners.
>>                     Most recently [1].
>>
>>                     > It is currently the latter for runners using
>>                     this code (which not all
>>                     > do, e.g. the ULR and Dataflow runners). I don't
>>                     think we want to
>>                     > ossify this decision as part of the spec. (Note
>>                     that even what's
>>                     > "known" and "unknown" can change from runner to
>>                     runner.)
>>                     This is interesting and unexpected for me. How do
>>                     runners decide about
>>                     how they encode elements between SDK harness and
>>                     the runner? How do they
>>                     inform the SDK harness about this decision? My
>>                     impression was that this
>>                     is well-defined at the model level. If not, then
>>                     we have the reason for
>>                     misunderstanding in this conversation. :-)
>>
>>                       Jan
>>
>>                     [1]
>>                     https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>                     <https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E>
>>
>>                     On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>                     > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský
>>                     <je.ik@seznam.cz <ma...@seznam.cz>> wrote:
>>                     >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>                     >>
>>                     >>> On Fri, Sep 3, 2021 at 11:42 AM Jan
>>                     Lukavský<je.ik@seznam.cz
>>                     <ma...@seznam.cz>> wrote:
>>                     >>>> Hi Robert,
>>                     >>>>
>>                     >>>>> There's another hitch here for TestStream.
>>                     For historical reasons,
>>                     >>>>> coders actually represent two encodings:
>>                     nested (aka self delimiting)
>>                     >>>>> and unnested. TestStream elements are given
>>                     as unnested encoded bytes,
>>                     >>>>> but the nested encoding is required for
>>                     sending data to the SDK. The
>>                     >>>>> runner can't go from <nested encoding> to
>>                     <unnested encoding> for an
>>                     >>>>> arbitrary unknown coder.
>>                     >>>>>
>>                     >>>>> (Even if it weren't for this complication,
>>                     to be able to send already
>>                     >>>>> encoded bytes of an unknown coder to the
>>                     SDK will also complicate the
>>                     >>>>> logic in choosing the coder to be used for
>>                     the channel and sending the
>>                     >>>>> data, which is some of what you're running
>>                     into (but can be solved
>>                     >>>>> differently for inlined reads as the coder
>>                     can always be known by the
>>                     >>>>> runner).)
>>                     >>>> It is hard for me to argue with "historical
>>                     reasons". But - the "nested"
>>                     >>>> and "unnested" coders look very similar to
>>                     SDK-coder and runner-coder
>>                     >>>> spaces.
>>                     >>> Unfortunately, they're actually orthogonal to
>>                     that.
>>                     >> Hm, do you mean the Context passed to the
>>                     encode/decode method? [1] That
>>                     >> seems to be deprecated, I assume that most
>>                     coders use the default
>>                     >> implementation and simply ignore the Context?
>>                     > Unfortunately the most basic coders (e.g.
>>                     bytes, string, kv, iterable)
>>                     > care about Context because they predated this
>>                     deprecation, and
>>                     > changing coders is hard (due to no way to
>>                     update the encoding for a
>>                     > streaming pipeline).
>>                     >
>>                     >> Even if not - whether or
>>                     >> not the elements are encoded using NESTED
>>                     Context or UNNESTED Context
>>                     >> should be part of the contract of TestStream,
>>                     right? Most likely it is
>>                     >> the UNNESTED one, if I understand correctly
>>                     what that does. Under what
>>                     >> conditions is the deprecated encode/decode
>>                     method used?
>>                     > Yes, it's the UNNESTED one.
>>                     >
>>                     >> [1]
>>                     >>
>>                     https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>                     <https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134>
>>                     >>
>>                     >>>> The runner's responsibility is not to go
>>                     from "<nested
>>                     >>>> encoding>" (SDK coder) to "<unnested
>>                     encoding>" for arbitrary coder.
>>                     >>>> That is really impossible. But a coder is a
>>                     function, right? Function
>>                     >>>> maps from universe A to universe B (in
>>                     general). TestStream provides a
>>                     >>>> set of elements, and these elements are the
>>                     "universe". For those
>>                     >>>> elements it also provides the encoded form,
>>                     which can be interpreted as
>>                     >>>> the definition of the coder.
>>                     >>> The problem here is that there is not "the
>>                     encoded form" for a Coder
>>                     >>> but two encoded forms, and we have the wrong
>>                     one. Things could be made
>>                     >>> to work if we had the other.
>>                     >> Which two encoded forms do you refer to?
>>                     Elements encoded by both the
>>                     >> SDK-coder and runner-coder (and I ignore the
>>                     Context here once again)
>>                     >> have the same binary representation (which
>>                     they must have, otherwise it
>>                     >> would be impossible to decode elements coming
>>                     from the runner to the
>>                     >> SDK-harness or vice-versa).
>>                     >>>> Therefore - technically (and formally) -
>>                     >>>> the SDK coder for the TestStream is known to
>>                     the runner, regardless of
>>                     >>>> the language the runner is written in.
>>                     >>>>
>>                     >>>> To move  this discussion forward, I think we
>>                     should look for answers to
>>                     >>>> the following questions:
>>                     >>>>
>>                     >>>>     a) do we have any clues that show, that
>>                     the proposed "in runner"
>>                     >>>> solution will not work?
>>                     >>> OK, thinking about it some more, in the
>>                     TestStream, we can use the
>>                     >>> happy coincidence that
>>                     >>>
>>                     >>>  LengthPrefixed(C).encode(x, nested=True) ==
>>                     >>> VarLong.encode(len(C.encode(x,
>>                     nested=False))) || C.encode(x,
>>                     >>> nested=False)
>>                     >>>
>>                     >>> (where || denotes concatenation) and the fact
>>                     that we have
>>                     >>>
>>                     >>>       C.encode(x, nested=False)
>>                     >>>
>>                     >>> in hand.
>>                     >>>
>>                     >>> A possible fix here for the OP's question is
>>                     that when rehydrating the
>>                     >>> TestStream transform it must behave
>>                     differently according to the coder
>>                     >>> used in the subsequent channel (e.g. for
>>                     known coders, it decodes the
>>                     >>> elements and emits them directly, but for
>>                     unknown coders, it prefixes
>>                     >>> them with their length and emits byte
>>                     strings. It gets more
>>                     >>> complicated for nested coders, e.g. for a
>>                     KV<known-coder,
>>                     >>> unknown-coder> the channel might be
>>                     LP(KV<known-coder, unknown-coder))
>>                     >>> or KV<known-coder, LP(unknown-coder)) which
>>                     have different encodings
>>                     >>> (and the latter, which is the default,
>>                     requires transcoding the bytes
>>                     >>> to inject the length in the middle which is
>>                     found by decoding the
>>                     >>> first component). As well as getting more
>>                     complex, this really seems
>>                     >>> to violate the spirit of separation of concerns.
>>                     >> How do we make the decision if the channel is
>>                     LP<KV<..>> or
>>                     >> KV<LP<unknown>, known>? From my understanding
>>                     it is always the latter,
>>                     >> because of [2].
>>                     >>
>>                     >> [2]
>>                     >>
>>                     https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>                     <https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48>
>>                     > It is currently the latter for runners using
>>                     this code (which not all
>>                     > do, e.g. the ULR and Dataflow runners). I don't
>>                     think we want to
>>                     > ossify this decision as part of the spec. (Note
>>                     that even what's
>>                     > "known" and "unknown" can change from runner to
>>                     runner.)
>>                     >
>>                     >>>>     b) do we think, that it will not be
>>                     robust enough to incorporate the
>>                     >>>> other use-cases (line generic transform
>>                     inlining, taking into account
>>                     >>>> that this applies only to runners that are
>>                     written in the same language
>>                     >>>> as the submitting SDK, because otherwise,
>>                     there is nothing to inline)?
>>                     >>> Being in the same language is not a
>>                     prerequisite to "inlining," e.g.
>>                     >>> the PubSub source on Dataflow is recognized
>>                     as such and not executed
>>                     >>> as SDK code but natively.
>>                     >> Agree, that is actually exactly what happens
>>                     with the TestStream. The
>>                     >> transform need not be in the same language, as
>>                     long as it is completely
>>                     >> understood by the runner, including the
>>                     SDK-coder (either explicitly -
>>                     >> which might be due to the PCollection coder
>>                     being composed of well-known
>>                     >> coders only, or implicitly like in the case of
>>                     TestStream, where the
>>                     >> elements are encoded using the SDK coder.
>>                     >>> It is more likely that inlining occurs in the
>>                     same language if there
>>                     >>> are UDFs involved.
>>                     >>>
>>                     >>>> I'm convinced, that the TestStream-decode
>>                     expansion solution is an
>>                     >>>> ad-hoc solution to a generic problem, which
>>                     is why I'm still bothering
>>                     >>>> this mailing list with my emails on this. :-)
>>                     >>>>
>>                     >>>> WDYT?
>>                     >>> While not a solution to the general problem,
>>                     I think the
>>                     >>> TestStream-only-does-bytes simplifies its
>>                     definition (primitives
>>                     >>> should have as simple/easy to implement
>>                     definitions as possible) and
>>                     >>> brings it closer to the other root we have:
>>                     Impulse. (We could go a
>>                     >>> step further and rather than emitting encoded
>>                     elements, with the data
>>                     >>> in the proto itself, it emits sequence
>>                     numbers, and a subsequent ParDo
>>                     >>> maps those to concrete elements (e.g. via an
>>                     in-memory map), but that
>>                     >>> further step doesn't buy much...)
>>                     >>>
>>                     >>> Only runners that want to do inlining would
>>                     have to take on the
>>                     >>> complexity of a fully generic solution.
>>                     >> I think that if the simplification brings
>>                     something, we can do that, but
>>                     >> I'd like to understand why we cannot (or
>>                     should not) use the generic
>>                     >> solution. I think it definitely *should* be
>>                     possible to use a generic
>>                     >> solution, because otherwise the solution would
>>                     not be generic. And it
>>                     >> would imply, that we are unable to do generic
>>                     transform inlining, which
>>                     >> I would find really strange. That would
>>                     immediately mean, that we are
>>                     >> unable to construct classical runner as a
>>                     special case of the portable
>>                     >> one, which would be bad I think.
>>                     >>
>>                     >> The elements in the TestStreamPayload are
>>                     encoded with pure SDK-coder,
>>                     >> or does this go through the
>>                     LengthPrefixUnknownCoders logic? If not,
>>                     >> then the problem would be there, because that
>>                     means, that the SDK-coder
>>                     >> cannot be (implicitly) defined in the runner.
>>                     If the elements would be
>>                     >> encoded using LP, then it would be possible to
>>                     decode them using
>>                     >> runner-coder and the problem should be solved,
>>                     or am I still missing
>>                     >> some key parts?
>>                     > Yes, the problem is precisely that there are
>>                     (unspecified) constraints
>>                     > on the coder used by the TestStreamPayload.
>>                     Just requiring that it be
>>                     > length prefixed is not enough, you have to make
>>                     constraints on
>>                     > sometimes pushing down the length prefixing if
>>                     it's a composite (like
>>                     > a KV) that depend on what the runner is
>>                     expected to support in terms
>>                     > of composites and/or the choices it chooses for
>>                     the channel (and the
>>                     > runner, not knowing the coder, can't transcode
>>                     between these choices).
>>                     >
>>                     > The simpler solution is to constrain this coder
>>                     to just be byte[]
>>                     > rather than let it be a little bit flexible
>>                     (but not wholly flexible).
>>                     >
>>                     > As for a fully generic solution, I think the
>>                     issue encountered with
>>                     > inlining Read vs. TestStream are related to
>>                     this, but not really the
>>                     > same. With TestStream one has an encoded
>>                     representation of the
>>                     > elements provided by the SDK that the Runner
>>                     and has no SDK
>>                     > representation/execution whereas with the Reads
>>                     one has unencoded
>>                     > elements in hand and a Coder that is understood
>>                     by both (so long as
>>                     > the channel can be negotiated correctly). FWIW,
>>                     I think the proper
>>                     > solution to inlining a Read (or other Transform
>>                     that would typically
>>                     > be executed in the SDK) is to treat it as a
>>                     special environment (where
>>                     > we know logically it can work) and then elide,
>>                     as possible, the
>>                     > various encodings, grpc calls, etc. that are
>>                     unneeded as everything is
>>                     > in process.
>>                     >
>>                     >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>                     >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan
>>                     Lukavský<je.ik@seznam.cz
>>                     <ma...@seznam.cz>> wrote:
>>                     >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>                     >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan
>>                     Lukavský<je.ik@seznam.cz
>>                     <ma...@seznam.cz>> wrote:
>>                     >>>>>>>> Hi,
>>                     >>>>>>>>
>>                     >>>>>>>> I had some more time thinking about this
>>                     and I'll try to recap that.
>>                     >>>>>>>> First some invariants:
>>                     >>>>>>>>
>>                     >>>>>>>>    a) each PCollection<T> has actually
>>                     two coders - an _SDK coder_ and a
>>                     >>>>>>>> _runner coder_. These coders have the
>>                     property, that each one can
>>                     >>>>>>>> _decode_ what the other encoded, but the
>>                     opposite is not true, the
>>                     >>>>>>>> coders cannot _encode_ what the other
>>                     _decoded_ (in general).
>>                     >>>>>>>>
>>                     >>>>>>>>    b) when is a PCollection<T> computed
>>                     inside an environment, the
>>                     >>>>>>>> elements are encoded using SDK coder on
>>                     the side of SDK-harness and
>>                     >>>>>>>> decoded using runner coder after
>>                     receiving in the runner
>>                     >>>>>>>>
>>                     >>>>>>>>    c) under specific circumstances, the
>>                     encode-decode step can be
>>                     >>>>>>>> optimized out, that is the case where
>>                     the SDK coder and all its
>>                     >>>>>>>> subcoders are all well-known to the
>>                     runner (in the present, that means
>>                     >>>>>>>> that all the parts present in the model
>>                     coders set). The reason for that
>>                     >>>>>>>> is that in this specific situation
>>                     runner_decode(sdk_encode(X)) = X.
>>                     >>>>>>>> This property is essential.
>>                     >>>>>>> However, in general, X can only pass from
>>                     the SDK to the runner (or
>>                     >>>>>>> vice versa) in encoded form.
>>                     >>>>>> In general yes, but we are (mostly)
>>                     talking transform inlining here, so
>>                     >>>>>> it that particular situation, the elements
>>                     might be passed in decoded form.
>>                     >>>>>>>>    d) from b) immediately follows, that
>>                     when a PTransform does not run in
>>                     >>>>>>>> an environment (and this might be due to
>>                     the transform being runner
>>                     >>>>>>>> native, inlined, source (e.g. Impulse or
>>                     TestStream)) the elements have
>>                     >>>>>>>> to be encoded by SDK coder, immediately
>>                     following decode by runner
>>                     >>>>>>>> coder. That (surprisingly) applies even
>>                     to situations when runner is
>>                     >>>>>>>> implemented using different language
>>                     than the client SDK, because it
>>                     >>>>>>>> implies that the type of produced
>>                     elements must be one of types encoded
>>                     >>>>>>>> using model coders (well-known to the
>>                     runner, otherwise the SDK will not
>>                     >>>>>>>> be able to consume it). But - due to
>>                     property c) - this means that this
>>                     >>>>>>>> encode-decode step can be optimized out.
>>                     This does not mean that it is
>>                     >>>>>>>> not (logically) present, though. This is
>>                     exactly the case of native
>>                     >>>>>>>> Impulse transform.
>>                     >>>>>>>>
>>                     >>>>>>>> Now, from that we can conclude that on
>>                     the boundary between executable
>>                     >>>>>>>> stages, or between runner (inlined)
>>                     transform and executable stage, each
>>                     >>>>>>>> PCollection has to be encoded using SDK
>>                     coder and immediately decoded by
>>                     >>>>>>>> runner coder, *unless this can be
>>                     optimized out* by property c).
>>                     >>>>>>>>
>>                     >>>>>>>> This gives us two options where to
>>                     implement this encode/decode step:
>>                     >>>>>>>>
>>                     >>>>>>>>    1) completely inside runner with the
>>                     possibility to optimize the
>>                     >>>>>>>> encode/decode step by identity under
>>                     right circumstances
>>                     >>>>>>>>
>>                     >>>>>>>>    2) partly in the runner and partly in
>>                     the SDK - that is we encode
>>                     >>>>>>>> elements of PCollection using SDK coder
>>                     into bytes, pass those to the
>>                     >>>>>>>> SDK harness and apply a custom decode
>>                     step there. This works because SDK
>>                     >>>>>>>> coder encoded elements are in byte[],
>>                     and that is well-known coder type.
>>                     >>>>>>>> We again only leverage property c) and
>>                     optimize the SDK coder encode,
>>                     >>>>>>>> runner decode step out.
>>                     >>>>>>>>
>>                     >>>>>>>> The option 2) is exactly the proposal of
>>                     TestStream producing byte[] and
>>                     >>>>>>>> decoding inside SDK-harness, the
>>                     TestStream is actually inlined
>>                     >>>>>>>> transform, the elements are produced
>>                     directly in runner (the SDK coder
>>                     >>>>>>>> is not known to the runner, but that
>>                     does not matter, because the
>>                     >>>>>>>> elements are already encoded by client).
>>                     >>>>>>>>
>>                     >>>>>>>>   From the above it seems to me, that
>>                     option 1) should be preferred, because:
>>                     >>>>>>>>
>>                     >>>>>>>>    i) it is generic, applicable to all
>>                     inlined transforms, any sources
>>                     >>>>>>>>
>>                     >>>>>>>>    ii) it is consistent with how things
>>                     logically work underneath
>>                     >>>>>>>>
>>                     >>>>>>>>    iii) it offers better room for
>>                     optimization - option 2) might result
>>                     >>>>>>>> in cases when the elements are passed
>>                     from the runner to the SDK-harness
>>                     >>>>>>>> only for the sake of the decoding from
>>                     SDK coder and immediately
>>                     >>>>>>>> encoding back using SDK-coder and
>>                     returned back to the runner. This
>>                     >>>>>>>> would be the case when TestStream would
>>                     be directly consumed by inlined
>>                     >>>>>>>> (or external) transform.
>>                     >>>>>>> (1) is not possible if the Coder in
>>                     question is not known to the
>>                     >>>>>>> Runner, which is why I proposed (2).
>>                     >>>>>> There is no particular need for the coder
>>                     to be known. If transform is
>>                     >>>>>> to be inlined, what *has* to be known is
>>                     the SDK-encoded form of the
>>                     >>>>>> elements. That holds true if:
>>                     >>>>>>
>>                     >>>>>>      a) either the SDK coder is known, or
>>                     >>>>>>
>>                     >>>>>>      b) encoded form of the produced
>>                     elements is known in advance
>>                     >>>>>>
>>                     >>>>>> For TestStream it is the case b). For
>>                     inlined primitive Read (or any
>>                     >>>>>> other transform which executes code) it is a).
>>                     >>>>> There's another hitch here for TestStream.
>>                     For historical reasons,
>>                     >>>>> coders actually represent two encodings:
>>                     nested (aka self delimiting)
>>                     >>>>> and unnested. TestStream elements are given
>>                     as unnested encoded bytes,
>>                     >>>>> but the nested encoding is required for
>>                     sending data to the SDK. The
>>                     >>>>> runner can't go from <nested encoding> to
>>                     <unnested encoding> for an
>>                     >>>>> arbitrary unknown coder.
>>                     >>>>>
>>                     >>>>> (Even if it weren't for this complication,
>>                     to be able to send already
>>                     >>>>> encoded bytes of an unknown coder to the
>>                     SDK will also complicate the
>>                     >>>>> logic in choosing the coder to be used for
>>                     the channel and sending the
>>                     >>>>> data, which is some of what you're running
>>                     into (but can be solved
>>                     >>>>> differently for inlined reads as the coder
>>                     can always be known by the
>>                     >>>>> runner).)
>>
>
>
>             -- 
>
>             	
>
>             Jack McCluskey
>             SWE - DataPLS PLAT/ Beam Go
>             RDU
>             jrmccluskey@gmail.com <ma...@gmail.com>
>
>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
The whole notion of an absolute set of known coders is a misnomer. It would
require all Runners and SDKs to be updated synchronously for every new
coder they might want to share.

Instead, what we have are

* Standard Coders which have well-defined, language-agnostic
representations and encodings, which may be used for interoperability and
efficiency, and
* Required Coders which are the minimum needed to execute the pipeline.

The latter consists only of bytes (for impulse), kv and iterable (for GBK),
windowed value (for windowing information) and length prefix (to be able to
handle anything else).


On Wed, Sep 8, 2021 at 3:03 PM Robert Burke <ro...@frantil.com> wrote:

> Is the claim that the Standard bytes and String_utf8 coders are not "known
> coders"?
>
>  What's the point of the standard coders if they are not the canonical
> "known coders" that can generally be expected to be known by runners/other
> SDKs?
>
>
> https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790
>
> The Go SDK rather heavily biases towards using the standard coders for
> their closes language equivalents rather than going into override/custom
> specified soup. (It's not possible to globally override the coders for the
> '[]byte' and 'string' types, nor is there often reason to.)
>
> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com> wrote:
>
>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <jr...@google.com>
>> wrote:
>>
>>> Hey all,
>>>
>>> Just catching up on the thread since I did the TestStream Go SDK
>>> implementation. The discussion about length prefixing behavior for known
>>> vs. unknown coders is interesting, since we ran into strings and byte
>>> slices getting extra length prefixes attached to them by Flink despite
>>> being known coders.
>>>
>>
>> Known to who?
>>
>>
>>> Based on what's been said, that isn't expected behavior, right?
>>>
>>
>> No, it's not.
>>
>> I would check to make sure the Go SDK is respecting the Coder (length
>> prefixed or not) that's set on the channel, rather than guessing at what it
>> expects it to be based on the Go type.
>>
>>
>>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>
>>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>>
>>>> Historically the DataflowRunner has been much more careful about not
>>>> breaking update, since this is a frequent operation by Dataflow users. I
>>>> think we've been less careful aboutt other runners, but as we see clearly
>>>> here Fllnk users do care about this as well, so we should probably test
>>>> upgrade compatibility for Flink.
>>>>
>>>> One strategy that Dataflow uses is to avoid embedding the Java
>>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>>> breakage (as we see with the issue you llnked to). Possibly similar
>>>> strategies should be investigated for Fllink.
>>>>
>>>> +1, that would be great!
>>>>
>>>>
>>>> Reuven
>>>>
>>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>
>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>> iterable)
>>>>> > care about Context because they predated this deprecation, and
>>>>> > changing coders is hard (due to no way to update the encoding for a
>>>>> > streaming pipeline).
>>>>> This is unrelated, but - regarding changing coders due to concerns
>>>>> about
>>>>> pipeline upgrades, we break this quite often, at least for some
>>>>> runners.
>>>>> Most recently [1].
>>>>>
>>>>> > It is currently the latter for runners using this code (which not all
>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>> This is interesting and unexpected for me. How do runners decide about
>>>>> how they encode elements between SDK harness and the runner? How do
>>>>> they
>>>>> inform the SDK harness about this decision? My impression was that
>>>>> this
>>>>> is well-defined at the model level. If not, then we have the reason
>>>>> for
>>>>> misunderstanding in this conversation. :-)
>>>>>
>>>>>   Jan
>>>>>
>>>>> [1]
>>>>>
>>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>>
>>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>>> >>
>>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>>> wrote:
>>>>> >>>> Hi Robert,
>>>>> >>>>
>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>> reasons,
>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>> delimiting)
>>>>> >>>>> and unnested. TestStream elements are given as unnested encoded
>>>>> bytes,
>>>>> >>>>> but the nested encoding is required for sending data to the SDK.
>>>>> The
>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>> for an
>>>>> >>>>> arbitrary unknown coder.
>>>>> >>>>>
>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>> already
>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>> complicate the
>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>> sending the
>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>> solved
>>>>> >>>>> differently for inlined reads as the coder can always be known
>>>>> by the
>>>>> >>>>> runner).)
>>>>> >>>> It is hard for me to argue with "historical reasons". But - the
>>>>> "nested"
>>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>>> runner-coder
>>>>> >>>> spaces.
>>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>>> >> Hm, do you mean the Context passed to the encode/decode method? [1]
>>>>> That
>>>>> >> seems to be deprecated, I assume that most coders use the default
>>>>> >> implementation and simply ignore the Context?
>>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv,
>>>>> iterable)
>>>>> > care about Context because they predated this deprecation, and
>>>>> > changing coders is hard (due to no way to update the encoding for a
>>>>> > streaming pipeline).
>>>>> >
>>>>> >> Even if not - whether or
>>>>> >> not the elements are encoded using NESTED Context or UNNESTED
>>>>> Context
>>>>> >> should be part of the contract of TestStream, right? Most likely it
>>>>> is
>>>>> >> the UNNESTED one, if I understand correctly what that does. Under
>>>>> what
>>>>> >> conditions is the deprecated encode/decode method used?
>>>>> > Yes, it's the UNNESTED one.
>>>>> >
>>>>> >> [1]
>>>>> >>
>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>>> >>
>>>>> >>>> The runner's responsibility is not to go from "<nested
>>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary
>>>>> coder.
>>>>> >>>> That is really impossible. But a coder is a function, right?
>>>>> Function
>>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>>> provides a
>>>>> >>>> set of elements, and these elements are the "universe". For those
>>>>> >>>> elements it also provides the encoded form, which can be
>>>>> interpreted as
>>>>> >>>> the definition of the coder.
>>>>> >>> The problem here is that there is not "the encoded form" for a
>>>>> Coder
>>>>> >>> but two encoded forms, and we have the wrong one. Things could be
>>>>> made
>>>>> >>> to work if we had the other.
>>>>> >> Which two encoded forms do you refer to? Elements encoded by both
>>>>> the
>>>>> >> SDK-coder and runner-coder (and I ignore the Context here once
>>>>> again)
>>>>> >> have the same binary representation (which they must have,
>>>>> otherwise it
>>>>> >> would be impossible to decode elements coming from the runner to the
>>>>> >> SDK-harness or vice-versa).
>>>>> >>>> Therefore - technically (and formally) -
>>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>>> regardless of
>>>>> >>>> the language the runner is written in.
>>>>> >>>>
>>>>> >>>> To move  this discussion forward, I think we should look for
>>>>> answers to
>>>>> >>>> the following questions:
>>>>> >>>>
>>>>> >>>>     a) do we have any clues that show, that the proposed "in
>>>>> runner"
>>>>> >>>> solution will not work?
>>>>> >>> OK, thinking about it some more, in the TestStream, we can use the
>>>>> >>> happy coincidence that
>>>>> >>>
>>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>>> >>> nested=False)
>>>>> >>>
>>>>> >>> (where || denotes concatenation) and the fact that we have
>>>>> >>>
>>>>> >>>       C.encode(x, nested=False)
>>>>> >>>
>>>>> >>> in hand.
>>>>> >>>
>>>>> >>> A possible fix here for the OP's question is that when rehydrating
>>>>> the
>>>>> >>> TestStream transform it must behave differently according to the
>>>>> coder
>>>>> >>> used in the subsequent channel (e.g. for known coders, it decodes
>>>>> the
>>>>> >>> elements and emits them directly, but for unknown coders, it
>>>>> prefixes
>>>>> >>> them with their length and emits byte strings. It gets more
>>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>>> unknown-coder))
>>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different
>>>>> encodings
>>>>> >>> (and the latter, which is the default, requires transcoding the
>>>>> bytes
>>>>> >>> to inject the length in the middle which is found by decoding the
>>>>> >>> first component). As well as getting more complex, this really
>>>>> seems
>>>>> >>> to violate the spirit of separation of concerns.
>>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>>> >> KV<LP<unknown>, known>? From my understanding it is always the
>>>>> latter,
>>>>> >> because of [2].
>>>>> >>
>>>>> >> [2]
>>>>> >>
>>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>> > It is currently the latter for runners using this code (which not all
>>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>>> > ossify this decision as part of the spec. (Note that even what's
>>>>> > "known" and "unknown" can change from runner to runner.)
>>>>> >
>>>>> >>>>     b) do we think, that it will not be robust enough to
>>>>> incorporate the
>>>>> >>>> other use-cases (line generic transform inlining, taking into
>>>>> account
>>>>> >>>> that this applies only to runners that are written in the same
>>>>> language
>>>>> >>>> as the submitting SDK, because otherwise, there is nothing to
>>>>> inline)?
>>>>> >>> Being in the same language is not a prerequisite to "inlining,"
>>>>> e.g.
>>>>> >>> the PubSub source on Dataflow is recognized as such and not
>>>>> executed
>>>>> >>> as SDK code but natively.
>>>>> >> Agree, that is actually exactly what happens with the TestStream.
>>>>> The
>>>>> >> transform need not be in the same language, as long as it is
>>>>> completely
>>>>> >> understood by the runner, including the SDK-coder (either
>>>>> explicitly -
>>>>> >> which might be due to the PCollection coder being composed of
>>>>> well-known
>>>>> >> coders only, or implicitly like in the case of TestStream, where the
>>>>> >> elements are encoded using the SDK coder.
>>>>> >>> It is more likely that inlining occurs in the same language if
>>>>> there
>>>>> >>> are UDFs involved.
>>>>> >>>
>>>>> >>>> I'm convinced, that the TestStream-decode expansion solution is an
>>>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>>>> bothering
>>>>> >>>> this mailing list with my emails on this. :-)
>>>>> >>>>
>>>>> >>>> WDYT?
>>>>> >>> While not a solution to the general problem, I think the
>>>>> >>> TestStream-only-does-bytes simplifies its definition (primitives
>>>>> >>> should have as simple/easy to implement definitions as possible)
>>>>> and
>>>>> >>> brings it closer to the other root we have: Impulse. (We could go a
>>>>> >>> step further and rather than emitting encoded elements, with the
>>>>> data
>>>>> >>> in the proto itself, it emits sequence numbers, and a subsequent
>>>>> ParDo
>>>>> >>> maps those to concrete elements (e.g. via an in-memory map), but
>>>>> that
>>>>> >>> further step doesn't buy much...)
>>>>> >>>
>>>>> >>> Only runners that want to do inlining would have to take on the
>>>>> >>> complexity of a fully generic solution.
>>>>> >> I think that if the simplification brings something, we can do
>>>>> that, but
>>>>> >> I'd like to understand why we cannot (or should not) use the generic
>>>>> >> solution. I think it definitely *should* be possible to use a
>>>>> generic
>>>>> >> solution, because otherwise the solution would not be generic. And
>>>>> it
>>>>> >> would imply, that we are unable to do generic transform inlining,
>>>>> which
>>>>> >> I would find really strange. That would immediately mean, that we
>>>>> are
>>>>> >> unable to construct classical runner as a special case of the
>>>>> portable
>>>>> >> one, which would be bad I think.
>>>>> >>
>>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>>> SDK-coder,
>>>>> >> or does this go through the LengthPrefixUnknownCoders logic? If not,
>>>>> >> then the problem would be there, because that means, that the
>>>>> SDK-coder
>>>>> >> cannot be (implicitly) defined in the runner. If the elements would
>>>>> be
>>>>> >> encoded using LP, then it would be possible to decode them using
>>>>> >> runner-coder and the problem should be solved, or am I still missing
>>>>> >> some key parts?
>>>>> > Yes, the problem is precisely that there are (unspecified)
>>>>> constraints
>>>>> > on the coder used by the TestStreamPayload. Just requiring that it be
>>>>> > length prefixed is not enough, you have to make constraints on
>>>>> > sometimes pushing down the length prefixing if it's a composite (like
>>>>> > a KV) that depend on what the runner is expected to support in terms
>>>>> > of composites and/or the choices it chooses for the channel (and the
>>>>> > runner, not knowing the coder, can't transcode between these
>>>>> choices).
>>>>> >
>>>>> > The simpler solution is to constrain this coder to just be byte[]
>>>>> > rather than let it be a little bit flexible (but not wholly
>>>>> flexible).
>>>>> >
>>>>> > As for a fully generic solution, I think the issue encountered with
>>>>> > inlining Read vs. TestStream are related to this, but not really the
>>>>> > same. With TestStream one has an encoded representation of the
>>>>> > elements provided by the SDK that the Runner and has no SDK
>>>>> > representation/execution whereas with the Reads one has unencoded
>>>>> > elements in hand and a Coder that is understood by both (so long as
>>>>> > the channel can be negotiated correctly). FWIW, I think the proper
>>>>> > solution to inlining a Read (or other Transform that would typically
>>>>> > be executed in the SDK) is to treat it as a special environment
>>>>> (where
>>>>> > we know logically it can work) and then elide, as possible, the
>>>>> > various encodings, grpc calls, etc. that are unneeded as everything
>>>>> is
>>>>> > in process.
>>>>> >
>>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>>>> wrote:
>>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
>>>>> wrote:
>>>>> >>>>>>>> Hi,
>>>>> >>>>>>>>
>>>>> >>>>>>>> I had some more time thinking about this and I'll try to
>>>>> recap that.
>>>>> >>>>>>>> First some invariants:
>>>>> >>>>>>>>
>>>>> >>>>>>>>       a) each PCollection<T> has actually two coders - an
>>>>> _SDK coder_ and a
>>>>> >>>>>>>> _runner coder_. These coders have the property, that each one
>>>>> can
>>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not
>>>>> true, the
>>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in general).
>>>>> >>>>>>>>
>>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>>> environment, the
>>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>>> SDK-harness and
>>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>>> >>>>>>>>
>>>>> >>>>>>>>       c) under specific circumstances, the encode-decode step
>>>>> can be
>>>>> >>>>>>>> optimized out, that is the case where the SDK coder and all
>>>>> its
>>>>> >>>>>>>> subcoders are all well-known to the runner (in the present,
>>>>> that means
>>>>> >>>>>>>> that all the parts present in the model coders set). The
>>>>> reason for that
>>>>> >>>>>>>> is that in this specific situation
>>>>> runner_decode(sdk_encode(X)) = X.
>>>>> >>>>>>>> This property is essential.
>>>>> >>>>>>> However, in general, X can only pass from the SDK to the
>>>>> runner (or
>>>>> >>>>>>> vice versa) in encoded form.
>>>>> >>>>>> In general yes, but we are (mostly) talking transform inlining
>>>>> here, so
>>>>> >>>>>> it that particular situation, the elements might be passed in
>>>>> decoded form.
>>>>> >>>>>>>>       d) from b) immediately follows, that when a PTransform
>>>>> does not run in
>>>>> >>>>>>>> an environment (and this might be due to the transform being
>>>>> runner
>>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>>>>> elements have
>>>>> >>>>>>>> to be encoded by SDK coder, immediately following decode by
>>>>> runner
>>>>> >>>>>>>> coder. That (surprisingly) applies even to situations when
>>>>> runner is
>>>>> >>>>>>>> implemented using different language than the client SDK,
>>>>> because it
>>>>> >>>>>>>> implies that the type of produced elements must be one of
>>>>> types encoded
>>>>> >>>>>>>> using model coders (well-known to the runner, otherwise the
>>>>> SDK will not
>>>>> >>>>>>>> be able to consume it). But - due to property c) - this means
>>>>> that this
>>>>> >>>>>>>> encode-decode step can be optimized out. This does not mean
>>>>> that it is
>>>>> >>>>>>>> not (logically) present, though. This is exactly the case of
>>>>> native
>>>>> >>>>>>>> Impulse transform.
>>>>> >>>>>>>>
>>>>> >>>>>>>> Now, from that we can conclude that on the boundary between
>>>>> executable
>>>>> >>>>>>>> stages, or between runner (inlined) transform and executable
>>>>> stage, each
>>>>> >>>>>>>> PCollection has to be encoded using SDK coder and immediately
>>>>> decoded by
>>>>> >>>>>>>> runner coder, *unless this can be optimized out* by property
>>>>> c).
>>>>> >>>>>>>>
>>>>> >>>>>>>> This gives us two options where to implement this
>>>>> encode/decode step:
>>>>> >>>>>>>>
>>>>> >>>>>>>>       1) completely inside runner with the possibility to
>>>>> optimize the
>>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>>> >>>>>>>>
>>>>> >>>>>>>>       2) partly in the runner and partly in the SDK - that is
>>>>> we encode
>>>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass
>>>>> those to the
>>>>> >>>>>>>> SDK harness and apply a custom decode step there. This works
>>>>> because SDK
>>>>> >>>>>>>> coder encoded elements are in byte[], and that is well-known
>>>>> coder type.
>>>>> >>>>>>>> We again only leverage property c) and optimize the SDK coder
>>>>> encode,
>>>>> >>>>>>>> runner decode step out.
>>>>> >>>>>>>>
>>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream producing
>>>>> byte[] and
>>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>>>>> inlined
>>>>> >>>>>>>> transform, the elements are produced directly in runner (the
>>>>> SDK coder
>>>>> >>>>>>>> is not known to the runner, but that does not matter, because
>>>>> the
>>>>> >>>>>>>> elements are already encoded by client).
>>>>> >>>>>>>>
>>>>> >>>>>>>>      From the above it seems to me, that option 1) should be
>>>>> preferred, because:
>>>>> >>>>>>>>
>>>>> >>>>>>>>       i) it is generic, applicable to all inlined transforms,
>>>>> any sources
>>>>> >>>>>>>>
>>>>> >>>>>>>>       ii) it is consistent with how things logically work
>>>>> underneath
>>>>> >>>>>>>>
>>>>> >>>>>>>>       iii) it offers better room for optimization - option 2)
>>>>> might result
>>>>> >>>>>>>> in cases when the elements are passed from the runner to the
>>>>> SDK-harness
>>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>>> immediately
>>>>> >>>>>>>> encoding back using SDK-coder and returned back to the
>>>>> runner. This
>>>>> >>>>>>>> would be the case when TestStream would be directly consumed
>>>>> by inlined
>>>>> >>>>>>>> (or external) transform.
>>>>> >>>>>>> (1) is not possible if the Coder in question is not known to
>>>>> the
>>>>> >>>>>>> Runner, which is why I proposed (2).
>>>>> >>>>>> There is no particular need for the coder to be known. If
>>>>> transform is
>>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded form
>>>>> of the
>>>>> >>>>>> elements. That holds true if:
>>>>> >>>>>>
>>>>> >>>>>>      a) either the SDK coder is known, or
>>>>> >>>>>>
>>>>> >>>>>>      b) encoded form of the produced elements is known in
>>>>> advance
>>>>> >>>>>>
>>>>> >>>>>> For TestStream it is the case b). For inlined primitive Read
>>>>> (or any
>>>>> >>>>>> other transform which executes code) it is a).
>>>>> >>>>> There's another hitch here for TestStream. For historical
>>>>> reasons,
>>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>>> delimiting)
>>>>> >>>>> and unnested. TestStream elements are given as unnested encoded
>>>>> bytes,
>>>>> >>>>> but the nested encoding is required for sending data to the SDK.
>>>>> The
>>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding>
>>>>> for an
>>>>> >>>>> arbitrary unknown coder.
>>>>> >>>>>
>>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>>> already
>>>>> >>>>> encoded bytes of an unknown coder to the SDK will also
>>>>> complicate the
>>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>>> sending the
>>>>> >>>>> data, which is some of what you're running into (but can be
>>>>> solved
>>>>> >>>>> differently for inlined reads as the coder can always be known
>>>>> by the
>>>>> >>>>> runner).)
>>>>>
>>>>
>>>
>>> --
>>>
>>>
>>> Jack McCluskey
>>> SWE - DataPLS PLAT/ Beam Go
>>> RDU
>>> jrmccluskey@gmail.com
>>>
>>>
>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Burke <ro...@frantil.com>.
Is the claim that the Standard bytes and String_utf8 coders are not "known
coders"?

 What's the point of the standard coders if they are not the canonical
"known coders" that can generally be expected to be known by runners/other
SDKs?

https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/beam_runner_api.proto#L790

The Go SDK rather heavily biases towards using the standard coders for
their closes language equivalents rather than going into override/custom
specified soup. (It's not possible to globally override the coders for the
'[]byte' and 'string' types, nor is there often reason to.)

On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw <ro...@google.com> wrote:

> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <jr...@google.com>
> wrote:
>
>> Hey all,
>>
>> Just catching up on the thread since I did the TestStream Go SDK
>> implementation. The discussion about length prefixing behavior for known
>> vs. unknown coders is interesting, since we ran into strings and byte
>> slices getting extra length prefixes attached to them by Flink despite
>> being known coders.
>>
>
> Known to who?
>
>
>> Based on what's been said, that isn't expected behavior, right?
>>
>
> No, it's not.
>
> I would check to make sure the Go SDK is respecting the Coder (length
> prefixed or not) that's set on the channel, rather than guessing at what it
> expects it to be based on the Go type.
>
>
>> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:
>>
>>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>>
>>> Historically the DataflowRunner has been much more careful about not
>>> breaking update, since this is a frequent operation by Dataflow users. I
>>> think we've been less careful aboutt other runners, but as we see clearly
>>> here Fllnk users do care about this as well, so we should probably test
>>> upgrade compatibility for Flink.
>>>
>>> One strategy that Dataflow uses is to avoid embedding the Java
>>> serialized form of a Coder in the graph, as this is a much higher risk of
>>> breakage (as we see with the issue you llnked to). Possibly similar
>>> strategies should be investigated for Fllink.
>>>
>>> +1, that would be great!
>>>
>>>
>>> Reuven
>>>
>>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>
>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
>>>> > care about Context because they predated this deprecation, and
>>>> > changing coders is hard (due to no way to update the encoding for a
>>>> > streaming pipeline).
>>>> This is unrelated, but - regarding changing coders due to concerns
>>>> about
>>>> pipeline upgrades, we break this quite often, at least for some
>>>> runners.
>>>> Most recently [1].
>>>>
>>>> > It is currently the latter for runners using this code (which not all
>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>> > ossify this decision as part of the spec. (Note that even what's
>>>> > "known" and "unknown" can change from runner to runner.)
>>>> This is interesting and unexpected for me. How do runners decide about
>>>> how they encode elements between SDK harness and the runner? How do
>>>> they
>>>> inform the SDK harness about this decision? My impression was that this
>>>> is well-defined at the model level. If not, then we have the reason for
>>>> misunderstanding in this conversation. :-)
>>>>
>>>>   Jan
>>>>
>>>> [1]
>>>>
>>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>>
>>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>>> >>
>>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>>> wrote:
>>>> >>>> Hi Robert,
>>>> >>>>
>>>> >>>>> There's another hitch here for TestStream. For historical reasons,
>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>> delimiting)
>>>> >>>>> and unnested. TestStream elements are given as unnested encoded
>>>> bytes,
>>>> >>>>> but the nested encoding is required for sending data to the SDK.
>>>> The
>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding> for
>>>> an
>>>> >>>>> arbitrary unknown coder.
>>>> >>>>>
>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>> already
>>>> >>>>> encoded bytes of an unknown coder to the SDK will also complicate
>>>> the
>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>> sending the
>>>> >>>>> data, which is some of what you're running into (but can be solved
>>>> >>>>> differently for inlined reads as the coder can always be known by
>>>> the
>>>> >>>>> runner).)
>>>> >>>> It is hard for me to argue with "historical reasons". But - the
>>>> "nested"
>>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>>> runner-coder
>>>> >>>> spaces.
>>>> >>> Unfortunately, they're actually orthogonal to that.
>>>> >> Hm, do you mean the Context passed to the encode/decode method? [1]
>>>> That
>>>> >> seems to be deprecated, I assume that most coders use the default
>>>> >> implementation and simply ignore the Context?
>>>> > Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
>>>> > care about Context because they predated this deprecation, and
>>>> > changing coders is hard (due to no way to update the encoding for a
>>>> > streaming pipeline).
>>>> >
>>>> >> Even if not - whether or
>>>> >> not the elements are encoded using NESTED Context or UNNESTED Context
>>>> >> should be part of the contract of TestStream, right? Most likely it
>>>> is
>>>> >> the UNNESTED one, if I understand correctly what that does. Under
>>>> what
>>>> >> conditions is the deprecated encode/decode method used?
>>>> > Yes, it's the UNNESTED one.
>>>> >
>>>> >> [1]
>>>> >>
>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>>> >>
>>>> >>>> The runner's responsibility is not to go from "<nested
>>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary
>>>> coder.
>>>> >>>> That is really impossible. But a coder is a function, right?
>>>> Function
>>>> >>>> maps from universe A to universe B (in general). TestStream
>>>> provides a
>>>> >>>> set of elements, and these elements are the "universe". For those
>>>> >>>> elements it also provides the encoded form, which can be
>>>> interpreted as
>>>> >>>> the definition of the coder.
>>>> >>> The problem here is that there is not "the encoded form" for a Coder
>>>> >>> but two encoded forms, and we have the wrong one. Things could be
>>>> made
>>>> >>> to work if we had the other.
>>>> >> Which two encoded forms do you refer to? Elements encoded by both the
>>>> >> SDK-coder and runner-coder (and I ignore the Context here once again)
>>>> >> have the same binary representation (which they must have, otherwise
>>>> it
>>>> >> would be impossible to decode elements coming from the runner to the
>>>> >> SDK-harness or vice-versa).
>>>> >>>> Therefore - technically (and formally) -
>>>> >>>> the SDK coder for the TestStream is known to the runner,
>>>> regardless of
>>>> >>>> the language the runner is written in.
>>>> >>>>
>>>> >>>> To move  this discussion forward, I think we should look for
>>>> answers to
>>>> >>>> the following questions:
>>>> >>>>
>>>> >>>>     a) do we have any clues that show, that the proposed "in
>>>> runner"
>>>> >>>> solution will not work?
>>>> >>> OK, thinking about it some more, in the TestStream, we can use the
>>>> >>> happy coincidence that
>>>> >>>
>>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>>> >>> nested=False)
>>>> >>>
>>>> >>> (where || denotes concatenation) and the fact that we have
>>>> >>>
>>>> >>>       C.encode(x, nested=False)
>>>> >>>
>>>> >>> in hand.
>>>> >>>
>>>> >>> A possible fix here for the OP's question is that when rehydrating
>>>> the
>>>> >>> TestStream transform it must behave differently according to the
>>>> coder
>>>> >>> used in the subsequent channel (e.g. for known coders, it decodes
>>>> the
>>>> >>> elements and emits them directly, but for unknown coders, it
>>>> prefixes
>>>> >>> them with their length and emits byte strings. It gets more
>>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>>> unknown-coder))
>>>> >>> or KV<known-coder, LP(unknown-coder)) which have different encodings
>>>> >>> (and the latter, which is the default, requires transcoding the
>>>> bytes
>>>> >>> to inject the length in the middle which is found by decoding the
>>>> >>> first component). As well as getting more complex, this really seems
>>>> >>> to violate the spirit of separation of concerns.
>>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>>> >> KV<LP<unknown>, known>? From my understanding it is always the
>>>> latter,
>>>> >> because of [2].
>>>> >>
>>>> >> [2]
>>>> >>
>>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>> > It is currently the latter for runners using this code (which not all
>>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>>> > ossify this decision as part of the spec. (Note that even what's
>>>> > "known" and "unknown" can change from runner to runner.)
>>>> >
>>>> >>>>     b) do we think, that it will not be robust enough to
>>>> incorporate the
>>>> >>>> other use-cases (line generic transform inlining, taking into
>>>> account
>>>> >>>> that this applies only to runners that are written in the same
>>>> language
>>>> >>>> as the submitting SDK, because otherwise, there is nothing to
>>>> inline)?
>>>> >>> Being in the same language is not a prerequisite to "inlining," e.g.
>>>> >>> the PubSub source on Dataflow is recognized as such and not executed
>>>> >>> as SDK code but natively.
>>>> >> Agree, that is actually exactly what happens with the TestStream. The
>>>> >> transform need not be in the same language, as long as it is
>>>> completely
>>>> >> understood by the runner, including the SDK-coder (either explicitly
>>>> -
>>>> >> which might be due to the PCollection coder being composed of
>>>> well-known
>>>> >> coders only, or implicitly like in the case of TestStream, where the
>>>> >> elements are encoded using the SDK coder.
>>>> >>> It is more likely that inlining occurs in the same language if there
>>>> >>> are UDFs involved.
>>>> >>>
>>>> >>>> I'm convinced, that the TestStream-decode expansion solution is an
>>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>>> bothering
>>>> >>>> this mailing list with my emails on this. :-)
>>>> >>>>
>>>> >>>> WDYT?
>>>> >>> While not a solution to the general problem, I think the
>>>> >>> TestStream-only-does-bytes simplifies its definition (primitives
>>>> >>> should have as simple/easy to implement definitions as possible) and
>>>> >>> brings it closer to the other root we have: Impulse. (We could go a
>>>> >>> step further and rather than emitting encoded elements, with the
>>>> data
>>>> >>> in the proto itself, it emits sequence numbers, and a subsequent
>>>> ParDo
>>>> >>> maps those to concrete elements (e.g. via an in-memory map), but
>>>> that
>>>> >>> further step doesn't buy much...)
>>>> >>>
>>>> >>> Only runners that want to do inlining would have to take on the
>>>> >>> complexity of a fully generic solution.
>>>> >> I think that if the simplification brings something, we can do that,
>>>> but
>>>> >> I'd like to understand why we cannot (or should not) use the generic
>>>> >> solution. I think it definitely *should* be possible to use a generic
>>>> >> solution, because otherwise the solution would not be generic. And it
>>>> >> would imply, that we are unable to do generic transform inlining,
>>>> which
>>>> >> I would find really strange. That would immediately mean, that we are
>>>> >> unable to construct classical runner as a special case of the
>>>> portable
>>>> >> one, which would be bad I think.
>>>> >>
>>>> >> The elements in the TestStreamPayload are encoded with pure
>>>> SDK-coder,
>>>> >> or does this go through the LengthPrefixUnknownCoders logic? If not,
>>>> >> then the problem would be there, because that means, that the
>>>> SDK-coder
>>>> >> cannot be (implicitly) defined in the runner. If the elements would
>>>> be
>>>> >> encoded using LP, then it would be possible to decode them using
>>>> >> runner-coder and the problem should be solved, or am I still missing
>>>> >> some key parts?
>>>> > Yes, the problem is precisely that there are (unspecified) constraints
>>>> > on the coder used by the TestStreamPayload. Just requiring that it be
>>>> > length prefixed is not enough, you have to make constraints on
>>>> > sometimes pushing down the length prefixing if it's a composite (like
>>>> > a KV) that depend on what the runner is expected to support in terms
>>>> > of composites and/or the choices it chooses for the channel (and the
>>>> > runner, not knowing the coder, can't transcode between these choices).
>>>> >
>>>> > The simpler solution is to constrain this coder to just be byte[]
>>>> > rather than let it be a little bit flexible (but not wholly flexible).
>>>> >
>>>> > As for a fully generic solution, I think the issue encountered with
>>>> > inlining Read vs. TestStream are related to this, but not really the
>>>> > same. With TestStream one has an encoded representation of the
>>>> > elements provided by the SDK that the Runner and has no SDK
>>>> > representation/execution whereas with the Reads one has unencoded
>>>> > elements in hand and a Coder that is understood by both (so long as
>>>> > the channel can be negotiated correctly). FWIW, I think the proper
>>>> > solution to inlining a Read (or other Transform that would typically
>>>> > be executed in the SDK) is to treat it as a special environment (where
>>>> > we know logically it can work) and then elide, as possible, the
>>>> > various encodings, grpc calls, etc. that are unneeded as everything is
>>>> > in process.
>>>> >
>>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>>> wrote:
>>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
>>>> wrote:
>>>> >>>>>>>> Hi,
>>>> >>>>>>>>
>>>> >>>>>>>> I had some more time thinking about this and I'll try to recap
>>>> that.
>>>> >>>>>>>> First some invariants:
>>>> >>>>>>>>
>>>> >>>>>>>>       a) each PCollection<T> has actually two coders - an _SDK
>>>> coder_ and a
>>>> >>>>>>>> _runner coder_. These coders have the property, that each one
>>>> can
>>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not true,
>>>> the
>>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in general).
>>>> >>>>>>>>
>>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>>> environment, the
>>>> >>>>>>>> elements are encoded using SDK coder on the side of
>>>> SDK-harness and
>>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>>> >>>>>>>>
>>>> >>>>>>>>       c) under specific circumstances, the encode-decode step
>>>> can be
>>>> >>>>>>>> optimized out, that is the case where the SDK coder and all its
>>>> >>>>>>>> subcoders are all well-known to the runner (in the present,
>>>> that means
>>>> >>>>>>>> that all the parts present in the model coders set). The
>>>> reason for that
>>>> >>>>>>>> is that in this specific situation
>>>> runner_decode(sdk_encode(X)) = X.
>>>> >>>>>>>> This property is essential.
>>>> >>>>>>> However, in general, X can only pass from the SDK to the runner
>>>> (or
>>>> >>>>>>> vice versa) in encoded form.
>>>> >>>>>> In general yes, but we are (mostly) talking transform inlining
>>>> here, so
>>>> >>>>>> it that particular situation, the elements might be passed in
>>>> decoded form.
>>>> >>>>>>>>       d) from b) immediately follows, that when a PTransform
>>>> does not run in
>>>> >>>>>>>> an environment (and this might be due to the transform being
>>>> runner
>>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>>>> elements have
>>>> >>>>>>>> to be encoded by SDK coder, immediately following decode by
>>>> runner
>>>> >>>>>>>> coder. That (surprisingly) applies even to situations when
>>>> runner is
>>>> >>>>>>>> implemented using different language than the client SDK,
>>>> because it
>>>> >>>>>>>> implies that the type of produced elements must be one of
>>>> types encoded
>>>> >>>>>>>> using model coders (well-known to the runner, otherwise the
>>>> SDK will not
>>>> >>>>>>>> be able to consume it). But - due to property c) - this means
>>>> that this
>>>> >>>>>>>> encode-decode step can be optimized out. This does not mean
>>>> that it is
>>>> >>>>>>>> not (logically) present, though. This is exactly the case of
>>>> native
>>>> >>>>>>>> Impulse transform.
>>>> >>>>>>>>
>>>> >>>>>>>> Now, from that we can conclude that on the boundary between
>>>> executable
>>>> >>>>>>>> stages, or between runner (inlined) transform and executable
>>>> stage, each
>>>> >>>>>>>> PCollection has to be encoded using SDK coder and immediately
>>>> decoded by
>>>> >>>>>>>> runner coder, *unless this can be optimized out* by property
>>>> c).
>>>> >>>>>>>>
>>>> >>>>>>>> This gives us two options where to implement this
>>>> encode/decode step:
>>>> >>>>>>>>
>>>> >>>>>>>>       1) completely inside runner with the possibility to
>>>> optimize the
>>>> >>>>>>>> encode/decode step by identity under right circumstances
>>>> >>>>>>>>
>>>> >>>>>>>>       2) partly in the runner and partly in the SDK - that is
>>>> we encode
>>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass those
>>>> to the
>>>> >>>>>>>> SDK harness and apply a custom decode step there. This works
>>>> because SDK
>>>> >>>>>>>> coder encoded elements are in byte[], and that is well-known
>>>> coder type.
>>>> >>>>>>>> We again only leverage property c) and optimize the SDK coder
>>>> encode,
>>>> >>>>>>>> runner decode step out.
>>>> >>>>>>>>
>>>> >>>>>>>> The option 2) is exactly the proposal of TestStream producing
>>>> byte[] and
>>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually inlined
>>>> >>>>>>>> transform, the elements are produced directly in runner (the
>>>> SDK coder
>>>> >>>>>>>> is not known to the runner, but that does not matter, because
>>>> the
>>>> >>>>>>>> elements are already encoded by client).
>>>> >>>>>>>>
>>>> >>>>>>>>      From the above it seems to me, that option 1) should be
>>>> preferred, because:
>>>> >>>>>>>>
>>>> >>>>>>>>       i) it is generic, applicable to all inlined transforms,
>>>> any sources
>>>> >>>>>>>>
>>>> >>>>>>>>       ii) it is consistent with how things logically work
>>>> underneath
>>>> >>>>>>>>
>>>> >>>>>>>>       iii) it offers better room for optimization - option 2)
>>>> might result
>>>> >>>>>>>> in cases when the elements are passed from the runner to the
>>>> SDK-harness
>>>> >>>>>>>> only for the sake of the decoding from SDK coder and
>>>> immediately
>>>> >>>>>>>> encoding back using SDK-coder and returned back to the runner.
>>>> This
>>>> >>>>>>>> would be the case when TestStream would be directly consumed
>>>> by inlined
>>>> >>>>>>>> (or external) transform.
>>>> >>>>>>> (1) is not possible if the Coder in question is not known to the
>>>> >>>>>>> Runner, which is why I proposed (2).
>>>> >>>>>> There is no particular need for the coder to be known. If
>>>> transform is
>>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded form of
>>>> the
>>>> >>>>>> elements. That holds true if:
>>>> >>>>>>
>>>> >>>>>>      a) either the SDK coder is known, or
>>>> >>>>>>
>>>> >>>>>>      b) encoded form of the produced elements is known in advance
>>>> >>>>>>
>>>> >>>>>> For TestStream it is the case b). For inlined primitive Read (or
>>>> any
>>>> >>>>>> other transform which executes code) it is a).
>>>> >>>>> There's another hitch here for TestStream. For historical reasons,
>>>> >>>>> coders actually represent two encodings: nested (aka self
>>>> delimiting)
>>>> >>>>> and unnested. TestStream elements are given as unnested encoded
>>>> bytes,
>>>> >>>>> but the nested encoding is required for sending data to the SDK.
>>>> The
>>>> >>>>> runner can't go from <nested encoding> to <unnested encoding> for
>>>> an
>>>> >>>>> arbitrary unknown coder.
>>>> >>>>>
>>>> >>>>> (Even if it weren't for this complication, to be able to send
>>>> already
>>>> >>>>> encoded bytes of an unknown coder to the SDK will also complicate
>>>> the
>>>> >>>>> logic in choosing the coder to be used for the channel and
>>>> sending the
>>>> >>>>> data, which is some of what you're running into (but can be solved
>>>> >>>>> differently for inlined reads as the coder can always be known by
>>>> the
>>>> >>>>> runner).)
>>>>
>>>
>>
>> --
>>
>>
>> Jack McCluskey
>> SWE - DataPLS PLAT/ Beam Go
>> RDU
>> jrmccluskey@gmail.com
>>
>>
>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey <jr...@google.com>
wrote:

> Hey all,
>
> Just catching up on the thread since I did the TestStream Go SDK
> implementation. The discussion about length prefixing behavior for known
> vs. unknown coders is interesting, since we ran into strings and byte
> slices getting extra length prefixes attached to them by Flink despite
> being known coders.
>

Known to who?


> Based on what's been said, that isn't expected behavior, right?
>

No, it's not.

I would check to make sure the Go SDK is respecting the Coder (length
prefixed or not) that's set on the channel, rather than guessing at what it
expects it to be based on the Go type.


> On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:
>
>> On 9/7/21 6:02 PM, Reuven Lax wrote:
>>
>> Historically the DataflowRunner has been much more careful about not
>> breaking update, since this is a frequent operation by Dataflow users. I
>> think we've been less careful aboutt other runners, but as we see clearly
>> here Fllnk users do care about this as well, so we should probably test
>> upgrade compatibility for Flink.
>>
>> One strategy that Dataflow uses is to avoid embedding the Java serialized
>> form of a Coder in the graph, as this is a much higher risk of breakage (as
>> we see with the issue you llnked to). Possibly similar strategies should be
>> investigated for Fllink.
>>
>> +1, that would be great!
>>
>>
>> Reuven
>>
>> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>>
>>> > Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
>>> > care about Context because they predated this deprecation, and
>>> > changing coders is hard (due to no way to update the encoding for a
>>> > streaming pipeline).
>>> This is unrelated, but - regarding changing coders due to concerns about
>>> pipeline upgrades, we break this quite often, at least for some runners.
>>> Most recently [1].
>>>
>>> > It is currently the latter for runners using this code (which not all
>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>> > ossify this decision as part of the spec. (Note that even what's
>>> > "known" and "unknown" can change from runner to runner.)
>>> This is interesting and unexpected for me. How do runners decide about
>>> how they encode elements between SDK harness and the runner? How do they
>>> inform the SDK harness about this decision? My impression was that this
>>> is well-defined at the model level. If not, then we have the reason for
>>> misunderstanding in this conversation. :-)
>>>
>>>   Jan
>>>
>>> [1]
>>>
>>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>>
>>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz> wrote:
>>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>> >>
>>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>
>>> wrote:
>>> >>>> Hi Robert,
>>> >>>>
>>> >>>>> There's another hitch here for TestStream. For historical reasons,
>>> >>>>> coders actually represent two encodings: nested (aka self
>>> delimiting)
>>> >>>>> and unnested. TestStream elements are given as unnested encoded
>>> bytes,
>>> >>>>> but the nested encoding is required for sending data to the SDK.
>>> The
>>> >>>>> runner can't go from <nested encoding> to <unnested encoding> for
>>> an
>>> >>>>> arbitrary unknown coder.
>>> >>>>>
>>> >>>>> (Even if it weren't for this complication, to be able to send
>>> already
>>> >>>>> encoded bytes of an unknown coder to the SDK will also complicate
>>> the
>>> >>>>> logic in choosing the coder to be used for the channel and sending
>>> the
>>> >>>>> data, which is some of what you're running into (but can be solved
>>> >>>>> differently for inlined reads as the coder can always be known by
>>> the
>>> >>>>> runner).)
>>> >>>> It is hard for me to argue with "historical reasons". But - the
>>> "nested"
>>> >>>> and "unnested" coders look very similar to SDK-coder and
>>> runner-coder
>>> >>>> spaces.
>>> >>> Unfortunately, they're actually orthogonal to that.
>>> >> Hm, do you mean the Context passed to the encode/decode method? [1]
>>> That
>>> >> seems to be deprecated, I assume that most coders use the default
>>> >> implementation and simply ignore the Context?
>>> > Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
>>> > care about Context because they predated this deprecation, and
>>> > changing coders is hard (due to no way to update the encoding for a
>>> > streaming pipeline).
>>> >
>>> >> Even if not - whether or
>>> >> not the elements are encoded using NESTED Context or UNNESTED Context
>>> >> should be part of the contract of TestStream, right? Most likely it is
>>> >> the UNNESTED one, if I understand correctly what that does. Under what
>>> >> conditions is the deprecated encode/decode method used?
>>> > Yes, it's the UNNESTED one.
>>> >
>>> >> [1]
>>> >>
>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>> >>
>>> >>>> The runner's responsibility is not to go from "<nested
>>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary coder.
>>> >>>> That is really impossible. But a coder is a function, right?
>>> Function
>>> >>>> maps from universe A to universe B (in general). TestStream
>>> provides a
>>> >>>> set of elements, and these elements are the "universe". For those
>>> >>>> elements it also provides the encoded form, which can be
>>> interpreted as
>>> >>>> the definition of the coder.
>>> >>> The problem here is that there is not "the encoded form" for a Coder
>>> >>> but two encoded forms, and we have the wrong one. Things could be
>>> made
>>> >>> to work if we had the other.
>>> >> Which two encoded forms do you refer to? Elements encoded by both the
>>> >> SDK-coder and runner-coder (and I ignore the Context here once again)
>>> >> have the same binary representation (which they must have, otherwise
>>> it
>>> >> would be impossible to decode elements coming from the runner to the
>>> >> SDK-harness or vice-versa).
>>> >>>> Therefore - technically (and formally) -
>>> >>>> the SDK coder for the TestStream is known to the runner, regardless
>>> of
>>> >>>> the language the runner is written in.
>>> >>>>
>>> >>>> To move  this discussion forward, I think we should look for
>>> answers to
>>> >>>> the following questions:
>>> >>>>
>>> >>>>     a) do we have any clues that show, that the proposed "in runner"
>>> >>>> solution will not work?
>>> >>> OK, thinking about it some more, in the TestStream, we can use the
>>> >>> happy coincidence that
>>> >>>
>>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>> >>> nested=False)
>>> >>>
>>> >>> (where || denotes concatenation) and the fact that we have
>>> >>>
>>> >>>       C.encode(x, nested=False)
>>> >>>
>>> >>> in hand.
>>> >>>
>>> >>> A possible fix here for the OP's question is that when rehydrating
>>> the
>>> >>> TestStream transform it must behave differently according to the
>>> coder
>>> >>> used in the subsequent channel (e.g. for known coders, it decodes the
>>> >>> elements and emits them directly, but for unknown coders, it prefixes
>>> >>> them with their length and emits byte strings. It gets more
>>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>>> >>> unknown-coder> the channel might be LP(KV<known-coder,
>>> unknown-coder))
>>> >>> or KV<known-coder, LP(unknown-coder)) which have different encodings
>>> >>> (and the latter, which is the default, requires transcoding the bytes
>>> >>> to inject the length in the middle which is found by decoding the
>>> >>> first component). As well as getting more complex, this really seems
>>> >>> to violate the spirit of separation of concerns.
>>> >> How do we make the decision if the channel is LP<KV<..>> or
>>> >> KV<LP<unknown>, known>? From my understanding it is always the latter,
>>> >> because of [2].
>>> >>
>>> >> [2]
>>> >>
>>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>> > It is currently the latter for runners using this code (which not all
>>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>>> > ossify this decision as part of the spec. (Note that even what's
>>> > "known" and "unknown" can change from runner to runner.)
>>> >
>>> >>>>     b) do we think, that it will not be robust enough to
>>> incorporate the
>>> >>>> other use-cases (line generic transform inlining, taking into
>>> account
>>> >>>> that this applies only to runners that are written in the same
>>> language
>>> >>>> as the submitting SDK, because otherwise, there is nothing to
>>> inline)?
>>> >>> Being in the same language is not a prerequisite to "inlining," e.g.
>>> >>> the PubSub source on Dataflow is recognized as such and not executed
>>> >>> as SDK code but natively.
>>> >> Agree, that is actually exactly what happens with the TestStream. The
>>> >> transform need not be in the same language, as long as it is
>>> completely
>>> >> understood by the runner, including the SDK-coder (either explicitly -
>>> >> which might be due to the PCollection coder being composed of
>>> well-known
>>> >> coders only, or implicitly like in the case of TestStream, where the
>>> >> elements are encoded using the SDK coder.
>>> >>> It is more likely that inlining occurs in the same language if there
>>> >>> are UDFs involved.
>>> >>>
>>> >>>> I'm convinced, that the TestStream-decode expansion solution is an
>>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>>> bothering
>>> >>>> this mailing list with my emails on this. :-)
>>> >>>>
>>> >>>> WDYT?
>>> >>> While not a solution to the general problem, I think the
>>> >>> TestStream-only-does-bytes simplifies its definition (primitives
>>> >>> should have as simple/easy to implement definitions as possible) and
>>> >>> brings it closer to the other root we have: Impulse. (We could go a
>>> >>> step further and rather than emitting encoded elements, with the data
>>> >>> in the proto itself, it emits sequence numbers, and a subsequent
>>> ParDo
>>> >>> maps those to concrete elements (e.g. via an in-memory map), but that
>>> >>> further step doesn't buy much...)
>>> >>>
>>> >>> Only runners that want to do inlining would have to take on the
>>> >>> complexity of a fully generic solution.
>>> >> I think that if the simplification brings something, we can do that,
>>> but
>>> >> I'd like to understand why we cannot (or should not) use the generic
>>> >> solution. I think it definitely *should* be possible to use a generic
>>> >> solution, because otherwise the solution would not be generic. And it
>>> >> would imply, that we are unable to do generic transform inlining,
>>> which
>>> >> I would find really strange. That would immediately mean, that we are
>>> >> unable to construct classical runner as a special case of the portable
>>> >> one, which would be bad I think.
>>> >>
>>> >> The elements in the TestStreamPayload are encoded with pure SDK-coder,
>>> >> or does this go through the LengthPrefixUnknownCoders logic? If not,
>>> >> then the problem would be there, because that means, that the
>>> SDK-coder
>>> >> cannot be (implicitly) defined in the runner. If the elements would be
>>> >> encoded using LP, then it would be possible to decode them using
>>> >> runner-coder and the problem should be solved, or am I still missing
>>> >> some key parts?
>>> > Yes, the problem is precisely that there are (unspecified) constraints
>>> > on the coder used by the TestStreamPayload. Just requiring that it be
>>> > length prefixed is not enough, you have to make constraints on
>>> > sometimes pushing down the length prefixing if it's a composite (like
>>> > a KV) that depend on what the runner is expected to support in terms
>>> > of composites and/or the choices it chooses for the channel (and the
>>> > runner, not knowing the coder, can't transcode between these choices).
>>> >
>>> > The simpler solution is to constrain this coder to just be byte[]
>>> > rather than let it be a little bit flexible (but not wholly flexible).
>>> >
>>> > As for a fully generic solution, I think the issue encountered with
>>> > inlining Read vs. TestStream are related to this, but not really the
>>> > same. With TestStream one has an encoded representation of the
>>> > elements provided by the SDK that the Runner and has no SDK
>>> > representation/execution whereas with the Reads one has unencoded
>>> > elements in hand and a Coder that is understood by both (so long as
>>> > the channel can be negotiated correctly). FWIW, I think the proper
>>> > solution to inlining a Read (or other Transform that would typically
>>> > be executed in the SDK) is to treat it as a special environment (where
>>> > we know logically it can work) and then elide, as possible, the
>>> > various encodings, grpc calls, etc. that are unneeded as everything is
>>> > in process.
>>> >
>>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>>> wrote:
>>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
>>> wrote:
>>> >>>>>>>> Hi,
>>> >>>>>>>>
>>> >>>>>>>> I had some more time thinking about this and I'll try to recap
>>> that.
>>> >>>>>>>> First some invariants:
>>> >>>>>>>>
>>> >>>>>>>>       a) each PCollection<T> has actually two coders - an _SDK
>>> coder_ and a
>>> >>>>>>>> _runner coder_. These coders have the property, that each one
>>> can
>>> >>>>>>>> _decode_ what the other encoded, but the opposite is not true,
>>> the
>>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in general).
>>> >>>>>>>>
>>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>>> environment, the
>>> >>>>>>>> elements are encoded using SDK coder on the side of SDK-harness
>>> and
>>> >>>>>>>> decoded using runner coder after receiving in the runner
>>> >>>>>>>>
>>> >>>>>>>>       c) under specific circumstances, the encode-decode step
>>> can be
>>> >>>>>>>> optimized out, that is the case where the SDK coder and all its
>>> >>>>>>>> subcoders are all well-known to the runner (in the present,
>>> that means
>>> >>>>>>>> that all the parts present in the model coders set). The reason
>>> for that
>>> >>>>>>>> is that in this specific situation runner_decode(sdk_encode(X))
>>> = X.
>>> >>>>>>>> This property is essential.
>>> >>>>>>> However, in general, X can only pass from the SDK to the runner
>>> (or
>>> >>>>>>> vice versa) in encoded form.
>>> >>>>>> In general yes, but we are (mostly) talking transform inlining
>>> here, so
>>> >>>>>> it that particular situation, the elements might be passed in
>>> decoded form.
>>> >>>>>>>>       d) from b) immediately follows, that when a PTransform
>>> does not run in
>>> >>>>>>>> an environment (and this might be due to the transform being
>>> runner
>>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>>> elements have
>>> >>>>>>>> to be encoded by SDK coder, immediately following decode by
>>> runner
>>> >>>>>>>> coder. That (surprisingly) applies even to situations when
>>> runner is
>>> >>>>>>>> implemented using different language than the client SDK,
>>> because it
>>> >>>>>>>> implies that the type of produced elements must be one of types
>>> encoded
>>> >>>>>>>> using model coders (well-known to the runner, otherwise the SDK
>>> will not
>>> >>>>>>>> be able to consume it). But - due to property c) - this means
>>> that this
>>> >>>>>>>> encode-decode step can be optimized out. This does not mean
>>> that it is
>>> >>>>>>>> not (logically) present, though. This is exactly the case of
>>> native
>>> >>>>>>>> Impulse transform.
>>> >>>>>>>>
>>> >>>>>>>> Now, from that we can conclude that on the boundary between
>>> executable
>>> >>>>>>>> stages, or between runner (inlined) transform and executable
>>> stage, each
>>> >>>>>>>> PCollection has to be encoded using SDK coder and immediately
>>> decoded by
>>> >>>>>>>> runner coder, *unless this can be optimized out* by property c).
>>> >>>>>>>>
>>> >>>>>>>> This gives us two options where to implement this encode/decode
>>> step:
>>> >>>>>>>>
>>> >>>>>>>>       1) completely inside runner with the possibility to
>>> optimize the
>>> >>>>>>>> encode/decode step by identity under right circumstances
>>> >>>>>>>>
>>> >>>>>>>>       2) partly in the runner and partly in the SDK - that is
>>> we encode
>>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass those
>>> to the
>>> >>>>>>>> SDK harness and apply a custom decode step there. This works
>>> because SDK
>>> >>>>>>>> coder encoded elements are in byte[], and that is well-known
>>> coder type.
>>> >>>>>>>> We again only leverage property c) and optimize the SDK coder
>>> encode,
>>> >>>>>>>> runner decode step out.
>>> >>>>>>>>
>>> >>>>>>>> The option 2) is exactly the proposal of TestStream producing
>>> byte[] and
>>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually inlined
>>> >>>>>>>> transform, the elements are produced directly in runner (the
>>> SDK coder
>>> >>>>>>>> is not known to the runner, but that does not matter, because
>>> the
>>> >>>>>>>> elements are already encoded by client).
>>> >>>>>>>>
>>> >>>>>>>>      From the above it seems to me, that option 1) should be
>>> preferred, because:
>>> >>>>>>>>
>>> >>>>>>>>       i) it is generic, applicable to all inlined transforms,
>>> any sources
>>> >>>>>>>>
>>> >>>>>>>>       ii) it is consistent with how things logically work
>>> underneath
>>> >>>>>>>>
>>> >>>>>>>>       iii) it offers better room for optimization - option 2)
>>> might result
>>> >>>>>>>> in cases when the elements are passed from the runner to the
>>> SDK-harness
>>> >>>>>>>> only for the sake of the decoding from SDK coder and immediately
>>> >>>>>>>> encoding back using SDK-coder and returned back to the runner.
>>> This
>>> >>>>>>>> would be the case when TestStream would be directly consumed by
>>> inlined
>>> >>>>>>>> (or external) transform.
>>> >>>>>>> (1) is not possible if the Coder in question is not known to the
>>> >>>>>>> Runner, which is why I proposed (2).
>>> >>>>>> There is no particular need for the coder to be known. If
>>> transform is
>>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded form of
>>> the
>>> >>>>>> elements. That holds true if:
>>> >>>>>>
>>> >>>>>>      a) either the SDK coder is known, or
>>> >>>>>>
>>> >>>>>>      b) encoded form of the produced elements is known in advance
>>> >>>>>>
>>> >>>>>> For TestStream it is the case b). For inlined primitive Read (or
>>> any
>>> >>>>>> other transform which executes code) it is a).
>>> >>>>> There's another hitch here for TestStream. For historical reasons,
>>> >>>>> coders actually represent two encodings: nested (aka self
>>> delimiting)
>>> >>>>> and unnested. TestStream elements are given as unnested encoded
>>> bytes,
>>> >>>>> but the nested encoding is required for sending data to the SDK.
>>> The
>>> >>>>> runner can't go from <nested encoding> to <unnested encoding> for
>>> an
>>> >>>>> arbitrary unknown coder.
>>> >>>>>
>>> >>>>> (Even if it weren't for this complication, to be able to send
>>> already
>>> >>>>> encoded bytes of an unknown coder to the SDK will also complicate
>>> the
>>> >>>>> logic in choosing the coder to be used for the channel and sending
>>> the
>>> >>>>> data, which is some of what you're running into (but can be solved
>>> >>>>> differently for inlined reads as the coder can always be known by
>>> the
>>> >>>>> runner).)
>>>
>>
>
> --
>
>
> Jack McCluskey
> SWE - DataPLS PLAT/ Beam Go
> RDU
> jrmccluskey@gmail.com
>
>
>

Re: Unexpected in TestStream in Portable Mode

Posted by Jack McCluskey <jr...@google.com>.
Hey all,

Just catching up on the thread since I did the TestStream Go SDK
implementation. The discussion about length prefixing behavior for known
vs. unknown coders is interesting, since we ran into strings and byte
slices getting extra length prefixes attached to them by Flink despite
being known coders. Based on what's been said, that isn't expected
behavior, right?

On Tue, Sep 7, 2021 at 2:46 PM Jan Lukavský <je...@seznam.cz> wrote:

> On 9/7/21 6:02 PM, Reuven Lax wrote:
>
> Historically the DataflowRunner has been much more careful about not
> breaking update, since this is a frequent operation by Dataflow users. I
> think we've been less careful aboutt other runners, but as we see clearly
> here Fllnk users do care about this as well, so we should probably test
> upgrade compatibility for Flink.
>
> One strategy that Dataflow uses is to avoid embedding the Java serialized
> form of a Coder in the graph, as this is a much higher risk of breakage (as
> we see with the issue you llnked to). Possibly similar strategies should be
> investigated for Fllink.
>
> +1, that would be great!
>
>
> Reuven
>
> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>
>> > Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
>> > care about Context because they predated this deprecation, and
>> > changing coders is hard (due to no way to update the encoding for a
>> > streaming pipeline).
>> This is unrelated, but - regarding changing coders due to concerns about
>> pipeline upgrades, we break this quite often, at least for some runners.
>> Most recently [1].
>>
>> > It is currently the latter for runners using this code (which not all
>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>> > ossify this decision as part of the spec. (Note that even what's
>> > "known" and "unknown" can change from runner to runner.)
>> This is interesting and unexpected for me. How do runners decide about
>> how they encode elements between SDK harness and the runner? How do they
>> inform the SDK harness about this decision? My impression was that this
>> is well-defined at the model level. If not, then we have the reason for
>> misunderstanding in this conversation. :-)
>>
>>   Jan
>>
>> [1]
>>
>> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>>
>> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz> wrote:
>> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>> >>
>> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>  wrote:
>> >>>> Hi Robert,
>> >>>>
>> >>>>> There's another hitch here for TestStream. For historical reasons,
>> >>>>> coders actually represent two encodings: nested (aka self
>> delimiting)
>> >>>>> and unnested. TestStream elements are given as unnested encoded
>> bytes,
>> >>>>> but the nested encoding is required for sending data to the SDK. The
>> >>>>> runner can't go from <nested encoding> to <unnested encoding> for an
>> >>>>> arbitrary unknown coder.
>> >>>>>
>> >>>>> (Even if it weren't for this complication, to be able to send
>> already
>> >>>>> encoded bytes of an unknown coder to the SDK will also complicate
>> the
>> >>>>> logic in choosing the coder to be used for the channel and sending
>> the
>> >>>>> data, which is some of what you're running into (but can be solved
>> >>>>> differently for inlined reads as the coder can always be known by
>> the
>> >>>>> runner).)
>> >>>> It is hard for me to argue with "historical reasons". But - the
>> "nested"
>> >>>> and "unnested" coders look very similar to SDK-coder and runner-coder
>> >>>> spaces.
>> >>> Unfortunately, they're actually orthogonal to that.
>> >> Hm, do you mean the Context passed to the encode/decode method? [1]
>> That
>> >> seems to be deprecated, I assume that most coders use the default
>> >> implementation and simply ignore the Context?
>> > Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
>> > care about Context because they predated this deprecation, and
>> > changing coders is hard (due to no way to update the encoding for a
>> > streaming pipeline).
>> >
>> >> Even if not - whether or
>> >> not the elements are encoded using NESTED Context or UNNESTED Context
>> >> should be part of the contract of TestStream, right? Most likely it is
>> >> the UNNESTED one, if I understand correctly what that does. Under what
>> >> conditions is the deprecated encode/decode method used?
>> > Yes, it's the UNNESTED one.
>> >
>> >> [1]
>> >>
>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>> >>
>> >>>> The runner's responsibility is not to go from "<nested
>> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary coder.
>> >>>> That is really impossible. But a coder is a function, right? Function
>> >>>> maps from universe A to universe B (in general). TestStream provides
>> a
>> >>>> set of elements, and these elements are the "universe". For those
>> >>>> elements it also provides the encoded form, which can be interpreted
>> as
>> >>>> the definition of the coder.
>> >>> The problem here is that there is not "the encoded form" for a Coder
>> >>> but two encoded forms, and we have the wrong one. Things could be made
>> >>> to work if we had the other.
>> >> Which two encoded forms do you refer to? Elements encoded by both the
>> >> SDK-coder and runner-coder (and I ignore the Context here once again)
>> >> have the same binary representation (which they must have, otherwise it
>> >> would be impossible to decode elements coming from the runner to the
>> >> SDK-harness or vice-versa).
>> >>>> Therefore - technically (and formally) -
>> >>>> the SDK coder for the TestStream is known to the runner, regardless
>> of
>> >>>> the language the runner is written in.
>> >>>>
>> >>>> To move  this discussion forward, I think we should look for answers
>> to
>> >>>> the following questions:
>> >>>>
>> >>>>     a) do we have any clues that show, that the proposed "in runner"
>> >>>> solution will not work?
>> >>> OK, thinking about it some more, in the TestStream, we can use the
>> >>> happy coincidence that
>> >>>
>> >>>       LengthPrefixed(C).encode(x, nested=True) ==
>> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>> >>> nested=False)
>> >>>
>> >>> (where || denotes concatenation) and the fact that we have
>> >>>
>> >>>       C.encode(x, nested=False)
>> >>>
>> >>> in hand.
>> >>>
>> >>> A possible fix here for the OP's question is that when rehydrating the
>> >>> TestStream transform it must behave differently according to the coder
>> >>> used in the subsequent channel (e.g. for known coders, it decodes the
>> >>> elements and emits them directly, but for unknown coders, it prefixes
>> >>> them with their length and emits byte strings. It gets more
>> >>> complicated for nested coders, e.g. for a KV<known-coder,
>> >>> unknown-coder> the channel might be LP(KV<known-coder, unknown-coder))
>> >>> or KV<known-coder, LP(unknown-coder)) which have different encodings
>> >>> (and the latter, which is the default, requires transcoding the bytes
>> >>> to inject the length in the middle which is found by decoding the
>> >>> first component). As well as getting more complex, this really seems
>> >>> to violate the spirit of separation of concerns.
>> >> How do we make the decision if the channel is LP<KV<..>> or
>> >> KV<LP<unknown>, known>? From my understanding it is always the latter,
>> >> because of [2].
>> >>
>> >> [2]
>> >>
>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>> > It is currently the latter for runners using this code (which not all
>> > do, e.g. the ULR and Dataflow runners). I don't think we want to
>> > ossify this decision as part of the spec. (Note that even what's
>> > "known" and "unknown" can change from runner to runner.)
>> >
>> >>>>     b) do we think, that it will not be robust enough to incorporate
>> the
>> >>>> other use-cases (line generic transform inlining, taking into account
>> >>>> that this applies only to runners that are written in the same
>> language
>> >>>> as the submitting SDK, because otherwise, there is nothing to
>> inline)?
>> >>> Being in the same language is not a prerequisite to "inlining," e.g.
>> >>> the PubSub source on Dataflow is recognized as such and not executed
>> >>> as SDK code but natively.
>> >> Agree, that is actually exactly what happens with the TestStream. The
>> >> transform need not be in the same language, as long as it is completely
>> >> understood by the runner, including the SDK-coder (either explicitly -
>> >> which might be due to the PCollection coder being composed of
>> well-known
>> >> coders only, or implicitly like in the case of TestStream, where the
>> >> elements are encoded using the SDK coder.
>> >>> It is more likely that inlining occurs in the same language if there
>> >>> are UDFs involved.
>> >>>
>> >>>> I'm convinced, that the TestStream-decode expansion solution is an
>> >>>> ad-hoc solution to a generic problem, which is why I'm still
>> bothering
>> >>>> this mailing list with my emails on this. :-)
>> >>>>
>> >>>> WDYT?
>> >>> While not a solution to the general problem, I think the
>> >>> TestStream-only-does-bytes simplifies its definition (primitives
>> >>> should have as simple/easy to implement definitions as possible) and
>> >>> brings it closer to the other root we have: Impulse. (We could go a
>> >>> step further and rather than emitting encoded elements, with the data
>> >>> in the proto itself, it emits sequence numbers, and a subsequent ParDo
>> >>> maps those to concrete elements (e.g. via an in-memory map), but that
>> >>> further step doesn't buy much...)
>> >>>
>> >>> Only runners that want to do inlining would have to take on the
>> >>> complexity of a fully generic solution.
>> >> I think that if the simplification brings something, we can do that,
>> but
>> >> I'd like to understand why we cannot (or should not) use the generic
>> >> solution. I think it definitely *should* be possible to use a generic
>> >> solution, because otherwise the solution would not be generic. And it
>> >> would imply, that we are unable to do generic transform inlining, which
>> >> I would find really strange. That would immediately mean, that we are
>> >> unable to construct classical runner as a special case of the portable
>> >> one, which would be bad I think.
>> >>
>> >> The elements in the TestStreamPayload are encoded with pure SDK-coder,
>> >> or does this go through the LengthPrefixUnknownCoders logic? If not,
>> >> then the problem would be there, because that means, that the SDK-coder
>> >> cannot be (implicitly) defined in the runner. If the elements would be
>> >> encoded using LP, then it would be possible to decode them using
>> >> runner-coder and the problem should be solved, or am I still missing
>> >> some key parts?
>> > Yes, the problem is precisely that there are (unspecified) constraints
>> > on the coder used by the TestStreamPayload. Just requiring that it be
>> > length prefixed is not enough, you have to make constraints on
>> > sometimes pushing down the length prefixing if it's a composite (like
>> > a KV) that depend on what the runner is expected to support in terms
>> > of composites and/or the choices it chooses for the channel (and the
>> > runner, not knowing the coder, can't transcode between these choices).
>> >
>> > The simpler solution is to constrain this coder to just be byte[]
>> > rather than let it be a little bit flexible (but not wholly flexible).
>> >
>> > As for a fully generic solution, I think the issue encountered with
>> > inlining Read vs. TestStream are related to this, but not really the
>> > same. With TestStream one has an encoded representation of the
>> > elements provided by the SDK that the Runner and has no SDK
>> > representation/execution whereas with the Reads one has unencoded
>> > elements in hand and a Coder that is understood by both (so long as
>> > the channel can be negotiated correctly). FWIW, I think the proper
>> > solution to inlining a Read (or other Transform that would typically
>> > be executed in the SDK) is to treat it as a special environment (where
>> > we know logically it can work) and then elide, as possible, the
>> > various encodings, grpc calls, etc. that are unneeded as everything is
>> > in process.
>> >
>> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>
>> wrote:
>> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
>> wrote:
>> >>>>>>>> Hi,
>> >>>>>>>>
>> >>>>>>>> I had some more time thinking about this and I'll try to recap
>> that.
>> >>>>>>>> First some invariants:
>> >>>>>>>>
>> >>>>>>>>       a) each PCollection<T> has actually two coders - an _SDK
>> coder_ and a
>> >>>>>>>> _runner coder_. These coders have the property, that each one can
>> >>>>>>>> _decode_ what the other encoded, but the opposite is not true,
>> the
>> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in general).
>> >>>>>>>>
>> >>>>>>>>       b) when is a PCollection<T> computed inside an
>> environment, the
>> >>>>>>>> elements are encoded using SDK coder on the side of SDK-harness
>> and
>> >>>>>>>> decoded using runner coder after receiving in the runner
>> >>>>>>>>
>> >>>>>>>>       c) under specific circumstances, the encode-decode step
>> can be
>> >>>>>>>> optimized out, that is the case where the SDK coder and all its
>> >>>>>>>> subcoders are all well-known to the runner (in the present, that
>> means
>> >>>>>>>> that all the parts present in the model coders set). The reason
>> for that
>> >>>>>>>> is that in this specific situation runner_decode(sdk_encode(X))
>> = X.
>> >>>>>>>> This property is essential.
>> >>>>>>> However, in general, X can only pass from the SDK to the runner
>> (or
>> >>>>>>> vice versa) in encoded form.
>> >>>>>> In general yes, but we are (mostly) talking transform inlining
>> here, so
>> >>>>>> it that particular situation, the elements might be passed in
>> decoded form.
>> >>>>>>>>       d) from b) immediately follows, that when a PTransform
>> does not run in
>> >>>>>>>> an environment (and this might be due to the transform being
>> runner
>> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>> elements have
>> >>>>>>>> to be encoded by SDK coder, immediately following decode by
>> runner
>> >>>>>>>> coder. That (surprisingly) applies even to situations when
>> runner is
>> >>>>>>>> implemented using different language than the client SDK,
>> because it
>> >>>>>>>> implies that the type of produced elements must be one of types
>> encoded
>> >>>>>>>> using model coders (well-known to the runner, otherwise the SDK
>> will not
>> >>>>>>>> be able to consume it). But - due to property c) - this means
>> that this
>> >>>>>>>> encode-decode step can be optimized out. This does not mean that
>> it is
>> >>>>>>>> not (logically) present, though. This is exactly the case of
>> native
>> >>>>>>>> Impulse transform.
>> >>>>>>>>
>> >>>>>>>> Now, from that we can conclude that on the boundary between
>> executable
>> >>>>>>>> stages, or between runner (inlined) transform and executable
>> stage, each
>> >>>>>>>> PCollection has to be encoded using SDK coder and immediately
>> decoded by
>> >>>>>>>> runner coder, *unless this can be optimized out* by property c).
>> >>>>>>>>
>> >>>>>>>> This gives us two options where to implement this encode/decode
>> step:
>> >>>>>>>>
>> >>>>>>>>       1) completely inside runner with the possibility to
>> optimize the
>> >>>>>>>> encode/decode step by identity under right circumstances
>> >>>>>>>>
>> >>>>>>>>       2) partly in the runner and partly in the SDK - that is we
>> encode
>> >>>>>>>> elements of PCollection using SDK coder into bytes, pass those
>> to the
>> >>>>>>>> SDK harness and apply a custom decode step there. This works
>> because SDK
>> >>>>>>>> coder encoded elements are in byte[], and that is well-known
>> coder type.
>> >>>>>>>> We again only leverage property c) and optimize the SDK coder
>> encode,
>> >>>>>>>> runner decode step out.
>> >>>>>>>>
>> >>>>>>>> The option 2) is exactly the proposal of TestStream producing
>> byte[] and
>> >>>>>>>> decoding inside SDK-harness, the TestStream is actually inlined
>> >>>>>>>> transform, the elements are produced directly in runner (the SDK
>> coder
>> >>>>>>>> is not known to the runner, but that does not matter, because the
>> >>>>>>>> elements are already encoded by client).
>> >>>>>>>>
>> >>>>>>>>      From the above it seems to me, that option 1) should be
>> preferred, because:
>> >>>>>>>>
>> >>>>>>>>       i) it is generic, applicable to all inlined transforms,
>> any sources
>> >>>>>>>>
>> >>>>>>>>       ii) it is consistent with how things logically work
>> underneath
>> >>>>>>>>
>> >>>>>>>>       iii) it offers better room for optimization - option 2)
>> might result
>> >>>>>>>> in cases when the elements are passed from the runner to the
>> SDK-harness
>> >>>>>>>> only for the sake of the decoding from SDK coder and immediately
>> >>>>>>>> encoding back using SDK-coder and returned back to the runner.
>> This
>> >>>>>>>> would be the case when TestStream would be directly consumed by
>> inlined
>> >>>>>>>> (or external) transform.
>> >>>>>>> (1) is not possible if the Coder in question is not known to the
>> >>>>>>> Runner, which is why I proposed (2).
>> >>>>>> There is no particular need for the coder to be known. If
>> transform is
>> >>>>>> to be inlined, what *has* to be known is the SDK-encoded form of
>> the
>> >>>>>> elements. That holds true if:
>> >>>>>>
>> >>>>>>      a) either the SDK coder is known, or
>> >>>>>>
>> >>>>>>      b) encoded form of the produced elements is known in advance
>> >>>>>>
>> >>>>>> For TestStream it is the case b). For inlined primitive Read (or
>> any
>> >>>>>> other transform which executes code) it is a).
>> >>>>> There's another hitch here for TestStream. For historical reasons,
>> >>>>> coders actually represent two encodings: nested (aka self
>> delimiting)
>> >>>>> and unnested. TestStream elements are given as unnested encoded
>> bytes,
>> >>>>> but the nested encoding is required for sending data to the SDK. The
>> >>>>> runner can't go from <nested encoding> to <unnested encoding> for an
>> >>>>> arbitrary unknown coder.
>> >>>>>
>> >>>>> (Even if it weren't for this complication, to be able to send
>> already
>> >>>>> encoded bytes of an unknown coder to the SDK will also complicate
>> the
>> >>>>> logic in choosing the coder to be used for the channel and sending
>> the
>> >>>>> data, which is some of what you're running into (but can be solved
>> >>>>> differently for inlined reads as the coder can always be known by
>> the
>> >>>>> runner).)
>>
>

-- 


Jack McCluskey
SWE - DataPLS PLAT/ Beam Go
RDU
jrmccluskey@gmail.com

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
On 9/7/21 6:02 PM, Reuven Lax wrote:
> Historically the DataflowRunner has been much more careful about not 
> breaking update, since this is a frequent operation by Dataflow users. 
> I think we've been less careful aboutt other runners, but as we see 
> clearly here Fllnk users do care about this as well, so we should 
> probably test upgrade compatibility for Flink.
>
> One strategy that Dataflow uses is to avoid embedding the Java 
> serialized form of a Coder in the graph, as this is a much higher risk 
> of breakage (as we see with the issue you llnked to). Possibly similar 
> strategies should be investigated for Fllink.
+1, that would be great!
>
> Reuven
>
> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je.ik@seznam.cz 
> <ma...@seznam.cz>> wrote:
>
>     > Unfortunately the most basic coders (e.g. bytes, string, kv,
>     iterable)
>     > care about Context because they predated this deprecation, and
>     > changing coders is hard (due to no way to update the encoding for a
>     > streaming pipeline).
>     This is unrelated, but - regarding changing coders due to concerns
>     about
>     pipeline upgrades, we break this quite often, at least for some
>     runners.
>     Most recently [1].
>
>     > It is currently the latter for runners using this code (which
>     not all
>     > do, e.g. the ULR and Dataflow runners). I don't think we want to
>     > ossify this decision as part of the spec. (Note that even what's
>     > "known" and "unknown" can change from runner to runner.)
>     This is interesting and unexpected for me. How do runners decide
>     about
>     how they encode elements between SDK harness and the runner? How
>     do they
>     inform the SDK harness about this decision? My impression was that
>     this
>     is well-defined at the model level. If not, then we have the
>     reason for
>     misunderstanding in this conversation. :-)
>
>       Jan
>
>     [1]
>     https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>     <https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E>
>
>     On 9/4/21 7:32 PM, Robert Bradshaw wrote:
>     > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je.ik@seznam.cz
>     <ma...@seznam.cz>> wrote:
>     >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>     >>
>     >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je.ik@seznam.cz
>     <ma...@seznam.cz>>  wrote:
>     >>>> Hi Robert,
>     >>>>
>     >>>>> There's another hitch here for TestStream. For historical
>     reasons,
>     >>>>> coders actually represent two encodings: nested (aka self
>     delimiting)
>     >>>>> and unnested. TestStream elements are given as unnested
>     encoded bytes,
>     >>>>> but the nested encoding is required for sending data to the
>     SDK. The
>     >>>>> runner can't go from <nested encoding> to <unnested
>     encoding> for an
>     >>>>> arbitrary unknown coder.
>     >>>>>
>     >>>>> (Even if it weren't for this complication, to be able to
>     send already
>     >>>>> encoded bytes of an unknown coder to the SDK will also
>     complicate the
>     >>>>> logic in choosing the coder to be used for the channel and
>     sending the
>     >>>>> data, which is some of what you're running into (but can be
>     solved
>     >>>>> differently for inlined reads as the coder can always be
>     known by the
>     >>>>> runner).)
>     >>>> It is hard for me to argue with "historical reasons". But -
>     the "nested"
>     >>>> and "unnested" coders look very similar to SDK-coder and
>     runner-coder
>     >>>> spaces.
>     >>> Unfortunately, they're actually orthogonal to that.
>     >> Hm, do you mean the Context passed to the encode/decode method?
>     [1] That
>     >> seems to be deprecated, I assume that most coders use the default
>     >> implementation and simply ignore the Context?
>     > Unfortunately the most basic coders (e.g. bytes, string, kv,
>     iterable)
>     > care about Context because they predated this deprecation, and
>     > changing coders is hard (due to no way to update the encoding for a
>     > streaming pipeline).
>     >
>     >> Even if not - whether or
>     >> not the elements are encoded using NESTED Context or UNNESTED
>     Context
>     >> should be part of the contract of TestStream, right? Most
>     likely it is
>     >> the UNNESTED one, if I understand correctly what that does.
>     Under what
>     >> conditions is the deprecated encode/decode method used?
>     > Yes, it's the UNNESTED one.
>     >
>     >> [1]
>     >>
>     https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>     <https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134>
>     >>
>     >>>> The runner's responsibility is not to go from "<nested
>     >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary
>     coder.
>     >>>> That is really impossible. But a coder is a function, right?
>     Function
>     >>>> maps from universe A to universe B (in general). TestStream
>     provides a
>     >>>> set of elements, and these elements are the "universe". For those
>     >>>> elements it also provides the encoded form, which can be
>     interpreted as
>     >>>> the definition of the coder.
>     >>> The problem here is that there is not "the encoded form" for a
>     Coder
>     >>> but two encoded forms, and we have the wrong one. Things could
>     be made
>     >>> to work if we had the other.
>     >> Which two encoded forms do you refer to? Elements encoded by
>     both the
>     >> SDK-coder and runner-coder (and I ignore the Context here once
>     again)
>     >> have the same binary representation (which they must have,
>     otherwise it
>     >> would be impossible to decode elements coming from the runner
>     to the
>     >> SDK-harness or vice-versa).
>     >>>> Therefore - technically (and formally) -
>     >>>> the SDK coder for the TestStream is known to the runner,
>     regardless of
>     >>>> the language the runner is written in.
>     >>>>
>     >>>> To move  this discussion forward, I think we should look for
>     answers to
>     >>>> the following questions:
>     >>>>
>     >>>>     a) do we have any clues that show, that the proposed "in
>     runner"
>     >>>> solution will not work?
>     >>> OK, thinking about it some more, in the TestStream, we can use the
>     >>> happy coincidence that
>     >>>
>     >>>       LengthPrefixed(C).encode(x, nested=True) ==
>     >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>     >>> nested=False)
>     >>>
>     >>> (where || denotes concatenation) and the fact that we have
>     >>>
>     >>>       C.encode(x, nested=False)
>     >>>
>     >>> in hand.
>     >>>
>     >>> A possible fix here for the OP's question is that when
>     rehydrating the
>     >>> TestStream transform it must behave differently according to
>     the coder
>     >>> used in the subsequent channel (e.g. for known coders, it
>     decodes the
>     >>> elements and emits them directly, but for unknown coders, it
>     prefixes
>     >>> them with their length and emits byte strings. It gets more
>     >>> complicated for nested coders, e.g. for a KV<known-coder,
>     >>> unknown-coder> the channel might be LP(KV<known-coder,
>     unknown-coder))
>     >>> or KV<known-coder, LP(unknown-coder)) which have different
>     encodings
>     >>> (and the latter, which is the default, requires transcoding
>     the bytes
>     >>> to inject the length in the middle which is found by decoding the
>     >>> first component). As well as getting more complex, this really
>     seems
>     >>> to violate the spirit of separation of concerns.
>     >> How do we make the decision if the channel is LP<KV<..>> or
>     >> KV<LP<unknown>, known>? From my understanding it is always the
>     latter,
>     >> because of [2].
>     >>
>     >> [2]
>     >>
>     https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>     <https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48>
>     > It is currently the latter for runners using this code (which
>     not all
>     > do, e.g. the ULR and Dataflow runners). I don't think we want to
>     > ossify this decision as part of the spec. (Note that even what's
>     > "known" and "unknown" can change from runner to runner.)
>     >
>     >>>>     b) do we think, that it will not be robust enough to
>     incorporate the
>     >>>> other use-cases (line generic transform inlining, taking into
>     account
>     >>>> that this applies only to runners that are written in the
>     same language
>     >>>> as the submitting SDK, because otherwise, there is nothing to
>     inline)?
>     >>> Being in the same language is not a prerequisite to
>     "inlining," e.g.
>     >>> the PubSub source on Dataflow is recognized as such and not
>     executed
>     >>> as SDK code but natively.
>     >> Agree, that is actually exactly what happens with the
>     TestStream. The
>     >> transform need not be in the same language, as long as it is
>     completely
>     >> understood by the runner, including the SDK-coder (either
>     explicitly -
>     >> which might be due to the PCollection coder being composed of
>     well-known
>     >> coders only, or implicitly like in the case of TestStream,
>     where the
>     >> elements are encoded using the SDK coder.
>     >>> It is more likely that inlining occurs in the same language if
>     there
>     >>> are UDFs involved.
>     >>>
>     >>>> I'm convinced, that the TestStream-decode expansion solution
>     is an
>     >>>> ad-hoc solution to a generic problem, which is why I'm still
>     bothering
>     >>>> this mailing list with my emails on this. :-)
>     >>>>
>     >>>> WDYT?
>     >>> While not a solution to the general problem, I think the
>     >>> TestStream-only-does-bytes simplifies its definition (primitives
>     >>> should have as simple/easy to implement definitions as
>     possible) and
>     >>> brings it closer to the other root we have: Impulse. (We could
>     go a
>     >>> step further and rather than emitting encoded elements, with
>     the data
>     >>> in the proto itself, it emits sequence numbers, and a
>     subsequent ParDo
>     >>> maps those to concrete elements (e.g. via an in-memory map),
>     but that
>     >>> further step doesn't buy much...)
>     >>>
>     >>> Only runners that want to do inlining would have to take on the
>     >>> complexity of a fully generic solution.
>     >> I think that if the simplification brings something, we can do
>     that, but
>     >> I'd like to understand why we cannot (or should not) use the
>     generic
>     >> solution. I think it definitely *should* be possible to use a
>     generic
>     >> solution, because otherwise the solution would not be generic.
>     And it
>     >> would imply, that we are unable to do generic transform
>     inlining, which
>     >> I would find really strange. That would immediately mean, that
>     we are
>     >> unable to construct classical runner as a special case of the
>     portable
>     >> one, which would be bad I think.
>     >>
>     >> The elements in the TestStreamPayload are encoded with pure
>     SDK-coder,
>     >> or does this go through the LengthPrefixUnknownCoders logic? If
>     not,
>     >> then the problem would be there, because that means, that the
>     SDK-coder
>     >> cannot be (implicitly) defined in the runner. If the elements
>     would be
>     >> encoded using LP, then it would be possible to decode them using
>     >> runner-coder and the problem should be solved, or am I still
>     missing
>     >> some key parts?
>     > Yes, the problem is precisely that there are (unspecified)
>     constraints
>     > on the coder used by the TestStreamPayload. Just requiring that
>     it be
>     > length prefixed is not enough, you have to make constraints on
>     > sometimes pushing down the length prefixing if it's a composite
>     (like
>     > a KV) that depend on what the runner is expected to support in terms
>     > of composites and/or the choices it chooses for the channel (and the
>     > runner, not knowing the coder, can't transcode between these
>     choices).
>     >
>     > The simpler solution is to constrain this coder to just be byte[]
>     > rather than let it be a little bit flexible (but not wholly
>     flexible).
>     >
>     > As for a fully generic solution, I think the issue encountered with
>     > inlining Read vs. TestStream are related to this, but not really the
>     > same. With TestStream one has an encoded representation of the
>     > elements provided by the SDK that the Runner and has no SDK
>     > representation/execution whereas with the Reads one has unencoded
>     > elements in hand and a Coder that is understood by both (so long as
>     > the channel can be negotiated correctly). FWIW, I think the proper
>     > solution to inlining a Read (or other Transform that would typically
>     > be executed in the SDK) is to treat it as a special environment
>     (where
>     > we know logically it can work) and then elide, as possible, the
>     > various encodings, grpc calls, etc. that are unneeded as
>     everything is
>     > in process.
>     >
>     >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>     >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je.ik@seznam.cz
>     <ma...@seznam.cz>>  wrote:
>     >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>     >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan
>     Lukavský<je.ik@seznam.cz <ma...@seznam.cz>> wrote:
>     >>>>>>>> Hi,
>     >>>>>>>>
>     >>>>>>>> I had some more time thinking about this and I'll try to
>     recap that.
>     >>>>>>>> First some invariants:
>     >>>>>>>>
>     >>>>>>>>       a) each PCollection<T> has actually two coders - an
>     _SDK coder_ and a
>     >>>>>>>> _runner coder_. These coders have the property, that each
>     one can
>     >>>>>>>> _decode_ what the other encoded, but the opposite is not
>     true, the
>     >>>>>>>> coders cannot _encode_ what the other _decoded_ (in general).
>     >>>>>>>>
>     >>>>>>>>       b) when is a PCollection<T> computed inside an
>     environment, the
>     >>>>>>>> elements are encoded using SDK coder on the side of
>     SDK-harness and
>     >>>>>>>> decoded using runner coder after receiving in the runner
>     >>>>>>>>
>     >>>>>>>>       c) under specific circumstances, the encode-decode
>     step can be
>     >>>>>>>> optimized out, that is the case where the SDK coder and
>     all its
>     >>>>>>>> subcoders are all well-known to the runner (in the
>     present, that means
>     >>>>>>>> that all the parts present in the model coders set). The
>     reason for that
>     >>>>>>>> is that in this specific situation
>     runner_decode(sdk_encode(X)) = X.
>     >>>>>>>> This property is essential.
>     >>>>>>> However, in general, X can only pass from the SDK to the
>     runner (or
>     >>>>>>> vice versa) in encoded form.
>     >>>>>> In general yes, but we are (mostly) talking transform
>     inlining here, so
>     >>>>>> it that particular situation, the elements might be passed
>     in decoded form.
>     >>>>>>>>       d) from b) immediately follows, that when a
>     PTransform does not run in
>     >>>>>>>> an environment (and this might be due to the transform
>     being runner
>     >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
>     elements have
>     >>>>>>>> to be encoded by SDK coder, immediately following decode
>     by runner
>     >>>>>>>> coder. That (surprisingly) applies even to situations
>     when runner is
>     >>>>>>>> implemented using different language than the client SDK,
>     because it
>     >>>>>>>> implies that the type of produced elements must be one of
>     types encoded
>     >>>>>>>> using model coders (well-known to the runner, otherwise
>     the SDK will not
>     >>>>>>>> be able to consume it). But - due to property c) - this
>     means that this
>     >>>>>>>> encode-decode step can be optimized out. This does not
>     mean that it is
>     >>>>>>>> not (logically) present, though. This is exactly the case
>     of native
>     >>>>>>>> Impulse transform.
>     >>>>>>>>
>     >>>>>>>> Now, from that we can conclude that on the boundary
>     between executable
>     >>>>>>>> stages, or between runner (inlined) transform and
>     executable stage, each
>     >>>>>>>> PCollection has to be encoded using SDK coder and
>     immediately decoded by
>     >>>>>>>> runner coder, *unless this can be optimized out* by
>     property c).
>     >>>>>>>>
>     >>>>>>>> This gives us two options where to implement this
>     encode/decode step:
>     >>>>>>>>
>     >>>>>>>>       1) completely inside runner with the possibility to
>     optimize the
>     >>>>>>>> encode/decode step by identity under right circumstances
>     >>>>>>>>
>     >>>>>>>>       2) partly in the runner and partly in the SDK -
>     that is we encode
>     >>>>>>>> elements of PCollection using SDK coder into bytes, pass
>     those to the
>     >>>>>>>> SDK harness and apply a custom decode step there. This
>     works because SDK
>     >>>>>>>> coder encoded elements are in byte[], and that is
>     well-known coder type.
>     >>>>>>>> We again only leverage property c) and optimize the SDK
>     coder encode,
>     >>>>>>>> runner decode step out.
>     >>>>>>>>
>     >>>>>>>> The option 2) is exactly the proposal of TestStream
>     producing byte[] and
>     >>>>>>>> decoding inside SDK-harness, the TestStream is actually
>     inlined
>     >>>>>>>> transform, the elements are produced directly in runner
>     (the SDK coder
>     >>>>>>>> is not known to the runner, but that does not matter,
>     because the
>     >>>>>>>> elements are already encoded by client).
>     >>>>>>>>
>     >>>>>>>>      From the above it seems to me, that option 1) should
>     be preferred, because:
>     >>>>>>>>
>     >>>>>>>>       i) it is generic, applicable to all inlined
>     transforms, any sources
>     >>>>>>>>
>     >>>>>>>>       ii) it is consistent with how things logically work
>     underneath
>     >>>>>>>>
>     >>>>>>>>       iii) it offers better room for optimization -
>     option 2) might result
>     >>>>>>>> in cases when the elements are passed from the runner to
>     the SDK-harness
>     >>>>>>>> only for the sake of the decoding from SDK coder and
>     immediately
>     >>>>>>>> encoding back using SDK-coder and returned back to the
>     runner. This
>     >>>>>>>> would be the case when TestStream would be directly
>     consumed by inlined
>     >>>>>>>> (or external) transform.
>     >>>>>>> (1) is not possible if the Coder in question is not known
>     to the
>     >>>>>>> Runner, which is why I proposed (2).
>     >>>>>> There is no particular need for the coder to be known. If
>     transform is
>     >>>>>> to be inlined, what *has* to be known is the SDK-encoded
>     form of the
>     >>>>>> elements. That holds true if:
>     >>>>>>
>     >>>>>>      a) either the SDK coder is known, or
>     >>>>>>
>     >>>>>>      b) encoded form of the produced elements is known in
>     advance
>     >>>>>>
>     >>>>>> For TestStream it is the case b). For inlined primitive
>     Read (or any
>     >>>>>> other transform which executes code) it is a).
>     >>>>> There's another hitch here for TestStream. For historical
>     reasons,
>     >>>>> coders actually represent two encodings: nested (aka self
>     delimiting)
>     >>>>> and unnested. TestStream elements are given as unnested
>     encoded bytes,
>     >>>>> but the nested encoding is required for sending data to the
>     SDK. The
>     >>>>> runner can't go from <nested encoding> to <unnested
>     encoding> for an
>     >>>>> arbitrary unknown coder.
>     >>>>>
>     >>>>> (Even if it weren't for this complication, to be able to
>     send already
>     >>>>> encoded bytes of an unknown coder to the SDK will also
>     complicate the
>     >>>>> logic in choosing the coder to be used for the channel and
>     sending the
>     >>>>> data, which is some of what you're running into (but can be
>     solved
>     >>>>> differently for inlined reads as the coder can always be
>     known by the
>     >>>>> runner).)
>

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
Makes sense, I missed that part. That is why a generic "inlining" scheme 
is problematic, because that depends on how does the runner encodes the 
elements on the wire. And it is where TestStream's output needs to be 
encoded into raw bytes, because the wire coder is unknown the the SDK 
when submitting the job.

Thanks for the clarification and for bearing with me!

  Jan

On 9/7/21 7:55 PM, Robert Bradshaw wrote:

> On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>>> It is currently the latter for runners using this code (which not all
>>> do, e.g. the ULR and Dataflow runners). I don't think we want to
>>> ossify this decision as part of the spec. (Note that even what's
>>> "known" and "unknown" can change from runner to runner.)
>> This is interesting and unexpected for me. How do runners decide about
>> how they encode elements between SDK harness and the runner? How do they
>> inform the SDK harness about this decision? My impression was that this
>> is well-defined at the model level. If not, then we have the reason for
>> misunderstanding in this conversation. :-)
> The coder id to use for a channel is specified by the runner on the
> channel (both the input and output) operation when sending a process
> bundle descriptor. They decide based on their capabilities (e.g. what
> coders they understand vs. what needs wrapping) SDK capabilities (e.g.
> for optimizations like param windowed value coder) and their needs
> (e.g. to do a GBK, they need the key and value bytes, not just the
> key-value bytes).

Re: Unexpected in TestStream in Portable Mode

Posted by Reuven Lax <re...@google.com>.
Historically the DataflowRunner has been much more careful about not
breaking update, since this is a frequent operation by Dataflow users. I
think we've been less careful aboutt other runners, but as we see clearly
here Fllnk users do care about this as well, so we should probably test
upgrade compatibility for Flink.

One strategy that Dataflow uses is to avoid embedding the Java serialized
form of a Coder in the graph, as this is a much higher risk of breakage (as
we see with the issue you llnked to). Possibly similar strategies should be
investigated for Fllink.

Reuven

On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:

> > Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
> > care about Context because they predated this deprecation, and
> > changing coders is hard (due to no way to update the encoding for a
> > streaming pipeline).
> This is unrelated, but - regarding changing coders due to concerns about
> pipeline upgrades, we break this quite often, at least for some runners.
> Most recently [1].
>
> > It is currently the latter for runners using this code (which not all
> > do, e.g. the ULR and Dataflow runners). I don't think we want to
> > ossify this decision as part of the spec. (Note that even what's
> > "known" and "unknown" can change from runner to runner.)
> This is interesting and unexpected for me. How do runners decide about
> how they encode elements between SDK harness and the runner? How do they
> inform the SDK harness about this decision? My impression was that this
> is well-defined at the model level. If not, then we have the reason for
> misunderstanding in this conversation. :-)
>
>   Jan
>
> [1]
>
> https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E
>
> On 9/4/21 7:32 PM, Robert Bradshaw wrote:
> > On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz> wrote:
> >> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
> >>
> >>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>  wrote:
> >>>> Hi Robert,
> >>>>
> >>>>> There's another hitch here for TestStream. For historical reasons,
> >>>>> coders actually represent two encodings: nested (aka self delimiting)
> >>>>> and unnested. TestStream elements are given as unnested encoded
> bytes,
> >>>>> but the nested encoding is required for sending data to the SDK. The
> >>>>> runner can't go from <nested encoding> to <unnested encoding> for an
> >>>>> arbitrary unknown coder.
> >>>>>
> >>>>> (Even if it weren't for this complication, to be able to send already
> >>>>> encoded bytes of an unknown coder to the SDK will also complicate the
> >>>>> logic in choosing the coder to be used for the channel and sending
> the
> >>>>> data, which is some of what you're running into (but can be solved
> >>>>> differently for inlined reads as the coder can always be known by the
> >>>>> runner).)
> >>>> It is hard for me to argue with "historical reasons". But - the
> "nested"
> >>>> and "unnested" coders look very similar to SDK-coder and runner-coder
> >>>> spaces.
> >>> Unfortunately, they're actually orthogonal to that.
> >> Hm, do you mean the Context passed to the encode/decode method? [1] That
> >> seems to be deprecated, I assume that most coders use the default
> >> implementation and simply ignore the Context?
> > Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
> > care about Context because they predated this deprecation, and
> > changing coders is hard (due to no way to update the encoding for a
> > streaming pipeline).
> >
> >> Even if not - whether or
> >> not the elements are encoded using NESTED Context or UNNESTED Context
> >> should be part of the contract of TestStream, right? Most likely it is
> >> the UNNESTED one, if I understand correctly what that does. Under what
> >> conditions is the deprecated encode/decode method used?
> > Yes, it's the UNNESTED one.
> >
> >> [1]
> >>
> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
> >>
> >>>> The runner's responsibility is not to go from "<nested
> >>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary coder.
> >>>> That is really impossible. But a coder is a function, right? Function
> >>>> maps from universe A to universe B (in general). TestStream provides a
> >>>> set of elements, and these elements are the "universe". For those
> >>>> elements it also provides the encoded form, which can be interpreted
> as
> >>>> the definition of the coder.
> >>> The problem here is that there is not "the encoded form" for a Coder
> >>> but two encoded forms, and we have the wrong one. Things could be made
> >>> to work if we had the other.
> >> Which two encoded forms do you refer to? Elements encoded by both the
> >> SDK-coder and runner-coder (and I ignore the Context here once again)
> >> have the same binary representation (which they must have, otherwise it
> >> would be impossible to decode elements coming from the runner to the
> >> SDK-harness or vice-versa).
> >>>> Therefore - technically (and formally) -
> >>>> the SDK coder for the TestStream is known to the runner, regardless of
> >>>> the language the runner is written in.
> >>>>
> >>>> To move  this discussion forward, I think we should look for answers
> to
> >>>> the following questions:
> >>>>
> >>>>     a) do we have any clues that show, that the proposed "in runner"
> >>>> solution will not work?
> >>> OK, thinking about it some more, in the TestStream, we can use the
> >>> happy coincidence that
> >>>
> >>>       LengthPrefixed(C).encode(x, nested=True) ==
> >>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
> >>> nested=False)
> >>>
> >>> (where || denotes concatenation) and the fact that we have
> >>>
> >>>       C.encode(x, nested=False)
> >>>
> >>> in hand.
> >>>
> >>> A possible fix here for the OP's question is that when rehydrating the
> >>> TestStream transform it must behave differently according to the coder
> >>> used in the subsequent channel (e.g. for known coders, it decodes the
> >>> elements and emits them directly, but for unknown coders, it prefixes
> >>> them with their length and emits byte strings. It gets more
> >>> complicated for nested coders, e.g. for a KV<known-coder,
> >>> unknown-coder> the channel might be LP(KV<known-coder, unknown-coder))
> >>> or KV<known-coder, LP(unknown-coder)) which have different encodings
> >>> (and the latter, which is the default, requires transcoding the bytes
> >>> to inject the length in the middle which is found by decoding the
> >>> first component). As well as getting more complex, this really seems
> >>> to violate the spirit of separation of concerns.
> >> How do we make the decision if the channel is LP<KV<..>> or
> >> KV<LP<unknown>, known>? From my understanding it is always the latter,
> >> because of [2].
> >>
> >> [2]
> >>
> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
> > It is currently the latter for runners using this code (which not all
> > do, e.g. the ULR and Dataflow runners). I don't think we want to
> > ossify this decision as part of the spec. (Note that even what's
> > "known" and "unknown" can change from runner to runner.)
> >
> >>>>     b) do we think, that it will not be robust enough to incorporate
> the
> >>>> other use-cases (line generic transform inlining, taking into account
> >>>> that this applies only to runners that are written in the same
> language
> >>>> as the submitting SDK, because otherwise, there is nothing to inline)?
> >>> Being in the same language is not a prerequisite to "inlining," e.g.
> >>> the PubSub source on Dataflow is recognized as such and not executed
> >>> as SDK code but natively.
> >> Agree, that is actually exactly what happens with the TestStream. The
> >> transform need not be in the same language, as long as it is completely
> >> understood by the runner, including the SDK-coder (either explicitly -
> >> which might be due to the PCollection coder being composed of well-known
> >> coders only, or implicitly like in the case of TestStream, where the
> >> elements are encoded using the SDK coder.
> >>> It is more likely that inlining occurs in the same language if there
> >>> are UDFs involved.
> >>>
> >>>> I'm convinced, that the TestStream-decode expansion solution is an
> >>>> ad-hoc solution to a generic problem, which is why I'm still bothering
> >>>> this mailing list with my emails on this. :-)
> >>>>
> >>>> WDYT?
> >>> While not a solution to the general problem, I think the
> >>> TestStream-only-does-bytes simplifies its definition (primitives
> >>> should have as simple/easy to implement definitions as possible) and
> >>> brings it closer to the other root we have: Impulse. (We could go a
> >>> step further and rather than emitting encoded elements, with the data
> >>> in the proto itself, it emits sequence numbers, and a subsequent ParDo
> >>> maps those to concrete elements (e.g. via an in-memory map), but that
> >>> further step doesn't buy much...)
> >>>
> >>> Only runners that want to do inlining would have to take on the
> >>> complexity of a fully generic solution.
> >> I think that if the simplification brings something, we can do that, but
> >> I'd like to understand why we cannot (or should not) use the generic
> >> solution. I think it definitely *should* be possible to use a generic
> >> solution, because otherwise the solution would not be generic. And it
> >> would imply, that we are unable to do generic transform inlining, which
> >> I would find really strange. That would immediately mean, that we are
> >> unable to construct classical runner as a special case of the portable
> >> one, which would be bad I think.
> >>
> >> The elements in the TestStreamPayload are encoded with pure SDK-coder,
> >> or does this go through the LengthPrefixUnknownCoders logic? If not,
> >> then the problem would be there, because that means, that the SDK-coder
> >> cannot be (implicitly) defined in the runner. If the elements would be
> >> encoded using LP, then it would be possible to decode them using
> >> runner-coder and the problem should be solved, or am I still missing
> >> some key parts?
> > Yes, the problem is precisely that there are (unspecified) constraints
> > on the coder used by the TestStreamPayload. Just requiring that it be
> > length prefixed is not enough, you have to make constraints on
> > sometimes pushing down the length prefixing if it's a composite (like
> > a KV) that depend on what the runner is expected to support in terms
> > of composites and/or the choices it chooses for the channel (and the
> > runner, not knowing the coder, can't transcode between these choices).
> >
> > The simpler solution is to constrain this coder to just be byte[]
> > rather than let it be a little bit flexible (but not wholly flexible).
> >
> > As for a fully generic solution, I think the issue encountered with
> > inlining Read vs. TestStream are related to this, but not really the
> > same. With TestStream one has an encoded representation of the
> > elements provided by the SDK that the Runner and has no SDK
> > representation/execution whereas with the Reads one has unencoded
> > elements in hand and a Coder that is understood by both (so long as
> > the channel can be negotiated correctly). FWIW, I think the proper
> > solution to inlining a Read (or other Transform that would typically
> > be executed in the SDK) is to treat it as a special environment (where
> > we know logically it can work) and then elide, as possible, the
> > various encodings, grpc calls, etc. that are unneeded as everything is
> > in process.
> >
> >>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
> >>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>  wrote:
> >>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
> >>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>
> wrote:
> >>>>>>>> Hi,
> >>>>>>>>
> >>>>>>>> I had some more time thinking about this and I'll try to recap
> that.
> >>>>>>>> First some invariants:
> >>>>>>>>
> >>>>>>>>       a) each PCollection<T> has actually two coders - an _SDK
> coder_ and a
> >>>>>>>> _runner coder_. These coders have the property, that each one can
> >>>>>>>> _decode_ what the other encoded, but the opposite is not true, the
> >>>>>>>> coders cannot _encode_ what the other _decoded_ (in general).
> >>>>>>>>
> >>>>>>>>       b) when is a PCollection<T> computed inside an environment,
> the
> >>>>>>>> elements are encoded using SDK coder on the side of SDK-harness
> and
> >>>>>>>> decoded using runner coder after receiving in the runner
> >>>>>>>>
> >>>>>>>>       c) under specific circumstances, the encode-decode step can
> be
> >>>>>>>> optimized out, that is the case where the SDK coder and all its
> >>>>>>>> subcoders are all well-known to the runner (in the present, that
> means
> >>>>>>>> that all the parts present in the model coders set). The reason
> for that
> >>>>>>>> is that in this specific situation runner_decode(sdk_encode(X)) =
> X.
> >>>>>>>> This property is essential.
> >>>>>>> However, in general, X can only pass from the SDK to the runner (or
> >>>>>>> vice versa) in encoded form.
> >>>>>> In general yes, but we are (mostly) talking transform inlining
> here, so
> >>>>>> it that particular situation, the elements might be passed in
> decoded form.
> >>>>>>>>       d) from b) immediately follows, that when a PTransform does
> not run in
> >>>>>>>> an environment (and this might be due to the transform being
> runner
> >>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the
> elements have
> >>>>>>>> to be encoded by SDK coder, immediately following decode by runner
> >>>>>>>> coder. That (surprisingly) applies even to situations when runner
> is
> >>>>>>>> implemented using different language than the client SDK, because
> it
> >>>>>>>> implies that the type of produced elements must be one of types
> encoded
> >>>>>>>> using model coders (well-known to the runner, otherwise the SDK
> will not
> >>>>>>>> be able to consume it). But - due to property c) - this means
> that this
> >>>>>>>> encode-decode step can be optimized out. This does not mean that
> it is
> >>>>>>>> not (logically) present, though. This is exactly the case of
> native
> >>>>>>>> Impulse transform.
> >>>>>>>>
> >>>>>>>> Now, from that we can conclude that on the boundary between
> executable
> >>>>>>>> stages, or between runner (inlined) transform and executable
> stage, each
> >>>>>>>> PCollection has to be encoded using SDK coder and immediately
> decoded by
> >>>>>>>> runner coder, *unless this can be optimized out* by property c).
> >>>>>>>>
> >>>>>>>> This gives us two options where to implement this encode/decode
> step:
> >>>>>>>>
> >>>>>>>>       1) completely inside runner with the possibility to
> optimize the
> >>>>>>>> encode/decode step by identity under right circumstances
> >>>>>>>>
> >>>>>>>>       2) partly in the runner and partly in the SDK - that is we
> encode
> >>>>>>>> elements of PCollection using SDK coder into bytes, pass those to
> the
> >>>>>>>> SDK harness and apply a custom decode step there. This works
> because SDK
> >>>>>>>> coder encoded elements are in byte[], and that is well-known
> coder type.
> >>>>>>>> We again only leverage property c) and optimize the SDK coder
> encode,
> >>>>>>>> runner decode step out.
> >>>>>>>>
> >>>>>>>> The option 2) is exactly the proposal of TestStream producing
> byte[] and
> >>>>>>>> decoding inside SDK-harness, the TestStream is actually inlined
> >>>>>>>> transform, the elements are produced directly in runner (the SDK
> coder
> >>>>>>>> is not known to the runner, but that does not matter, because the
> >>>>>>>> elements are already encoded by client).
> >>>>>>>>
> >>>>>>>>      From the above it seems to me, that option 1) should be
> preferred, because:
> >>>>>>>>
> >>>>>>>>       i) it is generic, applicable to all inlined transforms, any
> sources
> >>>>>>>>
> >>>>>>>>       ii) it is consistent with how things logically work
> underneath
> >>>>>>>>
> >>>>>>>>       iii) it offers better room for optimization - option 2)
> might result
> >>>>>>>> in cases when the elements are passed from the runner to the
> SDK-harness
> >>>>>>>> only for the sake of the decoding from SDK coder and immediately
> >>>>>>>> encoding back using SDK-coder and returned back to the runner.
> This
> >>>>>>>> would be the case when TestStream would be directly consumed by
> inlined
> >>>>>>>> (or external) transform.
> >>>>>>> (1) is not possible if the Coder in question is not known to the
> >>>>>>> Runner, which is why I proposed (2).
> >>>>>> There is no particular need for the coder to be known. If transform
> is
> >>>>>> to be inlined, what *has* to be known is the SDK-encoded form of the
> >>>>>> elements. That holds true if:
> >>>>>>
> >>>>>>      a) either the SDK coder is known, or
> >>>>>>
> >>>>>>      b) encoded form of the produced elements is known in advance
> >>>>>>
> >>>>>> For TestStream it is the case b). For inlined primitive Read (or any
> >>>>>> other transform which executes code) it is a).
> >>>>> There's another hitch here for TestStream. For historical reasons,
> >>>>> coders actually represent two encodings: nested (aka self delimiting)
> >>>>> and unnested. TestStream elements are given as unnested encoded
> bytes,
> >>>>> but the nested encoding is required for sending data to the SDK. The
> >>>>> runner can't go from <nested encoding> to <unnested encoding> for an
> >>>>> arbitrary unknown coder.
> >>>>>
> >>>>> (Even if it weren't for this complication, to be able to send already
> >>>>> encoded bytes of an unknown coder to the SDK will also complicate the
> >>>>> logic in choosing the coder to be used for the channel and sending
> the
> >>>>> data, which is some of what you're running into (but can be solved
> >>>>> differently for inlined reads as the coder can always be known by the
> >>>>> runner).)
>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
On Mon, Sep 6, 2021 at 1:29 AM Jan Lukavský <je...@seznam.cz> wrote:
>
> > It is currently the latter for runners using this code (which not all
> > do, e.g. the ULR and Dataflow runners). I don't think we want to
> > ossify this decision as part of the spec. (Note that even what's
> > "known" and "unknown" can change from runner to runner.)
> This is interesting and unexpected for me. How do runners decide about
> how they encode elements between SDK harness and the runner? How do they
> inform the SDK harness about this decision? My impression was that this
> is well-defined at the model level. If not, then we have the reason for
> misunderstanding in this conversation. :-)

The coder id to use for a channel is specified by the runner on the
channel (both the input and output) operation when sending a process
bundle descriptor. They decide based on their capabilities (e.g. what
coders they understand vs. what needs wrapping) SDK capabilities (e.g.
for optimizations like param windowed value coder) and their needs
(e.g. to do a GBK, they need the key and value bytes, not just the
key-value bytes).

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
> Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
> care about Context because they predated this deprecation, and
> changing coders is hard (due to no way to update the encoding for a
> streaming pipeline).
This is unrelated, but - regarding changing coders due to concerns about 
pipeline upgrades, we break this quite often, at least for some runners. 
Most recently [1].

> It is currently the latter for runners using this code (which not all
> do, e.g. the ULR and Dataflow runners). I don't think we want to
> ossify this decision as part of the spec. (Note that even what's
> "known" and "unknown" can change from runner to runner.)
This is interesting and unexpected for me. How do runners decide about 
how they encode elements between SDK harness and the runner? How do they 
inform the SDK harness about this decision? My impression was that this 
is well-defined at the model level. If not, then we have the reason for 
misunderstanding in this conversation. :-)

  Jan

[1] 
https://lists.apache.org/thread.html/r51ee0bbaba2dcef13524a189c1f579f209483418a1568acff0e2c789%40%3Cdev.beam.apache.org%3E

On 9/4/21 7:32 PM, Robert Bradshaw wrote:
> On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz> wrote:
>> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>>
>>> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>  wrote:
>>>> Hi Robert,
>>>>
>>>>> There's another hitch here for TestStream. For historical reasons,
>>>>> coders actually represent two encodings: nested (aka self delimiting)
>>>>> and unnested. TestStream elements are given as unnested encoded bytes,
>>>>> but the nested encoding is required for sending data to the SDK. The
>>>>> runner can't go from <nested encoding> to <unnested encoding> for an
>>>>> arbitrary unknown coder.
>>>>>
>>>>> (Even if it weren't for this complication, to be able to send already
>>>>> encoded bytes of an unknown coder to the SDK will also complicate the
>>>>> logic in choosing the coder to be used for the channel and sending the
>>>>> data, which is some of what you're running into (but can be solved
>>>>> differently for inlined reads as the coder can always be known by the
>>>>> runner).)
>>>> It is hard for me to argue with "historical reasons". But - the "nested"
>>>> and "unnested" coders look very similar to SDK-coder and runner-coder
>>>> spaces.
>>> Unfortunately, they're actually orthogonal to that.
>> Hm, do you mean the Context passed to the encode/decode method? [1] That
>> seems to be deprecated, I assume that most coders use the default
>> implementation and simply ignore the Context?
> Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
> care about Context because they predated this deprecation, and
> changing coders is hard (due to no way to update the encoding for a
> streaming pipeline).
>
>> Even if not - whether or
>> not the elements are encoded using NESTED Context or UNNESTED Context
>> should be part of the contract of TestStream, right? Most likely it is
>> the UNNESTED one, if I understand correctly what that does. Under what
>> conditions is the deprecated encode/decode method used?
> Yes, it's the UNNESTED one.
>
>> [1]
>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>>
>>>> The runner's responsibility is not to go from "<nested
>>>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary coder.
>>>> That is really impossible. But a coder is a function, right? Function
>>>> maps from universe A to universe B (in general). TestStream provides a
>>>> set of elements, and these elements are the "universe". For those
>>>> elements it also provides the encoded form, which can be interpreted as
>>>> the definition of the coder.
>>> The problem here is that there is not "the encoded form" for a Coder
>>> but two encoded forms, and we have the wrong one. Things could be made
>>> to work if we had the other.
>> Which two encoded forms do you refer to? Elements encoded by both the
>> SDK-coder and runner-coder (and I ignore the Context here once again)
>> have the same binary representation (which they must have, otherwise it
>> would be impossible to decode elements coming from the runner to the
>> SDK-harness or vice-versa).
>>>> Therefore - technically (and formally) -
>>>> the SDK coder for the TestStream is known to the runner, regardless of
>>>> the language the runner is written in.
>>>>
>>>> To move  this discussion forward, I think we should look for answers to
>>>> the following questions:
>>>>
>>>>     a) do we have any clues that show, that the proposed "in runner"
>>>> solution will not work?
>>> OK, thinking about it some more, in the TestStream, we can use the
>>> happy coincidence that
>>>
>>>       LengthPrefixed(C).encode(x, nested=True) ==
>>> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
>>> nested=False)
>>>
>>> (where || denotes concatenation) and the fact that we have
>>>
>>>       C.encode(x, nested=False)
>>>
>>> in hand.
>>>
>>> A possible fix here for the OP's question is that when rehydrating the
>>> TestStream transform it must behave differently according to the coder
>>> used in the subsequent channel (e.g. for known coders, it decodes the
>>> elements and emits them directly, but for unknown coders, it prefixes
>>> them with their length and emits byte strings. It gets more
>>> complicated for nested coders, e.g. for a KV<known-coder,
>>> unknown-coder> the channel might be LP(KV<known-coder, unknown-coder))
>>> or KV<known-coder, LP(unknown-coder)) which have different encodings
>>> (and the latter, which is the default, requires transcoding the bytes
>>> to inject the length in the middle which is found by decoding the
>>> first component). As well as getting more complex, this really seems
>>> to violate the spirit of separation of concerns.
>> How do we make the decision if the channel is LP<KV<..>> or
>> KV<LP<unknown>, known>? From my understanding it is always the latter,
>> because of [2].
>>
>> [2]
>> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
> It is currently the latter for runners using this code (which not all
> do, e.g. the ULR and Dataflow runners). I don't think we want to
> ossify this decision as part of the spec. (Note that even what's
> "known" and "unknown" can change from runner to runner.)
>
>>>>     b) do we think, that it will not be robust enough to incorporate the
>>>> other use-cases (line generic transform inlining, taking into account
>>>> that this applies only to runners that are written in the same language
>>>> as the submitting SDK, because otherwise, there is nothing to inline)?
>>> Being in the same language is not a prerequisite to "inlining," e.g.
>>> the PubSub source on Dataflow is recognized as such and not executed
>>> as SDK code but natively.
>> Agree, that is actually exactly what happens with the TestStream. The
>> transform need not be in the same language, as long as it is completely
>> understood by the runner, including the SDK-coder (either explicitly -
>> which might be due to the PCollection coder being composed of well-known
>> coders only, or implicitly like in the case of TestStream, where the
>> elements are encoded using the SDK coder.
>>> It is more likely that inlining occurs in the same language if there
>>> are UDFs involved.
>>>
>>>> I'm convinced, that the TestStream-decode expansion solution is an
>>>> ad-hoc solution to a generic problem, which is why I'm still bothering
>>>> this mailing list with my emails on this. :-)
>>>>
>>>> WDYT?
>>> While not a solution to the general problem, I think the
>>> TestStream-only-does-bytes simplifies its definition (primitives
>>> should have as simple/easy to implement definitions as possible) and
>>> brings it closer to the other root we have: Impulse. (We could go a
>>> step further and rather than emitting encoded elements, with the data
>>> in the proto itself, it emits sequence numbers, and a subsequent ParDo
>>> maps those to concrete elements (e.g. via an in-memory map), but that
>>> further step doesn't buy much...)
>>>
>>> Only runners that want to do inlining would have to take on the
>>> complexity of a fully generic solution.
>> I think that if the simplification brings something, we can do that, but
>> I'd like to understand why we cannot (or should not) use the generic
>> solution. I think it definitely *should* be possible to use a generic
>> solution, because otherwise the solution would not be generic. And it
>> would imply, that we are unable to do generic transform inlining, which
>> I would find really strange. That would immediately mean, that we are
>> unable to construct classical runner as a special case of the portable
>> one, which would be bad I think.
>>
>> The elements in the TestStreamPayload are encoded with pure SDK-coder,
>> or does this go through the LengthPrefixUnknownCoders logic? If not,
>> then the problem would be there, because that means, that the SDK-coder
>> cannot be (implicitly) defined in the runner. If the elements would be
>> encoded using LP, then it would be possible to decode them using
>> runner-coder and the problem should be solved, or am I still missing
>> some key parts?
> Yes, the problem is precisely that there are (unspecified) constraints
> on the coder used by the TestStreamPayload. Just requiring that it be
> length prefixed is not enough, you have to make constraints on
> sometimes pushing down the length prefixing if it's a composite (like
> a KV) that depend on what the runner is expected to support in terms
> of composites and/or the choices it chooses for the channel (and the
> runner, not knowing the coder, can't transcode between these choices).
>
> The simpler solution is to constrain this coder to just be byte[]
> rather than let it be a little bit flexible (but not wholly flexible).
>
> As for a fully generic solution, I think the issue encountered with
> inlining Read vs. TestStream are related to this, but not really the
> same. With TestStream one has an encoded representation of the
> elements provided by the SDK that the Runner and has no SDK
> representation/execution whereas with the Reads one has unencoded
> elements in hand and a Coder that is understood by both (so long as
> the channel can be negotiated correctly). FWIW, I think the proper
> solution to inlining a Read (or other Transform that would typically
> be executed in the SDK) is to treat it as a special environment (where
> we know logically it can work) and then elide, as possible, the
> various encodings, grpc calls, etc. that are unneeded as everything is
> in process.
>
>>>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>  wrote:
>>>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>  wrote:
>>>>>>>> Hi,
>>>>>>>>
>>>>>>>> I had some more time thinking about this and I'll try to recap that.
>>>>>>>> First some invariants:
>>>>>>>>
>>>>>>>>       a) each PCollection<T> has actually two coders - an _SDK coder_ and a
>>>>>>>> _runner coder_. These coders have the property, that each one can
>>>>>>>> _decode_ what the other encoded, but the opposite is not true, the
>>>>>>>> coders cannot _encode_ what the other _decoded_ (in general).
>>>>>>>>
>>>>>>>>       b) when is a PCollection<T> computed inside an environment, the
>>>>>>>> elements are encoded using SDK coder on the side of SDK-harness and
>>>>>>>> decoded using runner coder after receiving in the runner
>>>>>>>>
>>>>>>>>       c) under specific circumstances, the encode-decode step can be
>>>>>>>> optimized out, that is the case where the SDK coder and all its
>>>>>>>> subcoders are all well-known to the runner (in the present, that means
>>>>>>>> that all the parts present in the model coders set). The reason for that
>>>>>>>> is that in this specific situation runner_decode(sdk_encode(X)) = X.
>>>>>>>> This property is essential.
>>>>>>> However, in general, X can only pass from the SDK to the runner (or
>>>>>>> vice versa) in encoded form.
>>>>>> In general yes, but we are (mostly) talking transform inlining here, so
>>>>>> it that particular situation, the elements might be passed in decoded form.
>>>>>>>>       d) from b) immediately follows, that when a PTransform does not run in
>>>>>>>> an environment (and this might be due to the transform being runner
>>>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the elements have
>>>>>>>> to be encoded by SDK coder, immediately following decode by runner
>>>>>>>> coder. That (surprisingly) applies even to situations when runner is
>>>>>>>> implemented using different language than the client SDK, because it
>>>>>>>> implies that the type of produced elements must be one of types encoded
>>>>>>>> using model coders (well-known to the runner, otherwise the SDK will not
>>>>>>>> be able to consume it). But - due to property c) - this means that this
>>>>>>>> encode-decode step can be optimized out. This does not mean that it is
>>>>>>>> not (logically) present, though. This is exactly the case of native
>>>>>>>> Impulse transform.
>>>>>>>>
>>>>>>>> Now, from that we can conclude that on the boundary between executable
>>>>>>>> stages, or between runner (inlined) transform and executable stage, each
>>>>>>>> PCollection has to be encoded using SDK coder and immediately decoded by
>>>>>>>> runner coder, *unless this can be optimized out* by property c).
>>>>>>>>
>>>>>>>> This gives us two options where to implement this encode/decode step:
>>>>>>>>
>>>>>>>>       1) completely inside runner with the possibility to optimize the
>>>>>>>> encode/decode step by identity under right circumstances
>>>>>>>>
>>>>>>>>       2) partly in the runner and partly in the SDK - that is we encode
>>>>>>>> elements of PCollection using SDK coder into bytes, pass those to the
>>>>>>>> SDK harness and apply a custom decode step there. This works because SDK
>>>>>>>> coder encoded elements are in byte[], and that is well-known coder type.
>>>>>>>> We again only leverage property c) and optimize the SDK coder encode,
>>>>>>>> runner decode step out.
>>>>>>>>
>>>>>>>> The option 2) is exactly the proposal of TestStream producing byte[] and
>>>>>>>> decoding inside SDK-harness, the TestStream is actually inlined
>>>>>>>> transform, the elements are produced directly in runner (the SDK coder
>>>>>>>> is not known to the runner, but that does not matter, because the
>>>>>>>> elements are already encoded by client).
>>>>>>>>
>>>>>>>>      From the above it seems to me, that option 1) should be preferred, because:
>>>>>>>>
>>>>>>>>       i) it is generic, applicable to all inlined transforms, any sources
>>>>>>>>
>>>>>>>>       ii) it is consistent with how things logically work underneath
>>>>>>>>
>>>>>>>>       iii) it offers better room for optimization - option 2) might result
>>>>>>>> in cases when the elements are passed from the runner to the SDK-harness
>>>>>>>> only for the sake of the decoding from SDK coder and immediately
>>>>>>>> encoding back using SDK-coder and returned back to the runner. This
>>>>>>>> would be the case when TestStream would be directly consumed by inlined
>>>>>>>> (or external) transform.
>>>>>>> (1) is not possible if the Coder in question is not known to the
>>>>>>> Runner, which is why I proposed (2).
>>>>>> There is no particular need for the coder to be known. If transform is
>>>>>> to be inlined, what *has* to be known is the SDK-encoded form of the
>>>>>> elements. That holds true if:
>>>>>>
>>>>>>      a) either the SDK coder is known, or
>>>>>>
>>>>>>      b) encoded form of the produced elements is known in advance
>>>>>>
>>>>>> For TestStream it is the case b). For inlined primitive Read (or any
>>>>>> other transform which executes code) it is a).
>>>>> There's another hitch here for TestStream. For historical reasons,
>>>>> coders actually represent two encodings: nested (aka self delimiting)
>>>>> and unnested. TestStream elements are given as unnested encoded bytes,
>>>>> but the nested encoding is required for sending data to the SDK. The
>>>>> runner can't go from <nested encoding> to <unnested encoding> for an
>>>>> arbitrary unknown coder.
>>>>>
>>>>> (Even if it weren't for this complication, to be able to send already
>>>>> encoded bytes of an unknown coder to the SDK will also complicate the
>>>>> logic in choosing the coder to be used for the channel and sending the
>>>>> data, which is some of what you're running into (but can be solved
>>>>> differently for inlined reads as the coder can always be known by the
>>>>> runner).)

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
On Fri, Sep 3, 2021 at 12:50 PM Robert Bradshaw <ro...@google.com> wrote:
>
> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský <je...@seznam.cz> wrote:
> >
> > Hi Robert,
> >
> > > There's another hitch here for TestStream. For historical reasons,
> > > coders actually represent two encodings: nested (aka self delimiting)
> > > and unnested. TestStream elements are given as unnested encoded bytes,
> > > but the nested encoding is required for sending data to the SDK. The
> > > runner can't go from <nested encoding> to <unnested encoding> for an
> > > arbitrary unknown coder.
> > >
> > > (Even if it weren't for this complication, to be able to send already
> > > encoded bytes of an unknown coder to the SDK will also complicate the
> > > logic in choosing the coder to be used for the channel and sending the
> > > data, which is some of what you're running into (but can be solved
> > > differently for inlined reads as the coder can always be known by the
> > > runner).)
> > It is hard for me to argue with "historical reasons". But - the "nested"
> > and "unnested" coders look very similar to SDK-coder and runner-coder
> > spaces.
>
> Unfortunately, they're actually orthogonal to that.
>
> > The runner's responsibility is not to go from "<nested
> > encoding>" (SDK coder) to "<unnested encoding>" for arbitrary coder.
> > That is really impossible. But a coder is a function, right? Function
> > maps from universe A to universe B (in general). TestStream provides a
> > set of elements, and these elements are the "universe". For those
> > elements it also provides the encoded form, which can be interpreted as
> > the definition of the coder.
>
> The problem here is that there is not "the encoded form" for a Coder
> but two encoded forms, and we have the wrong one. Things could be made
> to work if we had the other.
>
> > Therefore - technically (and formally) -
> > the SDK coder for the TestStream is known to the runner, regardless of
> > the language the runner is written in.
> >
> > To move  this discussion forward, I think we should look for answers to
> > the following questions:
> >
> >   a) do we have any clues that show, that the proposed "in runner"
> > solution will not work?
>
> OK, thinking about it some more, in the TestStream, we can use the
> happy coincidence that
>
>     LengthPrefixed(C).encode(x, nested=True) ==
> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
> nested=False)
>
> (where || denotes concatenation) and the fact that we have
>
>     C.encode(x, nested=False)
>
> in hand.
>
> A possible fix here for the OP's question is that when rehydrating the
> TestStream transform it must behave differently according to the coder
> used in the subsequent channel (e.g. for known coders, it decodes the
> elements and emits them directly, but for unknown coders, it prefixes
> them with their length and emits byte strings. It gets more
> complicated for nested coders, e.g. for a KV<known-coder,
> unknown-coder> the channel might be LP(KV<known-coder, unknown-coder))
> or KV<known-coder, LP(unknown-coder)) which have different encodings
> (and the latter, which is the default, requires transcoding the bytes
> to inject the length in the middle which is found by decoding the
> first component). As well as getting more complex, this really seems
> to violate the spirit of separation of concerns.

Actually, it's worse than this. For KV<unknown, unknown> we would have
to force the channel to be LP(KV<unknown, unknown>). On the other
hand, for GBK the channel must be KV<LP(unknown), LP(unknown)>.

Corollary, it's impossible to have a TestStream followed by a GBK
without injecting an SDK operation in the middle to "split" the
encoded KV into K and V parts.

> >   b) do we think, that it will not be robust enough to incorporate the
> > other use-cases (line generic transform inlining, taking into account
> > that this applies only to runners that are written in the same language
> > as the submitting SDK, because otherwise, there is nothing to inline)?
>
> Being in the same language is not a prerequisite to "inlining," e.g.
> the PubSub source on Dataflow is recognized as such and not executed
> as SDK code but natively.
>
> It is more likely that inlining occurs in the same language if there
> are UDFs involved.
>
> > I'm convinced, that the TestStream-decode expansion solution is an
> > ad-hoc solution to a generic problem, which is why I'm still bothering
> > this mailing list with my emails on this. :-)
> >
> > WDYT?
>
> While not a solution to the general problem, I think the
> TestStream-only-does-bytes simplifies its definition (primitives
> should have as simple/easy to implement definitions as possible) and
> brings it closer to the other root we have: Impulse. (We could go a
> step further and rather than emitting encoded elements, with the data
> in the proto itself, it emits sequence numbers, and a subsequent ParDo
> maps those to concrete elements (e.g. via an in-memory map), but that
> further step doesn't buy much...)
>
> Only runners that want to do inlining would have to take on the
> complexity of a fully generic solution.
>
> > On 9/3/21 7:03 PM, Robert Bradshaw wrote:
> > > On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský <je...@seznam.cz> wrote:
> > >>
> > >> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
> > >>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský <je...@seznam.cz> wrote:
> > >>>> Hi,
> > >>>>
> > >>>> I had some more time thinking about this and I'll try to recap that.
> > >>>> First some invariants:
> > >>>>
> > >>>>     a) each PCollection<T> has actually two coders - an _SDK coder_ and a
> > >>>> _runner coder_. These coders have the property, that each one can
> > >>>> _decode_ what the other encoded, but the opposite is not true, the
> > >>>> coders cannot _encode_ what the other _decoded_ (in general).
> > >>>>
> > >>>>     b) when is a PCollection<T> computed inside an environment, the
> > >>>> elements are encoded using SDK coder on the side of SDK-harness and
> > >>>> decoded using runner coder after receiving in the runner
> > >>>>
> > >>>>     c) under specific circumstances, the encode-decode step can be
> > >>>> optimized out, that is the case where the SDK coder and all its
> > >>>> subcoders are all well-known to the runner (in the present, that means
> > >>>> that all the parts present in the model coders set). The reason for that
> > >>>> is that in this specific situation runner_decode(sdk_encode(X)) = X.
> > >>>> This property is essential.
> > >>> However, in general, X can only pass from the SDK to the runner (or
> > >>> vice versa) in encoded form.
> > >> In general yes, but we are (mostly) talking transform inlining here, so
> > >> it that particular situation, the elements might be passed in decoded form.
> > >>>>     d) from b) immediately follows, that when a PTransform does not run in
> > >>>> an environment (and this might be due to the transform being runner
> > >>>> native, inlined, source (e.g. Impulse or TestStream)) the elements have
> > >>>> to be encoded by SDK coder, immediately following decode by runner
> > >>>> coder. That (surprisingly) applies even to situations when runner is
> > >>>> implemented using different language than the client SDK, because it
> > >>>> implies that the type of produced elements must be one of types encoded
> > >>>> using model coders (well-known to the runner, otherwise the SDK will not
> > >>>> be able to consume it). But - due to property c) - this means that this
> > >>>> encode-decode step can be optimized out. This does not mean that it is
> > >>>> not (logically) present, though. This is exactly the case of native
> > >>>> Impulse transform.
> > >>>>
> > >>>> Now, from that we can conclude that on the boundary between executable
> > >>>> stages, or between runner (inlined) transform and executable stage, each
> > >>>> PCollection has to be encoded using SDK coder and immediately decoded by
> > >>>> runner coder, *unless this can be optimized out* by property c).
> > >>>>
> > >>>> This gives us two options where to implement this encode/decode step:
> > >>>>
> > >>>>     1) completely inside runner with the possibility to optimize the
> > >>>> encode/decode step by identity under right circumstances
> > >>>>
> > >>>>     2) partly in the runner and partly in the SDK - that is we encode
> > >>>> elements of PCollection using SDK coder into bytes, pass those to the
> > >>>> SDK harness and apply a custom decode step there. This works because SDK
> > >>>> coder encoded elements are in byte[], and that is well-known coder type.
> > >>>> We again only leverage property c) and optimize the SDK coder encode,
> > >>>> runner decode step out.
> > >>>>
> > >>>> The option 2) is exactly the proposal of TestStream producing byte[] and
> > >>>> decoding inside SDK-harness, the TestStream is actually inlined
> > >>>> transform, the elements are produced directly in runner (the SDK coder
> > >>>> is not known to the runner, but that does not matter, because the
> > >>>> elements are already encoded by client).
> > >>>>
> > >>>>    From the above it seems to me, that option 1) should be preferred, because:
> > >>>>
> > >>>>     i) it is generic, applicable to all inlined transforms, any sources
> > >>>>
> > >>>>     ii) it is consistent with how things logically work underneath
> > >>>>
> > >>>>     iii) it offers better room for optimization - option 2) might result
> > >>>> in cases when the elements are passed from the runner to the SDK-harness
> > >>>> only for the sake of the decoding from SDK coder and immediately
> > >>>> encoding back using SDK-coder and returned back to the runner. This
> > >>>> would be the case when TestStream would be directly consumed by inlined
> > >>>> (or external) transform.
> > >>> (1) is not possible if the Coder in question is not known to the
> > >>> Runner, which is why I proposed (2).
> > >> There is no particular need for the coder to be known. If transform is
> > >> to be inlined, what *has* to be known is the SDK-encoded form of the
> > >> elements. That holds true if:
> > >>
> > >>    a) either the SDK coder is known, or
> > >>
> > >>    b) encoded form of the produced elements is known in advance
> > >>
> > >> For TestStream it is the case b). For inlined primitive Read (or any
> > >> other transform which executes code) it is a).
> > > There's another hitch here for TestStream. For historical reasons,
> > > coders actually represent two encodings: nested (aka self delimiting)
> > > and unnested. TestStream elements are given as unnested encoded bytes,
> > > but the nested encoding is required for sending data to the SDK. The
> > > runner can't go from <nested encoding> to <unnested encoding> for an
> > > arbitrary unknown coder.
> > >
> > > (Even if it weren't for this complication, to be able to send already
> > > encoded bytes of an unknown coder to the SDK will also complicate the
> > > logic in choosing the coder to be used for the channel and sending the
> > > data, which is some of what you're running into (but can be solved
> > > differently for inlined reads as the coder can always be known by the
> > > runner).)

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
On Sat, Sep 4, 2021 at 6:52 AM Jan Lukavský <je...@seznam.cz> wrote:
>
> On 9/3/21 9:50 PM, Robert Bradshaw wrote:
>
> > On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>  wrote:
> >> Hi Robert,
> >>
> >>> There's another hitch here for TestStream. For historical reasons,
> >>> coders actually represent two encodings: nested (aka self delimiting)
> >>> and unnested. TestStream elements are given as unnested encoded bytes,
> >>> but the nested encoding is required for sending data to the SDK. The
> >>> runner can't go from <nested encoding> to <unnested encoding> for an
> >>> arbitrary unknown coder.
> >>>
> >>> (Even if it weren't for this complication, to be able to send already
> >>> encoded bytes of an unknown coder to the SDK will also complicate the
> >>> logic in choosing the coder to be used for the channel and sending the
> >>> data, which is some of what you're running into (but can be solved
> >>> differently for inlined reads as the coder can always be known by the
> >>> runner).)
> >> It is hard for me to argue with "historical reasons". But - the "nested"
> >> and "unnested" coders look very similar to SDK-coder and runner-coder
> >> spaces.
> > Unfortunately, they're actually orthogonal to that.
>
> Hm, do you mean the Context passed to the encode/decode method? [1] That
> seems to be deprecated, I assume that most coders use the default
> implementation and simply ignore the Context?

Unfortunately the most basic coders (e.g. bytes, string, kv, iterable)
care about Context because they predated this deprecation, and
changing coders is hard (due to no way to update the encoding for a
streaming pipeline).

> Even if not - whether or
> not the elements are encoded using NESTED Context or UNNESTED Context
> should be part of the contract of TestStream, right? Most likely it is
> the UNNESTED one, if I understand correctly what that does. Under what
> conditions is the deprecated encode/decode method used?

Yes, it's the UNNESTED one.

> [1]
> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134
>
> >> The runner's responsibility is not to go from "<nested
> >> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary coder.
> >> That is really impossible. But a coder is a function, right? Function
> >> maps from universe A to universe B (in general). TestStream provides a
> >> set of elements, and these elements are the "universe". For those
> >> elements it also provides the encoded form, which can be interpreted as
> >> the definition of the coder.
> > The problem here is that there is not "the encoded form" for a Coder
> > but two encoded forms, and we have the wrong one. Things could be made
> > to work if we had the other.
> Which two encoded forms do you refer to? Elements encoded by both the
> SDK-coder and runner-coder (and I ignore the Context here once again)
> have the same binary representation (which they must have, otherwise it
> would be impossible to decode elements coming from the runner to the
> SDK-harness or vice-versa).
> >> Therefore - technically (and formally) -
> >> the SDK coder for the TestStream is known to the runner, regardless of
> >> the language the runner is written in.
> >>
> >> To move  this discussion forward, I think we should look for answers to
> >> the following questions:
> >>
> >>    a) do we have any clues that show, that the proposed "in runner"
> >> solution will not work?
> > OK, thinking about it some more, in the TestStream, we can use the
> > happy coincidence that
> >
> >      LengthPrefixed(C).encode(x, nested=True) ==
> > VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
> > nested=False)
> >
> > (where || denotes concatenation) and the fact that we have
> >
> >      C.encode(x, nested=False)
> >
> > in hand.
> >
> > A possible fix here for the OP's question is that when rehydrating the
> > TestStream transform it must behave differently according to the coder
> > used in the subsequent channel (e.g. for known coders, it decodes the
> > elements and emits them directly, but for unknown coders, it prefixes
> > them with their length and emits byte strings. It gets more
> > complicated for nested coders, e.g. for a KV<known-coder,
> > unknown-coder> the channel might be LP(KV<known-coder, unknown-coder))
> > or KV<known-coder, LP(unknown-coder)) which have different encodings
> > (and the latter, which is the default, requires transcoding the bytes
> > to inject the length in the middle which is found by decoding the
> > first component). As well as getting more complex, this really seems
> > to violate the spirit of separation of concerns.
>
> How do we make the decision if the channel is LP<KV<..>> or
> KV<LP<unknown>, known>? From my understanding it is always the latter,
> because of [2].
>
> [2]
> https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48

It is currently the latter for runners using this code (which not all
do, e.g. the ULR and Dataflow runners). I don't think we want to
ossify this decision as part of the spec. (Note that even what's
"known" and "unknown" can change from runner to runner.)

> >>    b) do we think, that it will not be robust enough to incorporate the
> >> other use-cases (line generic transform inlining, taking into account
> >> that this applies only to runners that are written in the same language
> >> as the submitting SDK, because otherwise, there is nothing to inline)?
> > Being in the same language is not a prerequisite to "inlining," e.g.
> > the PubSub source on Dataflow is recognized as such and not executed
> > as SDK code but natively.
> Agree, that is actually exactly what happens with the TestStream. The
> transform need not be in the same language, as long as it is completely
> understood by the runner, including the SDK-coder (either explicitly -
> which might be due to the PCollection coder being composed of well-known
> coders only, or implicitly like in the case of TestStream, where the
> elements are encoded using the SDK coder.
> > It is more likely that inlining occurs in the same language if there
> > are UDFs involved.
> >
> >> I'm convinced, that the TestStream-decode expansion solution is an
> >> ad-hoc solution to a generic problem, which is why I'm still bothering
> >> this mailing list with my emails on this. :-)
> >>
> >> WDYT?
> > While not a solution to the general problem, I think the
> > TestStream-only-does-bytes simplifies its definition (primitives
> > should have as simple/easy to implement definitions as possible) and
> > brings it closer to the other root we have: Impulse. (We could go a
> > step further and rather than emitting encoded elements, with the data
> > in the proto itself, it emits sequence numbers, and a subsequent ParDo
> > maps those to concrete elements (e.g. via an in-memory map), but that
> > further step doesn't buy much...)
> >
> > Only runners that want to do inlining would have to take on the
> > complexity of a fully generic solution.
>
> I think that if the simplification brings something, we can do that, but
> I'd like to understand why we cannot (or should not) use the generic
> solution. I think it definitely *should* be possible to use a generic
> solution, because otherwise the solution would not be generic. And it
> would imply, that we are unable to do generic transform inlining, which
> I would find really strange. That would immediately mean, that we are
> unable to construct classical runner as a special case of the portable
> one, which would be bad I think.
>
> The elements in the TestStreamPayload are encoded with pure SDK-coder,
> or does this go through the LengthPrefixUnknownCoders logic? If not,
> then the problem would be there, because that means, that the SDK-coder
> cannot be (implicitly) defined in the runner. If the elements would be
> encoded using LP, then it would be possible to decode them using
> runner-coder and the problem should be solved, or am I still missing
> some key parts?

Yes, the problem is precisely that there are (unspecified) constraints
on the coder used by the TestStreamPayload. Just requiring that it be
length prefixed is not enough, you have to make constraints on
sometimes pushing down the length prefixing if it's a composite (like
a KV) that depend on what the runner is expected to support in terms
of composites and/or the choices it chooses for the channel (and the
runner, not knowing the coder, can't transcode between these choices).

The simpler solution is to constrain this coder to just be byte[]
rather than let it be a little bit flexible (but not wholly flexible).

As for a fully generic solution, I think the issue encountered with
inlining Read vs. TestStream are related to this, but not really the
same. With TestStream one has an encoded representation of the
elements provided by the SDK that the Runner and has no SDK
representation/execution whereas with the Reads one has unencoded
elements in hand and a Coder that is understood by both (so long as
the channel can be negotiated correctly). FWIW, I think the proper
solution to inlining a Read (or other Transform that would typically
be executed in the SDK) is to treat it as a special environment (where
we know logically it can work) and then elide, as possible, the
various encodings, grpc calls, etc. that are unneeded as everything is
in process.

> >> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
> >>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>  wrote:
> >>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
> >>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>  wrote:
> >>>>>> Hi,
> >>>>>>
> >>>>>> I had some more time thinking about this and I'll try to recap that.
> >>>>>> First some invariants:
> >>>>>>
> >>>>>>      a) each PCollection<T> has actually two coders - an _SDK coder_ and a
> >>>>>> _runner coder_. These coders have the property, that each one can
> >>>>>> _decode_ what the other encoded, but the opposite is not true, the
> >>>>>> coders cannot _encode_ what the other _decoded_ (in general).
> >>>>>>
> >>>>>>      b) when is a PCollection<T> computed inside an environment, the
> >>>>>> elements are encoded using SDK coder on the side of SDK-harness and
> >>>>>> decoded using runner coder after receiving in the runner
> >>>>>>
> >>>>>>      c) under specific circumstances, the encode-decode step can be
> >>>>>> optimized out, that is the case where the SDK coder and all its
> >>>>>> subcoders are all well-known to the runner (in the present, that means
> >>>>>> that all the parts present in the model coders set). The reason for that
> >>>>>> is that in this specific situation runner_decode(sdk_encode(X)) = X.
> >>>>>> This property is essential.
> >>>>> However, in general, X can only pass from the SDK to the runner (or
> >>>>> vice versa) in encoded form.
> >>>> In general yes, but we are (mostly) talking transform inlining here, so
> >>>> it that particular situation, the elements might be passed in decoded form.
> >>>>>>      d) from b) immediately follows, that when a PTransform does not run in
> >>>>>> an environment (and this might be due to the transform being runner
> >>>>>> native, inlined, source (e.g. Impulse or TestStream)) the elements have
> >>>>>> to be encoded by SDK coder, immediately following decode by runner
> >>>>>> coder. That (surprisingly) applies even to situations when runner is
> >>>>>> implemented using different language than the client SDK, because it
> >>>>>> implies that the type of produced elements must be one of types encoded
> >>>>>> using model coders (well-known to the runner, otherwise the SDK will not
> >>>>>> be able to consume it). But - due to property c) - this means that this
> >>>>>> encode-decode step can be optimized out. This does not mean that it is
> >>>>>> not (logically) present, though. This is exactly the case of native
> >>>>>> Impulse transform.
> >>>>>>
> >>>>>> Now, from that we can conclude that on the boundary between executable
> >>>>>> stages, or between runner (inlined) transform and executable stage, each
> >>>>>> PCollection has to be encoded using SDK coder and immediately decoded by
> >>>>>> runner coder, *unless this can be optimized out* by property c).
> >>>>>>
> >>>>>> This gives us two options where to implement this encode/decode step:
> >>>>>>
> >>>>>>      1) completely inside runner with the possibility to optimize the
> >>>>>> encode/decode step by identity under right circumstances
> >>>>>>
> >>>>>>      2) partly in the runner and partly in the SDK - that is we encode
> >>>>>> elements of PCollection using SDK coder into bytes, pass those to the
> >>>>>> SDK harness and apply a custom decode step there. This works because SDK
> >>>>>> coder encoded elements are in byte[], and that is well-known coder type.
> >>>>>> We again only leverage property c) and optimize the SDK coder encode,
> >>>>>> runner decode step out.
> >>>>>>
> >>>>>> The option 2) is exactly the proposal of TestStream producing byte[] and
> >>>>>> decoding inside SDK-harness, the TestStream is actually inlined
> >>>>>> transform, the elements are produced directly in runner (the SDK coder
> >>>>>> is not known to the runner, but that does not matter, because the
> >>>>>> elements are already encoded by client).
> >>>>>>
> >>>>>>     From the above it seems to me, that option 1) should be preferred, because:
> >>>>>>
> >>>>>>      i) it is generic, applicable to all inlined transforms, any sources
> >>>>>>
> >>>>>>      ii) it is consistent with how things logically work underneath
> >>>>>>
> >>>>>>      iii) it offers better room for optimization - option 2) might result
> >>>>>> in cases when the elements are passed from the runner to the SDK-harness
> >>>>>> only for the sake of the decoding from SDK coder and immediately
> >>>>>> encoding back using SDK-coder and returned back to the runner. This
> >>>>>> would be the case when TestStream would be directly consumed by inlined
> >>>>>> (or external) transform.
> >>>>> (1) is not possible if the Coder in question is not known to the
> >>>>> Runner, which is why I proposed (2).
> >>>> There is no particular need for the coder to be known. If transform is
> >>>> to be inlined, what *has* to be known is the SDK-encoded form of the
> >>>> elements. That holds true if:
> >>>>
> >>>>     a) either the SDK coder is known, or
> >>>>
> >>>>     b) encoded form of the produced elements is known in advance
> >>>>
> >>>> For TestStream it is the case b). For inlined primitive Read (or any
> >>>> other transform which executes code) it is a).
> >>> There's another hitch here for TestStream. For historical reasons,
> >>> coders actually represent two encodings: nested (aka self delimiting)
> >>> and unnested. TestStream elements are given as unnested encoded bytes,
> >>> but the nested encoding is required for sending data to the SDK. The
> >>> runner can't go from <nested encoding> to <unnested encoding> for an
> >>> arbitrary unknown coder.
> >>>
> >>> (Even if it weren't for this complication, to be able to send already
> >>> encoded bytes of an unknown coder to the SDK will also complicate the
> >>> logic in choosing the coder to be used for the channel and sending the
> >>> data, which is some of what you're running into (but can be solved
> >>> differently for inlined reads as the coder can always be known by the
> >>> runner).)

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
On 9/3/21 9:50 PM, Robert Bradshaw wrote:

> On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský<je...@seznam.cz>  wrote:
>> Hi Robert,
>>
>>> There's another hitch here for TestStream. For historical reasons,
>>> coders actually represent two encodings: nested (aka self delimiting)
>>> and unnested. TestStream elements are given as unnested encoded bytes,
>>> but the nested encoding is required for sending data to the SDK. The
>>> runner can't go from <nested encoding> to <unnested encoding> for an
>>> arbitrary unknown coder.
>>>
>>> (Even if it weren't for this complication, to be able to send already
>>> encoded bytes of an unknown coder to the SDK will also complicate the
>>> logic in choosing the coder to be used for the channel and sending the
>>> data, which is some of what you're running into (but can be solved
>>> differently for inlined reads as the coder can always be known by the
>>> runner).)
>> It is hard for me to argue with "historical reasons". But - the "nested"
>> and "unnested" coders look very similar to SDK-coder and runner-coder
>> spaces.
> Unfortunately, they're actually orthogonal to that.

Hm, do you mean the Context passed to the encode/decode method? [1] That 
seems to be deprecated, I assume that most coders use the default 
implementation and simply ignore the Context? Even if not - whether or 
not the elements are encoded using NESTED Context or UNNESTED Context 
should be part of the contract of TestStream, right? Most likely it is 
the UNNESTED one, if I understand correctly what that does. Under what 
conditions is the deprecated encode/decode method used?

[1] 
https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java#L134

>> The runner's responsibility is not to go from "<nested
>> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary coder.
>> That is really impossible. But a coder is a function, right? Function
>> maps from universe A to universe B (in general). TestStream provides a
>> set of elements, and these elements are the "universe". For those
>> elements it also provides the encoded form, which can be interpreted as
>> the definition of the coder.
> The problem here is that there is not "the encoded form" for a Coder
> but two encoded forms, and we have the wrong one. Things could be made
> to work if we had the other.
Which two encoded forms do you refer to? Elements encoded by both the 
SDK-coder and runner-coder (and I ignore the Context here once again) 
have the same binary representation (which they must have, otherwise it 
would be impossible to decode elements coming from the runner to the 
SDK-harness or vice-versa).
>> Therefore - technically (and formally) -
>> the SDK coder for the TestStream is known to the runner, regardless of
>> the language the runner is written in.
>>
>> To move  this discussion forward, I think we should look for answers to
>> the following questions:
>>
>>    a) do we have any clues that show, that the proposed "in runner"
>> solution will not work?
> OK, thinking about it some more, in the TestStream, we can use the
> happy coincidence that
>
>      LengthPrefixed(C).encode(x, nested=True) ==
> VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
> nested=False)
>
> (where || denotes concatenation) and the fact that we have
>
>      C.encode(x, nested=False)
>
> in hand.
>
> A possible fix here for the OP's question is that when rehydrating the
> TestStream transform it must behave differently according to the coder
> used in the subsequent channel (e.g. for known coders, it decodes the
> elements and emits them directly, but for unknown coders, it prefixes
> them with their length and emits byte strings. It gets more
> complicated for nested coders, e.g. for a KV<known-coder,
> unknown-coder> the channel might be LP(KV<known-coder, unknown-coder))
> or KV<known-coder, LP(unknown-coder)) which have different encodings
> (and the latter, which is the default, requires transcoding the bytes
> to inject the length in the middle which is found by decoding the
> first component). As well as getting more complex, this really seems
> to violate the spirit of separation of concerns.

How do we make the decision if the channel is LP<KV<..>> or 
KV<LP<unknown>, known>? From my understanding it is always the latter, 
because of [2].

[2] 
https://github.com/apache/beam/blob/c4e0b4ac0777f37f5eb775a8a83c56f66b3baac3/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48

>>    b) do we think, that it will not be robust enough to incorporate the
>> other use-cases (line generic transform inlining, taking into account
>> that this applies only to runners that are written in the same language
>> as the submitting SDK, because otherwise, there is nothing to inline)?
> Being in the same language is not a prerequisite to "inlining," e.g.
> the PubSub source on Dataflow is recognized as such and not executed
> as SDK code but natively.
Agree, that is actually exactly what happens with the TestStream. The 
transform need not be in the same language, as long as it is completely 
understood by the runner, including the SDK-coder (either explicitly - 
which might be due to the PCollection coder being composed of well-known 
coders only, or implicitly like in the case of TestStream, where the 
elements are encoded using the SDK coder.
> It is more likely that inlining occurs in the same language if there
> are UDFs involved.
>
>> I'm convinced, that the TestStream-decode expansion solution is an
>> ad-hoc solution to a generic problem, which is why I'm still bothering
>> this mailing list with my emails on this. :-)
>>
>> WDYT?
> While not a solution to the general problem, I think the
> TestStream-only-does-bytes simplifies its definition (primitives
> should have as simple/easy to implement definitions as possible) and
> brings it closer to the other root we have: Impulse. (We could go a
> step further and rather than emitting encoded elements, with the data
> in the proto itself, it emits sequence numbers, and a subsequent ParDo
> maps those to concrete elements (e.g. via an in-memory map), but that
> further step doesn't buy much...)
>
> Only runners that want to do inlining would have to take on the
> complexity of a fully generic solution.

I think that if the simplification brings something, we can do that, but 
I'd like to understand why we cannot (or should not) use the generic 
solution. I think it definitely *should* be possible to use a generic 
solution, because otherwise the solution would not be generic. And it 
would imply, that we are unable to do generic transform inlining, which 
I would find really strange. That would immediately mean, that we are 
unable to construct classical runner as a special case of the portable 
one, which would be bad I think.

The elements in the TestStreamPayload are encoded with pure SDK-coder, 
or does this go through the LengthPrefixUnknownCoders logic? If not, 
then the problem would be there, because that means, that the SDK-coder 
cannot be (implicitly) defined in the runner. If the elements would be 
encoded using LP, then it would be possible to decode them using 
runner-coder and the problem should be solved, or am I still missing 
some key parts?

>
>> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
>>> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský<je...@seznam.cz>  wrote:
>>>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský<je...@seznam.cz>  wrote:
>>>>>> Hi,
>>>>>>
>>>>>> I had some more time thinking about this and I'll try to recap that.
>>>>>> First some invariants:
>>>>>>
>>>>>>      a) each PCollection<T> has actually two coders - an _SDK coder_ and a
>>>>>> _runner coder_. These coders have the property, that each one can
>>>>>> _decode_ what the other encoded, but the opposite is not true, the
>>>>>> coders cannot _encode_ what the other _decoded_ (in general).
>>>>>>
>>>>>>      b) when is a PCollection<T> computed inside an environment, the
>>>>>> elements are encoded using SDK coder on the side of SDK-harness and
>>>>>> decoded using runner coder after receiving in the runner
>>>>>>
>>>>>>      c) under specific circumstances, the encode-decode step can be
>>>>>> optimized out, that is the case where the SDK coder and all its
>>>>>> subcoders are all well-known to the runner (in the present, that means
>>>>>> that all the parts present in the model coders set). The reason for that
>>>>>> is that in this specific situation runner_decode(sdk_encode(X)) = X.
>>>>>> This property is essential.
>>>>> However, in general, X can only pass from the SDK to the runner (or
>>>>> vice versa) in encoded form.
>>>> In general yes, but we are (mostly) talking transform inlining here, so
>>>> it that particular situation, the elements might be passed in decoded form.
>>>>>>      d) from b) immediately follows, that when a PTransform does not run in
>>>>>> an environment (and this might be due to the transform being runner
>>>>>> native, inlined, source (e.g. Impulse or TestStream)) the elements have
>>>>>> to be encoded by SDK coder, immediately following decode by runner
>>>>>> coder. That (surprisingly) applies even to situations when runner is
>>>>>> implemented using different language than the client SDK, because it
>>>>>> implies that the type of produced elements must be one of types encoded
>>>>>> using model coders (well-known to the runner, otherwise the SDK will not
>>>>>> be able to consume it). But - due to property c) - this means that this
>>>>>> encode-decode step can be optimized out. This does not mean that it is
>>>>>> not (logically) present, though. This is exactly the case of native
>>>>>> Impulse transform.
>>>>>>
>>>>>> Now, from that we can conclude that on the boundary between executable
>>>>>> stages, or between runner (inlined) transform and executable stage, each
>>>>>> PCollection has to be encoded using SDK coder and immediately decoded by
>>>>>> runner coder, *unless this can be optimized out* by property c).
>>>>>>
>>>>>> This gives us two options where to implement this encode/decode step:
>>>>>>
>>>>>>      1) completely inside runner with the possibility to optimize the
>>>>>> encode/decode step by identity under right circumstances
>>>>>>
>>>>>>      2) partly in the runner and partly in the SDK - that is we encode
>>>>>> elements of PCollection using SDK coder into bytes, pass those to the
>>>>>> SDK harness and apply a custom decode step there. This works because SDK
>>>>>> coder encoded elements are in byte[], and that is well-known coder type.
>>>>>> We again only leverage property c) and optimize the SDK coder encode,
>>>>>> runner decode step out.
>>>>>>
>>>>>> The option 2) is exactly the proposal of TestStream producing byte[] and
>>>>>> decoding inside SDK-harness, the TestStream is actually inlined
>>>>>> transform, the elements are produced directly in runner (the SDK coder
>>>>>> is not known to the runner, but that does not matter, because the
>>>>>> elements are already encoded by client).
>>>>>>
>>>>>>     From the above it seems to me, that option 1) should be preferred, because:
>>>>>>
>>>>>>      i) it is generic, applicable to all inlined transforms, any sources
>>>>>>
>>>>>>      ii) it is consistent with how things logically work underneath
>>>>>>
>>>>>>      iii) it offers better room for optimization - option 2) might result
>>>>>> in cases when the elements are passed from the runner to the SDK-harness
>>>>>> only for the sake of the decoding from SDK coder and immediately
>>>>>> encoding back using SDK-coder and returned back to the runner. This
>>>>>> would be the case when TestStream would be directly consumed by inlined
>>>>>> (or external) transform.
>>>>> (1) is not possible if the Coder in question is not known to the
>>>>> Runner, which is why I proposed (2).
>>>> There is no particular need for the coder to be known. If transform is
>>>> to be inlined, what *has* to be known is the SDK-encoded form of the
>>>> elements. That holds true if:
>>>>
>>>>     a) either the SDK coder is known, or
>>>>
>>>>     b) encoded form of the produced elements is known in advance
>>>>
>>>> For TestStream it is the case b). For inlined primitive Read (or any
>>>> other transform which executes code) it is a).
>>> There's another hitch here for TestStream. For historical reasons,
>>> coders actually represent two encodings: nested (aka self delimiting)
>>> and unnested. TestStream elements are given as unnested encoded bytes,
>>> but the nested encoding is required for sending data to the SDK. The
>>> runner can't go from <nested encoding> to <unnested encoding> for an
>>> arbitrary unknown coder.
>>>
>>> (Even if it weren't for this complication, to be able to send already
>>> encoded bytes of an unknown coder to the SDK will also complicate the
>>> logic in choosing the coder to be used for the channel and sending the
>>> data, which is some of what you're running into (but can be solved
>>> differently for inlined reads as the coder can always be known by the
>>> runner).)

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
On Fri, Sep 3, 2021 at 11:42 AM Jan Lukavský <je...@seznam.cz> wrote:
>
> Hi Robert,
>
> > There's another hitch here for TestStream. For historical reasons,
> > coders actually represent two encodings: nested (aka self delimiting)
> > and unnested. TestStream elements are given as unnested encoded bytes,
> > but the nested encoding is required for sending data to the SDK. The
> > runner can't go from <nested encoding> to <unnested encoding> for an
> > arbitrary unknown coder.
> >
> > (Even if it weren't for this complication, to be able to send already
> > encoded bytes of an unknown coder to the SDK will also complicate the
> > logic in choosing the coder to be used for the channel and sending the
> > data, which is some of what you're running into (but can be solved
> > differently for inlined reads as the coder can always be known by the
> > runner).)
> It is hard for me to argue with "historical reasons". But - the "nested"
> and "unnested" coders look very similar to SDK-coder and runner-coder
> spaces.

Unfortunately, they're actually orthogonal to that.

> The runner's responsibility is not to go from "<nested
> encoding>" (SDK coder) to "<unnested encoding>" for arbitrary coder.
> That is really impossible. But a coder is a function, right? Function
> maps from universe A to universe B (in general). TestStream provides a
> set of elements, and these elements are the "universe". For those
> elements it also provides the encoded form, which can be interpreted as
> the definition of the coder.

The problem here is that there is not "the encoded form" for a Coder
but two encoded forms, and we have the wrong one. Things could be made
to work if we had the other.

> Therefore - technically (and formally) -
> the SDK coder for the TestStream is known to the runner, regardless of
> the language the runner is written in.
>
> To move  this discussion forward, I think we should look for answers to
> the following questions:
>
>   a) do we have any clues that show, that the proposed "in runner"
> solution will not work?

OK, thinking about it some more, in the TestStream, we can use the
happy coincidence that

    LengthPrefixed(C).encode(x, nested=True) ==
VarLong.encode(len(C.encode(x, nested=False))) || C.encode(x,
nested=False)

(where || denotes concatenation) and the fact that we have

    C.encode(x, nested=False)

in hand.

A possible fix here for the OP's question is that when rehydrating the
TestStream transform it must behave differently according to the coder
used in the subsequent channel (e.g. for known coders, it decodes the
elements and emits them directly, but for unknown coders, it prefixes
them with their length and emits byte strings. It gets more
complicated for nested coders, e.g. for a KV<known-coder,
unknown-coder> the channel might be LP(KV<known-coder, unknown-coder))
or KV<known-coder, LP(unknown-coder)) which have different encodings
(and the latter, which is the default, requires transcoding the bytes
to inject the length in the middle which is found by decoding the
first component). As well as getting more complex, this really seems
to violate the spirit of separation of concerns.

>   b) do we think, that it will not be robust enough to incorporate the
> other use-cases (line generic transform inlining, taking into account
> that this applies only to runners that are written in the same language
> as the submitting SDK, because otherwise, there is nothing to inline)?

Being in the same language is not a prerequisite to "inlining," e.g.
the PubSub source on Dataflow is recognized as such and not executed
as SDK code but natively.

It is more likely that inlining occurs in the same language if there
are UDFs involved.

> I'm convinced, that the TestStream-decode expansion solution is an
> ad-hoc solution to a generic problem, which is why I'm still bothering
> this mailing list with my emails on this. :-)
>
> WDYT?

While not a solution to the general problem, I think the
TestStream-only-does-bytes simplifies its definition (primitives
should have as simple/easy to implement definitions as possible) and
brings it closer to the other root we have: Impulse. (We could go a
step further and rather than emitting encoded elements, with the data
in the proto itself, it emits sequence numbers, and a subsequent ParDo
maps those to concrete elements (e.g. via an in-memory map), but that
further step doesn't buy much...)

Only runners that want to do inlining would have to take on the
complexity of a fully generic solution.

> On 9/3/21 7:03 PM, Robert Bradshaw wrote:
> > On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský <je...@seznam.cz> wrote:
> >>
> >> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
> >>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský <je...@seznam.cz> wrote:
> >>>> Hi,
> >>>>
> >>>> I had some more time thinking about this and I'll try to recap that.
> >>>> First some invariants:
> >>>>
> >>>>     a) each PCollection<T> has actually two coders - an _SDK coder_ and a
> >>>> _runner coder_. These coders have the property, that each one can
> >>>> _decode_ what the other encoded, but the opposite is not true, the
> >>>> coders cannot _encode_ what the other _decoded_ (in general).
> >>>>
> >>>>     b) when is a PCollection<T> computed inside an environment, the
> >>>> elements are encoded using SDK coder on the side of SDK-harness and
> >>>> decoded using runner coder after receiving in the runner
> >>>>
> >>>>     c) under specific circumstances, the encode-decode step can be
> >>>> optimized out, that is the case where the SDK coder and all its
> >>>> subcoders are all well-known to the runner (in the present, that means
> >>>> that all the parts present in the model coders set). The reason for that
> >>>> is that in this specific situation runner_decode(sdk_encode(X)) = X.
> >>>> This property is essential.
> >>> However, in general, X can only pass from the SDK to the runner (or
> >>> vice versa) in encoded form.
> >> In general yes, but we are (mostly) talking transform inlining here, so
> >> it that particular situation, the elements might be passed in decoded form.
> >>>>     d) from b) immediately follows, that when a PTransform does not run in
> >>>> an environment (and this might be due to the transform being runner
> >>>> native, inlined, source (e.g. Impulse or TestStream)) the elements have
> >>>> to be encoded by SDK coder, immediately following decode by runner
> >>>> coder. That (surprisingly) applies even to situations when runner is
> >>>> implemented using different language than the client SDK, because it
> >>>> implies that the type of produced elements must be one of types encoded
> >>>> using model coders (well-known to the runner, otherwise the SDK will not
> >>>> be able to consume it). But - due to property c) - this means that this
> >>>> encode-decode step can be optimized out. This does not mean that it is
> >>>> not (logically) present, though. This is exactly the case of native
> >>>> Impulse transform.
> >>>>
> >>>> Now, from that we can conclude that on the boundary between executable
> >>>> stages, or between runner (inlined) transform and executable stage, each
> >>>> PCollection has to be encoded using SDK coder and immediately decoded by
> >>>> runner coder, *unless this can be optimized out* by property c).
> >>>>
> >>>> This gives us two options where to implement this encode/decode step:
> >>>>
> >>>>     1) completely inside runner with the possibility to optimize the
> >>>> encode/decode step by identity under right circumstances
> >>>>
> >>>>     2) partly in the runner and partly in the SDK - that is we encode
> >>>> elements of PCollection using SDK coder into bytes, pass those to the
> >>>> SDK harness and apply a custom decode step there. This works because SDK
> >>>> coder encoded elements are in byte[], and that is well-known coder type.
> >>>> We again only leverage property c) and optimize the SDK coder encode,
> >>>> runner decode step out.
> >>>>
> >>>> The option 2) is exactly the proposal of TestStream producing byte[] and
> >>>> decoding inside SDK-harness, the TestStream is actually inlined
> >>>> transform, the elements are produced directly in runner (the SDK coder
> >>>> is not known to the runner, but that does not matter, because the
> >>>> elements are already encoded by client).
> >>>>
> >>>>    From the above it seems to me, that option 1) should be preferred, because:
> >>>>
> >>>>     i) it is generic, applicable to all inlined transforms, any sources
> >>>>
> >>>>     ii) it is consistent with how things logically work underneath
> >>>>
> >>>>     iii) it offers better room for optimization - option 2) might result
> >>>> in cases when the elements are passed from the runner to the SDK-harness
> >>>> only for the sake of the decoding from SDK coder and immediately
> >>>> encoding back using SDK-coder and returned back to the runner. This
> >>>> would be the case when TestStream would be directly consumed by inlined
> >>>> (or external) transform.
> >>> (1) is not possible if the Coder in question is not known to the
> >>> Runner, which is why I proposed (2).
> >> There is no particular need for the coder to be known. If transform is
> >> to be inlined, what *has* to be known is the SDK-encoded form of the
> >> elements. That holds true if:
> >>
> >>    a) either the SDK coder is known, or
> >>
> >>    b) encoded form of the produced elements is known in advance
> >>
> >> For TestStream it is the case b). For inlined primitive Read (or any
> >> other transform which executes code) it is a).
> > There's another hitch here for TestStream. For historical reasons,
> > coders actually represent two encodings: nested (aka self delimiting)
> > and unnested. TestStream elements are given as unnested encoded bytes,
> > but the nested encoding is required for sending data to the SDK. The
> > runner can't go from <nested encoding> to <unnested encoding> for an
> > arbitrary unknown coder.
> >
> > (Even if it weren't for this complication, to be able to send already
> > encoded bytes of an unknown coder to the SDK will also complicate the
> > logic in choosing the coder to be used for the channel and sending the
> > data, which is some of what you're running into (but can be solved
> > differently for inlined reads as the coder can always be known by the
> > runner).)

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
Hi Robert,

> There's another hitch here for TestStream. For historical reasons,
> coders actually represent two encodings: nested (aka self delimiting)
> and unnested. TestStream elements are given as unnested encoded bytes,
> but the nested encoding is required for sending data to the SDK. The
> runner can't go from <nested encoding> to <unnested encoding> for an
> arbitrary unknown coder.
>
> (Even if it weren't for this complication, to be able to send already
> encoded bytes of an unknown coder to the SDK will also complicate the
> logic in choosing the coder to be used for the channel and sending the
> data, which is some of what you're running into (but can be solved
> differently for inlined reads as the coder can always be known by the
> runner).)
It is hard for me to argue with "historical reasons". But - the "nested" 
and "unnested" coders look very similar to SDK-coder and runner-coder 
spaces. The runner's responsibility is not to go from "<nested 
encoding>" (SDK coder) to "<unnested encoding>" for arbitrary coder. 
That is really impossible. But a coder is a function, right? Function 
maps from universe A to universe B (in general). TestStream provides a 
set of elements, and these elements are the "universe". For those 
elements it also provides the encoded form, which can be interpreted as 
the definition of the coder. Therefore - technically (and formally) - 
the SDK coder for the TestStream is known to the runner, regardless of 
the language the runner is written in.

To move  this discussion forward, I think we should look for answers to 
the following questions:

  a) do we have any clues that show, that the proposed "in runner" 
solution will not work?

  b) do we think, that it will not be robust enough to incorporate the 
other use-cases (line generic transform inlining, taking into accoung 
that this applies only to runners that are written in the same language 
as the submitting SDK, because otherwise, there is nothing to inline)?

I'm convinced, that the TestStream-decode expansion solution is an 
ad-hoc solution to a generic problem, which is why I'm still bothering 
this mailing list with my emails on this. :-)

WDYT?

  Jan

On 9/3/21 7:03 PM, Robert Bradshaw wrote:
> On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský <je...@seznam.cz> wrote:
>>
>> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
>>> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>> Hi,
>>>>
>>>> I had some more time thinking about this and I'll try to recap that.
>>>> First some invariants:
>>>>
>>>>     a) each PCollection<T> has actually two coders - an _SDK coder_ and a
>>>> _runner coder_. These coders have the property, that each one can
>>>> _decode_ what the other encoded, but the opposite is not true, the
>>>> coders cannot _encode_ what the other _decoded_ (in general).
>>>>
>>>>     b) when is a PCollection<T> computed inside an environment, the
>>>> elements are encoded using SDK coder on the side of SDK-harness and
>>>> decoded using runner coder after receiving in the runner
>>>>
>>>>     c) under specific circumstances, the encode-decode step can be
>>>> optimized out, that is the case where the SDK coder and all its
>>>> subcoders are all well-known to the runner (in the present, that means
>>>> that all the parts present in the model coders set). The reason for that
>>>> is that in this specific situation runner_decode(sdk_encode(X)) = X.
>>>> This property is essential.
>>> However, in general, X can only pass from the SDK to the runner (or
>>> vice versa) in encoded form.
>> In general yes, but we are (mostly) talking transform inlining here, so
>> it that particular situation, the elements might be passed in decoded form.
>>>>     d) from b) immediately follows, that when a PTransform does not run in
>>>> an environment (and this might be due to the transform being runner
>>>> native, inlined, source (e.g. Impulse or TestStream)) the elements have
>>>> to be encoded by SDK coder, immediately following decode by runner
>>>> coder. That (surprisingly) applies even to situations when runner is
>>>> implemented using different language than the client SDK, because it
>>>> implies that the type of produced elements must be one of types encoded
>>>> using model coders (well-known to the runner, otherwise the SDK will not
>>>> be able to consume it). But - due to property c) - this means that this
>>>> encode-decode step can be optimized out. This does not mean that it is
>>>> not (logically) present, though. This is exactly the case of native
>>>> Impulse transform.
>>>>
>>>> Now, from that we can conclude that on the boundary between executable
>>>> stages, or between runner (inlined) transform and executable stage, each
>>>> PCollection has to be encoded using SDK coder and immediately decoded by
>>>> runner coder, *unless this can be optimized out* by property c).
>>>>
>>>> This gives us two options where to implement this encode/decode step:
>>>>
>>>>     1) completely inside runner with the possibility to optimize the
>>>> encode/decode step by identity under right circumstances
>>>>
>>>>     2) partly in the runner and partly in the SDK - that is we encode
>>>> elements of PCollection using SDK coder into bytes, pass those to the
>>>> SDK harness and apply a custom decode step there. This works because SDK
>>>> coder encoded elements are in byte[], and that is well-known coder type.
>>>> We again only leverage property c) and optimize the SDK coder encode,
>>>> runner decode step out.
>>>>
>>>> The option 2) is exactly the proposal of TestStream producing byte[] and
>>>> decoding inside SDK-harness, the TestStream is actually inlined
>>>> transform, the elements are produced directly in runner (the SDK coder
>>>> is not known to the runner, but that does not matter, because the
>>>> elements are already encoded by client).
>>>>
>>>>    From the above it seems to me, that option 1) should be preferred, because:
>>>>
>>>>     i) it is generic, applicable to all inlined transforms, any sources
>>>>
>>>>     ii) it is consistent with how things logically work underneath
>>>>
>>>>     iii) it offers better room for optimization - option 2) might result
>>>> in cases when the elements are passed from the runner to the SDK-harness
>>>> only for the sake of the decoding from SDK coder and immediately
>>>> encoding back using SDK-coder and returned back to the runner. This
>>>> would be the case when TestStream would be directly consumed by inlined
>>>> (or external) transform.
>>> (1) is not possible if the Coder in question is not known to the
>>> Runner, which is why I proposed (2).
>> There is no particular need for the coder to be known. If transform is
>> to be inlined, what *has* to be known is the SDK-encoded form of the
>> elements. That holds true if:
>>
>>    a) either the SDK coder is known, or
>>
>>    b) encoded form of the produced elements is known in advance
>>
>> For TestStream it is the case b). For inlined primitive Read (or any
>> other transform which executes code) it is a).
> There's another hitch here for TestStream. For historical reasons,
> coders actually represent two encodings: nested (aka self delimiting)
> and unnested. TestStream elements are given as unnested encoded bytes,
> but the nested encoding is required for sending data to the SDK. The
> runner can't go from <nested encoding> to <unnested encoding> for an
> arbitrary unknown coder.
>
> (Even if it weren't for this complication, to be able to send already
> encoded bytes of an unknown coder to the SDK will also complicate the
> logic in choosing the coder to be used for the channel and sending the
> data, which is some of what you're running into (but can be solved
> differently for inlined reads as the coder can always be known by the
> runner).)

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský <je...@seznam.cz> wrote:
>
>
> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
> > On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský <je...@seznam.cz> wrote:
> >> Hi,
> >>
> >> I had some more time thinking about this and I'll try to recap that.
> >> First some invariants:
> >>
> >>    a) each PCollection<T> has actually two coders - an _SDK coder_ and a
> >> _runner coder_. These coders have the property, that each one can
> >> _decode_ what the other encoded, but the opposite is not true, the
> >> coders cannot _encode_ what the other _decoded_ (in general).
> >>
> >>    b) when is a PCollection<T> computed inside an environment, the
> >> elements are encoded using SDK coder on the side of SDK-harness and
> >> decoded using runner coder after receiving in the runner
> >>
> >>    c) under specific circumstances, the encode-decode step can be
> >> optimized out, that is the case where the SDK coder and all its
> >> subcoders are all well-known to the runner (in the present, that means
> >> that all the parts present in the model coders set). The reason for that
> >> is that in this specific situation runner_decode(sdk_encode(X)) = X.
> >> This property is essential.
> > However, in general, X can only pass from the SDK to the runner (or
> > vice versa) in encoded form.
> In general yes, but we are (mostly) talking transform inlining here, so
> it that particular situation, the elements might be passed in decoded form.
> >
> >>    d) from b) immediately follows, that when a PTransform does not run in
> >> an environment (and this might be due to the transform being runner
> >> native, inlined, source (e.g. Impulse or TestStream)) the elements have
> >> to be encoded by SDK coder, immediately following decode by runner
> >> coder. That (surprisingly) applies even to situations when runner is
> >> implemented using different language than the client SDK, because it
> >> implies that the type of produced elements must be one of types encoded
> >> using model coders (well-known to the runner, otherwise the SDK will not
> >> be able to consume it). But - due to property c) - this means that this
> >> encode-decode step can be optimized out. This does not mean that it is
> >> not (logically) present, though. This is exactly the case of native
> >> Impulse transform.
> >>
> >> Now, from that we can conclude that on the boundary between executable
> >> stages, or between runner (inlined) transform and executable stage, each
> >> PCollection has to be encoded using SDK coder and immediately decoded by
> >> runner coder, *unless this can be optimized out* by property c).
> >>
> >> This gives us two options where to implement this encode/decode step:
> >>
> >>    1) completely inside runner with the possibility to optimize the
> >> encode/decode step by identity under right circumstances
> >>
> >>    2) partly in the runner and partly in the SDK - that is we encode
> >> elements of PCollection using SDK coder into bytes, pass those to the
> >> SDK harness and apply a custom decode step there. This works because SDK
> >> coder encoded elements are in byte[], and that is well-known coder type.
> >> We again only leverage property c) and optimize the SDK coder encode,
> >> runner decode step out.
> >>
> >> The option 2) is exactly the proposal of TestStream producing byte[] and
> >> decoding inside SDK-harness, the TestStream is actually inlined
> >> transform, the elements are produced directly in runner (the SDK coder
> >> is not known to the runner, but that does not matter, because the
> >> elements are already encoded by client).
> >>
> >>   From the above it seems to me, that option 1) should be preferred, because:
> >>
> >>    i) it is generic, applicable to all inlined transforms, any sources
> >>
> >>    ii) it is consistent with how things logically work underneath
> >>
> >>    iii) it offers better room for optimization - option 2) might result
> >> in cases when the elements are passed from the runner to the SDK-harness
> >> only for the sake of the decoding from SDK coder and immediately
> >> encoding back using SDK-coder and returned back to the runner. This
> >> would be the case when TestStream would be directly consumed by inlined
> >> (or external) transform.
> > (1) is not possible if the Coder in question is not known to the
> > Runner, which is why I proposed (2).
>
> There is no particular need for the coder to be known. If transform is
> to be inlined, what *has* to be known is the SDK-encoded form of the
> elements. That holds true if:
>
>   a) either the SDK coder is known, or
>
>   b) encoded form of the produced elements is known in advance
>
> For TestStream it is the case b). For inlined primitive Read (or any
> other transform which executes code) it is a).

There's another hitch here for TestStream. For historical reasons,
coders actually represent two encodings: nested (aka self delimiting)
and unnested. TestStream elements are given as unnested encoded bytes,
but the nested encoding is required for sending data to the SDK. The
runner can't go from <nested encoding> to <unnested encoding> for an
arbitrary unknown coder.

(Even if it weren't for this complication, to be able to send already
encoded bytes of an unknown coder to the SDK will also complicate the
logic in choosing the coder to be used for the channel and sending the
data, which is some of what you're running into (but can be solved
differently for inlined reads as the coder can always be known by the
runner).)

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
On 9/3/21 1:06 AM, Robert Bradshaw wrote:
> On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský <je...@seznam.cz> wrote:
>> Hi,
>>
>> I had some more time thinking about this and I'll try to recap that.
>> First some invariants:
>>
>>    a) each PCollection<T> has actually two coders - an _SDK coder_ and a
>> _runner coder_. These coders have the property, that each one can
>> _decode_ what the other encoded, but the opposite is not true, the
>> coders cannot _encode_ what the other _decoded_ (in general).
>>
>>    b) when is a PCollection<T> computed inside an environment, the
>> elements are encoded using SDK coder on the side of SDK-harness and
>> decoded using runner coder after receiving in the runner
>>
>>    c) under specific circumstances, the encode-decode step can be
>> optimized out, that is the case where the SDK coder and all its
>> subcoders are all well-known to the runner (in the present, that means
>> that all the parts present in the model coders set). The reason for that
>> is that in this specific situation runner_decode(sdk_encode(X)) = X.
>> This property is essential.
> However, in general, X can only pass from the SDK to the runner (or
> vice versa) in encoded form.
In general yes, but we are (mostly) talking transform inlining here, so 
it that particular situation, the elements might be passed in decoded form.
>
>>    d) from b) immediately follows, that when a PTransform does not run in
>> an environment (and this might be due to the transform being runner
>> native, inlined, source (e.g. Impulse or TestStream)) the elements have
>> to be encoded by SDK coder, immediately following decode by runner
>> coder. That (surprisingly) applies even to situations when runner is
>> implemented using different language than the client SDK, because it
>> implies that the type of produced elements must be one of types encoded
>> using model coders (well-known to the runner, otherwise the SDK will not
>> be able to consume it). But - due to property c) - this means that this
>> encode-decode step can be optimized out. This does not mean that it is
>> not (logically) present, though. This is exactly the case of native
>> Impulse transform.
>>
>> Now, from that we can conclude that on the boundary between executable
>> stages, or between runner (inlined) transform and executable stage, each
>> PCollection has to be encoded using SDK coder and immediately decoded by
>> runner coder, *unless this can be optimized out* by property c).
>>
>> This gives us two options where to implement this encode/decode step:
>>
>>    1) completely inside runner with the possibility to optimize the
>> encode/decode step by identity under right circumstances
>>
>>    2) partly in the runner and partly in the SDK - that is we encode
>> elements of PCollection using SDK coder into bytes, pass those to the
>> SDK harness and apply a custom decode step there. This works because SDK
>> coder encoded elements are in byte[], and that is well-known coder type.
>> We again only leverage property c) and optimize the SDK coder encode,
>> runner decode step out.
>>
>> The option 2) is exactly the proposal of TestStream producing byte[] and
>> decoding inside SDK-harness, the TestStream is actually inlined
>> transform, the elements are produced directly in runner (the SDK coder
>> is not known to the runner, but that does not matter, because the
>> elements are already encoded by client).
>>
>>   From the above it seems to me, that option 1) should be preferred, because:
>>
>>    i) it is generic, applicable to all inlined transforms, any sources
>>
>>    ii) it is consistent with how things logically work underneath
>>
>>    iii) it offers better room for optimization - option 2) might result
>> in cases when the elements are passed from the runner to the SDK-harness
>> only for the sake of the decoding from SDK coder and immediately
>> encoding back using SDK-coder and returned back to the runner. This
>> would be the case when TestStream would be directly consumed by inlined
>> (or external) transform.
> (1) is not possible if the Coder in question is not known to the
> Runner, which is why I proposed (2).

There is no particular need for the coder to be known. If transform is 
to be inlined, what *has* to be known is the SDK-encoded form of the 
elements. That holds true if:

  a) either the SDK coder is known, or

  b) encoded form of the produced elements is known in advance

For TestStream it is the case b). For inlined primitive Read (or any 
other transform which executes code) it is a).

>
>> On 9/1/21 9:41 AM, Jan Lukavský wrote:
>>> On 9/1/21 9:15 AM, Robert Bradshaw wrote:
>>>
>>>> On Tue, Aug 31, 2021 at 11:48 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>> Sorry, I needed some time to let that sink in. :-)
>>>>>
>>>>> I think I understand why (and how) this will work for TestStream, still
>>>>> have a couple of notes, though:
>>>>>
>>>>>     a) the problem of type compatibility arises with the primitive
>>>>> Read as
>>>>> well, though we can solve it with different expansion for TestStream,
>>>>> that solution is not applicable to Read, because it has different
>>>>> contract
>>>>>
>>>>>     b) the same exact problem problem will arise every time we inline
>>>>> any
>>>>> transform that would otherwise be run in an environment
>>>>>
>>>>>     c) extracting bytes from TestStream actually bypasses "model coders"
>>>>> on the side of runner, should we do that in other cases as well?
>>>>>
>>>>> The TestStream problem is a special case, I think a more generic
>>>>> solution would be better.
>>>> Yep, TestStream is special and simpler. I'm not sure something generic
>>>> is always possible.
>>>>
>>>>> Two questions:
>>>>>
>>>>>     1) Could we create a mapping in runners-core-construction-java that
>>>>> would take Pipeline proto, and PCollectionId and create a mapping
>>>>> function from "sdk coder space" to "runner coder space"? That could be
>>>>> optimized to identity if the coder of the PCollection consists of model
>>>>> coders only. In that case the sdk coder and runner coder are identical,
>>>>> producing the same types. This mapping function could be reused by both
>>>>> portable TestStream, inlined primitive Read and any other future
>>>>> inlined
>>>>> transform.
>>>> This could help with the specific case of Java transforms being
>>>> inlined on a Java-based runner. I'd rather avoid Java-specific
>>>> solutions if possible. (E.g. the ULR is in Python, and Dataflow Runner
>>>> v2 is in C++.)
>>> The definition is generic, language agnostic. It can be rephrased as
>>> "if the SDK of a runner matches SDK of a transform, the transform can
>>> be inlined, and in order to be type-compatible coder mapping between
>>> SDK coder space and runner coder space has to be applied". Yes, when
>>> we implement this in "runners-core-construction-java", the "runner
>>> SDK" and "transform SDK" equals to Java SDK on both parts. But that is
>>> implied by the implementation, not the definition. For Go or Python we
>>> can implement exactly the same.
>>>> I'd say if a runner decides to truely inline a transform with an
>>>> alternative implementation than calling the SDK, it would only do so
>>>> in cases where it understands the input and output types, if any.
>>>> (With Read we have a case where the output type is not fixed, which
>>>> complicates things.) Generally I would probably structure most
>>>> "inlined" operations as one that happens to execute in an in-process
>>>> environment rather than an external one, which would mean that
>>>> "inlined" ones would not have to be handled specially, but could just
>>>> fall out due to the existing length prefixing logic.
>>>>
>>>> Of course just making SDFs work well on all runners would be the best
>>>> long term solution for Read, and avoid all these issues, but we may
>>>> need hacks in the meantime.
>>> I think the problem is broader than sources, so while it is true, that
>>> SDF would help, it is not the solution.
>>>>>     2) Why does runner need to understand the types it processes and
>>>>> does
>>>>> not work with raw bytes all the times, including model coders, the same
>>>>> way as is the proposed solution for TestStream, but for all transforms?
>>>>> The first step to every executable stage would then be to decode the
>>>>> input from raw bytes and only then process it. What is the benefit of
>>>>> runner understanding _some_ of the coders? It would be required for a
>>>>> runner to understand coders for root transforms (Impulse? any other?),
>>>>> but are there any other places where this is necessary?
>>>> A runner needs to understand the KV coder to execute GroupByKey (and
>>>> the result is a KV<K, Iterable<V>>, so IterableCoder is a requirement
>>>> as well). WindowedCoders are likewise needed so the runner can
>>>> deconstruct the timestamps, etc. Timers are sent via TimerCoders. A
>>>> runner also needs a way to deal with unknown coders, hence
>>>> LengthPrefixCoder. Asside from that, the runner can just deal in raw
>>>> bytes (though sometimes there are efficiency gains to understanding
>>>> more coders, e.g. wrapping tiny integers in LengthPrefix could be
>>>> rather wasteful).
>>>>
>>>> Well known coders, such as RowCoder, are of course useful for
>>>> communication between SDKs of different languages, even if the runner
>>>> doesn't care about them.
>>> +1, understood and agree.
>>>>> On 9/1/21 1:37 AM, Robert Bradshaw wrote:
>>>>>> On Tue, Aug 31, 2021 at 4:31 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>> If I can depict that:
>>>>>>>
>>>>>>> TestStream0 -> TestStream1 (DoFn), that is
>>>>>>>
>>>>>>> TestStream0 -> TestStream1 (executable stage) -> consumer 2
>>>>>>> (executable
>>>>>>> stage)
>>>>>> Yes. Expanding it out more,
>>>>>>
>>>>>> TestStream0 -> pc1 -> TestStream1 -> pc2 -> Consumer2 -> ...
>>>>>>
>>>>>> Where pc1 is a PCollection<bytes> and pc2 is a PCollection<T>.
>>>>>>
>>>>>>> In this scenario TestStream0 can produce byte[], and TestStream1 can
>>>>>>> consume them, the problem is that if the coder of TestStream0
>>>>>> I assume you mean TestStream1, as TestStream0 has no coder.
>>>>>>
>>>>>>> is some
>>>>>>> (recursively) known model coder (e.g. KvCoder), then consumer 2
>>>>>>> will not
>>>>>>> be able to decode that data.
>>>>>> We know that, by construction, TestStream1 and Consumer2 both are
>>>>>> executed in an environment that understands pc2's Coder<T>. The
>>>>>> difference here is that the runner need not understand Coder<T> as it
>>>>>> could inject length prefixing on pc2 if necessary.
>>>>>>
>>>>>>
>>>>>>> On 9/1/21 1:24 AM, Robert Bradshaw wrote:
>>>>>>>> On Tue, Aug 31, 2021 at 4:12 PM Jan Lukavský <je...@seznam.cz>
>>>>>>>> wrote:
>>>>>>>>>> Fortunately, there's no value in having the primitive TestStream
>>>>>>>>>> produce elements of arbitrary type (vs. sources, where the read is
>>>>>>>>>> inherently typed depending on the source and particular library
>>>>>>>>>> used
>>>>>>>>>> for reading it).
>>>>>>>>> I disagree with this one - the consumer(s) of elements from
>>>>>>>>> TestStream
>>>>>>>>> are (should be treated as) unknown to the runner, might be inlined,
>>>>>>>>> might be x-lang and whether or not will pass the boundary of
>>>>>>>>> executable
>>>>>>>>> stage, all these questions are important. The coder (and if it
>>>>>>>>> is model
>>>>>>>>> coder or not, and what are the sub-coders) must be precisely
>>>>>>>>> known to
>>>>>>>>> all parties that cooperate on the computation, because otherwise
>>>>>>>>> these
>>>>>>>>> parties might not agree on the binary representation.
>>>>>>>> What I'm saying is that there's little value in a primitive
>>>>>>>> TestStream that can produce all sorts of T, vs. a composite
>>>>>>>> TestStream
>>>>>>>> consisting of a TestStream0 primitive producing only bytes, followed
>>>>>>>> by a TestStream1 that decodes those bytes into T. This sidesteps all
>>>>>>>> issues of coder compatibility, as bytes is well-known and
>>>>>>>> TestStream1
>>>>>>>> is just a DoFn.
>>>>>>>>
>>>>>>>>> On 9/1/21 12:59 AM, Robert Bradshaw wrote:
>>>>>>>>>> Due to issues with Flink executing Reads as SDFs, there is an
>>>>>>>>>> alternative mode where Read is treated as a primitive and executed
>>>>>>>>>> directly in the runner. This, of course, requires that the
>>>>>>>>>> Read's UDF
>>>>>>>>>> be implemented in the same language as the Runner
>>>>>>>>>> (specifically, in
>>>>>>>>>> this case, Java) and their interpretation of what Coders should or
>>>>>>>>>> should not be wrapped agree (lest other complications arise).
>>>>>>>>>>
>>>>>>>>>> It's true one can view TestStream as a source, but it's a bit more
>>>>>>>>>> than that as it has the ability to wait for quiessence before
>>>>>>>>>> emitting
>>>>>>>>>> more elements/watermark updates to give stronger guarantees on
>>>>>>>>>> determinism, which generally requires deeper runner
>>>>>>>>>> integration. In
>>>>>>>>>> addition, TestStream is not associated with a UDF or specific
>>>>>>>>>> SDK the
>>>>>>>>>> way Sources are which is where the problem arises: TestStream
>>>>>>>>>> gives
>>>>>>>>>> the encoded form of the elements but the runner may need to
>>>>>>>>>> produce a
>>>>>>>>>> different encoded form of the elements (say, one with length
>>>>>>>>>> prefixing) which is not possible without knowing the Coder.
>>>>>>>>>> Fortunately, there's no value in having the primitive TestStream
>>>>>>>>>> produce elements of arbitrary type (vs. sources, where the read is
>>>>>>>>>> inherently typed depending on the source and particular library
>>>>>>>>>> used
>>>>>>>>>> for reading it).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, Aug 31, 2021 at 3:41 PM Ke Wu <ke...@gmail.com> wrote:
>>>>>>>>>>> Read does not have translation in portability, so the
>>>>>>>>>>> implementation is that it needs to be primitive transform
>>>>>>>>>>> explicitly implemented by the runner. The encoding/decoding
>>>>>>>>>>> has to happen in the runner.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Could you help me understand this a bit more? IIRC, Read is
>>>>>>>>>>> NOT being translated in portable mode exactly means it is a
>>>>>>>>>>> composite transform instead of primitive because all primitive
>>>>>>>>>>> transforms are required to be translated. In addition, Read is
>>>>>>>>>>> a composite transform of Impulse, which produces dummy bytes
>>>>>>>>>>> [1] to trigger subsequent ParDo/ExecutableStage, where
>>>>>>>>>>> decoding the actual source happens [2]
>>>>>>>>>>>
>>>>>>>>>>> There seems to be no role of the SDK harness with regard to
>>>>>>>>>>> the TestStream, because the elements are already encoded by
>>>>>>>>>>> the submitting SDK. The coders must match nevertheless,
>>>>>>>>>>> because you can have Events of type
>>>>>>>>>>> KV<KV<WindowedValue<Integer, Object>>> and what will and what
>>>>>>>>>>> will not get length-prefixed depends on which parts exactly
>>>>>>>>>>> are "known" (model) coders and which are not. Encoding the
>>>>>>>>>>> whole value as single byte array will not work for the
>>>>>>>>>>> consuming SDK harness, which will see that there should be
>>>>>>>>>>> nested KvCoders instead.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I don’t think I fully understand what you say here. TestStream
>>>>>>>>>>> is currently a primitive transform, therefore there is no role
>>>>>>>>>>> of SDK harness. This is what the proposal to change, to make
>>>>>>>>>>> TestStream a composite transform with a primitive transform
>>>>>>>>>>> and subsequent ParDo to decode to the desired format.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Impulse.java#L39
>>>>>>>>>>> [2]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L149
>>>>>>>>>>>
>>>>>>>>>>> On Aug 31, 2021, at 3:21 PM, Jan Lukavský <je...@seznam.cz>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>> On 9/1/21 12:13 AM, Ke Wu wrote:
>>>>>>>>>>>
>>>>>>>>>>> Hi Jan,
>>>>>>>>>>>
>>>>>>>>>>> Here is my understanding,
>>>>>>>>>>>
>>>>>>>>>>> Runner is being brought up by job server driver, which is up
>>>>>>>>>>> and running before the job submission, i.e. it is job
>>>>>>>>>>> agnostic. Therefore, the runner it brought up does not have
>>>>>>>>>>> any SDK coder available and artifact staging only happens for
>>>>>>>>>>> SDK workers.
>>>>>>>>>>>
>>>>>>>>>>> You are right that Read and TestStream are sources, however
>>>>>>>>>>> the one thing that distinguish them is that Read transform is
>>>>>>>>>>> a composite transform and the decoding happens in
>>>>>>>>>>> ParDo/ExecutableStage, i.e. on SDK worker.
>>>>>>>>>>>
>>>>>>>>>>> Read does not have translation in portability, so the
>>>>>>>>>>> implementation is that it needs to be primitive transform
>>>>>>>>>>> explicitly implemented by the runner. The encoding/decoding
>>>>>>>>>>> has to happen in the runner.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> The proposal here is also to make the public facing TestStream
>>>>>>>>>>> transform a composite transform instead of primitive now, so
>>>>>>>>>>> that the decoding would occur on the SDK worker side where SDK
>>>>>>>>>>> coder is available, and the primitive that powers TestStream,
>>>>>>>>>>> which will be directly translated by runner to always produce
>>>>>>>>>>> raw bytes, and these raw bytes will be decoded on the SDK
>>>>>>>>>>> worker side.
>>>>>>>>>>>
>>>>>>>>>>> There seems to be no role of the SDK harness with regard to
>>>>>>>>>>> the TestStream, because the elements are already encoded by
>>>>>>>>>>> the submitting SDK. The coders must match nevertheless,
>>>>>>>>>>> because you can have Events of type
>>>>>>>>>>> KV<KV<WindowedValue<Integer, Object>>> and what will and what
>>>>>>>>>>> will not get length-prefixed depends on which parts exactly
>>>>>>>>>>> are "known" (model) coders and which are not. Encoding the
>>>>>>>>>>> whole value as single byte array will not work for the
>>>>>>>>>>> consuming SDK harness, which will see that there should be
>>>>>>>>>>> nested KvCoders instead.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> Ke
>>>>>>>>>>>
>>>>>>>>>>> On Aug 31, 2021, at 2:56 PM, Jan Lukavský <je...@seznam.cz>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>> Sorry if I'm missing something obvious, but I don't quite see
>>>>>>>>>>> the difference between Read and TestStream regarding the
>>>>>>>>>>> discussed issue with coders. Couple of thoughts:
>>>>>>>>>>>
>>>>>>>>>>>       a) both Read and TestStream are _sources_ - they produce
>>>>>>>>>>> elements that are consumed by downstream transforms
>>>>>>>>>>>
>>>>>>>>>>>       b) the coder of a particular PCollection is defined by
>>>>>>>>>>> the Pipeline proto - it is the (client side) SDK that owns the
>>>>>>>>>>> Pipeline and that defines all the coders
>>>>>>>>>>>
>>>>>>>>>>>       c) runners must adhere to these coders, because otherwise
>>>>>>>>>>> there is risk of coder mismatch, most probably on edges like
>>>>>>>>>>> x-lang transforms or inlined transforms
>>>>>>>>>>>
>>>>>>>>>>> I tried the approach of encoding the output of Read into byte
>>>>>>>>>>> array as well, but that turns out to have the problem that
>>>>>>>>>>> once there is a (partially) known coder in play, this does not
>>>>>>>>>>> work, because the consuming transform (executable stage)
>>>>>>>>>>> expects to see the wire coder - that is not simply byte array,
>>>>>>>>>>> because the type of elements might be for instance KV<K, V>,
>>>>>>>>>>> where KvCoder is one of ModelCoders. That does not encode
>>>>>>>>>>> using LengthPrefixCoder and as such will be incompatible with
>>>>>>>>>>> LengthPrefixCoder(ByteArrayCoder). The TestStream needs to
>>>>>>>>>>> know the coder of elements, because that defines where exactly
>>>>>>>>>>> must or must not be inserted length-prefixing. The logic in
>>>>>>>>>>> LengthPrefixUnknownCoders [1] is recursive for ModelCoders.
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> https://github.com/apache/beam/blob/ff70e740a2155592dfcb302ff6303cc19660a268/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>>>>>>
>>>>>>>>>>> On 8/31/21 11:29 PM, Ke Wu wrote:
>>>>>>>>>>>
>>>>>>>>>>> Awesome! Thank you Luke and Robert.
>>>>>>>>>>>
>>>>>>>>>>> Also created https://issues.apache.org/jira/browse/BEAM-12828
>>>>>>>>>>> to track unit test conversion. I could take it after I updated
>>>>>>>>>>> Samza runner to support TestStream in portable mode.
>>>>>>>>>>>
>>>>>>>>>>> On Aug 31, 2021, at 2:05 PM, Robert Bradshaw
>>>>>>>>>>> <ro...@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>> Created https://issues.apache.org/jira/browse/BEAM-12827 to
>>>>>>>>>>> track this.
>>>>>>>>>>>
>>>>>>>>>>> +1 to converting tests to just use longs for better coverage
>>>>>>>>>>> for now.
>>>>>>>>>>>
>>>>>>>>>>> Also, yes, this is very similar to the issues encountered by
>>>>>>>>>>> Reads,
>>>>>>>>>>> but the solution is a bit simpler as there's no need for the
>>>>>>>>>>> TestStream primitive to interact with the decoded version of the
>>>>>>>>>>> elements (unlike Reads, where the sources often give elements in
>>>>>>>>>>> un-encoded form) and no user code to run.
>>>>>>>>>>>
>>>>>>>>>>> - Robert
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Aug 31, 2021 at 11:00 AM Jan Lukavský
>>>>>>>>>>> <je...@seznam.cz> wrote:
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> This looks (and likely has the same cause) similar to what I
>>>>>>>>>>> have experienced when making primitive Read supported by
>>>>>>>>>>> Flink. The final solution would be to make SDK coders known to
>>>>>>>>>>> the runner of the same SDK (already present in various
>>>>>>>>>>> different threads). But until then, the solution seems to be
>>>>>>>>>>> something like [1]. The root cause is that the executable
>>>>>>>>>>> stage expects its input to be encoded by the SDK harness, and
>>>>>>>>>>> that part is missing when the transform is inlined (like Read
>>>>>>>>>>> in my case, or TestStream in your case). The intoWireTypes
>>>>>>>>>>> method simulates precisely this part - it encodes the
>>>>>>>>>>> PCollection via coder defined in the SDK harness and then
>>>>>>>>>>> decodes it by coder defined by the runner (which match on
>>>>>>>>>>> binary level, but produce different types).
>>>>>>>>>>>
>>>>>>>>>>> Jan
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> https://github.com/apache/beam/blob/dd7945f9f259a2989f9396d1d7a8dcb122711a52/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java#L657
>>>>>>>>>>>
>>>>>>>>>>> On 8/31/21 7:27 PM, Luke Cwik wrote:
>>>>>>>>>>>
>>>>>>>>>>> I originally wasn't for making it a composite because it
>>>>>>>>>>> changes the "graph" structure but the more I thought about it
>>>>>>>>>>> the more I like it.
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Aug 31, 2021 at 10:06 AM Robert Bradshaw
>>>>>>>>>>> <ro...@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Aug 31, 2021 at 9:18 AM Luke Cwik <lc...@google.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Mon, Aug 30, 2021 at 7:07 PM Ke Wu <ke...@gmail.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>
>>>>>>>>>>> This is Ke. I am working on enable TestStream support for
>>>>>>>>>>> Samza Runner in portable mode and discovers something unexpected.
>>>>>>>>>>>
>>>>>>>>>>> In my implementation for Samza Runner, couple of tests are
>>>>>>>>>>> failing with errors like
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> java.lang.ClassCastException: java.lang.Integer cannot be cast
>>>>>>>>>>> to [B
>>>>>>>>>>>
>>>>>>>>>>> I noticed these tests have the same symptom on Flink Runner as
>>>>>>>>>>> well, which are currently excluded:
>>>>>>>>>>>
>>>>>>>>>>> https://issues.apache.org/jira/browse/BEAM-12048
>>>>>>>>>>> https://issues.apache.org/jira/browse/BEAM-12050
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> After some more digging, I realized that it is because the
>>>>>>>>>>> combination of following facts:
>>>>>>>>>>>
>>>>>>>>>>> TestStream is a primitive transform, therefore, Runners are
>>>>>>>>>>> supposed to translate directly, the most intuitive
>>>>>>>>>>> implementation for each runner to do is to parse the payload
>>>>>>>>>>> to decode TestStream.Event [1] on the Runner process to be
>>>>>>>>>>> handed over to subsequent stages.
>>>>>>>>>>> When TestStream used with Integers, i.e. VarIntCoder to
>>>>>>>>>>> initialize, since VarIntCoder is NOT a registered ModelCoder
>>>>>>>>>>> [2], it will be treated as custom coder during conversion to
>>>>>>>>>>> protobuf pipeline [3] and will be replaced with byte array
>>>>>>>>>>> coder [4] when runner sends data to SDK worker.
>>>>>>>>>>> Therefore an error occurs because the decoded TestStream.Event
>>>>>>>>>>> has Integer as its value but the remote input receiver is
>>>>>>>>>>> expecting byte array, causing java.lang.ClassCastException:
>>>>>>>>>>> java.lang.Integer cannot be cast to [B
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> In addition, I tried to update all these failed tests to use
>>>>>>>>>>> Long instead of Integer, and all tests will pass since
>>>>>>>>>>> VarLongCoder is a known coder. I do understand that runner
>>>>>>>>>>> process does not have user artifacts staged so it can only use
>>>>>>>>>>> coders in  beam model when communicating with SDK worker process.
>>>>>>>>>>>
>>>>>>>>>>> Couple of questions on this:
>>>>>>>>>>>
>>>>>>>>>>> 1. Is it expected that VarIntegerCoder is not a known coder?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Yes since no one has worked to make it a well known coder.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> The notion of "integer" vs. "long" is also language-specific
>>>>>>>>>>> detail as
>>>>>>>>>>> well, so not sure it makes sense as a well-known coder.
>>>>>>>>>>>
>>>>>>>>>>> It can be made a well known coder and this would solve the
>>>>>>>>>>> immediate problem but not the long term issue of portable
>>>>>>>>>>> TestStream not supporting arbitrary types.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> +1. Rather than making coder a property of TestStream, I would
>>>>>>>>>>> be in
>>>>>>>>>>> favor of the TestStream primitive always producing bytes
>>>>>>>>>>> (basically,
>>>>>>>>>>> by definition), and providing a composite that consists of this
>>>>>>>>>>> followed by a decoding to give us a typed TestStream.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 2. Is TestStream always supposed to be translated the payload
>>>>>>>>>>> as raw bytes in order that runner process can always send it
>>>>>>>>>>> to SDK worker with the default byte array coder and asks SDK
>>>>>>>>>>> worker to decode accordingly?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Having the runner treat it always as bytes and not T is likely
>>>>>>>>>>> the best solution but isn't necessary.
>>>>>>>>>>>
>>>>>>>>>>> 3. If Yes to 2), then does it mean, TestStream needs to be
>>>>>>>>>>> translated in a completely different way in portable mode from
>>>>>>>>>>> classic mode since in classic mode, translator can directly
>>>>>>>>>>> translates the payload to its final format.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> There are a few ways to fix the current implementation to work
>>>>>>>>>>> for all types. One way would be if we required the
>>>>>>>>>>> encoded_element to be the "nested" encoding and then ensured
>>>>>>>>>>> that the runner uses a WindowedValue<ByteArrayCoder in outer
>>>>>>>>>>> context> and the SDK used WindowedValue<T> (note that this
>>>>>>>>>>> isn't WindowedValue<LengthPrefix<T>>) for the wire coders.
>>>>>>>>>>> This is quite annoying cause the runner inserts length
>>>>>>>>>>> prefixing in a lot of places (effectively every time it sees
>>>>>>>>>>> an unknown type) so we would need to special case this and
>>>>>>>>>>> propagate this correction through any runner native transforms
>>>>>>>>>>> (e.g. GBK) until the SDK consumes it.
>>>>>>>>>>>
>>>>>>>>>>> Another way would be to ensure that the SDK always uses
>>>>>>>>>>> LengthPrefix<T> as the PCollection encoding and the
>>>>>>>>>>> encoded_element format. This would mean that the runner can
>>>>>>>>>>> translate it to a T if it so chooses and won't have the
>>>>>>>>>>> annoying special case propagation logic. This leaks the length
>>>>>>>>>>> prefixing into the SDK at graph construction time which is not
>>>>>>>>>>> what it was meant for.
>>>>>>>>>>>
>>>>>>>>>>> Swapping to use an existing well known type is by far the
>>>>>>>>>>> easiest approach as you had discovered and won't impact the
>>>>>>>>>>> correctness of the tests.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> Ke
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java#L52
>>>>>>>>>>> [2]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java#L65
>>>>>>>>>>> [3]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L99
>>>>>>>>>>> [4]
>>>>>>>>>>> https://github.com/apache/beam/blob/master/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L93
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský <je...@seznam.cz> wrote:
>
> Hi,
>
> I had some more time thinking about this and I'll try to recap that.
> First some invariants:
>
>   a) each PCollection<T> has actually two coders - an _SDK coder_ and a
> _runner coder_. These coders have the property, that each one can
> _decode_ what the other encoded, but the opposite is not true, the
> coders cannot _encode_ what the other _decoded_ (in general).
>
>   b) when is a PCollection<T> computed inside an environment, the
> elements are encoded using SDK coder on the side of SDK-harness and
> decoded using runner coder after receiving in the runner
>
>   c) under specific circumstances, the encode-decode step can be
> optimized out, that is the case where the SDK coder and all its
> subcoders are all well-known to the runner (in the present, that means
> that all the parts present in the model coders set). The reason for that
> is that in this specific situation runner_decode(sdk_encode(X)) = X.
> This property is essential.

However, in general, X can only pass from the SDK to the runner (or
vice versa) in encoded form.

>   d) from b) immediately follows, that when a PTransform does not run in
> an environment (and this might be due to the transform being runner
> native, inlined, source (e.g. Impulse or TestStream)) the elements have
> to be encoded by SDK coder, immediately following decode by runner
> coder. That (surprisingly) applies even to situations when runner is
> implemented using different language than the client SDK, because it
> implies that the type of produced elements must be one of types encoded
> using model coders (well-known to the runner, otherwise the SDK will not
> be able to consume it). But - due to property c) - this means that this
> encode-decode step can be optimized out. This does not mean that it is
> not (logically) present, though. This is exactly the case of native
> Impulse transform.
>
> Now, from that we can conclude that on the boundary between executable
> stages, or between runner (inlined) transform and executable stage, each
> PCollection has to be encoded using SDK coder and immediately decoded by
> runner coder, *unless this can be optimized out* by property c).
>
> This gives us two options where to implement this encode/decode step:
>
>   1) completely inside runner with the possibility to optimize the
> encode/decode step by identity under right circumstances
>
>   2) partly in the runner and partly in the SDK - that is we encode
> elements of PCollection using SDK coder into bytes, pass those to the
> SDK harness and apply a custom decode step there. This works because SDK
> coder encoded elements are in byte[], and that is well-known coder type.
> We again only leverage property c) and optimize the SDK coder encode,
> runner decode step out.
>
> The option 2) is exactly the proposal of TestStream producing byte[] and
> decoding inside SDK-harness, the TestStream is actually inlined
> transform, the elements are produced directly in runner (the SDK coder
> is not known to the runner, but that does not matter, because the
> elements are already encoded by client).
>
>  From the above it seems to me, that option 1) should be preferred, because:
>
>   i) it is generic, applicable to all inlined transforms, any sources
>
>   ii) it is consistent with how things logically work underneath
>
>   iii) it offers better room for optimization - option 2) might result
> in cases when the elements are passed from the runner to the SDK-harness
> only for the sake of the decoding from SDK coder and immediately
> encoding back using SDK-coder and returned back to the runner. This
> would be the case when TestStream would be directly consumed by inlined
> (or external) transform.

(1) is not possible if the Coder in question is not known to the
Runner, which is why I proposed (2).

> On 9/1/21 9:41 AM, Jan Lukavský wrote:
> > On 9/1/21 9:15 AM, Robert Bradshaw wrote:
> >
> >> On Tue, Aug 31, 2021 at 11:48 PM Jan Lukavský <je...@seznam.cz> wrote:
> >>> Sorry, I needed some time to let that sink in. :-)
> >>>
> >>> I think I understand why (and how) this will work for TestStream, still
> >>> have a couple of notes, though:
> >>>
> >>>    a) the problem of type compatibility arises with the primitive
> >>> Read as
> >>> well, though we can solve it with different expansion for TestStream,
> >>> that solution is not applicable to Read, because it has different
> >>> contract
> >>>
> >>>    b) the same exact problem problem will arise every time we inline
> >>> any
> >>> transform that would otherwise be run in an environment
> >>>
> >>>    c) extracting bytes from TestStream actually bypasses "model coders"
> >>> on the side of runner, should we do that in other cases as well?
> >>>
> >>> The TestStream problem is a special case, I think a more generic
> >>> solution would be better.
> >> Yep, TestStream is special and simpler. I'm not sure something generic
> >> is always possible.
> >>
> >>> Two questions:
> >>>
> >>>    1) Could we create a mapping in runners-core-construction-java that
> >>> would take Pipeline proto, and PCollectionId and create a mapping
> >>> function from "sdk coder space" to "runner coder space"? That could be
> >>> optimized to identity if the coder of the PCollection consists of model
> >>> coders only. In that case the sdk coder and runner coder are identical,
> >>> producing the same types. This mapping function could be reused by both
> >>> portable TestStream, inlined primitive Read and any other future
> >>> inlined
> >>> transform.
> >> This could help with the specific case of Java transforms being
> >> inlined on a Java-based runner. I'd rather avoid Java-specific
> >> solutions if possible. (E.g. the ULR is in Python, and Dataflow Runner
> >> v2 is in C++.)
> > The definition is generic, language agnostic. It can be rephrased as
> > "if the SDK of a runner matches SDK of a transform, the transform can
> > be inlined, and in order to be type-compatible coder mapping between
> > SDK coder space and runner coder space has to be applied". Yes, when
> > we implement this in "runners-core-construction-java", the "runner
> > SDK" and "transform SDK" equals to Java SDK on both parts. But that is
> > implied by the implementation, not the definition. For Go or Python we
> > can implement exactly the same.
> >>
> >> I'd say if a runner decides to truely inline a transform with an
> >> alternative implementation than calling the SDK, it would only do so
> >> in cases where it understands the input and output types, if any.
> >> (With Read we have a case where the output type is not fixed, which
> >> complicates things.) Generally I would probably structure most
> >> "inlined" operations as one that happens to execute in an in-process
> >> environment rather than an external one, which would mean that
> >> "inlined" ones would not have to be handled specially, but could just
> >> fall out due to the existing length prefixing logic.
> >>
> >> Of course just making SDFs work well on all runners would be the best
> >> long term solution for Read, and avoid all these issues, but we may
> >> need hacks in the meantime.
> > I think the problem is broader than sources, so while it is true, that
> > SDF would help, it is not the solution.
> >>
> >>>    2) Why does runner need to understand the types it processes and
> >>> does
> >>> not work with raw bytes all the times, including model coders, the same
> >>> way as is the proposed solution for TestStream, but for all transforms?
> >>> The first step to every executable stage would then be to decode the
> >>> input from raw bytes and only then process it. What is the benefit of
> >>> runner understanding _some_ of the coders? It would be required for a
> >>> runner to understand coders for root transforms (Impulse? any other?),
> >>> but are there any other places where this is necessary?
> >> A runner needs to understand the KV coder to execute GroupByKey (and
> >> the result is a KV<K, Iterable<V>>, so IterableCoder is a requirement
> >> as well). WindowedCoders are likewise needed so the runner can
> >> deconstruct the timestamps, etc. Timers are sent via TimerCoders. A
> >> runner also needs a way to deal with unknown coders, hence
> >> LengthPrefixCoder. Asside from that, the runner can just deal in raw
> >> bytes (though sometimes there are efficiency gains to understanding
> >> more coders, e.g. wrapping tiny integers in LengthPrefix could be
> >> rather wasteful).
> >>
> >> Well known coders, such as RowCoder, are of course useful for
> >> communication between SDKs of different languages, even if the runner
> >> doesn't care about them.
> > +1, understood and agree.
> >>
> >>> On 9/1/21 1:37 AM, Robert Bradshaw wrote:
> >>>> On Tue, Aug 31, 2021 at 4:31 PM Jan Lukavský <je...@seznam.cz> wrote:
> >>>>> If I can depict that:
> >>>>>
> >>>>> TestStream0 -> TestStream1 (DoFn), that is
> >>>>>
> >>>>> TestStream0 -> TestStream1 (executable stage) -> consumer 2
> >>>>> (executable
> >>>>> stage)
> >>>> Yes. Expanding it out more,
> >>>>
> >>>> TestStream0 -> pc1 -> TestStream1 -> pc2 -> Consumer2 -> ...
> >>>>
> >>>> Where pc1 is a PCollection<bytes> and pc2 is a PCollection<T>.
> >>>>
> >>>>> In this scenario TestStream0 can produce byte[], and TestStream1 can
> >>>>> consume them, the problem is that if the coder of TestStream0
> >>>> I assume you mean TestStream1, as TestStream0 has no coder.
> >>>>
> >>>>> is some
> >>>>> (recursively) known model coder (e.g. KvCoder), then consumer 2
> >>>>> will not
> >>>>> be able to decode that data.
> >>>> We know that, by construction, TestStream1 and Consumer2 both are
> >>>> executed in an environment that understands pc2's Coder<T>. The
> >>>> difference here is that the runner need not understand Coder<T> as it
> >>>> could inject length prefixing on pc2 if necessary.
> >>>>
> >>>>
> >>>>> On 9/1/21 1:24 AM, Robert Bradshaw wrote:
> >>>>>> On Tue, Aug 31, 2021 at 4:12 PM Jan Lukavský <je...@seznam.cz>
> >>>>>> wrote:
> >>>>>>>> Fortunately, there's no value in having the primitive TestStream
> >>>>>>>> produce elements of arbitrary type (vs. sources, where the read is
> >>>>>>>> inherently typed depending on the source and particular library
> >>>>>>>> used
> >>>>>>>> for reading it).
> >>>>>>> I disagree with this one - the consumer(s) of elements from
> >>>>>>> TestStream
> >>>>>>> are (should be treated as) unknown to the runner, might be inlined,
> >>>>>>> might be x-lang and whether or not will pass the boundary of
> >>>>>>> executable
> >>>>>>> stage, all these questions are important. The coder (and if it
> >>>>>>> is model
> >>>>>>> coder or not, and what are the sub-coders) must be precisely
> >>>>>>> known to
> >>>>>>> all parties that cooperate on the computation, because otherwise
> >>>>>>> these
> >>>>>>> parties might not agree on the binary representation.
> >>>>>> What I'm saying is that there's little value in a primitive
> >>>>>> TestStream that can produce all sorts of T, vs. a composite
> >>>>>> TestStream
> >>>>>> consisting of a TestStream0 primitive producing only bytes, followed
> >>>>>> by a TestStream1 that decodes those bytes into T. This sidesteps all
> >>>>>> issues of coder compatibility, as bytes is well-known and
> >>>>>> TestStream1
> >>>>>> is just a DoFn.
> >>>>>>
> >>>>>>> On 9/1/21 12:59 AM, Robert Bradshaw wrote:
> >>>>>>>> Due to issues with Flink executing Reads as SDFs, there is an
> >>>>>>>> alternative mode where Read is treated as a primitive and executed
> >>>>>>>> directly in the runner. This, of course, requires that the
> >>>>>>>> Read's UDF
> >>>>>>>> be implemented in the same language as the Runner
> >>>>>>>> (specifically, in
> >>>>>>>> this case, Java) and their interpretation of what Coders should or
> >>>>>>>> should not be wrapped agree (lest other complications arise).
> >>>>>>>>
> >>>>>>>> It's true one can view TestStream as a source, but it's a bit more
> >>>>>>>> than that as it has the ability to wait for quiessence before
> >>>>>>>> emitting
> >>>>>>>> more elements/watermark updates to give stronger guarantees on
> >>>>>>>> determinism, which generally requires deeper runner
> >>>>>>>> integration. In
> >>>>>>>> addition, TestStream is not associated with a UDF or specific
> >>>>>>>> SDK the
> >>>>>>>> way Sources are which is where the problem arises: TestStream
> >>>>>>>> gives
> >>>>>>>> the encoded form of the elements but the runner may need to
> >>>>>>>> produce a
> >>>>>>>> different encoded form of the elements (say, one with length
> >>>>>>>> prefixing) which is not possible without knowing the Coder.
> >>>>>>>> Fortunately, there's no value in having the primitive TestStream
> >>>>>>>> produce elements of arbitrary type (vs. sources, where the read is
> >>>>>>>> inherently typed depending on the source and particular library
> >>>>>>>> used
> >>>>>>>> for reading it).
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Tue, Aug 31, 2021 at 3:41 PM Ke Wu <ke...@gmail.com> wrote:
> >>>>>>>>> Read does not have translation in portability, so the
> >>>>>>>>> implementation is that it needs to be primitive transform
> >>>>>>>>> explicitly implemented by the runner. The encoding/decoding
> >>>>>>>>> has to happen in the runner.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Could you help me understand this a bit more? IIRC, Read is
> >>>>>>>>> NOT being translated in portable mode exactly means it is a
> >>>>>>>>> composite transform instead of primitive because all primitive
> >>>>>>>>> transforms are required to be translated. In addition, Read is
> >>>>>>>>> a composite transform of Impulse, which produces dummy bytes
> >>>>>>>>> [1] to trigger subsequent ParDo/ExecutableStage, where
> >>>>>>>>> decoding the actual source happens [2]
> >>>>>>>>>
> >>>>>>>>> There seems to be no role of the SDK harness with regard to
> >>>>>>>>> the TestStream, because the elements are already encoded by
> >>>>>>>>> the submitting SDK. The coders must match nevertheless,
> >>>>>>>>> because you can have Events of type
> >>>>>>>>> KV<KV<WindowedValue<Integer, Object>>> and what will and what
> >>>>>>>>> will not get length-prefixed depends on which parts exactly
> >>>>>>>>> are "known" (model) coders and which are not. Encoding the
> >>>>>>>>> whole value as single byte array will not work for the
> >>>>>>>>> consuming SDK harness, which will see that there should be
> >>>>>>>>> nested KvCoders instead.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> I don’t think I fully understand what you say here. TestStream
> >>>>>>>>> is currently a primitive transform, therefore there is no role
> >>>>>>>>> of SDK harness. This is what the proposal to change, to make
> >>>>>>>>> TestStream a composite transform with a primitive transform
> >>>>>>>>> and subsequent ParDo to decode to the desired format.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> [1]
> >>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Impulse.java#L39
> >>>>>>>>> [2]
> >>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L149
> >>>>>>>>>
> >>>>>>>>> On Aug 31, 2021, at 3:21 PM, Jan Lukavský <je...@seznam.cz>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>> On 9/1/21 12:13 AM, Ke Wu wrote:
> >>>>>>>>>
> >>>>>>>>> Hi Jan,
> >>>>>>>>>
> >>>>>>>>> Here is my understanding,
> >>>>>>>>>
> >>>>>>>>> Runner is being brought up by job server driver, which is up
> >>>>>>>>> and running before the job submission, i.e. it is job
> >>>>>>>>> agnostic. Therefore, the runner it brought up does not have
> >>>>>>>>> any SDK coder available and artifact staging only happens for
> >>>>>>>>> SDK workers.
> >>>>>>>>>
> >>>>>>>>> You are right that Read and TestStream are sources, however
> >>>>>>>>> the one thing that distinguish them is that Read transform is
> >>>>>>>>> a composite transform and the decoding happens in
> >>>>>>>>> ParDo/ExecutableStage, i.e. on SDK worker.
> >>>>>>>>>
> >>>>>>>>> Read does not have translation in portability, so the
> >>>>>>>>> implementation is that it needs to be primitive transform
> >>>>>>>>> explicitly implemented by the runner. The encoding/decoding
> >>>>>>>>> has to happen in the runner.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> The proposal here is also to make the public facing TestStream
> >>>>>>>>> transform a composite transform instead of primitive now, so
> >>>>>>>>> that the decoding would occur on the SDK worker side where SDK
> >>>>>>>>> coder is available, and the primitive that powers TestStream,
> >>>>>>>>> which will be directly translated by runner to always produce
> >>>>>>>>> raw bytes, and these raw bytes will be decoded on the SDK
> >>>>>>>>> worker side.
> >>>>>>>>>
> >>>>>>>>> There seems to be no role of the SDK harness with regard to
> >>>>>>>>> the TestStream, because the elements are already encoded by
> >>>>>>>>> the submitting SDK. The coders must match nevertheless,
> >>>>>>>>> because you can have Events of type
> >>>>>>>>> KV<KV<WindowedValue<Integer, Object>>> and what will and what
> >>>>>>>>> will not get length-prefixed depends on which parts exactly
> >>>>>>>>> are "known" (model) coders and which are not. Encoding the
> >>>>>>>>> whole value as single byte array will not work for the
> >>>>>>>>> consuming SDK harness, which will see that there should be
> >>>>>>>>> nested KvCoders instead.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Ke
> >>>>>>>>>
> >>>>>>>>> On Aug 31, 2021, at 2:56 PM, Jan Lukavský <je...@seznam.cz>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>> Sorry if I'm missing something obvious, but I don't quite see
> >>>>>>>>> the difference between Read and TestStream regarding the
> >>>>>>>>> discussed issue with coders. Couple of thoughts:
> >>>>>>>>>
> >>>>>>>>>      a) both Read and TestStream are _sources_ - they produce
> >>>>>>>>> elements that are consumed by downstream transforms
> >>>>>>>>>
> >>>>>>>>>      b) the coder of a particular PCollection is defined by
> >>>>>>>>> the Pipeline proto - it is the (client side) SDK that owns the
> >>>>>>>>> Pipeline and that defines all the coders
> >>>>>>>>>
> >>>>>>>>>      c) runners must adhere to these coders, because otherwise
> >>>>>>>>> there is risk of coder mismatch, most probably on edges like
> >>>>>>>>> x-lang transforms or inlined transforms
> >>>>>>>>>
> >>>>>>>>> I tried the approach of encoding the output of Read into byte
> >>>>>>>>> array as well, but that turns out to have the problem that
> >>>>>>>>> once there is a (partially) known coder in play, this does not
> >>>>>>>>> work, because the consuming transform (executable stage)
> >>>>>>>>> expects to see the wire coder - that is not simply byte array,
> >>>>>>>>> because the type of elements might be for instance KV<K, V>,
> >>>>>>>>> where KvCoder is one of ModelCoders. That does not encode
> >>>>>>>>> using LengthPrefixCoder and as such will be incompatible with
> >>>>>>>>> LengthPrefixCoder(ByteArrayCoder). The TestStream needs to
> >>>>>>>>> know the coder of elements, because that defines where exactly
> >>>>>>>>> must or must not be inserted length-prefixing. The logic in
> >>>>>>>>> LengthPrefixUnknownCoders [1] is recursive for ModelCoders.
> >>>>>>>>>
> >>>>>>>>> [1]
> >>>>>>>>> https://github.com/apache/beam/blob/ff70e740a2155592dfcb302ff6303cc19660a268/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
> >>>>>>>>>
> >>>>>>>>> On 8/31/21 11:29 PM, Ke Wu wrote:
> >>>>>>>>>
> >>>>>>>>> Awesome! Thank you Luke and Robert.
> >>>>>>>>>
> >>>>>>>>> Also created https://issues.apache.org/jira/browse/BEAM-12828
> >>>>>>>>> to track unit test conversion. I could take it after I updated
> >>>>>>>>> Samza runner to support TestStream in portable mode.
> >>>>>>>>>
> >>>>>>>>> On Aug 31, 2021, at 2:05 PM, Robert Bradshaw
> >>>>>>>>> <ro...@google.com> wrote:
> >>>>>>>>>
> >>>>>>>>> Created https://issues.apache.org/jira/browse/BEAM-12827 to
> >>>>>>>>> track this.
> >>>>>>>>>
> >>>>>>>>> +1 to converting tests to just use longs for better coverage
> >>>>>>>>> for now.
> >>>>>>>>>
> >>>>>>>>> Also, yes, this is very similar to the issues encountered by
> >>>>>>>>> Reads,
> >>>>>>>>> but the solution is a bit simpler as there's no need for the
> >>>>>>>>> TestStream primitive to interact with the decoded version of the
> >>>>>>>>> elements (unlike Reads, where the sources often give elements in
> >>>>>>>>> un-encoded form) and no user code to run.
> >>>>>>>>>
> >>>>>>>>> - Robert
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Tue, Aug 31, 2021 at 11:00 AM Jan Lukavský
> >>>>>>>>> <je...@seznam.cz> wrote:
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> This looks (and likely has the same cause) similar to what I
> >>>>>>>>> have experienced when making primitive Read supported by
> >>>>>>>>> Flink. The final solution would be to make SDK coders known to
> >>>>>>>>> the runner of the same SDK (already present in various
> >>>>>>>>> different threads). But until then, the solution seems to be
> >>>>>>>>> something like [1]. The root cause is that the executable
> >>>>>>>>> stage expects its input to be encoded by the SDK harness, and
> >>>>>>>>> that part is missing when the transform is inlined (like Read
> >>>>>>>>> in my case, or TestStream in your case). The intoWireTypes
> >>>>>>>>> method simulates precisely this part - it encodes the
> >>>>>>>>> PCollection via coder defined in the SDK harness and then
> >>>>>>>>> decodes it by coder defined by the runner (which match on
> >>>>>>>>> binary level, but produce different types).
> >>>>>>>>>
> >>>>>>>>> Jan
> >>>>>>>>>
> >>>>>>>>> [1]
> >>>>>>>>> https://github.com/apache/beam/blob/dd7945f9f259a2989f9396d1d7a8dcb122711a52/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java#L657
> >>>>>>>>>
> >>>>>>>>> On 8/31/21 7:27 PM, Luke Cwik wrote:
> >>>>>>>>>
> >>>>>>>>> I originally wasn't for making it a composite because it
> >>>>>>>>> changes the "graph" structure but the more I thought about it
> >>>>>>>>> the more I like it.
> >>>>>>>>>
> >>>>>>>>> On Tue, Aug 31, 2021 at 10:06 AM Robert Bradshaw
> >>>>>>>>> <ro...@google.com> wrote:
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Tue, Aug 31, 2021 at 9:18 AM Luke Cwik <lc...@google.com>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Mon, Aug 30, 2021 at 7:07 PM Ke Wu <ke...@gmail.com> wrote:
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Hello everyone,
> >>>>>>>>>
> >>>>>>>>> This is Ke. I am working on enable TestStream support for
> >>>>>>>>> Samza Runner in portable mode and discovers something unexpected.
> >>>>>>>>>
> >>>>>>>>> In my implementation for Samza Runner, couple of tests are
> >>>>>>>>> failing with errors like
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> java.lang.ClassCastException: java.lang.Integer cannot be cast
> >>>>>>>>> to [B
> >>>>>>>>>
> >>>>>>>>> I noticed these tests have the same symptom on Flink Runner as
> >>>>>>>>> well, which are currently excluded:
> >>>>>>>>>
> >>>>>>>>> https://issues.apache.org/jira/browse/BEAM-12048
> >>>>>>>>> https://issues.apache.org/jira/browse/BEAM-12050
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> After some more digging, I realized that it is because the
> >>>>>>>>> combination of following facts:
> >>>>>>>>>
> >>>>>>>>> TestStream is a primitive transform, therefore, Runners are
> >>>>>>>>> supposed to translate directly, the most intuitive
> >>>>>>>>> implementation for each runner to do is to parse the payload
> >>>>>>>>> to decode TestStream.Event [1] on the Runner process to be
> >>>>>>>>> handed over to subsequent stages.
> >>>>>>>>> When TestStream used with Integers, i.e. VarIntCoder to
> >>>>>>>>> initialize, since VarIntCoder is NOT a registered ModelCoder
> >>>>>>>>> [2], it will be treated as custom coder during conversion to
> >>>>>>>>> protobuf pipeline [3] and will be replaced with byte array
> >>>>>>>>> coder [4] when runner sends data to SDK worker.
> >>>>>>>>> Therefore an error occurs because the decoded TestStream.Event
> >>>>>>>>> has Integer as its value but the remote input receiver is
> >>>>>>>>> expecting byte array, causing java.lang.ClassCastException:
> >>>>>>>>> java.lang.Integer cannot be cast to [B
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> In addition, I tried to update all these failed tests to use
> >>>>>>>>> Long instead of Integer, and all tests will pass since
> >>>>>>>>> VarLongCoder is a known coder. I do understand that runner
> >>>>>>>>> process does not have user artifacts staged so it can only use
> >>>>>>>>> coders in  beam model when communicating with SDK worker process.
> >>>>>>>>>
> >>>>>>>>> Couple of questions on this:
> >>>>>>>>>
> >>>>>>>>> 1. Is it expected that VarIntegerCoder is not a known coder?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Yes since no one has worked to make it a well known coder.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> The notion of "integer" vs. "long" is also language-specific
> >>>>>>>>> detail as
> >>>>>>>>> well, so not sure it makes sense as a well-known coder.
> >>>>>>>>>
> >>>>>>>>> It can be made a well known coder and this would solve the
> >>>>>>>>> immediate problem but not the long term issue of portable
> >>>>>>>>> TestStream not supporting arbitrary types.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> +1. Rather than making coder a property of TestStream, I would
> >>>>>>>>> be in
> >>>>>>>>> favor of the TestStream primitive always producing bytes
> >>>>>>>>> (basically,
> >>>>>>>>> by definition), and providing a composite that consists of this
> >>>>>>>>> followed by a decoding to give us a typed TestStream.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 2. Is TestStream always supposed to be translated the payload
> >>>>>>>>> as raw bytes in order that runner process can always send it
> >>>>>>>>> to SDK worker with the default byte array coder and asks SDK
> >>>>>>>>> worker to decode accordingly?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Having the runner treat it always as bytes and not T is likely
> >>>>>>>>> the best solution but isn't necessary.
> >>>>>>>>>
> >>>>>>>>> 3. If Yes to 2), then does it mean, TestStream needs to be
> >>>>>>>>> translated in a completely different way in portable mode from
> >>>>>>>>> classic mode since in classic mode, translator can directly
> >>>>>>>>> translates the payload to its final format.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> There are a few ways to fix the current implementation to work
> >>>>>>>>> for all types. One way would be if we required the
> >>>>>>>>> encoded_element to be the "nested" encoding and then ensured
> >>>>>>>>> that the runner uses a WindowedValue<ByteArrayCoder in outer
> >>>>>>>>> context> and the SDK used WindowedValue<T> (note that this
> >>>>>>>>> isn't WindowedValue<LengthPrefix<T>>) for the wire coders.
> >>>>>>>>> This is quite annoying cause the runner inserts length
> >>>>>>>>> prefixing in a lot of places (effectively every time it sees
> >>>>>>>>> an unknown type) so we would need to special case this and
> >>>>>>>>> propagate this correction through any runner native transforms
> >>>>>>>>> (e.g. GBK) until the SDK consumes it.
> >>>>>>>>>
> >>>>>>>>> Another way would be to ensure that the SDK always uses
> >>>>>>>>> LengthPrefix<T> as the PCollection encoding and the
> >>>>>>>>> encoded_element format. This would mean that the runner can
> >>>>>>>>> translate it to a T if it so chooses and won't have the
> >>>>>>>>> annoying special case propagation logic. This leaks the length
> >>>>>>>>> prefixing into the SDK at graph construction time which is not
> >>>>>>>>> what it was meant for.
> >>>>>>>>>
> >>>>>>>>> Swapping to use an existing well known type is by far the
> >>>>>>>>> easiest approach as you had discovered and won't impact the
> >>>>>>>>> correctness of the tests.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Ke
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> [1]
> >>>>>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java#L52
> >>>>>>>>> [2]
> >>>>>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java#L65
> >>>>>>>>> [3]
> >>>>>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L99
> >>>>>>>>> [4]
> >>>>>>>>> https://github.com/apache/beam/blob/master/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L93
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
Hi,

I had some more time thinking about this and I'll try to recap that. 
First some invariants:

  a) each PCollection<T> has actually two coders - an _SDK coder_ and a 
_runner coder_. These coders have the property, that each one can 
_decode_ what the other encoded, but the opposite is not true, the 
coders cannot _encode_ what the other _decoded_ (in general).

  b) when is a PCollection<T> computed inside an environment, the 
elements are encoded using SDK coder on the side of SDK-harness and 
decoded using runner coder after receiving in the runner

  c) under specific circumstances, the encode-decode step can be 
optimized out, that is the case where the SDK coder and all its 
subcoders are all well-known to the runner (in the present, that means 
that all the parts present in the model coders set). The reason for that 
is that in this specific situation runner_decode(sdk_encode(X)) = X. 
This property is essential.

  d) from b) immediately follows, that when a PTransform does not run in 
an environment (and this might be due to the transform being runner 
native, inlined, source (e.g. Impulse or TestStream)) the elements have 
to be encoded by SDK coder, immediately following decode by runner 
coder. That (surprisingly) applies even to situations when runner is 
implemented using different language than the client SDK, because it 
implies that the type of produced elements must be one of types encoded 
using model coders (well-known to the runner, otherwise the SDK will not 
be able to consume it). But - due to property c) - this means that this 
encode-decode step can be optimized out. This does not mean that it is 
not (logically) present, though. This is exactly the case of native 
Impulse transform.

Now, from that we can conclude that on the boundary between executable 
stages, or between runner (inlined) transform and executable stage, each 
PCollection has to be encoded using SDK coder and immediately decoded by 
runner coder, *unless this can be optimized out* by property c).

This gives us two options where to implement this encode/decode step:

  1) completely inside runner with the possibility to optimize the 
encode/decode step by identity under right circumstances

  2) partly in the runner and partly in the SDK - that is we encode 
elements of PCollection using SDK coder into bytes, pass those to the 
SDK harness and apply a custom decode step there. This works because SDK 
coder encoded elements are in byte[], and that is well-known coder type. 
We again only leverage property c) and optimize the SDK coder encode, 
runner decode step out.

The option 2) is exactly the proposal of TestStream producing byte[] and 
decoding inside SDK-harness, the TestStream is actually inlined 
transform, the elements are produced directly in runner (the SDK coder 
is not known to the runner, but that does not matter, because the 
elements are already encoded by client).

 From the above it seems to me, that option 1) should be preferred, because:

  i) it is generic, applicable to all inlined transforms, any sources

  ii) it is consistent with how things logically work underneath

  iii) it offers better room for optimization - option 2) might result 
in cases when the elements are passed from the runner to the SDK-harness 
only for the sake of the decoding from SDK coder and immediately 
encoding back using SDK-coder and returned back to the runner. This 
would be the case when TestStream would be directly consumed by inlined 
(or external) transform.

  Jan

On 9/1/21 9:41 AM, Jan Lukavský wrote:
> On 9/1/21 9:15 AM, Robert Bradshaw wrote:
>
>> On Tue, Aug 31, 2021 at 11:48 PM Jan Lukavský <je...@seznam.cz> wrote:
>>> Sorry, I needed some time to let that sink in. :-)
>>>
>>> I think I understand why (and how) this will work for TestStream, still
>>> have a couple of notes, though:
>>>
>>>    a) the problem of type compatibility arises with the primitive 
>>> Read as
>>> well, though we can solve it with different expansion for TestStream,
>>> that solution is not applicable to Read, because it has different 
>>> contract
>>>
>>>    b) the same exact problem problem will arise every time we inline 
>>> any
>>> transform that would otherwise be run in an environment
>>>
>>>    c) extracting bytes from TestStream actually bypasses "model coders"
>>> on the side of runner, should we do that in other cases as well?
>>>
>>> The TestStream problem is a special case, I think a more generic
>>> solution would be better.
>> Yep, TestStream is special and simpler. I'm not sure something generic
>> is always possible.
>>
>>> Two questions:
>>>
>>>    1) Could we create a mapping in runners-core-construction-java that
>>> would take Pipeline proto, and PCollectionId and create a mapping
>>> function from "sdk coder space" to "runner coder space"? That could be
>>> optimized to identity if the coder of the PCollection consists of model
>>> coders only. In that case the sdk coder and runner coder are identical,
>>> producing the same types. This mapping function could be reused by both
>>> portable TestStream, inlined primitive Read and any other future 
>>> inlined
>>> transform.
>> This could help with the specific case of Java transforms being
>> inlined on a Java-based runner. I'd rather avoid Java-specific
>> solutions if possible. (E.g. the ULR is in Python, and Dataflow Runner
>> v2 is in C++.)
> The definition is generic, language agnostic. It can be rephrased as 
> "if the SDK of a runner matches SDK of a transform, the transform can 
> be inlined, and in order to be type-compatible coder mapping between 
> SDK coder space and runner coder space has to be applied". Yes, when 
> we implement this in "runners-core-construction-java", the "runner 
> SDK" and "transform SDK" equals to Java SDK on both parts. But that is 
> implied by the implementation, not the definition. For Go or Python we 
> can implement exactly the same.
>>
>> I'd say if a runner decides to truely inline a transform with an
>> alternative implementation than calling the SDK, it would only do so
>> in cases where it understands the input and output types, if any.
>> (With Read we have a case where the output type is not fixed, which
>> complicates things.) Generally I would probably structure most
>> "inlined" operations as one that happens to execute in an in-process
>> environment rather than an external one, which would mean that
>> "inlined" ones would not have to be handled specially, but could just
>> fall out due to the existing length prefixing logic.
>>
>> Of course just making SDFs work well on all runners would be the best
>> long term solution for Read, and avoid all these issues, but we may
>> need hacks in the meantime.
> I think the problem is broader than sources, so while it is true, that 
> SDF would help, it is not the solution.
>>
>>>    2) Why does runner need to understand the types it processes and 
>>> does
>>> not work with raw bytes all the times, including model coders, the same
>>> way as is the proposed solution for TestStream, but for all transforms?
>>> The first step to every executable stage would then be to decode the
>>> input from raw bytes and only then process it. What is the benefit of
>>> runner understanding _some_ of the coders? It would be required for a
>>> runner to understand coders for root transforms (Impulse? any other?),
>>> but are there any other places where this is necessary?
>> A runner needs to understand the KV coder to execute GroupByKey (and
>> the result is a KV<K, Iterable<V>>, so IterableCoder is a requirement
>> as well). WindowedCoders are likewise needed so the runner can
>> deconstruct the timestamps, etc. Timers are sent via TimerCoders. A
>> runner also needs a way to deal with unknown coders, hence
>> LengthPrefixCoder. Asside from that, the runner can just deal in raw
>> bytes (though sometimes there are efficiency gains to understanding
>> more coders, e.g. wrapping tiny integers in LengthPrefix could be
>> rather wasteful).
>>
>> Well known coders, such as RowCoder, are of course useful for
>> communication between SDKs of different languages, even if the runner
>> doesn't care about them.
> +1, understood and agree.
>>
>>> On 9/1/21 1:37 AM, Robert Bradshaw wrote:
>>>> On Tue, Aug 31, 2021 at 4:31 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>> If I can depict that:
>>>>>
>>>>> TestStream0 -> TestStream1 (DoFn), that is
>>>>>
>>>>> TestStream0 -> TestStream1 (executable stage) -> consumer 2 
>>>>> (executable
>>>>> stage)
>>>> Yes. Expanding it out more,
>>>>
>>>> TestStream0 -> pc1 -> TestStream1 -> pc2 -> Consumer2 -> ...
>>>>
>>>> Where pc1 is a PCollection<bytes> and pc2 is a PCollection<T>.
>>>>
>>>>> In this scenario TestStream0 can produce byte[], and TestStream1 can
>>>>> consume them, the problem is that if the coder of TestStream0
>>>> I assume you mean TestStream1, as TestStream0 has no coder.
>>>>
>>>>> is some
>>>>> (recursively) known model coder (e.g. KvCoder), then consumer 2 
>>>>> will not
>>>>> be able to decode that data.
>>>> We know that, by construction, TestStream1 and Consumer2 both are
>>>> executed in an environment that understands pc2's Coder<T>. The
>>>> difference here is that the runner need not understand Coder<T> as it
>>>> could inject length prefixing on pc2 if necessary.
>>>>
>>>>
>>>>> On 9/1/21 1:24 AM, Robert Bradshaw wrote:
>>>>>> On Tue, Aug 31, 2021 at 4:12 PM Jan Lukavský <je...@seznam.cz> 
>>>>>> wrote:
>>>>>>>> Fortunately, there's no value in having the primitive TestStream
>>>>>>>> produce elements of arbitrary type (vs. sources, where the read is
>>>>>>>> inherently typed depending on the source and particular library 
>>>>>>>> used
>>>>>>>> for reading it).
>>>>>>> I disagree with this one - the consumer(s) of elements from 
>>>>>>> TestStream
>>>>>>> are (should be treated as) unknown to the runner, might be inlined,
>>>>>>> might be x-lang and whether or not will pass the boundary of 
>>>>>>> executable
>>>>>>> stage, all these questions are important. The coder (and if it 
>>>>>>> is model
>>>>>>> coder or not, and what are the sub-coders) must be precisely 
>>>>>>> known to
>>>>>>> all parties that cooperate on the computation, because otherwise 
>>>>>>> these
>>>>>>> parties might not agree on the binary representation.
>>>>>> What I'm saying is that there's little value in a primitive
>>>>>> TestStream that can produce all sorts of T, vs. a composite 
>>>>>> TestStream
>>>>>> consisting of a TestStream0 primitive producing only bytes, followed
>>>>>> by a TestStream1 that decodes those bytes into T. This sidesteps all
>>>>>> issues of coder compatibility, as bytes is well-known and 
>>>>>> TestStream1
>>>>>> is just a DoFn.
>>>>>>
>>>>>>> On 9/1/21 12:59 AM, Robert Bradshaw wrote:
>>>>>>>> Due to issues with Flink executing Reads as SDFs, there is an
>>>>>>>> alternative mode where Read is treated as a primitive and executed
>>>>>>>> directly in the runner. This, of course, requires that the 
>>>>>>>> Read's UDF
>>>>>>>> be implemented in the same language as the Runner 
>>>>>>>> (specifically, in
>>>>>>>> this case, Java) and their interpretation of what Coders should or
>>>>>>>> should not be wrapped agree (lest other complications arise).
>>>>>>>>
>>>>>>>> It's true one can view TestStream as a source, but it's a bit more
>>>>>>>> than that as it has the ability to wait for quiessence before 
>>>>>>>> emitting
>>>>>>>> more elements/watermark updates to give stronger guarantees on
>>>>>>>> determinism, which generally requires deeper runner 
>>>>>>>> integration. In
>>>>>>>> addition, TestStream is not associated with a UDF or specific 
>>>>>>>> SDK the
>>>>>>>> way Sources are which is where the problem arises: TestStream 
>>>>>>>> gives
>>>>>>>> the encoded form of the elements but the runner may need to 
>>>>>>>> produce a
>>>>>>>> different encoded form of the elements (say, one with length
>>>>>>>> prefixing) which is not possible without knowing the Coder.
>>>>>>>> Fortunately, there's no value in having the primitive TestStream
>>>>>>>> produce elements of arbitrary type (vs. sources, where the read is
>>>>>>>> inherently typed depending on the source and particular library 
>>>>>>>> used
>>>>>>>> for reading it).
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Aug 31, 2021 at 3:41 PM Ke Wu <ke...@gmail.com> wrote:
>>>>>>>>> Read does not have translation in portability, so the 
>>>>>>>>> implementation is that it needs to be primitive transform 
>>>>>>>>> explicitly implemented by the runner. The encoding/decoding 
>>>>>>>>> has to happen in the runner.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Could you help me understand this a bit more? IIRC, Read is 
>>>>>>>>> NOT being translated in portable mode exactly means it is a 
>>>>>>>>> composite transform instead of primitive because all primitive 
>>>>>>>>> transforms are required to be translated. In addition, Read is 
>>>>>>>>> a composite transform of Impulse, which produces dummy bytes 
>>>>>>>>> [1] to trigger subsequent ParDo/ExecutableStage, where 
>>>>>>>>> decoding the actual source happens [2]
>>>>>>>>>
>>>>>>>>> There seems to be no role of the SDK harness with regard to 
>>>>>>>>> the TestStream, because the elements are already encoded by 
>>>>>>>>> the submitting SDK. The coders must match nevertheless, 
>>>>>>>>> because you can have Events of type 
>>>>>>>>> KV<KV<WindowedValue<Integer, Object>>> and what will and what 
>>>>>>>>> will not get length-prefixed depends on which parts exactly 
>>>>>>>>> are "known" (model) coders and which are not. Encoding the 
>>>>>>>>> whole value as single byte array will not work for the 
>>>>>>>>> consuming SDK harness, which will see that there should be 
>>>>>>>>> nested KvCoders instead.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> I don’t think I fully understand what you say here. TestStream 
>>>>>>>>> is currently a primitive transform, therefore there is no role 
>>>>>>>>> of SDK harness. This is what the proposal to change, to make 
>>>>>>>>> TestStream a composite transform with a primitive transform 
>>>>>>>>> and subsequent ParDo to decode to the desired format.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> [1] 
>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Impulse.java#L39
>>>>>>>>> [2] 
>>>>>>>>> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L149
>>>>>>>>>
>>>>>>>>> On Aug 31, 2021, at 3:21 PM, Jan Lukavský <je...@seznam.cz> 
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> On 9/1/21 12:13 AM, Ke Wu wrote:
>>>>>>>>>
>>>>>>>>> Hi Jan,
>>>>>>>>>
>>>>>>>>> Here is my understanding,
>>>>>>>>>
>>>>>>>>> Runner is being brought up by job server driver, which is up 
>>>>>>>>> and running before the job submission, i.e. it is job 
>>>>>>>>> agnostic. Therefore, the runner it brought up does not have 
>>>>>>>>> any SDK coder available and artifact staging only happens for 
>>>>>>>>> SDK workers.
>>>>>>>>>
>>>>>>>>> You are right that Read and TestStream are sources, however 
>>>>>>>>> the one thing that distinguish them is that Read transform is 
>>>>>>>>> a composite transform and the decoding happens in 
>>>>>>>>> ParDo/ExecutableStage, i.e. on SDK worker.
>>>>>>>>>
>>>>>>>>> Read does not have translation in portability, so the 
>>>>>>>>> implementation is that it needs to be primitive transform 
>>>>>>>>> explicitly implemented by the runner. The encoding/decoding 
>>>>>>>>> has to happen in the runner.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> The proposal here is also to make the public facing TestStream 
>>>>>>>>> transform a composite transform instead of primitive now, so 
>>>>>>>>> that the decoding would occur on the SDK worker side where SDK 
>>>>>>>>> coder is available, and the primitive that powers TestStream, 
>>>>>>>>> which will be directly translated by runner to always produce 
>>>>>>>>> raw bytes, and these raw bytes will be decoded on the SDK 
>>>>>>>>> worker side.
>>>>>>>>>
>>>>>>>>> There seems to be no role of the SDK harness with regard to 
>>>>>>>>> the TestStream, because the elements are already encoded by 
>>>>>>>>> the submitting SDK. The coders must match nevertheless, 
>>>>>>>>> because you can have Events of type 
>>>>>>>>> KV<KV<WindowedValue<Integer, Object>>> and what will and what 
>>>>>>>>> will not get length-prefixed depends on which parts exactly 
>>>>>>>>> are "known" (model) coders and which are not. Encoding the 
>>>>>>>>> whole value as single byte array will not work for the 
>>>>>>>>> consuming SDK harness, which will see that there should be 
>>>>>>>>> nested KvCoders instead.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Ke
>>>>>>>>>
>>>>>>>>> On Aug 31, 2021, at 2:56 PM, Jan Lukavský <je...@seznam.cz> 
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Sorry if I'm missing something obvious, but I don't quite see 
>>>>>>>>> the difference between Read and TestStream regarding the 
>>>>>>>>> discussed issue with coders. Couple of thoughts:
>>>>>>>>>
>>>>>>>>>      a) both Read and TestStream are _sources_ - they produce 
>>>>>>>>> elements that are consumed by downstream transforms
>>>>>>>>>
>>>>>>>>>      b) the coder of a particular PCollection is defined by 
>>>>>>>>> the Pipeline proto - it is the (client side) SDK that owns the 
>>>>>>>>> Pipeline and that defines all the coders
>>>>>>>>>
>>>>>>>>>      c) runners must adhere to these coders, because otherwise 
>>>>>>>>> there is risk of coder mismatch, most probably on edges like 
>>>>>>>>> x-lang transforms or inlined transforms
>>>>>>>>>
>>>>>>>>> I tried the approach of encoding the output of Read into byte 
>>>>>>>>> array as well, but that turns out to have the problem that 
>>>>>>>>> once there is a (partially) known coder in play, this does not 
>>>>>>>>> work, because the consuming transform (executable stage) 
>>>>>>>>> expects to see the wire coder - that is not simply byte array, 
>>>>>>>>> because the type of elements might be for instance KV<K, V>, 
>>>>>>>>> where KvCoder is one of ModelCoders. That does not encode 
>>>>>>>>> using LengthPrefixCoder and as such will be incompatible with 
>>>>>>>>> LengthPrefixCoder(ByteArrayCoder). The TestStream needs to 
>>>>>>>>> know the coder of elements, because that defines where exactly 
>>>>>>>>> must or must not be inserted length-prefixing. The logic in 
>>>>>>>>> LengthPrefixUnknownCoders [1] is recursive for ModelCoders.
>>>>>>>>>
>>>>>>>>> [1] 
>>>>>>>>> https://github.com/apache/beam/blob/ff70e740a2155592dfcb302ff6303cc19660a268/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>>>>
>>>>>>>>> On 8/31/21 11:29 PM, Ke Wu wrote:
>>>>>>>>>
>>>>>>>>> Awesome! Thank you Luke and Robert.
>>>>>>>>>
>>>>>>>>> Also created https://issues.apache.org/jira/browse/BEAM-12828 
>>>>>>>>> to track unit test conversion. I could take it after I updated 
>>>>>>>>> Samza runner to support TestStream in portable mode.
>>>>>>>>>
>>>>>>>>> On Aug 31, 2021, at 2:05 PM, Robert Bradshaw 
>>>>>>>>> <ro...@google.com> wrote:
>>>>>>>>>
>>>>>>>>> Created https://issues.apache.org/jira/browse/BEAM-12827 to 
>>>>>>>>> track this.
>>>>>>>>>
>>>>>>>>> +1 to converting tests to just use longs for better coverage 
>>>>>>>>> for now.
>>>>>>>>>
>>>>>>>>> Also, yes, this is very similar to the issues encountered by 
>>>>>>>>> Reads,
>>>>>>>>> but the solution is a bit simpler as there's no need for the
>>>>>>>>> TestStream primitive to interact with the decoded version of the
>>>>>>>>> elements (unlike Reads, where the sources often give elements in
>>>>>>>>> un-encoded form) and no user code to run.
>>>>>>>>>
>>>>>>>>> - Robert
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Tue, Aug 31, 2021 at 11:00 AM Jan Lukavský 
>>>>>>>>> <je...@seznam.cz> wrote:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> This looks (and likely has the same cause) similar to what I 
>>>>>>>>> have experienced when making primitive Read supported by 
>>>>>>>>> Flink. The final solution would be to make SDK coders known to 
>>>>>>>>> the runner of the same SDK (already present in various 
>>>>>>>>> different threads). But until then, the solution seems to be 
>>>>>>>>> something like [1]. The root cause is that the executable 
>>>>>>>>> stage expects its input to be encoded by the SDK harness, and 
>>>>>>>>> that part is missing when the transform is inlined (like Read 
>>>>>>>>> in my case, or TestStream in your case). The intoWireTypes 
>>>>>>>>> method simulates precisely this part - it encodes the 
>>>>>>>>> PCollection via coder defined in the SDK harness and then 
>>>>>>>>> decodes it by coder defined by the runner (which match on 
>>>>>>>>> binary level, but produce different types).
>>>>>>>>>
>>>>>>>>> Jan
>>>>>>>>>
>>>>>>>>> [1] 
>>>>>>>>> https://github.com/apache/beam/blob/dd7945f9f259a2989f9396d1d7a8dcb122711a52/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java#L657
>>>>>>>>>
>>>>>>>>> On 8/31/21 7:27 PM, Luke Cwik wrote:
>>>>>>>>>
>>>>>>>>> I originally wasn't for making it a composite because it 
>>>>>>>>> changes the "graph" structure but the more I thought about it 
>>>>>>>>> the more I like it.
>>>>>>>>>
>>>>>>>>> On Tue, Aug 31, 2021 at 10:06 AM Robert Bradshaw 
>>>>>>>>> <ro...@google.com> wrote:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Tue, Aug 31, 2021 at 9:18 AM Luke Cwik <lc...@google.com> 
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Mon, Aug 30, 2021 at 7:07 PM Ke Wu <ke...@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Hello everyone,
>>>>>>>>>
>>>>>>>>> This is Ke. I am working on enable TestStream support for 
>>>>>>>>> Samza Runner in portable mode and discovers something unexpected.
>>>>>>>>>
>>>>>>>>> In my implementation for Samza Runner, couple of tests are 
>>>>>>>>> failing with errors like
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> java.lang.ClassCastException: java.lang.Integer cannot be cast 
>>>>>>>>> to [B
>>>>>>>>>
>>>>>>>>> I noticed these tests have the same symptom on Flink Runner as 
>>>>>>>>> well, which are currently excluded:
>>>>>>>>>
>>>>>>>>> https://issues.apache.org/jira/browse/BEAM-12048
>>>>>>>>> https://issues.apache.org/jira/browse/BEAM-12050
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> After some more digging, I realized that it is because the 
>>>>>>>>> combination of following facts:
>>>>>>>>>
>>>>>>>>> TestStream is a primitive transform, therefore, Runners are 
>>>>>>>>> supposed to translate directly, the most intuitive 
>>>>>>>>> implementation for each runner to do is to parse the payload 
>>>>>>>>> to decode TestStream.Event [1] on the Runner process to be 
>>>>>>>>> handed over to subsequent stages.
>>>>>>>>> When TestStream used with Integers, i.e. VarIntCoder to 
>>>>>>>>> initialize, since VarIntCoder is NOT a registered ModelCoder 
>>>>>>>>> [2], it will be treated as custom coder during conversion to 
>>>>>>>>> protobuf pipeline [3] and will be replaced with byte array 
>>>>>>>>> coder [4] when runner sends data to SDK worker.
>>>>>>>>> Therefore an error occurs because the decoded TestStream.Event 
>>>>>>>>> has Integer as its value but the remote input receiver is 
>>>>>>>>> expecting byte array, causing java.lang.ClassCastException: 
>>>>>>>>> java.lang.Integer cannot be cast to [B
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> In addition, I tried to update all these failed tests to use 
>>>>>>>>> Long instead of Integer, and all tests will pass since 
>>>>>>>>> VarLongCoder is a known coder. I do understand that runner 
>>>>>>>>> process does not have user artifacts staged so it can only use 
>>>>>>>>> coders in  beam model when communicating with SDK worker process.
>>>>>>>>>
>>>>>>>>> Couple of questions on this:
>>>>>>>>>
>>>>>>>>> 1. Is it expected that VarIntegerCoder is not a known coder?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Yes since no one has worked to make it a well known coder.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> The notion of "integer" vs. "long" is also language-specific 
>>>>>>>>> detail as
>>>>>>>>> well, so not sure it makes sense as a well-known coder.
>>>>>>>>>
>>>>>>>>> It can be made a well known coder and this would solve the 
>>>>>>>>> immediate problem but not the long term issue of portable 
>>>>>>>>> TestStream not supporting arbitrary types.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> +1. Rather than making coder a property of TestStream, I would 
>>>>>>>>> be in
>>>>>>>>> favor of the TestStream primitive always producing bytes 
>>>>>>>>> (basically,
>>>>>>>>> by definition), and providing a composite that consists of this
>>>>>>>>> followed by a decoding to give us a typed TestStream.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 2. Is TestStream always supposed to be translated the payload 
>>>>>>>>> as raw bytes in order that runner process can always send it 
>>>>>>>>> to SDK worker with the default byte array coder and asks SDK 
>>>>>>>>> worker to decode accordingly?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Having the runner treat it always as bytes and not T is likely 
>>>>>>>>> the best solution but isn't necessary.
>>>>>>>>>
>>>>>>>>> 3. If Yes to 2), then does it mean, TestStream needs to be 
>>>>>>>>> translated in a completely different way in portable mode from 
>>>>>>>>> classic mode since in classic mode, translator can directly 
>>>>>>>>> translates the payload to its final format.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> There are a few ways to fix the current implementation to work 
>>>>>>>>> for all types. One way would be if we required the 
>>>>>>>>> encoded_element to be the "nested" encoding and then ensured 
>>>>>>>>> that the runner uses a WindowedValue<ByteArrayCoder in outer 
>>>>>>>>> context> and the SDK used WindowedValue<T> (note that this 
>>>>>>>>> isn't WindowedValue<LengthPrefix<T>>) for the wire coders. 
>>>>>>>>> This is quite annoying cause the runner inserts length 
>>>>>>>>> prefixing in a lot of places (effectively every time it sees 
>>>>>>>>> an unknown type) so we would need to special case this and 
>>>>>>>>> propagate this correction through any runner native transforms 
>>>>>>>>> (e.g. GBK) until the SDK consumes it.
>>>>>>>>>
>>>>>>>>> Another way would be to ensure that the SDK always uses 
>>>>>>>>> LengthPrefix<T> as the PCollection encoding and the 
>>>>>>>>> encoded_element format. This would mean that the runner can 
>>>>>>>>> translate it to a T if it so chooses and won't have the 
>>>>>>>>> annoying special case propagation logic. This leaks the length 
>>>>>>>>> prefixing into the SDK at graph construction time which is not 
>>>>>>>>> what it was meant for.
>>>>>>>>>
>>>>>>>>> Swapping to use an existing well known type is by far the 
>>>>>>>>> easiest approach as you had discovered and won't impact the 
>>>>>>>>> correctness of the tests.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Ke
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> [1] 
>>>>>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java#L52
>>>>>>>>> [2] 
>>>>>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java#L65
>>>>>>>>> [3] 
>>>>>>>>> https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L99
>>>>>>>>> [4] 
>>>>>>>>> https://github.com/apache/beam/blob/master/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L93
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Jan Lukavský <je...@seznam.cz>.
On 9/1/21 9:15 AM, Robert Bradshaw wrote:

> On Tue, Aug 31, 2021 at 11:48 PM Jan Lukavský <je...@seznam.cz> wrote:
>> Sorry, I needed some time to let that sink in. :-)
>>
>> I think I understand why (and how) this will work for TestStream, still
>> have a couple of notes, though:
>>
>>    a) the problem of type compatibility arises with the primitive Read as
>> well, though we can solve it with different expansion for TestStream,
>> that solution is not applicable to Read, because it has different contract
>>
>>    b) the same exact problem problem will arise every time we inline any
>> transform that would otherwise be run in an environment
>>
>>    c) extracting bytes from TestStream actually bypasses "model coders"
>> on the side of runner, should we do that in other cases as well?
>>
>> The TestStream problem is a special case, I think a more generic
>> solution would be better.
> Yep, TestStream is special and simpler. I'm not sure something generic
> is always possible.
>
>> Two questions:
>>
>>    1) Could we create a mapping in runners-core-construction-java that
>> would take Pipeline proto, and PCollectionId and create a mapping
>> function from "sdk coder space" to "runner coder space"? That could be
>> optimized to identity if the coder of the PCollection consists of model
>> coders only. In that case the sdk coder and runner coder are identical,
>> producing the same types. This mapping function could be reused by both
>> portable TestStream, inlined primitive Read and any other future inlined
>> transform.
> This could help with the specific case of Java transforms being
> inlined on a Java-based runner. I'd rather avoid Java-specific
> solutions if possible. (E.g. the ULR is in Python, and Dataflow Runner
> v2 is in C++.)
The definition is generic, language agnostic. It can be rephrased as "if 
the SDK of a runner matches SDK of a transform, the transform can be 
inlined, and in order to be type-compatible coder mapping between SDK 
coder space and runner coder space has to be applied". Yes, when we 
implement this in "runners-core-construction-java", the "runner SDK" and 
"transform SDK" equals to Java SDK on both parts. But that is implied by 
the implementation, not the definition. For Go or Python we can 
implement exactly the same.
>
> I'd say if a runner decides to truely inline a transform with an
> alternative implementation than calling the SDK, it would only do so
> in cases where it understands the input and output types, if any.
> (With Read we have a case where the output type is not fixed, which
> complicates things.) Generally I would probably structure most
> "inlined" operations as one that happens to execute in an in-process
> environment rather than an external one, which would mean that
> "inlined" ones would not have to be handled specially, but could just
> fall out due to the existing length prefixing logic.
>
> Of course just making SDFs work well on all runners would be the best
> long term solution for Read, and avoid all these issues, but we may
> need hacks in the meantime.
I think the problem is broader than sources, so while it is true, that 
SDF would help, it is not the solution.
>
>>    2) Why does runner need to understand the types it processes and does
>> not work with raw bytes all the times, including model coders, the same
>> way as is the proposed solution for TestStream, but for all transforms?
>> The first step to every executable stage would then be to decode the
>> input from raw bytes and only then process it. What is the benefit of
>> runner understanding _some_ of the coders? It would be required for a
>> runner to understand coders for root transforms (Impulse? any other?),
>> but are there any other places where this is necessary?
> A runner needs to understand the KV coder to execute GroupByKey (and
> the result is a KV<K, Iterable<V>>, so IterableCoder is a requirement
> as well). WindowedCoders are likewise needed so the runner can
> deconstruct the timestamps, etc. Timers are sent via TimerCoders. A
> runner also needs a way to deal with unknown coders, hence
> LengthPrefixCoder. Asside from that, the runner can just deal in raw
> bytes (though sometimes there are efficiency gains to understanding
> more coders, e.g. wrapping tiny integers in LengthPrefix could be
> rather wasteful).
>
> Well known coders, such as RowCoder, are of course useful for
> communication between SDKs of different languages, even if the runner
> doesn't care about them.
+1, understood and agree.
>
>> On 9/1/21 1:37 AM, Robert Bradshaw wrote:
>>> On Tue, Aug 31, 2021 at 4:31 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>> If I can depict that:
>>>>
>>>> TestStream0 -> TestStream1 (DoFn), that is
>>>>
>>>> TestStream0 -> TestStream1 (executable stage) -> consumer 2 (executable
>>>> stage)
>>> Yes. Expanding it out more,
>>>
>>> TestStream0 -> pc1 -> TestStream1 -> pc2 -> Consumer2 -> ...
>>>
>>> Where pc1 is a PCollection<bytes> and pc2 is a PCollection<T>.
>>>
>>>> In this scenario TestStream0 can produce byte[], and TestStream1 can
>>>> consume them, the problem is that if the coder of TestStream0
>>> I assume you mean TestStream1, as TestStream0 has no coder.
>>>
>>>> is some
>>>> (recursively) known model coder (e.g. KvCoder), then consumer 2 will not
>>>> be able to decode that data.
>>> We know that, by construction, TestStream1 and Consumer2 both are
>>> executed in an environment that understands pc2's Coder<T>. The
>>> difference here is that the runner need not understand Coder<T> as it
>>> could inject length prefixing on pc2 if necessary.
>>>
>>>
>>>> On 9/1/21 1:24 AM, Robert Bradshaw wrote:
>>>>> On Tue, Aug 31, 2021 at 4:12 PM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>> Fortunately, there's no value in having the primitive TestStream
>>>>>>> produce elements of arbitrary type (vs. sources, where the read is
>>>>>>> inherently typed depending on the source and particular library used
>>>>>>> for reading it).
>>>>>> I disagree with this one - the consumer(s) of elements from TestStream
>>>>>> are (should be treated as) unknown to the runner, might be inlined,
>>>>>> might be x-lang and whether or not will pass the boundary of executable
>>>>>> stage, all these questions are important. The coder (and if it is model
>>>>>> coder or not, and what are the sub-coders) must be precisely known to
>>>>>> all parties that cooperate on the computation, because otherwise these
>>>>>> parties might not agree on the binary representation.
>>>>> What I'm saying is that there's little value in a  primitive
>>>>> TestStream that can produce all sorts of T, vs. a composite TestStream
>>>>> consisting of a TestStream0 primitive producing only bytes, followed
>>>>> by a TestStream1 that decodes those bytes into T. This sidesteps all
>>>>> issues of coder compatibility, as bytes is well-known and TestStream1
>>>>> is just a DoFn.
>>>>>
>>>>>> On 9/1/21 12:59 AM, Robert Bradshaw wrote:
>>>>>>> Due to issues with Flink executing Reads as SDFs, there is an
>>>>>>> alternative mode where Read is treated as a primitive and executed
>>>>>>> directly in the runner. This, of course, requires that the Read's UDF
>>>>>>> be implemented in the same language as the Runner (specifically, in
>>>>>>> this case, Java) and their interpretation of what Coders should or
>>>>>>> should not be wrapped agree (lest other complications arise).
>>>>>>>
>>>>>>> It's true one can view TestStream as a source, but it's a bit more
>>>>>>> than that as it has the ability to wait for quiessence before emitting
>>>>>>> more elements/watermark updates to give stronger guarantees on
>>>>>>> determinism, which generally requires deeper runner integration. In
>>>>>>> addition, TestStream is not associated with a UDF or specific SDK the
>>>>>>> way Sources are which is where the problem arises: TestStream gives
>>>>>>> the encoded form of the elements but the runner may need to produce a
>>>>>>> different encoded form of the elements (say, one with length
>>>>>>> prefixing) which is not possible without knowing the Coder.
>>>>>>> Fortunately, there's no value in having the primitive TestStream
>>>>>>> produce elements of arbitrary type (vs. sources, where the read is
>>>>>>> inherently typed depending on the source and particular library used
>>>>>>> for reading it).
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Aug 31, 2021 at 3:41 PM Ke Wu <ke...@gmail.com> wrote:
>>>>>>>> Read does not have translation in portability, so the implementation is that it needs to be primitive transform explicitly implemented by the runner. The encoding/decoding has to happen in the runner.
>>>>>>>>
>>>>>>>>
>>>>>>>> Could you help me understand this a bit more? IIRC, Read is NOT being translated in portable mode exactly means it is a composite transform instead of primitive because all primitive transforms are required to be translated. In addition, Read is a composite transform of Impulse, which produces dummy bytes [1] to trigger subsequent ParDo/ExecutableStage, where decoding the actual source happens [2]
>>>>>>>>
>>>>>>>> There seems to be no role of the SDK harness with regard to the TestStream, because the elements are already encoded by the submitting SDK. The coders must match nevertheless, because you can have Events of type KV<KV<WindowedValue<Integer, Object>>> and what will and what will not get length-prefixed depends on which parts exactly are "known" (model) coders and which are not. Encoding the whole value as single byte array will not work for the consuming SDK harness, which will see that there should be nested KvCoders instead.
>>>>>>>>
>>>>>>>>
>>>>>>>> I don’t think I fully understand what you say here. TestStream is currently a primitive transform, therefore there is no role of SDK harness. This is what the proposal to change, to make TestStream a composite transform with a primitive transform and subsequent ParDo to decode to the desired format.
>>>>>>>>
>>>>>>>>
>>>>>>>> [1] https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Impulse.java#L39
>>>>>>>> [2] https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L149
>>>>>>>>
>>>>>>>> On Aug 31, 2021, at 3:21 PM, Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>>>
>>>>>>>> On 9/1/21 12:13 AM, Ke Wu wrote:
>>>>>>>>
>>>>>>>> Hi Jan,
>>>>>>>>
>>>>>>>> Here is my understanding,
>>>>>>>>
>>>>>>>> Runner is being brought up by job server driver, which is up and running before the job submission, i.e. it is job agnostic. Therefore, the runner it brought up does not have any SDK coder available and artifact staging only happens for SDK workers.
>>>>>>>>
>>>>>>>> You are right that Read and TestStream are sources, however the one thing that distinguish them is that Read transform is a composite transform and the decoding happens in ParDo/ExecutableStage, i.e. on SDK worker.
>>>>>>>>
>>>>>>>> Read does not have translation in portability, so the implementation is that it needs to be primitive transform explicitly implemented by the runner. The encoding/decoding has to happen in the runner.
>>>>>>>>
>>>>>>>>
>>>>>>>> The proposal here is also to make the public facing TestStream transform a composite transform instead of primitive now, so that the decoding would occur on the SDK worker side where SDK coder is available, and the primitive that powers TestStream, which will be directly translated by runner to always produce raw bytes, and these raw bytes will be decoded on the SDK worker side.
>>>>>>>>
>>>>>>>> There seems to be no role of the SDK harness with regard to the TestStream, because the elements are already encoded by the submitting SDK. The coders must match nevertheless, because you can have Events of type KV<KV<WindowedValue<Integer, Object>>> and what will and what will not get length-prefixed depends on which parts exactly are "known" (model) coders and which are not. Encoding the whole value as single byte array will not work for the consuming SDK harness, which will see that there should be nested KvCoders instead.
>>>>>>>>
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Ke
>>>>>>>>
>>>>>>>> On Aug 31, 2021, at 2:56 PM, Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>>>
>>>>>>>> Sorry if I'm missing something obvious, but I don't quite see the difference between Read and TestStream regarding the discussed issue with coders. Couple of thoughts:
>>>>>>>>
>>>>>>>>      a) both Read and TestStream are _sources_ - they produce elements that are consumed by downstream transforms
>>>>>>>>
>>>>>>>>      b) the coder of a particular PCollection is defined by the Pipeline proto - it is the (client side) SDK that owns the Pipeline and that defines all the coders
>>>>>>>>
>>>>>>>>      c) runners must adhere to these coders, because otherwise there is risk of coder mismatch, most probably on edges like x-lang transforms or inlined transforms
>>>>>>>>
>>>>>>>> I tried the approach of encoding the output of Read into byte array as well, but that turns out to have the problem that once there is a (partially) known coder in play, this does not work, because the consuming transform (executable stage) expects to see the wire coder - that is not simply byte array, because the type of elements might be for instance KV<K, V>, where KvCoder is one of ModelCoders. That does not encode using LengthPrefixCoder and as such will be incompatible with LengthPrefixCoder(ByteArrayCoder). The TestStream needs to know the coder of elements, because that defines where exactly must or must not be inserted length-prefixing. The logic in LengthPrefixUnknownCoders [1] is recursive for ModelCoders.
>>>>>>>>
>>>>>>>> [1] https://github.com/apache/beam/blob/ff70e740a2155592dfcb302ff6303cc19660a268/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
>>>>>>>>
>>>>>>>> On 8/31/21 11:29 PM, Ke Wu wrote:
>>>>>>>>
>>>>>>>> Awesome! Thank you Luke and Robert.
>>>>>>>>
>>>>>>>> Also created https://issues.apache.org/jira/browse/BEAM-12828 to track unit test conversion. I could take it after I updated Samza runner to support TestStream in portable mode.
>>>>>>>>
>>>>>>>> On Aug 31, 2021, at 2:05 PM, Robert Bradshaw <ro...@google.com> wrote:
>>>>>>>>
>>>>>>>> Created https://issues.apache.org/jira/browse/BEAM-12827 to track this.
>>>>>>>>
>>>>>>>> +1 to converting tests to just use longs for better coverage for now.
>>>>>>>>
>>>>>>>> Also, yes, this is very similar to the issues encountered by Reads,
>>>>>>>> but the solution is a bit simpler as there's no need for the
>>>>>>>> TestStream primitive to interact with the decoded version of the
>>>>>>>> elements (unlike Reads, where the sources often give elements in
>>>>>>>> un-encoded form) and no user code to run.
>>>>>>>>
>>>>>>>> - Robert
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Aug 31, 2021 at 11:00 AM Jan Lukavský <je...@seznam.cz> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> This looks (and likely has the same cause) similar to what I have experienced when making primitive Read supported by Flink. The final solution would be to make SDK coders known to the runner of the same SDK (already present in various different threads). But until then, the solution seems to be something like [1]. The root cause is that the executable stage expects its input to be encoded by the SDK harness, and that part is missing when the transform is inlined (like Read in my case, or TestStream in your case). The intoWireTypes method simulates precisely this part - it encodes the PCollection via coder defined in the SDK harness and then decodes it by coder defined by the runner (which match on binary level, but produce different types).
>>>>>>>>
>>>>>>>> Jan
>>>>>>>>
>>>>>>>> [1] https://github.com/apache/beam/blob/dd7945f9f259a2989f9396d1d7a8dcb122711a52/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java#L657
>>>>>>>>
>>>>>>>> On 8/31/21 7:27 PM, Luke Cwik wrote:
>>>>>>>>
>>>>>>>> I originally wasn't for making it a composite because it changes the "graph" structure but the more I thought about it the more I like it.
>>>>>>>>
>>>>>>>> On Tue, Aug 31, 2021 at 10:06 AM Robert Bradshaw <ro...@google.com> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Aug 31, 2021 at 9:18 AM Luke Cwik <lc...@google.com> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> On Mon, Aug 30, 2021 at 7:07 PM Ke Wu <ke...@gmail.com> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> Hello everyone,
>>>>>>>>
>>>>>>>> This is Ke. I am working on enable TestStream support for Samza Runner in portable mode and discovers something unexpected.
>>>>>>>>
>>>>>>>> In my implementation for Samza Runner, couple of tests are failing with errors like
>>>>>>>>
>>>>>>>>
>>>>>>>> java.lang.ClassCastException: java.lang.Integer cannot be cast to [B
>>>>>>>>
>>>>>>>> I noticed these tests have the same symptom on Flink Runner as well, which are currently excluded:
>>>>>>>>
>>>>>>>> https://issues.apache.org/jira/browse/BEAM-12048
>>>>>>>> https://issues.apache.org/jira/browse/BEAM-12050
>>>>>>>>
>>>>>>>>
>>>>>>>> After some more digging, I realized that it is because the combination of following facts:
>>>>>>>>
>>>>>>>> TestStream is a primitive transform, therefore, Runners are supposed to translate directly, the most intuitive implementation for each runner to do is to parse the payload to decode TestStream.Event [1] on the Runner process to be handed over to subsequent stages.
>>>>>>>> When TestStream used with Integers, i.e. VarIntCoder to initialize, since VarIntCoder is NOT a registered ModelCoder [2], it will be treated as custom coder during conversion to protobuf pipeline [3] and will be replaced with byte array coder [4] when runner sends data to SDK worker.
>>>>>>>> Therefore an error occurs because the decoded TestStream.Event has Integer as its value but the remote input receiver is expecting byte array, causing java.lang.ClassCastException: java.lang.Integer cannot be cast to [B
>>>>>>>>
>>>>>>>>
>>>>>>>> In addition, I tried to update all these failed tests to use Long instead of Integer, and all tests will pass since VarLongCoder is a known coder. I do understand that runner process does not have user artifacts staged so it can only use coders in  beam model when communicating with SDK worker process.
>>>>>>>>
>>>>>>>> Couple of questions on this:
>>>>>>>>
>>>>>>>> 1. Is it expected that VarIntegerCoder is not a known coder?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Yes since no one has worked to make it a well known coder.
>>>>>>>>
>>>>>>>>
>>>>>>>> The notion of "integer" vs. "long" is also language-specific detail as
>>>>>>>> well, so not sure it makes sense as a well-known coder.
>>>>>>>>
>>>>>>>> It can be made a well known coder and this would solve the immediate problem but not the long term issue of portable TestStream not supporting arbitrary types.
>>>>>>>>
>>>>>>>>
>>>>>>>> +1. Rather than making coder a property of TestStream, I would be in
>>>>>>>> favor of the TestStream primitive always producing bytes (basically,
>>>>>>>> by definition), and providing a composite that consists of this
>>>>>>>> followed by a decoding to give us a typed TestStream.
>>>>>>>>
>>>>>>>>
>>>>>>>> 2. Is TestStream always supposed to be translated the payload as raw bytes in order that runner process can always send it to SDK worker with the default byte array coder and asks SDK worker to decode accordingly?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Having the runner treat it always as bytes and not T is likely the best solution but isn't necessary.
>>>>>>>>
>>>>>>>> 3. If Yes to 2), then does it mean, TestStream needs to be translated in a completely different way in portable mode from classic mode since in classic mode, translator can directly translates the payload to its final format.
>>>>>>>>
>>>>>>>>
>>>>>>>> There are a few ways to fix the current implementation to work for all types. One way would be if we required the encoded_element to be the "nested" encoding and then ensured that the runner uses a WindowedValue<ByteArrayCoder in outer context> and the SDK used WindowedValue<T> (note that this isn't WindowedValue<LengthPrefix<T>>) for the wire coders. This is quite annoying cause the runner inserts length prefixing in a lot of places (effectively every time it sees an unknown type) so we would need to special case this and propagate this correction through any runner native transforms (e.g. GBK) until the SDK consumes it.
>>>>>>>>
>>>>>>>> Another way would be to ensure that the SDK always uses LengthPrefix<T> as the PCollection encoding and the encoded_element format. This would mean that the runner can translate it to a T if it so chooses and won't have the annoying special case propagation logic. This leaks the length prefixing into the SDK at graph construction time which is not what it was meant for.
>>>>>>>>
>>>>>>>> Swapping to use an existing well known type is by far the easiest approach as you had discovered and won't impact the correctness of the tests.
>>>>>>>>
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Ke
>>>>>>>>
>>>>>>>>
>>>>>>>> [1] https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java#L52
>>>>>>>> [2] https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java#L65
>>>>>>>> [3] https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L99
>>>>>>>> [4] https://github.com/apache/beam/blob/master/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L93
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>

Re: Unexpected in TestStream in Portable Mode

Posted by Robert Bradshaw <ro...@google.com>.
On Tue, Aug 31, 2021 at 11:48 PM Jan Lukavský <je...@seznam.cz> wrote:
>
> Sorry, I needed some time to let that sink in. :-)
>
> I think I understand why (and how) this will work for TestStream, still
> have a couple of notes, though:
>
>   a) the problem of type compatibility arises with the primitive Read as
> well, though we can solve it with different expansion for TestStream,
> that solution is not applicable to Read, because it has different contract
>
>   b) the same exact problem problem will arise every time we inline any
> transform that would otherwise be run in an environment
>
>   c) extracting bytes from TestStream actually bypasses "model coders"
> on the side of runner, should we do that in other cases as well?
>
> The TestStream problem is a special case, I think a more generic
> solution would be better.

Yep, TestStream is special and simpler. I'm not sure something generic
is always possible.

> Two questions:
>
>   1) Could we create a mapping in runners-core-construction-java that
> would take Pipeline proto, and PCollectionId and create a mapping
> function from "sdk coder space" to "runner coder space"? That could be
> optimized to identity if the coder of the PCollection consists of model
> coders only. In that case the sdk coder and runner coder are identical,
> producing the same types. This mapping function could be reused by both
> portable TestStream, inlined primitive Read and any other future inlined
> transform.

This could help with the specific case of Java transforms being
inlined on a Java-based runner. I'd rather avoid Java-specific
solutions if possible. (E.g. the ULR is in Python, and Dataflow Runner
v2 is in C++.)

I'd say if a runner decides to truely inline a transform with an
alternative implementation than calling the SDK, it would only do so
in cases where it understands the input and output types, if any.
(With Read we have a case where the output type is not fixed, which
complicates things.) Generally I would probably structure most
"inlined" operations as one that happens to execute in an in-process
environment rather than an external one, which would mean that
"inlined" ones would not have to be handled specially, but could just
fall out due to the existing length prefixing logic.

Of course just making SDFs work well on all runners would be the best
long term solution for Read, and avoid all these issues, but we may
need hacks in the meantime.

>   2) Why does runner need to understand the types it processes and does
> not work with raw bytes all the times, including model coders, the same
> way as is the proposed solution for TestStream, but for all transforms?
> The first step to every executable stage would then be to decode the
> input from raw bytes and only then process it. What is the benefit of
> runner understanding _some_ of the coders? It would be required for a
> runner to understand coders for root transforms (Impulse? any other?),
> but are there any other places where this is necessary?

A runner needs to understand the KV coder to execute GroupByKey (and
the result is a KV<K, Iterable<V>>, so IterableCoder is a requirement
as well). WindowedCoders are likewise needed so the runner can
deconstruct the timestamps, etc. Timers are sent via TimerCoders. A
runner also needs a way to deal with unknown coders, hence
LengthPrefixCoder. Asside from that, the runner can just deal in raw
bytes (though sometimes there are efficiency gains to understanding
more coders, e.g. wrapping tiny integers in LengthPrefix could be
rather wasteful).

Well known coders, such as RowCoder, are of course useful for
communication between SDKs of different languages, even if the runner
doesn't care about them.

> On 9/1/21 1:37 AM, Robert Bradshaw wrote:
> > On Tue, Aug 31, 2021 at 4:31 PM Jan Lukavský <je...@seznam.cz> wrote:
> >> If I can depict that:
> >>
> >> TestStream0 -> TestStream1 (DoFn), that is
> >>
> >> TestStream0 -> TestStream1 (executable stage) -> consumer 2 (executable
> >> stage)
> > Yes. Expanding it out more,
> >
> > TestStream0 -> pc1 -> TestStream1 -> pc2 -> Consumer2 -> ...
> >
> > Where pc1 is a PCollection<bytes> and pc2 is a PCollection<T>.
> >
> >> In this scenario TestStream0 can produce byte[], and TestStream1 can
> >> consume them, the problem is that if the coder of TestStream0
> > I assume you mean TestStream1, as TestStream0 has no coder.
> >
> >> is some
> >> (recursively) known model coder (e.g. KvCoder), then consumer 2 will not
> >> be able to decode that data.
> > We know that, by construction, TestStream1 and Consumer2 both are
> > executed in an environment that understands pc2's Coder<T>. The
> > difference here is that the runner need not understand Coder<T> as it
> > could inject length prefixing on pc2 if necessary.
> >
> >
> >> On 9/1/21 1:24 AM, Robert Bradshaw wrote:
> >>> On Tue, Aug 31, 2021 at 4:12 PM Jan Lukavský <je...@seznam.cz> wrote:
> >>>>> Fortunately, there's no value in having the primitive TestStream
> >>>>> produce elements of arbitrary type (vs. sources, where the read is
> >>>>> inherently typed depending on the source and particular library used
> >>>>> for reading it).
> >>>> I disagree with this one - the consumer(s) of elements from TestStream
> >>>> are (should be treated as) unknown to the runner, might be inlined,
> >>>> might be x-lang and whether or not will pass the boundary of executable
> >>>> stage, all these questions are important. The coder (and if it is model
> >>>> coder or not, and what are the sub-coders) must be precisely known to
> >>>> all parties that cooperate on the computation, because otherwise these
> >>>> parties might not agree on the binary representation.
> >>> What I'm saying is that there's little value in a  primitive
> >>> TestStream that can produce all sorts of T, vs. a composite TestStream
> >>> consisting of a TestStream0 primitive producing only bytes, followed
> >>> by a TestStream1 that decodes those bytes into T. This sidesteps all
> >>> issues of coder compatibility, as bytes is well-known and TestStream1
> >>> is just a DoFn.
> >>>
> >>>> On 9/1/21 12:59 AM, Robert Bradshaw wrote:
> >>>>> Due to issues with Flink executing Reads as SDFs, there is an
> >>>>> alternative mode where Read is treated as a primitive and executed
> >>>>> directly in the runner. This, of course, requires that the Read's UDF
> >>>>> be implemented in the same language as the Runner (specifically, in
> >>>>> this case, Java) and their interpretation of what Coders should or
> >>>>> should not be wrapped agree (lest other complications arise).
> >>>>>
> >>>>> It's true one can view TestStream as a source, but it's a bit more
> >>>>> than that as it has the ability to wait for quiessence before emitting
> >>>>> more elements/watermark updates to give stronger guarantees on
> >>>>> determinism, which generally requires deeper runner integration. In
> >>>>> addition, TestStream is not associated with a UDF or specific SDK the
> >>>>> way Sources are which is where the problem arises: TestStream gives
> >>>>> the encoded form of the elements but the runner may need to produce a
> >>>>> different encoded form of the elements (say, one with length
> >>>>> prefixing) which is not possible without knowing the Coder.
> >>>>> Fortunately, there's no value in having the primitive TestStream
> >>>>> produce elements of arbitrary type (vs. sources, where the read is
> >>>>> inherently typed depending on the source and particular library used
> >>>>> for reading it).
> >>>>>
> >>>>>
> >>>>> On Tue, Aug 31, 2021 at 3:41 PM Ke Wu <ke...@gmail.com> wrote:
> >>>>>> Read does not have translation in portability, so the implementation is that it needs to be primitive transform explicitly implemented by the runner. The encoding/decoding has to happen in the runner.
> >>>>>>
> >>>>>>
> >>>>>> Could you help me understand this a bit more? IIRC, Read is NOT being translated in portable mode exactly means it is a composite transform instead of primitive because all primitive transforms are required to be translated. In addition, Read is a composite transform of Impulse, which produces dummy bytes [1] to trigger subsequent ParDo/ExecutableStage, where decoding the actual source happens [2]
> >>>>>>
> >>>>>> There seems to be no role of the SDK harness with regard to the TestStream, because the elements are already encoded by the submitting SDK. The coders must match nevertheless, because you can have Events of type KV<KV<WindowedValue<Integer, Object>>> and what will and what will not get length-prefixed depends on which parts exactly are "known" (model) coders and which are not. Encoding the whole value as single byte array will not work for the consuming SDK harness, which will see that there should be nested KvCoders instead.
> >>>>>>
> >>>>>>
> >>>>>> I don’t think I fully understand what you say here. TestStream is currently a primitive transform, therefore there is no role of SDK harness. This is what the proposal to change, to make TestStream a composite transform with a primitive transform and subsequent ParDo to decode to the desired format.
> >>>>>>
> >>>>>>
> >>>>>> [1] https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Impulse.java#L39
> >>>>>> [2] https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L149
> >>>>>>
> >>>>>> On Aug 31, 2021, at 3:21 PM, Jan Lukavský <je...@seznam.cz> wrote:
> >>>>>>
> >>>>>> On 9/1/21 12:13 AM, Ke Wu wrote:
> >>>>>>
> >>>>>> Hi Jan,
> >>>>>>
> >>>>>> Here is my understanding,
> >>>>>>
> >>>>>> Runner is being brought up by job server driver, which is up and running before the job submission, i.e. it is job agnostic. Therefore, the runner it brought up does not have any SDK coder available and artifact staging only happens for SDK workers.
> >>>>>>
> >>>>>> You are right that Read and TestStream are sources, however the one thing that distinguish them is that Read transform is a composite transform and the decoding happens in ParDo/ExecutableStage, i.e. on SDK worker.
> >>>>>>
> >>>>>> Read does not have translation in portability, so the implementation is that it needs to be primitive transform explicitly implemented by the runner. The encoding/decoding has to happen in the runner.
> >>>>>>
> >>>>>>
> >>>>>> The proposal here is also to make the public facing TestStream transform a composite transform instead of primitive now, so that the decoding would occur on the SDK worker side where SDK coder is available, and the primitive that powers TestStream, which will be directly translated by runner to always produce raw bytes, and these raw bytes will be decoded on the SDK worker side.
> >>>>>>
> >>>>>> There seems to be no role of the SDK harness with regard to the TestStream, because the elements are already encoded by the submitting SDK. The coders must match nevertheless, because you can have Events of type KV<KV<WindowedValue<Integer, Object>>> and what will and what will not get length-prefixed depends on which parts exactly are "known" (model) coders and which are not. Encoding the whole value as single byte array will not work for the consuming SDK harness, which will see that there should be nested KvCoders instead.
> >>>>>>
> >>>>>>
> >>>>>> Best,
> >>>>>> Ke
> >>>>>>
> >>>>>> On Aug 31, 2021, at 2:56 PM, Jan Lukavský <je...@seznam.cz> wrote:
> >>>>>>
> >>>>>> Sorry if I'm missing something obvious, but I don't quite see the difference between Read and TestStream regarding the discussed issue with coders. Couple of thoughts:
> >>>>>>
> >>>>>>     a) both Read and TestStream are _sources_ - they produce elements that are consumed by downstream transforms
> >>>>>>
> >>>>>>     b) the coder of a particular PCollection is defined by the Pipeline proto - it is the (client side) SDK that owns the Pipeline and that defines all the coders
> >>>>>>
> >>>>>>     c) runners must adhere to these coders, because otherwise there is risk of coder mismatch, most probably on edges like x-lang transforms or inlined transforms
> >>>>>>
> >>>>>> I tried the approach of encoding the output of Read into byte array as well, but that turns out to have the problem that once there is a (partially) known coder in play, this does not work, because the consuming transform (executable stage) expects to see the wire coder - that is not simply byte array, because the type of elements might be for instance KV<K, V>, where KvCoder is one of ModelCoders. That does not encode using LengthPrefixCoder and as such will be incompatible with LengthPrefixCoder(ByteArrayCoder). The TestStream needs to know the coder of elements, because that defines where exactly must or must not be inserted length-prefixing. The logic in LengthPrefixUnknownCoders [1] is recursive for ModelCoders.
> >>>>>>
> >>>>>> [1] https://github.com/apache/beam/blob/ff70e740a2155592dfcb302ff6303cc19660a268/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCoders.java#L48
> >>>>>>
> >>>>>> On 8/31/21 11:29 PM, Ke Wu wrote:
> >>>>>>
> >>>>>> Awesome! Thank you Luke and Robert.
> >>>>>>
> >>>>>> Also created https://issues.apache.org/jira/browse/BEAM-12828 to track unit test conversion. I could take it after I updated Samza runner to support TestStream in portable mode.
> >>>>>>
> >>>>>> On Aug 31, 2021, at 2:05 PM, Robert Bradshaw <ro...@google.com> wrote:
> >>>>>>
> >>>>>> Created https://issues.apache.org/jira/browse/BEAM-12827 to track this.
> >>>>>>
> >>>>>> +1 to converting tests to just use longs for better coverage for now.
> >>>>>>
> >>>>>> Also, yes, this is very similar to the issues encountered by Reads,
> >>>>>> but the solution is a bit simpler as there's no need for the
> >>>>>> TestStream primitive to interact with the decoded version of the
> >>>>>> elements (unlike Reads, where the sources often give elements in
> >>>>>> un-encoded form) and no user code to run.
> >>>>>>
> >>>>>> - Robert
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Aug 31, 2021 at 11:00 AM Jan Lukavský <je...@seznam.cz> wrote:
> >>>>>>
> >>>>>>
> >>>>>> This looks (and likely has the same cause) similar to what I have experienced when making primitive Read supported by Flink. The final solution would be to make SDK coders known to the runner of the same SDK (already present in various different threads). But until then, the solution seems to be something like [1]. The root cause is that the executable stage expects its input to be encoded by the SDK harness, and that part is missing when the transform is inlined (like Read in my case, or TestStream in your case). The intoWireTypes method simulates precisely this part - it encodes the PCollection via coder defined in the SDK harness and then decodes it by coder defined by the runner (which match on binary level, but produce different types).
> >>>>>>
> >>>>>> Jan
> >>>>>>
> >>>>>> [1] https://github.com/apache/beam/blob/dd7945f9f259a2989f9396d1d7a8dcb122711a52/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java#L657
> >>>>>>
> >>>>>> On 8/31/21 7:27 PM, Luke Cwik wrote:
> >>>>>>
> >>>>>> I originally wasn't for making it a composite because it changes the "graph" structure but the more I thought about it the more I like it.
> >>>>>>
> >>>>>> On Tue, Aug 31, 2021 at 10:06 AM Robert Bradshaw <ro...@google.com> wrote:
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Aug 31, 2021 at 9:18 AM Luke Cwik <lc...@google.com> wrote:
> >>>>>>
> >>>>>>
> >>>>>> On Mon, Aug 30, 2021 at 7:07 PM Ke Wu <ke...@gmail.com> wrote:
> >>>>>>
> >>>>>>
> >>>>>> Hello everyone,
> >>>>>>
> >>>>>> This is Ke. I am working on enable TestStream support for Samza Runner in portable mode and discovers something unexpected.
> >>>>>>
> >>>>>> In my implementation for Samza Runner, couple of tests are failing with errors like
> >>>>>>
> >>>>>>
> >>>>>> java.lang.ClassCastException: java.lang.Integer cannot be cast to [B
> >>>>>>
> >>>>>> I noticed these tests have the same symptom on Flink Runner as well, which are currently excluded:
> >>>>>>
> >>>>>> https://issues.apache.org/jira/browse/BEAM-12048
> >>>>>> https://issues.apache.org/jira/browse/BEAM-12050
> >>>>>>
> >>>>>>
> >>>>>> After some more digging, I realized that it is because the combination of following facts:
> >>>>>>
> >>>>>> TestStream is a primitive transform, therefore, Runners are supposed to translate directly, the most intuitive implementation for each runner to do is to parse the payload to decode TestStream.Event [1] on the Runner process to be handed over to subsequent stages.
> >>>>>> When TestStream used with Integers, i.e. VarIntCoder to initialize, since VarIntCoder is NOT a registered ModelCoder [2], it will be treated as custom coder during conversion to protobuf pipeline [3] and will be replaced with byte array coder [4] when runner sends data to SDK worker.
> >>>>>> Therefore an error occurs because the decoded TestStream.Event has Integer as its value but the remote input receiver is expecting byte array, causing java.lang.ClassCastException: java.lang.Integer cannot be cast to [B
> >>>>>>
> >>>>>>
> >>>>>> In addition, I tried to update all these failed tests to use Long instead of Integer, and all tests will pass since VarLongCoder is a known coder. I do understand that runner process does not have user artifacts staged so it can only use coders in  beam model when communicating with SDK worker process.
> >>>>>>
> >>>>>> Couple of questions on this:
> >>>>>>
> >>>>>> 1. Is it expected that VarIntegerCoder is not a known coder?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> Yes since no one has worked to make it a well known coder.
> >>>>>>
> >>>>>>
> >>>>>> The notion of "integer" vs. "long" is also language-specific detail as
> >>>>>> well, so not sure it makes sense as a well-known coder.
> >>>>>>
> >>>>>> It can be made a well known coder and this would solve the immediate problem but not the long term issue of portable TestStream not supporting arbitrary types.
> >>>>>>
> >>>>>>
> >>>>>> +1. Rather than making coder a property of TestStream, I would be in
> >>>>>> favor of the TestStream primitive always producing bytes (basically,
> >>>>>> by definition), and providing a composite that consists of this
> >>>>>> followed by a decoding to give us a typed TestStream.
> >>>>>>
> >>>>>>
> >>>>>> 2. Is TestStream always supposed to be translated the payload as raw bytes in order that runner process can always send it to SDK worker with the default byte array coder and asks SDK worker to decode accordingly?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> Having the runner treat it always as bytes and not T is likely the best solution but isn't necessary.
> >>>>>>
> >>>>>> 3. If Yes to 2), then does it mean, TestStream needs to be translated in a completely different way in portable mode from classic mode since in classic mode, translator can directly translates the payload to its final format.
> >>>>>>
> >>>>>>
> >>>>>> There are a few ways to fix the current implementation to work for all types. One way would be if we required the encoded_element to be the "nested" encoding and then ensured that the runner uses a WindowedValue<ByteArrayCoder in outer context> and the SDK used WindowedValue<T> (note that this isn't WindowedValue<LengthPrefix<T>>) for the wire coders. This is quite annoying cause the runner inserts length prefixing in a lot of places (effectively every time it sees an unknown type) so we would need to special case this and propagate this correction through any runner native transforms (e.g. GBK) until the SDK consumes it.
> >>>>>>
> >>>>>> Another way would be to ensure that the SDK always uses LengthPrefix<T> as the PCollection encoding and the encoded_element format. This would mean that the runner can translate it to a T if it so chooses and won't have the annoying special case propagation logic. This leaks the length prefixing into the SDK at graph construction time which is not what it was meant for.
> >>>>>>
> >>>>>> Swapping to use an existing well known type is by far the easiest approach as you had discovered and won't impact the correctness of the tests.
> >>>>>>
> >>>>>>
> >>>>>> Best,
> >>>>>> Ke
> >>>>>>
> >>>>>>
> >>>>>> [1] https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java#L52
> >>>>>> [2] https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java#L65
> >>>>>> [3] https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java#L99
> >>>>>> [4] https://github.com/apache/beam/blob/master/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java#L93
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>