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

PubSubSinkTask.flush should flush tracking for all input partitions and only those partitions #306

Open
wants to merge 3 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -347,6 +347,7 @@ private ByteString handleValue(Schema schema, Object value, Map<String, String>
@Override
public void flush(Map<TopicPartition, OffsetAndMetadata> partitionOffsets) {
log.debug("Flushing...");
RuntimeException maybeException = null;
// Process results of all the outstanding futures specified by each TopicPartition.
for (Map.Entry<TopicPartition, OffsetAndMetadata> partitionOffset :
partitionOffsets.entrySet()) {
Expand All @@ -356,20 +357,28 @@ public void flush(Map<TopicPartition, OffsetAndMetadata> partitionOffsets) {
if (outstandingFuturesForTopic == null) {
continue;
}
int partition = partitionOffset.getKey().partition();
OutstandingFuturesForPartition outstandingFutures =
outstandingFuturesForTopic.get(partitionOffset.getKey().partition());
outstandingFuturesForTopic.get(partition);
if (outstandingFutures == null) {
continue;
}
try {
ApiFutures.allAsList(outstandingFutures.futures).get();
// Only wait for partition to complete if the flush hasn't already failed.
if (maybeException == null) {
ApiFutures.allAsList(outstandingFutures.futures).get();
}
} catch (Exception e) {
throw new RuntimeException(e);
maybeException = new RuntimeException(e);
} finally {
// Always clear tracking for flushed partitions
outstandingFutures.futures.clear();
}
}
allOutstandingFutures.clear();
if (maybeException != null) {
// If any partitions had an exceptional future, throw after all cleanup
throw maybeException;
}
}

/** Publish all the messages in a partition and store the Future's for each publish request. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
package com.google.pubsub.kafka.sink;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
Expand All @@ -25,6 +26,7 @@
import static org.mockito.Mockito.when;

import com.google.api.core.ApiFuture;
import com.google.api.core.SettableApiFuture;
import com.google.cloud.pubsub.v1.Publisher;
import com.google.protobuf.ByteString;
import com.google.pubsub.kafka.common.ConnectorUtils;
Expand Down Expand Up @@ -281,7 +283,7 @@ public void testNullSchema() {
public void testPutWherePublishesAreInvoked() {
props.put(CloudPubSubSinkConnector.MAX_BUFFER_SIZE_CONFIG, CPS_MIN_BATCH_SIZE1);
task.start(props);
List<SinkRecord> records = getSampleRecords();
List<SinkRecord> records = getSampleRecords(0);
task.put(records);
ArgumentCaptor<PubsubMessage> captor = ArgumentCaptor.forClass(PubsubMessage.class);
verify(publisher, times(2)).publish(captor.capture());
Expand Down Expand Up @@ -349,7 +351,7 @@ public void testFlushWithNoPublishInPut() throws Exception {
task.start(props);
Map<TopicPartition, OffsetAndMetadata> partitionOffsets = new HashMap<>();
partitionOffsets.put(new TopicPartition(KAFKA_TOPIC, 0), null);
List<SinkRecord> records = getSampleRecords();
List<SinkRecord> records = getSampleRecords(0);
ApiFuture<String> goodFuture = getSuccessfulPublishFuture();
when(publisher.publish(any(PubsubMessage.class))).thenReturn(goodFuture);
task.put(records);
Expand All @@ -362,18 +364,66 @@ public void testFlushWithNoPublishInPut() throws Exception {
* Tests that if a Future that is being processed in flush() failed with an exception, that an
* exception is thrown.
*/
@Test(expected = RuntimeException.class)
@Test
public void testFlushExceptionCase() throws Exception {
task.start(props);
Map<TopicPartition, OffsetAndMetadata> partitionOffsets = new HashMap<>();
partitionOffsets.put(new TopicPartition(KAFKA_TOPIC, 0), null);
List<SinkRecord> records = getSampleRecords();
List<SinkRecord> records = getSampleRecords(0);
ApiFuture<String> badFuture = getFailedPublishFuture();
when(publisher.publish(any(PubsubMessage.class))).thenReturn(badFuture);
task.put(records);
assertThrows(RuntimeException.class, () -> task.flush(partitionOffsets));
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use assertThrows instead of expected so the following verifies are actually reached.

verify(publisher, times(2)).publish(any(PubsubMessage.class));
verify(badFuture, times(2)).addListener(any(Runnable.class), any(Executor.class));
}

/**
* Tests that flush clears tracking of all partitions it is invoked for even during exceptions and
* propagates an exception.
*/
@Test
public void testFlushMultiPartitionExceptionCase() throws Exception {
task.start(props);
Map<TopicPartition, OffsetAndMetadata> partitionOffsets = new HashMap<>();
partitionOffsets.put(new TopicPartition(KAFKA_TOPIC, 0), null);
partitionOffsets.put(new TopicPartition(KAFKA_TOPIC, 1), null);
List<SinkRecord> recordsP0 = getSampleRecords(0);
List<SinkRecord> recordsP1 = getSampleRecords(1);
when(publisher.publish(any(PubsubMessage.class))).thenReturn(getFailedPublishFuture());
task.put(recordsP0);
task.put(recordsP1);
assertThrows(RuntimeException.class, () -> task.flush(partitionOffsets));
verify(publisher, times(4)).publish(any(PubsubMessage.class));
// The second flush should not throw an exception since all tracking was cleared
task.flush(partitionOffsets);
verify(publisher, times(1)).publish(any(PubsubMessage.class));
verify(badFuture, times(1)).addListener(any(Runnable.class), any(Executor.class));
}

/**
* Tests that flush clears tracking of only the partitions it is invoked for so it cannot be
* considered complete for a subsequent flush on another partition that may have failed.
*/
@Test
public void testFlushOneOfMultiplePartitionsCase() throws Exception {
task.start(props);
Map<TopicPartition, OffsetAndMetadata> partitionOffsetsP0 = new HashMap<>();
partitionOffsetsP0.put(new TopicPartition(KAFKA_TOPIC, 0), null);
Map<TopicPartition, OffsetAndMetadata> partitionOffsetsP1 = new HashMap<>();
partitionOffsetsP1.put(new TopicPartition(KAFKA_TOPIC, 1), null);
List<SinkRecord> recordsP0 = getSampleRecords(0);
List<SinkRecord> recordsP1 = getSampleRecords(1);
when(publisher.publish(any(PubsubMessage.class)))
.thenReturn(getSuccessfulPublishFuture())
.thenReturn(getSuccessfulPublishFuture())
.thenReturn(getFailedPublishFuture())
.thenReturn(getFailedPublishFuture());
task.put(recordsP0);
task.put(recordsP1);
verify(publisher, times(4)).publish(any(PubsubMessage.class));

task.flush(partitionOffsetsP0);
// The second flush should not complete successfully
assertThrows(RuntimeException.class, () -> task.flush(partitionOffsetsP1));
}

/**
Expand Down Expand Up @@ -590,7 +640,7 @@ public void testFlushExceptionThenNoExceptionCase() throws Exception {
task.start(props);
Map<TopicPartition, OffsetAndMetadata> partitionOffsets = new HashMap<>();
partitionOffsets.put(new TopicPartition(KAFKA_TOPIC, 0), null);
List<SinkRecord> records = getSampleRecords();
List<SinkRecord> records = getSampleRecords(0);
ApiFuture<String> badFuture = getFailedPublishFuture();
ApiFuture<String> goodFuture = getSuccessfulPublishFuture();
when(publisher.publish(any(PubsubMessage.class))).thenReturn(badFuture).thenReturn(badFuture).thenReturn(goodFuture);
Expand All @@ -599,7 +649,7 @@ public void testFlushExceptionThenNoExceptionCase() throws Exception {
task.flush(partitionOffsets);
} catch (RuntimeException e) {
}
records = getSampleRecords();
records = getSampleRecords(0);
task.put(records);
task.flush(partitionOffsets);
verify(publisher, times(4)).publish(any(PubsubMessage.class));
Expand All @@ -620,12 +670,12 @@ public void testPublisherShutdownOnStop() throws Exception {
}

/** Get some sample SinkRecords's to use in the tests. */
private List<SinkRecord> getSampleRecords() {
private List<SinkRecord> getSampleRecords(int partition) {
List<SinkRecord> records = new ArrayList<>();
records.add(
new SinkRecord(
KAFKA_TOPIC,
0,
partition,
STRING_SCHEMA,
KAFKA_MESSAGE_KEY1,
BYTE_STRING_SCHEMA,
Expand All @@ -634,7 +684,7 @@ private List<SinkRecord> getSampleRecords() {
records.add(
new SinkRecord(
KAFKA_TOPIC,
0,
partition,
STRING_SCHEMA,
KAFKA_MESSAGE_KEY1,
BYTE_STRING_SCHEMA,
Expand Down