Skip to content
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
26 changes: 26 additions & 0 deletions external/storm-kafka-client/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,20 @@
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>${storm.kafka.client.version}</version>
<exclusions>
<exclusion>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
</exclusions>
</dependency>
<!--test dependencies -->
<dependency>
Expand All @@ -66,6 +80,18 @@
<version>4.11</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>info.batey.kafka</groupId>
<artifactId>kafka-unit</artifactId>
<version>0.6</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>log4j-over-slf4j</artifactId>
<version>${log4j-over-slf4j.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public class KafkaSpout<K, V> extends BaseRichSpout {
private KafkaSpoutStreams kafkaSpoutStreams; // Object that wraps all the logic to declare output fields and emit tuples
private transient KafkaSpoutTuplesBuilder<K, V> tuplesBuilder; // Object that contains the logic to build tuples for each ConsumerRecord

private transient Map<TopicPartition, OffsetEntry> acked; // Tuples that were successfully acked. These tuples will be committed periodically when the commit timer expires, after consumer rebalance, or on close/deactivate
transient Map<TopicPartition, OffsetEntry> acked; // Tuples that were successfully acked. These tuples will be committed periodically when the commit timer expires, after consumer rebalance, or on close/deactivate
private transient Set<KafkaSpoutMessageId> emitted; // Tuples that have been emitted but that are "on the wire", i.e. pending being acked or failed
private transient Iterator<ConsumerRecord<K, V>> waitingToEmit; // Records that have been polled and are queued to be emitted in the nextTuple() call. One record is emitted per nextTuple()
private transient long numUncommittedOffsets; // Number of offsets that have been polled and emitted but not yet been committed
Expand Down Expand Up @@ -266,19 +266,22 @@ private void doSeekRetriableTopicPartitions() {
if (offsetAndMeta != null) {
kafkaConsumer.seek(rtp, offsetAndMeta.offset() + 1); // seek to the next offset that is ready to commit in next commit cycle
} else {
kafkaConsumer.seekToEnd(toArrayList(rtp)); // Seek to last committed offset
kafkaConsumer.seek(rtp, acked.get(rtp).committedOffset + 1); // Seek to last committed offset
Copy link
Contributor

@srdo srdo Sep 19, 2016

Choose a reason for hiding this comment

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

Nice catch. Is this code being hit by the tests?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yup :) . shouldReplayInOrderFailedMessages and shouldReplayOutOfOrderFailedMessages Tests.

Copy link
Contributor

@hmcl hmcl Sep 20, 2016

Choose a reason for hiding this comment

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

@jfenc91 Why is seekToEnd not correct?

Copy link
Contributor

Choose a reason for hiding this comment

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

Guessing here, but if no tuples have been acked since partition reassignment and some have failed, this case should be hit. The right thing to do is then to start replaying from the last committed offset. If the consumer just skips to the end of the partition, it may skip some messages, and findNextCommitOffset will break on subsequent calls because the offsets that can be committed aren't continuous anymore.

@jfenc91 The tests you mention ack a message before a tuple is failed. Doesn't that prevent the spout from hitting this case, since there's now an offset in acked?

Copy link
Contributor

Choose a reason for hiding this comment

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

Hm. This probably isn't great if auto commit mode is enabled. It'll make the spout return to where it started processing the partition every time a tuple is failed. It might make sense to disable tuple retries when auto commit is enabled, I don't think we can support it properly in auto commit mode with the current RetryService interface, since we don't update committedOffset in auto commit mode.

I think having fail() do nothing if auto commit is enabled makes sense. Otherwise, RetryService needs to return the list of offsets that should be retried so this code can seek directly to the smallest retry offset per partition instead.

Copy link
Contributor

Choose a reason for hiding this comment

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

Autocommit mode comes with no guarantees of delivery. It is supported for backwards compatibility.

Copy link
Contributor

Choose a reason for hiding this comment

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

@jfenc91 About this being tested: If the two tests you mentioned run fine without this change, would you mind adding one that tests this? Maybe something like have a topic with a few messages in it and fail the first message, then check that the spout doesn't skip the messages following the failed message?

Copy link
Contributor Author

@jfenc91 jfenc91 Sep 21, 2016

Choose a reason for hiding this comment

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

So shouldReplayOutOfOrderFailedMessages does fail the first tuple and with changing the line that test does fail. You are right, the other passes. So, I am renaming the test that checks this case to: shouldReplayFirstTupleFailedOutOfOrder to make it more clear. I would add another, but I hate to duplicate unit tests without adding extra value.

Copy link
Contributor

Choose a reason for hiding this comment

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

Looks good, thanks.

Copy link
Contributor

Choose a reason for hiding this comment

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

@jfenc91 @srdo this is correct. @jfenc91 good finding!

}
}
}

