You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/11/29 21:26:12 UTC

[GitHub] [beam] ahmedabu98 commented on a diff in pull request #24290: Implemented SchemaTransforms for SingleStoreIO

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


##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.schematransform;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.io.singlestore.SingleStoreIO;
+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.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB read jobs configured
+ * using {@link SingleStoreSchemaTransformReadConfiguration}.
+ */
+public class SingleStoreSchemaTransformReadProvider
+    extends TypedSchemaTransformProvider<SingleStoreSchemaTransformReadConfiguration> {
+
+  private static final String API = "singlestore";
+  private static final String OUTPUT_TAG = "OUTPUT";
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<SingleStoreSchemaTransformReadConfiguration> configurationClass() {
+    return SingleStoreSchemaTransformReadConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(SingleStoreSchemaTransformReadConfiguration configuration) {
+    return new SingleStoreReadSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */
+  @Override
+  public String identifier() {
+    return String.format("%s:read", API);
+  }
+
+  /**
+   * Implementation of the {@link TypedSchemaTransformProvider} inputCollectionNames method. Since
+   * no input is expected, this returns an empty list.
+   */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Collections.emptyList();
+  }
+
+  /**
+   * Implementation of the {@link TypedSchemaTransformProvider} outputCollectionNames method. Since
+   * a single output is expected, this returns a list with a single name.
+   */
+  @Override
+  public List<String> outputCollectionNames() {
+    return Collections.singletonList(OUTPUT_TAG);
+  }
+
+  /**
+   * An implementation of {@link SchemaTransform} for SingleStoreDB read jobs configured using
+   * {@link SingleStoreSchemaTransformReadConfiguration}.
+   */
+  private static class SingleStoreReadSchemaTransform implements SchemaTransform {
+    private final SingleStoreSchemaTransformReadConfiguration configuration;
+
+    SingleStoreReadSchemaTransform(SingleStoreSchemaTransformReadConfiguration configuration) {
+      this.configuration = configuration;
+    }
+
+    /** Implements {@link SchemaTransform} buildTransform method. */
+    @Override
+    public PTransform<PCollectionRowTuple, PCollectionRowTuple> buildTransform() {
+      return new PCollectionRowTupleTransform(configuration);
+    }
+  }
+
+  /**
+   * An implementation of {@link PTransform} for SingleStoreDB read jobs configured using {@link
+   * SingleStoreSchemaTransformReadConfiguration}.
+   */
+  static class PCollectionRowTupleTransform
+      extends PTransform<PCollectionRowTuple, PCollectionRowTuple> {
+
+    private final SingleStoreSchemaTransformReadConfiguration configuration;
+
+    PCollectionRowTupleTransform(SingleStoreSchemaTransformReadConfiguration configuration) {
+      this.configuration = configuration;
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      if (!input.getAll().isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format(
+                "%s %s input is expected to be empty",
+                input.getClass().getSimpleName(), getClass().getSimpleName()));
+      }
+      SingleStoreIO.DataSourceConfiguration dataSourceConfiguration =
+          configuration.getDataSourceConfiguration();
+      String table = configuration.getTable();
+      String query = configuration.getQuery();
+      Boolean outputParallelization = configuration.getOutputParallelization();
+
+      SingleStoreIO.ReadRows read = SingleStoreIO.readRows();
+
+      if (dataSourceConfiguration != null) {
+        read = read.withDataSourceConfiguration(dataSourceConfiguration);
+      }
+
+      if (table != null) {

Review Comment:
   ```suggestion
         if (!Strings.isNullOrEmpty(table)) {
   ```



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.schematransform;
+
+import com.google.auto.value.AutoValue;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.singlestore.SingleStoreIO;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * Configuration for reading from SignleStoreDB.
+ *
+ * <p>This class is meant to be used with {@link SingleStoreSchemaTransformReadProvider}.
+ */
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class SingleStoreSchemaTransformReadConfiguration {
+
+  /** Instantiates a {@link SingleStoreSchemaTransformReadConfiguration.Builder}. */
+  public static Builder builder() {
+    return new AutoValue_SingleStoreSchemaTransformReadConfiguration.Builder();
+  }
+
+  private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema();
+  private static final TypeDescriptor<SingleStoreSchemaTransformReadConfiguration> TYPE_DESCRIPTOR =
+      TypeDescriptor.of(SingleStoreSchemaTransformReadConfiguration.class);
+  private static final SerializableFunction<SingleStoreSchemaTransformReadConfiguration, Row>
+      ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR);
+
+  /** Serializes configuration to a {@link Row}. */
+  public Row toBeamRow() {
+    return ROW_SERIALIZABLE_FUNCTION.apply(this);
+  }
+
+  @Nullable
+  public abstract SingleStoreIO.DataSourceConfiguration getDataSourceConfiguration();
+
+  @Nullable
+  public abstract String getQuery();
+
+  @Nullable
+  public abstract String getTable();
+
+  @Nullable
+  public abstract Boolean getOutputParallelization();
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+
+    public abstract Builder setDataSourceConfiguration(SingleStoreIO.DataSourceConfiguration value);

Review Comment:
   When a remote SDK tries to prepare a configuration Row object to use this IO, how would it set the dataSourceConfiguration? The `DataSourceConfiguration` POJO only exists in the Java SDK



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsProvider.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.schematransform;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.io.singlestore.SingleStoreIO;
+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.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB parallel read jobs
+ * configured using {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration}.
+ */
+public class SingleStoreSchemaTransformReadWithPartitionsProvider

Review Comment:
   There's a lot of overlap between this and the `SingleStoreSchemaTransformReadProvider` and configuration classes. I think the only difference is two configuration parameters (this one uses the initialNumReaders parameter and the other uses the outputParallelization parameter).
   
   Would it make sense to combine these two sets of classes into one that includes both parameters? You can add a new `readWithPartitions` boolean parameter that would distinguish between the two read modes. 



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.schematransform;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.io.singlestore.SingleStoreIO;
+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.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB read jobs configured
+ * using {@link SingleStoreSchemaTransformReadConfiguration}.
+ */
+public class SingleStoreSchemaTransformReadProvider
+    extends TypedSchemaTransformProvider<SingleStoreSchemaTransformReadConfiguration> {
+
+  private static final String API = "singlestore";
+  private static final String OUTPUT_TAG = "OUTPUT";
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<SingleStoreSchemaTransformReadConfiguration> configurationClass() {
+    return SingleStoreSchemaTransformReadConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(SingleStoreSchemaTransformReadConfiguration configuration) {
+    return new SingleStoreReadSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */
+  @Override
+  public String identifier() {
+    return String.format("%s:read", API);
+  }
+
+  /**
+   * Implementation of the {@link TypedSchemaTransformProvider} inputCollectionNames method. Since
+   * no input is expected, this returns an empty list.
+   */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Collections.emptyList();
+  }
+
+  /**
+   * Implementation of the {@link TypedSchemaTransformProvider} outputCollectionNames method. Since
+   * a single output is expected, this returns a list with a single name.
+   */
+  @Override
+  public List<String> outputCollectionNames() {
+    return Collections.singletonList(OUTPUT_TAG);
+  }
+
+  /**
+   * An implementation of {@link SchemaTransform} for SingleStoreDB read jobs configured using
+   * {@link SingleStoreSchemaTransformReadConfiguration}.
+   */
+  private static class SingleStoreReadSchemaTransform implements SchemaTransform {
+    private final SingleStoreSchemaTransformReadConfiguration configuration;
+
+    SingleStoreReadSchemaTransform(SingleStoreSchemaTransformReadConfiguration configuration) {
+      this.configuration = configuration;
+    }
+
+    /** Implements {@link SchemaTransform} buildTransform method. */
+    @Override
+    public PTransform<PCollectionRowTuple, PCollectionRowTuple> buildTransform() {
+      return new PCollectionRowTupleTransform(configuration);
+    }
+  }
+
+  /**
+   * An implementation of {@link PTransform} for SingleStoreDB read jobs configured using {@link
+   * SingleStoreSchemaTransformReadConfiguration}.
+   */
+  static class PCollectionRowTupleTransform
+      extends PTransform<PCollectionRowTuple, PCollectionRowTuple> {
+
+    private final SingleStoreSchemaTransformReadConfiguration configuration;
+
+    PCollectionRowTupleTransform(SingleStoreSchemaTransformReadConfiguration configuration) {
+      this.configuration = configuration;
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      if (!input.getAll().isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format(
+                "%s %s input is expected to be empty",
+                input.getClass().getSimpleName(), getClass().getSimpleName()));
+      }
+      SingleStoreIO.DataSourceConfiguration dataSourceConfiguration =
+          configuration.getDataSourceConfiguration();
+      String table = configuration.getTable();
+      String query = configuration.getQuery();
+      Boolean outputParallelization = configuration.getOutputParallelization();
+
+      SingleStoreIO.ReadRows read = SingleStoreIO.readRows();
+
+      if (dataSourceConfiguration != null) {
+        read = read.withDataSourceConfiguration(dataSourceConfiguration);
+      }
+
+      if (table != null) {

Review Comment:
   Strings.isNullOrEmpty to catches the `""` edge case. Consider using it for these checks in the other SchemaTransformProvider classes



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.schematransform;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.io.singlestore.SingleStoreIO;
+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.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB write jobs configured
+ * using {@link SingleStoreSchemaTransformWriteConfiguration}.
+ */
+public class SingleStoreSchemaTransformWriteProvider
+    extends TypedSchemaTransformProvider<SingleStoreSchemaTransformWriteConfiguration> {
+
+  private static final String API = "singlestore";
+  private static final String OUTPUT_TAG = "OUTPUT";
+  public static final String INPUT_TAG = "INPUT";
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<SingleStoreSchemaTransformWriteConfiguration> configurationClass() {
+    return SingleStoreSchemaTransformWriteConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(SingleStoreSchemaTransformWriteConfiguration configuration) {
+    return new SingleStoreWriteSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */
+  @Override
+  public String identifier() {
+    return String.format("%s:write", API);

Review Comment:
   See previous comment for URN conventions



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsProvider.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.schematransform;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.io.singlestore.SingleStoreIO;
+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.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB parallel read jobs
+ * configured using {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration}.
+ */
+public class SingleStoreSchemaTransformReadWithPartitionsProvider

Review Comment:
   Some thought should be put into this decision. Merging the two read modes would make sense as it is now, but if it's likely that these two modes will develop down the line to have many more differences then keeping them separate makes more sense.



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.schematransform;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.io.singlestore.SingleStoreIO;
+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.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB read jobs configured
+ * using {@link SingleStoreSchemaTransformReadConfiguration}.
+ */
+public class SingleStoreSchemaTransformReadProvider
+    extends TypedSchemaTransformProvider<SingleStoreSchemaTransformReadConfiguration> {
+
+  private static final String API = "singlestore";
+  private static final String OUTPUT_TAG = "OUTPUT";
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<SingleStoreSchemaTransformReadConfiguration> configurationClass() {
+    return SingleStoreSchemaTransformReadConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(SingleStoreSchemaTransformReadConfiguration configuration) {
+    return new SingleStoreReadSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */
+  @Override
+  public String identifier() {
+    return String.format("%s:read", API);
+  }

Review Comment:
   It's encouraged to adhere to the URN conventions here: https://beam.apache.org/documentation/programming-guide/#1314-defining-a-urn
   
   for example, something like `beam:schematransform:org.apache.beam:singlestore_read:v1`



##########
sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.schematransform;
+
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.io.singlestore.SingleStoreIO;
+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.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB read jobs configured
+ * using {@link SingleStoreSchemaTransformReadConfiguration}.
+ */
+public class SingleStoreSchemaTransformReadProvider
+    extends TypedSchemaTransformProvider<SingleStoreSchemaTransformReadConfiguration> {
+
+  private static final String API = "singlestore";
+  private static final String OUTPUT_TAG = "OUTPUT";
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<SingleStoreSchemaTransformReadConfiguration> configurationClass() {
+    return SingleStoreSchemaTransformReadConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(SingleStoreSchemaTransformReadConfiguration configuration) {
+    return new SingleStoreReadSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */
+  @Override
+  public String identifier() {
+    return String.format("%s:read", API);
+  }
+
+  /**
+   * Implementation of the {@link TypedSchemaTransformProvider} inputCollectionNames method. Since
+   * no input is expected, this returns an empty list.
+   */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Collections.emptyList();
+  }
+
+  /**
+   * Implementation of the {@link TypedSchemaTransformProvider} outputCollectionNames method. Since
+   * a single output is expected, this returns a list with a single name.
+   */
+  @Override
+  public List<String> outputCollectionNames() {
+    return Collections.singletonList(OUTPUT_TAG);
+  }
+
+  /**
+   * An implementation of {@link SchemaTransform} for SingleStoreDB read jobs configured using
+   * {@link SingleStoreSchemaTransformReadConfiguration}.
+   */
+  private static class SingleStoreReadSchemaTransform implements SchemaTransform {
+    private final SingleStoreSchemaTransformReadConfiguration configuration;
+
+    SingleStoreReadSchemaTransform(SingleStoreSchemaTransformReadConfiguration configuration) {
+      this.configuration = configuration;
+    }
+
+    /** Implements {@link SchemaTransform} buildTransform method. */
+    @Override
+    public PTransform<PCollectionRowTuple, PCollectionRowTuple> buildTransform() {
+      return new PCollectionRowTupleTransform(configuration);
+    }
+  }
+
+  /**
+   * An implementation of {@link PTransform} for SingleStoreDB read jobs configured using {@link
+   * SingleStoreSchemaTransformReadConfiguration}.
+   */
+  static class PCollectionRowTupleTransform
+      extends PTransform<PCollectionRowTuple, PCollectionRowTuple> {
+
+    private final SingleStoreSchemaTransformReadConfiguration configuration;
+
+    PCollectionRowTupleTransform(SingleStoreSchemaTransformReadConfiguration configuration) {
+      this.configuration = configuration;
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      if (!input.getAll().isEmpty()) {
+        throw new IllegalArgumentException(
+            String.format(
+                "%s %s input is expected to be empty",
+                input.getClass().getSimpleName(), getClass().getSimpleName()));
+      }
+      SingleStoreIO.DataSourceConfiguration dataSourceConfiguration =
+          configuration.getDataSourceConfiguration();
+      String table = configuration.getTable();
+      String query = configuration.getQuery();
+      Boolean outputParallelization = configuration.getOutputParallelization();
+
+      SingleStoreIO.ReadRows read = SingleStoreIO.readRows();
+
+      if (dataSourceConfiguration != null) {
+        read = read.withDataSourceConfiguration(dataSourceConfiguration);
+      }
+
+      if (table != null) {
+        read = read.withTable(table);
+      }
+
+      if (query != null) {

Review Comment:
   ```suggestion
         if (!Strings.isNullOrEmpty(query)) {
   ```



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org