-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-10652] Allow Clustering without Partition in BigQuery #16578
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
59 commits
Select commit
Hold shift + click to select a range
b58b1a6
[BEAM-10652] removed check that blocked clustering without partitioning
brucearctor f111ff2
Merge branch 'BEAM-10652' of github.com:brucearctor/beam
brucearctor f6a98e1
Merge branch 'master' of github.com:apache/beam
brucearctor f28ced6
Merge branch 'master' of github.com:apache/beam
brucearctor 1b91ec7
Merge branch 'master' of github.com:apache/beam
brucearctor c033139
[BEAM-10652] allow clustering without requiring partition
brucearctor 2cf0dbe
newline
brucearctor 60b440f
added needed null
brucearctor ab9c9cb
remove testClusteringThrowsWithoutPartitioning
brucearctor 49378fd
update clustering
brucearctor c214c95
formatting
brucearctor b2dcf0a
now compiles
brucearctor 40c7c66
passes spotless
brucearctor a944c33
update doc
brucearctor de76a43
focus on single test
brucearctor 4556a16
Merge branch 'master' of github.com:apache/beam
brucearctor e382310
Merge branch 'BEAM-10652' of github.com:brucearctor/beam into BEAM-10652
brucearctor 471f210
Merge branch 'master' of github.com:apache/beam into BEAM-10652
brucearctor 393094b
Merge branch 'master' of github.com:apache/beam into BEAM-10652
brucearctor 46defbd
spotless
brucearctor 9aeca4d
run all ITs
brucearctor 22e43d6
spotless
brucearctor 7ea7837
testing with time partitioning
brucearctor 314feab
checking
brucearctor 12429b4
set clustering independant of partitioning
brucearctor adb4c5d
remove timepart from it
brucearctor 7011de1
spotless
brucearctor 029222b
removed test
brucearctor 29fb91e
added TODO
brucearctor 8c22606
removed block of unneded code/comment
brucearctor 0d2cf18
remove override to v3 coder
brucearctor e587c0a
Spotless cleanup
brucearctor 39c059f
re-add override to v3 coder
brucearctor 81dbb74
spotless
brucearctor 9a3f81b
adding checksum ( wrong value )
brucearctor e4c9154
added needed query var
brucearctor 8a30468
use tableName as var
brucearctor 6255ff0
DATASET NAME
brucearctor e2ff347
project name in query
brucearctor 43bc3c2
update query
brucearctor f9f706e
change tests
brucearctor 7c55fc2
remove unneeded imports
brucearctor f044c13
remove rest of forgotten
brucearctor 88f76db
add rows
brucearctor 66786c7
16000 bytes
brucearctor 8a94697
bigint
brucearctor c86590f
streaming test
brucearctor 093f7d0
spotless
brucearctor db2164c
methods
brucearctor e8b71c7
end stream
brucearctor 94af35c
stream method and naming
brucearctor a82157c
nostream
brucearctor d6710a0
streaming
brucearctor 5b08ef8
streamingoptions
brucearctor 2b3c0cf
without streaming example
brucearctor bae4bd8
string column instead of date -- related to BEAM-13753
brucearctor 27ec3d7
mor strings
brucearctor 3ba86f4
spotless
brucearctor 90de8f9
revert, only DEFAULT and FILE_LOADS
brucearctor File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
185 changes: 185 additions & 0 deletions
185
...loud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryClusteringIT.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,185 @@ | ||
| /* | ||
| * 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.io.gcp.bigquery; | ||
|
|
||
| import com.google.api.services.bigquery.Bigquery; | ||
| import com.google.api.services.bigquery.model.Clustering; | ||
| import com.google.api.services.bigquery.model.Table; | ||
| 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.math.BigInteger; | ||
| import java.util.Arrays; | ||
| import org.apache.beam.sdk.Pipeline; | ||
| import org.apache.beam.sdk.coders.Coder; | ||
| import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; | ||
| import org.apache.beam.sdk.options.Default; | ||
| import org.apache.beam.sdk.options.Description; | ||
| import org.apache.beam.sdk.options.ExperimentalOptions; | ||
| import org.apache.beam.sdk.options.PipelineOptionsFactory; | ||
| import org.apache.beam.sdk.testing.TestPipeline; | ||
| import org.apache.beam.sdk.testing.TestPipelineOptions; | ||
| import org.apache.beam.sdk.transforms.DoFn; | ||
| import org.apache.beam.sdk.transforms.ParDo; | ||
| import org.apache.beam.sdk.values.ValueInSingleWindow; | ||
| import org.checkerframework.checker.nullness.qual.Nullable; | ||
| import org.junit.Assert; | ||
| import org.junit.Before; | ||
| import org.junit.Test; | ||
| import org.junit.runner.RunWith; | ||
| import org.junit.runners.JUnit4; | ||
|
|
||
| /** Integration test that clusters sample data in BigQuery. */ | ||
| @RunWith(JUnit4.class) | ||
| public class BigQueryClusteringIT { | ||
| private static final Long EXPECTED_BYTES = 16000L; | ||
| private static final BigInteger EXPECTED_ROWS = new BigInteger("1000"); | ||
| private static final String WEATHER_SAMPLES_TABLE = | ||
| "clouddataflow-readonly:samples.weather_stations"; | ||
| private static final String DATASET_NAME = "BigQueryClusteringIT"; | ||
| private static final Clustering CLUSTERING = | ||
| new Clustering().setFields(Arrays.asList("station_number")); | ||
| private static final TableSchema SCHEMA = | ||
| new TableSchema() | ||
| .setFields( | ||
| Arrays.asList( | ||
| new TableFieldSchema().setName("station_number").setType("INTEGER"), | ||
| new TableFieldSchema().setName("date").setType("DATE"))); | ||
|
|
||
| private Bigquery bqClient; | ||
| private BigQueryClusteringITOptions options; | ||
|
|
||
| @Before | ||
| public void setUp() { | ||
| PipelineOptionsFactory.register(BigQueryClusteringITOptions.class); | ||
| options = TestPipeline.testingPipelineOptions().as(BigQueryClusteringITOptions.class); | ||
| options.setTempLocation(options.getTempRoot() + "/temp-it/"); | ||
| bqClient = BigqueryClient.getNewBigqueryClient(options.getAppName()); | ||
| } | ||
|
|
||
| /** Customized PipelineOptions for BigQueryClustering Integration Test. */ | ||
| public interface BigQueryClusteringITOptions | ||
| extends TestPipelineOptions, ExperimentalOptions, BigQueryOptions { | ||
| @Description("Table to read from, specified as " + "<project_id>:<dataset_id>.<table_id>") | ||
| @Default.String(WEATHER_SAMPLES_TABLE) | ||
| String getBqcInput(); | ||
|
|
||
| void setBqcInput(String value); | ||
| } | ||
|
|
||
| static class KeepStationNumberAndConvertDate extends DoFn<TableRow, TableRow> { | ||
| @ProcessElement | ||
| public void processElement(ProcessContext c) { | ||
| String day = (String) c.element().get("day"); | ||
| String month = (String) c.element().get("month"); | ||
| String year = (String) c.element().get("year"); | ||
|
|
||
| TableRow row = new TableRow(); | ||
| row.set("station_number", c.element().get("station_number")); | ||
| row.set("date", String.format("%s-%s-%s", year, month, day)); | ||
| c.output(row); | ||
| } | ||
| } | ||
|
|
||
| static class ClusteredDestinations extends DynamicDestinations<TableRow, TableDestination> { | ||
| private final String tableName; | ||
|
|
||
| public ClusteredDestinations(String tableName) { | ||
| this.tableName = tableName; | ||
| } | ||
|
|
||
| @Override | ||
| public @Nullable Coder<TableDestination> getDestinationCoder() { | ||
| return TableDestinationCoderV3.of(); | ||
| } | ||
|
|
||
| @Override | ||
| public TableDestination getDestination(ValueInSingleWindow<TableRow> element) { | ||
| return new TableDestination( | ||
| String.format("%s.%s", DATASET_NAME, tableName), null, null, CLUSTERING); | ||
| } | ||
|
|
||
| @Override | ||
| public TableDestination getTable(TableDestination destination) { | ||
| return destination; | ||
| } | ||
|
|
||
| @Override | ||
| public TableSchema getSchema(TableDestination destination) { | ||
| return SCHEMA; | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| public void testE2EBigQueryClusteringNoPartitionTableFunction() throws Exception { | ||
| String tableName = "weather_stations_clustered_table_function_" + System.currentTimeMillis(); | ||
|
|
||
| Pipeline p = Pipeline.create(options); | ||
|
|
||
| p.apply(BigQueryIO.readTableRows().from(options.getBqcInput())) | ||
| .apply(ParDo.of(new KeepStationNumberAndConvertDate())) | ||
| .apply( | ||
| BigQueryIO.writeTableRows() | ||
| .to( | ||
| (ValueInSingleWindow<TableRow> vsw) -> | ||
| new TableDestination( | ||
| String.format("%s.%s", DATASET_NAME, tableName), | ||
| null, | ||
| null, | ||
| CLUSTERING)) | ||
| .withClustering(CLUSTERING) | ||
| .withSchema(SCHEMA) | ||
| .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) | ||
| .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE) | ||
| .withMethod(BigQueryIO.Write.Method.DEFAULT)); | ||
|
|
||
| p.run().waitUntilFinish(); | ||
|
|
||
| Table table = bqClient.tables().get(options.getProject(), DATASET_NAME, tableName).execute(); | ||
|
|
||
| Assert.assertEquals(CLUSTERING, table.getClustering()); | ||
brucearctor marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| Assert.assertEquals(EXPECTED_BYTES, table.getNumBytes()); | ||
| Assert.assertEquals(EXPECTED_ROWS, table.getNumRows()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testE2EBigQueryClusteringNoPartitionDynamicDestinations() throws Exception { | ||
| String tableName = | ||
| "weather_stations_clustered_dynamic_destinations_" + System.currentTimeMillis(); | ||
|
|
||
| Pipeline p = Pipeline.create(options); | ||
|
|
||
| p.apply(BigQueryIO.readTableRows().from(options.getBqcInput())) | ||
| .apply(ParDo.of(new KeepStationNumberAndConvertDate())) | ||
| .apply( | ||
| BigQueryIO.writeTableRows() | ||
| .to(new ClusteredDestinations(tableName)) | ||
| .withClustering(CLUSTERING) | ||
| .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) | ||
| .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE) | ||
| .withMethod(BigQueryIO.Write.Method.FILE_LOADS)); | ||
|
|
||
| p.run().waitUntilFinish(); | ||
|
|
||
| Table table = bqClient.tables().get(options.getProject(), DATASET_NAME, tableName).execute(); | ||
|
|
||
| Assert.assertEquals(CLUSTERING, table.getClustering()); | ||
brucearctor marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| Assert.assertEquals(EXPECTED_ROWS, table.getNumRows()); | ||
| Assert.assertEquals(EXPECTED_BYTES, table.getNumBytes()); | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.