Repository: beam-site
Updated Branches:
  refs/heads/asf-site 3fd0ed617 -> 408f290eb


Adding testing related info: TestPipeline, API surface.


Project: http://git-wip-us.apache.org/repos/asf/beam-site/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam-site/commit/3c0c5322
Tree: http://git-wip-us.apache.org/repos/asf/beam-site/tree/3c0c5322
Diff: http://git-wip-us.apache.org/repos/asf/beam-site/diff/3c0c5322

Branch: refs/heads/asf-site
Commit: 3c0c5322cbd6ac608a61708622765119cde194dc
Parents: 3fd0ed6
Author: Stas Levin <[email protected]>
Authored: Mon Feb 20 18:21:37 2017 +0200
Committer: Stas Levin <[email protected]>
Committed: Fri Feb 24 16:01:57 2017 +0200

----------------------------------------------------------------------
 src/contribute/testing.md | 155 +++++++++++++++++++++++++++++++++++++++++
 1 file changed, 155 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam-site/blob/3c0c5322/src/contribute/testing.md
----------------------------------------------------------------------
diff --git a/src/contribute/testing.md b/src/contribute/testing.md
index 1d54b0e..c3439a3 100644
--- a/src/contribute/testing.md
+++ b/src/contribute/testing.md
@@ -371,3 +371,158 @@ RunnableOnService tests are tests built to use the Beam 
TestPipeline class,
 which enables test authors to write simple functionality verification. They are
 meant to use some of the built-in utilities of the SDK, namely PAssert, to
 verify that the simple pipelines they run end in the correct state.
+
+
+### Effective use of the TestPipeline JUnit rule
+
+`TestPipeline` is JUnit rule designed to facilitate testing pipelines. 
+In combination with `PAssert`, the two can be used for testing and 
+writing assertions over pipelines. However, in order for these assertions 
+to be effective, the constructed pipeline **must** be run by a pipeline 
+runner. If the pipeline is not run (i.e., executed) then the 
+constructed `PAssert` statements will not be triggered, and will thus 
+be ineffective. 
+
+To prevent such cases, `TestPipeline` has some protection mechanisms in place.
+
+__Abandoned node detection (performed automatically)__
+
+Abandoned nodes are `PTransforms`, `PAsserts` included, that were not 
+executed by the pipeline runner. Abandoned nodes are most likely to occur 
+due to the one of the following scenarios:
+ 1. Lack of a `pipeline.run()` statement at the end of a test. 
+ 2. Addition of `PTransform`s  after the pipeline has already run.
+
+Abandoned node detection is *automatically enabled* when a real pipeline 
+runner (i.e. not a `CrashingRunner`) and/or a 
+`@NeedsRunner` / `@RunnableOnService` annotation are detected. 
+
+Consider the following test:
+  
+```java
+// Note the @Rule annotation here
+@Rule
+public final transient TestPipeline pipeline = TestPipeline.create();
+
+@Test
+@Category(NeedsRunner.class)
+public void myPipelineTest() throws Exception {
+
+final PCollection<String> pCollection = 
+  pipeline
+    .apply("Create", Create.of(WORDS).withCoder(StringUtf8Coder.of()))
+    .apply(
+        "Map1",
+        MapElements.via(
+            new SimpleFunction<String, String>() {
+
+              @Override
+              public String apply(final String input) {
+                return WHATEVER;
+              }
+            }));
+            
+PAssert.that(pCollection).containsInAnyOrder(WHATEVER);       
+
+/* ERROR: pipeline.run() is missing, PAsserts are ineffective */
+}
+```
+
+```py
+# Unsupported in Beam's Python SDK.
+```
+ 
+The `PAssert` at the end of this test method will not be executed, since 
+`pipeline` is never run, making this test ineffective. If this test method 
+is run using an actual pipeline runner, an exception will be thrown 
+indicating that there was no `run()` invocation in the test.
+
+Exceptions that are thrown prior to executing a pipeline, will fail 
+the test unless handled by an `ExpectedException` rule.
+
+Consider the following test:  
+  
+```java
+// Note the @Rule annotation here
+@Rule
+public final transient TestPipeline pipeline = TestPipeline.create();
+
+@Test
+public void testReadingFailsTableDoesNotExist() throws Exception {
+  final String table = "TEST-TABLE";
+
+  BigtableIO.Read read =
+      BigtableIO.read()
+          .withBigtableOptions(BIGTABLE_OPTIONS)
+          .withTableId(table)
+          .withBigtableService(service);
+
+  // Exception will be thrown by read.validate() when read is applied.
+  thrown.expect(IllegalArgumentException.class);
+  thrown.expectMessage(String.format("Table %s does not exist", table));
+
+  p.apply(read);
+}
+```
+
+```py
+# Unsupported in Beam's Python SDK.
+```  
+  
+The application of the `read` transform throws an exception, which is then 
+handled by the `thrown` `ExpectedException` rule. 
+In light of this exception, the fact this test has abandoned nodes 
+(the `read` transform) does not play a role since the test fails before 
+the pipeline would have been executed (had there been a `run()` statement). 
+   
+__Auto-add `pipeline.run()` (disabled by default)__
+
+A `TestPipeline` instance can be configured to auto-add a missing `run()` 
+statement by setting `testPipeline.enableAutoRunIfMissing(true/false)`. 
+If this feature is enabled, no exception will be thrown in case of a 
+missing `run()` statement, instead, one will be added automatically.
+
+
+### API Surface testing
+
+The surface of an API is the set of public classes that are exposed to the 
+outer world. In order to keep the API tight and avoid unnecessarily exposing 
+classes, Beam provides the `ApiSurface` utility class. 
+Using the `ApiSurface` class,  we can assert the API surface against an 
+expected set of classes.
+
+Consider the following snippet:
+```java
+@Test
+public void testMyApiSurface() throws Exception {
+  
+    final Package thisPackage = getClass().getPackage();
+    final ClassLoader thisClassLoader = getClass().getClassLoader();
+    
+    final ApiSurface apiSurface =
+        ApiSurface.ofPackage(thisPackage, thisClassLoader)
+            .pruningPattern("org[.]apache[.]beam[.].*Test.*")
+            .pruningPattern("org[.]apache[.]beam[.].*IT")
+            .pruningPattern("java[.]lang.*");
+    
+    @SuppressWarnings("unchecked")
+    final Set<Matcher<Class<?>>> allowed =
+        ImmutableSet.of(
+            classesInPackage("org.apache.beam.x"),
+            classesInPackage("org.apache.beam.y"),
+            classesInPackage("org.apache.beam.z"),
+            Matchers.<Class<?>>equalTo(Other.class));
+    
+    assertThat(apiSurface, containsOnlyClassesMatching(allowed));
+}
+```
+
+```py
+# Unsupported in Beam's Python SDK.
+```
+
+This test will fail if the classes exposed by `getClass().getPackage()`, 
except 
+classes which reside under `"org[.]apache[.]beam[.].*Test.*"`,  
+`"org[.]apache[.]beam[.].*IT"` or `"java[.]lang.*"`, belong to neither
+of the packages: `org.apache.beam.x`, `org.apache.beam.y`, 
`org.apache.beam.z`, 
+nor equal to `Other.class`.

Reply via email to