Skip to content

Commit 173dd48

Browse files
authored
[yaml] Adding Spanner IO Providers for Beam YAML (#31987)
* 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
1 parent 7a88e6f commit 173dd48

File tree

9 files changed

+645
-30
lines changed

9 files changed

+645
-30
lines changed

sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/ErrorHandling.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,13 +18,14 @@
1818
package org.apache.beam.sdk.schemas.transforms.providers;
1919

2020
import com.google.auto.value.AutoValue;
21+
import java.io.Serializable;
2122
import javax.annotation.Nullable;
2223
import org.apache.beam.sdk.schemas.Schema;
2324
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
2425
import org.apache.beam.sdk.values.Row;
2526

2627
@AutoValue
27-
public abstract class ErrorHandling {
28+
public abstract class ErrorHandling implements Serializable {
2829
@SchemaFieldDescription("The name of the output PCollection containing failed writes.")
2930
public abstract String getOutput();
3031

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationUtils.java

Lines changed: 60 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,8 +26,11 @@
2626
import com.google.cloud.Timestamp;
2727
import com.google.cloud.spanner.Key;
2828
import com.google.cloud.spanner.Mutation;
29+
import com.google.cloud.spanner.Value;
2930
import java.math.BigDecimal;
31+
import java.util.HashMap;
3032
import java.util.List;
33+
import java.util.Map;
3134
import java.util.stream.StreamSupport;
3235
import org.apache.beam.sdk.schemas.Schema;
3336
import org.apache.beam.sdk.transforms.SerializableFunction;
@@ -351,4 +354,61 @@ private static void addIterableToMutationBuilder(
351354
beamIterableType.getTypeName()));
352355
}
353356
}
357+
358+
public static Row createRowFromMutation(Schema schema, Mutation mutation) {
359+
Map<String, Object> mutationHashMap = new HashMap<>();
360+
mutation
361+
.asMap()
362+
.forEach(
363+
(column, value) -> mutationHashMap.put(column, convertValueToBeamFieldType(value)));
364+
return Row.withSchema(schema).withFieldValues(mutationHashMap).build();
365+
}
366+
367+
public static Object convertValueToBeamFieldType(Value value) {
368+
switch (value.getType().getCode()) {
369+
case BOOL:
370+
return value.getBool();
371+
case BYTES:
372+
return value.getBytes();
373+
case DATE:
374+
return value.getDate();
375+
case INT64:
376+
return value.getInt64();
377+
case FLOAT64:
378+
return value.getFloat64();
379+
case NUMERIC:
380+
return value.getNumeric();
381+
case TIMESTAMP:
382+
return value.getTimestamp();
383+
case STRING:
384+
return value.getString();
385+
case JSON:
386+
return value.getJson();
387+
case ARRAY:
388+
switch (value.getType().getArrayElementType().getCode()) {
389+
case BOOL:
390+
return value.getBoolArray();
391+
case BYTES:
392+
return value.getBytesArray();
393+
case DATE:
394+
return value.getDateArray();
395+
case INT64:
396+
return value.getInt64Array();
397+
case FLOAT64:
398+
return value.getFloat64Array();
399+
case NUMERIC:
400+
return value.getNumericArray();
401+
case TIMESTAMP:
402+
return value.getTimestampArray();
403+
case STRING:
404+
return value.getStringArray();
405+
case JSON:
406+
return value.getJsonArray();
407+
default:
408+
return value.toString();
409+
}
410+
default:
411+
return value.toString();
412+
}
413+
}
354414
}
Lines changed: 235 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,235 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.beam.sdk.io.gcp.spanner;
19+
20+
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
21+
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
22+
23+
import com.google.auto.service.AutoService;
24+
import com.google.auto.value.AutoValue;
25+
import com.google.cloud.spanner.Struct;
26+
import java.io.Serializable;
27+
import java.util.Collections;
28+
import java.util.List;
29+
import javax.annotation.Nullable;
30+
import org.apache.beam.sdk.schemas.AutoValueSchema;
31+
import org.apache.beam.sdk.schemas.Schema;
32+
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
33+
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
34+
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
35+
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
36+
import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
37+
import org.apache.beam.sdk.transforms.MapElements;
38+
import org.apache.beam.sdk.values.PCollection;
39+
import org.apache.beam.sdk.values.PCollectionRowTuple;
40+
import org.apache.beam.sdk.values.Row;
41+
import org.apache.beam.sdk.values.TypeDescriptor;
42+
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
43+
import org.checkerframework.checker.initialization.qual.Initialized;
44+
import org.checkerframework.checker.nullness.qual.NonNull;
45+
import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
46+
47+
@SuppressWarnings({
48+
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
49+
})
50+
/**
51+
* A provider for reading from Cloud Spanner using a Schema Transform Provider.
52+
*
53+
* <p>This provider enables reading from Cloud Spanner using a specified SQL query or by directly
54+
* accessing a table and its columns. It supports configuration through the {@link
55+
* SpannerReadSchemaTransformConfiguration} class, allowing users to specify project, instance,
56+
* database, table, query, and columns.
57+
*
58+
* <p>The transformation leverages the {@link SpannerIO} to perform the read operation and maps the
59+
* results to Beam rows, preserving the schema.
60+
*
61+
* <p>Example usage in a YAML pipeline using query:
62+
*
63+
* <pre>{@code
64+
* pipeline:
65+
* transforms:
66+
* - type: ReadFromSpanner
67+
* name: ReadShipments
68+
* # Columns: shipment_id, customer_id, shipment_date, shipment_cost, customer_name, customer_email
69+
* config:
70+
* project_id: 'apache-beam-testing'
71+
* instance_id: 'shipment-test'
72+
* database_id: 'shipment'
73+
* query: 'SELECT * FROM shipments'
74+
* }</pre>
75+
*
76+
* <p>Example usage in a YAML pipeline using a table and columns:
77+
*
78+
* <pre>{@code
79+
* pipeline:
80+
* transforms:
81+
* - type: ReadFromSpanner
82+
* name: ReadShipments
83+
* # Columns: shipment_id, customer_id, shipment_date, shipment_cost, customer_name, customer_email
84+
* config:
85+
* project_id: 'apache-beam-testing'
86+
* instance_id: 'shipment-test'
87+
* database_id: 'shipment'
88+
* table: 'shipments'
89+
* columns: ['customer_id', 'customer_name']
90+
* }</pre>
91+
*/
92+
@AutoService(SchemaTransformProvider.class)
93+
public class SpannerReadSchemaTransformProvider
94+
extends TypedSchemaTransformProvider<
95+
SpannerReadSchemaTransformProvider.SpannerReadSchemaTransformConfiguration> {
96+
97+
static class SpannerSchemaTransformRead extends SchemaTransform implements Serializable {
98+
private final SpannerReadSchemaTransformConfiguration configuration;
99+
100+
SpannerSchemaTransformRead(SpannerReadSchemaTransformConfiguration configuration) {
101+
configuration.validate();
102+
this.configuration = configuration;
103+
}
104+
105+
@Override
106+
public PCollectionRowTuple expand(PCollectionRowTuple input) {
107+
checkNotNull(input, "Input to SpannerReadSchemaTransform cannot be null.");
108+
SpannerIO.Read read =
109+
SpannerIO.readWithSchema()
110+
.withProjectId(configuration.getProjectId())
111+
.withInstanceId(configuration.getInstanceId())
112+
.withDatabaseId(configuration.getDatabaseId());
113+
114+
if (!Strings.isNullOrEmpty(configuration.getQuery())) {
115+
read = read.withQuery(configuration.getQuery());
116+
} else {
117+
read = read.withTable(configuration.getTableId()).withColumns(configuration.getColumns());
118+
}
119+
PCollection<Struct> spannerRows = input.getPipeline().apply(read);
120+
Schema schema = spannerRows.getSchema();
121+
PCollection<Row> rows =
122+
spannerRows.apply(
123+
MapElements.into(TypeDescriptor.of(Row.class))
124+
.via((Struct struct) -> StructUtils.structToBeamRow(struct, schema)));
125+
126+
return PCollectionRowTuple.of("output", rows.setRowSchema(schema));
127+
}
128+
}
129+
130+
@Override
131+
public @UnknownKeyFor @NonNull @Initialized String identifier() {
132+
return "beam:schematransform:org.apache.beam:spanner_read:v1";
133+
}
134+
135+
@Override
136+
public @UnknownKeyFor @NonNull @Initialized List<@UnknownKeyFor @NonNull @Initialized String>
137+
inputCollectionNames() {
138+
return Collections.emptyList();
139+
}
140+
141+
@Override
142+
public @UnknownKeyFor @NonNull @Initialized List<@UnknownKeyFor @NonNull @Initialized String>
143+
outputCollectionNames() {
144+
return Collections.singletonList("output");
145+
}
146+
147+
@DefaultSchema(AutoValueSchema.class)
148+
@AutoValue
149+
public abstract static class SpannerReadSchemaTransformConfiguration implements Serializable {
150+
@AutoValue.Builder
151+
@Nullable
152+
public abstract static class Builder {
153+
public abstract Builder setProjectId(String projectId);
154+
155+
public abstract Builder setInstanceId(String instanceId);
156+
157+
public abstract Builder setDatabaseId(String databaseId);
158+
159+
public abstract Builder setTableId(String tableId);
160+
161+
public abstract Builder setQuery(String query);
162+
163+
public abstract Builder setColumns(List<String> columns);
164+
165+
public abstract SpannerReadSchemaTransformConfiguration build();
166+
}
167+
168+
public void validate() {
169+
String invalidConfigMessage = "Invalid Cloud Spanner Read configuration: ";
170+
checkArgument(
171+
!Strings.isNullOrEmpty(this.getInstanceId()),
172+
invalidConfigMessage + "Instance ID must be specified.");
173+
checkArgument(
174+
!Strings.isNullOrEmpty(this.getDatabaseId()),
175+
invalidConfigMessage + "Database ID must be specified.");
176+
if (Strings.isNullOrEmpty(this.getQuery())) {
177+
checkArgument(
178+
!Strings.isNullOrEmpty(this.getTableId()),
179+
invalidConfigMessage + "Table name must be specified for table read.");
180+
checkArgument(
181+
this.getColumns() != null && !this.getColumns().isEmpty(),
182+
invalidConfigMessage + "Columns must be specified for table read.");
183+
} else {
184+
checkArgument(
185+
!Strings.isNullOrEmpty(this.getQuery()),
186+
invalidConfigMessage + "Query must be specified for query read.");
187+
checkArgument(
188+
Strings.isNullOrEmpty(this.getTableId()),
189+
invalidConfigMessage + "Table name should not be specified when using a query.");
190+
checkArgument(
191+
this.getColumns() == null || this.getColumns().isEmpty(),
192+
invalidConfigMessage + "Columns should not be specified when using a query.");
193+
}
194+
}
195+
196+
public static Builder builder() {
197+
return new AutoValue_SpannerReadSchemaTransformProvider_SpannerReadSchemaTransformConfiguration
198+
.Builder();
199+
}
200+
201+
@SchemaFieldDescription("Specifies the GCP project ID.")
202+
@Nullable
203+
public abstract String getProjectId();
204+
205+
@SchemaFieldDescription("Specifies the Cloud Spanner instance.")
206+
public abstract String getInstanceId();
207+
208+
@SchemaFieldDescription("Specifies the Cloud Spanner database.")
209+
public abstract String getDatabaseId();
210+
211+
@SchemaFieldDescription("Specifies the Cloud Spanner table.")
212+
@Nullable
213+
public abstract String getTableId();
214+
215+
@SchemaFieldDescription("Specifies the SQL query to execute.")
216+
@Nullable
217+
public abstract String getQuery();
218+
219+
@SchemaFieldDescription("Specifies the columns to read from the table.")
220+
@Nullable
221+
public abstract List<String> getColumns();
222+
}
223+
224+
@Override
225+
protected @UnknownKeyFor @NonNull @Initialized Class<SpannerReadSchemaTransformConfiguration>
226+
configurationClass() {
227+
return SpannerReadSchemaTransformConfiguration.class;
228+
}
229+
230+
@Override
231+
protected @UnknownKeyFor @NonNull @Initialized SchemaTransform from(
232+
SpannerReadSchemaTransformConfiguration configuration) {
233+
return new SpannerSchemaTransformRead(configuration);
234+
}
235+
}

0 commit comments

Comments
 (0)