TheNeuralBit commented on a change in pull request #12498:
URL: https://github.com/apache/beam/pull/12498#discussion_r467112288
##########
File path: settings.gradle
##########
@@ -181,3 +181,4 @@ include "beam-test-tools"
project(":beam-test-tools").dir = file(".test-infra/tools")
include "beam-test-jenkins"
project(":beam-test-jenkins").dir = file(".test-infra/jenkins")
+include 'sdks:java:extensions:schemaio-expansion-service'
Review comment:
nit: can you move this up with the other :sdks:java entries and match
the formatting
##########
File path: sdks/java/extensions/schemaio-expansion-service/build.gradle
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+plugins {
+ id 'org.apache.beam.module'
+}
+
+applyJavaNature(
+ enableChecker:false,
Review comment:
Can you remove this? We've disabled checker framework in most projects
because we just added it and we're still cleaning up all the bugs it's finding,
but for a new project we should turn checker framework on and fix any issues it
finds.
##########
File path:
sdks/java/extensions/schemaio-expansion-service/src/main/java/org/apache/beam/sdk/extensions/schemaio/expansion/package-info.java
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+/** External Transform Registration for SchemaIOs. */
+package org.apache.beam.sdk.extensions.schemaio.expansion;
Review comment:
```suggestion
@DefaultQualifier(NonNull.class)
package org.apache.beam.sdk.extensions.schemaio.expansion;
import org.checkerframework.framework.qual.DefaultQualifier;
import org.checkerframework.checker.nullness.qual.NonNull;
```
We should make everything default to NonNull. After you do this you'll
likely get a lot of build errors from checker framework that will require you
to add `@Nullable` on anything that can be null.
(See https://checkerframework.org/manual/#default-qualifier)
##########
File path: sdks/python/apache_beam/io/jdbc.py
##########
@@ -168,41 +182,33 @@ def __init__(
:param expansion_service: The address (host:port) of the ExpansionService.
"""
- super(WriteToJdbc, self).__init__(
- self.URN,
- NamedTupleBasedPayloadBuilder(
- WriteToJdbcSchema(
- driver_class_name=driver_class_name,
- jdbc_url=jdbc_url,
- username=username,
- password=password,
- statement=statement,
- connection_properties=connection_properties,
- connection_init_sqls=connection_init_sqls,
+ super(WriteToJdbc, self).__init__(
+ self.URN,
+ NamedTupleBasedPayloadBuilder(
+ ReadFromWriteToJdbcSchema(
+ location=jdbc_url,
Review comment:
I think the better analog for "location" would actually be the table
name, and the JDBC URL would be part of the configuration. Unfortunately it
looks like the typical use-case (i.e. what we have in xlang_jdbcio_it_test)
just has the table name implicitly in the query:
https://github.com/apache/beam/blob/84cbd9bdfbe45221b8f3302cedad7bdb47cd2f5a/sdks/python/apache_beam/io/external/xlang_jdbcio_it_test.py#L151
and in the statement:
https://github.com/apache/beam/blob/84cbd9bdfbe45221b8f3302cedad7bdb47cd2f5a/sdks/python/apache_beam/io/external/xlang_jdbcio_it_test.py#L120
Maybe what we should do here is require a `table_name` parameter that we
will use for the location. Then, in the JdbcSchemaIOProvider, we can actually
generate the query and statement. The user would still have the ability to
override both of these if they prefer. WDYT?
##########
File path:
sdks/java/extensions/schemaio-expansion-service/src/main/java/org/apache/beam/sdk/extensions/schemaio/expansion/ExternalSchemaIOTransformRegistrar.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.extensions.schemaio.expansion;
+
+import com.google.auto.service.AutoService;
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.ServiceLoader;
+import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.SchemaApi;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.schemas.io.SchemaIOProvider;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.Row;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+@Experimental(Experimental.Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class ExternalSchemaIOTransformRegistrar implements
ExternalTransformRegistrar {
+
+ @Override
+ public Map<String, ExternalTransformBuilder<?, ?, ?>>
knownBuilderInstances() {
+ ImmutableMap.Builder builder = ImmutableMap.<String,
ExternalTransformRegistrar>builder();
+ try {
+ for (SchemaIOProvider schemaIOProvider :
ServiceLoader.load(SchemaIOProvider.class)) {
+ builder.put(
+ "beam:external:java:" + schemaIOProvider.identifier() + ":read:v1",
+ new ReaderBuilder(schemaIOProvider));
+ builder.put(
+ "beam:external:java:" + schemaIOProvider.identifier() +
":write:v1",
+ new WriterBuilder(schemaIOProvider));
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e.getMessage());
+ }
+ return builder.build();
+ }
+
+ public static class Configuration {
+ String location = "";
+ byte[] config = new byte[0];
+ @Nullable byte[] dataSchema = null;
+
+ public void setLocation(String location) {
+ this.location = location;
+ }
+
+ public void setConfig(byte[] config) {
+ this.config = config;
+ }
+
+ public void setDataSchema(byte[] dataSchema) {
+ this.dataSchema = dataSchema;
+ }
+ }
+
+ @Nullable
+ private static Schema translateSchema(@Nullable byte[] schemaBytes) throws
Exception {
+ if (schemaBytes == null) {
+ return null;
+ }
+ SchemaApi.Schema protoSchema = SchemaApi.Schema.parseFrom(schemaBytes);
+ return SchemaTranslation.schemaFromProto(protoSchema);
+ }
+
+ private static Row translateRow(byte[] rowBytes, Schema configSchema) throws
Exception {
+ RowCoder rowCoder = RowCoder.of(configSchema);
+ InputStream stream = new ByteArrayInputStream(rowBytes);
+ return rowCoder.decode(stream);
+ }
+
+ private static class ReaderBuilder
+ implements ExternalTransformBuilder<Configuration, PBegin,
PCollection<Row>> {
+ SchemaIOProvider schemaIOProvider;
+
+ ReaderBuilder(SchemaIOProvider schemaIOProvider) {
+ this.schemaIOProvider = schemaIOProvider;
+ }
+
+ @Override
+ public PTransform<PBegin, PCollection<Row>> buildExternal(Configuration
configuration) {
+ try {
+ return schemaIOProvider
+ .from(
+ configuration.location,
+ translateRow(configuration.config,
schemaIOProvider.configurationSchema()),
+ translateSchema(configuration.dataSchema))
+ .buildReader();
+ } catch (Exception e) {
+ throw new RuntimeException("Could not convert configuration proto to
row or schema.");
+ }
+ }
+ }
+
+ private static class WriterBuilder
+ implements ExternalTransformBuilder<Configuration, PCollection<Row>,
PDone> {
+ SchemaIOProvider schemaIOProvider;
+
+ WriterBuilder(SchemaIOProvider schemaIOProvider) {
+ this.schemaIOProvider = schemaIOProvider;
+ }
+
+ @Override
+ public PTransform<PCollection<Row>, PDone> buildExternal(Configuration
configuration) {
+ try {
+ return (PTransform<PCollection<Row>, PDone>)
+ schemaIOProvider
+ .from(
+ configuration.location,
+ translateRow(configuration.config,
schemaIOProvider.configurationSchema()),
+ translateSchema(configuration.dataSchema))
+ .buildWriter();
+ } catch (Exception e) {
+ throw new RuntimeException("Could not convert configuration proto to
row or schema.");
Review comment:
This isn't ideal since it will swallow up any other exception that
occurs inside the try block, and when it does catch something from translateRow
or translateSchema it's not clear where the problem is. Could you instead add a
try-catch in translateRow and translateSchema that catches the specific
exception(s) that might be thrown, and re-throws them with a RuntimeException?
When you re-throw you should always reference the original exception as
well, like `throw new RuntimeException("Some additional context", e)`
##########
File path:
sdks/java/extensions/schemaio-expansion-service/src/main/java/org/apache/beam/sdk/extensions/schemaio/expansion/ExternalSchemaIOTransformRegistrar.java
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.extensions.schemaio.expansion;
+
+import com.google.auto.service.AutoService;
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.ServiceLoader;
+import org.apache.beam.model.pipeline.v1.SchemaApi;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.schemas.io.SchemaIOProvider;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.Row;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+@Experimental(Experimental.Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class ExternalSchemaIOTransformRegistrar implements
ExternalTransformRegistrar {
+ private static final String URN = "beam:external:java:schemaio:v1";
Review comment:
I don't think this URN is used.
##########
File path:
sdks/java/extensions/schemaio-expansion-service/src/main/java/org/apache/beam/sdk/extensions/schemaio/expansion/ExternalSchemaIOTransformRegistrar.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.extensions.schemaio.expansion;
+
+import com.google.auto.service.AutoService;
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.ServiceLoader;
+import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.SchemaApi;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.schemas.io.SchemaIOProvider;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.Row;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+@Experimental(Experimental.Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class ExternalSchemaIOTransformRegistrar implements
ExternalTransformRegistrar {
+
+ @Override
+ public Map<String, ExternalTransformBuilder<?, ?, ?>>
knownBuilderInstances() {
+ ImmutableMap.Builder builder = ImmutableMap.<String,
ExternalTransformRegistrar>builder();
+ try {
+ for (SchemaIOProvider schemaIOProvider :
ServiceLoader.load(SchemaIOProvider.class)) {
+ builder.put(
+ "beam:external:java:" + schemaIOProvider.identifier() + ":read:v1",
+ new ReaderBuilder(schemaIOProvider));
+ builder.put(
+ "beam:external:java:" + schemaIOProvider.identifier() +
":write:v1",
Review comment:
Let's make these URNs mention schemaio to distinguish them in case there
are non-schemaio versions. Maybe `beam:external:java:schemaio:<identifier>`
##########
File path:
sdks/java/extensions/schemaio-expansion-service/src/main/java/org/apache/beam/sdk/extensions/schemaio/expansion/ExternalSchemaIOTransformRegistrar.java
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.extensions.schemaio.expansion;
+
+import com.google.auto.service.AutoService;
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.ServiceLoader;
+import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.SchemaApi;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.schemas.io.SchemaIOProvider;
+import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.Row;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+@Experimental(Experimental.Kind.PORTABILITY)
+@AutoService(ExternalTransformRegistrar.class)
+public class ExternalSchemaIOTransformRegistrar implements
ExternalTransformRegistrar {
+
+ @Override
+ public Map<String, ExternalTransformBuilder<?, ?, ?>>
knownBuilderInstances() {
+ ImmutableMap.Builder builder = ImmutableMap.<String,
ExternalTransformRegistrar>builder();
+ try {
+ for (SchemaIOProvider schemaIOProvider :
ServiceLoader.load(SchemaIOProvider.class)) {
+ builder.put(
+ "beam:external:java:" + schemaIOProvider.identifier() + ":read:v1",
+ new ReaderBuilder(schemaIOProvider));
+ builder.put(
+ "beam:external:java:" + schemaIOProvider.identifier() +
":write:v1",
+ new WriterBuilder(schemaIOProvider));
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e.getMessage());
+ }
+ return builder.build();
+ }
+
+ public static class Configuration {
+ String location = "";
+ byte[] config = new byte[0];
+ @Nullable byte[] dataSchema = null;
+
+ public void setLocation(String location) {
+ this.location = location;
+ }
+
+ public void setConfig(byte[] config) {
+ this.config = config;
+ }
+
+ public void setDataSchema(byte[] dataSchema) {
+ this.dataSchema = dataSchema;
+ }
+ }
+
+ @Nullable
+ private static Schema translateSchema(@Nullable byte[] schemaBytes) throws
Exception {
+ if (schemaBytes == null) {
+ return null;
+ }
+ SchemaApi.Schema protoSchema = SchemaApi.Schema.parseFrom(schemaBytes);
+ return SchemaTranslation.schemaFromProto(protoSchema);
+ }
+
+ private static Row translateRow(byte[] rowBytes, Schema configSchema) throws
Exception {
+ RowCoder rowCoder = RowCoder.of(configSchema);
+ InputStream stream = new ByteArrayInputStream(rowBytes);
+ return rowCoder.decode(stream);
+ }
+
+ private static class ReaderBuilder
+ implements ExternalTransformBuilder<Configuration, PBegin,
PCollection<Row>> {
+ SchemaIOProvider schemaIOProvider;
+
+ ReaderBuilder(SchemaIOProvider schemaIOProvider) {
+ this.schemaIOProvider = schemaIOProvider;
+ }
+
+ @Override
+ public PTransform<PBegin, PCollection<Row>> buildExternal(Configuration
configuration) {
+ try {
+ return schemaIOProvider
+ .from(
+ configuration.location,
+ translateRow(configuration.config,
schemaIOProvider.configurationSchema()),
+ translateSchema(configuration.dataSchema))
+ .buildReader();
+ } catch (Exception e) {
+ throw new RuntimeException("Could not convert configuration proto to
row or schema.");
+ }
+ }
+ }
+
+ private static class WriterBuilder
Review comment:
There should probably be some unit tests that exercise WriterBuilder and
ReaderBuilder. You could create instances that reference a mock
SchemaIOProvider and verify they do what we expect.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]