forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-6225] [cassandra] Add a CassandraAppendTableSink.
This closes apache#3748.
- Loading branch information
Showing
9 changed files
with
454 additions
and
12 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
88 changes: 88 additions & 0 deletions
88
...c/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraAppendTableSink.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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); | ||
} | ||
} | ||
} |
42 changes: 42 additions & 0 deletions
42
...andra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraRowSink.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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; | ||
} | ||
} |
162 changes: 162 additions & 0 deletions
162
...main/java/org/apache/flink/streaming/connectors/cassandra/CassandraRowWriteAheadSink.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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; | ||
} | ||
} | ||
} |
Oops, something went wrong.