johnjcasey commented on code in PR #23535:
URL: https://github.com/apache/beam/pull/23535#discussion_r1008335684


##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/DataSourceConfiguration.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.io.singlestore;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import javax.sql.DataSource;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * A POJO describing a SingleStoreDB {@link DataSource} by providing all 
properties needed to create
+ * it.
+ */
+@AutoValue
+public abstract class DataSourceConfiguration implements Serializable {
+  abstract @Nullable ValueProvider<String> getEndpoint();

Review Comment:
   These don't need to be valueproviders. ValueProvider was needed for legacy 
templates, but with flextemplates they are no longer required



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/DataSourceConfiguration.java:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.io.singlestore;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import javax.sql.DataSource;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * A POJO describing a SingleStoreDB {@link DataSource} by providing all 
properties needed to create
+ * it.
+ */
+@AutoValue
+public abstract class DataSourceConfiguration implements Serializable {
+  abstract @Nullable ValueProvider<String> getEndpoint();
+
+  abstract @Nullable ValueProvider<String> getUsername();
+
+  abstract @Nullable ValueProvider<String> getPassword();
+
+  abstract @Nullable ValueProvider<String> getDatabase();
+
+  abstract @Nullable ValueProvider<String> getConnectionProperties();
+
+  abstract Builder builder();
+
+  @AutoValue.Builder
+  abstract static class Builder {
+    abstract Builder setEndpoint(ValueProvider<String> endpoint);
+
+    abstract Builder setUsername(ValueProvider<String> username);
+
+    abstract Builder setPassword(ValueProvider<String> password);
+
+    abstract Builder setDatabase(ValueProvider<String> database);
+
+    abstract Builder setConnectionProperties(ValueProvider<String> 
connectionProperties);
+
+    abstract DataSourceConfiguration build();
+  }
+
+  public static DataSourceConfiguration create(String endpoint) {
+    checkNotNull(endpoint, "endpoint can not be null");
+    return create(ValueProvider.StaticValueProvider.of(endpoint));
+  }
+
+  public static DataSourceConfiguration create(ValueProvider<String> endpoint) 
{
+    checkNotNull(endpoint, "endpoint can not be null");
+    return new 
AutoValue_DataSourceConfiguration.Builder().setEndpoint(endpoint).build();
+  }
+
+  public DataSourceConfiguration withUsername(String username) {
+    checkNotNull(username, "username can not be null");
+    return withUsername(ValueProvider.StaticValueProvider.of(username));
+  }
+
+  public DataSourceConfiguration withUsername(ValueProvider<String> username) {
+    checkNotNull(username, "username can not be null");
+    return builder().setUsername(username).build();
+  }
+
+  public DataSourceConfiguration withPassword(String password) {
+    checkNotNull(password, "password can not be null");
+    return withPassword(ValueProvider.StaticValueProvider.of(password));
+  }
+
+  public DataSourceConfiguration withPassword(ValueProvider<String> password) {
+    checkNotNull(password, "password can not be null");
+    return builder().setPassword(password).build();
+  }
+
+  public DataSourceConfiguration withDatabase(String database) {
+    checkNotNull(database, "database can not be null");
+    return withDatabase(ValueProvider.StaticValueProvider.of(database));
+  }
+
+  public DataSourceConfiguration withDatabase(ValueProvider<String> database) {
+    checkNotNull(database, "database can not be null");
+    return builder().setDatabase(database).build();
+  }
+
+  /**
+   * Sets the connection properties passed to driver.connect(...). Format of 
the string must be
+   * [propertyName=property;]*
+   *
+   * <p>NOTE - The "user" and "password" properties can be add via {@link 
#withUsername(String)},
+   * {@link #withPassword(String)}, so they do not need to be included here.
+   *
+   * <p>Full list of supported properties can be found here {@link <a
+   * 
href="https://docs.singlestore.com/managed-service/en/developer-resources/connect-with-application-development-tools/connect-with-java-jdbc/the-singlestore-jdbc-driver.html#connection-string-parameters";>...</a>}
+   */
+  public DataSourceConfiguration withConnectionProperties(String 
connectionProperties) {
+    checkNotNull(connectionProperties, "connectionProperties can not be null");
+    return 
withConnectionProperties(ValueProvider.StaticValueProvider.of(connectionProperties));
+  }
+
+  /** Same as {@link #withConnectionProperties(String)} but accepting a 
ValueProvider. */
+  public DataSourceConfiguration withConnectionProperties(
+      ValueProvider<String> connectionProperties) {
+    checkNotNull(connectionProperties, "connectionProperties can not be null");
+    return builder().setConnectionProperties(connectionProperties).build();
+  }
+
+  public static void populateDisplayData(
+      @Nullable DataSourceConfiguration dataSourceConfiguration, 
DisplayData.Builder builder) {
+    if (dataSourceConfiguration != null) {
+      builder.addIfNotNull(DisplayData.item("endpoint", 
dataSourceConfiguration.getEndpoint()));
+      builder.addIfNotNull(DisplayData.item("username", 
dataSourceConfiguration.getUsername()));
+      builder.addIfNotNull(DisplayData.item("database", 
dataSourceConfiguration.getDatabase()));
+      builder.addIfNotNull(
+          DisplayData.item(
+              "connectionProperties", 
dataSourceConfiguration.getConnectionProperties()));
+    }
+  }
+
+  public DataSource getDataSource() {
+    String endpoint = Util.getRequiredArgument(getEndpoint(), "endpoint can 
not be null");

Review Comment:
   Prefer using org.apache.beam.sdk.util.Preconditions for some of these checks



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/Util.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.io.singlestore;
+
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+
+public class Util {

Review Comment:
   Can we rename this to SingleStoreUtil, or something more descriptive? Also, 
please add some class level commentary



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/Write.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.io.singlestore;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import javax.sql.DataSource;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.commons.dbcp2.DelegatingStatement;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Instant;
+
+@AutoValue
+public abstract class Write<T> extends PTransform<PCollection<T>, PDone> {
+
+  private static final int DEFAULT_BATCH_SIZE = 100000;
+  private static final int BUFFER_SIZE = 524288;
+
+  abstract @Nullable DataSourceConfiguration getDataSourceConfiguration();
+
+  abstract @Nullable ValueProvider<String> getTable();
+
+  abstract @Nullable ValueProvider<Integer> getBatchSize();
+
+  abstract @Nullable UserDataMapper<T> getUserDataMapper();
+
+  abstract Builder<T> toBuilder();
+
+  @AutoValue.Builder
+  abstract static class Builder<T> {
+    abstract Builder<T> setDataSourceConfiguration(DataSourceConfiguration 
dataSourceConfiguration);
+
+    abstract Builder<T> setTable(ValueProvider<String> table);
+
+    abstract Builder<T> setBatchSize(ValueProvider<Integer> batchSize);
+
+    abstract Builder<T> setUserDataMapper(UserDataMapper<T> userDataMapper);
+
+    abstract Write<T> build();
+  }
+
+  public Write<T> withDataSourceConfiguration(DataSourceConfiguration config) {
+    checkNotNull(config, "dataSourceConfiguration can not be null");
+    return toBuilder().setDataSourceConfiguration(config).build();
+  }
+
+  public Write<T> withTable(String table) {
+    checkNotNull(table, "table can not be null");
+    return withTable(ValueProvider.StaticValueProvider.of(table));
+  }
+
+  public Write<T> withTable(ValueProvider<String> table) {
+    checkNotNull(table, "table can not be null");
+    return toBuilder().setTable(table).build();
+  }
+
+  public Write<T> withUserDataMapper(UserDataMapper<T> userDataMapper) {
+    checkNotNull(userDataMapper, "userDataMapper can not be null");
+    return toBuilder().setUserDataMapper(userDataMapper).build();
+  }
+
+  /**
+   * Provide a maximum number of rows that is written by one SQL statement. 
Default is 100000.
+   *
+   * @param batchSize maximum number of rows that is written by one SQL 
statement
+   */
+  public Write<T> withBatchSize(Integer batchSize) {
+    checkNotNull(batchSize, "batchSize can not be null");
+    return withBatchSize(ValueProvider.StaticValueProvider.of(batchSize));
+  }
+
+  /** Same as {@link #withBatchSize(Integer)} but accepting a ValueProvider. */
+  public Write<T> withBatchSize(ValueProvider<Integer> batchSize) {
+    checkNotNull(batchSize, "batchSize can not be null");
+    return toBuilder().setBatchSize(batchSize).build();
+  }
+
+  @Override
+  public PDone expand(PCollection<T> input) {
+    DataSourceConfiguration dataSourceConfiguration =
+        Util.getRequiredArgument(
+            getDataSourceConfiguration(), "withDataSourceConfiguration() is 
required");
+    String table = Util.getRequiredArgument(getTable(), "withTable() is 
required");
+    UserDataMapper<T> userDataMapper =
+        Util.getRequiredArgument(getUserDataMapper(), "withUserDataMapper() is 
required");
+    int batchSize = Util.getArgumentWithDefault(getBatchSize(), 
DEFAULT_BATCH_SIZE);
+    checkArgument(batchSize > 0, "batchSize should be greater then 0");
+
+    input
+        .apply(
+            ParDo.of(
+                new DoFn<T, List<String>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext context) throws 
Exception {
+                    context.output(userDataMapper.mapRow(context.element()));
+                  }
+                }))
+        .setCoder(ListCoder.of(StringUtf8Coder.of()))
+        .apply(ParDo.of(new BatchFn<>(batchSize)))
+        .apply(ParDo.of(new WriteFn<Void>(dataSourceConfiguration, table)))

Review Comment:
   returning void or pdone is considered an antipattern. Better would be to 
return the status of the writes. This enables users to use the output of this 
transform to drive future behavior



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/ReadWithPartitions.java:
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.io.singlestore;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.auto.value.AutoValue;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import javax.sql.DataSource;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.range.OffsetRange;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoValue
+public abstract class ReadWithPartitions<T> extends PTransform<PBegin, 
PCollection<T>> {

Review Comment:
   Probably worth adding documentation arguing that ReadWithPartitions is 
preferred for performance reasons



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/ReadWithPartitions.java:
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.io.singlestore;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.auto.value.AutoValue;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import javax.sql.DataSource;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.range.OffsetRange;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoValue
+public abstract class ReadWithPartitions<T> extends PTransform<PBegin, 
PCollection<T>> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReadWithPartitions.class);
+
+  abstract @Nullable DataSourceConfiguration getDataSourceConfiguration();
+
+  abstract @Nullable ValueProvider<String> getQuery();
+
+  abstract @Nullable ValueProvider<String> getTable();
+
+  abstract @Nullable RowMapper<T> getRowMapper();
+
+  abstract @Nullable ValueProvider<Integer> getInitialNumReaders();
+
+  abstract Builder<T> toBuilder();
+
+  @AutoValue.Builder
+  abstract static class Builder<T> {
+    abstract Builder<T> setDataSourceConfiguration(DataSourceConfiguration 
dataSourceConfiguration);
+
+    abstract Builder<T> setQuery(ValueProvider<String> query);
+
+    abstract Builder<T> setTable(ValueProvider<String> table);
+
+    abstract Builder<T> setRowMapper(RowMapper<T> rowMapper);
+
+    abstract Builder<T> setInitialNumReaders(ValueProvider<Integer> 
initialNumReaders);
+
+    abstract ReadWithPartitions<T> build();
+  }
+
+  public ReadWithPartitions<T> 
withDataSourceConfiguration(DataSourceConfiguration config) {
+    checkNotNull(config, "dataSourceConfiguration can not be null");
+    return toBuilder().setDataSourceConfiguration(config).build();
+  }
+
+  public ReadWithPartitions<T> withQuery(String query) {
+    checkNotNull(query, "query can not be null");
+    return withQuery(ValueProvider.StaticValueProvider.of(query));
+  }
+
+  public ReadWithPartitions<T> withQuery(ValueProvider<String> query) {
+    checkNotNull(query, "query can not be null");
+    return toBuilder().setQuery(query).build();
+  }
+
+  public ReadWithPartitions<T> withTable(String table) {
+    checkNotNull(table, "table can not be null");
+    return withTable(ValueProvider.StaticValueProvider.of(table));
+  }
+
+  public ReadWithPartitions<T> withTable(ValueProvider<String> table) {
+    checkNotNull(table, "table can not be null");
+    return toBuilder().setTable(table).build();
+  }
+
+  public ReadWithPartitions<T> withRowMapper(RowMapper<T> rowMapper) {
+    checkNotNull(rowMapper, "rowMapper can not be null");
+    return toBuilder().setRowMapper(rowMapper).build();
+  }
+
+  /** Pre-split initial restriction and start initialNumReaders reading at the 
very beginning. */
+  public ReadWithPartitions<T> withInitialNumReaders(Integer 
initialNumReaders) {
+    checkNotNull(initialNumReaders, "initialNumReaders can not be null");
+    return 
withInitialNumReaders(ValueProvider.StaticValueProvider.of(initialNumReaders));
+  }
+
+  /** Same as {@link #withInitialNumReaders(Integer)} but accepting a 
ValueProvider. */
+  public ReadWithPartitions<T> withInitialNumReaders(ValueProvider<Integer> 
initialNumReaders) {
+    checkNotNull(initialNumReaders, "initialNumReaders can not be null");
+    return toBuilder().setInitialNumReaders(initialNumReaders).build();
+  }
+
+  @Override
+  public PCollection<T> expand(PBegin input) {
+    DataSourceConfiguration dataSourceConfiguration =
+        Util.getRequiredArgument(
+            getDataSourceConfiguration(), "withDataSourceConfiguration() is 
required");
+    String database =
+        Util.getRequiredArgument(
+            dataSourceConfiguration.getDatabase(),
+            "withDatabase() is required for DataSourceConfiguration in order 
to perform readWithPartitions");
+    RowMapper<T> rowMapper =
+        Util.getRequiredArgument(getRowMapper(), "withRowMapper() is 
required");
+
+    int initialNumReaders = 
Util.getArgumentWithDefault(getInitialNumReaders(), 1);
+    checkArgument(
+        initialNumReaders >= 1, "withInitialNumReaders() should be greater or 
equal to 1");
+
+    String actualQuery = Util.getSelectQuery(getTable(), getQuery());
+
+    Coder<T> coder =
+        Util.inferCoder(
+            rowMapper,
+            input.getPipeline().getCoderRegistry(),
+            input.getPipeline().getSchemaRegistry(),
+            LOG);
+
+    return input
+        .apply(Create.of((Void) null))
+        .apply(
+            ParDo.of(
+                new ReadWithPartitions.ReadWithPartitionsFn<>(
+                    dataSourceConfiguration, actualQuery, database, rowMapper, 
initialNumReaders)))
+        .setCoder(coder);
+  }
+
+  private static class ReadWithPartitionsFn<ParameterT, OutputT> extends 
DoFn<ParameterT, OutputT> {
+    DataSourceConfiguration dataSourceConfiguration;
+    String query;
+    String database;
+    RowMapper<OutputT> rowMapper;
+    int initialNumReaders;
+
+    ReadWithPartitionsFn(
+        DataSourceConfiguration dataSourceConfiguration,
+        String query,
+        String database,
+        RowMapper<OutputT> rowMapper,
+        int initialNumReaders) {
+      this.dataSourceConfiguration = dataSourceConfiguration;
+      this.query = query;
+      this.database = database;
+      this.rowMapper = rowMapper;
+      this.initialNumReaders = initialNumReaders;
+    }
+
+    @ProcessElement
+    public void processElement(
+        ProcessContext context, RestrictionTracker<OffsetRange, Long> tracker) 
throws Exception {
+      DataSource dataSource = dataSourceConfiguration.getDataSource();
+      Connection conn = dataSource.getConnection();
+      try {
+        for (long partition = tracker.currentRestriction().getFrom();

Review Comment:
   either because it is outside of the tracker range, or because it is beyond 
the total partition count.



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/Write.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.io.singlestore;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import javax.sql.DataSource;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.commons.dbcp2.DelegatingStatement;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Instant;
+
+@AutoValue
+public abstract class Write<T> extends PTransform<PCollection<T>, PDone> {
+
+  private static final int DEFAULT_BATCH_SIZE = 100000;
+  private static final int BUFFER_SIZE = 524288;
+
+  abstract @Nullable DataSourceConfiguration getDataSourceConfiguration();
+
+  abstract @Nullable ValueProvider<String> getTable();
+
+  abstract @Nullable ValueProvider<Integer> getBatchSize();
+
+  abstract @Nullable UserDataMapper<T> getUserDataMapper();
+
+  abstract Builder<T> toBuilder();
+
+  @AutoValue.Builder
+  abstract static class Builder<T> {
+    abstract Builder<T> setDataSourceConfiguration(DataSourceConfiguration 
dataSourceConfiguration);
+
+    abstract Builder<T> setTable(ValueProvider<String> table);
+
+    abstract Builder<T> setBatchSize(ValueProvider<Integer> batchSize);
+
+    abstract Builder<T> setUserDataMapper(UserDataMapper<T> userDataMapper);
+
+    abstract Write<T> build();
+  }
+
+  public Write<T> withDataSourceConfiguration(DataSourceConfiguration config) {
+    checkNotNull(config, "dataSourceConfiguration can not be null");
+    return toBuilder().setDataSourceConfiguration(config).build();
+  }
+
+  public Write<T> withTable(String table) {
+    checkNotNull(table, "table can not be null");
+    return withTable(ValueProvider.StaticValueProvider.of(table));
+  }
+
+  public Write<T> withTable(ValueProvider<String> table) {
+    checkNotNull(table, "table can not be null");
+    return toBuilder().setTable(table).build();
+  }
+
+  public Write<T> withUserDataMapper(UserDataMapper<T> userDataMapper) {
+    checkNotNull(userDataMapper, "userDataMapper can not be null");
+    return toBuilder().setUserDataMapper(userDataMapper).build();
+  }
+
+  /**
+   * Provide a maximum number of rows that is written by one SQL statement. 
Default is 100000.
+   *
+   * @param batchSize maximum number of rows that is written by one SQL 
statement
+   */
+  public Write<T> withBatchSize(Integer batchSize) {
+    checkNotNull(batchSize, "batchSize can not be null");
+    return withBatchSize(ValueProvider.StaticValueProvider.of(batchSize));
+  }
+
+  /** Same as {@link #withBatchSize(Integer)} but accepting a ValueProvider. */
+  public Write<T> withBatchSize(ValueProvider<Integer> batchSize) {
+    checkNotNull(batchSize, "batchSize can not be null");
+    return toBuilder().setBatchSize(batchSize).build();
+  }
+
+  @Override
+  public PDone expand(PCollection<T> input) {
+    DataSourceConfiguration dataSourceConfiguration =
+        Util.getRequiredArgument(
+            getDataSourceConfiguration(), "withDataSourceConfiguration() is 
required");
+    String table = Util.getRequiredArgument(getTable(), "withTable() is 
required");
+    UserDataMapper<T> userDataMapper =
+        Util.getRequiredArgument(getUserDataMapper(), "withUserDataMapper() is 
required");
+    int batchSize = Util.getArgumentWithDefault(getBatchSize(), 
DEFAULT_BATCH_SIZE);
+    checkArgument(batchSize > 0, "batchSize should be greater then 0");
+
+    input
+        .apply(
+            ParDo.of(
+                new DoFn<T, List<String>>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext context) throws 
Exception {
+                    context.output(userDataMapper.mapRow(context.element()));
+                  }
+                }))
+        .setCoder(ListCoder.of(StringUtf8Coder.of()))
+        .apply(ParDo.of(new BatchFn<>(batchSize)))
+        .apply(ParDo.of(new WriteFn<Void>(dataSourceConfiguration, table)))
+        .setCoder(VoidCoder.of());
+
+    return PDone.in(input.getPipeline());
+  }
+
+  private static class BatchFn<ParameterT> extends DoFn<ParameterT, 
Iterable<ParameterT>> {
+    List<ParameterT> batch = new ArrayList<>();
+    int batchSize;
+
+    BatchFn(int batchSize) {
+      this.batchSize = batchSize;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext context) {
+      batch.add(context.element());
+      if (batch.size() >= batchSize) {
+        context.output(batch);
+        batch = new ArrayList<>();
+      }
+    }
+
+    @FinishBundle
+    public void finish(FinishBundleContext context) {
+      if (batch.size() > 0) {
+        context.output(batch, Instant.now(), GlobalWindow.INSTANCE);
+        batch = new ArrayList<>();
+      }
+    }
+  }
+
+  private static class WriteFn<OutputT> extends DoFn<Iterable<List<String>>, 
OutputT> {
+    DataSourceConfiguration dataSourceConfiguration;
+    String table;
+
+    WriteFn(DataSourceConfiguration dataSourceConfiguration, String table) {
+      this.dataSourceConfiguration = dataSourceConfiguration;
+      this.table = table;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext context) throws Exception {
+      DataSource dataSource = dataSourceConfiguration.getDataSource();
+
+      Connection conn = dataSource.getConnection();
+      try {
+        Statement stmt = conn.createStatement();
+        try (PipedOutputStream baseStream = new PipedOutputStream();
+            InputStream inputStream = new PipedInputStream(baseStream, 
BUFFER_SIZE)) {
+          ((com.singlestore.jdbc.Statement) ((DelegatingStatement) 
stmt).getInnermostDelegate())
+              .setNextLocalInfileInputStream(inputStream);
+
+          final Exception[] writeException = new Exception[1];
+
+          Thread dataWritingThread =

Review Comment:
   Avoid creating threads inside of dofns. In Beam, asynchronous and 
multithreading behavior is best handled by the Beam framework itself. 



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