documentation(xlang-python): updating docs as per comments
diff --git a/sdks/python/apache_beam/transforms/validate_runner_xlang_test.py b/sdks/python/apache_beam/transforms/validate_runner_xlang_test.py
index 4d6b56b..beddfb3 100644
--- a/sdks/python/apache_beam/transforms/validate_runner_xlang_test.py
+++ b/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
+  (https://beam.apache.org/documentation/programming-guide/#combine)
+  –> Flatten
+  (https://beam.apache.org/documentation/programming-guide/#flatten)
+  –> Partition
+  (https://beam.apache.org/documentation/programming-guide/#partition)
+
+  See Runner Validation Test Plan for Cross-language transforms at
+https://docs.google.com/document/d/1xQp0ElIV84b8OCVz8CD2hvbiWdR8w4BvWxPTZJZA6NA
+  for further details.
+"""
+
 from __future__ import absolute_import
 
 import logging
@@ -46,6 +82,15 @@
         'localhost:%s' % os.environ.get('EXPANSION_PORT'))
 
   def run_prefix(self, pipeline):
+    """
+    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
+    """
     with pipeline as p:
       res = (
           p
@@ -57,6 +102,15 @@
       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)
+    Boundary conditions checked –
+     –> PCollectionTuple to external transforms
+     –> PCollectionTuple from external transforms
+    """
     with pipeline as p:
       main1 = p | 'Main1' >> beam.Create(
           ['a', 'bb'], reshuffle=False).with_output_types(unicode)
@@ -70,6 +124,15 @@
       assert_that(res['side'], equal_to(['ss']), label='CheckSide')
 
   def run_group_by_key(self, pipeline):
+    """
+    Target transform – GroupByKey
+    (https://beam.apache.org/documentation/programming-guide/#groupbykey)
+    Test scenario – Grouping a collection of KV<K,V> to a collection of
+    KV<K, Iterable<V>> by key
+    Boundary conditions checked –
+     –> PCollection<KV<?, ?>> to external transforms
+     –> PCollection<KV<?, Iterable<?>>> from external transforms
+    """
     with pipeline as p:
       res = (
           p
@@ -81,6 +144,15 @@
       assert_that(res, equal_to(['0:1,2', '1:3']))
 
   def run_cogroup_by_key(self, pipeline):
+    """
+    Target transform – CoGroupByKey
+    (https://beam.apache.org/documentation/programming-guide/#cogroupbykey)
+    Test scenario – Grouping multiple input collections with keys to a
+    collection of KV<K, CoGbkResult> by key
+    Boundary conditions checked –
+     –> KeyedPCollectionTuple<?> to external transforms
+     –> PCollection<KV<?, Iterable<?>>> from external transforms
+    """
     with pipeline as p:
       col1 = p | 'create_col1' >> beam.Create(
           [(0, "1"), (0, "2"), (1, "3")], reshuffle=False).with_output_types(
@@ -95,6 +167,15 @@
       assert_that(res, equal_to(['0:1,2,4', '1:3,5,6']))
 
   def run_combine_globally(self, pipeline):
+    """
+    Target transform – Combine
+    (https://beam.apache.org/documentation/programming-guide/#combine)
+    Test scenario – Combining elements globally with a predefined simple
+    CombineFn
+    Boundary conditions checked –
+     –> PCollection<?> to external transforms
+     –> PCollection<?> from external transforms
+    """
     with pipeline as p:
       res = (
           p
@@ -104,6 +185,15 @@
       assert_that(res, equal_to([6]))
 
   def run_combine_per_key(self, pipeline):
+    """
+    Target transform – Combine
+    (https://beam.apache.org/documentation/programming-guide/#combine)
+    Test scenario – Combining elements per key with a predefined simple
+    merging function
+    Boundary conditions checked –
+     –> PCollection<?> to external transforms
+     –> PCollection<?> from external transforms
+    """
     with pipeline as p:
       res = (
           p
@@ -114,6 +204,14 @@
       assert_that(res, equal_to([('a', 3), ('b', 3)]))
 
   def run_flatten(self, pipeline):
+    """
+    Target transform – Flatten
+    (https://beam.apache.org/documentation/programming-guide/#flatten)
+    Test scenario – Merging multiple collections into a single collection
+    Boundary conditions checked –
+     –> PCollectionList<?> to external transforms
+     –> PCollection<?> from external transforms
+    """
     with pipeline as p:
       col1 = p | 'col1' >> beam.Create([1, 2, 3]).with_output_types(int)
       col2 = p | 'col2' >> beam.Create([4, 5, 6]).with_output_types(int)
@@ -123,6 +221,15 @@
       assert_that(res, equal_to([1, 2, 3, 4, 5, 6]))
 
   def run_partition(self, pipeline):
+    """
+    Target transform – Partition
+    (https://beam.apache.org/documentation/programming-guide/#partition)
+    Test scenario – Splitting a single collection into multiple collections
+    with a predefined simple PartitionFn
+    Boundary conditions checked –
+     –> PCollection<?> to external transforms
+     –> PCollectionList<?> from external transforms
+    """
     with pipeline as p:
       res = (
           p