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

    https://github.com/apache/flink/pull/1771#discussion_r63869055
  
    --- Diff: 
flink-streaming-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraCommitter.java
 ---
    @@ -0,0 +1,131 @@
    +/**
    + * 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.Cluster;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.Session;
    +import org.apache.flink.api.java.ClosureCleaner;
    +import org.apache.flink.streaming.runtime.operators.CheckpointCommitter;
    +
    +/**
    + * CheckpointCommitter that saves information about completed checkpoints 
within a separate table in a cassandra
    + * database.
    + * <p/>
    + * Entries are in the form |operator_id | subtask_id | 
last_completed_checkpoint|
    + */
    +public class CassandraCommitter extends CheckpointCommitter {
    +   private ClusterBuilder builder;
    +   private transient Cluster cluster;
    +   private transient Session session;
    +
    +   private String keySpace = "flink_auxiliary";
    +   private String table = "checkpoints_";
    +
    +   private transient PreparedStatement deleteStatement;
    +   private transient PreparedStatement updateStatement;
    +   private transient PreparedStatement selectStatement;
    +
    +   public CassandraCommitter(ClusterBuilder builder) {
    +           this.builder = builder;
    +           ClosureCleaner.clean(builder, true);
    +   }
    +
    +   public CassandraCommitter(ClusterBuilder builder, String keySpace) {
    +           this(builder);
    +           this.keySpace = keySpace;
    +   }
    +
    +   /**
    +    * Internally used to set the job ID after instantiation.
    +    *
    +    * @param id
    +    * @throws Exception
    +    */
    +   public void setJobId(String id) throws Exception {
    +           super.setJobId(id);
    +           table += id;
    +   }
    +
    +   /**
    +    * Generates the necessary tables to store information.
    +    *
    +    * @return
    +    * @throws Exception
    +    */
    +   @Override
    +   public void createResource() throws Exception {
    +           cluster = builder.getCluster();
    +           session = cluster.connect();
    +
    +           session.execute(String.format("CREATE KEYSPACE IF NOT EXISTS %s 
with replication={'class':'SimpleStrategy', 'replication_factor':3};", 
keySpace));
    +           session.execute(String.format("CREATE TABLE IF NOT EXISTS %s.%s 
(sink_id text, sub_id int, checkpoint_id bigint, PRIMARY KEY (sink_id, 
sub_id));", keySpace, table));
    +
    +           try {
    +                   session.close();
    +           } catch (Exception e) {
    +                   LOG.error("Error while closing session.", e);
    --- End diff --
    
    This means that a failed closing operation cannot leave the external system 
in a corrupted state?


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