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

    https://github.com/apache/flink/pull/1771#discussion_r58561402
  
    --- Diff: 
flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraTupleWriteAheadSink.java
 ---
    @@ -0,0 +1,136 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.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 org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.tuple.Tuple;
    +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
    +import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
    +import org.apache.flink.streaming.runtime.operators.GenericAtLeastOnceSink;
    +
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +/**
    + * Sink that emits its input elements into a Cassandra database. 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.
    + *
    + * @param <IN> Type of the elements emitted by this sink
    + */
    +public class CassandraTupleWriteAheadSink<IN extends Tuple> extends 
GenericAtLeastOnceSink<IN> {
    +   protected transient Cluster cluster;
    +   protected transient Session session;
    +
    +   private final String insertQuery;
    +   private transient PreparedStatement preparedStatement;
    +
    +   private transient Throwable exception = null;
    +   private transient FutureCallback<ResultSet> callback;
    +
    +   private ClusterBuilder builder;
    +
    +   private int updatesSent = 0;
    +   private AtomicInteger updatesConfirmed = new AtomicInteger(0);
    +
    +   private transient Object[] fields;
    +
    +   protected CassandraTupleWriteAheadSink(String insertQuery, 
TypeSerializer<IN> serializer, ClusterBuilder builder, String jobID, 
CheckpointCommitter committer) throws Exception {
    +           super(committer, serializer, jobID);
    +           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.");
    +           }
    +           this.callback = new FutureCallback<ResultSet>() {
    +                   @Override
    +                   public void onSuccess(ResultSet resultSet) {
    +                           updatesConfirmed.incrementAndGet();
    +                   }
    +
    +                   @Override
    +                   public void onFailure(Throwable throwable) {
    +                           exception = throwable;
    --- End diff --
    
    Error logging here as well


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to