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

ASF GitHub Bot commented on FLINK-4500:
---------------------------------------

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

    https://github.com/apache/flink/pull/4605#discussion_r139482639
  
    --- Diff: 
flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBaseTest.java
 ---
    @@ -0,0 +1,469 @@
    +/*
    + * 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 org.apache.flink.configuration.Configuration;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.ResultSet;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.google.common.util.concurrent.ListenableFuture;
    +
    +import org.junit.Assert;
    +import org.junit.BeforeClass;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.mockito.invocation.InvocationOnMock;
    +import org.mockito.stubbing.Answer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.TimeUnit;
    +
    +import static org.mockito.Mockito.mock;
    +import static org.powermock.api.mockito.PowerMockito.when;
    +
    +/**
    + * Tests for the {@link CassandraSinkBase}.
    + */
    +public class CassandraSinkBaseTest {
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(CassandraSinkBaseTest.class);
    +
    +   private static final String DUMMY_QUERY_STMT = "CQL_Dummy_Stmt";
    +
    +   private static final String DUMMY_MESSAGE = "Dummy_msg";
    +
    +   private static final int MAX_THREAD_NUM = 3;
    +
    +   private static final int MAX_THREAD_POOL_SIZE = 2;
    +
    +   @BeforeClass
    +   public static void doSetUp() throws Exception {
    +
    +   }
    +
    +   @BeforeClass
    +   public static void doTearDown() throws Exception {
    --- End diff --
    
    empty method that can be removed


> Cassandra sink can lose messages
> --------------------------------
>
>                 Key: FLINK-4500
>                 URL: https://issues.apache.org/jira/browse/FLINK-4500
>             Project: Flink
>          Issue Type: Bug
>          Components: Cassandra Connector
>    Affects Versions: 1.1.0
>            Reporter: Elias Levy
>            Assignee: Michael Fong
>
> The problem is the same as I pointed out with the Kafka producer sink 
> (FLINK-4027).  The CassandraTupleSink's send() and CassandraPojoSink's send() 
> both send data asynchronously to Cassandra and record whether an error occurs 
> via a future callback.  But CassandraSinkBase does not implement 
> Checkpointed, so it can't stop checkpoint from happening even though the are 
> Cassandra queries in flight from the checkpoint that may fail.  If they do 
> fail, they would subsequently not be replayed when the job recovered, and 
> would thus be lost.
> In addition, 
> CassandraSinkBase's close should check whether there is a pending exception 
> and throw it, rather than silently close.  It should also wait for any 
> pending async queries to complete and check their status before closing.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to