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
Expand Up @@ -72,10 +72,8 @@ 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_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"];
BIGQUERY_WRITE = 5 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:bigquery_write:v1"];
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,18 +15,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.io.gcp.bigquery;

import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;
package org.apache.beam.sdk.io.gcp.bigquery.providers;

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;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryWriteConfiguration;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
Expand All @@ -38,7 +37,7 @@

/**
* An implementation of {@link TypedSchemaTransformProvider} for BigQuery write jobs configured
* using {@link BigQueryWriteConfiguration}.
* using {@link org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryWriteConfiguration}.
*
* <p><b>Internal only:</b> This class is actively being worked on, and it will likely change. We
* provide no backwards compatibility guarantees, and it should not be implemented outside the Beam
Expand All @@ -61,7 +60,7 @@ protected SchemaTransform from(BigQueryWriteConfiguration configuration) {

@Override
public String identifier() {
return getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_FILE_LOADS);
return "beam:schematransform:org.apache.beam:bigquery_fileloads:v1";
Copy link
Contributor

Choose a reason for hiding this comment

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

I think it's still fine to define these URNs in the proto.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I feel like we don't need to right now? Since we're using a wrapper over file loads and storage api writes

Copy link
Contributor

Choose a reason for hiding this comment

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

May be will be useful for any Python wrappers that directly use specific schema-transforms ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In that case maybe we can do this in a separate PR that targets all schematransforms?

}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
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 @@ -28,7 +27,6 @@
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 @@ -100,7 +98,7 @@ protected SchemaTransform from(BigQueryWriteConfiguration configuration) {

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

Choose a reason for hiding this comment

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

Ditto.

}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
/**
* Configuration for writing to BigQuery with SchemaTransforms. Used by {@link
* BigQueryStorageWriteApiSchemaTransformProvider} and {@link
* org.apache.beam.sdk.io.gcp.bigquery.BigQueryFileLoadsWriteSchemaTransformProvider}.
* BigQueryFileLoadsWriteSchemaTransformProvider}.
*/
@DefaultSchema(AutoValueSchema.class)
@AutoValue
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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.providers;

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

import com.google.auto.service.AutoService;
import org.apache.beam.model.pipeline.v1.ExternalTransforms;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionRowTuple;

/**
* A BigQuery Write SchemaTransformProvider that routes to either {@link
* BigQueryFileLoadsWriteSchemaTransformProvider} or {@link
* BigQueryStorageWriteApiSchemaTransformProvider}.
*
* <p>Internal only. Used by the Managed Transform layer.
*/
@Internal
@AutoService(SchemaTransformProvider.class)
public class BigQueryWriteSchemaTransformProvider
extends TypedSchemaTransformProvider<BigQueryWriteConfiguration> {
@Override
public String identifier() {
return getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_WRITE);
}

@Override
protected SchemaTransform from(BigQueryWriteConfiguration configuration) {
return new BigQueryWriteRouter(configuration);
}

static class BigQueryWriteRouter extends SchemaTransform {
private final BigQueryWriteConfiguration configuration;

BigQueryWriteRouter(BigQueryWriteConfiguration configuration) {
configuration.validate();
this.configuration = configuration;
}

@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
if (input.getSinglePCollection().isBounded().equals(PCollection.IsBounded.BOUNDED)) {
return input.apply(new BigQueryFileLoadsWriteSchemaTransformProvider().from(configuration));
} else { // UNBOUNDED
return input.apply(
new BigQueryStorageWriteApiSchemaTransformProvider().from(configuration));
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.io.gcp.bigquery;
package org.apache.beam.sdk.io.gcp.bigquery.providers;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
Expand All @@ -24,10 +24,11 @@
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryFileLoadsWriteSchemaTransformProvider.BigQueryWriteSchemaTransform;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryWriteConfiguration;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryOptions;
import org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryFileLoadsWriteSchemaTransformProvider.BigQueryWriteSchemaTransform;
import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices;
import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService;
import org.apache.beam.sdk.io.gcp.testing.FakeJobService;
Expand All @@ -46,7 +47,10 @@
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Test for {@link BigQueryFileLoadsWriteSchemaTransformProvider}. */
/**
* Test for {@link
* org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryFileLoadsWriteSchemaTransformProvider}.
*/
@RunWith(JUnit4.class)
public class BigQueryFileLoadsWriteSchemaTransformProviderTest {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ 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_STORAGE_WRITE))
.put(BIGQUERY, getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_WRITE))
.build();

/**
Expand Down Expand Up @@ -135,17 +135,14 @@ public static ManagedTransform read(String source) {
* </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(supportedIdentifiers)
.setSupportedIdentifiers(new ArrayList<>(WRITE_TRANSFORMS.values()))
.build();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,7 @@
*/
package org.apache.beam.sdk.managed;

