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

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_r53281258
  
    --- Diff: 
flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/connectors/cassandra/streaming/CassandraMapperSink.java
 ---
    @@ -0,0 +1,86 @@
    +/*
    + * 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.Serializable;
    +
    +import org.apache.flink.configuration.Configuration;
    +
    +import com.datastax.driver.mapping.MappingManager;
    +import com.datastax.driver.mapping.Mapper;
    +import com.google.common.base.Preconditions;
    +import com.google.common.util.concurrent.ListenableFuture;
    +import 
org.apache.flink.connectors.cassandra.streaming.examples.WriteCassandraMapperSink;
    +
    +/**
    + * Flink Sink to save data into a Cassandra cluster using {@link Mapper}, 
which
    + * it uses annotations from {@link com.datastax.driver.mapping}. See 
example.
    + * {@link WriteCassandraMapperSink }
    + *
    + * @param <IN>                     Type of the elements emitted by this 
sink
    + */
    +public abstract class CassandraMapperSink<IN extends Serializable> extends
    +           BaseCassandraSink<IN,Void> {
    +
    +   private static final long serialVersionUID = 1L;
    +
    +   protected Class<IN> clazz;
    +
    +   protected transient Mapper<IN> mapper;
    +
    +   protected transient MappingManager mappingManager;
    +
    +   /**
    +    * Constructor for creating a CassandraMapperSink
    +    *
    +    * @param clazz                 Class<IN> instance
    +    */
    +   public CassandraMapperSink(Class<IN> clazz) {
    +           this(null, clazz);
    +   }
    +
    +
    +   /**
    +    * The main constructor for creating CassandraMapperSink
    +    *
    +    * @param createQuery   Query to create dynamically a Table into 
Cassandra
    +    * @param clazz                 Class<IN> instance
    +    */
    +   public CassandraMapperSink(String createQuery, Class<IN> clazz) {
    +           super(createQuery);
    +           Preconditions.checkNotNull(clazz, "Clazz cannot be null");
    +           this.clazz = clazz;
    +   }
    +
    +   @Override
    +   public void open(Configuration configuration) {
    +           super.open(configuration);
    +           try {
    +                   this.mappingManager = new MappingManager(session);
    +                   this.mapper = mappingManager.mapper(clazz);
    +           } catch (Exception e) {
    +                   logError(e.getMessage());
    --- End diff --
    
    when you throw an exception anyway we don't need to log anything.
    
    regardless, the usage of logError is inconsistent, it is only used in this 
method, although you log things at other places.


> 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