lostluck commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r680188647



##########
File path: sdks/go/pkg/beam/testing/teststream/teststream.go
##########
@@ -0,0 +1,150 @@
+// 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 teststream contains code configuring the TestStream primitive for
+// use in testing code that is meant to be run on streaming data sources.
+// See https://beam.apache.org/blog/test-stream/ for more information.
+// TestStream is supported on Flink.

Review comment:
       Blank lines here will make it render more readably.
   
   ```suggestion
   // use in testing code that is meant to be run on streaming data sources.
   //
   // See https://beam.apache.org/blog/test-stream/ for more information.
   //
   // TestStream is supported on the Flink runner.
   ```

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream.go
##########
@@ -0,0 +1,131 @@
+// 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 teststream contains code configuring the TestStream primitive for
+// use in testing code that is meant to be run on streaming data sources.
+// TestStream is not supported on the Go direct runner.
+package teststream
+
+import (
+       "bytes"
+       "fmt"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/protox"
+
+       pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+)
+
+const urn = "beam:transform:teststream:v1"
+
+// Config holds information used to create a TestStreamPayload object.
+type Config struct {
+       ElmCoder  *coder.Coder
+       Events    []*pipepb.TestStreamPayload_Event
+       Endpoint  *pipepb.ApiServiceDescriptor
+       Watermark int64
+}
+
+// MakeConfig initializes a Config struct to begin inserting TestStream 
events/endpoints into.
+// Requires that users provide the coder for the elements they are trying to 
emit.
+func MakeConfig(c *coder.Coder) Config {
+       return Config{ElmCoder: c,
+               Events:    []*pipepb.TestStreamPayload_Event{},
+               Endpoint:  &pipepb.ApiServiceDescriptor{},
+               Watermark: 0,
+       }
+}
+
+// SetEndpoint sets a URL for a TestStreamService that will emit events 
instead of having them
+// defined manually. Currently does not support authentication, so the 
TestStreamService should
+// be accessed in a trusted context.
+func (c *Config) SetEndpoint(url string) {

Review comment:
       Orthogonality is nice, but will delay the error case to later, 
dislocating it from where the mistake occured.
   
   Alternatively, we could have a separate constructor function for Config that 
takes in the EndPoint and Coder so it avoids the 2 calls, and allows a user to 
inline the configuration in the teststream.Create call. Or even further, we 
just have a `CreateWithEndpoint(beam.Scope, endpoint string, c coder.Coder) 
beam.PCollection` call which avoids users from being able to misconfigure 
something (mixing endpoints and events in a config). Config could still handle 
these details for serializing as an implementation detail, but only the 
`CreateWithEndpoint` method can access and modify them before serialization. 
Basically, we can ensure correctness by construction, by preventing users from 
building something in-advisable.
   
   Orthogonality is great, but unless all the options make sense with each 
other, it leaves confusing spaces for users to make cofiguration mistakes.

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream_test.go
##########
@@ -0,0 +1,121 @@
+// 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 teststream
+
+import (
+       "bytes"
+       "testing"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+)
+
+func TestMakeConfig(t *testing.T) {

Review comment:
       ```suggestion
   func TestNewConfig(t *testing.T) {
   ```

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream_test.go
##########
@@ -0,0 +1,121 @@
+// 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 teststream
+
+import (
+       "bytes"
+       "testing"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+)
+
+func TestMakeConfig(t *testing.T) {
+       con := NewConfig()
+       if con.elmCoder != nil {
+               t.Fatalf("coder is not correct, expected nil, got %v", 
con.elmCoder.Kind)
+       }
+       if len(con.events) != 0 {
+               t.Fatalf("config has too many elements, expected 0, got %v", 
len(con.events))

Review comment:
       These should both be `t.Errorf` calls instead of `t.Fatalf`. Basically, 
if a test can keep going and provide useful output, it should keep going. These 
a failure's don't block or change each other, so we can give ourselves more 
information about what expectations are broken all at once.

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream.go
##########
@@ -0,0 +1,150 @@
+// 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 teststream contains code configuring the TestStream primitive for
+// use in testing code that is meant to be run on streaming data sources.
+// See https://beam.apache.org/blog/test-stream/ for more information.
+// TestStream is supported on Flink.
+package teststream
+
+import (
+       "bytes"
+       "fmt"
+       "reflect"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/protox"
+
+       pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+)
+
+const urn = "beam:transform:teststream:v1"
+
+// Config holds information used to create a TestStreamPayload object.
+type Config struct {
+       elmCoder  *coder.Coder
+       events    []*pipepb.TestStreamPayload_Event
+       endpoint  *pipepb.ApiServiceDescriptor
+       watermark int64
+}
+
+// MakeConfig initializes a Config struct to begin inserting TestStream 
events/endpoints into.
+// Requires that users provide the coder for the elements they are trying to 
emit.
+func NewConfig() Config {
+       return Config{elmCoder: nil,
+               events:    []*pipepb.TestStreamPayload_Event{},
+               endpoint:  &pipepb.ApiServiceDescriptor{},
+               watermark: 0,
+       }
+}
+
+// SetEndpoint sets a URL for a TestStreamService that will emit events 
instead of having them
+// defined manually. Currently does not support authentication, so the 
TestStreamService should
+// be accessed in a trusted context.
+func (c *Config) SetEndpoint(url string) {
+       c.endpoint.Url = url
+}
+
+// createPayload converts the Config object into a TestStreamPayload to be 
sent to the runner.
+func (c *Config) createPayload() *pipepb.TestStreamPayload {
+       return &pipepb.TestStreamPayload{CoderId: "c0", Events: c.events, 
Endpoint: c.endpoint}
+}
+
+// AdvanceWatermark adds an event to the Config Events struct advancing the 
watermark for the PCollection
+// to the given timestamp. Timestamp is in milliseconds
+func (c *Config) AdvanceWatermark(timestamp int64) error {
+       if c.watermark >= timestamp {
+               return fmt.Errorf("watermark must be monotonally increasing, is 
at %v, got %v", c.watermark, timestamp)
+       }
+       watermarkAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceWatermark{NewWatermark: timestamp}
+       watermarkEvent := 
&pipepb.TestStreamPayload_Event_WatermarkEvent{WatermarkEvent: watermarkAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
watermarkEvent})
+       c.watermark = timestamp
+       return nil
+}
+
+// AdvanceWatermarkToInfinity advances the watermark to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+       return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event advancing the processing time by a 
given duration.
+// This advancement is applied to all of the PCollections output by the 
TestStream.
+func (c *Config) AdvanceProcessingTime(duration int64) {
+       processingAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceProcessingTime{AdvanceDuration: duration}
+       processingEvent := 
&pipepb.TestStreamPayload_Event_ProcessingTimeEvent{ProcessingTimeEvent: 
processingAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
processingEvent})
+}
+
+// AdvanceProcessingTimeToInfinity moves the TestStream processing time to the 
largest possible
+// timestamp.
+func (c *Config) AdvanceProcessingTimeToInfinity() {
+       c.AdvanceProcessingTime(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AddElements adds a number of elements to the Config object at the specified 
event timestamp.
+// The encoder will panic if there is a type mismatch between the provided 
coder and the
+// elements.
+func (c *Config) AddElements(timestamp int64, elements ...interface{}) error {
+       if c.elmCoder == nil {
+               elmType := reflect.TypeOf(elements[0])

Review comment:
       I'm open to suggestions on how we can make it easier for external to 
beam packages to be able to do something like this, short of exposing 
everything. Essentially the risk being managed is users mis-assuming that 
certain hoops need to be jumped through to get something working....

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream_test.go
##########
@@ -0,0 +1,121 @@
+// 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 teststream
+
+import (
+       "bytes"
+       "testing"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+)
+
+func TestMakeConfig(t *testing.T) {
+       con := NewConfig()
+       if con.elmCoder != nil {
+               t.Fatalf("coder is not correct, expected nil, got %v", 
con.elmCoder.Kind)
+       }
+       if len(con.events) != 0 {
+               t.Fatalf("config has too many elements, expected 0, got %v", 
len(con.events))
+       }
+       if con.endpoint.Url != "" {
+               t.Errorf("config has URL endpoint when it should be empty")
+       }
+}
+
+func TestAdvanceWatermark(t *testing.T) {
+       con := NewConfig()
+       con.AdvanceWatermark(500)
+       if w := con.watermark; w != 500 {
+               t.Errorf("default watermark expected 500, got %v", w)
+       }
+       if len(con.events) != 1 {
+               t.Errorf("expected only 1 event in config, got %v", 
len(con.events))

Review comment:
       Meanwhile here, a t.Fatalf would be better because the next check 
requires at least one element, or the code will panic, preventing clean test 
output in that failure.

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream.go
##########
@@ -0,0 +1,150 @@
+// 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 teststream contains code configuring the TestStream primitive for
+// use in testing code that is meant to be run on streaming data sources.
+// See https://beam.apache.org/blog/test-stream/ for more information.
+// TestStream is supported on Flink.
+package teststream
+
+import (
+       "bytes"
+       "fmt"
+       "reflect"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/protox"
+
+       pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+)
+
+const urn = "beam:transform:teststream:v1"
+
+// Config holds information used to create a TestStreamPayload object.
+type Config struct {
+       elmCoder  *coder.Coder
+       events    []*pipepb.TestStreamPayload_Event
+       endpoint  *pipepb.ApiServiceDescriptor
+       watermark int64
+}
+
+// MakeConfig initializes a Config struct to begin inserting TestStream 
events/endpoints into.
+// Requires that users provide the coder for the elements they are trying to 
emit.
+func NewConfig() Config {
+       return Config{elmCoder: nil,
+               events:    []*pipepb.TestStreamPayload_Event{},
+               endpoint:  &pipepb.ApiServiceDescriptor{},
+               watermark: 0,
+       }
+}
+
+// SetEndpoint sets a URL for a TestStreamService that will emit events 
instead of having them
+// defined manually. Currently does not support authentication, so the 
TestStreamService should
+// be accessed in a trusted context.
+func (c *Config) SetEndpoint(url string) {
+       c.endpoint.Url = url
+}
+
+// createPayload converts the Config object into a TestStreamPayload to be 
sent to the runner.
+func (c *Config) createPayload() *pipepb.TestStreamPayload {
+       return &pipepb.TestStreamPayload{CoderId: "c0", Events: c.events, 
Endpoint: c.endpoint}
+}
+
+// AdvanceWatermark adds an event to the Config Events struct advancing the 
watermark for the PCollection
+// to the given timestamp. Timestamp is in milliseconds
+func (c *Config) AdvanceWatermark(timestamp int64) error {
+       if c.watermark >= timestamp {
+               return fmt.Errorf("watermark must be monotonally increasing, is 
at %v, got %v", c.watermark, timestamp)
+       }
+       watermarkAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceWatermark{NewWatermark: timestamp}
+       watermarkEvent := 
&pipepb.TestStreamPayload_Event_WatermarkEvent{WatermarkEvent: watermarkAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
watermarkEvent})
+       c.watermark = timestamp
+       return nil
+}
+
+// AdvanceWatermarkToInfinity advances the watermark to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+       return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event advancing the processing time by a 
given duration.
+// This advancement is applied to all of the PCollections output by the 
TestStream.
+func (c *Config) AdvanceProcessingTime(duration int64) {
+       processingAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceProcessingTime{AdvanceDuration: duration}
+       processingEvent := 
&pipepb.TestStreamPayload_Event_ProcessingTimeEvent{ProcessingTimeEvent: 
processingAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
processingEvent})
+}
+
+// AdvanceProcessingTimeToInfinity moves the TestStream processing time to the 
largest possible
+// timestamp.
+func (c *Config) AdvanceProcessingTimeToInfinity() {
+       c.AdvanceProcessingTime(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AddElements adds a number of elements to the Config object at the specified 
event timestamp.
+// The encoder will panic if there is a type mismatch between the provided 
coder and the
+// elements.

Review comment:
       Consider the following documentation:
   
   ```suggestion
   // AddElements adds a number of elements to the stream at the specified 
event timestamp. Must be called with
   // at least one element.
   //
   // On the first call, a coder will be inferred from the passed in elements, 
which must be of all the same type.
   // Type mismatches on this or subsequent calls will cause AddElements to 
return an error.
   ```
   
   So a couple things about this suggestion. 
   
   1. Since the method signature doesn't require at least one element. (It 
would need to be `AddElements(timestamp int64, element interface{}, elements 
...interface{})` for the compiler to require it at least one, but we don't need 
to go that far TBH, we can return an error.)
   
   2.  It tells the user that it's going to keep them from making a weird 
mistake like mixing element types, and that it's going to be inferring the 
coder from the type.
   
   

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream.go
##########
@@ -0,0 +1,150 @@
+// 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 teststream contains code configuring the TestStream primitive for
+// use in testing code that is meant to be run on streaming data sources.
+// See https://beam.apache.org/blog/test-stream/ for more information.
+// TestStream is supported on Flink.
+package teststream
+
+import (
+       "bytes"
+       "fmt"
+       "reflect"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/protox"
+
+       pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+)
+
+const urn = "beam:transform:teststream:v1"
+
+// Config holds information used to create a TestStreamPayload object.
+type Config struct {
+       elmCoder  *coder.Coder
+       events    []*pipepb.TestStreamPayload_Event
+       endpoint  *pipepb.ApiServiceDescriptor
+       watermark int64
+}
+
+// MakeConfig initializes a Config struct to begin inserting TestStream 
events/endpoints into.
+// Requires that users provide the coder for the elements they are trying to 
emit.
+func NewConfig() Config {
+       return Config{elmCoder: nil,
+               events:    []*pipepb.TestStreamPayload_Event{},
+               endpoint:  &pipepb.ApiServiceDescriptor{},
+               watermark: 0,
+       }
+}
+
+// SetEndpoint sets a URL for a TestStreamService that will emit events 
instead of having them
+// defined manually. Currently does not support authentication, so the 
TestStreamService should
+// be accessed in a trusted context.
+func (c *Config) SetEndpoint(url string) {
+       c.endpoint.Url = url
+}
+
+// createPayload converts the Config object into a TestStreamPayload to be 
sent to the runner.
+func (c *Config) createPayload() *pipepb.TestStreamPayload {
+       return &pipepb.TestStreamPayload{CoderId: "c0", Events: c.events, 
Endpoint: c.endpoint}
+}
+
+// AdvanceWatermark adds an event to the Config Events struct advancing the 
watermark for the PCollection
+// to the given timestamp. Timestamp is in milliseconds
+func (c *Config) AdvanceWatermark(timestamp int64) error {
+       if c.watermark >= timestamp {
+               return fmt.Errorf("watermark must be monotonally increasing, is 
at %v, got %v", c.watermark, timestamp)
+       }
+       watermarkAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceWatermark{NewWatermark: timestamp}
+       watermarkEvent := 
&pipepb.TestStreamPayload_Event_WatermarkEvent{WatermarkEvent: watermarkAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
watermarkEvent})
+       c.watermark = timestamp
+       return nil
+}
+
+// AdvanceWatermarkToInfinity advances the watermark to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+       return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event advancing the processing time by a 
given duration.
+// This advancement is applied to all of the PCollections output by the 
TestStream.
+func (c *Config) AdvanceProcessingTime(duration int64) {
+       processingAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceProcessingTime{AdvanceDuration: duration}
+       processingEvent := 
&pipepb.TestStreamPayload_Event_ProcessingTimeEvent{ProcessingTimeEvent: 
processingAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
processingEvent})
+}
+
+// AdvanceProcessingTimeToInfinity moves the TestStream processing time to the 
largest possible
+// timestamp.
+func (c *Config) AdvanceProcessingTimeToInfinity() {
+       c.AdvanceProcessingTime(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AddElements adds a number of elements to the Config object at the specified 
event timestamp.
+// The encoder will panic if there is a type mismatch between the provided 
coder and the
+// elements.
+func (c *Config) AddElements(timestamp int64, elements ...interface{}) error {
+       if c.elmCoder == nil {
+               elmType := reflect.TypeOf(elements[0])
+               var newCoder *coder.Coder
+               switch elmType.Kind() {
+               case reflect.Bool:
+                       newCoder = coder.NewBool()
+               case reflect.String:
+                       newCoder = coder.NewString()
+               case reflect.Float32, reflect.Float64:
+                       newCoder = coder.NewDouble()
+               case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, 
reflect.Int64:
+                       newCoder = coder.NewVarInt()
+               default:
+                       return fmt.Errorf("unsupported element type %v", 
elmType.Kind())
+               }
+               c.elmCoder = newCoder
+       }
+       newElements := []*pipepb.TestStreamPayload_TimestampedElement{}
+       enc := beam.NewElementEncoder(c.elmCoder.T.Type())
+       for _, e := range elements {
+               var buf bytes.Buffer
+               if err := enc.Encode(e, &buf); err != nil {
+                       return fmt.Errorf("encoding value %v failed, got %v", 
e, err)
+               }
+               newElements = append(newElements, 
&pipepb.TestStreamPayload_TimestampedElement{EncodedElement: buf.Bytes(), 
Timestamp: timestamp})
+       }
+       addElementsEvent := 
&pipepb.TestStreamPayload_Event_AddElements{Elements: newElements}
+       elementEvent := 
&pipepb.TestStreamPayload_Event_ElementEvent{ElementEvent: addElementsEvent}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
elementEvent})
+       return nil
+}
+
+// Create inserts a TestStream primitive into a pipeline, taking a scope and a 
Config object and
+// producing an array of output PCollections. The TestStream must be the first 
PTransform in the
+// pipeline.
+func Create(s beam.Scope, c Config) beam.PCollection {
+       pyld := protox.MustEncode(c.createPayload())
+       outputs := []beam.FullType{c.elmCoder.T}
+
+       outputMap := beam.External(s, urn, pyld, []beam.PCollection{}, outputs, 
false)

Review comment:
       Some cleanup from moving to `beam.External` from `beam.ExternalTagged`. 
`beam.External` already returns a `[]beam.PCollection`. Since the outputs are 
dictated by the the passed in slice of fulltypes, it should already only return 
a length 1 slice.

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream.go
##########
@@ -0,0 +1,150 @@
+// 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 teststream contains code configuring the TestStream primitive for
+// use in testing code that is meant to be run on streaming data sources.
+// See https://beam.apache.org/blog/test-stream/ for more information.
+// TestStream is supported on Flink.
+package teststream
+
+import (
+       "bytes"
+       "fmt"
+       "reflect"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/protox"
+
+       pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+)
+
+const urn = "beam:transform:teststream:v1"
+
+// Config holds information used to create a TestStreamPayload object.
+type Config struct {
+       elmCoder  *coder.Coder
+       events    []*pipepb.TestStreamPayload_Event
+       endpoint  *pipepb.ApiServiceDescriptor
+       watermark int64
+}
+
+// MakeConfig initializes a Config struct to begin inserting TestStream 
events/endpoints into.
+// Requires that users provide the coder for the elements they are trying to 
emit.
+func NewConfig() Config {
+       return Config{elmCoder: nil,
+               events:    []*pipepb.TestStreamPayload_Event{},
+               endpoint:  &pipepb.ApiServiceDescriptor{},
+               watermark: 0,
+       }
+}
+
+// SetEndpoint sets a URL for a TestStreamService that will emit events 
instead of having them
+// defined manually. Currently does not support authentication, so the 
TestStreamService should
+// be accessed in a trusted context.
+func (c *Config) SetEndpoint(url string) {
+       c.endpoint.Url = url
+}
+
+// createPayload converts the Config object into a TestStreamPayload to be 
sent to the runner.
+func (c *Config) createPayload() *pipepb.TestStreamPayload {
+       return &pipepb.TestStreamPayload{CoderId: "c0", Events: c.events, 
Endpoint: c.endpoint}
+}
+
+// AdvanceWatermark adds an event to the Config Events struct advancing the 
watermark for the PCollection
+// to the given timestamp. Timestamp is in milliseconds
+func (c *Config) AdvanceWatermark(timestamp int64) error {
+       if c.watermark >= timestamp {
+               return fmt.Errorf("watermark must be monotonally increasing, is 
at %v, got %v", c.watermark, timestamp)
+       }
+       watermarkAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceWatermark{NewWatermark: timestamp}
+       watermarkEvent := 
&pipepb.TestStreamPayload_Event_WatermarkEvent{WatermarkEvent: watermarkAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
watermarkEvent})
+       c.watermark = timestamp
+       return nil
+}
+
+// AdvanceWatermarkToInfinity advances the watermark to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+       return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event advancing the processing time by a 
given duration.
+// This advancement is applied to all of the PCollections output by the 
TestStream.
+func (c *Config) AdvanceProcessingTime(duration int64) {
+       processingAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceProcessingTime{AdvanceDuration: duration}
+       processingEvent := 
&pipepb.TestStreamPayload_Event_ProcessingTimeEvent{ProcessingTimeEvent: 
processingAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
processingEvent})
+}
+
+// AdvanceProcessingTimeToInfinity moves the TestStream processing time to the 
largest possible
+// timestamp.
+func (c *Config) AdvanceProcessingTimeToInfinity() {
+       c.AdvanceProcessingTime(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AddElements adds a number of elements to the Config object at the specified 
event timestamp.
+// The encoder will panic if there is a type mismatch between the provided 
coder and the
+// elements.
+func (c *Config) AddElements(timestamp int64, elements ...interface{}) error {
+       if c.elmCoder == nil {
+               elmType := reflect.TypeOf(elements[0])

Review comment:
       I'm increasingly annoyed that the node needs to know the coder it's got. 
It doesn't seem like it would need to since that information is on the 
subsequent PCollection. But it's also not great that we need to duplicate coder 
inference here either.
   
   One could work around this by having a 2nd DoFn that actually contains the 
serialized data and keeps it in specified order. Then instead of the real data, 
the TestStream is only given successive int64s with the time stamps. Means we 
always can pass in coder.NewVarInt(), and the 2nd DoFn spits out whatever 
element the returned index has from it's cache. Basically the 2nd DoFn ends up 
as a variant on `beam.Create`'s implementation. 
   
   Users as a rule won't be able to tell the difference, and it would resolve a 
small chunk of the issues we're having without moving this to the `beam` 
package with it's own Graph and graphx plumbing.

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream.go
##########
@@ -0,0 +1,150 @@
+// 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 teststream contains code configuring the TestStream primitive for
+// use in testing code that is meant to be run on streaming data sources.
+// See https://beam.apache.org/blog/test-stream/ for more information.
+// TestStream is supported on Flink.
+package teststream
+
+import (
+       "bytes"
+       "fmt"
+       "reflect"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/protox"
+
+       pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+)
+
+const urn = "beam:transform:teststream:v1"
+
+// Config holds information used to create a TestStreamPayload object.
+type Config struct {
+       elmCoder  *coder.Coder
+       events    []*pipepb.TestStreamPayload_Event
+       endpoint  *pipepb.ApiServiceDescriptor
+       watermark int64
+}
+
+// MakeConfig initializes a Config struct to begin inserting TestStream 
events/endpoints into.

Review comment:
       ```suggestion
   // NewConfig returns a Config to build a sequence of a test stream's events.
   ```

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream.go
##########
@@ -0,0 +1,150 @@
+// 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 teststream contains code configuring the TestStream primitive for
+// use in testing code that is meant to be run on streaming data sources.
+// See https://beam.apache.org/blog/test-stream/ for more information.
+// TestStream is supported on Flink.
+package teststream
+
+import (
+       "bytes"
+       "fmt"
+       "reflect"
+
+       "github.com/apache/beam/sdks/go/pkg/beam"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/coder"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+       "github.com/apache/beam/sdks/go/pkg/beam/core/util/protox"
+
+       pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+)
+
+const urn = "beam:transform:teststream:v1"
+
+// Config holds information used to create a TestStreamPayload object.
+type Config struct {
+       elmCoder  *coder.Coder
+       events    []*pipepb.TestStreamPayload_Event
+       endpoint  *pipepb.ApiServiceDescriptor
+       watermark int64
+}
+
+// MakeConfig initializes a Config struct to begin inserting TestStream 
events/endpoints into.
+// Requires that users provide the coder for the elements they are trying to 
emit.
+func NewConfig() Config {
+       return Config{elmCoder: nil,
+               events:    []*pipepb.TestStreamPayload_Event{},
+               endpoint:  &pipepb.ApiServiceDescriptor{},
+               watermark: 0,
+       }
+}
+
+// SetEndpoint sets a URL for a TestStreamService that will emit events 
instead of having them
+// defined manually. Currently does not support authentication, so the 
TestStreamService should
+// be accessed in a trusted context.
+func (c *Config) SetEndpoint(url string) {
+       c.endpoint.Url = url
+}
+
+// createPayload converts the Config object into a TestStreamPayload to be 
sent to the runner.
+func (c *Config) createPayload() *pipepb.TestStreamPayload {
+       return &pipepb.TestStreamPayload{CoderId: "c0", Events: c.events, 
Endpoint: c.endpoint}
+}
+
+// AdvanceWatermark adds an event to the Config Events struct advancing the 
watermark for the PCollection
+// to the given timestamp. Timestamp is in milliseconds
+func (c *Config) AdvanceWatermark(timestamp int64) error {
+       if c.watermark >= timestamp {
+               return fmt.Errorf("watermark must be monotonally increasing, is 
at %v, got %v", c.watermark, timestamp)
+       }
+       watermarkAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceWatermark{NewWatermark: timestamp}
+       watermarkEvent := 
&pipepb.TestStreamPayload_Event_WatermarkEvent{WatermarkEvent: watermarkAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
watermarkEvent})
+       c.watermark = timestamp
+       return nil
+}
+
+// AdvanceWatermarkToInfinity advances the watermark to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+       return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event advancing the processing time by a 
given duration.
+// This advancement is applied to all of the PCollections output by the 
TestStream.
+func (c *Config) AdvanceProcessingTime(duration int64) {
+       processingAdvance := 
&pipepb.TestStreamPayload_Event_AdvanceProcessingTime{AdvanceDuration: duration}
+       processingEvent := 
&pipepb.TestStreamPayload_Event_ProcessingTimeEvent{ProcessingTimeEvent: 
processingAdvance}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
processingEvent})
+}
+
+// AdvanceProcessingTimeToInfinity moves the TestStream processing time to the 
largest possible
+// timestamp.
+func (c *Config) AdvanceProcessingTimeToInfinity() {
+       c.AdvanceProcessingTime(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AddElements adds a number of elements to the Config object at the specified 
event timestamp.
+// The encoder will panic if there is a type mismatch between the provided 
coder and the
+// elements.
+func (c *Config) AddElements(timestamp int64, elements ...interface{}) error {
+       if c.elmCoder == nil {
+               elmType := reflect.TypeOf(elements[0])
+               var newCoder *coder.Coder
+               switch elmType.Kind() {
+               case reflect.Bool:
+                       newCoder = coder.NewBool()
+               case reflect.String:
+                       newCoder = coder.NewString()
+               case reflect.Float32, reflect.Float64:
+                       newCoder = coder.NewDouble()
+               case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, 
reflect.Int64:
+                       newCoder = coder.NewVarInt()
+               default:
+                       return fmt.Errorf("unsupported element type %v", 
elmType.Kind())
+               }
+               c.elmCoder = newCoder
+       }
+       newElements := []*pipepb.TestStreamPayload_TimestampedElement{}
+       enc := beam.NewElementEncoder(c.elmCoder.T.Type())
+       for _, e := range elements {
+               var buf bytes.Buffer
+               if err := enc.Encode(e, &buf); err != nil {
+                       return fmt.Errorf("encoding value %v failed, got %v", 
e, err)
+               }
+               newElements = append(newElements, 
&pipepb.TestStreamPayload_TimestampedElement{EncodedElement: buf.Bytes(), 
Timestamp: timestamp})
+       }
+       addElementsEvent := 
&pipepb.TestStreamPayload_Event_AddElements{Elements: newElements}
+       elementEvent := 
&pipepb.TestStreamPayload_Event_ElementEvent{ElementEvent: addElementsEvent}
+       c.events = append(c.events, &pipepb.TestStreamPayload_Event{Event: 
elementEvent})
+       return nil
+}
+
+// Create inserts a TestStream primitive into a pipeline, taking a scope and a 
Config object and
+// producing an array of output PCollections. The TestStream must be the first 
PTransform in the

Review comment:
       ```suggestion
   // producing an output PCollection. The TestStream must be the first 
PTransform in the
   ```




-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to