You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/07/30 15:12:53 UTC

[GitHub] [beam] jrmccluskey opened a new pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

jrmccluskey opened a new pull request #15253:
URL: https://github.com/apache/beam/pull/15253


   Create the teststream package to provide an interface for configuring the TestStream primitive in the Go SDK. Provides a type to set up a TestStreamConfig proto and insert it into the pipeline. Currently only supported on Flink.
   
   ------------------------
   
   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).
   
   `ValidatesRunner` compliance status (on master branch)
   --------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>Lang</th>
         <th>ULR</th>
         <th>Dataflow</th>
         <th>Flink</th>
         <th>Samza</th>
         <th>Spark</th>
         <th>Twister2</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>Go</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon">
           </a>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Samza/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
       </tr>
       <tr>
         <td>Java</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_ULR/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_ULR/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon?subject=V1">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming/lastCompletedBuild/badge/icon?subject=V1+Streaming">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon?subject=V1+Java+11">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2/lastCompletedBuild/badge/icon?subject=V2">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/lastCompletedBuild/badge/icon?subject=V2+Streaming">
           </a><br>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon?subject=Java+8">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/badge/icon?subject=Java+11">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon?subject=Portable">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon?subject=Portable+Streaming">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Samza/lastCompletedBuild/badge/icon?subject=Portable">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon?subject=Portable">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon?subject=Structured+Streaming">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/badge/icon">
           </a>
         </td>
       </tr>
       <tr>
         <td>Python</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon?subject=V1">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon?subject=V2">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon?subject=ValCont">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/badge/icon?subject=Portable">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Samza/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
       </tr>
       <tr>
         <td>XLang</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Dataflow/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Dataflow/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Samza/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
       </tr>
     </tbody>
   </table>
   
   Examples testing status on various runners
   --------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>Lang</th>
         <th>ULR</th>
         <th>Dataflow</th>
         <th>Flink</th>
         <th>Samza</th>
         <th>Spark</th>
         <th>Twister2</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>Go</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
       <tr>
         <td>Java</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Cron/lastCompletedBuild/badge/icon?subject=V1">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Java11_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Java11_Cron/lastCompletedBuild/badge/icon?subject=V1+Java11">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2/lastCompletedBuild/badge/icon?subject=V2">
           </a><br>
         </td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
       <tr>
         <td>Python</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
       <tr>
         <td>XLang</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
     </tbody>
   </table>
   
   Post-Commit SDK/Transform Integration Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>Go</th>
         <th>Java</th>
         <th>Python</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon?subject=3.6">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon?subject=3.7">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/badge/icon?subject=3.8">
           </a>
         </td>
       </tr>
     </tbody>
   </table>
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>---</th>
         <th>Java</th>
         <th>Python</th>
         <th>Go</th>
         <th>Website</th>
         <th>Whitespace</th>
         <th>Typescript</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>Non-portable</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon">
           </a><br>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon?subject=Tests">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon?subject=Lint">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/badge/icon?subject=Docker">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/badge/icon?subject=Docs">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
       </tr>
       <tr>
         <td>Portable</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_GoPortable_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_GoPortable_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
     </tbody>
   </table>
   
   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.
   
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r681989714



