Skip to content

Commit

Permalink
[yaml] Adding Spanner IO Providers for Beam YAML (apache#31987)
Browse files Browse the repository at this point in the history
* Add Spanner IO providers to YAML SDK

* add handling logic for more datatypes

* delete examples

* minor changes

* minor change

* add integration test

* add docs

* minor change

* minor changes

1. Removed serialiazability from ErrorHandling.java
2. Removed double map definitions from MutationUtils.java
3. Added checkNotNull in spanner write provider
4. Modified some variables in spanner wrapper
5. Change instance id in integration tests

* Update spanner_wrapper.py

import retry

* minor changes

1. replace checknotnull with checkargument in spanner read provider
2. use the correct table  name (tmp_table) in integration test

* minor changes

1. Added serializable to error handling
2. Corrected validation methods in spanner read
3. Added retry import and removed default project name in spanner wrapper
4. Corrected instance and database names in spanner integration test
5. Corrected table name in query

* formatting

* Update SpannerWriteSchemaTransformProvider.java

* correct lint failures

* correct lint failures

* correct lint failures

* Update SpannerReadSchemaTransformProvider.java

* correct lint failures

* Update SpannerWriteSchemaTransformProvider.java

* spanner version update
  • Loading branch information
reeba212 committed Dec 4, 2024
1 parent d32c5c5 commit 9ec72b0
Show file tree
Hide file tree
Showing 9 changed files with 645 additions and 30 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,14 @@
package org.apache.beam.sdk.schemas.transforms.providers;

import com.google.auto.value.AutoValue;
import java.io.Serializable;
import javax.annotation.Nullable;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
import org.apache.beam.sdk.values.Row;

@AutoValue
public abstract class ErrorHandling {
public abstract class ErrorHandling implements Serializable {
@SchemaFieldDescription("The name of the output PCollection containing failed writes.")
public abstract String getOutput();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,11 @@
import com.google.cloud.Timestamp;
import com.google.cloud.spanner.Key;
import com.google.cloud.spanner.Mutation;
import com.google.cloud.spanner.Value;
import java.math.BigDecimal;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.StreamSupport;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.transforms.SerializableFunction;
Expand Down Expand Up @@ -351,4 +354,61 @@ private static void addIterableToMutationBuilder(
beamIterableType.getTypeName()));
}
}

public static Row createRowFromMutation(Schema schema, Mutation mutation) {
Map<String, Object> mutationHashMap = new HashMap<>();
mutation
.asMap()
.forEach(
(column, value) -> mutationHashMap.put(column, convertValueToBeamFieldType(value)));
return Row.withSchema(schema).withFieldValues(mutationHashMap).build();
}

