Skip to content

Commit

Permalink
[FLINK-6225] [cassandra] Add a CassandraAppendTableSink.
Browse files Browse the repository at this point in the history
This closes apache#3748.
  • Loading branch information
Jing Fan authored and fhueske committed Nov 1, 2017
1 parent 0df8e07 commit 1809cad
Show file tree
Hide file tree
Showing 9 changed files with 454 additions and 12 deletions.
44 changes: 43 additions & 1 deletion docs/dev/table/sourceSinks.md
Original file line number Diff line number Diff line change
Expand Up @@ -492,7 +492,8 @@ The following table lists the `TableSink`s which are provided with Flink.

| **Class name** | **Maven dependency** | **Batch?** | **Streaming?** | **Description**
| `CsvTableSink` | `flink-table` | Y | Append | A simple sink for CSV files.
| `JDBCAppendTableSink` | `flink-jdbc` | Y | Append | Writes tables to a JDBC database.
| `JDBCAppendTableSink` | `flink-jdbc` | Y | Append | Writes a Table to a JDBC table.
| `CassandraAppendTableSink` | `flink-connector-cassandra` | N | Append | Writes a Table to a Cassandra table.
| `Kafka08JsonTableSink` | `flink-connector-kafka-0.8` | N | Append | A Kafka 0.8 sink with JSON encoding.
| `Kafka09JsonTableSink` | `flink-connector-kafka-0.9` | N | Append | A Kafka 0.9 sink with JSON encoding.

Expand Down Expand Up @@ -583,6 +584,47 @@ Similar to using <code>JDBCOutputFormat</code>, you have to explicitly specify t

{% top %}

### CassandraAppendTableSink

The `CassandraAppendTableSink` emits a `Table` to a Cassandra table. The sink only supports append-only streaming tables. It cannot be used to emit a `Table` that is continuously updated. See the [documentation on Table to Stream conversions](./streaming.html#table-to-stream-conversion) for details.

The `CassandraAppendTableSink` inserts all rows at least once into the Cassandra table if checkpointing is enabled. However, you can specify the query as upsert query.

To use the `CassandraAppendTableSink`, you have to add the Cassandra connector dependency (<code>flink-connector-cassandra</code>) to your project. The example below shows how to use the `CassandraAppendTableSink`.

<div class="codetabs" markdown="1">
<div data-lang="java" markdown="1">
{% highlight java %}

ClusterBuilder builder = ... // configure Cassandra cluster connection

CassandraAppendTableSink sink = new CassandraAppendTableSink(
builder,
// the query must match the schema of the table
INSERT INTO flink.myTable (id, name, value) VALUES (?, ?, ?));

Table table = ...
table.writeToSink(sink);
{% endhighlight %}
</div>

<div data-lang="scala" markdown="1">
{% highlight scala %}
val builder: ClusterBuilder = ... // configure Cassandra cluster connection

val sink: CassandraAppendTableSink = new CassandraAppendTableSink(
builder,
// the query must match the schema of the table
INSERT INTO flink.myTable (id, name, value) VALUES (?, ?, ?))

val table: Table = ???
table.writeToSink(sink)
{% endhighlight %}
</div>
</div>

{% top %}

Define a TableSource
--------------------

Expand Down
6 changes: 6 additions & 0 deletions flink-connectors/flink-connector-cassandra/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -197,5 +197,11 @@ under the License.
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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:https://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 org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.table.sinks.AppendStreamTableSink;
import org.apache.flink.types.Row;
import org.apache.flink.util.Preconditions;

import java.util.Properties;

/**
* An {@link AppendStreamTableSink} to write an append stream Table to a Cassandra table.
*/
public class CassandraAppendTableSink implements AppendStreamTableSink<Row> {

private final ClusterBuilder builder;
private final String cql;
private String[] fieldNames;
private TypeInformation[] fieldTypes;
private final Properties properties;

public CassandraAppendTableSink(ClusterBuilder builder, String cql) {
this.builder = Preconditions.checkNotNull(builder, "ClusterBuilder must not be null.");
this.cql = Preconditions.checkNotNull(cql, "CQL query must not be null.");
this.properties = new Properties();
}

public CassandraAppendTableSink(ClusterBuilder builder, String cql, Properties properties) {
this.builder = Preconditions.checkNotNull(builder, "ClusterBuilder must not be null.");
this.cql = Preconditions.checkNotNull(cql, "CQL query must not be null.");
this.properties = Preconditions.checkNotNull(properties, "Properties must not be null.");
}

@Override
public TypeInformation<Row> getOutputType() {
return new RowTypeInfo(fieldTypes);
}

@Override
public String[] getFieldNames() {
return this.fieldNames;
}

@Override
public TypeInformation<?>[] getFieldTypes() {
return this.fieldTypes;
}

@Override
public CassandraAppendTableSink configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
CassandraAppendTableSink cassandraTableSink = new CassandraAppendTableSink(this.builder, this.cql, this.properties);
cassandraTableSink.fieldNames = Preconditions.checkNotNull(fieldNames, "Field names must not be null.");
cassandraTableSink.fieldTypes = Preconditions.checkNotNull(fieldTypes, "Field types must not be null.");
Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
"Number of provided field names and types does not match.");
return cassandraTableSink;
}

