You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2020/04/09 21:38:00 UTC

[jira] [Work logged] (BEAM-9722) Add SnowflakeIO to Java SDK

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

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

                Author: ASF GitHub Bot
            Created on: 09/Apr/20 21:37
            Start Date: 09/Apr/20 21:37
    Worklog Time Spent: 10m 
      Work Description: takidau commented on pull request #11360: [WIP][BEAM-9722] added SnowflakeIO with Read operation
URL: https://github.com/apache/beam/pull/11360#discussion_r406491845
 
 

 ##########
 File path: sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java
 ##########
 @@ -0,0 +1,691 @@
+/*
+ * 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.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+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.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.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.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 Google Cloud
+ * Storage.
+ *
+ * <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#withPortNumber(int)} 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);
 
 Review comment:
   options.getSchema()
 
----------------------------------------------------------------
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:
users@infra.apache.org


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

            Worklog Id:     (was: 419800)
    Remaining Estimate: 0h
            Time Spent: 10m

> Add SnowflakeIO 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: Kasia Kucharczyk
>            Priority: Major
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> Creating java [Snowflake|https://docs.snowflake.com/en/] connector which includes:
> * batch processing connector provided by COPY operation from [SnowflakeJDBC|https://github.com/snowflakedb/snowflake-jdbc] which will load/unload files staged on GCP,
> * streaming processing thanks to [SnowPipe|https://docs.snowflake.com/en/user-guide/data-load-snowpipe.html] mechanism and [Snowflake Ingest Java library|https://github.com/snowflakedb/snowflake-ingest-java],
> * Snowflake authentication mechanism.



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