public static Object convertValueToBeamFieldType(Value value) {
switch (value.getType().getCode()) {
case BOOL:
return value.getBool();
case BYTES:
return value.getBytes();
case DATE:
return value.getDate();
case INT64:
return value.getInt64();
case FLOAT64:
return value.getFloat64();
case NUMERIC:
return value.getNumeric();
case TIMESTAMP:
return value.getTimestamp();
case STRING:
return value.getString();
case JSON:
return value.getJson();
case ARRAY:
switch (value.getType().getArrayElementType().getCode()) {
case BOOL:
return value.getBoolArray();
case BYTES:
return value.getBytesArray();
case DATE:
return value.getDateArray();
case INT64:
return value.getInt64Array();
case FLOAT64:
return value.getFloat64Array();
case NUMERIC:
return value.getNumericArray();
case TIMESTAMP:
return value.getTimestampArray();
case STRING:
return value.getStringArray();
case JSON:
return value.getJsonArray();
default:
return value.toString();
}
default:
return value.toString();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,235 @@
/*
* 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.spanner;

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;

import com.google.auto.service.AutoService;
import com.google.auto.value.AutoValue;
import com.google.cloud.spanner.Struct;
import java.io.Serializable;
import java.util.Collections;
import java.util.List;
import javax.annotation.Nullable;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
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.transforms.MapElements;
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.sdk.values.TypeDescriptor;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
import org.checkerframework.checker.initialization.qual.Initialized;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.checkerframework.checker.nullness.qual.UnknownKeyFor;

@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
/**
* A provider for reading from Cloud Spanner using a Schema Transform Provider.
*
* <p>This provider enables reading from Cloud Spanner using a specified SQL query or by directly
* accessing a table and its columns. It supports configuration through the {@link
* SpannerReadSchemaTransformConfiguration} class, allowing users to specify project, instance,
* database, table, query, and columns.
*
* <p>The transformation leverages the {@link SpannerIO} to perform the read operation and maps the
* results to Beam rows, preserving the schema.
*
* <p>Example usage in a YAML pipeline using query:
*
* <pre>{@code
* pipeline:
* transforms:
* - type: ReadFromSpanner
* name: ReadShipments
* # Columns: shipment_id, customer_id, shipment_date, shipment_cost, customer_name, customer_email
* config:
* project_id: 'apache-beam-testing'
* instance_id: 'shipment-test'
* database_id: 'shipment'
* query: 'SELECT * FROM shipments'
* }</pre>
*
* <p>Example usage in a YAML pipeline using a table and columns:
*
* <pre>{@code
* pipeline:
* transforms:
* - type: ReadFromSpanner
* name: ReadShipments
* # Columns: shipment_id, customer_id, shipment_date, shipment_cost, customer_name, customer_email
* config:
* project_id: 'apache-beam-testing'
* instance_id: 'shipment-test'
* database_id: 'shipment'
* table: 'shipments'
* columns: ['customer_id', 'customer_name']
* }</pre>
*/
@AutoService(SchemaTransformProvider.class)
public class SpannerReadSchemaTransformProvider
extends TypedSchemaTransformProvider<
SpannerReadSchemaTransformProvider.SpannerReadSchemaTransformConfiguration> {

static class SpannerSchemaTransformRead extends SchemaTransform implements Serializable {
private final SpannerReadSchemaTransformConfiguration configuration;

SpannerSchemaTransformRead(SpannerReadSchemaTransformConfiguration configuration) {
configuration.validate();
this.configuration = configuration;
}

@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
checkNotNull(input, "Input to SpannerReadSchemaTransform cannot be null.");
SpannerIO.Read read =
SpannerIO.readWithSchema()
.withProjectId(configuration.getProjectId())
.withInstanceId(configuration.getInstanceId())
.withDatabaseId(configuration.getDatabaseId());

if (!Strings.isNullOrEmpty(configuration.getQuery())) {
read = read.withQuery(configuration.getQuery());
} else {
read = read.withTable(configuration.getTableId()).withColumns(configuration.getColumns());
}
PCollection<Struct> spannerRows = input.getPipeline().apply(read);
Schema schema = spannerRows.getSchema();
PCollection<Row> rows =
spannerRows.apply(
MapElements.into(TypeDescriptor.of(Row.class))
.via((Struct struct) -> StructUtils.structToBeamRow(struct, schema)));

return PCollectionRowTuple.of("output", rows.setRowSchema(schema));
}
}

@Override
public @UnknownKeyFor @NonNull @Initialized String identifier() {
return "beam:schematransform:org.apache.beam:spanner_read:v1";
}

@Override
public @UnknownKeyFor @NonNull @Initialized List<@UnknownKeyFor @NonNull @Initialized String>
inputCollectionNames() {
return Collections.emptyList();
}

@Override
public @UnknownKeyFor @NonNull @Initialized List<@UnknownKeyFor @NonNull @Initialized String>
outputCollectionNames() {
return Collections.singletonList("output");
}

@DefaultSchema(AutoValueSchema.class)
@AutoValue
public abstract static class SpannerReadSchemaTransformConfiguration implements Serializable {
@AutoValue.Builder
@Nullable
public abstract static class Builder {
public abstract Builder setProjectId(String projectId);

public abstract Builder setInstanceId(String instanceId);

public abstract Builder setDatabaseId(String databaseId);

public abstract Builder setTableId(String tableId);

public abstract Builder setQuery(String query);

public abstract Builder setColumns(List<String> columns);

public abstract SpannerReadSchemaTransformConfiguration build();
}

public void validate() {
String invalidConfigMessage = "Invalid Cloud Spanner Read configuration: ";
checkArgument(
!Strings.isNullOrEmpty(this.getInstanceId()),
invalidConfigMessage + "Instance ID must be specified.");
checkArgument(
!Strings.isNullOrEmpty(this.getDatabaseId()),
invalidConfigMessage + "Database ID must be specified.");
if (Strings.isNullOrEmpty(this.getQuery())) {
checkArgument(
!Strings.isNullOrEmpty(this.getTableId()),
invalidConfigMessage + "Table name must be specified for table read.");
checkArgument(
this.getColumns() != null && !this.getColumns().isEmpty(),
invalidConfigMessage + "Columns must be specified for table read.");
} else {
checkArgument(
!Strings.isNullOrEmpty(this.getQuery()),
invalidConfigMessage + "Query must be specified for query read.");
checkArgument(
Strings.isNullOrEmpty(this.getTableId()),
invalidConfigMessage + "Table name should not be specified when using a query.");
checkArgument(
this.getColumns() == null || this.getColumns().isEmpty(),
invalidConfigMessage + "Columns should not be specified when using a query.");
}
}

public static Builder builder() {
return new AutoValue_SpannerReadSchemaTransformProvider_SpannerReadSchemaTransformConfiguration
.Builder();
}

@SchemaFieldDescription("Specifies the GCP project ID.")
@Nullable
public abstract String getProjectId();

@SchemaFieldDescription("Specifies the Cloud Spanner instance.")
public abstract String getInstanceId();

@SchemaFieldDescription("Specifies the Cloud Spanner database.")
public abstract String getDatabaseId();

@SchemaFieldDescription("Specifies the Cloud Spanner table.")
@Nullable
public abstract String getTableId();

@SchemaFieldDescription("Specifies the SQL query to execute.")
@Nullable
public abstract String getQuery();

@SchemaFieldDescription("Specifies the columns to read from the table.")
@Nullable
public abstract List<String> getColumns();
}

@Override
protected @UnknownKeyFor @NonNull @Initialized Class<SpannerReadSchemaTransformConfiguration>
configurationClass() {
return SpannerReadSchemaTransformConfiguration.class;
}

@Override
protected @UnknownKeyFor @NonNull @Initialized SchemaTransform from(
SpannerReadSchemaTransformConfiguration configuration) {
return new SpannerSchemaTransformRead(configuration);
}
}
Loading

0 comments on commit 9ec72b0

Please sign in to comment.