[ https://issues.apache.org/jira/browse/BEAM-7013?focusedWorklogId=297340&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-297340 ]
ASF GitHub Bot logged work on BEAM-7013: ---------------------------------------- Author: ASF GitHub Bot Created on: 19/Aug/19 17:52 Start Date: 19/Aug/19 17:52 Worklog Time Spent: 10m Work Description: boyuanzz commented on pull request #9144: [BEAM-7013] Integrating ZetaSketch's HLL++ algorithm with Beam URL: https://github.com/apache/beam/pull/9144#discussion_r315333397 ########## File path: sdks/java/extensions/zetasketch/src/test/java/org/apache/beam/sdk/extensions/zetasketch/BigQueryHllSketchCompatibilityIT.java ########## @@ -0,0 +1,145 @@ +/* + * 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.extensions.zetasketch; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.Method; +import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord; +import org.apache.beam.sdk.io.gcp.testing.BigqueryMatcher; +import org.apache.beam.sdk.options.ApplicationNameOptions; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Integration tests for HLL++ sketch compatibility between Beam and BigQuery. The tests verifies + * that HLL++ sketches created in Beam can be processed by BigQuery, and vice versa. + */ +@RunWith(JUnit4.class) +public class BigQueryHllSketchCompatibilityIT { + + private static final String DATASET_NAME = "zetasketch_compatibility_test"; + + // Table for testReadSketchFromBigQuery() + // Schema: only one STRING field named "data". + // Content: prepopulated with 4 rows: "Apple", "Orange", "Banana", "Orange" + private static final String DATA_TABLE_NAME = "hll_data"; + private static final String DATA_FIELD_NAME = "data"; + private static final String QUERY_RESULT_FIELD_NAME = "sketch"; + private static final Long EXPECTED_COUNT = 3L; + + // Table for testWriteSketchToBigQuery() + // Schema: only one BYTES field named "sketch". + // Content: will be overridden by the sketch computed by the test pipeline each time the test runs + private static final String SKETCH_TABLE_NAME = "hll_sketch"; + private static final String SKETCH_FIELD_NAME = "sketch"; + private static final List<String> TEST_DATA = + Arrays.asList("Apple", "Orange", "Banana", "Orange"); + // SHA-1 hash of string "[3]", the string representation of a row that has only one field 3 in it + private static final String EXPECTED_CHECKSUM = "f1e31df9806ce94c5bdbbfff9608324930f4d3f1"; + + /** + * Test that HLL++ sketch computed in BigQuery can be processed by Beam. Hll sketch is computed by + * {@code HLL_COUNT.INIT} in BigQuery and read into Beam; the test verifies that we can run {@link + * HllCount.MergePartial} and {@link HllCount.Extract} on the sketch in Beam to get the correct + * estimated count. + */ + @Test + public void testReadSketchFromBigQuery() { + String tableSpec = String.format("%s.%s", DATASET_NAME, DATA_TABLE_NAME); + String query = + String.format( + "SELECT HLL_COUNT.INIT(%s) AS %s FROM %s", + DATA_FIELD_NAME, QUERY_RESULT_FIELD_NAME, tableSpec); + SerializableFunction<SchemaAndRecord, byte[]> parseQueryResultToByteArray = + (SchemaAndRecord schemaAndRecord) -> + // BigQuery BYTES type corresponds to Java java.nio.ByteBuffer type + ((ByteBuffer) schemaAndRecord.getRecord().get(QUERY_RESULT_FIELD_NAME)).array(); + + TestPipelineOptions options = + TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class); + + Pipeline p = Pipeline.create(options); + PCollection<Long> result = + p.apply( + BigQueryIO.read(parseQueryResultToByteArray) + .fromQuery(query) + .usingStandardSql() + .withMethod(Method.DIRECT_READ) + .withCoder(ByteArrayCoder.of())) + .apply(HllCount.MergePartial.globally()) // no-op, only for testing MergePartial + .apply(HllCount.Extract.globally()); + PAssert.thatSingleton(result).isEqualTo(EXPECTED_COUNT); + p.run().waitUntilFinish(); + } + + /** + * Test that HLL++ sketch computed in Beam can be processed by BigQuery. Hll sketch is computed by + * {@link HllCount.Init} in Beam and written to BigQuery; the test verifies that we can run {@code + * HLL_COUNT.EXTRACT()} on the sketch in BigQuery to get the correct estimated count. + */ + @Test + public void testWriteSketchToBigQuery() { + String tableSpec = String.format("%s.%s", DATASET_NAME, SKETCH_TABLE_NAME); + String query = + String.format("SELECT HLL_COUNT.EXTRACT(%s) FROM %s", SKETCH_FIELD_NAME, tableSpec); + TableSchema tableSchema = + new TableSchema() + .setFields( + Collections.singletonList( + new TableFieldSchema().setName(SKETCH_FIELD_NAME).setType("BYTES"))); + + TestPipelineOptions options = + TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class); + // After the pipeline finishes, BigqueryMatcher will send a query to retrieve the estimated + // count and verifies its correctness using checksum. + options.setOnSuccessMatcher( + BigqueryMatcher.createUsingStandardSql( + options.as(ApplicationNameOptions.class).getAppName(), + options.as(GcpOptions.class).getProject(), + query, + EXPECTED_CHECKSUM)); + + Pipeline p = Pipeline.create(options); + p.apply(Create.of(TEST_DATA)) + .apply(HllCount.Init.forStrings().globally()) + .apply( + BigQueryIO.<byte[]>write() + .to(tableSpec) + .withSchema(tableSchema) + .withFormatFunction(sketch -> new TableRow().set(SKETCH_FIELD_NAME, sketch)) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)); Review comment: Same above. Without creating dataset and table, how can you write to a table? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 297340) Time Spent: 21h 10m (was: 21h) > A new count distinct transform based on BigQuery compatible HyperLogLog++ > implementation > ---------------------------------------------------------------------------------------- > > Key: BEAM-7013 > URL: https://issues.apache.org/jira/browse/BEAM-7013 > Project: Beam > Issue Type: New Feature > Components: extensions-java-sketching, sdk-java-core > Reporter: Yueyang Qiu > Assignee: Yueyang Qiu > Priority: Major > Fix For: 2.16.0 > > Time Spent: 21h 10m > Remaining Estimate: 0h > -- This message was sent by Atlassian Jira (v8.3.2#803003)