* {@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..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
@@ -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,9 +50,8 @@
*
* In order to run tests on a pipeline runner, the following conditions must be met: *
{@code [
* "--runner=com.google.cloud.dataflow.sdk.testing.TestDataflowPipelineRunner",
* "--project=mygcpproject",
@@ -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