Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-13059][Cassandra Connector] Release Semaphore correctly on Exception in send() #8967

Merged
merged 4 commits into from Sep 3, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -128,8 +128,14 @@ public void snapshotState(FunctionSnapshotContext ctx) throws Exception {
@Override
public void invoke(IN value) throws Exception {
checkAsyncErrors();
tryAcquire();
final ListenableFuture<V> result = send(value);
tryAcquire(1);
final ListenableFuture<V> result;
try {
result = send(value);
} catch (Exception e) {
semaphore.release();
throw e;
}
Futures.addCallback(result, callback);
}

Expand All @@ -139,11 +145,12 @@ protected Session createSession() {

public abstract ListenableFuture<V> send(IN value);

private void tryAcquire() throws InterruptedException, TimeoutException {
if (!semaphore.tryAcquire(config.getMaxConcurrentRequestsTimeout().toMillis(), TimeUnit.MILLISECONDS)) {
private void tryAcquire(int permits) throws InterruptedException, TimeoutException {
if (!semaphore.tryAcquire(permits, config.getMaxConcurrentRequestsTimeout().toMillis(), TimeUnit.MILLISECONDS)) {
throw new TimeoutException(
String.format(
"Failed to acquire 1 permit of %d to send value in %s.",
"Failed to acquire %d out of %d permits to send value in %s.",
permits,
config.getMaxConcurrentRequests(),
config.getMaxConcurrentRequestsTimeout()
)
Expand All @@ -158,8 +165,8 @@ private void checkAsyncErrors() throws Exception {
}
}

private void flush() {
semaphore.acquireUninterruptibly(config.getMaxConcurrentRequests());
private void flush() throws InterruptedException, TimeoutException {
tryAcquire(config.getMaxConcurrentRequests());
semaphore.release(config.getMaxConcurrentRequests());
}

Expand Down
Expand Up @@ -28,6 +28,7 @@
import com.datastax.driver.core.Cluster;
import com.datastax.driver.core.ResultSet;
import com.datastax.driver.core.Session;
import com.datastax.driver.core.exceptions.InvalidQueryException;
import com.datastax.driver.core.exceptions.NoHostAvailableException;
import com.google.common.util.concurrent.ListenableFuture;
import org.junit.Assert;
Expand Down Expand Up @@ -180,7 +181,7 @@ public void go() throws Exception {
}
};
t.start();
while (t.getState() != Thread.State.WAITING) {
while (t.getState() != Thread.State.TIMED_WAITING) {
Thread.sleep(5);
}

Expand Down Expand Up @@ -212,7 +213,7 @@ public void go() throws Exception {
}
};
t.start();
while (t.getState() != Thread.State.WAITING) {
while (t.getState() != Thread.State.TIMED_WAITING) {
Thread.sleep(5);
}

Expand Down Expand Up @@ -272,6 +273,26 @@ public void testReleaseOnFailure() throws Exception {
}
}

@Test(timeout = DEFAULT_TEST_TIMEOUT)
public void testReleaseOnSendException() throws Exception {
final CassandraSinkBaseConfig config = CassandraSinkBaseConfig.newBuilder()
.setMaxConcurrentRequests(1)
.build();

try (TestCassandraSink testCassandraSink = createOpenedSendExceptionTestCassandraSink(config)) {
Assert.assertEquals(1, testCassandraSink.getAvailablePermits());
Assert.assertEquals(0, testCassandraSink.getAcquiredPermits());

try {
testCassandraSink.invoke("N/A");
} catch (Exception e) {
Assert.assertTrue(e instanceof InvalidQueryException);
Assert.assertEquals(1, testCassandraSink.getAvailablePermits());
Assert.assertEquals(0, testCassandraSink.getAcquiredPermits());
}
}
}

@Test(timeout = DEFAULT_TEST_TIMEOUT)
public void testTimeoutExceptionOnInvoke() throws Exception {
final CassandraSinkBaseConfig config = CassandraSinkBaseConfig.newBuilder()
Expand Down Expand Up @@ -331,6 +352,12 @@ private OneInputStreamOperatorTestHarness<String, Object> createOpenedTestHarnes
return testHarness;
}

private TestCassandraSink createOpenedSendExceptionTestCassandraSink(CassandraSinkBaseConfig config) {
final TestCassandraSink testCassandraSink = new SendExceptionTestCassandraSink(config);
testCassandraSink.open(new Configuration());
return testCassandraSink;
}

private static class TestCassandraSink extends CassandraSinkBase<String, ResultSet> implements AutoCloseable {

private static final ClusterBuilder builder;
Expand Down Expand Up @@ -379,4 +406,15 @@ void enqueueCompletableFuture(CompletableFuture<ResultSet> completableFuture) {
resultSetFutures.offer(ResultSetFutures.fromCompletableFuture(completableFuture));
}
}

private static class SendExceptionTestCassandraSink extends TestCassandraSink {
SendExceptionTestCassandraSink(CassandraSinkBaseConfig config) {
super(config, new NoOpCassandraFailureHandler());
}

@Override
public ListenableFuture<ResultSet> send(String value) {
throw new InvalidQueryException("For test purposes");
}
}
}