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

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

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

    https://github.com/apache/flink/pull/1771#discussion_r56156990
  
    --- Diff: 
flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/batch/connectors/cassandra/CassandraInputFormat.java
 ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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.batch.connectors.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.ResultSet;
    +import com.datastax.driver.core.Row;
    +import com.datastax.driver.core.Session;
    +import com.google.common.base.Strings;
    +import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
    +import org.apache.flink.api.common.io.NonParallelInput;
    +import org.apache.flink.api.common.io.RichInputFormat;
    +import org.apache.flink.api.common.io.statistics.BaseStatistics;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.io.GenericInputSplit;
    +import org.apache.flink.core.io.InputSplit;
    +import org.apache.flink.core.io.InputSplitAssigner;
    +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +/**
    + * InputFormat to read data from Apache Cassandra and generate ${@link 
Tuple}.
    + *
    + * @param <OUT> type of Tuple
    + */
    +public class CassandraInputFormat<OUT extends Tuple> extends 
RichInputFormat<OUT, InputSplit> implements NonParallelInput {
    +   private static final Logger LOG = 
LoggerFactory.getLogger(CassandraInputFormat.class);
    +
    +   private final String query;
    +   private final ClusterBuilder builder;
    +
    +   private transient Cluster cluster;
    +   private transient Session session;
    +   private transient ResultSet resultSet;
    +
    +   public CassandraInputFormat(String query, ClusterBuilder builder) {
    +           if (Strings.isNullOrEmpty(query)) {
    +                   throw new IllegalArgumentException("Query cannot be 
null or empty");
    +           }
    +           if (builder == null) {
    +                   throw new IllegalArgumentException("Builder cannot be 
null.");
    +           }
    +           this.query = query;
    +           this.builder = builder;
    +   }
    +
    +   @Override
    +   public void configure(Configuration parameters) {
    +           this.cluster = builder.getCluster();
    +   }
    +
    +   @Override
    +   public BaseStatistics getStatistics(BaseStatistics cachedStatistics) 
throws IOException {
    +           return cachedStatistics;
    +   }
    +
    +   /**
    +    * Opens a Session and executes the query.
    +    *
    +    * @param ignored
    +    * @throws IOException
    +    */
    +   @Override
    +   public void open(InputSplit ignored) throws IOException {
    +           this.session = cluster.connect();
    +           this.resultSet = session.execute(query);
    +   }
    +
    +   @Override
    +   public boolean reachedEnd() throws IOException {
    +           return resultSet.isExhausted();
    +   }
    +
    +   @Override
    +   public OUT nextRecord(OUT reuse) throws IOException {
    +           final Row item = resultSet.one();
    +           for (int i = 0; i < reuse.getArity(); i++) {
    +                   reuse.setField(item.getObject(i), i);
    +           }
    +           return reuse;
    +   }
    +
    +   @Override
    +   public InputSplit[] createInputSplits(int minNumSplits) throws 
IOException {
    +           GenericInputSplit[] split = {new GenericInputSplit(0, 1)};
    +           return split;
    +   }
    +
    +   @Override
    +   public InputSplitAssigner getInputSplitAssigner(InputSplit[] 
inputSplits) {
    +           return new DefaultInputSplitAssigner(inputSplits);
    +   }
    +
    +   /**
    +    * Closes all resources used.
    +    */
    +   @Override
    +   public void close() throws IOException {
    +           try {
    +                   session.close();
    +           } catch (Exception e) {
    +                   LOG.info("Inputformat couldn't be closed - " + 
e.getMessage());
    +           }
    +
    +           try {
    +                   cluster.close();
    +           } catch (Exception e) {
    +                   LOG.info("Inputformat couldn't be closed - " + 
e.getMessage());
    --- End diff --
    
    I would add the cause in both cases.


> 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