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

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

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

    https://github.com/apache/flink/pull/1660#discussion_r53281448
  
    --- Diff: 
flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/connectors/cassandra/streaming/BaseCassandraSink.java
 ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.connectors.cassandra.streaming;
    +
    +import java.io.IOException;
    +
    +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.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.Session;
    +import com.google.common.base.Strings;
    +import com.google.common.util.concurrent.FutureCallback;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.ListenableFuture;
    +
    +/**
    + * BaseCassandraSink is the common abstract class of {@link 
CassandraMapperSink} and {@link CassandraSink}.
    + *
    + * The {@link Cluster} is built via {@link 
ClusterConfigurator#configureCluster(Cluster.Builder cluster) configureCluster}
    + * inherited by {@link ClusterConfigurator}
    + *
    + * The {@link Session} is liable to maintain the connection between our 
Sink and the Cassandra Cluster.
    + *
    + * @param <IN> Type of the elements emitted by this sink
    + */
    +public abstract class BaseCassandraSink<IN,V> extends RichSinkFunction<IN> 
implements ClusterConfigurator {
    +
    +   private static final long serialVersionUID = 1L;
    +   private static final Logger LOG = 
LoggerFactory.getLogger(BaseCassandraSink.class);
    +
    +   protected final String createQuery;
    +   protected transient Cluster cluster;
    +   protected transient Session session;
    +   protected transient Throwable asyncException = null;
    +   protected transient FutureCallback<V> callback;
    +
    +   public BaseCassandraSink(String createQuery){
    +           this.createQuery = createQuery;
    --- End diff --
    
    if you verify that the insertQuery is not empty, you should also check that 
for createQuery


> Add a connector for streaming data into Cassandra
> -------------------------------------------------
>
>                 Key: FLINK-3311
>                 URL: https://issues.apache.org/jira/browse/FLINK-3311
>             Project: Flink
>          Issue Type: New Feature
>          Components: Streaming Connectors
>            Reporter: Robert Metzger
>            Assignee: Andrea Sella
>
> We had users in the past asking for a Flink+Cassandra integration.
> It seems that there is a well-developed java client for connecting into 
> Cassandra: https://github.com/datastax/java-driver (ASL 2.0)
> There are also tutorials out there on how to start a local cassandra instance 
> (for the tests): 
> http://prettyprint.me/prettyprint.me/2010/02/14/running-cassandra-as-an-embedded-service/index.html
> For the data types, I think we should support TupleX types, and map standard 
> java types to the respective cassandra types.
> In addition, it seems that there is a object mapper from datastax to store 
> POJOs in Cassandra (there are annotations for defining the primary key and 
> types)



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

Reply via email to