If it was made into a portable coder that would expand the set of
transforms that would be usable outside of Java though.

On Wed, Jun 10, 2026 at 8:55 AM Ganesh Sivakumar <
[email protected]> wrote:

> Thanks, I got it, It's a sdk specific coder.
>
> On Tue, 9 Jun, 2026, 9:57 pm Robert Bradshaw via dev, <[email protected]>
> wrote:
>
>> On Tue, Jun 9, 2026 at 6:36 AM Ganesh Sivakumar <
>> [email protected]> wrote:
>>
>>> I had worked on coders implementation and started testing java pipelines
>>> on the runner. Noticed something interesting, Typically Pipeline object
>>> contain map of coder id and the Coder object like:
>>> ```
>>> "ByteArrayCoder": Coder {
>>>                     spec: Some(
>>>                         FunctionSpec {
>>>                             urn: "beam:coder:bytes:v1",
>>>                             payload: [],
>>>                         },
>>>                     ),
>>>                     component_coder_ids: [],
>>>                 },
>>> ``
>>> Which makes perfect sense for the runner to use the coder_id of
>>> pcollection to get the right coder implementation based on urn and
>>> encode/decode when that pcollection arrives. But my pipeline had a dofn
>>> that gets the string input and prints it, it's the end of pipeline and dofn
>>> returns void.  DoFn<String,Void>. Coder for this looked like :
>>>
>>> ```
>>>                 "VoidCoder": Coder {
>>>                     spec: Some(
>>>                         FunctionSpec {
>>>                             urn: "beam:coders:javasdk:0.1",
>>>                             payload: [
>>> ```
>>> I understand VoidCoder is for handling empty values, basically do
>>> nothing when we receive the coder?  But why doesn't it have its own urn
>>> like others, what's the purpose of beam:coders:javasdk:0.1.
>>>
>>
>> This is because the VoidCoder was never "made portable" in the sense of
>> becoming a transparent Coder with a well-defined spec suitable for
>> traversing cross-language boundaries. "beam:coders:javasdk:0.1" means "a
>> java specific coder, defined by its java serialized bytes" and is what's
>> used user-defined coders.
>>
>> It would certainly make sense to update VoidCoder to a true
>> cross-language coder.
>>
>> Since it's a dofn,if worker executes it and output pcol is void, worker
>>> sends no elements to runner in data channel?
>>>
>>
>> As an aside, IIRC, the void coder might send a single byte (e.g. so one
>> could send a list of exactly N voids) that carry no semantic meaning. Worth
>> checking the implementation.
>>
>>
>>> On Wed, May 20, 2026 at 5:26 PM Ganesh Sivakumar <
>>> [email protected]> wrote:
>>>
>>>> Thanks, This is a really valuable reference. I'm working on coders Rust
>>>> implementation, will reach out if there are any questions.
>>>>
>>>> On Tue, May 19, 2026 at 7:11 PM Robert Burke <[email protected]>
>>>> wrote:
>>>>
>>>>> Just a quick reply
>>>>>
>>>>> You're right that you use the Beam coders to interpret the bytes. You
>>>>> just need an implementation of them in rust (and in every language 
>>>>> building
>>>>> Beam components).
>>>>>
>>>>> For the Prism runner (written in Go, default for Python and Go) we use
>>>>> the Go SDK coder implementations, because they were already present. But
>>>>> not every thing makes sense to use directly from the SDK within a runner
>>>>> context.
>>>>>
>>>>>
>>>>> https://github.com/apache/beam/blob/master/sdks/go/pkg/beam/runners/prism/internal/coders.go
>>>>>
>>>>> For example, for timers, it made more sense to reimplement certain
>>>>> portions within the engine portion of prism, than to route towards SDK
>>>>> constructs.
>>>>>
>>>>>
>>>>> https://github.com/apache/beam/blob/master/sdks/go/pkg/beam/runners/prism/internal/engine/timers.go#L135
>>>>>
>>>>>
>>>>> I wrote up the flow that Prism uses for managing bundles here. There
>>>>> are flowcharts that provide the broad strokes, and I hope they are useful
>>>>> to someone building their own runner.
>>>>>
>>>>>
>>>>> https://github.com/apache/beam/blob/master/sdks/go/pkg/beam/runners/prism/internal/README.md
>>>>>
>>>>>
>>>>> Finally, while the Beam Pipeline Protos and runner APIs dictate how
>>>>> things communicate between runners and SDKs. The rest is up to you.
>>>>> I have a languishing "hobby" Go SDK that uses a different approach to
>>>>> coders to make them easier to deal with and manipulate, vs just the
>>>>> bytestream approach.
>>>>>
>>>>> https://github.com/lostluck/beam-go/blob/main/coders/coders.go
>>>>>
>>>>> It mostly wraps a byte buffer, and then allows callers to pop or push
>>>>> values to it. But this ends up playing very well for the garbage collector
>>>>> in Go.
>>>>>
>>>>> Rust has different constraints and problems to solve, so don't feel
>>>>> constrained by how the other languages do it.
>>>>>
>>>>> Hope this helps, and let me know if you have questions.
>>>>> Robert Burke
>>>>>
>>>>> On Tue, May 19, 2026, 5:29 AM Ganesh Sivakumar <
>>>>> [email protected]> wrote:
>>>>>
>>>>>> Hey Everyone,
>>>>>>
>>>>>> I am working on a new Rust based portable Beam Runner and I'm at
>>>>>> pipeline execution phase where the Rust runner side needs to
>>>>>> communicate with the worker sdk harness(Java) and execute the stages(
>>>>>> stages are nothing but a set of fused transforms, formed using the greedy
>>>>>> fusion approach from Beam Java utils, rewritten in Rust for the runner)
>>>>>>
>>>>>> For the stage to run on a worker, the runner needs to register the
>>>>>> stage information with the worker and then send a run request via grpc
>>>>>> channels. The worker will execute the transforms in the stage and send 
>>>>>> the
>>>>>> output back to runner via data grpc channel in the form of Elements [1]
>>>>>> Elements contain the output data as raw bytes which the runner needs to
>>>>>> decode to get the actual data like String, Int or POJO. Typically other
>>>>>> runners do it with Beam's defined coders for encoding and decoding. But 
>>>>>> for
>>>>>> Rust there isn't Beam coders implementation. Curious if anyone previously
>>>>>> worked on coders for cross languages, or similar things, and how did you
>>>>>> implement in general.
>>>>>>
>>>>>> Thanks,
>>>>>> Ganesh.
>>>>>>
>>>>>> [1] -
>>>>>> https://github.com/apache/beam/blob/55eb624e5cd00e546ab19fc411281a0e5f596142/model/fn-execution/src/main/proto/org/apache/beam/model/fn_execution/v1/beam_fn_api.proto#L724
>>>>>>
>>>>>

Reply via email to