flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-3311) Add a connector for streaming data into Cassandra
Date Tue, 05 Apr 2016 15:37:26 GMT

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

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_r58561096
  
    --- Diff: flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBase.java
---
    @@ -0,0 +1,93 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.Session;
    +import com.google.common.util.concurrent.FutureCallback;
    +import com.google.common.util.concurrent.Futures;
    +import com.google.common.util.concurrent.ListenableFuture;
    +import org.apache.flink.api.java.ClosureCleaner;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +/**
    + * CassandraSinkBase is the common abstract class of {@link CassandraPojoSink} and {@link
CassandraTupleSink}.
    + *
    + * @param <IN> Type of the elements emitted by this sink
    + */
    +public abstract class CassandraSinkBase<IN, V> extends RichSinkFunction<IN>
{
    +	protected static final Logger LOG = LoggerFactory.getLogger(CassandraSinkBase.class);
    +	protected transient Cluster cluster;
    +	protected transient Session session;
    +
    +	protected transient Throwable exception = null;
    +	protected transient FutureCallback<V> callback;
    +
    +	private final ClusterBuilder builder;
    +
    +	protected CassandraSinkBase(ClusterBuilder builder) {
    +		this.builder = builder;
    +		ClosureCleaner.clean(builder, true);
    +	}
    +
    +	@Override
    +	public void open(Configuration configuration) {
    +		this.callback = new FutureCallback<V>() {
    +			@Override
    +			public void onSuccess(V ignored) {
    +			}
    +
    +			@Override
    +			public void onFailure(Throwable t) {
    +				exception = t;
    --- End diff --
    
    Can you log the exception as well. Maybe invoke() is never called after the failure and
nobody knows what's going on


> 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)

Mime
View raw message