pabloem commented on code in PR #24629:
URL: https://github.com/apache/beam/pull/24629#discussion_r1053590539
##########
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:
the main thing that converts this into unbounded is the `return
ProcessContinuation.resume().withResumeDelay(Duration.standardSeconds(1));` -
when it returns that, we are asking the runner to checkpoint processing (save
the current progress, and pass the data tat we produced so far downstream, and
call process again in a second to continue)
##########
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:
done
##########
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:
done thanks : D
##########
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:
i made it package protected but not private as its used from other classes
(the test classes)
--
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]