TheNeuralBit commented on a change in pull request #12498:
URL: https://github.com/apache/beam/pull/12498#discussion_r470904963



##########
File path: 
sdks/java/extensions/schemaio-expansion-service/src/test/java/org/apache/beam/sdk/extensions/schemaio/expansion/ExternalSchemaIOTransformRegistrarTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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 static org.junit.Assert.assertThrows;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.coders.RowCoder;
+import 
org.apache.beam.sdk.extensions.schemaio.expansion.ExternalSchemaIOTransformRegistrar.Configuration;
+import 
org.apache.beam.sdk.extensions.schemaio.expansion.ExternalSchemaIOTransformRegistrar.ReaderBuilder;
+import 
org.apache.beam.sdk.extensions.schemaio.expansion.ExternalSchemaIOTransformRegistrar.WriterBuilder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.schemas.io.SchemaIO;
+import org.apache.beam.sdk.schemas.io.SchemaIOProvider;
+import org.apache.beam.sdk.values.Row;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
+
+/** Unit tests for {@link ExternalSchemaIOTransformRegistrar}. */
+@RunWith(JUnit4.class)
+public class ExternalSchemaIOTransformRegistrarTest {
+  String location = "test";
+  Schema validDataSchema = 
Schema.builder().addStringField("dataField").build();
+  Schema validConfigSchema = 
Schema.builder().addStringField("configField").build();
+  Row validConfigRow = 
Row.withSchema(validConfigSchema).addValue("value").build();
+
+  byte[] validSchemaBytes = SchemaTranslation.schemaToProto(validDataSchema, 
true).toByteArray();
+  byte[] invalidBytes = "Nice try".getBytes(Charset.defaultCharset());

Review comment:
       Nice :)

