[ https://issues.apache.org/jira/browse/FLINK-3311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15151293#comment-15151293 ]
ASF GitHub Bot commented on FLINK-3311: --------------------------------------- Github user rmetzger commented on a diff in the pull request: https://github.com/apache/flink/pull/1660#discussion_r53240407 --- Diff: flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/connectors/cassandra/batch/CassandraOutputFormat.java --- @@ -0,0 +1,126 @@ +/* + * 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.batch; + +import java.io.IOException; + +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connectors.cassandra.streaming.ClusterConfigurator; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.ResultSetFuture; +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; + +public abstract class CassandraOutputFormat<OUT extends Tuple> extends + RichOutputFormat<OUT> implements ClusterConfigurator { + + private static final long serialVersionUID = 1L; + + private final String createQuery; + private final String insertQuery; + + private transient Cluster cluster; + private transient Session session; + private transient PreparedStatement ps; + private transient FutureCallback<ResultSet> callback; + private transient Throwable asyncException = null; + + public CassandraOutputFormat(String insertQuery) { + this(null, insertQuery); + } + + public CassandraOutputFormat(String createQuery, String insertQuery) { + if(Strings.isNullOrEmpty(insertQuery)){ + throw new IllegalArgumentException("insertQuery cannot be null or empty"); + } + this.insertQuery = insertQuery; + this.createQuery = createQuery; + } + + @Override + public void configure(Configuration parameters) { + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + + this.cluster = configureCluster(Cluster.builder()).build(); + this.session = cluster.connect(); + this.ps = session.prepare(insertQuery); + this.callback = new FutureCallback<ResultSet>() { + + @Override + public void onSuccess(ResultSet ignored) { + } + + @Override + public void onFailure(Throwable t) { + asyncException = t; + } + }; + + // execute createQuery with only parallelism of 1 + if(createQuery != null){ + int parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); + if(parallelism > 1) { + throw new IllegalStateException("It is allowed to create the table only with parallelism of 1"); --- End diff -- This effectively means the output format works only with parallelism of 1, right? > 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)