##########
File path: sdks/go/pkg/beam/testing/teststream/teststream_test.go
##########
@@ -0,0 +1,116 @@
+// 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"
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+)
+
+func TestNewConfig(t *testing.T) {
+	con := NewConfig()
+	if con.elmType != nil {
+		t.Errorf("type is not correct, expected nil, got %v", con.elmType)
+	}
+	if len(con.events) != 0 {
+		t.Errorf("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.Fatalf("expected only 1 event in config, got %v", len(con.events))
+	}
+	if eventWatermark := con.events[0].GetWatermarkEvent().NewWatermark; eventWatermark != 500 {
+		t.Errorf("expected watermark in event was 500, got %v", eventWatermark)

Review comment:
       Done.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lostluck commented on pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
lostluck commented on pull request #15253:
URL: https://github.com/apache/beam/pull/15253#issuecomment-892116624


   Run Go Flink ValidatesRunner


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r680148726



##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag
+// to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+	return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event into the Config Events struct advancing the processing time by a given

Review comment:
       Done.

##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag
+// to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+	return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event into the Config Events struct 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 timestamp.

Review comment:
       Done.

##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag

Review comment:
       Done.

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

Review comment:
       Done.

##########
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) {
+	c.Endpoint.Url = url
+}
+
+// CreatePayload converts the Config object into a TestStreamPayload to be sent to the runner.
+func (c *Config) CreatePayload() *pipepb.TestStreamPayload {

Review comment:
       Done.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lostluck commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r680110178



##########
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) {
+	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}

Review comment:
       Correct! That's what's expected for now. Add a comment to the "teststream.Create" method to call out this current limitation that the test stream must be the first PTransform in a pipeline.

##########
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) {
+	c.Endpoint.Url = url
+}
+
+// CreatePayload converts the Config object into a TestStreamPayload to be sent to the runner.
+func (c *Config) CreatePayload() *pipepb.TestStreamPayload {

Review comment:
       Since the proto payload is an implementation detail, this method doesn't need to be exported and can remain internal to the package.

##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag
+// to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+	return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event into the Config Events struct 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 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 {
+	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
+}
+
+// TestStream inserts a TestStream primitive into a pipeline, taking a scope and a Config object and
+// producing an array of output PCollections.
+func TestStream(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)
+
+	var ret []beam.PCollection
+	for _, val := range outputMap {
+		ret = append(ret, val)
+	}
+	return ret

Review comment:
       Agreed. Since implementations seem to all be single output collections, lets punt on tagged multi-collection map support for now. A later iteration can add a `CreateN` or `CreateMulti`call which would return a map.

##########
File path: sdks/go/test/integration/primitives/teststream.go
##########
@@ -0,0 +1,54 @@
+// 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 primitives
+
+import (
+	"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/testing/passert"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/teststream"
+)
+
+// TestStreamSequence tests the TestStream primitive by inserting string elements
+// then advancing the watermark past the poin where they were inserted.

Review comment:
       ```suggestion
   // then advancing the watermark past the point where they were inserted.
   ```

##########
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 {

Review comment:
       Conventionally if a constructor function is returning a pointer, it's totally OK to call it "New<foo>" instead of Make. Shorter, and idiomatic to a certain degree.

##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag
+// to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+	return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event into the Config Events struct advancing the processing time by a given

Review comment:
       We can omit implementation details like "Config Events struct " in the comment, instead leaning on the abstraction. TestStream is specifying a sequence of events, so the comments can be phrased accordingly. 

##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag
+// to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+	return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event into the Config Events struct 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 timestamp.

Review comment:
       It would be good to specify that the timestamp is the "event timestamp" for the element, and not when the element is being handled in processing time. This ties the comment to the Beam model.
   
   The panic comment about mismatching is good.

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

Review comment:
       Add a link to the blogpost describing test stream to the Package doc.
   
   We can clarify that teststream is only supported on Flink.
   
   It looks like it should be supported on Dataflow, so that's worth trying and calling out. It's only weird to mention because Dataflow costs money vs the intent of this is for testing... 

##########
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:
       In the "auto define coder" world, this is where a user would be required to call both SetCoder and SetEndpoint, but it would also be correct to have SetEndpoint also take the coder in question, as that's when it's relevant.

##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. Timestamp is

Review comment:
       Remove the tag comment since we're not worrying about multiple PCollections now.

##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag

Review comment:
       Remove the tag part of the comment.

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

Review comment:
       None of these fields need to be exported, as the Config type isn't getting serialized, and the users should be configuring them with the methods. This may not be java, but we can do some encapsulation and hiding implementation details from users.

##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag
+// to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+	return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event into the Config Events struct 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 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 {
+	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
+}
+
+// TestStream inserts a TestStream primitive into a pipeline, taking a scope and a Config object and
+// producing an array of output PCollections.
+func TestStream(s beam.Scope, c Config) []beam.PCollection {

Review comment:
       The only beef I have is calling this `TestStream` is that users would invoke it as `teststream.TestStream` which is stuttery and doesn't really describe what's going on.
   
   Consider instead `Create` which then becomes a call like `input := teststream.Create(s, cfg)`  One might thinkg that `New` could work like `teststream.New(s, cfg)` but that usually implies that one is getting some kind of "TestStream" pointer typed value, but in this case it's a PCollection. So I'm leaning torward `Create`.
   

##########
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 {

Review comment:
       At present the Go SDK has avoided users needing to interact with the coder objects. We can have the coder set by the first call to AddElements, and check that subsequent elements are of the same type at future calls.
   
   It would be reasonable to have a "SetCoder" method, which then validates that the specified coder uses the same type.
   
   This does mean that converting values to the event protos would need to be delayed until later, but overall it is more convenient to users that they don't need to deal with it if they don't need to.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r681990037



##########
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 the Flink runner.
+package teststream
+
+import (
+	"bytes"
+	"fmt"
+	"reflect"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+	"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 {
+	elmType   beam.FullType
+	events    []*pipepb.TestStreamPayload_Event
+	endpoint  *pipepb.ApiServiceDescriptor
+	watermark int64
+}
+
+// NewConfig returns a Config to build a sequence of a test stream's events.
+// Requires that users provide the coder for the elements they are trying to emit.
+func NewConfig() Config {
+	return Config{elmType: nil,
+		events:    []*pipepb.TestStreamPayload_Event{},
+		endpoint:  &pipepb.ApiServiceDescriptor{},
+		watermark: 0,

Review comment:
       Done.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lostluck commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
lostluck commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r681860491



##########
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 the Flink runner.
+package teststream
+
+import (
+	"bytes"
+	"fmt"
+	"reflect"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+	"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 {
+	elmType   beam.FullType
+	events    []*pipepb.TestStreamPayload_Event
+	endpoint  *pipepb.ApiServiceDescriptor
+	watermark int64
+}
+
+// NewConfig returns a Config to build a sequence of a test stream's events.
+// Requires that users provide the coder for the elements they are trying to emit.
+func NewConfig() Config {
+	return Config{elmType: 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 stream at the specified event timestamp. Must be called with
+// at least one element.
+//
+// On the first call, a type 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.
+func (c *Config) AddElements(timestamp int64, elements ...interface{}) error {
+	t := reflect.TypeOf(elements[0])
+	if c.elmType == nil {
+		c.elmType = typex.New(t)
+	} else if c.elmType.Type() != t {
+		return fmt.Errorf("element type mismatch, previous additions were of type %v, tried to add type %v", c.elmType, t)
+	}
+	newElements := []*pipepb.TestStreamPayload_TimestampedElement{}
+	enc := beam.NewElementEncoder(t)
+	for _, e := range elements {
+		var buf bytes.Buffer

Review comment:
       Consider validating that the rest of the elements in the slice are actually the same type to provide a clearer error message than "encoding value %v failed..."
   
   Something like "element %d was type %T but previous additions were of type %v", i, e, c.elmType  ?

##########
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 the Flink runner.
+package teststream
+
+import (
+	"bytes"
+	"fmt"
+	"reflect"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+	"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 {
+	elmType   beam.FullType
+	events    []*pipepb.TestStreamPayload_Event
+	endpoint  *pipepb.ApiServiceDescriptor
+	watermark int64
+}
+
+// NewConfig returns a Config to build a sequence of a test stream's events.
+// Requires that users provide the coder for the elements they are trying to emit.
+func NewConfig() Config {
+	return Config{elmType: 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}

Review comment:
       Add a comment explaining what the "c0" is and why we can get away with hard coding it.

##########
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 the Flink runner.
+package teststream
+
+import (
+	"bytes"
+	"fmt"
+	"reflect"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+	"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 {
+	elmType   beam.FullType
+	events    []*pipepb.TestStreamPayload_Event
+	endpoint  *pipepb.ApiServiceDescriptor
+	watermark int64
+}
+
+// NewConfig returns a Config to build a sequence of a test stream's events.
+// Requires that users provide the coder for the elements they are trying to emit.
+func NewConfig() Config {
+	return Config{elmType: nil,
+		events:    []*pipepb.TestStreamPayload_Event{},
+		endpoint:  &pipepb.ApiServiceDescriptor{},
+		watermark: 0,

Review comment:
       Nit: should probably start the watermark at the [mtime.MinTimestamp](https://github.com/apache/beam/blob/master/sdks/go/pkg/beam/core/graph/mtime/time.go#L32) 

##########
File path: sdks/go/pkg/beam/testing/teststream/teststream_test.go
##########
@@ -0,0 +1,116 @@
+// 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"
+	"reflect"
+	"testing"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+)
+
+func TestNewConfig(t *testing.T) {
+	con := NewConfig()
+	if con.elmType != nil {
+		t.Errorf("type is not correct, expected nil, got %v", con.elmType)
+	}
+	if len(con.events) != 0 {
+		t.Errorf("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.Fatalf("expected only 1 event in config, got %v", len(con.events))
+	}
+	if eventWatermark := con.events[0].GetWatermarkEvent().NewWatermark; eventWatermark != 500 {
+		t.Errorf("expected watermark in event was 500, got %v", eventWatermark)

Review comment:
       Style nit: Go idiom prefers using "want" over "expected". Not required if want is more convoluted to use of course.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on pull request #15253:
URL: https://github.com/apache/beam/pull/15253#issuecomment-889982219


   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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r681168372



##########
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:
       Changed to the new, coder-free approach




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on pull request #15253:
URL: https://github.com/apache/beam/pull/15253#issuecomment-889981621


   R: @lostluck 


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r681167960



##########
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:
       Done

##########
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:
       Done.

##########
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:
       Done.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on pull request #15253:
URL: https://github.com/apache/beam/pull/15253#issuecomment-892071549






-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r680149449



##########
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 {

Review comment:
       Done.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lostluck merged pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
lostluck merged pull request #15253:
URL: https://github.com/apache/beam/pull/15253


   


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r681990355



##########
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 the Flink runner.
+package teststream
+
+import (
+	"bytes"
+	"fmt"
+	"reflect"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+	"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 {
+	elmType   beam.FullType
+	events    []*pipepb.TestStreamPayload_Event
+	endpoint  *pipepb.ApiServiceDescriptor
+	watermark int64
+}
+
+// NewConfig returns a Config to build a sequence of a test stream's events.
+// Requires that users provide the coder for the elements they are trying to emit.
+func NewConfig() Config {
+	return Config{elmType: 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}

Review comment:
       Done.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on pull request #15253:
URL: https://github.com/apache/beam/pull/15253#issuecomment-891255910


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

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey removed a comment on pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey removed a comment on pull request #15253:
URL: https://github.com/apache/beam/pull/15253#issuecomment-891255910


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

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lostluck commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r680165494



##########
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:
       I suppose someone setting an endpoint will not be manually adding elements so taking a coder here isn't the end of the world. Or we could provide a SetCoder() function and have a check here to enforce that the config's elmCoder field is non-null so that it fails at build-time if there's a problem.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r680015359



##########
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) {
+	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}

Review comment:
       Hard-coding the c0 coder is a little hacky but works. The translation code always inserts a coder for the type and it's c0 as long as the TestStream is the first thing inserted into the pipeline from what I've seen. 




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r680149037



##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. Timestamp is

Review comment:
       Done.

##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag
+// to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+	return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event into the Config Events struct 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 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 {
+	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
+}
+
+// TestStream inserts a TestStream primitive into a pipeline, taking a scope and a Config object and
+// producing an array of output PCollections.
+func TestStream(s beam.Scope, c Config) []beam.PCollection {

Review comment:
       Done.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r680013819



##########
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) {
+	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 a PCollection
+// to the given timestamp. if the tag is empty, this is applied to the default PCollection. 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 for the PCollection corresponding to the tag
+// to the maximum timestamp.
+func (c *Config) AdvanceWatermarkToInfinity() error {
+	return c.AdvanceWatermark(mtime.MaxTimestamp.Milliseconds())
+}
+
+// AdvanceProcessingTime adds an event into the Config Events struct 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 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 {
+	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
+}
+
+// TestStream inserts a TestStream primitive into a pipeline, taking a scope and a Config object and
+// producing an array of output PCollections.
+func TestStream(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)
+
+	var ret []beam.PCollection
+	for _, val := range outputMap {
+		ret = append(ret, val)
+	}
+	return ret

Review comment:
       This should only ever be a single PCollection, so I was thinking this should be changed so that the return type is just a beam.PCollection, but didn't know if there was a clean looking way to do 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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r681994334



##########
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 the Flink runner.
+package teststream
+
+import (
+	"bytes"
+	"fmt"
+	"reflect"
+
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/typex"
+	"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 {
+	elmType   beam.FullType
+	events    []*pipepb.TestStreamPayload_Event
+	endpoint  *pipepb.ApiServiceDescriptor
+	watermark int64
+}
+
+// NewConfig returns a Config to build a sequence of a test stream's events.
+// Requires that users provide the coder for the elements they are trying to emit.
+func NewConfig() Config {
+	return Config{elmType: 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 stream at the specified event timestamp. Must be called with
+// at least one element.
+//
+// On the first call, a type 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.
+func (c *Config) AddElements(timestamp int64, elements ...interface{}) error {
+	t := reflect.TypeOf(elements[0])
+	if c.elmType == nil {
+		c.elmType = typex.New(t)
+	} else if c.elmType.Type() != t {
+		return fmt.Errorf("element type mismatch, previous additions were of type %v, tried to add type %v", c.elmType, t)
+	}
+	newElements := []*pipepb.TestStreamPayload_TimestampedElement{}
+	enc := beam.NewElementEncoder(t)
+	for _, e := range elements {
+		var buf bytes.Buffer

Review comment:
       Done.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r680164222



##########
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 {

Review comment:
       Changed NewConfig() to not take any arguments, handle assigning the coder when elements are added. 
   
   Assuming we allow users to provide custom coders through a SetCoder method, how would we validate that the new coder can encode everything the previous coder could?




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r681094769



##########
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:
       Looking at it now I think we could completely ditch keeping up with the coder and keep up with the element type passed into AddElements instead. `beam.Create` just gets the type of the first element and defines the encoder off of that using `beam.NewElementEncoder` in the same way I have now. End result we can remove any references to coders in `AddElement.` 




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on pull request #15253:
URL: https://github.com/apache/beam/pull/15253#issuecomment-891258098


   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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] jrmccluskey commented on a change in pull request #15253: [BEAM-11088] Add TestStream package to Go SDK testing capabilities

Posted by GitBox <gi...@apache.org>.
jrmccluskey commented on a change in pull request #15253:
URL: https://github.com/apache/beam/pull/15253#discussion_r681188337



##########
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:
       Switched to `CreateWithEndpoint` and have users provide a FullType object for the output. Ideally that would allow users to create more complex outputs to run through a TestStream assuming they have the TestStreamService set up. Could also just change it to take a `reflect.Type` object if going with FullType is overkill.




-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org