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(-)
----------------------------------------------------------------------