[ 
https://issues.apache.org/jira/browse/FLINK-3967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15299619#comment-15299619
 ] 

ASF GitHub Bot commented on FLINK-3967:
---------------------------------------

Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2031#discussion_r64527910
  
    --- Diff: 
flink-streaming-connectors/flink-connector-rethinkdb/src/main/java/org/apache/flink/streaming/connectors/rethinkdb/FlinkRethinkDbSink.java
 ---
    @@ -0,0 +1,289 @@
    +/*
    + * 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.flink.streaming.connectors.rethinkdb;
    +
    +import java.io.Serializable;
    +import java.util.HashMap;
    +import java.util.Objects;
    +
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.rethinkdb.RethinkDB;
    +import com.rethinkdb.gen.ast.Insert;
    +import com.rethinkdb.gen.ast.Table;
    +import com.rethinkdb.net.Connection;
    +
    +/**
    + * This class is the Flink sink for RethinkDB which is a tcp/JSON protocol 
based document
    + * oriented NoSQL database.
    + * 
    + * <p/>
    + * This sink provides two constuctors:
    + * <p/>
    + * {@link #FlinkRethinkDbSink(String hostname, int hostport, String 
database, String table, JSONSerializationSchema schema)}, and
    + * <p/>
    + * {@link #FlinkRethinkDbSink(String hostname, int hostport, String 
database, String table, JSONSerializationSchema schema, ConflictStrategy 
conflictStrategy)}
    + * <p/>
    + * 
    + * The parameter for the constructor are as follows:
    + * <p/>
    + * <ul>
    + * <li>hostname - the rethinkdb hostname</li>
    + * <li>hosport - the rethinkdb port for the driver to connect</li>
    + * <li>database - the rethinkdb database name to which the table 
belongs</li>
    + * <li>table - the rethinkdb table name where documents are inserted</li>
    + * <li>schema - the schema tranfromer that converts input to JSONObject, 
or JSONArray</li>
    + * <li>conflictStrategy - the conflict resolution strategy in case 
inserted document has id which exists in the db</li>
    + * </ul>
    + * <p/>
    + *
    + * The user can also set:
    + * <p/>
    + * <ul>
    + * <li>username - default is admin</li>
    + * <li>password - default is blank</li>
    + * </ul>
    + * <p/> with the {@link #setUsernameAndPassword(String, String)} method.
    + * <p/>
    + * <b>NOTE: If multiple documents are getting inserted (eg: using 
JSONArray), the sink 
    + * checks if there is an error entry in the result HashMap and throws a 
runtime exception if errors
    + * counts is not zero.  The exception message contains the results 
HashMap. 
    + * In case of multiple errors only the first error is noted in the result 
HashMap.
    + * </b>
    + * 
    + * @see {@link ConflictStrategy} for conflict resolution strategies
    + * 
    + * @param <OUT> a value that can be transformed into a {@link 
org.json.simple.JSONArray;} or {@link org.json.simple.JSONObject}
    + */
    +public class FlinkRethinkDbSink<OUT> extends RichSinkFunction<OUT> 
implements Serializable{
    --- End diff --
    
    This class should be named RethinkDBSink


> Provide RethinkDB Sink for Flink
> --------------------------------
>
>                 Key: FLINK-3967
>                 URL: https://issues.apache.org/jira/browse/FLINK-3967
>             Project: Flink
>          Issue Type: New Feature
>          Components: Streaming, Streaming Connectors
>    Affects Versions: 1.0.3
>         Environment: All
>            Reporter: Mans Singh
>            Assignee: Mans Singh
>            Priority: Minor
>              Labels: features
>             Fix For: 1.1.0
>
>   Original Estimate: 48h
>  Remaining Estimate: 48h
>
> Provide Sink to stream data from flink to rethink db.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to