You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by lc...@apache.org on 2016/10/19 23:29:16 UTC
[1/3] incubator-beam git commit: [BEAM-605] Create Bigquery Verifier
Repository: incubator-beam
Updated Branches:
refs/heads/master c472e1227 -> 6c6f824aa
[BEAM-605] Create Bigquery Verifier
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8e225d7c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8e225d7c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8e225d7c
Branch: refs/heads/master
Commit: 8e225d7c36812cbcc831269d76279700e29131f7
Parents: c472e12
Author: Mark Liu <ma...@markliu-macbookpro.roam.corp.google.com>
Authored: Thu Sep 22 14:34:20 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Oct 19 16:22:56 2016 -0700
----------------------------------------------------------------------
.../examples/cookbook/BigQueryTornadoesIT.java | 14 +-
.../beam/sdk/testing/BigqueryMatcher.java | 235 +++++++++++++++++++
.../beam/sdk/testing/BigqueryMatcherTest.java | 176 ++++++++++++++
3 files changed, 424 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8e225d7c/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java
index 8bcab4a..7e15389 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java
@@ -18,7 +18,10 @@
package org.apache.beam.examples.cookbook;
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.BigqueryMatcher;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.TestPipelineOptions;
import org.junit.Test;
@@ -35,7 +38,10 @@ public class BigQueryTornadoesIT {
* Options for the BigQueryTornadoes Integration Test.
*/
public interface BigQueryTornadoesITOptions
- extends TestPipelineOptions, BigQueryTornadoes.Options {
+ extends TestPipelineOptions, BigQueryTornadoes.Options, BigQueryOptions {
+ @Default.String("043e8e6ee32384df0cda4c241b8ab897f2ce0f2f")
+ String getChecksum();
+ void setChecksum(String value);
}
@Test
@@ -46,6 +52,12 @@ public class BigQueryTornadoesIT {
options.setOutput(String.format("%s.%s",
"BigQueryTornadoesIT", "monthly_tornadoes_" + System.currentTimeMillis()));
+ String query =
+ String.format("SELECT month, tornado_count FROM [%s]", options.getOutput());
+ options.setOnSuccessMatcher(
+ new BigqueryMatcher(
+ options.getAppName(), options.getProject(), query, options.getChecksum()));
+
BigQueryTornadoes.main(TestPipeline.convertToArgs(options));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8e225d7c/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
new file mode 100644
index 0000000..7646caa
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
@@ -0,0 +1,235 @@
+/*
+ * 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;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.api.client.auth.oauth2.Credential;
+import com.google.api.client.googleapis.auth.oauth2.GoogleCredential;
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.BigqueryScopes;
+import com.google.api.services.bigquery.model.QueryRequest;
+import com.google.api.services.bigquery.model.QueryResponse;
+import com.google.api.services.bigquery.model.TableCell;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.apache.beam.sdk.util.Transport;
+import org.hamcrest.Description;
+import org.hamcrest.TypeSafeMatcher;
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A matcher to verify data in BigQuery by processing given query
+ * and comparing with content's checksum.
+ *
+ * <p>Example:
+ * <pre>{@code [
+ * assertThat(job, new BigqueryMatcher(appName, projectId, queryString, expectedChecksum));
+ * ]}</pre>
+ */
+@NotThreadSafe
+public class BigqueryMatcher extends TypeSafeMatcher<PipelineResult>
+ implements SerializableMatcher<PipelineResult> {
+ private static final Logger LOG = LoggerFactory.getLogger(BigqueryMatcher.class);
+
+ // The maximum number of retries to execute a BigQuery RPC
+ static final int MAX_QUERY_RETRIES = 4;
+
+ // The initial backoff for executing a BigQuery RPC
+ private static final Duration INITIAL_BACKOFF = Duration.standardSeconds(1L);
+
+ // The total number of rows in query response to be formatted for debugging purpose
+ private static final int TOTAL_FORMATTED_ROWS = 20;
+
+ // The backoff factory with initial configs
+ static final FluentBackoff BACKOFF_FACTORY =
+ FluentBackoff.DEFAULT
+ .withMaxRetries(MAX_QUERY_RETRIES)
+ .withInitialBackoff(INITIAL_BACKOFF);
+
+ private final String applicationName;
+ private final String projectId;
+ private final String query;
+ private final String expectedChecksum;
+ private String actualChecksum;
+ private transient QueryResponse response;
+
+ public BigqueryMatcher(
+ String applicationName, String projectId, String query, String expectedChecksum) {
+ validateArgument("applicationName", applicationName);
+ validateArgument("projectId", projectId);
+ validateArgument("query", query);
+ validateArgument("expectedChecksum", expectedChecksum);
+
+ this.applicationName = applicationName;
+ this.projectId = projectId;
+ this.query = query;
+ this.expectedChecksum = expectedChecksum;
+ }
+
+ @Override
+ protected boolean matchesSafely(PipelineResult pipelineResult) {
+ LOG.info("Verifying Bigquery data");
+ Bigquery bigqueryClient = newBigqueryClient(applicationName);
+
+ // execute query
+ LOG.debug("Executing query: {}", query);
+ try {
+ QueryRequest queryContent = new QueryRequest();
+ queryContent.setQuery(query);
+
+ response = queryWithRetries(
+ bigqueryClient, queryContent, Sleeper.DEFAULT, BACKOFF_FACTORY.backoff());
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to fetch BigQuery data.", e);
+ }
+
+ // validate BigQuery response
+ if (response == null || response.getRows() == null || response.getRows().isEmpty()) {
+ return false;
+ }
+
+ // compute checksum
+ actualChecksum = generateHash(response.getRows());
+ LOG.debug("Generated a SHA1 checksum based on queried data: {}", actualChecksum);
+
+ return expectedChecksum.equals(actualChecksum);
+ }
+
+ @VisibleForTesting
+ Bigquery newBigqueryClient(String applicationName) {
+ HttpTransport transport = Transport.getTransport();
+ JsonFactory jsonFactory = Transport.getJsonFactory();
+ Credential credential = getDefaultCredential(transport, jsonFactory);
+
+ return new Bigquery.Builder(transport, jsonFactory, credential)
+ .setApplicationName(applicationName)
+ .build();
+ }
+
+ @VisibleForTesting
+ QueryResponse queryWithRetries(Bigquery bigqueryClient, QueryRequest queryContent,
+ Sleeper sleeper, BackOff backOff)
+ throws IOException, InterruptedException {
+ IOException lastException = null;
+ do {
+ try {
+ return bigqueryClient.jobs().query(projectId, queryContent).execute();
+ } catch (IOException e) {
+ // ignore and retry
+ LOG.warn("Ignore the error and retry the query.");
+ lastException = e;
+ }
+ } while(BackOffUtils.next(sleeper, backOff));
+ throw new IOException(
+ String.format(
+ "Unable to get BigQuery response after retrying %d times", MAX_QUERY_RETRIES),
+ lastException);
+ }
+
+ private void validateArgument(String name, String value) {
+ checkArgument(
+ !Strings.isNullOrEmpty(value), "Expected valid %s, but was %s", name, value);
+ }
+
+ private Credential getDefaultCredential(HttpTransport transport, JsonFactory jsonFactory) {
+ GoogleCredential credential;
+ try {
+ credential = GoogleCredential.getApplicationDefault(transport, jsonFactory);
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to get application default credential.", e);
+ }
+
+ if (credential.createScopedRequired()) {
+ Collection<String> bigqueryScope =
+ Lists.newArrayList(BigqueryScopes.CLOUD_PLATFORM_READ_ONLY);
+ credential = credential.createScoped(bigqueryScope);
+ }
+ return credential;
+ }
+
+ private String generateHash(List<TableRow> rows) {
+ List<HashCode> rowHashes = Lists.newArrayList();
+ for (TableRow row : rows) {
+ List<HashCode> cellHashes = Lists.newArrayList();
+ for (TableCell cell : row.getF()) {
+ cellHashes.add(Hashing.sha1().hashString(cell.toString(), StandardCharsets.UTF_8));
+ }
+ rowHashes.add(Hashing.combineUnordered(cellHashes));
+ }
+ return Hashing.combineUnordered(rowHashes).toString();
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description
+ .appendText("Expected checksum is (")
+ .appendText(expectedChecksum)
+ .appendText(")");
+ }
+
+ @Override
+ public void describeMismatchSafely(PipelineResult pResult, Description description) {
+ String info;
+ if (response == null || response.getRows() == null || response.getRows().isEmpty()) {
+ // invalid query response
+ info = String.format("Invalid BigQuery response: %s", Objects.toString(response));
+ } else {
+ // checksum mismatch
+ info = String.format("was (%s).%n"
+ + "\tTotal number of rows are: %d.%n"
+ + "\tQueried data details:%s",
+ actualChecksum, response.getTotalRows(), formatRows(TOTAL_FORMATTED_ROWS));
+ }
+ description.appendText(info);
+ }
+
+ private String formatRows(int totalNumRows) {
+ StringBuilder samples = new StringBuilder();
+ List<TableRow> rows = response.getRows();
+ for (int i = 0; i < totalNumRows && i < rows.size(); i++) {
+ samples.append("\n\t\t");
+ for (TableCell field : rows.get(i).getF()) {
+ samples.append(String.format("%-10s", field.getV()));
+ }
+ }
+ if (rows.size() > totalNumRows) {
+ samples.append("\n\t\t....");
+ }
+ return samples.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8e225d7c/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
new file mode 100644
index 0000000..1c427f8
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.api.services.bigquery.Bigquery;
+import com.google.api.services.bigquery.model.QueryRequest;
+import com.google.api.services.bigquery.model.QueryResponse;
+import com.google.api.services.bigquery.model.TableCell;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.math.BigInteger;
+import org.apache.beam.sdk.PipelineResult;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+/**
+ * Tests for {@link BigqueryMatcher}.
+ */
+@RunWith(JUnit4.class)
+public class BigqueryMatcherTest {
+ private final String appName = "test-app";
+ private final String projectId = "test-project";
+ private final String query = "test-query";
+
+ @Rule public ExpectedException thrown = ExpectedException.none();
+ @Rule public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper();
+ @Mock private Bigquery mockBigqueryClient;
+ @Mock private Bigquery.Jobs mockJobs;
+ @Mock private Bigquery.Jobs.Query mockQuery;
+ @Mock private PipelineResult mockResult;
+
+ @Before
+ public void setUp() throws IOException {
+ MockitoAnnotations.initMocks(this);
+ when(mockBigqueryClient.jobs()).thenReturn(mockJobs);
+ when(mockJobs.query(anyString(), any(QueryRequest.class))).thenReturn(mockQuery);
+ }
+
+ @Test
+ public void testBigqueryMatcherThatSucceeds() throws Exception {
+ BigqueryMatcher matcher = spy(
+ new BigqueryMatcher(
+ appName, projectId, query, "8d1bbbf1f523f924b98c88b00c5811e041c2f855"));
+ doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
+ when(mockQuery.execute()).thenReturn(createResponseContainingTestData());
+
+ assertThat(mockResult, matcher);
+ verify(matcher).newBigqueryClient(eq(appName));
+ verify(mockJobs).query(eq(projectId), eq(new QueryRequest().setQuery(query)));
+ }
+
+ @Test
+ public void testBigqueryMatcherFailsForChecksumMismatch() throws IOException {
+ BigqueryMatcher matcher = spy(
+ new BigqueryMatcher(appName, projectId, query, "incorrect-checksum"));
+ doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
+ when(mockQuery.execute()).thenReturn(createResponseContainingTestData());
+
+ try {
+ assertThat(mockResult, matcher);
+ } catch (AssertionError expected) {
+ assertThat(expected.getMessage(), containsString("Total number of rows are: 1"));
+ assertThat(expected.getMessage(), containsString("abc"));
+ verify(matcher).newBigqueryClient(eq(appName));
+ verify(mockJobs).query(eq(projectId), eq(new QueryRequest().setQuery(query)));
+ }
+ }
+
+ @Test
+ public void testBigqueryMatcherFailsWhenResponseIsNull() throws IOException {
+ testMatcherFailsSinceInvalidQueryResponse(null);
+ }
+
+ @Test
+ public void testBigqueryMatcherFailsWhenNullRowsInResponse() throws IOException {
+ testMatcherFailsSinceInvalidQueryResponse(new QueryResponse());
+ }
+
+ @Test
+ public void testBigqueryMatcherFailsWhenEmptyRowsInResponse() throws IOException {
+ QueryResponse response = new QueryResponse();
+ response.setRows(Lists.<TableRow>newArrayList());
+
+ testMatcherFailsSinceInvalidQueryResponse(response);
+ }
+
+ private void testMatcherFailsSinceInvalidQueryResponse(QueryResponse response)
+ throws IOException {
+ BigqueryMatcher matcher = spy(
+ new BigqueryMatcher(appName, projectId, query, "some-checksum"));
+ doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
+ when(mockQuery.execute()).thenReturn(response);
+
+ try {
+ assertThat(mockResult, matcher);
+ } catch (AssertionError expected) {
+ assertThat(expected.getMessage(), containsString("Invalid BigQuery response:"));
+ verify(matcher).newBigqueryClient(eq(appName));
+ verify(mockJobs).query(eq(projectId), eq(new QueryRequest().setQuery(query)));
+ return;
+ }
+ // Note that fail throws an AssertionError which is why it is placed out here
+ // instead of inside the try-catch block.
+ fail("AssertionError is expected.");
+ }
+
+ @Test
+ public void testQueryWithRetriesWhenServiceFails() throws Exception {
+ BigqueryMatcher matcher = spy(
+ new BigqueryMatcher(appName, projectId, query, "some-checksum"));
+ when(mockQuery.execute()).thenThrow(new IOException());
+
+ thrown.expect(IOException.class);
+ thrown.expectMessage("Unable to get BigQuery response after retrying");
+
+ matcher.queryWithRetries(
+ mockBigqueryClient,
+ new QueryRequest(),
+ fastClock,
+ BigqueryMatcher.BACKOFF_FACTORY.backoff());
+
+ verify(matcher).newBigqueryClient(eq(appName));
+ verify(mockJobs, times(BigqueryMatcher.MAX_QUERY_RETRIES))
+ .query(eq(projectId), eq(new QueryRequest().setQuery(query)));
+ }
+
+ private QueryResponse createResponseContainingTestData() {
+ TableCell field1 = new TableCell();
+ field1.setV("abc");
+ TableCell field2 = new TableCell();
+ field2.setV("2");
+ TableCell field3 = new TableCell();
+ field3.setV("testing BigQuery matcher.");
+ TableRow row = new TableRow();
+ row.setF(Lists.newArrayList(field1, field2, field3));
+
+ QueryResponse response = new QueryResponse();
+ response.setRows(Lists.newArrayList(row));
+ response.setTotalRows(BigInteger.ONE);
+ return response;
+ }
+}
[2/3] incubator-beam git commit: Add BigQuery Verifier to
WindowedWordCountIT
Posted by lc...@apache.org.
Add BigQuery Verifier to WindowedWordCountIT
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/dd46523d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/dd46523d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/dd46523d
Branch: refs/heads/master
Commit: dd46523dc2bca4aee11265a2fb065cc137920b1d
Parents: 8e225d7
Author: Mark Liu <ma...@markliu-macbookpro.roam.corp.google.com>
Authored: Thu Oct 6 14:34:55 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Oct 19 16:22:57 2016 -0700
----------------------------------------------------------------------
.../apache/beam/examples/WindowedWordCountIT.java | 11 +++++++++++
.../beam/examples/cookbook/BigQueryTornadoesIT.java | 2 +-
.../apache/beam/sdk/testing/BigqueryMatcher.java | 16 ++++++++++------
.../beam/sdk/testing/BigqueryMatcherTest.java | 2 +-
4 files changed, 23 insertions(+), 8 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dd46523d/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java
index 379d1b0..6742654 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java
@@ -19,8 +19,10 @@ package org.apache.beam.examples;
import java.io.IOException;
import org.apache.beam.examples.WindowedWordCount.Options;
+import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.testing.BigqueryMatcher;
import org.apache.beam.sdk.testing.StreamingIT;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.TestPipelineOptions;
@@ -40,6 +42,9 @@ public class WindowedWordCountIT {
*/
public interface WindowedWordCountITOptions
extends Options, TestPipelineOptions, StreamingOptions {
+ @Default.String("ff54f6f42b2afeb146206c1e8e915deaee0362b4")
+ String getChecksum();
+ void setChecksum(String value);
}
@Test
@@ -59,6 +64,12 @@ public class WindowedWordCountIT {
TestPipeline.testingPipelineOptions().as(WindowedWordCountITOptions.class);
options.setStreaming(isStreaming);
+ String query = String.format("SELECT word, SUM(count) FROM [%s:%s.%s] GROUP BY word",
+ options.getProject(), options.getBigQueryDataset(), options.getBigQueryTable());
+ options.setOnSuccessMatcher(
+ new BigqueryMatcher(
+ options.getAppName(), options.getProject(), query, options.getChecksum()));
+
WindowedWordCount.main(TestPipeline.convertToArgs(options));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dd46523d/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java
----------------------------------------------------------------------
diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java
index 7e15389..27a5a8f 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java
@@ -39,7 +39,7 @@ public class BigQueryTornadoesIT {
*/
public interface BigQueryTornadoesITOptions
extends TestPipelineOptions, BigQueryTornadoes.Options, BigQueryOptions {
- @Default.String("043e8e6ee32384df0cda4c241b8ab897f2ce0f2f")
+ @Default.String("1ab4c7ec460b94bbb3c3885b178bf0e6bed56e1f")
String getChecksum();
void setChecksum(String value);
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dd46523d/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
index 7646caa..95208ce 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/BigqueryMatcher.java
@@ -40,8 +40,10 @@ import com.google.common.hash.Hashing;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
+import java.util.Collections;
import java.util.List;
import java.util.Objects;
+import javax.annotation.Nonnull;
import javax.annotation.concurrent.NotThreadSafe;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.util.FluentBackoff;
@@ -182,14 +184,16 @@ public class BigqueryMatcher extends TypeSafeMatcher<PipelineResult>
return credential;
}
- private String generateHash(List<TableRow> rows) {
+ private String generateHash(@Nonnull List<TableRow> rows) {
List<HashCode> rowHashes = Lists.newArrayList();
for (TableRow row : rows) {
- List<HashCode> cellHashes = Lists.newArrayList();
+ List<String> cellsInOneRow = Lists.newArrayList();
for (TableCell cell : row.getF()) {
- cellHashes.add(Hashing.sha1().hashString(cell.toString(), StandardCharsets.UTF_8));
+ cellsInOneRow.add(Objects.toString(cell.getV()));
+ Collections.sort(cellsInOneRow);
}
- rowHashes.add(Hashing.combineUnordered(cellHashes));
+ rowHashes.add(
+ Hashing.sha1().hashString(cellsInOneRow.toString(), StandardCharsets.UTF_8));
}
return Hashing.combineUnordered(rowHashes).toString();
}
@@ -222,13 +226,13 @@ public class BigqueryMatcher extends TypeSafeMatcher<PipelineResult>
StringBuilder samples = new StringBuilder();
List<TableRow> rows = response.getRows();
for (int i = 0; i < totalNumRows && i < rows.size(); i++) {
- samples.append("\n\t\t");
+ samples.append(String.format("%n\t\t"));
for (TableCell field : rows.get(i).getF()) {
samples.append(String.format("%-10s", field.getV()));
}
}
if (rows.size() > totalNumRows) {
- samples.append("\n\t\t....");
+ samples.append(String.format("%n\t\t..."));
}
return samples.toString();
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dd46523d/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
index 1c427f8..d0ae765 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/BigqueryMatcherTest.java
@@ -74,7 +74,7 @@ public class BigqueryMatcherTest {
public void testBigqueryMatcherThatSucceeds() throws Exception {
BigqueryMatcher matcher = spy(
new BigqueryMatcher(
- appName, projectId, query, "8d1bbbf1f523f924b98c88b00c5811e041c2f855"));
+ appName, projectId, query, "9bb47f5c90d2a99cad526453dff5ed5ec74650dc"));
doReturn(mockBigqueryClient).when(matcher).newBigqueryClient(anyString());
when(mockQuery.execute()).thenReturn(createResponseContainingTestData());
[3/3] incubator-beam git commit: [BEAM-605] Create Bigquery Verifier
Posted by lc...@apache.org.
[BEAM-605] Create Bigquery Verifier
This closes #990
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/6c6f824a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/6c6f824a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/6c6f824a
Branch: refs/heads/master
Commit: 6c6f824aa25b90f094ecec189c5ad6a0ccf365cf
Parents: c472e12 dd46523
Author: Luke Cwik <lc...@google.com>
Authored: Wed Oct 19 16:23:21 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Oct 19 16:23:21 2016 -0700
----------------------------------------------------------------------
.../beam/examples/WindowedWordCountIT.java | 11 +
.../examples/cookbook/BigQueryTornadoesIT.java | 14 +-
.../beam/sdk/testing/BigqueryMatcher.java | 239 +++++++++++++++++++
.../beam/sdk/testing/BigqueryMatcherTest.java | 176 ++++++++++++++
4 files changed, 439 insertions(+), 1 deletion(-)
----------------------------------------------------------------------