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 2020/06/24 08:24:13 UTC

[GitHub] [beam] pskevin opened a new pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

pskevin opened a new pull request #12071:
URL: https://github.com/apache/beam/pull/12071


   Added documentation, to both the [java](https://github.com/apache/beam/blob/master/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java) and [python](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/validate_runner_xlang_test.py) equivalents, explaining the design of the test suite tasked with validating a cross-language runner. It is directly inspired from [Runner Validation Test Plan for Cross-language transforms](https://docs.google.com/document/d/1xQp0ElIV84b8OCVz8CD2hvbiWdR8w4BvWxPTZJZA6NA/edit?usp=sharing).
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [x] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [x] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   Lang | SDK | Dataflow | Flink | Samza | Spark
   --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/)
   Java | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/)
   Python | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python2_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python35_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/)
   XLang | --- | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/)
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website
   --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/)
   Portable | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/) | --- | ---
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] chamikaramj commented on a change in pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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



##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +54,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the
+ * box. In spite of this, there always exists a possibility of rough edges existing. It could be
+ * caused due to unpolished implementation of any part of the execution code path, for example: –>
+ * Transform expansion [SDK] –> Pipeline construction [SDK] –> Cross-language artifact staging

Review comment:
       Does "->" have a special meaning ? If not we should just use comma here.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +54,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the
