Skip to content
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

Managed BigQueryIO #31486

Merged
merged 30 commits into from
Nov 12, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
30 commits
Select commit Hold shift + click to select a range
ce14b96
managed bigqueryio
ahmedabu98 Jun 3, 2024
550c1b4
spotless
ahmedabu98 Jun 4, 2024
c94de3c
move managed dependency to test only
ahmedabu98 Jun 4, 2024
912dc08
Merge branch 'master' of https://github.com/ahmedabu98/beam into mana…
ahmedabu98 Jun 5, 2024
f436e62
cleanup after merging snake_case PR
ahmedabu98 Jun 5, 2024
fe60904
choose write method based on boundedness and pipeline options
ahmedabu98 Jul 9, 2024
7d405cf
Merge branch 'master' of https://github.com/ahmedabu98/beam into mana…
ahmedabu98 Jul 9, 2024
d45159f
rename bigquery write config class
ahmedabu98 Jul 9, 2024
989ad0f
spotless
ahmedabu98 Jul 9, 2024
b9b49e7
change read output tag to 'output'
ahmedabu98 Jul 9, 2024
a119bbc
spotless
ahmedabu98 Jul 9, 2024
74bc178
revert logic that depends on DataflowServiceOptions. switching BQ met…
ahmedabu98 Jul 16, 2024
528b504
spotless
ahmedabu98 Jul 16, 2024
dcc398a
fix typo
ahmedabu98 Jul 29, 2024
36edc38
separate BQ write config to a new class
ahmedabu98 Aug 6, 2024
f9be86c
fix doc
ahmedabu98 Aug 6, 2024
bd1e534
Merge branch 'master' of https://github.com/ahmedabu98/beam into mana…
ahmedabu98 Oct 25, 2024
a26765e
resolve after syncing to HEAD
ahmedabu98 Oct 25, 2024
725f7bd
spotless
ahmedabu98 Oct 26, 2024
2631104
fork on batch/streaming
ahmedabu98 Nov 5, 2024
770cf50
cleanup
ahmedabu98 Nov 5, 2024
0a70466
spotless
ahmedabu98 Nov 5, 2024
01a01f7
move forking logic to BQ schematransform side
ahmedabu98 Nov 6, 2024
697c0b8
add file loads translation and tests; add test checks that the correc…
ahmedabu98 Nov 7, 2024
105474b
set top-level wrapper to be the underlying managed BQ transform urn; …
ahmedabu98 Nov 8, 2024
d6b9e69
move unit tests to respectvie schematransform test classes
ahmedabu98 Nov 8, 2024
c0767d7
Merge branch 'master' of https://github.com/ahmedabu98/beam into mana…
ahmedabu98 Nov 8, 2024
a600f62
Merge branch 'master' of https://github.com/ahmedabu98/beam into mana…
ahmedabu98 Nov 8, 2024
ad4dcd9
expose to Python SDK as well
ahmedabu98 Nov 11, 2024
6f325ce
Merge branch 'master' of https://github.com/ahmedabu98/beam into mana…
ahmedabu98 Nov 11, 2024
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
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 2
"modification": 1
}
Original file line number Diff line number Diff line change
Expand Up @@ -72,8 +72,10 @@ message ManagedTransforms {
"beam:schematransform:org.apache.beam:kafka_write:v1"];
BIGQUERY_READ = 4 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:bigquery_storage_read:v1"];
BIGQUERY_WRITE = 5 [(org.apache.beam.model.pipeline.v1.beam_urn) =
BIGQUERY_STORAGE_WRITE = 5 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:bigquery_storage_write:v2"];
BIGQUERY_FILE_LOADS = 6 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:bigquery_fileloads:v1"];
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,12 @@
*/
package org.apache.beam.sdk.io.gcp.bigquery;

import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;

import com.google.auto.service.AutoService;
import java.util.Collections;
import java.util.List;
import org.apache.beam.model.pipeline.v1.ExternalTransforms;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
Expand Down Expand Up @@ -49,8 +52,6 @@
public class BigQueryFileLoadsWriteSchemaTransformProvider
extends TypedSchemaTransformProvider<BigQueryWriteConfiguration> {

private static final String IDENTIFIER =
"beam:schematransform:org.apache.beam:bigquery_fileloads:v1";
static final String INPUT_TAG = "input";

@Override
Expand All @@ -60,7 +61,7 @@ protected SchemaTransform from(BigQueryWriteConfiguration configuration) {

@Override
public String identifier() {
return IDENTIFIER;
return getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_FILE_LOADS);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
*/
package org.apache.beam.sdk.io.gcp.bigquery.providers;

import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;

Expand All @@ -26,6 +27,7 @@
import java.util.Collections;
import java.util.List;
import javax.annotation.Nullable;
import org.apache.beam.model.pipeline.v1.ExternalTransforms;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead;
Expand Down Expand Up @@ -78,7 +80,7 @@ protected SchemaTransform from(BigQueryDirectReadSchemaTransformConfiguration co

@Override
public String identifier() {
return "beam:schematransform:org.apache.beam:bigquery_storage_read:v1"; // getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_READ);
return getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_READ);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.beam.sdk.io.gcp.bigquery.providers;

import static org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryWriteConfiguration.DYNAMIC_DESTINATIONS;
import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;

import com.google.api.services.bigquery.model.TableConstraints;
Expand All @@ -27,6 +28,7 @@
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import org.apache.beam.model.pipeline.v1.ExternalTransforms;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method;
Expand Down Expand Up @@ -98,7 +100,7 @@ protected SchemaTransform from(BigQueryWriteConfiguration configuration) {

@Override
public String identifier() {
return "beam:schematransform:org.apache.beam:bigquery_storage_write:v2"; // getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_WRITE);
return getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_STORAGE_WRITE);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,12 @@
import java.util.stream.LongStream;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
import org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryDirectReadSchemaTransformProvider;
import org.apache.beam.sdk.io.gcp.testing.BigqueryClient;
import org.apache.beam.sdk.managed.Managed;
import org.apache.beam.sdk.schemas.Schema;
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.MapElements;
import org.apache.beam.sdk.transforms.PeriodicImpulse;
Expand All @@ -39,15 +39,22 @@
import org.apache.beam.sdk.values.TypeDescriptors;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.joda.time.Duration;
import org.joda.time.Instant;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** This class tests the execution of {@link Managed} BigQueryIO. */
@RunWith(JUnit4.class)
public class BigQueryManagedIT {
@Rule public TestName testName = new TestName();
@Rule public transient TestPipeline writePipeline = TestPipeline.create();
@Rule public transient TestPipeline readPipeline = TestPipeline.create();

private static final Schema SCHEMA =
Schema.of(
Schema.Field.of("str", Schema.FieldType.STRING),
Expand Down Expand Up @@ -79,34 +86,58 @@ public static void setUpTestEnvironment() throws IOException, InterruptedExcepti
public static void cleanup() {
BQ_CLIENT.deleteDataset(PROJECT, BIG_QUERY_DATASET_ID);
}
@Test
public void testBatchFileLoadsWriteRead() {
String table =
String.format("%s:%s.%s", PROJECT, BIG_QUERY_DATASET_ID, testName.getMethodName());
Map<String, Object> config = ImmutableMap.of("table", table);

// file loads requires a GCS temp location
String tempLocation = writePipeline.getOptions().as(TestPipelineOptions.class).getTempRoot();
writePipeline.getOptions().setTempLocation(tempLocation);

// batch write
PCollectionRowTuple.of("input", getInput(writePipeline, false))
.apply(Managed.write(Managed.BIGQUERY).withConfig(config));
writePipeline.run().waitUntilFinish();

// read and validate
PCollection<Row> outputRows =
readPipeline
.apply(Managed.read(Managed.BIGQUERY).withConfig(config))
.getSinglePCollection();
PAssert.that(outputRows).containsInAnyOrder(ROWS);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also confirm that we end up using the correct sink here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Initially did not know how we would perform such a check. Will think about it again and give it a try

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added tests that verifies by looking at the pipeline proto, PTAL!


readPipeline.run().waitUntilFinish();
}

@Test
public void testStreamingStorageWriteRead() {
String table = String.format("%s:%s.managed_storage_write_read", PROJECT, BIG_QUERY_DATASET_ID);

Map<String, Object> writeConfig =
ImmutableMap.<String, Object>builder().put("table", table).build();
Pipeline p = Pipeline.create();
PCollectionRowTuple.of("input", getInput(p, true))
.apply(Managed.write(Managed.BIGQUERY).withConfig(writeConfig));
p.run().waitUntilFinish();

Map<String, Object> readConfig =
ImmutableMap.<String, Object>builder().put("table", table).build();
Pipeline q = Pipeline.create();
String table =
String.format("%s:%s.%s", PROJECT, BIG_QUERY_DATASET_ID, testName.getMethodName());
Map<String, Object> config = ImmutableMap.of("table", table);

// streaming write
PCollectionRowTuple.of("input", getInput(writePipeline, true))
.apply(Managed.write(Managed.BIGQUERY).withConfig(config));
writePipeline.run().waitUntilFinish();

// read and validate
PCollection<Row> outputRows =
PCollectionRowTuple.empty(p)
.apply(Managed.read(Managed.BIGQUERY).withConfig(readConfig))
.get(BigQueryDirectReadSchemaTransformProvider.OUTPUT_TAG);
readPipeline
.apply(Managed.read(Managed.BIGQUERY).withConfig(config))
.getSinglePCollection();
PAssert.that(outputRows).containsInAnyOrder(ROWS);
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
q.run().waitUntilFinish();

readPipeline.run().waitUntilFinish();
}

public PCollection<Row> getInput(Pipeline p, boolean isStreaming) {
if (isStreaming) {
return p.apply(
PeriodicImpulse.create()
.stopAfter(Duration.millis(20))
.startAt(new Instant(0))
.stopAt(new Instant(19))
.withInterval(Duration.millis(1)))
.apply(
MapElements.into(TypeDescriptors.rows())
Expand Down
61 changes: 61 additions & 0 deletions sdks/java/managed/expansion-service/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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.
*/

apply plugin: 'org.apache.beam.module'
apply plugin: 'application'
mainClassName = "org.apache.beam.sdk.expansion.service.ExpansionService"

applyJavaNature(
automaticModuleName: 'org.apache.beam.sdk.managed.expansion.service',
exportJavadoc: false,
validateShadowJar: false,
shadowClosure: {},
)

// TODO(https://github.com/apache/beam/pull/32486/) Use library.java.kafka_clients once >=3.1.0 is set as default
configurations.runtimeClasspath {
// Pin kafka-clients version due to <3.1.0 missing auth callback classes
resolutionStrategy.force 'org.apache.kafka:kafka-clients:3.1.2'
}

shadowJar {
mergeServiceFiles()
outputs.upToDateWhen { false }
}

description = "Apache Beam :: SDKs :: Java :: Managed :: Expansion Service"
ext.summary = "Expansion service for Managed Transforms"

dependencies {
runtimeOnly project(":sdks:java:expansion-service")

// **** IcebergIO and dependencies ****
runtimeOnly project(":sdks:java:io:iceberg")
// Needed when writing to GCS
runtimeOnly library.java.bigdataoss_gcs_connector
runtimeOnly library.java.hadoop_client
// For HiveCatalog
runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:1.4.2")
runtimeOnly project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow")

// **** KafkaIO and dependencies ****
runtimeOnly project(":sdks:java:io:kafka")
runtimeOnly library.java.kafka_clients

runtimeOnly library.java.slf4j_jdk14
}
19 changes: 19 additions & 0 deletions sdks/java/managed/expansion-service/container/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
plugins {
id 'java'
}

group = 'org.apache.beam.sdk.managed.expansion.service.container'
version = '2.61.0-SNAPSHOT'

repositories {
mavenCentral()
}

dependencies {
testImplementation platform('org.junit:junit-bom:5.9.1')
testImplementation 'org.junit.jupiter:junit-jupiter'
}

test {
useJUnitPlatform()
}
Original file line number Diff line number Diff line change
Expand Up @@ -99,15 +99,17 @@ public class Managed {
ImmutableMap.<String, String>builder()
.put(ICEBERG, getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_WRITE))
.put(KAFKA, getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_WRITE))
.put(BIGQUERY, getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_WRITE))
.put(BIGQUERY, getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_STORAGE_WRITE))
ahmedabu98 marked this conversation as resolved.
Show resolved Hide resolved
.build();

/**
* Instantiates a {@link Managed.ManagedTransform} transform for the specified source. The
* supported managed sources are:
*
* <ul>
* <li>{@link Managed#ICEBERG} : Read from Apache Iceberg
* <li>{@link Managed#ICEBERG} : Read from Apache Iceberg tables
* <li>{@link Managed#KAFKA} : Read from Apache Kafka topics
* <li>{@link Managed#BIGQUERY} : Read from GCP BigQuery tables
* </ul>
*/
public static ManagedTransform read(String source) {
Expand All @@ -127,18 +129,23 @@ public static ManagedTransform read(String source) {
* managed sinks are:
*
* <ul>
* <li>{@link Managed#ICEBERG} : Write to Apache Iceberg
* <li>{@link Managed#ICEBERG} : Write to Apache Iceberg tables
* <li>{@link Managed#KAFKA} : Write to Apache Kafka topics
* <li>{@link Managed#BIGQUERY} : Write to GCP BigQuery tables
chamikaramj marked this conversation as resolved.
Show resolved Hide resolved
* </ul>
*/
public static ManagedTransform write(String sink) {
List<String> supportedIdentifiers = new ArrayList<>(WRITE_TRANSFORMS.values());
supportedIdentifiers.add(getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_FILE_LOADS));

return new AutoValue_Managed_ManagedTransform.Builder()
.setIdentifier(
Preconditions.checkNotNull(
WRITE_TRANSFORMS.get(sink.toLowerCase()),
"An unsupported sink was specified: '%s'. Please specify one of the following sinks: %s",
sink,
WRITE_TRANSFORMS.keySet()))
.setSupportedIdentifiers(new ArrayList<>(WRITE_TRANSFORMS.values()))
.setSupportedIdentifiers(supportedIdentifiers)
.build();
}

Expand Down
Loading
Loading