Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 10 additions & 0 deletions runners/google-cloud-dataflow-java/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -287,11 +287,21 @@
<artifactId>java-sdk-all</artifactId>
</dependency>

<dependency>
<groupId>org.apache.avro</groupId>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm surprised this validation required adding dependencies on avro and datastore. Is there a reason these dependencies appeared now but not before?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Previously there was no direct dependency on Avro/Datastore in Dataflow runner. This change adds integration tests which directly reference classes in these packages.

<artifactId>avro</artifactId>
</dependency>

<dependency>
<groupId>com.google.api-client</groupId>
<artifactId>google-api-client</artifactId>
</dependency>

<dependency>
<groupId>com.google.apis</groupId>
<artifactId>google-api-services-datastore-protobuf</artifactId>
</dependency>

<dependency>
<groupId>com.google.oauth-client</groupId>
<artifactId>google-oauth-client</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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.runners.dataflow.io;

import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;

import static org.hamcrest.Matchers.hasItem;
import static org.junit.Assert.assertThat;

import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
import org.apache.beam.runners.dataflow.transforms.DataflowDisplayDataEvaluator;
import org.apache.beam.sdk.io.AvroIO;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;

import org.apache.avro.Schema;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

import java.util.Set;

/**
* {@link DataflowPipelineRunner} specific tests for {@link AvroIO} transforms.
*/
@RunWith(JUnit4.class)
public class DataflowAvroIOTest {
@Test
public void testPrimitiveWriteDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();

AvroIO.Write.Bound<?> write = AvroIO.Write
.to("foo")
.withSchema(Schema.create(Schema.Type.STRING))
.withoutValidation();

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
assertThat("AvroIO.Write should include the file pattern in its primitive transform",
displayData, hasItem(hasDisplayItem("fileNamePattern")));
}

@Test
public void testPrimitiveReadDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();

AvroIO.Read.Bound<?> read = AvroIO.Read.from("foo.*")
.withSchema(Schema.create(Schema.Type.STRING))
.withoutValidation();

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
assertThat("AvroIO.Read should include the file pattern in its primitive transform",
displayData, hasItem(hasDisplayItem("filePattern")));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.beam.runners.dataflow.io;

import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey;

import static org.hamcrest.Matchers.hasItem;
import static org.junit.Assert.assertThat;
Expand All @@ -39,6 +38,30 @@
* Unit tests for Dataflow usage of {@link BigQueryIO} transforms.
*/
public class DataflowBigQueryIOTest {
@Test
public void testTableSourcePrimitiveDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();
BigQueryIO.Read.Bound read = BigQueryIO.Read
.from("project:dataset.tableId")
.withoutValidation();

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
assertThat("BigQueryIO.Read should include the table spec in its primitive display data",
displayData, hasItem(hasDisplayItem("table")));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This assertion reads unfortunately -- hasItem(hasDisplayItem("table")) -- but I don't see a clean way to fix it without introducing a data structure to wrap the Set<DisplayData>. Let's leave it unless you have any thoughts.

}

@Test
public void testQuerySourcePrimitiveDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();
BigQueryIO.Read.Bound read = BigQueryIO.Read
.fromQuery("foobar")
.withoutValidation();

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
assertThat("BigQueryIO.Read should include the query in its primitive display data",
displayData, hasItem(hasDisplayItem("query")));
}

@Test
public void testBatchSinkPrimitiveDisplayData() {
DataflowPipelineOptions options = DataflowDisplayDataEvaluator.getDefaultOptions();
Expand All @@ -63,9 +86,9 @@ private void testSinkPrimitiveDisplayData(DataflowPipelineOptions options) {

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
assertThat("BigQueryIO.Write should include the table spec in its primitive display data",
displayData, hasItem(hasDisplayItem(hasKey("tableSpec"))));
displayData, hasItem(hasDisplayItem("tableSpec")));

assertThat("BigQueryIO.Write should include the table schema in its primitive display data",
displayData, hasItem(hasDisplayItem(hasKey("schema"))));
displayData, hasItem(hasDisplayItem("schema")));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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.runners.dataflow.io;

import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;

import static org.hamcrest.Matchers.hasItem;
import static org.junit.Assert.assertThat;

import org.apache.beam.runners.dataflow.transforms.DataflowDisplayDataEvaluator;
import org.apache.beam.sdk.io.DatastoreIO;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PInput;

import com.google.api.services.datastore.DatastoreV1;

import org.junit.Test;

import java.util.Set;

/**
* Unit tests for Dataflow usage of {@link DatastoreIO} transforms.
*/
public class DataflowDatastoreIOTest {
@Test
public void testSourcePrimitiveDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();
PTransform<PInput, ?> read = DatastoreIO.readFrom(
"myDataset", DatastoreV1.Query.newBuilder().build());

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
assertThat("DatastoreIO read should include the dataset in its primitive display data",
displayData, hasItem(hasDisplayItem("dataset")));
}

@Test
public void testSinkPrimitiveDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();
PTransform<PCollection<DatastoreV1.Entity>, ?> write = DatastoreIO.writeTo("myDataset");

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
assertThat("DatastoreIO write should include the dataset in its primitive display data",
displayData, hasItem(hasDisplayItem("dataset")));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.runners.dataflow.io;