// ======== emit =========
private void emit() {
emitTupleIfNotEmitted(waitingToEmit.next());
waitingToEmit.remove();
while(!emitTupleIfNotEmitted(waitingToEmit.next()) && waitingToEmit.hasNext()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

@jfenc91 I believe that there is no need for this while loop here. I ran all your test cases without it and they all pass. Unless there is a particular reason to have this loop in here, I would suggest that you remove it.

Copy link
Contributor

Choose a reason for hiding this comment

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

This is a performance improvement. emitTupleIfNotEmitted does what it says. It may or may not emit a tuple. If it does not emit a tuple (because it has already been emitted) then the spout will exit nextTuple() having emitted nothing. In that case storm will sleep for 1 ms, even though waitingToEmit is not necessarily empty.

waitingToEmit.remove();
}
}

// emits one tuple per record
private void emitTupleIfNotEmitted(ConsumerRecord<K, V> record) {

//Emits one tuple per record
//@return true if tuple was emitted
private boolean emitTupleIfNotEmitted(ConsumerRecord<K, V> record) {
final TopicPartition tp = new TopicPartition(record.topic(), record.partition());
final KafkaSpoutMessageId msgId = new KafkaSpoutMessageId(record);

Expand All @@ -295,7 +298,9 @@ private void emitTupleIfNotEmitted(ConsumerRecord<K, V> record) {
retryService.remove(msgId); // re-emitted hence remove from failed
}
LOG.trace("Emitted tuple [{}] for record [{}]", tuple, record);
return true;
}
return false;
}

