-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Add schema to SpannerIO read #32008
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
Add schema to SpannerIO read #32008
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,56 @@ | ||
| /* | ||
| * 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.sdk.io.gcp.spanner.StructUtils.structTypeToBeamRowSchema; | ||
|
|
||
| import com.google.cloud.spanner.ReadContext; | ||
| import com.google.cloud.spanner.ResultSet; | ||
| import com.google.cloud.spanner.Statement; | ||
| import org.apache.beam.sdk.schemas.Schema; | ||
|
|
||
| class SpannerQuerySourceDef implements SpannerSourceDef { | ||
|
|
||
| private final SpannerConfig config; | ||
| private final Statement query; | ||
|
|
||
| static SpannerQuerySourceDef create(SpannerConfig config, Statement query) { | ||
| return new SpannerQuerySourceDef(config, query); | ||
| } | ||
|
|
||
| private SpannerQuerySourceDef(SpannerConfig config, Statement query) { | ||
| this.config = config; | ||
| this.query = query; | ||
| } | ||
|
|
||
| /** {@inheritDoc} */ | ||
| @Override | ||
| public Schema getBeamSchema() { | ||
| Schema beamSchema; | ||
| try (SpannerAccessor spannerAccessor = SpannerAccessor.getOrCreate(config)) { | ||
| try (ReadContext readContext = spannerAccessor.getDatabaseClient().singleUse()) { | ||
| ResultSet result = readContext.analyzeQuery(query, ReadContext.QueryAnalyzeMode.PLAN); | ||
| result.next(); | ||
| beamSchema = structTypeToBeamRowSchema(result.getMetadata().getRowType(), true); | ||
| } | ||
| } catch (Exception e) { | ||
| throw new SpannerSchemaRetrievalException("Exception while trying to retrieve schema", e); | ||
| } | ||
| return beamSchema; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,29 @@ | ||
| /* | ||
| * 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; | ||
|
|
||
| /** Exception to signal that Spanner schema retrieval failed. */ | ||
| public class SpannerSchemaRetrievalException extends RuntimeException { | ||
| SpannerSchemaRetrievalException(String message, Throwable cause) { | ||
| super(message, cause); | ||
| } | ||
|
|
||
| SpannerSchemaRetrievalException(String message) { | ||
| super(message); | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,36 @@ | ||
| /* | ||
| * 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 java.io.Serializable; | ||
| import org.apache.beam.sdk.schemas.Schema; | ||
|
|
||
| /** | ||
| * Represents a source used for {@link SpannerIO#read()}. Currently, this could be either a table or | ||
| * a query. Direct read sources are not yet supported. | ||
| */ | ||
| interface SpannerSourceDef extends Serializable { | ||
|
|
||
| /** | ||
| * Extract the Beam {@link Schema} corresponding to this source. | ||
| * | ||
| * @return Beam schema of the source | ||
| * @throws SpannerSchemaRetrievalException if schema retrieval fails | ||
| */ | ||
| Schema getBeamSchema(); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,63 @@ | ||
| /* | ||
| * 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.sdk.io.gcp.spanner.StructUtils.structTypeToBeamRowSchema; | ||
|
|
||
| import com.google.cloud.spanner.KeySet; | ||
| import com.google.cloud.spanner.Options; | ||
| import com.google.cloud.spanner.ReadContext; | ||
| import com.google.cloud.spanner.ResultSet; | ||
| import org.apache.beam.sdk.schemas.Schema; | ||
|
|
||
| class SpannerTableSourceDef implements SpannerSourceDef { | ||
|
|
||
| private final SpannerConfig config; | ||
| private final String table; | ||
| private final Iterable<String> columns; | ||
|
|
||
| static SpannerTableSourceDef create( | ||
| SpannerConfig config, String table, Iterable<String> columns) { | ||
| return new SpannerTableSourceDef(config, table, columns); | ||
| } | ||
|
|
||
| private SpannerTableSourceDef(SpannerConfig config, String table, Iterable<String> columns) { | ||
| this.table = table; | ||
| this.config = config; | ||
| this.columns = columns; | ||
| } | ||
|
|
||
| /** {@inheritDoc} */ | ||
| @Override | ||
| public Schema getBeamSchema() { | ||
| Schema beamSchema; | ||
| try (SpannerAccessor spannerAccessor = SpannerAccessor.getOrCreate(config)) { | ||
| try (ReadContext readContext = spannerAccessor.getDatabaseClient().singleUse()) { | ||
| ResultSet result = readContext.read(table, KeySet.all(), columns, Options.limit(1)); | ||
| if (result.next()) { | ||
| beamSchema = structTypeToBeamRowSchema(result.getMetadata().getRowType(), true); | ||
| } else { | ||
| throw new SpannerSchemaRetrievalException("Cannot find Spanner table."); | ||
| } | ||
| } | ||
| } catch (Exception e) { | ||
| throw new SpannerSchemaRetrievalException("Exception while trying to retrieve schema", e); | ||
| } | ||
| return beamSchema; | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ | |
| import com.google.cloud.Timestamp; | ||
| import com.google.cloud.spanner.Struct; | ||
| import com.google.cloud.spanner.Type; | ||
| import com.google.spanner.v1.StructType; | ||
| import java.math.BigDecimal; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
|
|
@@ -38,6 +39,20 @@ | |
|
|
||
| final class StructUtils { | ||
|
|
||
| private static final SpannerIO.Read.ToBeamRowFunction STRUCT_TO_BEAM_ROW_FUNCTION = | ||
| schema -> (Struct struct) -> structToBeamRow(struct, schema); | ||
|
|
||
| public static SpannerIO.Read.ToBeamRowFunction structToBeamRow() { | ||
| return STRUCT_TO_BEAM_ROW_FUNCTION; | ||
| } | ||
|
|
||
| private static final SpannerIO.Read.FromBeamRowFunction STRUCT_FROM_BEAM_ROW_FUNCTION = | ||
damccorm marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| ignored -> StructUtils::beamRowToStruct; | ||
|
|
||
| public static SpannerIO.Read.FromBeamRowFunction structFromBeamRow() { | ||
| return STRUCT_FROM_BEAM_ROW_FUNCTION; | ||
| } | ||
|
|
||
| // It's not possible to pass nulls as values even with a field is nullable | ||
| @SuppressWarnings({ | ||
| "nullness" // TODO(https://github.com/apache/beam/issues/20497) | ||
|
|
@@ -52,6 +67,58 @@ public static Row structToBeamRow(Struct struct, Schema schema) { | |
| return Row.withSchema(schema).withFieldValues(structValues).build(); | ||
| } | ||
|
|
||
| public static Schema structTypeToBeamRowSchema(StructType structType, boolean isRead) { | ||
| Schema.Builder beamSchema = Schema.builder(); | ||
| structType | ||
| .getFieldsList() | ||
| .forEach( | ||
| field -> { | ||
| Schema.FieldType fieldType; | ||
| try { | ||
| fieldType = convertSpannerTypeToBeamFieldType(field.getType()); | ||
| } catch (IllegalArgumentException e) { | ||
| throw new IllegalArgumentException( | ||
| "Error processing struct to row: " + e.getMessage()); | ||
| } | ||
| // Treat reads from Spanner as Nullable and leave Null handling to Spanner | ||
| if (isRead) { | ||
| beamSchema.addNullableField(field.getName(), fieldType); | ||
| } else { | ||
| beamSchema.addField(field.getName(), fieldType); | ||
| } | ||
| }); | ||
| return beamSchema.build(); | ||
| } | ||
|
|
||
| public static Schema.FieldType convertSpannerTypeToBeamFieldType( | ||
| com.google.spanner.v1.Type spannerType) { | ||
| switch (spannerType.getCode()) { | ||
| case BOOL: | ||
| return Schema.FieldType.BOOLEAN; | ||
| case BYTES: | ||
| return Schema.FieldType.BYTES; | ||
| case TIMESTAMP: | ||
| case DATE: | ||
| return Schema.FieldType.DATETIME; | ||
| case INT64: | ||
| return Schema.FieldType.INT64; | ||
| case FLOAT32: | ||
| return Schema.FieldType.FLOAT; | ||
| case FLOAT64: | ||
| return Schema.FieldType.DOUBLE; | ||
| case NUMERIC: | ||
| return Schema.FieldType.DECIMAL; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is it expected that the datatype of the corresponding values in the rowStruct to be of certain datatype? Like what is the datatype expected for DECIMAL? Does Spanner client library give the data in the expected datatype for all these types?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since data types are not 1:1 for many different sources in Beam (BigQuery has lots of data types that are exclusive to BigQuery and do not exist as Java types) we map to the closest Java type. For example, As I said in my other comment, all the data types that were added in this PR were tested and all the data conversions worked successfully |
||
| case ARRAY: | ||
| return Schema.FieldType.array( | ||
| convertSpannerTypeToBeamFieldType(spannerType.getArrayElementType())); | ||
| case STRUCT: | ||
| throw new IllegalArgumentException( | ||
| String.format("Unsupported type '%s'.", spannerType.getCode())); | ||
| default: | ||
| return Schema.FieldType.STRING; | ||
| } | ||
| } | ||
|
|
||
| public static Struct beamRowToStruct(Row row) { | ||
| Struct.Builder structBuilder = Struct.newBuilder(); | ||
| List<Schema.Field> fields = row.getSchema().getFields(); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.