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

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

                Author: ASF GitHub Bot
            Created on: 22/Feb/20 15:07
            Start Date: 22/Feb/20 15:07
    Worklog Time Spent: 10m 
      Work Description: iemejia commented on pull request #10604: BEAM-2546 
Beam IO for InfluxDB
URL: https://github.com/apache/beam/pull/10604#discussion_r382917044
 
 

 ##########
 File path: 
sdks/java/io/influxdb/src/main/java/org/apache/beam/sdk/io/influxdb/InfluxDBIO.java
 ##########
 @@ -0,0 +1,624 @@
+package org.apache.beam.sdk.io.influxdb;
+
+import com.google.auto.value.AutoValue;
+import okhttp3.OkHttpClient;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.HasDisplayData;
+import org.apache.beam.sdk.values.*;
+import org.influxdb.BatchOptions;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.*;
+import org.influxdb.dto.Query;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.net.ssl.*;
+import java.io.Serializable;
+import java.security.cert.CertificateException;
+import java.util.*;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+/**
+ *  IO to read and write to InfluxDB.
+  <h3>Reading from InfluxDB datasource</h3>
+        *
+        * <p>InfluxDBIO source returns a bounded collection of {@code String} 
as a {@code PCollection<String>}.
+        *
+        * <p>To configure the InfluxDB source, you have to provide a {@link 
DataSourceConfiguration} using<br>
+        * {@link DataSourceConfiguration#create(String, String, String)}(durl, 
username and password).
+        * Optionally, {@link DataSourceConfiguration#withUsername(String)} and 
{@link
+        * DataSourceConfiguration#withPassword(String)} allows you to define 
username and password.
+         *
+         * <p>For example:
+        *
+        * <pre>{@code
+        * PCollection<Stringn> collection = pipeline.apply(InfluxDBIO.read()
+        *   .withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(
+        *          "https://localhost:8086","username","password";))
+        *   .withDatabase("metrics")
+        *   .withRetentionPolicy("autogen")
+        *   .withSslInvalidHostNameAllowed(true)
+        *   withSslEnabled(true));
+        * }</pre>
+        *
+        * <p> For example (Read from query):
+        * <pre>{@code
+        * PCollection<Stringn> collection = pipeline.apply(InfluxDBIO.read()
+        *   .withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(
+        *          "https://localhost:8086","username","password";))
+        *   .withDatabase("metrics")
+        *   .withQuery("Select * from cpu")
+        *   .withRetentionPolicy("autogen")
+        *   .withSslInvalidHostNameAllowed(true)
+        *   withSslEnabled(true));
+        * }</pre>
+        * <h3>Writing to Influx datasource</h3>
+        *
+        * <p>InfluxDB sink supports writing records into a database. It writes 
a {@link PCollection} to the
+        * database by converting each T. The T should implement  
getLineProtocol() from {@link LineProtocolConvertable}.
+        *
+        * <p>Like the source, to configure the sink, you have to provide a 
{@link DataSourceConfiguration}.
+        *
+        * <pre>{@code
+        * pipeline
+        *   .apply(...)
+        *   .apply(InfluxDb.write()
+        *      
.withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(
+        *            "https://localhost:8086","username","password";))
+        *   .withRetentionPolicy("autogen")
+        *   .withDatabase("metrics")
+        *   .withSslInvalidHostNameAllowed(true)
+        *   withSslEnabled(true));
+        *    );
+        * }</pre>
+ * *
+ */
+@Experimental(Experimental.Kind.SOURCE_SINK)
+public class InfluxDBIO {
+    private static final Logger LOG = 
LoggerFactory.getLogger(InfluxDBIO.class);
+
+    public static Write write() {
+        return new AutoValue_InfluxDBIO_Write.Builder()
+                .build();
+    }
+    public static Read read() {
+        return new AutoValue_InfluxDBIO_Read.Builder()
+                .build();
+    }
+
+    @AutoValue
+    public abstract static class Read extends PTransform<PBegin, 
PCollection<String>> {
+        @Nullable
+        abstract Boolean sslInvalidHostNameAllowed();
+        @Nullable
+        abstract String retentionPolicy();
+        @Nullable
+        abstract String database();
+        @Nullable
+        abstract String query();
+        @Nullable
+        abstract Boolean sslEnabled();
+        @Nullable
+        abstract DataSourceConfiguration dataSourceConfiguration();
+        @Nullable
+        abstract List<String> metric();
+        abstract Builder builder();
+
+        @AutoValue.Builder
+        abstract static class Builder {
+            abstract Builder 
setDataSourceConfiguration(DataSourceConfiguration configuration);
+            abstract Builder setDatabase(String database);
+            abstract Builder setSslInvalidHostNameAllowed(Boolean value);
+            abstract Builder setRetentionPolicy(String retentionPolicy);
+            abstract Builder setQuery(String query);
+            abstract Builder setSslEnabled(Boolean sslEnabled);
+            abstract Builder setMetric(List<String> metric);
+
+            abstract Read build();
+        }
+
+        /** Reads from the InfluxDB instance indicated by the given 
configuration. */
+        public Read withDataSourceConfiguration(DataSourceConfiguration 
configuration) {
+            checkArgument(configuration != null, "configuration can not be 
null");
+            return builder().setDataSourceConfiguration(configuration).build();
+        }
+
+        /** Reads from the specified database. */
+        public Read withDatabase(String database) {
+            return 
builder().setDatabase(database).setDataSourceConfiguration(dataSourceConfiguration()).build();
+        }
+        /** Reads from the specified query. */
+        public Read withQuery(String query) {
+            return builder().setQuery(query).build();
+        }
+
+        public Read withMetric(List<String> metric){
 
 Review comment:
   Maybe varargs will make a simpler API here (String... metric)
 
----------------------------------------------------------------
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: 391140)
    Time Spent: 4.5h  (was: 4h 20m)

> Create InfluxDbIO
> -----------------
>
>                 Key: BEAM-2546
>                 URL: https://issues.apache.org/jira/browse/BEAM-2546
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-ideas
>            Reporter: Jean-Baptiste Onofré
>            Assignee: Jean-Baptiste Onofré
>            Priority: Major
>          Time Spent: 4.5h
>  Remaining Estimate: 0h
>




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

Reply via email to