From 60d5471f74df1da5979dc31a6a7ee11c6fe1a850 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Tue, 12 Apr 2016 17:49:07 -0700 Subject: [PATCH 1/3] [BEAM-151] Add support for RunnableOnService validation tests to runners package Default implementation for Flink and Spark are to have it disabled. Default implementation for Dataflow requires you to specify the runnableOnServicePipelineOptions system property with a valid project/runner/... to be able to execute on Google Cloud Dataflow. --- examples/java/pom.xml | 11 ----- examples/java8/pom.xml | 11 ----- pom.xml | 7 --- runners/google-cloud-dataflow-java/pom.xml | 49 ++++++++++++------- runners/pom.xml | 38 ++++++++++++++ sdks/java/core/pom.xml | 16 ------ .../sdk/testing/RunnableOnService.java | 13 +++-- .../dataflow/sdk/testing/TestPipeline.java | 25 +++++----- .../sdk/testing/DataflowJUnitTestRunner.java | 5 +- .../sdk/testing/TestPipelineTest.java | 5 +- sdks/java/java8tests/pom.xml | 15 ------ 11 files changed, 91 insertions(+), 104 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index a90973af742e..bcac57d1f94d 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -36,17 +36,6 @@ jar - - - DataflowPipelineTests - - true - com.google.cloud.dataflow.sdk.testing.RunnableOnService - both - - - - diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index 822341b0d69f..fe10dbc0eeda 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -37,17 +37,6 @@ jar - - - DataflowPipelineTests - - true - com.google.cloud.dataflow.sdk.testing.RunnableOnService - both - - - - diff --git a/pom.xml b/pom.xml index 08073a229a12..de29eb8153fb 100644 --- a/pom.xml +++ b/pom.xml @@ -289,13 +289,6 @@ maven-surefire-plugin 2.18.1 - ${testParallelValue} - 4 - ${testGroups} - - ${runIntegrationTestOnService} - ${dataflowProjectName} - false false true diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 3d62b0ef7323..715e35990aaf 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -21,14 +21,11 @@ 4.0.0 - - org.apache.beam - parent - 0.1.0-incubating-SNAPSHOT - ../../pom.xml + org.apache.beam + runners-parent + 0.1.0-incubating-SNAPSHOT + ../pom.xml google-cloud-dataflow-java-runner @@ -41,21 +38,37 @@ 0.1.0-incubating-SNAPSHOT ${maven.build.timestamp} yyyy-MM-dd HH:mm - com.google.cloud.dataflow - false - none - - + - DataflowPipelineTests - - true - com.google.cloud.dataflow.sdk.testing.RunnableOnService - both - + disable-integration-tests + + + !integrationTestPipelineOptions + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + disable-integration-test + integration-test + + test + + + true + + + + + + diff --git a/runners/pom.xml b/runners/pom.xml index cba25a3a1256..013881f91edc 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -39,4 +39,42 @@ spark + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.18.1 + + + execute-integration-tests + integration-test + + test + + + com.google.cloud.dataflow.sdk.testing.RunnableOnService + all + 4 + + org.apache.beam:java-sdk-all + + + ${integrationTestPipelineOptions} + + + + + + + org.apache.maven.surefire + surefire-junit47 + 2.18.1 + + + + + + diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index aa72b11c667f..06ceb2286b1c 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -39,24 +39,8 @@ ${maven.build.timestamp} yyyy-MM-dd HH:mm - com.google.cloud.dataflow - false - none - - - - - DataflowPipelineTests - - true - com.google.cloud.dataflow.sdk.testing.RunnableOnService - both - - - - diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java index 4490c3c79f8c..ea14774e9d97 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/RunnableOnService.java @@ -18,14 +18,17 @@ package com.google.cloud.dataflow.sdk.testing; /** - * Category tag for tests that can be run on the - * {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner} if the - * {@code runIntegrationTestOnService} System property is set to true. - * Example usage: + * Category tag for validation tests which utilize {@link TestPipeline} for execution and + * {@link PAssert} for validation. Example usage: *

  *     {@literal @}Test
  *     {@literal @}Category(RunnableOnService.class)
- *     public void testParDo() {...
+ *     public void testParDo() {
+ *       Pipeline p = TestPipeline.create();
+ *       p.apply(...);
+ *       PAssert.that(p);
+ *       p.run();
+ *     }
  * 
*/ public interface RunnableOnService {} diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java index fe667a44feae..f2ca2577aa63 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java @@ -27,6 +27,7 @@ import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.util.TestCredential; import com.google.common.base.Optional; +import com.google.common.base.Strings; import com.google.common.collect.Iterators; import com.fasterxml.jackson.databind.ObjectMapper; @@ -49,7 +50,6 @@ * *

