You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by GitBox <gi...@apache.org> on 2020/03/23 17:37:31 UTC

[GitHub] [beam] lostluck opened a new pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

lostluck opened a new pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197
 
 
   This adds a Reshuffle transform to the Go SDK.
   
   * In particular, it configures windowing & trigggers for a GBK to allow for fusion breaks, where parallelism needs to increase, or decrease due to data bundling properties.
   * Previous element window and timestamps are preserved.
   * The SDK operations are wrapped with a higher level reshuffle URN so runners can optimize this step better.
   
   Note: While this should work for streaming cases, it hasn't been tested with them yet, due to the current state of streaming the Go SDK.
   
   It further adds one small optimization for the internal Decoding interface, called the DecodeTo method to avoid extra allocations to the heap incurred by returning a *FullValue. Can't avoid extra allocations for KV types at present, but value PCollections should have lower overhead. A subsequent PR will use the DecodeTo method at a other applicable places in the decode stack.
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark
   --- | --- | --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/)
   Java | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/)
   Python | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/) | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/)
   XLang | --- | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/)
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website
   --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/) 
   Portable | --- | [![Build Status](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/) | --- | ---
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-605211109
 
 
   Retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r396887810
 
 

 ##########
 File path: sdks/go/pkg/beam/gbk.go
 ##########
 @@ -95,3 +95,52 @@ func TryCoGroupByKey(s Scope, cols ...PCollection) (PCollection, error) {
 	ret.SetCoder(NewCoder(ret.Type()))
 	return ret, nil
 }
+
+// Reshuffle copies a PCollection of the same kind and using the same element
+// coder, and maintains the same windowing information. Importantly, it allows
+// the result PCollection to be processed with a different sharding, in a
+// different stage than the input PCollection.
+//
+// For example, if a computation needs a lot of parallelism but
+// produces only a small amount of output data, then the computation
+// producing the data can run with as much parallelism as needed,
+// while the output file is written with a smaller amount of
+// parallelism, using the following pattern:
+//
+//   pc := bigHairyComputationNeedingParallelism(scope) // PCollection<string>
+//   resharded := beam.Reshard(scope, pc)                // PCollection<string>
 
 Review comment:
   Here and elsewhere in this comment, Reshuffle is referred to as "Reshard". I think it's fine to refer to it as a reshard informally, since that's what it functionally is, but the places where it's used as a proper noun should be switched to Reshuffle.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r396823199
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
 ##########
 @@ -259,59 +298,82 @@ type customDecoder struct {
 	dec Decoder
 }
 
-func (c *customDecoder) Decode(r io.Reader) (*FullValue, error) {
+func (c *customDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
 	// (1) Read length-prefixed encoded data
 
 	size, err := coder.DecodeVarInt(r)
 	if err != nil {
-		return nil, err
+		return err
 	}
 	data, err := ioutilx.ReadN(r, (int)(size))
 	if err != nil {
-		return nil, err
+		return err
 	}
 
 	// (2) Call decode
 
 	val, err := c.dec.Decode(c.t, data)
 	if err != nil {
+		return err
+	}
+	*fv = FullValue{Elm: val}
+	return err
 
 Review comment:
   I know this is just preserving the existing behavior, but it seems weird to `return err` here instead of `return nil`, even if it is guaranteed to be `nil` at this point.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r396856077
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/exec/reshuffle.go
 ##########
 @@ -0,0 +1,170 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+	"bytes"
+	"context"
+	"fmt"
+	"io"
+	"math/rand"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+	"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
+)
+
+// ReshuffleInput is a Node.
+type ReshuffleInput struct {
+	UID   UnitID
+	SID   StreamID
+	Coder *coder.Coder // Coder for the input PCollection.
+	Seed  int64
+	Out   Node
+
+	r    *rand.Rand
+	enc  ElementEncoder
+	wEnc WindowEncoder
+	b    bytes.Buffer
+	// ret is a cached allocations for passing to the next Unit. Units never modify the passed in FullValue.
+	ret FullValue
+}
+
+// ID returns the unit debug id.
+func (n *ReshuffleInput) ID() UnitID {
+	return n.UID
+}
+
+// Up initializes the value and window encoders, and the random source.
+func (n *ReshuffleInput) Up(ctx context.Context) error {
+	n.enc = MakeElementEncoder(coder.SkipW(n.Coder))
+	n.wEnc = MakeWindowEncoder(n.Coder.Window)
+	n.r = rand.New(rand.NewSource(n.Seed))
+	return nil
+}
+
+// StartBundle is a no-op.
+func (n *ReshuffleInput) StartBundle(ctx context.Context, id string, data DataContext) error {
+	return MultiStartBundle(ctx, id, data, n.Out)
+}
+
+func (n *ReshuffleInput) ProcessElement(ctx context.Context, value *FullValue, values ...ReStream) error {
+	n.b.Reset()
+	if err := EncodeWindowedValueHeader(n.wEnc, value.Windows, value.Timestamp, &n.b); err != nil {
+		return err
+	}
+	if err := n.enc.Encode(value, &n.b); err != nil {
+		return errors.WithContextf(err, "encoding element %v with coder %v", value, n.Coder)
+	}
+	n.ret = FullValue{Elm: n.r.Int(), Elm2: n.b.Bytes(), Timestamp: value.Timestamp}
+	if err := n.Out.ProcessElement(ctx, &n.ret); err != nil {
+		return err
+	}
+	return nil
+}
+
+// FinishBundle propagates finish bundle, and clears cached state.
+func (n *ReshuffleInput) FinishBundle(ctx context.Context) error {
+	n.b = bytes.Buffer{}
+	n.ret = FullValue{}
+	return MultiFinishBundle(ctx, n.Out)
+}
+
+// Down is a no-op.
+func (n *ReshuffleInput) Down(ctx context.Context) error {
+	return nil
+}
+
+func (n *ReshuffleInput) String() string {
+	return fmt.Sprintf("ReshuffleInput[%v] Coder:%v", n.SID, n.Coder)
+}
+
+// ReshuffleOutput is a Node.
+type ReshuffleOutput struct {
+	UID   UnitID
+	SID   StreamID
+	Coder *coder.Coder // Coder for the receiving PCollection.
+	Out   Node
+
+	b    bytes.Buffer
+	dec  ElementDecoder
+	wDec WindowDecoder
+	ret  FullValue
+}
+
+// ID returns the unit debug id.
+func (n *ReshuffleOutput) ID() UnitID {
+	return n.UID
+}
+
+// Up initializes the value and window encoders, and the random source.
+func (n *ReshuffleOutput) Up(ctx context.Context) error {
+	n.dec = MakeElementDecoder(coder.SkipW(n.Coder))
+	n.wDec = MakeWindowDecoder(n.Coder.Window)
+	return nil
+}
+
+// StartBundle is a no-op.
+func (n *ReshuffleOutput) StartBundle(ctx context.Context, id string, data DataContext) error {
+	return MultiStartBundle(ctx, id, data, n.Out)
+}
+
+func (n *ReshuffleOutput) ProcessElement(ctx context.Context, value *FullValue, values ...ReStream) error {
+	// Marshal the pieces into a temporary buffer since they must be transmitted on FnAPI as a single
+	// unit.
+	vs, err := values[0].Open()
 
 Review comment:
   It's strange to me that `values[]` can have multiple elements, but this method ends up actually reading all the values from the first element of it. Could you explain why that happens? Are there sometimes multiple ReStreams representing different things?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r396853348
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/exec/reshuffle.go
 ##########
 @@ -0,0 +1,170 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+	"bytes"
+	"context"
+	"fmt"
+	"io"
+	"math/rand"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+	"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
+)
+
+// ReshuffleInput is a Node.
+type ReshuffleInput struct {
+	UID   UnitID
+	SID   StreamID
+	Coder *coder.Coder // Coder for the input PCollection.
+	Seed  int64
+	Out   Node
+
+	r    *rand.Rand
+	enc  ElementEncoder
+	wEnc WindowEncoder
+	b    bytes.Buffer
+	// ret is a cached allocations for passing to the next Unit. Units never modify the passed in FullValue.
+	ret FullValue
+}
+
+// ID returns the unit debug id.
+func (n *ReshuffleInput) ID() UnitID {
+	return n.UID
+}
+
+// Up initializes the value and window encoders, and the random source.
+func (n *ReshuffleInput) Up(ctx context.Context) error {
+	n.enc = MakeElementEncoder(coder.SkipW(n.Coder))
+	n.wEnc = MakeWindowEncoder(n.Coder.Window)
+	n.r = rand.New(rand.NewSource(n.Seed))
+	return nil
+}
+
+// StartBundle is a no-op.
+func (n *ReshuffleInput) StartBundle(ctx context.Context, id string, data DataContext) error {
+	return MultiStartBundle(ctx, id, data, n.Out)
+}
+
+func (n *ReshuffleInput) ProcessElement(ctx context.Context, value *FullValue, values ...ReStream) error {
+	n.b.Reset()
+	if err := EncodeWindowedValueHeader(n.wEnc, value.Windows, value.Timestamp, &n.b); err != nil {
+		return err
+	}
+	if err := n.enc.Encode(value, &n.b); err != nil {
+		return errors.WithContextf(err, "encoding element %v with coder %v", value, n.Coder)
+	}
+	n.ret = FullValue{Elm: n.r.Int(), Elm2: n.b.Bytes(), Timestamp: value.Timestamp}
+	if err := n.Out.ProcessElement(ctx, &n.ret); err != nil {
+		return err
+	}
+	return nil
 
 Review comment:
   ```suggestion
   	return n.Out.ProcessElement(ctx, &n.ret)
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r396855047
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/exec/reshuffle.go
 ##########
 @@ -0,0 +1,170 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+	"bytes"
+	"context"
+	"fmt"
+	"io"
+	"math/rand"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+	"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
+)
+
+// ReshuffleInput is a Node.
+type ReshuffleInput struct {
+	UID   UnitID
+	SID   StreamID
+	Coder *coder.Coder // Coder for the input PCollection.
+	Seed  int64
+	Out   Node
+
+	r    *rand.Rand
+	enc  ElementEncoder
+	wEnc WindowEncoder
+	b    bytes.Buffer
+	// ret is a cached allocations for passing to the next Unit. Units never modify the passed in FullValue.
+	ret FullValue
+}
+
+// ID returns the unit debug id.
+func (n *ReshuffleInput) ID() UnitID {
+	return n.UID
+}
+
+// Up initializes the value and window encoders, and the random source.
+func (n *ReshuffleInput) Up(ctx context.Context) error {
+	n.enc = MakeElementEncoder(coder.SkipW(n.Coder))
+	n.wEnc = MakeWindowEncoder(n.Coder.Window)
+	n.r = rand.New(rand.NewSource(n.Seed))
+	return nil
+}
+
+// StartBundle is a no-op.
+func (n *ReshuffleInput) StartBundle(ctx context.Context, id string, data DataContext) error {
+	return MultiStartBundle(ctx, id, data, n.Out)
+}
+
+func (n *ReshuffleInput) ProcessElement(ctx context.Context, value *FullValue, values ...ReStream) error {
+	n.b.Reset()
+	if err := EncodeWindowedValueHeader(n.wEnc, value.Windows, value.Timestamp, &n.b); err != nil {
+		return err
+	}
+	if err := n.enc.Encode(value, &n.b); err != nil {
+		return errors.WithContextf(err, "encoding element %v with coder %v", value, n.Coder)
+	}
+	n.ret = FullValue{Elm: n.r.Int(), Elm2: n.b.Bytes(), Timestamp: value.Timestamp}
+	if err := n.Out.ProcessElement(ctx, &n.ret); err != nil {
+		return err
+	}
+	return nil
+}
+
+// FinishBundle propagates finish bundle, and clears cached state.
+func (n *ReshuffleInput) FinishBundle(ctx context.Context) error {
+	n.b = bytes.Buffer{}
+	n.ret = FullValue{}
+	return MultiFinishBundle(ctx, n.Out)
+}
+
+// Down is a no-op.
+func (n *ReshuffleInput) Down(ctx context.Context) error {
+	return nil
+}
+
+func (n *ReshuffleInput) String() string {
+	return fmt.Sprintf("ReshuffleInput[%v] Coder:%v", n.SID, n.Coder)
+}
+
+// ReshuffleOutput is a Node.
+type ReshuffleOutput struct {
+	UID   UnitID
+	SID   StreamID
+	Coder *coder.Coder // Coder for the receiving PCollection.
+	Out   Node
+
+	b    bytes.Buffer
+	dec  ElementDecoder
+	wDec WindowDecoder
+	ret  FullValue
+}
+
+// ID returns the unit debug id.
+func (n *ReshuffleOutput) ID() UnitID {
+	return n.UID
+}
+
+// Up initializes the value and window encoders, and the random source.
+func (n *ReshuffleOutput) Up(ctx context.Context) error {
+	n.dec = MakeElementDecoder(coder.SkipW(n.Coder))
+	n.wDec = MakeWindowDecoder(n.Coder.Window)
+	return nil
+}
+
+// StartBundle is a no-op.
+func (n *ReshuffleOutput) StartBundle(ctx context.Context, id string, data DataContext) error {
+	return MultiStartBundle(ctx, id, data, n.Out)
+}
+
+func (n *ReshuffleOutput) ProcessElement(ctx context.Context, value *FullValue, values ...ReStream) error {
+	// Marshal the pieces into a temporary buffer since they must be transmitted on FnAPI as a single
+	// unit.
+	vs, err := values[0].Open()
+	if err != nil {
+		return errors.WithContextf(err, "decoding values for %v with coder %v", value, n.Coder)
+	}
+	defer vs.Close()
+	for {
+		v, err := vs.Read()
+		if err != nil {
+			if err == io.EOF {
+				return nil
+			}
+			return errors.WithContextf(err, "reading values for %v", n)
+		}
+		n.b = *bytes.NewBuffer(v.Elm.([]byte))
+		ws, ts, err := DecodeWindowedValueHeader(n.wDec, &n.b)
+		if err != nil {
+			return errors.WithContextf(err, "decoding windows for %v", n)
+		}
+		if err := n.dec.DecodeTo(&n.b, &n.ret); err != nil {
+			return errors.WithContextf(err, "decoding element for %v", n)
+		}
+		n.ret.Windows = ws
+		n.ret.Timestamp = ts
+		if err := n.Out.ProcessElement(ctx, &n.ret); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// FinishBundle propagates finish bundle to downstream nodes.
+func (n *ReshuffleOutput) FinishBundle(ctx context.Context) error {
+	n.b = bytes.Buffer{}
 
 Review comment:
   Should this also be clearing `n.ret` like `ReshuffleInput` does?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r397451550
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/graphx/translate.go
 ##########
 @@ -473,6 +479,164 @@ func (m *marshaller) addNode(n *graph.Node) string {
 	return m.makeNode(id, m.coders.Add(n.Coder), n)
 }
 
+// expandReshuffle translates resharding to a composite reshuffle
+// transform.
+//
+// With proper runner support, the SDK doesn't need to do anything.
+// However, we still need to provide a backup plan in terms of other
+// PTransforms in the event the runner doesn't have a native implementation.
+//
+// In particular, the "backup plan" needs to:
+//
+//  * Encode the windowed element, preserving timestamps.
+//  * Add random keys to the encoded windowed element []bytes
+//  * GroupByKey (in the global window).
+//  * Explode the resulting elements list.
+//  * Decode the windowed element []bytes.
+//
+// While a simple reshard can be written in user terms, (timestamps and windows
+// are accessible to user functions) there are some framework internal
+// optimizations that can be done if the framework is aware of the reshard, though
+// ideally this is handled on the runner side.
+//
+// User code is able to write reshards, but it's easier to access
+// the window coders framework side, which is critical for the reshard
+// to function with unbounded inputs.
+func (m *marshaller) expandReshuffle(edge NamedEdge) string {
+	id := edgeID(edge.Edge)
+	var kvCoderID, gbkCoderID string
+	{
+		kv := makeUnionCoder()
+		kvCoderID = m.coders.Add(kv)
+		gbkCoderID = m.coders.Add(coder.NewCoGBK(kv.Components))
+	}
+
+	var subtransforms []string
+
+	in := edge.Edge.Input[0]
+
+	origInput := m.addNode(in.From)
+	// We need to preserve the old windowing/triggering here
+	// for re-instatement after the GBK.
+	preservedWSId := m.pcollections[origInput].GetWindowingStrategyId()
+
+	// Get the windowing strategy from before:
+	postReify := fmt.Sprintf("%v_%v_reifyts", nodeID(in.From), id)
+	m.makeNode(postReify, kvCoderID, in.From)
+
+	// We need to replace postReify's windowing strategy with one appropriate
+	// for reshuffles.
+	{
+		wfn := window.NewGlobalWindows()
+		m.pcollections[postReify].WindowingStrategyId =
+			m.internWindowingStrategy(&pb.WindowingStrategy{
+				// Not segregated by time...
+				WindowFn: makeWindowFn(wfn),
+				// ...output after every element is received...
+				Trigger: &pb.Trigger{
+					// Should this be an Always trigger instead?
+					Trigger: &pb.Trigger_ElementCount_{
+						ElementCount: &pb.Trigger_ElementCount{
+							ElementCount: 1,
+						},
+					},
+				},
+				// ...and after outputing, discard the output elements...
+				AccumulationMode: pb.AccumulationMode_DISCARDING,
+				// ...and since every pane should have 1 element,
+				// try to preserve the timestamp.
+				OutputTime: pb.OutputTime_EARLIEST_IN_PANE,
+				// Defaults copied from marshalWindowingStrategy.
+				// TODO(BEAM-3304): migrate to user side operations once trigger support is in.
+				EnvironmentId:   m.addDefaultEnv(),
+				MergeStatus:     pb.MergeStatus_NON_MERGING,
+				WindowCoderId:   m.coders.AddWindowCoder(makeWindowCoder(wfn)),
+				ClosingBehavior: pb.ClosingBehavior_EMIT_IF_NONEMPTY,
+				AllowedLateness: 0,
+				OnTimeBehavior:  pb.OnTimeBehavior_FIRE_ALWAYS,
+			})
+	}
+
+	// Inputs (i)
+
+	inputID := fmt.Sprintf("%v_reifyts", id)
+	payload := &pb.ParDoPayload{
+		DoFn: &pb.FunctionSpec{
+			Urn: URNReshuffleInput,
+			Payload: []byte(protox.MustEncodeBase64(&v1.TransformPayload{
+				Urn: URNReshuffleInput,
+			})),
+		},
+	}
+	input := &pb.PTransform{
+		UniqueName: inputID,
+		Spec: &pb.FunctionSpec{
+			Urn:     URNParDo,
+			Payload: protox.MustEncode(payload),
+		},
+		Inputs:        map[string]string{"i0": nodeID(in.From)},
+		Outputs:       map[string]string{"i0": postReify},
+		EnvironmentId: m.addDefaultEnv(),
+	}
+	m.transforms[inputID] = input
+	subtransforms = append(subtransforms, inputID)
+
+	outNode := edge.Edge.Output[0].To
+
+	// GBK
+
+	gbkOut := fmt.Sprintf("%v_out", nodeID(outNode))
+	m.makeNode(gbkOut, gbkCoderID, outNode)
+
+	gbkID := fmt.Sprintf("%v_gbk", id)
+	gbk := &pb.PTransform{
+		UniqueName: gbkID,
+		Spec:       &pb.FunctionSpec{Urn: URNGBK},
+		Inputs:     map[string]string{"i0": postReify},
 
 Review comment:
   Oh ok, I see it now. I was completely misinterpreting the expansion here. So if I understand correctly, it should look like this, right? (With pcollections/nodes in square brackets and transforms/edges as arrows)
   
   ```
   [in.From] ---input---> [postReify] ---gbk---> [gbkOut] ---output---> [out.To]
   ```
   
   Where `input` and `output` are the newly added Reshuffle transforms. That looks right to me, and rereading the code it looks consistent with that.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r397308582
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/exec/coder.go
 ##########
 @@ -259,59 +298,82 @@ type customDecoder struct {
 	dec Decoder
 }
 
-func (c *customDecoder) Decode(r io.Reader) (*FullValue, error) {
+func (c *customDecoder) DecodeTo(r io.Reader, fv *FullValue) error {
 	// (1) Read length-prefixed encoded data
 
 	size, err := coder.DecodeVarInt(r)
 	if err != nil {
-		return nil, err
+		return err
 	}
 	data, err := ioutilx.ReadN(r, (int)(size))
 	if err != nil {
-		return nil, err
+		return err
 	}
 
 	// (2) Call decode
 
 	val, err := c.dec.Decode(c.t, data)
 	if err != nil {
+		return err
+	}
+	*fv = FullValue{Elm: val}
+	return err
 
 Review comment:
   Good catch. Thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-602862121
 
 
   Post commits run and pass which is a good sign! 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-602851450
 
 
   Run Go Postcommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r396886566
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/graphx/translate.go
 ##########
 @@ -473,6 +479,164 @@ func (m *marshaller) addNode(n *graph.Node) string {
 	return m.makeNode(id, m.coders.Add(n.Coder), n)
 }
 
+// expandReshuffle translates resharding to a composite reshuffle
+// transform.
+//
+// With proper runner support, the SDK doesn't need to do anything.
+// However, we still need to provide a backup plan in terms of other
+// PTransforms in the event the runner doesn't have a native implementation.
+//
+// In particular, the "backup plan" needs to:
+//
+//  * Encode the windowed element, preserving timestamps.
+//  * Add random keys to the encoded windowed element []bytes
+//  * GroupByKey (in the global window).
+//  * Explode the resulting elements list.
+//  * Decode the windowed element []bytes.
+//
+// While a simple reshard can be written in user terms, (timestamps and windows
+// are accessible to user functions) there are some framework internal
+// optimizations that can be done if the framework is aware of the reshard, though
+// ideally this is handled on the runner side.
+//
+// User code is able to write reshards, but it's easier to access
+// the window coders framework side, which is critical for the reshard
+// to function with unbounded inputs.
+func (m *marshaller) expandReshuffle(edge NamedEdge) string {
+	id := edgeID(edge.Edge)
+	var kvCoderID, gbkCoderID string
+	{
+		kv := makeUnionCoder()
+		kvCoderID = m.coders.Add(kv)
+		gbkCoderID = m.coders.Add(coder.NewCoGBK(kv.Components))
+	}
+
+	var subtransforms []string
+
+	in := edge.Edge.Input[0]
+
+	origInput := m.addNode(in.From)
+	// We need to preserve the old windowing/triggering here
+	// for re-instatement after the GBK.
+	preservedWSId := m.pcollections[origInput].GetWindowingStrategyId()
+
+	// Get the windowing strategy from before:
+	postReify := fmt.Sprintf("%v_%v_reifyts", nodeID(in.From), id)
+	m.makeNode(postReify, kvCoderID, in.From)
+
+	// We need to replace postReify's windowing strategy with one appropriate
+	// for reshuffles.
+	{
+		wfn := window.NewGlobalWindows()
+		m.pcollections[postReify].WindowingStrategyId =
+			m.internWindowingStrategy(&pb.WindowingStrategy{
+				// Not segregated by time...
+				WindowFn: makeWindowFn(wfn),
+				// ...output after every element is received...
+				Trigger: &pb.Trigger{
+					// Should this be an Always trigger instead?
+					Trigger: &pb.Trigger_ElementCount_{
+						ElementCount: &pb.Trigger_ElementCount{
+							ElementCount: 1,
+						},
+					},
+				},
+				// ...and after outputing, discard the output elements...
+				AccumulationMode: pb.AccumulationMode_DISCARDING,
+				// ...and since every pane should have 1 element,
+				// try to preserve the timestamp.
+				OutputTime: pb.OutputTime_EARLIEST_IN_PANE,
+				// Defaults copied from marshalWindowingStrategy.
+				// TODO(BEAM-3304): migrate to user side operations once trigger support is in.
+				EnvironmentId:   m.addDefaultEnv(),
+				MergeStatus:     pb.MergeStatus_NON_MERGING,
+				WindowCoderId:   m.coders.AddWindowCoder(makeWindowCoder(wfn)),
+				ClosingBehavior: pb.ClosingBehavior_EMIT_IF_NONEMPTY,
+				AllowedLateness: 0,
+				OnTimeBehavior:  pb.OnTimeBehavior_FIRE_ALWAYS,
+			})
+	}
+
+	// Inputs (i)
+
+	inputID := fmt.Sprintf("%v_reifyts", id)
+	payload := &pb.ParDoPayload{
+		DoFn: &pb.FunctionSpec{
+			Urn: URNReshuffleInput,
+			Payload: []byte(protox.MustEncodeBase64(&v1.TransformPayload{
+				Urn: URNReshuffleInput,
+			})),
+		},
+	}
+	input := &pb.PTransform{
+		UniqueName: inputID,
+		Spec: &pb.FunctionSpec{
+			Urn:     URNParDo,
+			Payload: protox.MustEncode(payload),
+		},
+		Inputs:        map[string]string{"i0": nodeID(in.From)},
+		Outputs:       map[string]string{"i0": postReify},
+		EnvironmentId: m.addDefaultEnv(),
+	}
+	m.transforms[inputID] = input
+	subtransforms = append(subtransforms, inputID)
+
+	outNode := edge.Edge.Output[0].To
+
+	// GBK
+
+	gbkOut := fmt.Sprintf("%v_out", nodeID(outNode))
+	m.makeNode(gbkOut, gbkCoderID, outNode)
+
+	gbkID := fmt.Sprintf("%v_gbk", id)
+	gbk := &pb.PTransform{
+		UniqueName: gbkID,
+		Spec:       &pb.FunctionSpec{Urn: URNGBK},
+		Inputs:     map[string]string{"i0": postReify},
 
 Review comment:
   Is this input supposed to be `postReify`? I would've expected `inputID` from the previous step.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-602817069
 
 
   I'm definitely not merging this until both the PostCommit runs, and someone more familiar with windowing/trigger semantics looks over the configuration I copied over from python:
   https://github.com/apache/beam/pull/11197/files#diff-ef420fdb9afbce0674282b4ed4481042R530

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-602805178
 
 
   R: @youngoli 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r399474243
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/graphx/translate.go
 ##########
 @@ -473,6 +479,164 @@ func (m *marshaller) addNode(n *graph.Node) string {
 	return m.makeNode(id, m.coders.Add(n.Coder), n)
 }
 
+// expandReshuffle translates resharding to a composite reshuffle
+// transform.
+//
+// With proper runner support, the SDK doesn't need to do anything.
+// However, we still need to provide a backup plan in terms of other
+// PTransforms in the event the runner doesn't have a native implementation.
+//
+// In particular, the "backup plan" needs to:
+//
+//  * Encode the windowed element, preserving timestamps.
+//  * Add random keys to the encoded windowed element []bytes
+//  * GroupByKey (in the global window).
+//  * Explode the resulting elements list.
+//  * Decode the windowed element []bytes.
+//
+// While a simple reshard can be written in user terms, (timestamps and windows
+// are accessible to user functions) there are some framework internal
+// optimizations that can be done if the framework is aware of the reshard, though
+// ideally this is handled on the runner side.
+//
+// User code is able to write reshards, but it's easier to access
+// the window coders framework side, which is critical for the reshard
+// to function with unbounded inputs.
+func (m *marshaller) expandReshuffle(edge NamedEdge) string {
+	id := edgeID(edge.Edge)
+	var kvCoderID, gbkCoderID string
+	{
+		kv := makeUnionCoder()
+		kvCoderID = m.coders.Add(kv)
+		gbkCoderID = m.coders.Add(coder.NewCoGBK(kv.Components))
+	}
+
+	var subtransforms []string
+
+	in := edge.Edge.Input[0]
+
+	origInput := m.addNode(in.From)
+	// We need to preserve the old windowing/triggering here
+	// for re-instatement after the GBK.
+	preservedWSId := m.pcollections[origInput].GetWindowingStrategyId()
+
+	// Get the windowing strategy from before:
+	postReify := fmt.Sprintf("%v_%v_reifyts", nodeID(in.From), id)
+	m.makeNode(postReify, kvCoderID, in.From)
+
+	// We need to replace postReify's windowing strategy with one appropriate
+	// for reshuffles.
+	{
+		wfn := window.NewGlobalWindows()
+		m.pcollections[postReify].WindowingStrategyId =
+			m.internWindowingStrategy(&pb.WindowingStrategy{
+				// Not segregated by time...
+				WindowFn: makeWindowFn(wfn),
+				// ...output after every element is received...
+				Trigger: &pb.Trigger{
+					// Should this be an Always trigger instead?
 
 Review comment:
   Thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-602750930
 
 
   Run Go Postcommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck merged pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck merged pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lukecwik commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lukecwik commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-603292995
 
 
   R: @reuvenlax 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-602835398
 
 
   Retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r397314344
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/exec/reshuffle.go
 ##########
 @@ -0,0 +1,170 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+	"bytes"
+	"context"
+	"fmt"
+	"io"
+	"math/rand"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+	"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
+)
+
+// ReshuffleInput is a Node.
+type ReshuffleInput struct {
+	UID   UnitID
+	SID   StreamID
+	Coder *coder.Coder // Coder for the input PCollection.
+	Seed  int64
+	Out   Node
+
+	r    *rand.Rand
+	enc  ElementEncoder
+	wEnc WindowEncoder
+	b    bytes.Buffer
+	// ret is a cached allocations for passing to the next Unit. Units never modify the passed in FullValue.
+	ret FullValue
+}
+
+// ID returns the unit debug id.
+func (n *ReshuffleInput) ID() UnitID {
+	return n.UID
+}
+
+// Up initializes the value and window encoders, and the random source.
+func (n *ReshuffleInput) Up(ctx context.Context) error {
+	n.enc = MakeElementEncoder(coder.SkipW(n.Coder))
+	n.wEnc = MakeWindowEncoder(n.Coder.Window)
+	n.r = rand.New(rand.NewSource(n.Seed))
+	return nil
+}
+
+// StartBundle is a no-op.
+func (n *ReshuffleInput) StartBundle(ctx context.Context, id string, data DataContext) error {
+	return MultiStartBundle(ctx, id, data, n.Out)
+}
+
+func (n *ReshuffleInput) ProcessElement(ctx context.Context, value *FullValue, values ...ReStream) error {
+	n.b.Reset()
+	if err := EncodeWindowedValueHeader(n.wEnc, value.Windows, value.Timestamp, &n.b); err != nil {
+		return err
+	}
+	if err := n.enc.Encode(value, &n.b); err != nil {
+		return errors.WithContextf(err, "encoding element %v with coder %v", value, n.Coder)
+	}
+	n.ret = FullValue{Elm: n.r.Int(), Elm2: n.b.Bytes(), Timestamp: value.Timestamp}
+	if err := n.Out.ProcessElement(ctx, &n.ret); err != nil {
+		return err
+	}
+	return nil
+}
+
+// FinishBundle propagates finish bundle, and clears cached state.
+func (n *ReshuffleInput) FinishBundle(ctx context.Context) error {
+	n.b = bytes.Buffer{}
+	n.ret = FullValue{}
+	return MultiFinishBundle(ctx, n.Out)
+}
+
+// Down is a no-op.
+func (n *ReshuffleInput) Down(ctx context.Context) error {
+	return nil
+}
+
+func (n *ReshuffleInput) String() string {
+	return fmt.Sprintf("ReshuffleInput[%v] Coder:%v", n.SID, n.Coder)
+}
+
+// ReshuffleOutput is a Node.
+type ReshuffleOutput struct {
+	UID   UnitID
+	SID   StreamID
+	Coder *coder.Coder // Coder for the receiving PCollection.
+	Out   Node
+
+	b    bytes.Buffer
+	dec  ElementDecoder
+	wDec WindowDecoder
+	ret  FullValue
+}
+
+// ID returns the unit debug id.
+func (n *ReshuffleOutput) ID() UnitID {
+	return n.UID
+}
+
+// Up initializes the value and window encoders, and the random source.
+func (n *ReshuffleOutput) Up(ctx context.Context) error {
+	n.dec = MakeElementDecoder(coder.SkipW(n.Coder))
+	n.wDec = MakeWindowDecoder(n.Coder.Window)
+	return nil
+}
+
+// StartBundle is a no-op.
+func (n *ReshuffleOutput) StartBundle(ctx context.Context, id string, data DataContext) error {
+	return MultiStartBundle(ctx, id, data, n.Out)
+}
+
+func (n *ReshuffleOutput) ProcessElement(ctx context.Context, value *FullValue, values ...ReStream) error {
+	// Marshal the pieces into a temporary buffer since they must be transmitted on FnAPI as a single
+	// unit.
+	vs, err := values[0].Open()
 
 Review comment:
   This is the Go SDK future proofing itself, against CoGBK supporting multiple datastreams from the runner. Functionally, only datasource.go would need to change in that case.  You can see a comment to that effect in datasource.go, and then nearly everything else deals with the values streams properly under that assumption. 
   
   In this case, we know that if this code is being used, it's coming from a single GBK, which means there's only a single stream of values, and then since we're framework side, we just handle the stream directly. In that way, it's similar to how we're handling CoGBKs presently, with synthetic inject and expand steps to get to the right number of joined streams, even though the Runner is only providing us with a single data stream for the grouped data.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-605214792
 
 
   Run Go Postcommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on issue #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#issuecomment-603372518
 
 
   Run Go Postcommit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r397315541
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/exec/reshuffle.go
 ##########
 @@ -0,0 +1,170 @@
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package exec
+
+import (
+	"bytes"
+	"context"
+	"fmt"
+	"io"
+	"math/rand"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+	"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
+)
+
+// ReshuffleInput is a Node.
+type ReshuffleInput struct {
+	UID   UnitID
+	SID   StreamID
+	Coder *coder.Coder // Coder for the input PCollection.
+	Seed  int64
+	Out   Node
+
+	r    *rand.Rand
+	enc  ElementEncoder
+	wEnc WindowEncoder
+	b    bytes.Buffer
+	// ret is a cached allocations for passing to the next Unit. Units never modify the passed in FullValue.
+	ret FullValue
+}
+
+// ID returns the unit debug id.
+func (n *ReshuffleInput) ID() UnitID {
+	return n.UID
+}
+
+// Up initializes the value and window encoders, and the random source.
+func (n *ReshuffleInput) Up(ctx context.Context) error {
+	n.enc = MakeElementEncoder(coder.SkipW(n.Coder))
+	n.wEnc = MakeWindowEncoder(n.Coder.Window)
+	n.r = rand.New(rand.NewSource(n.Seed))
+	return nil
+}
+
+// StartBundle is a no-op.
+func (n *ReshuffleInput) StartBundle(ctx context.Context, id string, data DataContext) error {
+	return MultiStartBundle(ctx, id, data, n.Out)
+}
+
+func (n *ReshuffleInput) ProcessElement(ctx context.Context, value *FullValue, values ...ReStream) error {
+	n.b.Reset()
+	if err := EncodeWindowedValueHeader(n.wEnc, value.Windows, value.Timestamp, &n.b); err != nil {
+		return err
+	}
+	if err := n.enc.Encode(value, &n.b); err != nil {
+		return errors.WithContextf(err, "encoding element %v with coder %v", value, n.Coder)
+	}
+	n.ret = FullValue{Elm: n.r.Int(), Elm2: n.b.Bytes(), Timestamp: value.Timestamp}
+	if err := n.Out.ProcessElement(ctx, &n.ret); err != nil {
+		return err
+	}
+	return nil
+}
+
+// FinishBundle propagates finish bundle, and clears cached state.
+func (n *ReshuffleInput) FinishBundle(ctx context.Context) error {
+	n.b = bytes.Buffer{}
+	n.ret = FullValue{}
+	return MultiFinishBundle(ctx, n.Out)
+}
+
+// Down is a no-op.
+func (n *ReshuffleInput) Down(ctx context.Context) error {
+	return nil
+}
+
+func (n *ReshuffleInput) String() string {
+	return fmt.Sprintf("ReshuffleInput[%v] Coder:%v", n.SID, n.Coder)
+}
+
+// ReshuffleOutput is a Node.
+type ReshuffleOutput struct {
+	UID   UnitID
+	SID   StreamID
+	Coder *coder.Coder // Coder for the receiving PCollection.
+	Out   Node
+
+	b    bytes.Buffer
+	dec  ElementDecoder
+	wDec WindowDecoder
+	ret  FullValue
+}
+
+// ID returns the unit debug id.
+func (n *ReshuffleOutput) ID() UnitID {
+	return n.UID
+}
+
+// Up initializes the value and window encoders, and the random source.
+func (n *ReshuffleOutput) Up(ctx context.Context) error {
+	n.dec = MakeElementDecoder(coder.SkipW(n.Coder))
+	n.wDec = MakeWindowDecoder(n.Coder.Window)
+	return nil
+}
+
+// StartBundle is a no-op.
+func (n *ReshuffleOutput) StartBundle(ctx context.Context, id string, data DataContext) error {
+	return MultiStartBundle(ctx, id, data, n.Out)
+}
+
+func (n *ReshuffleOutput) ProcessElement(ctx context.Context, value *FullValue, values ...ReStream) error {
+	// Marshal the pieces into a temporary buffer since they must be transmitted on FnAPI as a single
+	// unit.
+	vs, err := values[0].Open()
+	if err != nil {
+		return errors.WithContextf(err, "decoding values for %v with coder %v", value, n.Coder)
+	}
+	defer vs.Close()
+	for {
+		v, err := vs.Read()
+		if err != nil {
+			if err == io.EOF {
+				return nil
+			}
+			return errors.WithContextf(err, "reading values for %v", n)
+		}
+		n.b = *bytes.NewBuffer(v.Elm.([]byte))
+		ws, ts, err := DecodeWindowedValueHeader(n.wDec, &n.b)
+		if err != nil {
+			return errors.WithContextf(err, "decoding windows for %v", n)
+		}
+		if err := n.dec.DecodeTo(&n.b, &n.ret); err != nil {
+			return errors.WithContextf(err, "decoding element for %v", n)
+		}
+		n.ret.Windows = ws
+		n.ret.Timestamp = ts
+		if err := n.Out.ProcessElement(ctx, &n.ret); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// FinishBundle propagates finish bundle to downstream nodes.
+func (n *ReshuffleOutput) FinishBundle(ctx context.Context) error {
+	n.b = bytes.Buffer{}
 
 Review comment:
   Yes it should. Thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] dpmills commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
dpmills commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r399468862
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/graphx/translate.go
 ##########
 @@ -473,6 +479,164 @@ func (m *marshaller) addNode(n *graph.Node) string {
 	return m.makeNode(id, m.coders.Add(n.Coder), n)
 }
 
+// expandReshuffle translates resharding to a composite reshuffle
+// transform.
+//
+// With proper runner support, the SDK doesn't need to do anything.
+// However, we still need to provide a backup plan in terms of other
+// PTransforms in the event the runner doesn't have a native implementation.
+//
+// In particular, the "backup plan" needs to:
+//
+//  * Encode the windowed element, preserving timestamps.
+//  * Add random keys to the encoded windowed element []bytes
+//  * GroupByKey (in the global window).
+//  * Explode the resulting elements list.
+//  * Decode the windowed element []bytes.
+//
+// While a simple reshard can be written in user terms, (timestamps and windows
+// are accessible to user functions) there are some framework internal
+// optimizations that can be done if the framework is aware of the reshard, though
+// ideally this is handled on the runner side.
+//
+// User code is able to write reshards, but it's easier to access
+// the window coders framework side, which is critical for the reshard
+// to function with unbounded inputs.
+func (m *marshaller) expandReshuffle(edge NamedEdge) string {
+	id := edgeID(edge.Edge)
+	var kvCoderID, gbkCoderID string
+	{
+		kv := makeUnionCoder()
+		kvCoderID = m.coders.Add(kv)
+		gbkCoderID = m.coders.Add(coder.NewCoGBK(kv.Components))
+	}
+
+	var subtransforms []string
+
+	in := edge.Edge.Input[0]
+
+	origInput := m.addNode(in.From)
+	// We need to preserve the old windowing/triggering here
+	// for re-instatement after the GBK.
+	preservedWSId := m.pcollections[origInput].GetWindowingStrategyId()
+
+	// Get the windowing strategy from before:
+	postReify := fmt.Sprintf("%v_%v_reifyts", nodeID(in.From), id)
+	m.makeNode(postReify, kvCoderID, in.From)
+
+	// We need to replace postReify's windowing strategy with one appropriate
+	// for reshuffles.
+	{
+		wfn := window.NewGlobalWindows()
+		m.pcollections[postReify].WindowingStrategyId =
+			m.internWindowingStrategy(&pb.WindowingStrategy{
+				// Not segregated by time...
+				WindowFn: makeWindowFn(wfn),
+				// ...output after every element is received...
+				Trigger: &pb.Trigger{
+					// Should this be an Always trigger instead?
 
 Review comment:
   In answer to this comment, yes it should be Always

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
youngoli commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r396887966
 
 

 ##########
 File path: sdks/go/pkg/beam/gbk.go
 ##########
 @@ -95,3 +95,52 @@ func TryCoGroupByKey(s Scope, cols ...PCollection) (PCollection, error) {
 	ret.SetCoder(NewCoder(ret.Type()))
 	return ret, nil
 }
+
+// Reshuffle copies a PCollection of the same kind and using the same element
+// coder, and maintains the same windowing information. Importantly, it allows
+// the result PCollection to be processed with a different sharding, in a
+// different stage than the input PCollection.
+//
+// For example, if a computation needs a lot of parallelism but
+// produces only a small amount of output data, then the computation
+// producing the data can run with as much parallelism as needed,
+// while the output file is written with a smaller amount of
+// parallelism, using the following pattern:
+//
+//   pc := bigHairyComputationNeedingParallelism(scope) // PCollection<string>
+//   resharded := beam.Reshard(scope, pc)                // PCollection<string>
+//
+// Another use case is when one has a non-deterministic DoFn followed by one
+// that performs externally-visible side effects. Inserting a Reshard
+// between these DoFns ensures that retries of the second DoFn will always be
+// the same, which is necessary to make side effects idempotent.
+//
+// A Reshuffle will force a break in the optimized pipeline. Consequently,
+// this operation should be used sparingly, only after determining that the
+// pipeline without reshard is broken in some way and performing an extra
+// operation is worth the cost.
+func Reshuffle(s Scope, col PCollection) PCollection {
+	return Must(TryReshuffle(s, col))
+}
+
+// TryReshuffle inserts a Reshard into the pipeline, and returns an error if
 
 Review comment:
   Same as previous comment, using Reshard instead of Reshuffle. The error message a few lines below also does that.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [beam] lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #11197: [BEAM-8292] Portable Reshuffle for Go SDK
URL: https://github.com/apache/beam/pull/11197#discussion_r397305306
 
 

 ##########
 File path: sdks/go/pkg/beam/core/runtime/graphx/translate.go
 ##########
 @@ -473,6 +479,164 @@ func (m *marshaller) addNode(n *graph.Node) string {
 	return m.makeNode(id, m.coders.Add(n.Coder), n)
 }
 
+// expandReshuffle translates resharding to a composite reshuffle
+// transform.
+//
+// With proper runner support, the SDK doesn't need to do anything.
+// However, we still need to provide a backup plan in terms of other
+// PTransforms in the event the runner doesn't have a native implementation.
+//
+// In particular, the "backup plan" needs to:
+//
+//  * Encode the windowed element, preserving timestamps.
+//  * Add random keys to the encoded windowed element []bytes
+//  * GroupByKey (in the global window).
+//  * Explode the resulting elements list.
+//  * Decode the windowed element []bytes.
+//
+// While a simple reshard can be written in user terms, (timestamps and windows
+// are accessible to user functions) there are some framework internal
+// optimizations that can be done if the framework is aware of the reshard, though
+// ideally this is handled on the runner side.
+//
+// User code is able to write reshards, but it's easier to access
+// the window coders framework side, which is critical for the reshard
+// to function with unbounded inputs.
+func (m *marshaller) expandReshuffle(edge NamedEdge) string {
+	id := edgeID(edge.Edge)
+	var kvCoderID, gbkCoderID string
+	{
+		kv := makeUnionCoder()
+		kvCoderID = m.coders.Add(kv)
+		gbkCoderID = m.coders.Add(coder.NewCoGBK(kv.Components))
+	}
+
+	var subtransforms []string
+
+	in := edge.Edge.Input[0]
+
+	origInput := m.addNode(in.From)
+	// We need to preserve the old windowing/triggering here
+	// for re-instatement after the GBK.
+	preservedWSId := m.pcollections[origInput].GetWindowingStrategyId()
+
+	// Get the windowing strategy from before:
+	postReify := fmt.Sprintf("%v_%v_reifyts", nodeID(in.From), id)
+	m.makeNode(postReify, kvCoderID, in.From)
+
+	// We need to replace postReify's windowing strategy with one appropriate
+	// for reshuffles.
+	{
+		wfn := window.NewGlobalWindows()
+		m.pcollections[postReify].WindowingStrategyId =
+			m.internWindowingStrategy(&pb.WindowingStrategy{
+				// Not segregated by time...
+				WindowFn: makeWindowFn(wfn),
+				// ...output after every element is received...
+				Trigger: &pb.Trigger{
+					// Should this be an Always trigger instead?
+					Trigger: &pb.Trigger_ElementCount_{
+						ElementCount: &pb.Trigger_ElementCount{
+							ElementCount: 1,
+						},
+					},
+				},
+				// ...and after outputing, discard the output elements...
+				AccumulationMode: pb.AccumulationMode_DISCARDING,
+				// ...and since every pane should have 1 element,
+				// try to preserve the timestamp.
+				OutputTime: pb.OutputTime_EARLIEST_IN_PANE,
+				// Defaults copied from marshalWindowingStrategy.
+				// TODO(BEAM-3304): migrate to user side operations once trigger support is in.
+				EnvironmentId:   m.addDefaultEnv(),
+				MergeStatus:     pb.MergeStatus_NON_MERGING,
+				WindowCoderId:   m.coders.AddWindowCoder(makeWindowCoder(wfn)),
+				ClosingBehavior: pb.ClosingBehavior_EMIT_IF_NONEMPTY,
+				AllowedLateness: 0,
+				OnTimeBehavior:  pb.OnTimeBehavior_FIRE_ALWAYS,
+			})
+	}
+
+	// Inputs (i)
+
+	inputID := fmt.Sprintf("%v_reifyts", id)
+	payload := &pb.ParDoPayload{
+		DoFn: &pb.FunctionSpec{
+			Urn: URNReshuffleInput,
+			Payload: []byte(protox.MustEncodeBase64(&v1.TransformPayload{
+				Urn: URNReshuffleInput,
+			})),
+		},
+	}
+	input := &pb.PTransform{
+		UniqueName: inputID,
+		Spec: &pb.FunctionSpec{
+			Urn:     URNParDo,
+			Payload: protox.MustEncode(payload),
+		},
+		Inputs:        map[string]string{"i0": nodeID(in.From)},
+		Outputs:       map[string]string{"i0": postReify},
+		EnvironmentId: m.addDefaultEnv(),
+	}
+	m.transforms[inputID] = input
+	subtransforms = append(subtransforms, inputID)
+
+	outNode := edge.Edge.Output[0].To
+
+	// GBK
+
+	gbkOut := fmt.Sprintf("%v_out", nodeID(outNode))
+	m.makeNode(gbkOut, gbkCoderID, outNode)
+
+	gbkID := fmt.Sprintf("%v_gbk", id)
+	gbk := &pb.PTransform{
+		UniqueName: gbkID,
+		Spec:       &pb.FunctionSpec{Urn: URNGBK},
+		Inputs:     map[string]string{"i0": postReify},
 
 Review comment:
   Yup. postReify is the PCollection that has the random keys and the full windowed value as serialized bytes.  
   The input from the previous step is used on line 577. 
   in represents strictly inbound data (and specifically the main input), and From indicates the "Node" in the graph. In this model of the pipeline PCollections are Nodes, and Transforms are Edges. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services