private void commitOffsetsForAckedTuples() {
Expand Down Expand Up @@ -451,7 +456,7 @@ public int compare(KafkaSpoutMessageId m1, KafkaSpoutMessageId m2) {
/**
* This class is not thread safe
*/
private class OffsetEntry {
class OffsetEntry {
private final TopicPartition tp;
private final long initialFetchOffset; /* First offset to be fetched. It is either set to the beginning, end, or to the first uncommitted offset.
* Initial value depends on offset strategy. See KafkaSpoutConsumerRebalanceListener */
Expand Down Expand Up @@ -479,16 +484,17 @@ public OffsetAndMetadata findNextCommitOffset() {
KafkaSpoutMessageId nextCommitMsg = null; // this is a convenience variable to make it faster to create OffsetAndMetadata

for (KafkaSpoutMessageId currAckedMsg : ackedMsgs) { // complexity is that of a linear scan on a TreeMap
if ((currOffset = currAckedMsg.offset()) == initialFetchOffset || currOffset == nextCommitOffset + 1) { // found the next offset to commit
if ((currOffset = currAckedMsg.offset()) == nextCommitOffset + 1) { // found the next offset to commit
Copy link
Contributor

@hmcl hmcl Sep 20, 2016

Choose a reason for hiding this comment

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

With this change is the code going to work for a topic that does not yet have any records and/or commits in it?

I recall testing the condition using initialFetchOffset was necessary. Why isn't it necessary?

Copy link
Contributor

Choose a reason for hiding this comment

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

nextCommitOffset starts out as initialFetchOffset - 1 due to L467 and L481. Assuming a tuple isn't acked after being committed, the two checks should work out to be the same thing.

Copy link
Contributor

Choose a reason for hiding this comment

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

@srdo @jfenc91 you are right, this simplification is correct. Probably the code was refactored in a way that made this redundant. Good catch!

found = true;
nextCommitMsg = currAckedMsg;
nextCommitOffset = currOffset;
} else if (currAckedMsg.offset() > nextCommitOffset + 1) { // offset found is not continuous to the offsets listed to go in the next commit, so stop search
LOG.debug("topic-partition [{}] has non-continuous offset [{}]. It will be processed in a subsequent batch.", tp, currOffset);
break;
} else {
LOG.debug("topic-partition [{}] has unexpected offset [{}].", tp, currOffset);
break;
//Received a redundant ack. Ignore and continue processing.
Copy link
Contributor

Choose a reason for hiding this comment

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

Feel free to correct me, but it's my impression that Storm doesn't really support double acking. If a bolt acks the same tuple multiple times, I think Storm ends up failing the tuple tree. Have you seen double acking occur from Storm's side?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So I added this because I was seeing acks on tuples that were behind the already committed offset. With that break statement in place, the result is a complete halt in processing. While this isn't pretty this is the only solution I could see.

Copy link
Contributor

Choose a reason for hiding this comment

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

Do you have a test case where we can reproduce this consistently? This code is already running in a large production environment, and according to the feedback I received, there are no issues of this nature.

I am also a bit confused on what I mean by multiple acks. I am pretty sure storm guarantees that a tuple it's either acked (once), failed (once), or times out (which is equivalent to failing)

Copy link
Contributor

Choose a reason for hiding this comment

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

It still seems strange to me if Storm acks a message multiple times. You're right though that the break statement should be removed, if the spout gets into a state where an offset <= committed is acked, the offset should probably just be logged and dropped since it's already committed. I'd still like to know how the spout gets into this state though.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I saw this a couple of times before I figured it out. I have not taken the time to reproduce this in a toy/test case, but given the error message this is clearly a storm or storm-kafka-client issue. I got to this state in about 30 minutes of running a topology processing 800k-300k tuples a minute with about 10s latency. The input to the topology was on the order of 2k-10k tuples per minute with a bolt that separated each input into multiple tuples. At startup there was a high amount of failures after the separation (I was making requests against an unwarmed ELB). I would guess that that is enough to reproduce with random data/failures.

Copy link
Contributor

Choose a reason for hiding this comment

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

@jfenc91 @srdo This behavior still seems quite surprising to me. However, this fix probably won't have any bad side effect, so it should be OK. Perhaps you can add to the log message that we are ignoring it as it is out of order, duplicate, or something like that.

@revans2 can you please comment on why, or if it is possible a tuple to be acked more than once?

Copy link
Contributor

Choose a reason for hiding this comment

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

The issue is not with storm itself, but with the internal bookkeeping of the spout. If Kafka rebalances partitions (or there are other recoverable errors) the spout resets all of it's internal bookkeeping, but there is no way to un-emit something. So everything that was emitted, but not fully acked/committed when the recovery happened is still outstanding, and will come back for the spout to process.

  1. emit A offset B
  2. oh crap, recovering...
  3. Recovered handling partition A again
  4. emit A offset B again
  5. ack A offset B (the first one)
  6. ack A offset B (the second recovered one)

Copy link
Contributor

Choose a reason for hiding this comment

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

It can also happen if the Kafka cluster has unclean leader election enabled. In that case, the log end offset in Kafka may become smaller than the last offset the spout committed. It's a weird edge case, since most people probably don't need to have at least once in Storm while allowing message loss in Kafka. Bit of a newbie trap though, since unclean leader election is enabled by default in Kafka.

LOG.warn("topic-partition [{}] has unexpected offset [{}]. Current committed Offset [{}]",
tp, currOffset, committedOffset);
}
}

Expand Down Expand Up @@ -532,6 +538,10 @@ public void commit(OffsetAndMetadata committedOffset) {
LOG.trace("{}", this);
}

long getCommittedOffset() {
return committedOffset;
}

public boolean isEmpty() {
return ackedMsgs.isEmpty();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,240 @@
/*
* 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.storm.kafka.spout;

import info.batey.kafka.unit.KafkaUnitRule;
import kafka.producer.KeyedMessage;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
import org.apache.storm.spout.SpoutOutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.tuple.Values;
import org.junit.Rule;
import org.junit.Test;
import org.mockito.ArgumentCaptor;

import static org.junit.Assert.*;

import java.util.Map;
import java.util.stream.IntStream;
import static org.mockito.Mockito.*;
import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.*;

public class SingleTopicKafkaSpoutTest {
Copy link
Contributor

Choose a reason for hiding this comment

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

The blocks of code B1, B2, B3, B4 have pretty much the same initialization code, with exception of perhaps one or two values that can be parameterized. Can you please create an initialization or setup method with this code, and avoid all of the code duplication.

I didn't look in particular detail in other parts of this code, but it may also be easy to do the same thing for the multiple play and verify blocks of code in this class. If that's the case, that would be ideal.

Copy link
Contributor Author

@jfenc91 jfenc91 Nov 14, 2016

Choose a reason for hiding this comment

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

I addressed the initialization part. I think that abstracting the different test cases away algorithmically would add too much complexity. It is nice to have test cases that are easy to understand.


private class SpoutContext {
public KafkaSpout<String, String> spout;
public SpoutOutputCollector collector;

public SpoutContext(KafkaSpout<String, String> spout,
SpoutOutputCollector collector) {
this.spout = spout;
this.collector = collector;
}
}

@Rule
public KafkaUnitRule kafkaUnitRule = new KafkaUnitRule();

void populateTopicData(String topicName, int msgCount) {
kafkaUnitRule.getKafkaUnit().createTopic(topicName);

IntStream.range(0, msgCount).forEach(value -> {
KeyedMessage<String, String> keyedMessage = new KeyedMessage<>(
topicName, Integer.toString(value),
Integer.toString(value));

kafkaUnitRule.getKafkaUnit().sendMessages(keyedMessage);
});
}

SpoutContext initializeSpout(int msgCount) {
populateTopicData(SingleTopicKafkaSpoutConfiguration.TOPIC, msgCount);
int kafkaPort = kafkaUnitRule.getKafkaPort();

TopologyContext topology = mock(TopologyContext.class);
SpoutOutputCollector collector = mock(SpoutOutputCollector.class);
Map conf = mock(Map.class);

KafkaSpout<String, String> spout = new KafkaSpout<>(getKafkaSpoutConfig(getKafkaSpoutStreams(), kafkaPort));
spout.open(conf, topology, collector);
spout.activate();
return new SpoutContext(spout, collector);
}
/*
* Asserts that the next possible offset to commit or the committed offset is the provided offset.
* An offset that is ready to be committed is not guarenteed to be already committed.
*/
private void assertOffsetCommitted(int offset, KafkaSpout.OffsetEntry entry) {

boolean currentOffsetMatch = entry.getCommittedOffset() == offset;
OffsetAndMetadata nextOffset = entry.findNextCommitOffset();
boolean nextOffsetMatch = nextOffset != null && nextOffset.offset() == offset;
assertTrue("Next offset: " +
entry.findNextCommitOffset() +
" OR current offset: " +
entry.getCommittedOffset() +
" must equal desired offset: " +
offset,
currentOffsetMatch | nextOffsetMatch);
}

@Test
public void shouldContinueWithSlowDoubleAcks() throws Exception {
int messageCount = 20;
SpoutContext context = initializeSpout(messageCount);

//play 1st tuple
ArgumentCaptor<Object> messageIdToDoubleAck = ArgumentCaptor.forClass(Object.class);
context.spout.nextTuple();
verify(context.collector).emit(anyObject(), anyObject(), messageIdToDoubleAck.capture());
context.spout.ack(messageIdToDoubleAck.getValue());

IntStream.range(0, messageCount/2).forEach(value -> {
context.spout.nextTuple();
});

context.spout.ack(messageIdToDoubleAck.getValue());

IntStream.range(0, messageCount).forEach(value -> {
context.spout.nextTuple();
});

ArgumentCaptor<Object> remainingIds = ArgumentCaptor.forClass(Object.class);

verify(context.collector, times(messageCount)).emit(
eq(SingleTopicKafkaSpoutConfiguration.STREAM),
anyObject(),
remainingIds.capture());
remainingIds.getAllValues().iterator().forEachRemaining(context.spout::ack);

context.spout.acked.values().forEach(item -> {
assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item);
});
}

@Test
public void shouldEmitAllMessages() throws Exception {
int messageCount = 10;
SpoutContext context = initializeSpout(messageCount);


IntStream.range(0, messageCount).forEach(value -> {
context.spout.nextTuple();
ArgumentCaptor<Object> messageId = ArgumentCaptor.forClass(Object.class);
verify(context.collector).emit(
eq(SingleTopicKafkaSpoutConfiguration.STREAM),
eq(new Values(SingleTopicKafkaSpoutConfiguration.TOPIC,
Integer.toString(value),
Integer.toString(value))),
messageId.capture());
context.spout.ack(messageId.getValue());
reset(context.collector);
});

context.spout.acked.values().forEach(item -> {
assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item);
});
}

@Test
public void shouldReplayInOrderFailedMessages() throws Exception {
int messageCount = 10;
SpoutContext context = initializeSpout(messageCount);

//play and ack 1 tuple
ArgumentCaptor<Object> messageIdAcked = ArgumentCaptor.forClass(Object.class);
context.spout.nextTuple();
verify(context.collector).emit(anyObject(), anyObject(), messageIdAcked.capture());
context.spout.ack(messageIdAcked.getValue());
reset(context.collector);

//play and fail 1 tuple
ArgumentCaptor<Object> messageIdFailed = ArgumentCaptor.forClass(Object.class);
context.spout.nextTuple();
verify(context.collector).emit(anyObject(), anyObject(), messageIdFailed.capture());
context.spout.fail(messageIdFailed.getValue());
reset(context.collector);

//pause so that failed tuples will be retried
Thread.sleep(200);


//allow for some calls to nextTuple() to fail to emit a tuple
IntStream.range(0, messageCount + 5).forEach(value -> {
Copy link
Contributor

@srdo srdo Sep 19, 2016

Choose a reason for hiding this comment

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

What kind of failure can happen here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

A call to the spout's emit() does not always mean a tuple is emitted. I will improve the comment.

context.spout.nextTuple();
});

ArgumentCaptor<Object> remainingMessageIds = ArgumentCaptor.forClass(Object.class);

//1 message replayed, messageCount - 2 messages emitted for the first time
verify(context.collector, times(messageCount - 1)).emit(
eq(SingleTopicKafkaSpoutConfiguration.STREAM),
anyObject(),
remainingMessageIds.capture());
remainingMessageIds.getAllValues().iterator().forEachRemaining(context.spout::ack);

context.spout.acked.values().forEach(item -> {
assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item);
});
}

@Test
public void shouldReplayFirstTupleFailedOutOfOrder() throws Exception {
int messageCount = 10;
SpoutContext context = initializeSpout(messageCount);


//play 1st tuple
ArgumentCaptor<Object> messageIdToFail = ArgumentCaptor.forClass(Object.class);
context.spout.nextTuple();
verify(context.collector).emit(anyObject(), anyObject(), messageIdToFail.capture());
reset(context.collector);

//play 2nd tuple
ArgumentCaptor<Object> messageIdToAck = ArgumentCaptor.forClass(Object.class);
context.spout.nextTuple();
verify(context.collector).emit(anyObject(), anyObject(), messageIdToAck.capture());
reset(context.collector);

//ack 2nd tuple
context.spout.ack(messageIdToAck.getValue());
//fail 1st tuple
context.spout.fail(messageIdToFail.getValue());

//pause so that failed tuples will be retried
Thread.sleep(200);

//allow for some calls to nextTuple() to fail to emit a tuple
IntStream.range(0, messageCount + 5).forEach(value -> {
context.spout.nextTuple();
});

ArgumentCaptor<Object> remainingIds = ArgumentCaptor.forClass(Object.class);
//1 message replayed, messageCount - 2 messages emitted for the first time
verify(context.collector, times(messageCount - 1)).emit(
eq(SingleTopicKafkaSpoutConfiguration.STREAM),
anyObject(),
remainingIds.capture());
remainingIds.getAllValues().iterator().forEachRemaining(context.spout::ack);

context.spout.acked.values().forEach(item -> {
assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item);
});
}
}
Loading