[ 
https://issues.apache.org/jira/browse/BEAM-9722?focusedWorklogId=435617&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-435617
 ]

ASF GitHub Bot logged work on BEAM-9722:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 20/May/20 19:04
            Start Date: 20/May/20 19:04
    Worklog Time Spent: 10m 
      Work Description: DariuszAniszewski commented on a change in pull request 
#11360:
URL: https://github.com/apache/beam/pull/11360#discussion_r428244242



##########
File path: 
sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java
##########
@@ -0,0 +1,735 @@
+/*
+ * 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.snowflake;
+
+import static org.apache.beam.sdk.io.TextIO.readFiles;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import com.opencsv.CSVParser;
+import com.opencsv.CSVParserBuilder;
+import java.io.IOException;
+import java.io.Serializable;
+import java.security.PrivateKey;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import javax.sql.DataSource;
+import net.snowflake.client.jdbc.SnowflakeBasicDataSource;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import 
org.apache.beam.sdk.io.snowflake.credentials.KeyPairSnowflakeCredentials;
+import 
org.apache.beam.sdk.io.snowflake.credentials.OAuthTokenSnowflakeCredentials;
+import org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentials;
+import 
org.apache.beam.sdk.io.snowflake.credentials.UsernamePasswordSnowflakeCredentials;
+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.SerializableFunction;
+import org.apache.beam.sdk.transforms.Wait;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.HasDisplayData;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * IO to read and write data on Snowflake.
+ *
+ * <p>SnowflakeIO uses <a 
href="https://docs.snowflake.net/manuals/user-guide/jdbc.html";>Snowflake
+ * JDBC</a> driver under the hood, but data isn't read/written using JDBC 
directly. Instead,
+ * SnowflakeIO uses dedicated <b>COPY</b> operations to read/write data 
from/to a cloud bucket. By
+ * now only Google Cloud Storage is supported.
+ *
+ * <p>To configure SnowflakeIO to read/write from your Snowflake instance, you 
have to provide a
+ * {@link DataSourceConfiguration} using {@link
+ * DataSourceConfiguration#create(SnowflakeCredentials)}, where {@link 
SnowflakeCredentials might be
+ * created using {@link 
org.apache.beam.sdk.io.snowflake.credentials.SnowflakeCredentialsFactory}}.
+ * Additionally one of {@link DataSourceConfiguration#withServerName(String)} 
or {@link
+ * DataSourceConfiguration#withUrl(String)} must be used to tell SnowflakeIO 
which instance to use.
+ * <br>
+ * There are also other options available to configure connection to Snowflake:
+ *
+ * <ul>
+ *   <li>{@link DataSourceConfiguration#withWarehouse(String)} to specify 
which Warehouse to use
+ *   <li>{@link DataSourceConfiguration#withDatabase(String)} to specify which 
Database to connect
+ *       to
+ *   <li>{@link DataSourceConfiguration#withSchema(String)} to specify which 
schema to use
+ *   <li>{@link DataSourceConfiguration#withRole(String)} to specify which 
role to use
+ *   <li>{@link DataSourceConfiguration#withLoginTimeout(Integer)} to specify 
the timeout for the
+ *       login
+ *   <li>{@link DataSourceConfiguration#withPortNumber(Integer)} to specify 
custom port of Snowflake
+ *       instance
+ * </ul>
+ *
+ * <p>For example:
+ *
+ * <pre>{@code
+ * SnowflakeIO.DataSourceConfiguration dataSourceConfiguration =
+ *     
SnowflakeIO.DataSourceConfiguration.create(SnowflakeCredentialsFactory.of(options))
+ *         .withServerName(options.getServerName())
+ *         .withWarehouse(options.getWarehouse())
+ *         .withDatabase(options.getDatabase())
+ *         .withSchema(options.getSchema());
+ * }</pre>
+ *
+ * <h3>Reading from Snowflake</h3>
+ *
+ * <p>SnowflakeIO.Read returns a bounded collection of {@code T} as a {@code 
PCollection<T>}. T is
+ * the type returned by the provided {@link CsvMapper}.
+ *
+ * <p>For example
+ *
+ * <pre>{@code
+ * PCollection<GenericRecord> items = pipeline.apply(
+ *  SnowflakeIO.<GenericRecord>read()
+ *    .withDataSourceConfiguration(dataSourceConfiguration)
+ *    .fromQuery(QUERY)
+ *    .withStagingBucketName(stagingBucketName)
+ *    .withIntegrationName(integrationName)
+ *    .withCsvMapper(...)
+ *    .withCoder(...));
+ * }</pre>
+ *
+ * <p><b>Important</b> When reading data from Snowflake, temporary CSV files 
are created on the
+ * specified stagingBucketName in directory named `sf_copy_csv_[RANDOM 
CHARS]_[TIMESTAMP]`. This
+ * directory and all the files are deleted automatically by default, but in 
case of failed pipeline
+ * they will remain and have to be removed manually.
+ */
+public class SnowflakeIO {
+  private static final Logger LOG = LoggerFactory.getLogger(SnowflakeIO.class);
+
+  private static final String CSV_QUOTE_CHAR = "'";
+  /**
+   * Read data from Snowflake.
+   *
+   * @param snowflakeService user-defined {@link SnowflakeService}
+   * @param <T> Type of the data to be read.
+   */
+  public static <T> Read<T> read(SnowflakeService snowflakeService) {
+    return new AutoValue_SnowflakeIO_Read.Builder<T>()
+        .setSnowflakeService(snowflakeService)
+        .build();
+  }
+
+  /**
+   * Read data from Snowflake.
+   *
+   * @param <T> Type of the data to be read.
+   */
+  public static <T> Read<T> read() {
+    return read(new SnowflakeServiceImpl());
+  }
+
+  /**
+   * Interface for user-defined function mapping parts of CSV line into T. 
Used for
+   * SnowflakeIO.Read.
+   *
+   * @param <T> Type of data to be read.
+   */
+  @FunctionalInterface
+  public interface CsvMapper<T> extends Serializable {
+    T mapRow(String[] parts) throws Exception;
+  }
+
+  /** Implementation of {@link #read()}. */
+  @AutoValue
+  public abstract static class Read<T> extends PTransform<PBegin, 
PCollection<T>> {
+    @Nullable
+    abstract SerializableFunction<Void, DataSource> getDataSourceProviderFn();
+
+    @Nullable
+    abstract String getQuery();
+
+    @Nullable
+    abstract String getTable();
+
+    @Nullable
+    abstract String getIntegrationName();
+
+    @Nullable
+    abstract String getStagingBucketName();
+
+    @Nullable
+    abstract CsvMapper<T> getCsvMapper();
+
+    @Nullable
+    abstract Coder<T> getCoder();
+
+    @Nullable
+    abstract SnowflakeService getSnowflakeService();
+
+    abstract Builder<T> toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder<T> {
+      abstract Builder<T> setDataSourceProviderFn(
+          SerializableFunction<Void, DataSource> dataSourceProviderFn);
+
+      abstract Builder<T> setQuery(String query);
+
+      abstract Builder<T> setTable(String table);
+
+      abstract Builder<T> setIntegrationName(String integrationName);
+
+      abstract Builder<T> setStagingBucketName(String stagingBucketName);
+
+      abstract Builder<T> setCsvMapper(CsvMapper<T> csvMapper);
+
+      abstract Builder<T> setCoder(Coder<T> coder);
+
+      abstract Builder<T> setSnowflakeService(SnowflakeService 
snowflakeService);
+
+      abstract Read<T> build();
+    }
+
+    /**
+     * Setting information about Snowflake server.
+     *
+     * @param config - An instance of {@link DataSourceConfiguration}.
+     */
+    public Read<T> withDataSourceConfiguration(final DataSourceConfiguration 
config) {
+
+      try {
+        Connection connection = config.buildDatasource().getConnection();
+        connection.close();
+      } catch (SQLException e) {
+        throw new IllegalArgumentException(
+            "Invalid DataSourceConfiguration. Underlying cause: " + e);
+      }
+      return withDataSourceProviderFn(new 
DataSourceProviderFromDataSourceConfiguration(config));
+    }
+
+    /**
+     * Setting function that will provide {@link DataSourceConfiguration} in 
runtime.
+     *
+     * @param dataSourceProviderFn a {@link SerializableFunction}.
+     */
+    public Read<T> withDataSourceProviderFn(
+        SerializableFunction<Void, DataSource> dataSourceProviderFn) {
+      return toBuilder().setDataSourceProviderFn(dataSourceProviderFn).build();
+    }
+
+    /**
+     * A query to be executed in Snowflake.
+     *
+     * @param query - String with query.
+     */
+    public Read<T> fromQuery(String query) {
+      return toBuilder().setQuery(query).build();
+    }
+
+    /**
+     * A table name to be read in Snowflake.
+     *
+     * @param table - String with the name of the table.
+     */
+    public Read<T> fromTable(String table) {
+      return toBuilder().setTable(table).build();
+    }
+
+    /**
+     * Name of the cloud bucket (GCS by now) to use as tmp location of CSVs 
during COPY statement.
+     *
+     * @param stagingBucketName - String with the name of the bucket.
+     */
+    public Read<T> withStagingBucketName(String stagingBucketName) {
+      return toBuilder().setStagingBucketName(stagingBucketName).build();
+    }
+
+    /**
+     * Name of the Storage Integration in Snowflake to be used. See
+     * 
https://docs.snowflake.com/en/sql-reference/sql/create-storage-integration.html 
for
+     * reference.
+     *
+     * @param integrationName - String with the name of the Storage 
Integration.
+     */
+    public Read<T> withIntegrationName(String integrationName) {
+      return toBuilder().setIntegrationName(integrationName).build();
+    }
+
+    /**
+     * User-defined function mapping CSV lines into user data.
+     *
+     * @param csvMapper - an instance of {@link CsvMapper}.
+     */
+    public Read<T> withCsvMapper(CsvMapper<T> csvMapper) {
+      return toBuilder().setCsvMapper(csvMapper).build();
+    }
+
+    /**
+     * A Coder to be used by the output PCollection generated by the source.
+     *
+     * @param coder - an instance of {@link Coder}.
+     */
+    public Read<T> withCoder(Coder<T> coder) {
+      return toBuilder().setCoder(coder).build();
+    }
+
+    @Override
+    public PCollection<T> expand(PBegin input) {
+      // Either table or query is required. If query is present, it's being 
used, table is used
+      // otherwise
+      checkArgument(
+          getQuery() != null || getTable() != null, "fromTable() or 
fromQuery() is required");
+      checkArgument(
+          !(getQuery() != null && getTable() != null),
+          "fromTable() and fromQuery() are not allowed together");
+      checkArgument(getCsvMapper() != null, "withCsvMapper() is required");
+      checkArgument(getCoder() != null, "withCoder() is required");
+      checkArgument(getIntegrationName() != null, "withIntegrationName() is 
required");
+      checkArgument(getStagingBucketName() != null, "withStagingBucketName() 
is required");
+      checkArgument(
+          (getDataSourceProviderFn() != null),
+          "withDataSourceConfiguration() or withDataSourceProviderFn() is 
required");
+
+      String gcpTmpDirName = makeTmpDirName();
+      String stagingBucketDir = String.format("%s/%s/", 
getStagingBucketName(), gcpTmpDirName);
+
+      PCollection<Void> emptyCollection = input.apply(Create.of((Void) null));
+
+      PCollection<T> output =
+          emptyCollection
+              .apply(
+                  ParDo.of(
+                      new CopyIntoStageFn(

Review comment:
       Thanks @chamikaramj for detailed explanation. I've added `Reshuffle` 
step in [latest 
commit](https://github.com/apache/beam/pull/11360/commits/3ba192a57217a9b251f71f77771b68ceb2882b02)




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

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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 435617)
    Time Spent: 5h 40m  (was: 5.5h)

> Add batch SnowflakeIO.Read to Java SDK
> --------------------------------------
>
>                 Key: BEAM-9722
>                 URL: https://issues.apache.org/jira/browse/BEAM-9722
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-ideas
>            Reporter: Kasia Kucharczyk
>            Assignee: Dariusz Aniszewski
>            Priority: P2
>          Time Spent: 5h 40m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to