import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;

import static org.hamcrest.Matchers.hasItem;
import static org.junit.Assert.assertThat;

import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
import org.apache.beam.runners.dataflow.transforms.DataflowDisplayDataEvaluator;
import org.apache.beam.sdk.io.PubsubIO;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;

import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

import java.util.Set;

/**
* {@link DataflowPipelineRunner} specific tests for {@link PubsubIO} transforms.
*/
@RunWith(JUnit4.class)
public class DataflowPubsubIOTest {
@Test
public void testPrimitiveWriteDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();
PubsubIO.Write.Bound<?> write = PubsubIO.Write
.topic("projects/project/topics/topic");

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
assertThat("PubsubIO.Write should include the topic in its primitive display data",
displayData, hasItem(hasDisplayItem("topic")));
}

@Test
public void testPrimitiveReadDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();
PubsubIO.Read.Bound<String> read = PubsubIO.Read.topic("projects/project/topics/topic");

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
assertThat("PubsubIO.Read should include the topic in its primitive display data",
displayData, hasItem(hasDisplayItem("topic")));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ private void applyRead(Pipeline pipeline, String path) {
}

@Test
public void testPrimitiveDisplayData() {
public void testPrimitiveWriteDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();

TextIO.Write.Bound<?> write = TextIO.Write.to("foobar");
Expand All @@ -137,4 +137,17 @@ public void testPrimitiveDisplayData() {
assertThat("TextIO.Write should include the file prefix in its primitive display data",
displayData, hasItem(hasDisplayItem(hasValue(startsWith("foobar")))));
}

@Test
public void testPrimitiveReadDisplayData() {
DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();

TextIO.Read.Bound<String> read = TextIO.Read
.from("foobar")
.withoutValidation();

Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
assertThat("TextIO.Read should include the file prefix in its primitive display data",
displayData, hasItem(hasDisplayItem(hasValue(startsWith("foobar")))));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ private DataflowDisplayDataEvaluator() {}
public static DataflowPipelineOptions getDefaultOptions() {
DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);

options.setRunner(DataflowPipelineRunner.class);
options.setProject("foobar");
options.setTempLocation("gs://bucket/tmpLocation");
options.setFilesToStage(Lists.<String>newArrayList());
Expand All @@ -66,6 +67,6 @@ public static DisplayDataEvaluator create() {
* the {@link DataflowPipelineRunner} with the specified {@code options}.
*/
public static DisplayDataEvaluator create(DataflowPipelineOptions options) {
return DisplayDataEvaluator.forRunner(DataflowPipelineRunner.class, options);
return DisplayDataEvaluator.create(options);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -802,6 +802,12 @@ public synchronized long getEstimatedSizeBytes(PipelineOptions options) throws E
protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception {
// Do nothing.
}

@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder.add(DisplayData.item("table", jsonTable));
}
}

/**
Expand Down Expand Up @@ -893,6 +899,11 @@ protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception {
tableService.deleteDataset(tableToRemove.getProjectId(), tableToRemove.getDatasetId());
}

@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder.add(DisplayData.item("query", query));
}
private synchronized JobStatistics dryRunQueryIfNeeded(BigQueryOptions bqOptions)
throws InterruptedException, IOException {
if (dryRunJobStats.get() == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1030,6 +1030,11 @@ private void publish() throws IOException {
checkState(n == output.size());
output.clear();
}

@Override
public void populateDisplayData(DisplayData.Builder builder) {
Bound.this.populateDisplayData(builder);
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
import static org.apache.beam.sdk.io.BigQueryIO.fromJsonString;
import static org.apache.beam.sdk.io.BigQueryIO.toJsonString;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThat;
Expand Down Expand Up @@ -710,8 +710,8 @@ public void testBuildSinkDisplayData() {

DisplayData displayData = DisplayData.from(write);

assertThat(displayData, hasDisplayItem(hasKey("table")));
assertThat(displayData, hasDisplayItem(hasKey("schema")));
assertThat(displayData, hasDisplayItem("table"));
assertThat(displayData, hasDisplayItem("schema"));
assertThat(displayData,
hasDisplayItem("createDisposition", CreateDisposition.CREATE_IF_NEEDED.toString()));
assertThat(displayData,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.beam.sdk.io;

import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom;

import static org.hamcrest.Matchers.containsString;
Expand Down Expand Up @@ -353,7 +352,7 @@ public void populateDisplayData(DisplayData.Builder builder) {
DisplayData compressedSourceDisplayData = DisplayData.from(compressedSource);
DisplayData gzipDisplayData = DisplayData.from(gzipSource);

assertThat(compressedSourceDisplayData, hasDisplayItem(hasKey("compressionMode")));
assertThat(compressedSourceDisplayData, hasDisplayItem("compressionMode"));
assertThat(gzipDisplayData, hasDisplayItem("compressionMode", CompressionMode.GZIP.toString()));
assertThat(compressedSourceDisplayData, hasDisplayItem("source", inputSource.getClass()));
assertThat(compressedSourceDisplayData, includesDisplayDataFrom(inputSource));
Expand Down
Loading