AdalbertMemSQL commented on code in PR #23535: URL: https://github.com/apache/beam/pull/23535#discussion_r1017947911
########## 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: The same loop is presented in this blog https://beam.apache.org/blog/splittable-do-fn/ ``` public void process(ProcessContext c, OffsetRangeTracker tracker) { for (long i = tracker.currentRestriction().getFrom(); tracker.tryClaim(i); ++i) { c.output(KV.of(c.element().getKey(), i)); } } ``` I thought that `tryClaim` will just return `false` when an invalid partition is provided. -- 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]