In order to run tests on a pipeline runner, the following conditions must be met: *

    - *
  • System property "runIntegrationTestOnService" must be set to true.
  • *
  • System property "dataflowOptions" must contain a JSON delimited list of pipeline options. * For example: *
    {@code [
    @@ -77,7 +77,7 @@
      * containing the message from the {@link PAssert} that failed.
      */
     public class TestPipeline extends Pipeline {
    -  private static final String PROPERTY_DATAFLOW_OPTIONS = "dataflowOptions";
    +  private static final String PROPERTY_BEAM_TEST_PIPELINE_OPTIONS = "beamTestPipelineOptions";
       private static final ObjectMapper MAPPER = new ObjectMapper();
     
       /**
    @@ -126,34 +126,31 @@ public String toString() {
        */
       public static PipelineOptions testingPipelineOptions() {
         try {
    -      @Nullable String systemDataflowOptions = System.getProperty(PROPERTY_DATAFLOW_OPTIONS);
    +      @Nullable String beamTestPipelineOptions =
    +          System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS);
    +
           PipelineOptions options =
    -          systemDataflowOptions == null
    +          Strings.isNullOrEmpty(beamTestPipelineOptions)
                   ? PipelineOptionsFactory.create()
                   : PipelineOptionsFactory.fromArgs(
                           MAPPER.readValue(
    -                          System.getProperty(PROPERTY_DATAFLOW_OPTIONS), String[].class))
    +                          System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS), String[].class))
                       .as(PipelineOptions.class);
     
           options.as(ApplicationNameOptions.class).setAppName(getAppName());
    -      if (!isIntegrationTest()) {
    +      // If no options were specified, use a test credential object on all pipelines.
    +      if (Strings.isNullOrEmpty(beamTestPipelineOptions)) {
             options.as(GcpOptions.class).setGcpCredential(new TestCredential());
           }
           options.setStableUniqueNames(CheckEnabled.ERROR);
           return options;
         } catch (IOException e) {
           throw new RuntimeException("Unable to instantiate test options from system property "
    -          + PROPERTY_DATAFLOW_OPTIONS + ":" + System.getProperty(PROPERTY_DATAFLOW_OPTIONS), e);
    +          + PROPERTY_BEAM_TEST_PIPELINE_OPTIONS + ":"
    +          + System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS), e);
         }
       }
     
    -  /**
    -   * Returns whether a {@link TestPipeline} should be treated as an integration test.
    -   */
    -  private static boolean isIntegrationTest() {
    -    return Boolean.parseBoolean(System.getProperty("runIntegrationTestOnService"));
    -  }
    -
       /** Returns the class + method name of the test, or a default name. */
       private static String getAppName() {
         Optional stackTraceElement = findCallersStackTrace();
    diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/testing/DataflowJUnitTestRunner.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/testing/DataflowJUnitTestRunner.java
    index ffb71c09781a..f0f341a2959a 100644
    --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/testing/DataflowJUnitTestRunner.java
    +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/testing/DataflowJUnitTestRunner.java
    @@ -94,9 +94,8 @@ public static void main(String ... args) throws Exception {
     
         // Set system properties required by TestPipeline so that it is able to execute tests
         // on the service.
    -    String dataflowPipelineOptions = new ObjectMapper().writeValueAsString(args);
    -    System.setProperty("runIntegrationTestOnService", "true");
    -    System.setProperty("dataflowOptions", dataflowPipelineOptions);
    +    String beamTestPipelineOptions = new ObjectMapper().writeValueAsString(args);
    +    System.setProperty("beamTestPipelineOptions", beamTestPipelineOptions);
     
         // Run the set of tests
         boolean success = true;
    diff --git a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java
    index d6a2b63a6a4b..147cc490b209 100644
    --- a/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java
    +++ b/sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/testing/TestPipelineTest.java
    @@ -52,7 +52,7 @@ public void testCreationOfPipelineOptions() throws Exception {
           "--runner=DirectPipelineRunner",
           "--project=testProject"
         });
    -    System.getProperties().put("dataflowOptions", stringOptions);
    +    System.getProperties().put("beamTestPipelineOptions", stringOptions);
         GcpOptions options =
             TestPipeline.testingPipelineOptions().as(GcpOptions.class);
         assertEquals(DirectPipelineRunner.class, options.getRunner());
    @@ -61,9 +61,6 @@ public void testCreationOfPipelineOptions() throws Exception {
     
       @Test
       public void testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase() throws Exception {
    -        ObjectMapper mapper = new ObjectMapper();
    -    String stringOptions = mapper.writeValueAsString(new String[]{});
    -    System.getProperties().put("dataflowOptions", stringOptions);
         PipelineOptions options = TestPipeline.testingPipelineOptions();
         assertThat(options.as(ApplicationNameOptions.class).getAppName(), startsWith(
             "TestPipelineTest-testCreationOfPipelineOptionsFromReallyVerboselyNamedTestCase"));
    diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml
    index 59e2843119fd..cd7174a729fa 100644
    --- a/sdks/java/java8tests/pom.xml
    +++ b/sdks/java/java8tests/pom.xml
    @@ -37,22 +37,9 @@
     
       jar
     
    -  
    -    
    -      DataflowPipelineTests
    -      
    -        true
    -        com.google.cloud.dataflow.sdk.testing.RunnableOnService
    -        both
    -      
    -    
    -  
    -
       
    -
         
           
    -
             
               maven-compiler-plugin
               
    @@ -60,12 +47,10 @@
                  1.8
               
             
    -
           
         
     
         
    -
           
             maven-compiler-plugin
           
    
    From 86e61c5f91f02340b42e3608b0b949f6384ae3d9 Mon Sep 17 00:00:00 2001
    From: Luke Cwik 
    Date: Tue, 12 Apr 2016 19:50:49 -0700
    Subject: [PATCH 2/3] [BEAM-151] !fixup Add the runnable-on-service-tests
     execution
    
    ---
     runners/flink/examples/pom.xml             |  25 +
     runners/flink/runner/pom.xml               |  24 +
     runners/google-cloud-dataflow-java/pom.xml |  14 +-
     runners/pom.xml                            |  11 +-
     runners/spark/pom.xml                      | 783 +++++++++++----------
     5 files changed, 461 insertions(+), 396 deletions(-)
    
    diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml
    index 1c0a4dd5aa7b..d27f2db2f361 100644
    --- a/runners/flink/examples/pom.xml
    +++ b/runners/flink/examples/pom.xml
    @@ -42,6 +42,31 @@
         1
       
     
    +  
    +    
    +      disable-runnable-on-service-tests
    +      
    +        true
    +      
    +      
    +        
    +          
    +            org.apache.maven.plugins
    +            maven-surefire-plugin
    +            
    +              
    +                runnable-on-service-tests
    +                
    +                  true
    +                
    +              
    +            
    +          
    +        
    +      
    +    
    +  
    +
       
     
         
    diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
    index 7a1011a8e88d..89c8d9949e66 100644
    --- a/runners/flink/runner/pom.xml
    +++ b/runners/flink/runner/pom.xml
    @@ -34,6 +34,30 @@
     
       jar
     
    +  
    +    
    +      disable-runnable-on-service-tests
    +      
    +        true
    +      
    +      
    +        
    +          
    +            org.apache.maven.plugins
    +            maven-surefire-plugin
    +            
    +              
    +                runnable-on-service-tests
    +                
    +                  true
    +                
    +              
    +            
    +          
    +        
    +      
    +    
    +  
     
       
         
    diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
    index 715e35990aaf..e54bbf95bb3f 100644
    --- a/runners/google-cloud-dataflow-java/pom.xml
    +++ b/runners/google-cloud-dataflow-java/pom.xml
    @@ -40,13 +40,15 @@
         yyyy-MM-dd HH:mm
       
     
    -  
    +  
       
         
    -      disable-integration-tests
    +      disable-runnable-on-service-tests
           
             
    -          !integrationTestPipelineOptions
    +          !runnableOnServicePipelineOptions
             
           
           
    @@ -56,11 +58,7 @@
                 maven-surefire-plugin
                 
                   
    -                disable-integration-test
    -                integration-test
    -                
    -                  test
    -                
    +                runnable-on-service-tests
                     
                       true
                     
    diff --git a/runners/pom.xml b/runners/pom.xml
    index 013881f91edc..9e2eb4305c69 100644
    --- a/runners/pom.xml
    +++ b/runners/pom.xml
    @@ -48,7 +48,7 @@
               2.18.1
               
                 
    -              execute-integration-tests
    +              runnable-on-service-tests
                   integration-test
                   
                     test
    @@ -61,18 +61,11 @@
                       org.apache.beam:java-sdk-all
                     
                     
    -                  ${integrationTestPipelineOptions}
    +                  ${runnableOnServicePipelineOptions}
                     
                   
                 
               
    -          
    -          
    -            org.apache.maven.surefire
    -            surefire-junit47
    -            2.18.1
    -          
    -        
             
           
         
    diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
    index 477691023a13..f12d8a6a3c17 100644
    --- a/runners/spark/pom.xml
    +++ b/runners/spark/pom.xml
    @@ -19,396 +19,421 @@
              xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
              xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     
    -    4.0.0
    +  4.0.0
     
    -    
    -        org.apache.beam
    -        runners-parent
    -        0.1.0-incubating-SNAPSHOT
    -    
    +  
    +    org.apache.beam
    +    runners-parent
    +    0.1.0-incubating-SNAPSHOT
    +  
     
    -    spark-runner
    +  spark-runner
     
    -    Apache Beam :: Runners :: Spark
    -    jar
    +  Apache Beam :: Runners :: Spark
    +  jar
     
    -    
    -        UTF-8
    -        UTF-8
    -        1.7
    -        1.6.1
    -        0.1.0-incubating-SNAPSHOT
    -    
    +  
    +    UTF-8
    +    UTF-8
    +    1.7
    +    1.6.1
    +    0.1.0-incubating-SNAPSHOT
    +  
     
    -    
    -        
    -            org.apache.spark
    -            spark-core_2.10
    -            ${spark.version}
    -            provided
    -        
    -        
    -            org.apache.spark
    -            spark-streaming_2.10
    -            ${spark.version}
    -            provided
    -        
    -        
    -            org.apache.spark
    -            spark-streaming-kafka_2.10
    -            ${spark.version}
    -            provided
    -        
    -        
    -            org.apache.kafka
    -            kafka_2.10
    -            0.8.2.1
    -            provided
    -        
    -        
    -            com.google.guava
    -            guava
    -            ${guava.version}
    -        
    -        
    -            org.apache.beam
    -            java-sdk-all
    -            ${beam.version}
    -            
    -                
    -                
    -                    org.slf4j
    -                    slf4j-jdk14
    -                
    -            
    -        
    -        
    -            org.apache.beam
    -            java-examples-all
    -            ${beam.version}
    -            
    -                
    -                
    -                    org.slf4j
    -                    slf4j-jdk14
    -                
    -            
    -        
    -        
    -            org.apache.avro
    -            avro-mapred
    -            1.7.7
    -            hadoop2
    -            
    -                
    -                
    -                    org.mortbay.jetty
    -                    servlet-api
    -                
    -            
    -        
    +  
    +    
    +      org.apache.spark
    +      spark-core_2.10
    +      ${spark.version}
    +      provided
    +    
    +    
    +      org.apache.spark
    +      spark-streaming_2.10
    +      ${spark.version}
    +      provided
    +    
    +    
    +      org.apache.spark
    +      spark-streaming-kafka_2.10
    +      ${spark.version}
    +      provided
    +    
    +    
    +      org.apache.kafka
    +      kafka_2.10
    +      0.8.2.1
    +      provided
    +    
    +    
    +      com.google.guava
    +      guava
    +      ${guava.version}
    +    
    +    
    +      org.apache.beam
    +      java-sdk-all
    +      ${beam.version}
    +      
    +        
    +        
    +          org.slf4j
    +          slf4j-jdk14
    +        
    +      
    +    
    +    
    +      org.apache.beam
    +      java-examples-all
    +      ${beam.version}
    +      
    +        
    +        
    +          org.slf4j
    +          slf4j-jdk14
    +        
    +      
    +    
    +    
    +      org.apache.avro
    +      avro-mapred
    +      1.7.7
    +      hadoop2
    +      
    +        
    +        
    +          org.mortbay.jetty
    +          servlet-api
    +        
    +      
    +    
     
    -        
    -        
    -            junit
    -            junit
    -            4.12
    -            test
    -        
    -        
    -            org.hamcrest
    -            hamcrest-all
    -            1.3
    -            test
    -        
    -    
    +    
    +    
    +      junit
    +      junit
    +      4.12
    +      test
    +    
    +    
    +      org.hamcrest
    +      hamcrest-all
    +      1.3
    +      test
    +    
    +  
     
    -    
    -        
    -            
    -                
    -                    org.apache.maven.plugins
    -                    maven-compiler-plugin
    -                    3.5
    -                    
    -                        ${java.version}
    -                        ${java.version}
    -                        
    -                            -Xlint:all,-serial
    -                        
    -                    
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-surefire-plugin
    -                    2.19.1
    -                    
    -                        1
    -                        false
    -                    
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-checkstyle-plugin
    -                    2.17
    -                    
    -                        build-resources/header-file.txt
    -                        build-resources/checkstyle.xml
    -                    
    -                    
    -                        
    -                            validate
    -                            validate
    -                            
    -                                check
    -                            
    -                        
    -                    
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-source-plugin
    -                    2.4
    -                    
    -                        
    -                            attach-sources
    -                            
    -                                jar-no-fork
    -                            
    -                        
    -                    
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-javadoc-plugin
    -                    2.10.3
    -                    
    -                        ${java.version}
    -                    
    -                    
    -                        
    -                            attach-javadocs
    -                            
    -                                jar
    -                            
    -                        
    -                    
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-clean-plugin
    -                    3.0.0
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-deploy-plugin
    -                    2.8.2
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-install-plugin
    -                    2.5.2
    -                    
    -                        true
    -                    
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-resources-plugin
    -                    2.7
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-jar-plugin
    -                    2.6
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-site-plugin
    -                    3.4
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-release-plugin
    -                    2.5.3
    -                    
    -                        
    -                            org.apache.maven.scm
    -                            maven-scm-provider-gitexe
    -                            1.9.2
    -                        
    -                    
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-gpg-plugin
    -                    1.6
    -                    
    -                        
    -                            sign-artifacts
    -                            verify
    -                            
    -                              sign
    -                            
    -                        
    -                    
    -                
    -                
    -                    org.apache.rat
    -                    apache-rat-plugin
    -                    0.11
    -                    
    -                        
    -                            .travis.yml
    -                            **/*.conf
    -                            **/*.iml
    -                            **/*.md
    -                            **/*.txt
    -                            **/.project
    -                            **/.checkstyle
    -                            **/.classpath
    -                            **/.settings/
    -                            **/gen/**
    -                            **/resources/**
    -                            **/target/**
    -                            **/dependency-reduced-pom.xml
    -                            false
    -                        
    -                    
    -                    
    -                        
    -                            verify
    -                            verify
    -                            
    -                                check
    -                            
    -                        
    -                    
    -                
    -                
    -                    org.codehaus.mojo
    -                    exec-maven-plugin
    -                    1.4.0
    -                    
    -                        java
    -                        test 
    -                        
    -                            -classpath
    -                            
    -                            ${mainClass}
    -                            --inputFile=${input}
    -                            --output=${output}
    -                            --runner=${runner}
    -                            --sparkMaster=${sparkMaster}
    -                        
    -                    
    -                
    -                
    -                    org.jacoco
    -                    jacoco-maven-plugin
    -                    0.7.5.201505241946
    -                    
    -                        
    -                            
    -                                prepare-agent
    -                            
    -                        
    -                        
    -                            report
    -                            test
    -                            
    -                               report
    -                            
    -                        
    -                    
    -                
    -                
    -                    org.apache.maven.plugins
    -                    maven-shade-plugin
    -                    2.4.3
    -                    
    -                        
    -                            package
    -                            
    -                                shade
    -                            
    -                            
    -                                
    -                                    
    -                                    
    -                                        com.google.common
    -                                        org.apache.beam.spark.relocated.com.google.common
    -                                    
    -                                
    -                                true
    -                                spark-app
    -                                
    -                                    
    -                                
    -                            
    -                        
    -                    
    -                
    -            
    -        
    +  
    +    
    +      
    +        
    +          org.apache.maven.plugins
    +          maven-compiler-plugin
    +          3.5
    +          
    +            ${java.version}
    +            ${java.version}
    +            
    +              -Xlint:all,-serial
    +            
    +          
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-surefire-plugin
    +          2.19.1
    +          
    +            1
    +            false
    +          
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-checkstyle-plugin
    +          2.17
    +          
    +            build-resources/header-file.txt
    +            build-resources/checkstyle.xml
    +          
    +          
    +            
    +              validate
    +              validate
    +              
    +                check
    +              
    +            
    +          
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-source-plugin
    +          2.4
    +          
    +            
    +              attach-sources
    +              
    +                jar-no-fork
    +              
    +            
    +          
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-javadoc-plugin
    +          2.10.3
    +          
    +            ${java.version}
    +          
    +          
    +            
    +              attach-javadocs
    +              
    +                jar
    +              
    +            
    +          
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-clean-plugin
    +          3.0.0
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-deploy-plugin
    +          2.8.2
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-install-plugin
    +          2.5.2
    +          
    +            true
    +          
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-resources-plugin
    +          2.7
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-jar-plugin
    +          2.6
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-site-plugin
    +          3.4
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-release-plugin
    +          2.5.3
    +          
    +            
    +              org.apache.maven.scm
    +              maven-scm-provider-gitexe
    +              1.9.2
    +            
    +          
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-gpg-plugin
    +          1.6
    +          
    +            
    +              sign-artifacts
    +              verify
    +              
    +                sign
    +              
    +            
    +          
    +        
    +        
    +          org.apache.rat
    +          apache-rat-plugin
    +          0.11
    +          
    +            
    +              .travis.yml
    +              **/*.conf
    +              **/*.iml
    +              **/*.md
    +              **/*.txt
    +              **/.project
    +              **/.checkstyle
    +              **/.classpath
    +              **/.settings/
    +              **/gen/**
    +              **/resources/**
    +              **/target/**
    +              **/dependency-reduced-pom.xml
    +              false
    +            
    +          
    +          
    +            
    +              verify
    +              verify
    +              
    +                check
    +              
    +            
    +          
    +        
    +        
    +          org.codehaus.mojo
    +          exec-maven-plugin
    +          1.4.0
    +          
    +            java
    +            test 
    +            
    +              -classpath
    +              
    +              ${mainClass}
    +              --inputFile=${input}
    +              --output=${output}
    +              --runner=${runner}
    +              --sparkMaster=${sparkMaster}
    +            
    +          
    +        
    +        
    +          org.jacoco
    +          jacoco-maven-plugin
    +          0.7.5.201505241946
    +          
    +            
    +              
    +                prepare-agent
    +              
    +            
    +            
    +              report
    +              test
    +              
    +                report
    +              
    +            
    +          
    +        
    +        
    +          org.apache.maven.plugins
    +          maven-shade-plugin
    +          2.4.3
    +          
    +            
    +              package
    +              
    +                shade
    +              
    +              
    +                
    +                  
    +                  
    +                    com.google.common
    +                    org.apache.beam.spark.relocated.com.google.common
    +                  
    +                
    +                true
    +                spark-app
    +                
    +                  
    +                
    +              
    +            
    +          
    +        
    +      
    +    
    +    
    +      
    +        org.apache.maven.plugins
    +        maven-checkstyle-plugin
    +      
    +      
    +        org.apache.rat
    +        apache-rat-plugin
    +      
    +      
    +        org.apache.maven.plugins
    +        maven-source-plugin
    +      
    +      
    +        org.apache.maven.plugins
    +        maven-javadoc-plugin
    +      
    +      
    +        org.apache.maven.plugins
    +        maven-shade-plugin
    +      
    +    
    +  
    +
    +  
    +    
    +      
    +        org.apache.maven.plugins
    +        maven-checkstyle-plugin
    +      
    +      
    +        org.apache.maven.plugins
    +        maven-javadoc-plugin
    +        
    +          
    +            aggregate
    +            
    +              aggregate
    +              test-aggregate
    +            
    +          
    +        
    +      
    +    
    +  
    +
    +  
    +    
    +      jacoco
    +      
             
    -            
    -                org.apache.maven.plugins
    -                maven-checkstyle-plugin
    -            
    -            
    -                org.apache.rat
    -              apache-rat-plugin
    -            
    -            
    -                org.apache.maven.plugins
    -                maven-source-plugin
    -            
    -            
    -                org.apache.maven.plugins
    -                maven-javadoc-plugin
    -            
    -            
    -                org.apache.maven.plugins
    -                maven-shade-plugin
    -            
    +          
    +            org.jacoco
    +            jacoco-maven-plugin
    +          
             
    -    
    +      
    +    
     
    -    
    +    
    +      disable-runnable-on-service-tests
    +      
    +        true
    +      
    +      
             
    -            
    -                org.apache.maven.plugins
    -                maven-checkstyle-plugin
    -            
    -            
    -                org.apache.maven.plugins
    -                maven-javadoc-plugin
    -                
    -                    
    -                        aggregate
    -                        
    -                            aggregate
    -                            test-aggregate
    -                        
    -                    
    -                
    -            
    +          
    +            org.apache.maven.plugins
    +            maven-surefire-plugin
    +            
    +              
    +                runnable-on-service-tests
    +                
    +                  true
    +                
    +              
    +            
    +          
             
    -    
    -
    -    
    -        
    -            jacoco
    -            
    -                
    -                    
    -                        org.jacoco
    -                        jacoco-maven-plugin
    -                    
    -                
    -            
    -        
    -    
    +      
    +    
    +  
     
     
    
    From c1b4f3350b830f726e169999cfe6db0d6b337022 Mon Sep 17 00:00:00 2001
    From: Luke Cwik 
    Date: Tue, 12 Apr 2016 19:54:45 -0700
    Subject: [PATCH 3/3] [BEAM-151] !fixup Fix comment in TestPipeline
    
    ---
     .../com/google/cloud/dataflow/sdk/testing/TestPipeline.java   | 4 ++--
     1 file changed, 2 insertions(+), 2 deletions(-)
    
    diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java
    index f2ca2577aa63..17b1538a9bbf 100644
    --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java
    +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/testing/TestPipeline.java
    @@ -50,8 +50,8 @@
      *
      * 

    In order to run tests on a pipeline runner, the following conditions must be met: *

      - *
    • System property "dataflowOptions" must contain a JSON delimited list of pipeline options. - * For example: + *
    • System property "beamTestPipelineOptions" must contain a JSON delimited list of pipeline + * options. For example: *
      {@code [
        *     "--runner=com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineRunner",
        *     "--project=mygcpproject",