import static org.apache.beam.sdk.managed.Managed.BIGQUERY;
import static org.apache.beam.sdk.managed.Managed.WRITE_TRANSFORMS;
import static org.apache.beam.sdk.managed.ManagedTransformConstants.MAPPINGS;
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.auto.service.AutoService;
Expand All @@ -35,7 +32,6 @@
import java.util.Map;
import java.util.ServiceLoader;
import javax.annotation.Nullable;
import org.apache.beam.model.pipeline.v1.ExternalTransforms;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.fs.MatchResult;
import org.apache.beam.sdk.schemas.AutoValueSchema;
Expand All @@ -48,7 +44,6 @@
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
import org.apache.beam.sdk.schemas.utils.YamlUtils;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionRowTuple;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
Expand Down Expand Up @@ -102,8 +97,6 @@ public static Builder builder() {
@SchemaFieldDescription("YAML string config used to build the underlying SchemaTransform.")
public abstract @Nullable String getConfig();

public abstract Builder toBuilder();

@AutoValue.Builder
public abstract static class Builder {
public abstract Builder setTransformIdentifier(String identifier);
Expand Down Expand Up @@ -141,62 +134,53 @@ private Map<String, Object> resolveUnderlyingConfig() {

return YamlUtils.yamlStringToMap(yamlTransformConfig);
}

@VisibleForTesting
ManagedConfig resolveUnderlyingTransform(PCollectionRowTuple input) {
String identifier = getTransformIdentifier();
if (identifier.equals(WRITE_TRANSFORMS.get(BIGQUERY))) {
if (input.getSinglePCollection().isBounded().equals(PCollection.IsBounded.BOUNDED)) {
identifier = getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_FILE_LOADS);
}
}
return toBuilder().setTransformIdentifier(identifier).build();
}
}

@Override
protected SchemaTransform from(ManagedConfig managedConfig) {
managedConfig.validate();
return new ManagedSchemaTransform(managedConfig, getAllProviders());
SchemaTransformProvider schemaTransformProvider =
Preconditions.checkNotNull(
getAllProviders().get(managedConfig.getTransformIdentifier()),
"Could not find a transform with the identifier "
+ "%s. This could be either due to the dependency with the "
+ "transform not being available in the classpath or due to "
+ "the specified transform not being supported.",
managedConfig.getTransformIdentifier());

return new ManagedSchemaTransform(managedConfig, schemaTransformProvider);
}

static class ManagedSchemaTransform extends SchemaTransform {
private ManagedConfig managedConfig;
private final Map<String, SchemaTransformProvider> providers;
private final ManagedConfig managedConfig;
private final Row underlyingRowConfig;
private final SchemaTransformProvider underlyingTransformProvider;

ManagedSchemaTransform(
ManagedConfig managedConfig, Map<String, SchemaTransformProvider> providers) {
this.providers = providers;
this.managedConfig = managedConfig;
}

@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
managedConfig = managedConfig.resolveUnderlyingTransform(input);

SchemaTransformProvider schemaTransformProvider =
Preconditions.checkNotNull(
providers.get(managedConfig.getTransformIdentifier()),
"Could not find a transform with the identifier "
+ "%s. This could be either due to the dependency with the "
+ "transform not being available in the classpath or due to "
+ "the specified transform not being supported.",
managedConfig.getTransformIdentifier());
Schema transformConfigSchema = schemaTransformProvider.configurationSchema();
Row transformRowConfig;
ManagedConfig managedConfig, SchemaTransformProvider underlyingTransformProvider) {
// parse config before expansion to check if it matches underlying transform's config schema
Schema transformConfigSchema = underlyingTransformProvider.configurationSchema();
Row underlyingRowConfig;
try {
transformRowConfig = getRowConfig(managedConfig, transformConfigSchema);
underlyingRowConfig = getRowConfig(managedConfig, transformConfigSchema);
} catch (Exception e) {
throw new IllegalArgumentException(
"Encountered an error when retrieving a Row configuration", e);
}

this.underlyingRowConfig = underlyingRowConfig;
this.underlyingTransformProvider = underlyingTransformProvider;
this.managedConfig = managedConfig;
}

@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
LOG.debug(
"Building transform \"{}\" with configuration: {}",
schemaTransformProvider.identifier(),
transformRowConfig);
underlyingTransformProvider.identifier(),
underlyingRowConfig);

return input.apply(schemaTransformProvider.from(transformRowConfig));
return input.apply(underlyingTransformProvider.from(underlyingRowConfig));
}

public ManagedConfig getManagedConfig() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ public class ManagedTransformConstants {
getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_READ),
BIGQUERY_READ_MAPPINGS)
.put(
getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_STORAGE_WRITE),
getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_WRITE),
BIGQUERY_WRITE_MAPPINGS)
.build();
}
Loading
Loading