ahmedabu98 commented on code in PR #24629:
URL: https://github.com/apache/beam/pull/24629#discussion_r1052681022


##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaSourceConsumerFn.java:
##########
@@ -89,12 +91,17 @@
    *
    * @param connectorClass Supported Debezium connector class
    * @param fn a SourceRecordMapper
-   * @param minutesToRun Maximum time to run (in minutes)
+   * @param milisecondsToRun Maximum time to run (in milliseconds)
    */
-  KafkaSourceConsumerFn(Class<?> connectorClass, SourceRecordMapper<T> fn, 
long minutesToRun) {
+  KafkaSourceConsumerFn(
+      Class<?> connectorClass,
+      SourceRecordMapper<T> fn,
+      Integer maxRecords,

Review Comment:
   Can you update the documentation above this class (L68 & 72) with the new 
parameters?



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.io.debezium;
+
+import com.google.auto.value.AutoValue;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.checkerframework.checker.initialization.qual.Initialized;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DebeziumReadSchemaTransformProvider
+    extends TypedSchemaTransformProvider<
+        
DebeziumReadSchemaTransformProvider.DebeziumReadSchemaTransformConfiguration> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DebeziumReadSchemaTransformProvider.class);
+  private final Boolean isTest;
+  private final Integer testLimitRecords;
+  private final Long testLimitMilliseconds;
+
+  DebeziumReadSchemaTransformProvider() {
+    this(false, -1, Long.MAX_VALUE);
+  }
+
+  @VisibleForTesting
+  DebeziumReadSchemaTransformProvider(Boolean isTest, Integer recordLimit, 
Long timeLimitMs) {
+    this.isTest = isTest;
+    this.testLimitRecords = recordLimit;
+    this.testLimitMilliseconds = timeLimitMs;
+  }
+
+  @Override
+  protected @NonNull @Initialized 
Class<DebeziumReadSchemaTransformConfiguration>
+      configurationClass() {
+    return DebeziumReadSchemaTransformConfiguration.class;
+  }
+
+  @Override
+  protected @NonNull @Initialized SchemaTransform from(
+      DebeziumReadSchemaTransformConfiguration configuration) {
+    return new SchemaTransform() {
+      @Override
+      public @UnknownKeyFor @NonNull @Initialized PTransform<
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple,
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple>
+          buildTransform() {
+        return new PTransform<PCollectionRowTuple, PCollectionRowTuple>() {
+          @Override
+          public PCollectionRowTuple expand(PCollectionRowTuple input) {
+            // TODO(pabloem): Test this behavior
+            if (!Arrays.stream(Connectors.values())
+                .map(Objects::toString)
+                .collect(Collectors.toSet())
+                .contains(configuration.getDatabase())) {
+              throw new IllegalArgumentException(
+                  "Unsupported dabase "
+                      + configuration.getDatabase()
+                      + ". Unable to select a JDBC driver for it. Supported 
Databases are: "
+                      + String.join(
+                          ", ",
+                          Arrays.stream(Connectors.values())

Review Comment:
   This supported databases array is used twice, maybe declare it above and use 
it as a variable?



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.io.debezium;
+
+import com.google.auto.value.AutoValue;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.checkerframework.checker.initialization.qual.Initialized;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DebeziumReadSchemaTransformProvider
+    extends TypedSchemaTransformProvider<
+        
DebeziumReadSchemaTransformProvider.DebeziumReadSchemaTransformConfiguration> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DebeziumReadSchemaTransformProvider.class);
+  private final Boolean isTest;
+  private final Integer testLimitRecords;
+  private final Long testLimitMilliseconds;
+
+  DebeziumReadSchemaTransformProvider() {
+    this(false, -1, Long.MAX_VALUE);
+  }
+
+  @VisibleForTesting
+  DebeziumReadSchemaTransformProvider(Boolean isTest, Integer recordLimit, 
Long timeLimitMs) {
+    this.isTest = isTest;
+    this.testLimitRecords = recordLimit;
+    this.testLimitMilliseconds = timeLimitMs;
+  }
+
+  @Override
+  protected @NonNull @Initialized 
Class<DebeziumReadSchemaTransformConfiguration>
+      configurationClass() {
+    return DebeziumReadSchemaTransformConfiguration.class;
+  }
+
+  @Override
+  protected @NonNull @Initialized SchemaTransform from(
+      DebeziumReadSchemaTransformConfiguration configuration) {
+    return new SchemaTransform() {
+      @Override
+      public @UnknownKeyFor @NonNull @Initialized PTransform<
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple,
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple>
+          buildTransform() {
+        return new PTransform<PCollectionRowTuple, PCollectionRowTuple>() {
+          @Override
+          public PCollectionRowTuple expand(PCollectionRowTuple input) {
+            // TODO(pabloem): Test this behavior
+            if (!Arrays.stream(Connectors.values())
+                .map(Objects::toString)
+                .collect(Collectors.toSet())
+                .contains(configuration.getDatabase())) {
+              throw new IllegalArgumentException(
+                  "Unsupported dabase "
+                      + configuration.getDatabase()
+                      + ". Unable to select a JDBC driver for it. Supported 
Databases are: "
+                      + String.join(
+                          ", ",
+                          Arrays.stream(Connectors.values())
+                              .map(Object::toString)
+                              .collect(Collectors.toList())));
+            }
+            Class<?> connectorClass =
+                
Objects.requireNonNull(Connectors.valueOf(configuration.getDatabase()))
+                    .getConnector();
+            DebeziumIO.ConnectorConfiguration connectorConfiguration =
+                DebeziumIO.ConnectorConfiguration.create()
+                    .withUsername(configuration.getUsername())
+                    .withPassword(configuration.getPassword())
+                    .withHostName(configuration.getHost())
+                    .withPort(Integer.toString(configuration.getPort()))
+                    .withConnectorClass(connectorClass);
+            connectorConfiguration =
+                connectorConfiguration
+                    .withConnectionProperty("table.include.list", 
configuration.getTable())
+                    .withConnectionProperty("include.schema.changes", "false")
+                    .withConnectionProperty("database.server.name", 
"beam-pipeline-server");
+            if (configuration.getDatabase().equals("POSTGRES")) {
+              LOG.info(
+                  "As Database is POSTGRES, we set the `database.dbname` 
property to {}.",
+                  configuration.getTable().substring(0, 
configuration.getTable().indexOf(".")));
+              connectorConfiguration =
+                  connectorConfiguration.withConnectionProperty(
+                      "database.dbname",
+                      configuration.getTable().substring(0, 
configuration.getTable().indexOf(".")));
+            }
+
+            final List<String> debeziumConnectionProperties =
+                configuration.getDebeziumConnectionProperties();
+            if (debeziumConnectionProperties != null) {
+              for (String connectionProperty : debeziumConnectionProperties) {
+                String[] parts = connectionProperty.split("=", -1);
+                String key = parts[0];
+                String value = parts[1];
+                connectorConfiguration.withConnectionProperty(key, value);
+              }
+            }
+
+            DebeziumIO.Read<Row> readTransform =
+                
DebeziumIO.<Row>read().withConnectorConfiguration(connectorConfiguration);
+
+            if (isTest) {
+              readTransform =
+                  readTransform
+                      .withMaxNumberOfRecords(testLimitRecords)
+                      .withMaxTimeToRun(testLimitMilliseconds);
+            }
+
+            // TODO(pabloem): Database connection issues can be debugged here.
+            Schema recordSchema = readTransform.getRecordSchema();
+            LOG.info(
+                "Computed schema for table {} from {}: {}",
+                configuration.getTable(),
+                configuration.getDatabase(),
+                recordSchema);
+            SourceRecordMapper<Row> formatFn =
+                KafkaConnectUtils.beamRowFromSourceRecordFn(recordSchema);
+            readTransform =
+                
readTransform.withFormatFunction(formatFn).withCoder(RowCoder.of(recordSchema));
+
+            return PCollectionRowTuple.of("output", 
input.getPipeline().apply(readTransform));
+          }
+        };
+      }
+    };
+  }
+
+  @Override
+  public @NonNull @Initialized String identifier() {
+    return "beam:schematransform:org.apache.beam:debezium_read:v1";
+  }
+
+  @Override
+  public @NonNull @Initialized List<@NonNull @Initialized String> 
inputCollectionNames() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public @NonNull @Initialized List<@NonNull @Initialized String> 
outputCollectionNames() {
+    return Collections.singletonList("output");
+  }
+
+  @AutoValue
+  public abstract static class DebeziumReadSchemaTransformConfiguration {

Review Comment:
   I mentioned two things above that can be considered for validation, maybe 
there are others. This would make this SchemaTransform more user-friendly but I 
think these are not immediate blockers (maybe just add a TODO).



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.io.debezium;
+
+import com.google.auto.value.AutoValue;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.checkerframework.checker.initialization.qual.Initialized;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DebeziumReadSchemaTransformProvider
+    extends TypedSchemaTransformProvider<
+        
DebeziumReadSchemaTransformProvider.DebeziumReadSchemaTransformConfiguration> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DebeziumReadSchemaTransformProvider.class);
+  private final Boolean isTest;
+  private final Integer testLimitRecords;
+  private final Long testLimitMilliseconds;
+
+  DebeziumReadSchemaTransformProvider() {
+    this(false, -1, Long.MAX_VALUE);
+  }
+
+  @VisibleForTesting
+  DebeziumReadSchemaTransformProvider(Boolean isTest, Integer recordLimit, 
Long timeLimitMs) {

Review Comment:
   Should this be private?



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaSourceConsumerFn.java:
##########
@@ -127,6 +134,45 @@ public Coder<OffsetHolder> getRestrictionCoder() {
     return SerializableCoder.of(OffsetHolder.class);
   }
 
+  protected SourceRecord getOneRecord(Map<String, String> configuration) {
+    try {
+      SourceConnector connector = 
connectorClass.getDeclaredConstructor().newInstance();
+      connector.start(configuration);
+
+      SourceTask task = (SourceTask) 
connector.taskClass().getDeclaredConstructor().newInstance();
+      task.initialize(new BeamSourceTaskContext(null));
+      task.start(connector.taskConfigs(1).get(0));
+      List<SourceRecord> records = Lists.newArrayList();
+      int loops = 0;
+      while (records.size() == 0) {
+        if (loops > 3) {
+          throw new RuntimeException("could not fetch database schema");
+        }
+        records = task.poll();
+        // Waiting for the Database snapshot to finish.
+        Thread.sleep(2000);

Review Comment:
   Did you consider using backoff here?



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.io.debezium;
+
+import com.google.auto.value.AutoValue;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.checkerframework.checker.initialization.qual.Initialized;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DebeziumReadSchemaTransformProvider
+    extends TypedSchemaTransformProvider<
+        
DebeziumReadSchemaTransformProvider.DebeziumReadSchemaTransformConfiguration> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DebeziumReadSchemaTransformProvider.class);
+  private final Boolean isTest;
+  private final Integer testLimitRecords;
+  private final Long testLimitMilliseconds;
+
+  DebeziumReadSchemaTransformProvider() {
+    this(false, -1, Long.MAX_VALUE);
+  }
+
+  @VisibleForTesting
+  DebeziumReadSchemaTransformProvider(Boolean isTest, Integer recordLimit, 
Long timeLimitMs) {
+    this.isTest = isTest;
+    this.testLimitRecords = recordLimit;
+    this.testLimitMilliseconds = timeLimitMs;
+  }
+
+  @Override
+  protected @NonNull @Initialized 
Class<DebeziumReadSchemaTransformConfiguration>
+      configurationClass() {
+    return DebeziumReadSchemaTransformConfiguration.class;
+  }
+
+  @Override
+  protected @NonNull @Initialized SchemaTransform from(
+      DebeziumReadSchemaTransformConfiguration configuration) {
+    return new SchemaTransform() {
+      @Override
+      public @UnknownKeyFor @NonNull @Initialized PTransform<
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple,
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple>
+          buildTransform() {
+        return new PTransform<PCollectionRowTuple, PCollectionRowTuple>() {
+          @Override
+          public PCollectionRowTuple expand(PCollectionRowTuple input) {
+            // TODO(pabloem): Test this behavior
+            if (!Arrays.stream(Connectors.values())
+                .map(Objects::toString)
+                .collect(Collectors.toSet())
+                .contains(configuration.getDatabase())) {
+              throw new IllegalArgumentException(
+                  "Unsupported dabase "
+                      + configuration.getDatabase()
+                      + ". Unable to select a JDBC driver for it. Supported 
Databases are: "
+                      + String.join(
+                          ", ",
+                          Arrays.stream(Connectors.values())
+                              .map(Object::toString)
+                              .collect(Collectors.toList())));
+            }
+            Class<?> connectorClass =
+                
Objects.requireNonNull(Connectors.valueOf(configuration.getDatabase()))
+                    .getConnector();
+            DebeziumIO.ConnectorConfiguration connectorConfiguration =
+                DebeziumIO.ConnectorConfiguration.create()
+                    .withUsername(configuration.getUsername())
+                    .withPassword(configuration.getPassword())
+                    .withHostName(configuration.getHost())
+                    .withPort(Integer.toString(configuration.getPort()))
+                    .withConnectorClass(connectorClass);
+            connectorConfiguration =
+                connectorConfiguration
+                    .withConnectionProperty("table.include.list", 
configuration.getTable())
+                    .withConnectionProperty("include.schema.changes", "false")
+                    .withConnectionProperty("database.server.name", 
"beam-pipeline-server");
+            if (configuration.getDatabase().equals("POSTGRES")) {
+              LOG.info(
+                  "As Database is POSTGRES, we set the `database.dbname` 
property to {}.",
+                  configuration.getTable().substring(0, 
configuration.getTable().indexOf(".")));

Review Comment:
   `.indexOf(".")` will return -1 if there's no `.` in the table configuration 
field and the substring will throw an error. 
   
   If having the dot is a must, there should be a validation check in 
`DebeziumReadSchemaTransformConfiguration` for this. 



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.io.debezium;
+
+import com.google.auto.value.AutoValue;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.checkerframework.checker.initialization.qual.Initialized;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DebeziumReadSchemaTransformProvider
+    extends TypedSchemaTransformProvider<
+        
DebeziumReadSchemaTransformProvider.DebeziumReadSchemaTransformConfiguration> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DebeziumReadSchemaTransformProvider.class);
+  private final Boolean isTest;
+  private final Integer testLimitRecords;
+  private final Long testLimitMilliseconds;
+
+  DebeziumReadSchemaTransformProvider() {
+    this(false, -1, Long.MAX_VALUE);
+  }
+
+  @VisibleForTesting
+  DebeziumReadSchemaTransformProvider(Boolean isTest, Integer recordLimit, 
Long timeLimitMs) {
+    this.isTest = isTest;
+    this.testLimitRecords = recordLimit;
+    this.testLimitMilliseconds = timeLimitMs;
+  }
+
+  @Override
+  protected @NonNull @Initialized 
Class<DebeziumReadSchemaTransformConfiguration>
+      configurationClass() {
+    return DebeziumReadSchemaTransformConfiguration.class;
+  }
+
+  @Override
+  protected @NonNull @Initialized SchemaTransform from(
+      DebeziumReadSchemaTransformConfiguration configuration) {
+    return new SchemaTransform() {
+      @Override
+      public @UnknownKeyFor @NonNull @Initialized PTransform<
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple,
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple>
+          buildTransform() {
+        return new PTransform<PCollectionRowTuple, PCollectionRowTuple>() {
+          @Override
+          public PCollectionRowTuple expand(PCollectionRowTuple input) {
+            // TODO(pabloem): Test this behavior
+            if (!Arrays.stream(Connectors.values())
+                .map(Objects::toString)
+                .collect(Collectors.toSet())
+                .contains(configuration.getDatabase())) {
+              throw new IllegalArgumentException(
+                  "Unsupported dabase "
+                      + configuration.getDatabase()
+                      + ". Unable to select a JDBC driver for it. Supported 
Databases are: "
+                      + String.join(
+                          ", ",
+                          Arrays.stream(Connectors.values())
+                              .map(Object::toString)
+                              .collect(Collectors.toList())));
+            }
+            Class<?> connectorClass =
+                
Objects.requireNonNull(Connectors.valueOf(configuration.getDatabase()))
+                    .getConnector();
+            DebeziumIO.ConnectorConfiguration connectorConfiguration =
+                DebeziumIO.ConnectorConfiguration.create()
+                    .withUsername(configuration.getUsername())
+                    .withPassword(configuration.getPassword())
+                    .withHostName(configuration.getHost())
+                    .withPort(Integer.toString(configuration.getPort()))
+                    .withConnectorClass(connectorClass);
+            connectorConfiguration =
+                connectorConfiguration
+                    .withConnectionProperty("table.include.list", 
configuration.getTable())
+                    .withConnectionProperty("include.schema.changes", "false")
+                    .withConnectionProperty("database.server.name", 
"beam-pipeline-server");
+            if (configuration.getDatabase().equals("POSTGRES")) {
+              LOG.info(
+                  "As Database is POSTGRES, we set the `database.dbname` 
property to {}.",
+                  configuration.getTable().substring(0, 
configuration.getTable().indexOf(".")));
+              connectorConfiguration =
+                  connectorConfiguration.withConnectionProperty(
+                      "database.dbname",
+                      configuration.getTable().substring(0, 
configuration.getTable().indexOf(".")));
+            }
+
+            final List<String> debeziumConnectionProperties =
+                configuration.getDebeziumConnectionProperties();
+            if (debeziumConnectionProperties != null) {
+              for (String connectionProperty : debeziumConnectionProperties) {
+                String[] parts = connectionProperty.split("=", -1);
+                String key = parts[0];
+                String value = parts[1];

Review Comment:
   Similar to above, I think some validation may be needed for  
`debeziumConnectionProperties`. A customized error message would help when 
users input connection properties that don't follow the "abc=xyz" standard. 



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.io.debezium;
+
+import com.google.auto.value.AutoValue;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.checkerframework.checker.initialization.qual.Initialized;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DebeziumReadSchemaTransformProvider
+    extends TypedSchemaTransformProvider<
+        
DebeziumReadSchemaTransformProvider.DebeziumReadSchemaTransformConfiguration> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DebeziumReadSchemaTransformProvider.class);
+  private final Boolean isTest;
+  private final Integer testLimitRecords;
+  private final Long testLimitMilliseconds;
+
+  DebeziumReadSchemaTransformProvider() {
+    this(false, -1, Long.MAX_VALUE);
+  }
+
+  @VisibleForTesting
+  DebeziumReadSchemaTransformProvider(Boolean isTest, Integer recordLimit, 
Long timeLimitMs) {
+    this.isTest = isTest;
+    this.testLimitRecords = recordLimit;
+    this.testLimitMilliseconds = timeLimitMs;
+  }
+
+  @Override
+  protected @NonNull @Initialized 
Class<DebeziumReadSchemaTransformConfiguration>
+      configurationClass() {
+    return DebeziumReadSchemaTransformConfiguration.class;
+  }
+
+  @Override
+  protected @NonNull @Initialized SchemaTransform from(
+      DebeziumReadSchemaTransformConfiguration configuration) {
+    return new SchemaTransform() {
+      @Override
+      public @UnknownKeyFor @NonNull @Initialized PTransform<
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple,
+              @UnknownKeyFor @NonNull @Initialized PCollectionRowTuple>
+          buildTransform() {
+        return new PTransform<PCollectionRowTuple, PCollectionRowTuple>() {
+          @Override
+          public PCollectionRowTuple expand(PCollectionRowTuple input) {
+            // TODO(pabloem): Test this behavior
+            if (!Arrays.stream(Connectors.values())
+                .map(Objects::toString)
+                .collect(Collectors.toSet())
+                .contains(configuration.getDatabase())) {
+              throw new IllegalArgumentException(
+                  "Unsupported dabase "

Review Comment:
   ```suggestion
                     "Unsupported database "
   ```



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaSourceConsumerFn.java:
##########
@@ -184,22 +230,24 @@ public ProcessContinuation process(
 
           receiver.output(json);
         }
-
         task.commit();
+        records = task.poll();

Review Comment:
   Just outta curiosity, is this (+ the while) what turns it into an unbounded 
DoFn?



##########
sdks/java/io/debezium/src/test/java/org/apache/beam/io/debezium/DebeziumIOPostgresSqlConnectorIT.java:
##########
@@ -51,6 +63,91 @@ public class DebeziumIOPostgresSqlConnectorIT {
           .withExposedPorts(5432)
           .withDatabaseName("inventory");
 
+  static final Schema TABLE_SCHEMA =
+      Schema.builder()
+          .addInt32Field("id")
+          .addStringField("first_name")
+          .addStringField("last_name")
+          .addStringField("email")
+          .build();
+
+  static DataSource getPostgresDatasource() {
+    PGSimpleDataSource dataSource = new PGSimpleDataSource();
+    dataSource.setDatabaseName("inventory");
+    dataSource.setServerName(POSTGRES_SQL_CONTAINER.getContainerIpAddress());
+    dataSource.setPortNumber(POSTGRES_SQL_CONTAINER.getMappedPort(5432));
+    dataSource.setUser("debezium");
+    dataSource.setPassword("dbz");
+    return dataSource;
+  }
+
+  @Test
+  public void testDebeziumSchemaTransformPostgresRead() throws 
InterruptedException {
+    long writeSize = 500L;
+    long testTime = writeSize * 200L;
+    POSTGRES_SQL_CONTAINER.start();
+
+    PipelineOptions options = PipelineOptionsFactory.create();
+    Pipeline writePipeline = Pipeline.create(options);
+    writePipeline
+        .apply(GenerateSequence.from(0).to(writeSize).withRate(10, 
Duration.standardSeconds(1)))
+        .apply(
+            MapElements.into(TypeDescriptors.rows())
+                .via(
+                    num ->
+                        Row.withSchema(TABLE_SCHEMA)
+                            .withFieldValue(
+                                "id",
+                                // We need this tricky conversion because the 
original "customers"
+                                // table already
+                                // contains rows 1001, 1002, 1003, 1004.
+                                num <= 1000
+                                    ? Long.valueOf(num).intValue()
+                                    : Long.valueOf(num).intValue() + 4)
+                            .withFieldValue("first_name", Long.toString(num))
+                            .withFieldValue("last_name", 
Long.toString(writeSize - num))
+                            .withFieldValue("email", Long.toString(num) + 
"@beamail.com")
+                            // TODO(pabloem): Add other data types
+                            .build()))
+        .setRowSchema(TABLE_SCHEMA)
+        .apply(
+            JdbcIO.<Row>write()
+                .withTable("inventory.inventory.customers")

Review Comment:
   Is this a typo for `inventory.customers`?



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaConnectUtils.java:
##########
@@ -74,4 +77,45 @@ public static Schema.FieldType beamSchemaTypeFromKafkaType(
                 "Unable to convert Kafka field schema %s to Beam Schema", 
kafkaFieldSchema));
     }
   }
+
+  public static SourceRecordMapper<Row> beamRowFromSourceRecordFn(final Schema 
recordSchema) {
+    return new SourceRecordMapper<Row>() {
+      @Override
+      public Row mapSourceRecord(SourceRecord sourceRecord) throws Exception {
+        return beamRowFromKafkaStruct((Struct) sourceRecord.value(), 
recordSchema);
+      }
+
+      private Row beamRowFromKafkaStruct(Struct kafkaStruct, Schema 
beamSchema) {
+        Row.Builder rowBuilder = Row.withSchema(beamSchema);
+        for (Schema.Field f : beamSchema.getFields()) {
+          Object structField = kafkaStruct.getWithoutDefault(f.getName());
+          switch (kafkaStruct.schema().field(f.getName()).schema().type()) {
+            case ARRAY:
+            case MAP:
+              // TODO(pabloem): Handle nested structs
+              throw new IllegalArgumentException("UNABLE TO CONVERT FIELD " + 
f);
+            case STRUCT:
+              Schema fieldSchema = f.getType().getRowSchema();
+              if (fieldSchema == null) {
+                throw new IllegalArgumentException(
+                    "Improper schema for Beam record " + fieldSchema);

Review Comment:
   ```suggestion
                       String.format("Improper schema for Beam record: %s has 
no row schema to build a Row from.", f.getName());
   ```



##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaSourceConsumerFn.java:
##########
@@ -89,12 +91,17 @@
    *
    * @param connectorClass Supported Debezium connector class
    * @param fn a SourceRecordMapper
-   * @param minutesToRun Maximum time to run (in minutes)
+   * @param milisecondsToRun Maximum time to run (in milliseconds)
    */
-  KafkaSourceConsumerFn(Class<?> connectorClass, SourceRecordMapper<T> fn, 
long minutesToRun) {
+  KafkaSourceConsumerFn(
+      Class<?> connectorClass,
+      SourceRecordMapper<T> fn,
+      Integer maxRecords,

Review Comment:
   P.S. also the documentation for the constructors



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to