Skip to content
Merged
Show file tree
Hide file tree
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 Sep 22, 2021
f111ff2
Merge branch 'BEAM-10652' of github.com:brucearctor/beam
brucearctor Sep 24, 2021
f6a98e1
Merge branch 'master' of github.com:apache/beam
brucearctor Oct 3, 2021
f28ced6
Merge branch 'master' of github.com:apache/beam
brucearctor Oct 4, 2021
1b91ec7
Merge branch 'master' of github.com:apache/beam
brucearctor Oct 4, 2021
c033139
[BEAM-10652] allow clustering without requiring partition
brucearctor Sep 23, 2021
2cf0dbe
newline
brucearctor Sep 23, 2021
60b440f
added needed null
brucearctor Sep 23, 2021
ab9c9cb
remove testClusteringThrowsWithoutPartitioning
brucearctor Sep 23, 2021
49378fd
update clustering
brucearctor Oct 4, 2021
c214c95
formatting
brucearctor Oct 4, 2021
b2dcf0a
now compiles
brucearctor Oct 4, 2021
40c7c66
passes spotless
brucearctor Oct 4, 2021
a944c33
update doc
brucearctor Oct 13, 2021
de76a43
focus on single test
brucearctor Oct 13, 2021
4556a16
Merge branch 'master' of github.com:apache/beam
brucearctor Oct 25, 2021
e382310
Merge branch 'BEAM-10652' of github.com:brucearctor/beam into BEAM-10652
brucearctor Jan 17, 2022
471f210
Merge branch 'master' of github.com:apache/beam into BEAM-10652
brucearctor Jan 17, 2022
393094b
Merge branch 'master' of github.com:apache/beam into BEAM-10652
brucearctor Jan 17, 2022
46defbd
spotless
brucearctor Jan 17, 2022
9aeca4d
run all ITs
brucearctor Jan 18, 2022
22e43d6
spotless
brucearctor Jan 18, 2022
7ea7837
testing with time partitioning
brucearctor Jan 19, 2022
314feab
checking
brucearctor Jan 19, 2022
12429b4
set clustering independant of partitioning
brucearctor Jan 19, 2022
adb4c5d
remove timepart from it
brucearctor Jan 19, 2022
7011de1
spotless
brucearctor Jan 19, 2022
029222b
removed test
brucearctor Jan 20, 2022
29fb91e
added TODO
brucearctor Jan 20, 2022
8c22606
removed block of unneded code/comment
brucearctor Jan 20, 2022
0d2cf18
remove override to v3 coder
brucearctor Feb 11, 2022
e587c0a
Spotless cleanup
brucearctor Feb 11, 2022
39c059f
re-add override to v3 coder
brucearctor Feb 12, 2022
81dbb74
spotless
brucearctor Feb 12, 2022
9a3f81b
adding checksum ( wrong value )
brucearctor Feb 14, 2022
e4c9154
added needed query var
brucearctor Feb 14, 2022
8a30468
use tableName as var
brucearctor Feb 14, 2022
6255ff0
DATASET NAME
brucearctor Feb 15, 2022
e2ff347
project name in query
brucearctor Feb 15, 2022
43bc3c2
update query
brucearctor Feb 15, 2022
f9f706e
change tests
brucearctor Feb 15, 2022
7c55fc2
remove unneeded imports
brucearctor Feb 15, 2022
f044c13
remove rest of forgotten
brucearctor Feb 15, 2022
88f76db
add rows
brucearctor Feb 15, 2022
66786c7
16000 bytes
brucearctor Feb 15, 2022
8a94697
bigint
brucearctor Feb 16, 2022
c86590f
streaming test
brucearctor Feb 22, 2022
093f7d0
spotless
brucearctor Feb 22, 2022
db2164c
methods
brucearctor Feb 23, 2022
e8b71c7
end stream
brucearctor Feb 23, 2022
94af35c
stream method and naming
brucearctor Feb 23, 2022
a82157c
nostream
brucearctor Feb 23, 2022
d6710a0
streaming
brucearctor Feb 23, 2022
5b08ef8
streamingoptions
brucearctor Feb 23, 2022
2b3c0cf
without streaming example
brucearctor Feb 24, 2022
bae4bd8
string column instead of date -- related to BEAM-13753
brucearctor Feb 24, 2022
27ec3d7
mor strings
brucearctor Feb 24, 2022
3ba86f4
spotless
brucearctor Feb 24, 2022
90de8f9
revert, only DEFAULT and FILE_LOADS
brucearctor Feb 25, 2022
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
Original file line number Diff line number Diff line change
Expand Up @@ -2228,9 +2228,9 @@ public Write<T> withClustering(Clustering clustering) {
/**
* Allows writing to clustered tables when {@link #to(SerializableFunction)} or {@link
* #to(DynamicDestinations)} is used. The returned {@link TableDestination} objects should
* specify the time partitioning and clustering fields per table. If writing to a single table,
* use {@link #withClustering(Clustering)} instead to pass a {@link Clustering} instance that
* specifies the static clustering fields to use.
* specify the clustering fields per table. If writing to a single table, use {@link
* #withClustering(Clustering)} instead to pass a {@link Clustering} instance that specifies the
* static clustering fields to use.
*
* <p>Setting this option enables use of {@link TableDestinationCoderV3} which encodes
* clustering information. The updated coder is compatible with non-clustered tables, so can be
Expand Down Expand Up @@ -2661,11 +2661,6 @@ public WriteResult expand(PCollection<T> input) {
"The supplied getTableFunction object can directly set TimePartitioning."
+ " There is no need to call BigQueryIO.Write.withTimePartitioning.");
}
if (getClustering() != null && getClustering().getFields() != null) {
checkArgument(
getJsonTimePartitioning() != null,
"Clustering fields can only be set when TimePartitioning is set.");
}

DynamicDestinations<T, ?> dynamicDestinations = getDynamicDestinations();
if (dynamicDestinations == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -485,10 +485,9 @@ private PendingJob startLoad(
}
if (timePartitioning != null) {
loadConfig.setTimePartitioning(timePartitioning);
// only set clustering if timePartitioning is set
if (clustering != null) {
loadConfig.setClustering(clustering);
}
}
if (clustering != null) {
loadConfig.setClustering(clustering);
}
if (kmsKey != null) {
loadConfig.setDestinationEncryptionConfiguration(
Expand Down
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());
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());
Assert.assertEquals(EXPECTED_ROWS, table.getNumRows());
Assert.assertEquals(EXPECTED_BYTES, table.getNumBytes());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -512,15 +512,6 @@ public void testClusteringStorageApi() throws Exception {
}
}

@Test(expected = IllegalArgumentException.class)
public void testClusteringThrowsWithoutPartitioning() throws Exception {
if (useStorageApi || !useStreaming) {
throw new IllegalArgumentException();
}
p.enableAbandonedNodeEnforcement(false);
testTimePartitioningClustering(Method.STREAMING_INSERTS, false, true);
}

@Test
public void testClusteringTableFunction() throws Exception {
TableRow row1 = new TableRow().set("date", "2018-01-01").set("number", "1");
Expand Down