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

[Backport] KafkaLookupExtractorFactory: shutdown kafka consumer on close() #3574

Merged
merged 1 commit into from
Oct 16, 2016
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.
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 @@ -82,6 +82,7 @@ public String fromBytes(byte[] bytes)
private final AtomicReference<Map<String, String>> mapRef = new AtomicReference<>(null);
private final AtomicBoolean started = new AtomicBoolean(false);

private volatile ConsumerConnector consumerConnector;
private volatile ListenableFuture<?> future = null;

@JsonProperty
Expand Down Expand Up @@ -194,8 +195,12 @@ public boolean start()
public void run()
{
while (!executorService.isShutdown()) {
final ConsumerConnector consumerConnector = buildConnector(kafkaProperties);
consumerConnector = buildConnector(kafkaProperties);
try {
if (executorService.isShutdown()) {
break;
}

final List<KafkaStream<String, String>> streams = consumerConnector.createMessageStreamsByFilter(
new Whitelist(Pattern.quote(topic)), 1, DEFAULT_STRING_DECODER, DEFAULT_STRING_DECODER
);
Expand Down Expand Up @@ -283,7 +288,7 @@ public void onFailure(Throwable t)
}
}

// Overriden in tests
// Overridden in tests
ConsumerConnector buildConnector(Properties properties)
{
return new kafka.javaapi.consumer.ZookeeperConsumerConnector(
Expand All @@ -301,6 +306,11 @@ public boolean close()
}
started.set(false);
executorService.shutdown();

if (consumerConnector != null) {
consumerConnector.shutdown();
}

final ListenableFuture<?> future = this.future;
if (future != null) {
if (!future.isDone() && !future.cancel(false)) {
Expand Down Expand Up @@ -363,7 +373,7 @@ public byte[] getCacheKey()
.putLong(startCount)
.array();
} else {
// If the number of things added HAS changed during the coruse of this extractor's life, we CANNOT cache
// If the number of things added HAS changed during the course of this extractor's life, we CANNOT cache
final byte[] scrambler = StringUtils.toUtf8(UUID.randomUUID().toString());
return ByteBuffer
.allocate(idutf8.length + 1 + scrambler.length + 1)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -283,7 +283,7 @@ public Object answer() throws Throwable {
threadWasInterrupted.set(Thread.currentThread().isInterrupted());
return null;
}
}).once();
}).times(2);

EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator);
final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
Expand Down Expand Up @@ -364,6 +364,8 @@ public void testStopDeleteError()
.andReturn(new ConcurrentHashMap<String, String>())
.once();
EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(false).once();
consumerConnector.shutdown();
EasyMock.expectLastCall().anyTimes();

EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator);
final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
Expand Down Expand Up @@ -404,7 +406,7 @@ public void testStartStopStart()
.once();
EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once();
consumerConnector.shutdown();
EasyMock.expectLastCall().once();
EasyMock.expectLastCall().times(2);
EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator);
final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
cacheManager,
Expand Down Expand Up @@ -444,7 +446,7 @@ public void testStartStartStop()
.once();
EasyMock.expect(cacheManager.delete(EasyMock.anyString())).andReturn(true).once();
consumerConnector.shutdown();
EasyMock.expectLastCall().once();
EasyMock.expectLastCall().times(3);
EasyMock.replay(cacheManager, kafkaStream, consumerConnector, consumerIterator);
final KafkaLookupExtractorFactory factory = new KafkaLookupExtractorFactory(
cacheManager,
Expand Down