You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by al...@apache.org on 2018/04/05 00:49:36 UTC

[beam] branch master updated: [BEAM-3250] Migrate Apex and Gearpump ValidatesRunner tests to Gradle (#5025)

This is an automated email from the ASF dual-hosted git repository.

altay pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new e42278c  [BEAM-3250] Migrate Apex and Gearpump ValidatesRunner tests to Gradle (#5025)
e42278c is described below

commit e42278c485152fac204e8018cf3134d9d35048fe
Author: Henning Rohde <he...@seekerror.org>
AuthorDate: Wed Apr 4 17:49:30 2018 -0700

    [BEAM-3250] Migrate Apex and Gearpump ValidatesRunner tests to Gradle (#5025)
    
    * [BEAM-3250] Migrate Apex and Gearpump ValidatesRunner tests to Gradle
---
 ...eam_PostCommit_Java_ValidatesRunner_Apex.groovy | 30 ++++++++-----
 ...PostCommit_Java_ValidatesRunner_Gearpump.groovy | 26 ++++++++---
 runners/apex/build.gradle                          | 47 +++++++++++++++++---
 runners/flink/build.gradle                         |  2 +-
 runners/gearpump/build.gradle                      | 51 +++++++++++++++++++---
 .../beam/sdk/testing/UsesParDoLifecycle.java       | 24 ++++++++++
 .../beam/sdk/transforms/ParDoLifecycleTest.java    | 25 ++++++-----
 7 files changed, 164 insertions(+), 41 deletions(-)

diff --git a/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Apex.groovy b/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Apex.groovy
index c16a1e2..512cfa9 100644
--- a/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Apex.groovy
+++ b/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Apex.groovy
@@ -19,15 +19,22 @@
 import common_job_properties
 
 // This job runs the suite of ValidatesRunner tests against the Apex runner.
-mavenJob('beam_PostCommit_Java_ValidatesRunner_Apex') {
+job('beam_PostCommit_Java_ValidatesRunner_Apex_Gradle') {
   description('Runs the ValidatesRunner suite on the Apex runner.')
+  previousNames('beam_PostCommit_Java_ValidatesRunner_Apex')
   previousNames('beam_PostCommit_Java_RunnableOnService_Apex')
 
   // Set common parameters.
   common_job_properties.setTopLevelMainJobProperties(delegate)
 
-  // Set maven parameters.
-  common_job_properties.setMavenConfig(delegate)
+  def gradle_switches = [
+    // Gradle log verbosity enough to diagnose basic build issues
+    "--info",
+    // Continue the build even if there is a failure to show as many potential failures as possible.
+    '--continue',
+    // Until we verify the build cache is working appropriately, force rerunning all tasks
+    '--rerun-tasks',
+  ]
 
   // Sets that this is a PostCommit job.
   common_job_properties.setPostCommit(delegate)
@@ -38,11 +45,14 @@ mavenJob('beam_PostCommit_Java_ValidatesRunner_Apex') {
     'Apache Apex Runner ValidatesRunner Tests',
     'Run Apex ValidatesRunner')
 
-  // Maven goals for this job.
-  goals('''clean verify --projects runners/apex \
-      --also-make \
-      --batch-mode \
-      --errors \
-      --activate-profiles validates-runner-tests \
-      --activate-profiles local-validates-runner-tests''')
+  // Gradle goals for this job.
+  steps {
+    gradle {
+      rootBuildScriptDir(common_job_properties.checkoutDir)
+      tasks(':runners:apex:validatesRunner')
+      for (String gradle_switch : gradle_switches) {
+        switches(gradle_switch)
+      }
+    }
+  }
 }
diff --git a/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Gearpump.groovy b/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Gearpump.groovy
index e1cbafe..8ba0a71 100644
--- a/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Gearpump.groovy
+++ b/.test-infra/jenkins/job_beam_PostCommit_Java_ValidatesRunner_Gearpump.groovy
@@ -20,9 +20,9 @@ import common_job_properties
 
 // This job runs the suite of ValidatesRunner tests against the Gearpump
 // runner.
-mavenJob('beam_PostCommit_Java_ValidatesRunner_Gearpump') {
+job('beam_PostCommit_Java_ValidatesRunner_Gearpump_Gradle') {
   description('Runs the ValidatesRunner suite on the Gearpump runner.')
-
+  previousNames('beam_PostCommit_Java_ValidatesRunner_Gearpump')
   previousNames('beam_PostCommit_Java_RunnableOnService_Gearpump')
 
   // Set common parameters.
@@ -30,8 +30,14 @@ mavenJob('beam_PostCommit_Java_ValidatesRunner_Gearpump') {
       delegate,
       'gearpump-runner')
 
-  // Set maven parameters.
-  common_job_properties.setMavenConfig(delegate)
+  def gradle_switches = [
+    // Gradle log verbosity enough to diagnose basic build issues
+    "--info",
+    // Continue the build even if there is a failure to show as many potential failures as possible.
+    '--continue',
+    // Until we verify the build cache is working appropriately, force rerunning all tasks
+    '--rerun-tasks',
+  ]
 
   // Sets that this is a PostCommit job.
   // 0 5 31 2 * will run on Feb 31 (i.e. never) according to job properties.
@@ -44,6 +50,14 @@ mavenJob('beam_PostCommit_Java_ValidatesRunner_Gearpump') {
     'Apache Gearpump Runner ValidatesRunner Tests',
     'Run Gearpump ValidatesRunner')
 
-  // Maven goals for this job.
-  goals('-B -e clean verify -am -pl runners/gearpump -DforkCount=0 -DvalidatesRunnerPipelineOptions=\'[ "--runner=TestGearpumpRunner"]\'')
+  // Gradle goals for this job.
+  steps {
+    gradle {
+      rootBuildScriptDir(common_job_properties.checkoutDir)
+      tasks(':runners:gearpump:validatesRunner')
+      for (String gradle_switch : gradle_switches) {
+        switches(gradle_switch)
+      }
+    }
+  }
 }
diff --git a/runners/apex/build.gradle b/runners/apex/build.gradle
index 800e594..1ddb855 100644
--- a/runners/apex/build.gradle
+++ b/runners/apex/build.gradle
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+import groovy.json.JsonOutput
+
 apply from: project(":").file("build_rules.gradle")
 applyJavaNature(artifactId: "beam-runners-apex")
 
@@ -30,6 +32,11 @@ description = "Apache Beam :: Runners :: Apex"
  */
 evaluationDependsOn(":model:fn-execution")
 evaluationDependsOn(":runners:core-java")
+evaluationDependsOn(":sdks:java:core")
+
+configurations {
+  validatesRunner
+}
 
 dependencies {
   shadow project(path: ":model:pipeline", configuration: "shadow")
@@ -42,13 +49,15 @@ dependencies {
   shadow library.java.commons_lang3
   shadow library.java.findbugs_jsr305
   shadow library.java.apex_engine
-  testCompile project(path: ":sdks:java:core", configuration: "shadowTest")
+  shadowTest project(path: ":sdks:java:core", configuration: "shadowTest")
   // ApexStateInternalsTest extends abstract StateInternalsTest
-  testCompile project(":runners:core-java").sourceSets.test.output
-  testCompile library.java.hamcrest_core
-  testCompile library.java.junit
-  testCompile library.java.mockito_core
-  testCompile library.java.jackson_dataformat_yaml
+  shadowTest project(":runners:core-java").sourceSets.test.output
+  shadowTest library.java.hamcrest_core
+  shadowTest library.java.junit
+  shadowTest library.java.mockito_core
+  shadowTest library.java.jackson_dataformat_yaml
+  validatesRunner project(path: ":sdks:java:core", configuration: "shadowTest")
+  validatesRunner project(path: project.path, configuration: "shadow")
 }
 
 // TODO: Update this so that the generated file is added to the explicitly added instead of
@@ -73,5 +82,31 @@ task buildDependencyTree(type: DependencyReportTask) {
 }
 compileJava.dependsOn buildDependencyTree
 
+task validatesRunnerBatch(type: Test) {
+  group = "Verification"
+  systemProperty "beamTestPipelineOptions", JsonOutput.toJson([
+          "--runner=TestApexRunner",
+  ])
+
+  classpath = configurations.validatesRunner
+  testClassesDirs = files(project(":sdks:java:core").sourceSets.test.output.classesDirs)
+  useJUnit {
+    includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
+    excludeCategories 'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesTimersInParDo'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesSplittableParDo'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesAttemptedMetrics'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
+  }
+}
+
+task validatesRunner {
+  group = "Verification"
+  description "Validates Apex runner"
+  dependsOn validatesRunnerBatch
+}
+
 // Generates :runners:apex:runQuickstartJavaApex
 createJavaQuickstartValidationTask(name: 'Apex')
diff --git a/runners/flink/build.gradle b/runners/flink/build.gradle
index d6673ae..3df8891 100644
--- a/runners/flink/build.gradle
+++ b/runners/flink/build.gradle
@@ -124,7 +124,7 @@ createValidatesRunnerTask(name: "validatesRunnerStreaming", streaming: true)
 
 task validatesRunner {
   group = "Verification"
-  description "Validates batch and streaming runners"
+  description "Validates Flink runner"
   dependsOn validatesRunnerBatch
   dependsOn validatesRunnerStreaming
 }
diff --git a/runners/gearpump/build.gradle b/runners/gearpump/build.gradle
index cd96dc4..b9181c0 100644
--- a/runners/gearpump/build.gradle
+++ b/runners/gearpump/build.gradle
@@ -16,13 +16,21 @@
  * limitations under the License.
  */
 
+import groovy.json.JsonOutput
+
 apply from: project(":").file("build_rules.gradle")
 applyJavaNature(artifactId: "beam-runners-gearpump")
 
 description = "Apache Beam :: Runners :: Gearpump"
 
+evaluationDependsOn(":sdks:java:core")
+
 def gearpump_version = "0.8.4"
 
+configurations {
+  validatesRunner
+}
+
 dependencies {
   compile library.java.guava
   compileOnly "com.typesafe:config:1.3.0"
@@ -35,10 +43,41 @@ dependencies {
   shadow library.java.joda_time
   shadow library.java.jackson_annotations
   shadow library.java.findbugs_jsr305
-  testCompile project(path: ":sdks:java:core", configuration: "shadowTest")
-  testCompile library.java.junit
-  testCompile library.java.hamcrest_core
-  testCompile library.java.jackson_databind
-  testCompile library.java.jackson_dataformat_yaml
-  testCompile library.java.mockito_core
+  shadowTest project(path: ":sdks:java:core", configuration: "shadowTest")
+  shadowTest library.java.junit
+  shadowTest library.java.hamcrest_core
+  shadowTest library.java.jackson_databind
+  shadowTest library.java.jackson_dataformat_yaml
+  shadowTest library.java.mockito_core
+  validatesRunner project(path: ":sdks:java:core", configuration: "shadowTest")
+  validatesRunner project(path: project.path, configuration: "shadow")
+}
+
+task validatesRunnerStreaming(type: Test) {
+  group = "Verification"
+  systemProperty "beamTestPipelineOptions", JsonOutput.toJson([
+          "--runner=TestGearpumpRunner",
+          "--streaming=true",
+  ])
+
+  classpath = configurations.validatesRunner
+  testClassesDirs = files(project(":sdks:java:core").sourceSets.test.output.classesDirs)
+  useJUnit {
+    includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
+    excludeCategories 'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesStatefulParDo'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesTimersInParDo'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesSplittableParDo'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesAttemptedMetrics'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesCustomWindowMerging'
+    excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
+  }
+}
+
+task validatesRunner {
+  group = "Verification"
+  description "Validates Gearpump runner"
+  dependsOn validatesRunnerStreaming
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesParDoLifecycle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesParDoLifecycle.java
new file mode 100644
index 0000000..9616165
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesParDoLifecycle.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.testing;
+
+/**
+ * Category tag for the ParDoLifecycleTest for exclusion (BEAM-3241).
+ */
+public interface UsesParDoLifecycle {}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
index 9ecc31d..3ba516e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
@@ -31,6 +31,7 @@ import org.apache.beam.sdk.state.StateSpec;
 import org.apache.beam.sdk.state.StateSpecs;
 import org.apache.beam.sdk.state.ValueState;
 import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.UsesParDoLifecycle;
 import org.apache.beam.sdk.testing.UsesStatefulParDo;
 import org.apache.beam.sdk.testing.ValidatesRunner;
 import org.apache.beam.sdk.values.KV;
@@ -53,7 +54,7 @@ public class ParDoLifecycleTest implements Serializable {
   public final transient TestPipeline p = TestPipeline.create();
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testOldFnCallSequence() {
     PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4)))
         .and(p.apply("Polite", Create.of(3, 5, 6, 7)))
@@ -64,7 +65,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testOldFnCallSequenceMulti() {
     PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4)))
         .and(p.apply("Polite", Create.of(3, 5, 6, 7)))
@@ -133,7 +134,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testFnCallSequence() {
     PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4)))
         .and(p.apply("Polite", Create.of(3, 5, 6, 7)))
@@ -144,7 +145,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testFnCallSequenceMulti() {
     PCollectionList.of(p.apply("Impolite", Create.of(1, 2, 4)))
         .and(p.apply("Polite", Create.of(3, 5, 6, 7)))
@@ -157,7 +158,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category({ValidatesRunner.class, UsesStatefulParDo.class})
+  @Category({ValidatesRunner.class, UsesStatefulParDo.class, UsesParDoLifecycle.class})
   public void testFnCallSequenceStateful() {
     PCollectionList.of(p.apply("Impolite", Create.of(KV.of("a", 1), KV.of("b", 2), KV.of("a", 4))))
         .and(
@@ -233,7 +234,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testTeardownCalledAfterExceptionInStartBundle() {
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.START_BUNDLE);
     p
@@ -251,7 +252,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testTeardownCalledAfterExceptionInProcessElement() {
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.PROCESS_ELEMENT);
     p
@@ -269,7 +270,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testTeardownCalledAfterExceptionInFinishBundle() {
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.FINISH_BUNDLE);
     p
@@ -287,7 +288,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testWithContextTeardownCalledAfterExceptionInSetup() {
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.SETUP);
     p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn));
@@ -302,7 +303,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testWithContextTeardownCalledAfterExceptionInStartBundle() {
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.START_BUNDLE);
     p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn));
@@ -317,7 +318,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testWithContextTeardownCalledAfterExceptionInProcessElement() {
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.PROCESS_ELEMENT);
     p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn));
@@ -332,7 +333,7 @@ public class ParDoLifecycleTest implements Serializable {
   }
 
   @Test
-  @Category(ValidatesRunner.class)
+  @Category({ValidatesRunner.class, UsesParDoLifecycle.class})
   public void testWithContextTeardownCalledAfterExceptionInFinishBundle() {
     ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.FINISH_BUNDLE);
     p.apply(Create.of(1, 2, 3)).apply(ParDo.of(fn));

-- 
To stop receiving notification emails like this one, please contact
altay@apache.org.