+ * box. In spite of this, there always exists a possibility of rough edges existing. It could be
+ * caused due to unpolished implementation of any part of the execution code path, for example: –>
+ * Transform expansion [SDK] –> Pipeline construction [SDK] –> Cross-language artifact staging
+ * [Runner] –> Language specific serialization/deserialization of PCollection (and other data types)
+ * [Runner/SDK]
+ *
+ * <p>In an effort to improve developer visibility into potential problems, this test suite
+ * validates correct execution of 5 Core Beam transforms when used as cross-language transforms
+ * within the Java SDK from any foreign SDK: –> ParDo
+ * (https://beam.apache.org/documentation/programming-guide/#pardo) –> GroupByKey

Review comment:
       Ditto regarding ->
   Also please move the link to the same line as the corresponding transform for easy readability of anyone who is just reading code.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -135,6 +172,15 @@ public void multiInputOutputWithSideInputTest() {
     PAssert.that(pTuple.get("side")).containsInAnyOrder("ss");
   }
 
+  /**
+   * Motivation behind groupByKeyTest.
+   *
+   * <p>Target transform – GroupByKey
+   * (https://beam.apache.org/documentation/programming-guide/#groupbykey) Test scenario – Grouping

Review comment:
       Move "Test scenario" to a new line here and below.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -110,6 +131,14 @@ private void waitForReady() {
     }
   }
 
+  /**
+   * Motivation behind singleInputOutputTest.
+   *
+   * <p>Target transform – ParDo (https://beam.apache.org/documentation/programming-guide/#pardo)
+   * Test scenario – Mapping elements from a single input collection to a single output collection
+   * Boundary conditions checked – –> PCollection<?> to external transforms –> PCollection<?> from

Review comment:
       Ditto regarding using comma instead of "->" here and below.

##########
File path: sdks/python/apache_beam/transforms/validate_runner_xlang_test.py
##########
@@ -14,6 +14,42 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
+
+"""
+###########################################################
+Runner Validation Test Suite for Cross-language Transforms
+###########################################################
+ As per Beams's Portability Framework design, Cross-language transforms
+ should work out of the box. In spite of this, there always exists a
+ possibility of rough edges existing. It could be caused due to unpolished
+ implementation of any part of the execution code path, for example:
+ - Transform expansion [SDK]
+ - Pipeline construction [SDK]
+ - Cross-language artifact staging [Runner]
+ - Language specific serialization/deserialization of PCollection (and
+ other data types) [Runner/SDK]
+
+ In an effort to improve developer visibility into potential problems,
+ this test suite validates correct execution of 5 Core Beam transforms when
+ used as cross-language transforms within the Python SDK from any foreign SDK:
+  - ParDo
+  (https://beam.apache.org/documentation/programming-guide/#pardo)
+  - GroupByKey
+  (https://beam.apache.org/documentation/programming-guide/#groupbykey)
+  - CoGroupByKey
+  (https://beam.apache.org/documentation/programming-guide/#cogroupbykey)
+  - Combine

Review comment:
       This reads much better :)

##########
File path: sdks/python/apache_beam/transforms/validate_runner_xlang_test.py
##########
@@ -57,6 +102,15 @@ def run_prefix(self, pipeline):
       assert_that(res, equal_to(['0a', '0b']))
 
   def run_multi_input_output_with_sideinput(self, pipeline):
+    """
+    Target transform - ParDo
+    (https://beam.apache.org/documentation/programming-guide/#pardo)
+    Test scenario - Mapping elements from multiple input collections (main
+    and side) to multiple output collections (main and side)

Review comment:
       Missing full stop here and below.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -110,6 +131,14 @@ private void waitForReady() {
     }
   }
 
+  /**
+   * Motivation behind singleInputOutputTest.
+   *
+   * <p>Target transform – ParDo (https://beam.apache.org/documentation/programming-guide/#pardo)
+   * Test scenario – Mapping elements from a single input collection to a single output collection
+   * Boundary conditions checked – –> PCollection<?> to external transforms –> PCollection<?> from
+   * external transforms

Review comment:
       Missing full stop here and below.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] chamikaramj commented on a change in pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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



##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -110,6 +137,20 @@ private void waitForReady() {
     }
   }
 
+  /**
+   * Motivation behind <i>singleInputOutputTest</i>.
+   *
+   * <ul>
+   *   <li><b>Target transform</b> – {@link ParDo}

Review comment:
       Lets link to website instead of Java version. For example, https://beam.apache.org/documentation/programming-guide/#pardo

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -120,6 +161,20 @@ public void singleInputOutputTest() throws IOException {
     PAssert.that(col).containsInAnyOrder("01", "02", "03");
   }
 
+  /**
+   * Motivation behind <i>multiInputOutputWithSideInputTest</i>.
+   *
+   * <ul>
+   *   <li><b>Target transform</b> – {@link ParDo}

Review comment:
       Ditto.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -110,6 +137,20 @@ private void waitForReady() {
     }
   }
 
+  /**
+   * Motivation behind <i>singleInputOutputTest</i>.
+   *
+   * <ul>
+   *   <li><b>Target transform</b> – {@link ParDo}
+   *   <li><b>Test scenario</b> – Mapping elements from a single input collection to a single output
+   *       collection
+   *   <li><b>Boundary conditions checked</b> –
+   *       <ul>
+   *         <li>PCollection<?> to external transforms

Review comment:
       Lets refer to following fo PCollection.
   https://beam.apache.org/documentation/programming-guide/#pcollections

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -187,6 +283,20 @@ public void combineGloballyTest() {
     PAssert.that(col).containsInAnyOrder(6L);
   }
 
+  /**
+   * Motivation behind <i>combinePerKeyTest</i>.

Review comment:
       Lets refer to https://beam.apache.org/documentation/programming-guide/#combine

##########
File path: sdks/python/apache_beam/transforms/validate_runner_xlang_test.py
##########
@@ -14,6 +14,38 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
+
+"""
+###########################################################
+Runner Validation Test Suite for Cross-language Transforms
+###########################################################
+As per Beams's Portability Framework design, Cross-language transforms should
+work out of the box. In spite of this, there always exists a possibility of

Review comment:
       Same as above, change to something more direct like following.
   
   Cross-language transforms runner validation test suite for Java SDK.
   
   These tests check whether core Beam transforms work correctly when used as cross-language transforms from Java SDK. This is needed to make sure that cross-language transforms framework works correctly for various corner cases. Please see documentation for individual tests for more details regarding cases covered by each test. Please see here<link to doc> for more details.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -154,6 +223,20 @@ public void groupByKeyTest() {
     PAssert.that(col).containsInAnyOrder("0:1,2", "1:3");
   }
 
+  /**
+   * Motivation behind <i>coGroupByKeyTest</i>.
+   *
+   * <ul>
+   *   <li><b>Target transform</b> – {@link CoGroupByKey}

Review comment:
       Lets refer to https://beam.apache.org/documentation/programming-guide/#cogroupbykey

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -177,6 +260,19 @@ public void coGroupByKeyTest() {
     PAssert.that(col).containsInAnyOrder("0:1,2,4", "1:3,5,6");
   }
 
+  /**
+   * Motivation behind <i>combineGloballyTest</i>.

Review comment:
       Lets refer to https://beam.apache.org/documentation/programming-guide/#combine

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -197,6 +307,19 @@ public void combinePerKeyTest() {
     PAssert.that(col).containsInAnyOrder(KV.of("a", 3L), KV.of("b", 3L));
   }
 
+  /**
+   * Motivation behind <i>flattenTest</i>.

Review comment:
       Lets refer to https://beam.apache.org/documentation/programming-guide/#flatten

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +60,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the
+ * box. In spite of this, there always exists a possibility of rough edges existing. It could be
+ * caused due to unpolished implementation of any part of the execution code path, for example:
+ *
+ * <ul>
+ *   <li>Transform expansion [SDK]
+ *   <li>Pipeline construction [SDK]
+ *   <li>Cross-language artifact staging [Runner]
+ *   <li>Language specific serialization/deserialization of PCollection (and other data types)
+ *       [Runner/SDK]
+ * </ul>
+ *
+ * <p>In an effort to improve developer visibility into potential problems, this test suite
+ * validates a cross-language runner against <i>5 Core Beam transforms</i> from any foreign language

Review comment:
       These tests actually tests whether versions of these transforms in foreign SDKs work from Java. I don't think we need to refer to Java versions here. We can just just mention that we test "core Beam transforms" here and link to Website below when possible without referring to (or importing) Java versions.

##########
File path: sdks/python/apache_beam/transforms/validate_runner_xlang_test.py
##########
@@ -14,6 +14,38 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
+
+"""
+###########################################################
+Runner Validation Test Suite for Cross-language Transforms
+###########################################################
+As per Beams's Portability Framework design, Cross-language transforms should
+work out of the box. In spite of this, there always exists a possibility of
+rough edges existing. It could be caused due to unpolished implementation of any
+part of the execution code path, for example:
+
+* Transform expansion [SDK]

Review comment:
       Lets refer to same Website links here as well for corresponding transforms.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +60,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the

Review comment:
       I think this can be replaced by something like following which gives a more direct explanation:
   
   Cross-language transforms runner validation test suite for Java SDK.
   
   These tests check whether core Beam transforms work correctly when used as cross-language transforms from Java SDK. This is needed to make sure that cross-language transforms framework works correctly for various corner cases. Please see documentation for individual tests for more details regarding cases covered by each test. Please see here<link to doc> for more details.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -209,6 +332,20 @@ public void flattenTest() {
     PAssert.that(col).containsInAnyOrder(1L, 2L, 3L, 4L, 5L, 6L);
   }
 
+  /**
+   * Motivation behind <i>partitionTest</i>.

Review comment:
       Lets refer to https://beam.apache.org/documentation/programming-guide/#partition

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -135,6 +190,20 @@ public void multiInputOutputWithSideInputTest() {
     PAssert.that(pTuple.get("side")).containsInAnyOrder("ss");
   }
 
+  /**
+   * Motivation behind <i>groupByKeyTest</i>.
+   *
+   * <ul>
+   *   <li><b>Target transform</b> – {@link GroupByKey}

Review comment:
       Lets refer to https://beam.apache.org/documentation/programming-guide/#groupbykey




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] chamikaramj commented on pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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


   Retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin commented on a change in pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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



##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +54,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the
+ * box. In spite of this, there always exists a possibility of rough edges existing. It could be
+ * caused due to unpolished implementation of any part of the execution code path, for example: –>
+ * Transform expansion [SDK] –> Pipeline construction [SDK] –> Cross-language artifact staging

Review comment:
       Oh wow. Technically, "–> Transform expansion [SDK]" is supposed to be one bullet point (and applies to others as well). Seems like spotless didn't like it and put them as you see it. I'll change it to commas.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +54,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the
+ * box. In spite of this, there always exists a possibility of rough edges existing. It could be
+ * caused due to unpolished implementation of any part of the execution code path, for example: –>
+ * Transform expansion [SDK] –> Pipeline construction [SDK] –> Cross-language artifact staging
+ * [Runner] –> Language specific serialization/deserialization of PCollection (and other data types)
+ * [Runner/SDK]
+ *
+ * <p>In an effort to improve developer visibility into potential problems, this test suite
+ * validates correct execution of 5 Core Beam transforms when used as cross-language transforms
+ * within the Java SDK from any foreign SDK: –> ParDo
+ * (https://beam.apache.org/documentation/programming-guide/#pardo) –> GroupByKey

Review comment:
       Ditto as my previous comment. None of this is organized as I intended it originally. Either spotless or lint has removed all new lines because none of the text is separated as markdown (when compared to the previous version). 
   My guess with the link is that since it's presence of the same line would exceed the allowed number of characters per line, it was just wrapped as you see it.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -110,6 +131,14 @@ private void waitForReady() {
     }
   }
 
+  /**
+   * Motivation behind singleInputOutputTest.
+   *
+   * <p>Target transform – ParDo (https://beam.apache.org/documentation/programming-guide/#pardo)
+   * Test scenario – Mapping elements from a single input collection to a single output collection
+   * Boundary conditions checked – –> PCollection<?> to external transforms –> PCollection<?> from

Review comment:
       Ditto as my comment earlier.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -110,6 +131,14 @@ private void waitForReady() {
     }
   }
 
+  /**
+   * Motivation behind singleInputOutputTest.
+   *
+   * <p>Target transform – ParDo (https://beam.apache.org/documentation/programming-guide/#pardo)
+   * Test scenario – Mapping elements from a single input collection to a single output collection
+   * Boundary conditions checked – –> PCollection<?> to external transforms –> PCollection<?> from
+   * external transforms

Review comment:
       Thanks for catching that.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -135,6 +172,15 @@ public void multiInputOutputWithSideInputTest() {
     PAssert.that(pTuple.get("side")).containsInAnyOrder("ss");
   }
 
+  /**
+   * Motivation behind groupByKeyTest.
+   *
+   * <p>Target transform – GroupByKey
+   * (https://beam.apache.org/documentation/programming-guide/#groupbykey) Test scenario – Grouping

Review comment:
       Ditto as my comment earlier. I'll see how to resolve this.

##########
File path: sdks/python/apache_beam/transforms/validate_runner_xlang_test.py
##########
@@ -14,6 +14,42 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
+
+"""
+###########################################################
+Runner Validation Test Suite for Cross-language Transforms
+###########################################################
+ As per Beams's Portability Framework design, Cross-language transforms
+ should work out of the box. In spite of this, there always exists a
+ possibility of rough edges existing. It could be caused due to unpolished
+ implementation of any part of the execution code path, for example:
+ - Transform expansion [SDK]
+ - Pipeline construction [SDK]
+ - Cross-language artifact staging [Runner]
+ - Language specific serialization/deserialization of PCollection (and
+ other data types) [Runner/SDK]
+
+ In an effort to improve developer visibility into potential problems,
+ this test suite validates correct execution of 5 Core Beam transforms when
+ used as cross-language transforms within the Python SDK from any foreign SDK:
+  - ParDo
+  (https://beam.apache.org/documentation/programming-guide/#pardo)
+  - GroupByKey
+  (https://beam.apache.org/documentation/programming-guide/#groupbykey)
+  - CoGroupByKey
+  (https://beam.apache.org/documentation/programming-guide/#cogroupbykey)
+  - Combine

Review comment:
       Thanks. :)

##########
File path: sdks/python/apache_beam/transforms/validate_runner_xlang_test.py
##########
@@ -57,6 +102,15 @@ def run_prefix(self, pipeline):
       assert_that(res, equal_to(['0a', '0b']))
 
   def run_multi_input_output_with_sideinput(self, pipeline):
+    """
+    Target transform - ParDo
+    (https://beam.apache.org/documentation/programming-guide/#pardo)
+    Test scenario - Mapping elements from multiple input collections (main
+    and side) to multiple output collections (main and side)

Review comment:
       Thanks for catching that. Will make the necessary change.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin commented on a change in pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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



##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -110,6 +137,20 @@ private void waitForReady() {
     }
   }
 
+  /**
+   * Motivation behind <i>singleInputOutputTest</i>.
+   *
+   * <ul>

Review comment:
       Makes sense. I've removed markup from both files making it more human-readable. Thanks for the advice @ihji!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] chamikaramj commented on pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] ihji commented on a change in pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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



##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +60,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the
+ * box. In spite of this, there always exists a possibility of rough edges existing. It could be
+ * caused due to unpolished implementation of any part of the execution code path, for example:
+ *
+ * <ul>
+ *   <li>Transform expansion [SDK]
+ *   <li>Pipeline construction [SDK]
+ *   <li>Cross-language artifact staging [Runner]
+ *   <li>Language specific serialization/deserialization of PCollection (and other data types)
+ *       [Runner/SDK]
+ * </ul>
+ *
+ * <p>In an effort to improve developer visibility into potential problems, this test suite
+ * validates a cross-language runner against <i>5 Core Beam transforms</i> from any foreign language

Review comment:
       Linking to Java transforms also conflicts with the description: from any foreign language.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -110,6 +137,20 @@ private void waitForReady() {
     }
   }
 
+  /**
+   * Motivation behind <i>singleInputOutputTest</i>.
+   *
+   * <ul>

Review comment:
       Not sure whether we should use markup here since these comments will be mostly read from the source code, not from the formatted doc pages (this class doesn't create APIs, it's only for tests). Markups could hinder readability in the raw form.

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +60,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the

Review comment:
       +1 for more direct and concise explanation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] chamikaramj commented on pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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


   Retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin commented on a change in pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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



##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +60,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the

Review comment:
       I've made it more explicit now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin commented on a change in pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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



##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -110,6 +137,20 @@ private void waitForReady() {
     }
   }
 
+  /**
+   * Motivation behind <i>singleInputOutputTest</i>.
+   *
+   * <ul>
+   *   <li><b>Target transform</b> – {@link ParDo}

Review comment:
       Acknowledged and reflected in the latest changes!

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -120,6 +161,20 @@ public void singleInputOutputTest() throws IOException {
     PAssert.that(col).containsInAnyOrder("01", "02", "03");
   }
 
+  /**
+   * Motivation behind <i>multiInputOutputWithSideInputTest</i>.
+   *
+   * <ul>
+   *   <li><b>Target transform</b> – {@link ParDo}

Review comment:
       Acknowledged and reflected in the latest changes!

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -135,6 +190,20 @@ public void multiInputOutputWithSideInputTest() {
     PAssert.that(pTuple.get("side")).containsInAnyOrder("ss");
   }
 
+  /**
+   * Motivation behind <i>groupByKeyTest</i>.
+   *
+   * <ul>
+   *   <li><b>Target transform</b> – {@link GroupByKey}

Review comment:
       Acknowledged and reflected in the latest changes!

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -154,6 +223,20 @@ public void groupByKeyTest() {
     PAssert.that(col).containsInAnyOrder("0:1,2", "1:3");
   }
 
+  /**
+   * Motivation behind <i>coGroupByKeyTest</i>.
+   *
+   * <ul>
+   *   <li><b>Target transform</b> – {@link CoGroupByKey}

Review comment:
       Acknowledged and reflected in the latest changes!

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -197,6 +307,19 @@ public void combinePerKeyTest() {
     PAssert.that(col).containsInAnyOrder(KV.of("a", 3L), KV.of("b", 3L));
   }
 
+  /**
+   * Motivation behind <i>flattenTest</i>.

Review comment:
       Acknowledged and reflected in the latest changes!

##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -209,6 +332,20 @@ public void flattenTest() {
     PAssert.that(col).containsInAnyOrder(1L, 2L, 3L, 4L, 5L, 6L);
   }
 
+  /**
+   * Motivation behind <i>partitionTest</i>.

Review comment:
       Acknowledged and reflected in the latest changes!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin closed pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

Posted by GitBox <gi...@apache.org>.
pskevin closed pull request #12071:
URL: https://github.com/apache/beam/pull/12071


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] chamikaramj commented on pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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


   R: @ihji can you take a look since I'll be OOO.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin commented on pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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


   R: @chamikaramj 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin commented on pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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


   Closing this pull request in favor of an [updated](https://github.com/apache/beam/pull/12465) counterpart which isn't plagued with botched git workflows.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin commented on pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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


   @chamikaramj @ihji I've addressed the concerns and reflected the changes. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] chamikaramj commented on pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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


   @pskevin please let us know when comments here have been addressed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin commented on a change in pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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



##########
File path: runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
##########
@@ -54,15 +60,36 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test External transforms. */
+/**
+ * Runner Validation Test Suite for Cross-language Transforms.
+ *
+ * <p>As per Beams's Portability Framework design, Cross-language transforms should work out of the
+ * box. In spite of this, there always exists a possibility of rough edges existing. It could be
+ * caused due to unpolished implementation of any part of the execution code path, for example:
+ *
+ * <ul>
+ *   <li>Transform expansion [SDK]
+ *   <li>Pipeline construction [SDK]
+ *   <li>Cross-language artifact staging [Runner]
+ *   <li>Language specific serialization/deserialization of PCollection (and other data types)
+ *       [Runner/SDK]
+ * </ul>
+ *
+ * <p>In an effort to improve developer visibility into potential problems, this test suite
+ * validates a cross-language runner against <i>5 Core Beam transforms</i> from any foreign language

Review comment:
       Acknowledged and reflected in the latest changes!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] chamikaramj commented on pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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


   Seems like commit history got messed up.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [beam] pskevin commented on a change in pull request #12071: [BEAM-9932] Add documentation describing cross-language test pipelines

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



##########
File path: sdks/python/apache_beam/transforms/validate_runner_xlang_test.py
##########
@@ -14,6 +14,38 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
+
+"""
+###########################################################
+Runner Validation Test Suite for Cross-language Transforms
+###########################################################
+As per Beams's Portability Framework design, Cross-language transforms should
+work out of the box. In spite of this, there always exists a possibility of

Review comment:
       Acknowledged and reflected in the latest changes!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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