This is an automated email from the ASF dual-hosted git repository.
yhu pushed a commit to branch release-2.65
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/release-2.65 by this push:
new e7098bdcbe0 fix: correct spanner column schema type parser (#34868)
e7098bdcbe0 is described below
commit e7098bdcbe004f7caa7fd69c461aad1c9e20fd34
Author: Niel Markwick <[email protected]>
AuthorDate: Tue May 6 21:22:25 2025 +0200
fix: correct spanner column schema type parser (#34868)
Regex for parsing array types did not take into account
sized primitive types, eg ARRAY<STRING(MAX)>
Fixes #34863
---
.../beam/sdk/io/gcp/spanner/SpannerSchema.java | 6 ++--
.../beam/sdk/io/gcp/spanner/SpannerSchemaTest.java | 33 +++++++++++++---------
2 files changed, 24 insertions(+), 15 deletions(-)
diff --git
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java
index ebe5815d568..5483364ff5a 100644
---
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java
+++
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java
@@ -208,8 +208,10 @@ public abstract class SpannerSchema implements
Serializable {
return Type.bytes();
}
if (spannerType.startsWith("ARRAY")) {
- // Substring "ARRAY<xxx>"
- Pattern pattern = Pattern.compile("ARRAY<([^>(]+)>");
+ // find 'xxx' in string ARRAY<xxxxx>
+ // Graph DBs may have suffixes, eg
ARRAY<FLOAT32>(vector_length=>256)
+ //
+ Pattern pattern = Pattern.compile("ARRAY<([^>]+)>");
Matcher matcher = pattern.matcher(originalSpannerType);
if (matcher.find()) {
diff --git
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java
index 14f3dbdd4ed..b02f5428ed7 100644
---
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java
+++
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java
@@ -33,22 +33,26 @@ public class SpannerSchemaTest {
public void testSingleTable() throws Exception {
SpannerSchema schema =
SpannerSchema.builder()
- .addColumn("test", "pk", "STRING(48)")
- .addKeyPart("test", "pk", false)
- .addColumn("test", "maxKey", "STRING(MAX)")
- .addColumn("test", "numericVal", "NUMERIC")
- .addColumn("test", "jsonVal", "JSON")
- .addColumn("test", "protoVal", "PROTO<customer.app.TestMessage>")
- .addColumn("test", "enumVal", "ENUM<customer.app.TestEnum>")
- .addColumn("test", "tokens", "TOKENLIST")
- .addColumn("test", "uuidCol", "UUID")
- .addColumn("test", "arrayVal",
"ARRAY<FLOAT32>(vector_length=>256)")
- .addColumn("test", "arrayValue", "ARRAY<FLOAT32>")
+ .addColumn("test", "pk_0", "STRING(48)")
+ .addKeyPart("test", "pk_0", false)
+ .addColumn("test", "maxKey_1", "STRING(MAX)")
+ .addColumn("test", "numericVal_2", "NUMERIC")
+ .addColumn("test", "jsonVal_3", "JSON")
+ .addColumn("test", "protoVal_4", "PROTO<customer.app.TestMessage>")
+ .addColumn("test", "enumVal_5", "ENUM<customer.app.TestEnum>")
+ .addColumn("test", "tokens_6", "TOKENLIST")
+ .addColumn("test", "uuidCol_7", "UUID")
+ .addColumn("test", "arrayVal_8",
"ARRAY<FLOAT32>(vector_length=>256)")
+ .addColumn("test", "sizedArrayVal_9", "ARRAY<STRING(MAX)>")
+ .addColumn("test", "sizedByteVal_10", "ARRAY<BYTES(1024)>")
+ .addColumn("test", "hexSizedByteVal_11", "ARRAY<BYTES(0x400)>")
+ .addColumn("test", "arrayValue_12", "ARRAY<FLOAT32>")
.build();
assertEquals(1, schema.getTables().size());
- assertEquals(10, schema.getColumns("test").size());
+ assertEquals(13, schema.getColumns("test").size());
assertEquals(1, schema.getKeyParts("test").size());
+ assertEquals(Type.numeric(), schema.getColumns("test").get(2).getType());
assertEquals(Type.json(), schema.getColumns("test").get(3).getType());
assertEquals(
Type.proto("customer.app.TestMessage"),
schema.getColumns("test").get(4).getType());
@@ -57,7 +61,10 @@ public class SpannerSchemaTest {
assertEquals(Type.bytes(), schema.getColumns("test").get(6).getType());
assertEquals(Type.string(), schema.getColumns("test").get(7).getType());
assertEquals(Type.array(Type.float32()),
schema.getColumns("test").get(8).getType());
- assertEquals(Type.array(Type.float32()),
schema.getColumns("test").get(9).getType());
+ assertEquals(Type.array(Type.string()),
schema.getColumns("test").get(9).getType());
+ assertEquals(Type.array(Type.bytes()),
schema.getColumns("test").get(10).getType());
+ assertEquals(Type.array(Type.bytes()),
schema.getColumns("test").get(11).getType());
+ assertEquals(Type.array(Type.float32()),
schema.getColumns("test").get(12).getType());
}
@Test