##########
File path: sdks/python/apache_beam/io/external/xlang_jdbcio_it_test.py
##########
@@ -144,11 +144,11 @@ def test_xlang_jdbc_read(self):
       result = (
           p
           | 'Read from jdbc' >> ReadFromJdbc(
+              table_name=table_name,

Review comment:
       Ideally we would keep both sets of tests, one that relies on just the 
table name, and one that sets the SELECT/INSERT statement. We can leave this 
for a follow-up though if you file a jira and add a TODO

##########
File path: sdks/python/apache_beam/io/jdbc.py
##########
@@ -96,25 +98,37 @@
 
 
 def default_io_expansion_service():
-  return BeamJarExpansionService('sdks:java:io:expansion-service:shadowJar')
+    return BeamJarExpansionService(
+        ':sdks:java:extensions:schemaio-expansion-service:shadowJar')
 
 
-WriteToJdbcSchema = typing.NamedTuple(
-    'WriteToJdbcSchema',
+ReadFromWriteToJdbcSchema = typing.NamedTuple(

Review comment:
       Filed BEAM-10709 for this

##########
File path: sdks/python/apache_beam/io/jdbc.py
##########
@@ -96,25 +98,37 @@
 
 
 def default_io_expansion_service():
-  return BeamJarExpansionService('sdks:java:io:expansion-service:shadowJar')
+    return BeamJarExpansionService(
+        ':sdks:java:extensions:schemaio-expansion-service:shadowJar')
 
 
-WriteToJdbcSchema = typing.NamedTuple(
-    'WriteToJdbcSchema',
+ReadFromWriteToJdbcSchema = typing.NamedTuple(
+    'ReadFromWriteToJdbcSchema',
+    [
+        ('location', unicode),
+        ('config', bytes)
+    ],
+)
+
+Config = typing.NamedTuple(
+    'Config',
     [
         ('driver_class_name', unicode),
         ('jdbc_url', unicode),
         ('username', unicode),
         ('password', unicode),
         ('connection_properties', typing.Optional[unicode]),
         ('connection_init_sqls', typing.Optional[typing.List[unicode]]),
-        ('statement', unicode),
+        ('write_statement', typing.Optional[unicode]),
+        ('read_query', typing.Optional[unicode]),
+        ('fetch_size', typing.Optional[int]),
+        ('output_parallelization', typing.Optional[bool]),
     ],
 )
 
 
 class WriteToJdbc(ExternalTransform):
-  """A PTransform which writes Rows to the specified database via JDBC.
+    """A PTransform which writes Rows to the specified database via JDBC.

Review comment:
       nit: looks like there's an extraneous whitespace change here.
   
   Please update this docstring (and the Read counterpart) based on our change 
to table_name/statement/query. It should point out that table_name is required 
and statement/query are optional, but can be overriden

##########
File path: sdks/python/apache_beam/io/jdbc.py
##########
@@ -96,25 +98,37 @@
 
 
 def default_io_expansion_service():
-  return BeamJarExpansionService('sdks:java:io:expansion-service:shadowJar')
+    return BeamJarExpansionService(
+        ':sdks:java:extensions:schemaio-expansion-service:shadowJar')
 
 
-WriteToJdbcSchema = typing.NamedTuple(
-    'WriteToJdbcSchema',
+ReadFromWriteToJdbcSchema = typing.NamedTuple(

Review comment:
       nit: WDYT about calling this JdbcConfigSchema?
   

##########
File path: sdks/java/extensions/schemaio-expansion-service/build.gradle
##########
@@ -0,0 +1,47 @@
+/*
+ * 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(
+        ignoreRawtypeErrors: true,
+        automaticModuleName: 
'org.apache.beam.sdk.extensions.schemaio.expansion',
+        validateShadowJar: false,
+        shadowClosure: {
+            manifest {
+                attributes "Main-Class": 
"org.apache.beam.sdk.expansion.service.ExpansionService"
+            }
+        }
+)
+
+dependencies {
+    compile project(path: ":sdks:java:expansion-service")
+    compile project(":sdks:java:io:jdbc")
+    compile library.java.postgres
+    testCompile library.java.junit
+    testCompile library.java.powermock_mockito
+    testCompile library.java.mockito_core
+    testCompile "org.checkerframework:checker-qual:3.5.0"

Review comment:
       Assuming this is because of BEAM-10632? Let's add a TODO here so we 
remember to remove it later
   ```suggestion
       testCompile library.java.mockito_core
       // TODO(BEAM-10632): remove this dependency
       testCompile "org.checkerframework:checker-qual:3.5.0"
   ```

##########
File path: sdks/java/extensions/schemaio-expansion-service/build.gradle
##########
@@ -0,0 +1,47 @@
+/*
+ * 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(
+        ignoreRawtypeErrors: true,
+        automaticModuleName: 
'org.apache.beam.sdk.extensions.schemaio.expansion',
+        validateShadowJar: false,
+        shadowClosure: {
+            manifest {
+                attributes "Main-Class": 
"org.apache.beam.sdk.expansion.service.ExpansionService"
+            }
+        }
+)
+
+dependencies {
+    compile project(path: ":sdks:java:expansion-service")
+    compile project(":sdks:java:io:jdbc")
+    compile library.java.postgres
+    testCompile library.java.junit
+    testCompile library.java.powermock_mockito
+    testCompile library.java.mockito_core
+    testCompile "org.checkerframework:checker-qual:3.5.0"
+}
+
+task runExpansionService (type: JavaExec) {
+    main = "org.apache.beam.sdk.expansion.service.ExpansionService"
+    classpath = sourceSets.main.runtimeClasspath
+    args = [project.findProperty(":constructionService.port") ?: "8097"]
+}

Review comment:
       I don't think we need this task, Python is responsible for running the 
expansion service from the compiled jar using subprocess.

##########
File path: sdks/python/apache_beam/io/jdbc.py
##########
@@ -96,25 +98,37 @@
 
 
 def default_io_expansion_service():
-  return BeamJarExpansionService('sdks:java:io:expansion-service:shadowJar')
+    return BeamJarExpansionService(
+        ':sdks:java:extensions:schemaio-expansion-service:shadowJar')
 
 
-WriteToJdbcSchema = typing.NamedTuple(
-    'WriteToJdbcSchema',
+ReadFromWriteToJdbcSchema = typing.NamedTuple(

Review comment:
       I think the next improvement to SchemaIO/Provider should be to formally 
differentiate read-only and write-only configuration, it's confusing to combine 
them this way when it's not needed. The only place its _needed_ is in SQL, so 
we should do it there.
   (you don't need to do anything here, I'm just getting some thoughts out)




----------------------------------------------------------------
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]


Reply via email to