@Override
public void emitDataStream(DataStream<Row> dataStream) {
try {
CassandraSink.addSink(dataStream)
.setClusterBuilder(this.builder)
.setQuery(this.cql)
.build();
} catch (Exception e) {
throw new RuntimeException(e);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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:https://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 org.apache.flink.types.Row;

/**
* A SinkFunction to write Row records into a Cassandra table.
*/
public class CassandraRowSink extends AbstractCassandraTupleSink<Row> {

private final int rowArity;

public CassandraRowSink(int rowArity, String insertQuery, ClusterBuilder builder) {
super(insertQuery, builder);
this.rowArity = rowArity;
}

@Override
protected Object[] extract(Row record) {
Object[] al = new Object[rowArity];
for (int i = 0; i < rowArity; i++) {
al[i] = record.getField(i);
}
return al;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,162 @@
/*
* 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:https://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 org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.api.java.typeutils.runtime.RowSerializer;
import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
import org.apache.flink.streaming.runtime.operators.GenericWriteAheadSink;
import org.apache.flink.types.Row;

import com.datastax.driver.core.BoundStatement;
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.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;

import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;

/**
* Sink that emits its input elements into a Cassandra table. This sink stores incoming records within a
* {@link org.apache.flink.runtime.state.AbstractStateBackend}, and only commits them to Cassandra
* if a checkpoint is completed.
*
*/
public class CassandraRowWriteAheadSink extends GenericWriteAheadSink<Row> {
private static final long serialVersionUID = 1L;

protected transient Cluster cluster;
protected transient Session session;

private final String insertQuery;
private transient PreparedStatement preparedStatement;

private ClusterBuilder builder;

private int arity;
private transient Object[] fields;

protected CassandraRowWriteAheadSink(String insertQuery, TypeSerializer<Row> serializer, ClusterBuilder builder, CheckpointCommitter committer) throws Exception {
super(committer, serializer, UUID.randomUUID().toString().replace("-", "_"));
this.insertQuery = insertQuery;
this.builder = builder;
ClosureCleaner.clean(builder, true);
}

public void open() throws Exception {
super.open();
if (!getRuntimeContext().isCheckpointingEnabled()) {
throw new IllegalStateException("The write-ahead log requires checkpointing to be enabled.");
}
cluster = builder.getCluster();
session = cluster.connect();
preparedStatement = session.prepare(insertQuery);

arity = ((RowSerializer) serializer).getArity();
fields = new Object[arity];
}

@Override
public void close() throws Exception {
super.close();
try {
if (session != null) {
session.close();
}
} catch (Exception e) {
LOG.error("Error while closing session.", e);
}
try {
if (cluster != null) {
cluster.close();
}
} catch (Exception e) {
LOG.error("Error while closing cluster.", e);
}
}

@Override
protected boolean sendValues(Iterable<Row> values, long checkpointId, long timestamp) throws Exception {
final AtomicInteger updatesCount = new AtomicInteger(0);
final AtomicInteger updatesConfirmed = new AtomicInteger(0);

final AtomicReference<Throwable> exception = new AtomicReference<>();

FutureCallback<ResultSet> callback = new FutureCallback<ResultSet>() {
@Override
public void onSuccess(ResultSet resultSet) {
updatesConfirmed.incrementAndGet();
if (updatesCount.get() > 0) { // only set if all updates have been sent
if (updatesCount.get() == updatesConfirmed.get()) {
synchronized (updatesConfirmed) {
updatesConfirmed.notifyAll();
}
}
}
}

@Override
public void onFailure(Throwable throwable) {
if (exception.compareAndSet(null, throwable)) {
LOG.error("Error while sending value.", throwable);
synchronized (updatesConfirmed) {
updatesConfirmed.notifyAll();
}
}
}
};

//set values for prepared statement
int updatesSent = 0;
for (Row value : values) {
for (int x = 0; x < arity; x++) {
fields[x] = value.getField(x);
}
//insert values and send to cassandra
BoundStatement s = preparedStatement.bind(fields);
s.setDefaultTimestamp(timestamp);
ResultSetFuture result = session.executeAsync(s);
updatesSent++;
if (result != null) {
//add callback to detect errors
Futures.addCallback(result, callback);
}
}
updatesCount.set(updatesSent);

synchronized (updatesConfirmed) {
while (exception.get() == null && updatesSent != updatesConfirmed.get()) {
updatesConfirmed.wait();
}
}

if (exception.get() != null) {
LOG.warn("Sending a value failed.", exception.get());
return false;
} else {
return true;
}
}
}
Loading

0 comments on commit 1809cad

Please sign in to comment.