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 @@ -52,6 +52,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 @@ -65,6 +79,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
Copy link
Contributor

Choose a reason for hiding this comment

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

I believe this is for testing purposes, right? There are ways for testing frameworks to access private fields, if really necessary. I am in favor of not break encapsulation and expose internal state for the purposes of testing.

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 agree to some extent, though this also needs to be fixed on master if we change it here. Breaking encapsulation with package private fields is not a big issue IMO since it doesn't change the public API. Also it probably wouldn't hurt to split some of the code into smaller packages, which limits visibility.

I'd really like to avoid using reflection if possible. It's brittle and prevents the IDE from helping you when you're refactoring (renaming a field for example).

How would you feel about adding acked to the testing constructor and injecting it from the tests instead? That way the tests still have access to it without the field being package private. We could also pretty easily move OffsetEntry out into the spout.internal package.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@hmcl (sorry, forgot to ping you on the other comment) The same applies to Timer btw, thinking of your comment about the spout code getting a little big. We could easily move that to the internal package too.

Copy link
Contributor

Choose a reason for hiding this comment

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

@srdo using reflection for tests only wouldn't look to concerning to me, but I also agree with your point of not being able to do find usages in the IDE. I would also say that if the tests and the source class are correctly implemented, the tests should never need to query the internal state to assert of the correctness of the code. If one has to do that, that to me feels like something is not as good as it could be.

In my opinion package private methods have a very specific use, which is for the construction of libraries. Although they are not part of the public API, they are easy to misuse by clients who may want to take shortcuts. With this said, I think that having a package where we put some of the nested classes would make the code significantly more modular. I really think that it is getting to the point where I start seeing if / else blocks all over the place, added to instance of ... and I really want to avoid that at all costs... otherwise soon we wont' have a hand on this.

However extracting the nested classes will also add an extra complexity, because there is a significant amount of state that the OffsetEntry class uses that would have to be passed around if it becomes it's own class. I still thing we should do it, though. When we do it should be in such a way that we can plugin a different implementation of OffsetEntry. Perhaps also rename this class, as it is really an OffsetManager. What do you think ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, I agree that we should move OffsetEntry out of KafkaSpout. I also agree that we should be able to test the spout without peeking directly into the spout's fields.

Here's what I suggest:

  • We move Timer out into a separate class, and use a TimerFactory to create instances in the KafkaSpout (basically the same solution as we use for KafkaConsumer mocking on master). That way we can write tests where time is simulated, which is helpful if we want to test that the spout commits offsets correctly (or any other periodic action).

  • We move and rename OffsetEntry out into a new class. It looks like the only part of KafkaSpout state it reads is numUncommittedOffsets from OffsetEntry.commit(). Since that method returns void, we can solve that problem very easily by returning numCommittedOffsets from there. If we want to support other OffsetEntry implementations we can just set up a factory for that too, but I think that can be postponed until we actually need to support multiple implementations.

  • We rewrite the offending tests here to use a mocked KafkaConsumer and KafkaTimer instead of KafkaUnit, and we make the tests check that the offsets are committed on the consumer instead of directly reading KafkaSpout.acked. I think KafkaUnit is a good idea for integration tests, but I don't think it fits that well for the tests in this PR.

I'll try implementing this on master, if it works out I'll open a PR there first. We can port the improvements to 1.x once we agree on the solution on master.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@hmcl The changes are here #1832. I'd like to merge this branch, then we can backport the refactoring in a separate PR once it has been merged to master.

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
}
}
}

// ======== emit =========
private void emit() {
emitTupleIfNotEmitted(waitingToEmit.next());
waitingToEmit.remove();
while(!emitTupleIfNotEmitted(waitingToEmit.next()) && waitingToEmit.hasNext()) {
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
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.
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,245 @@
/*
* 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 static org.mockito.Mockito.*;
import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.*;

public class SingleTopicKafkaSpoutTest {

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);

for (int i = 0; i < msgCount; i++){
KeyedMessage<String, String> keyedMessage = new KeyedMessage<>(
topicName, Integer.toString(i),
Integer.toString(i));

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(anyString(), anyList(), messageIdToDoubleAck.capture());
context.spout.ack(messageIdToDoubleAck.getValue());

for (int i = 0; i < messageCount/2; i++) {
context.spout.nextTuple();
};

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

for (int i = 0; i < messageCount; i++) {
context.spout.nextTuple();
};

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

verify(context.collector, times(messageCount)).emit(
eq(SingleTopicKafkaSpoutConfiguration.STREAM),
anyList(),
remainingIds.capture());
for (Object id : remainingIds.getAllValues()) {
context.spout.ack(id);
}

for(Object item : context.spout.acked.values()) {
assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item);
};
}

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


for (int i = 0; i < messageCount; i++) {
context.spout.nextTuple();
ArgumentCaptor<Object> messageId = ArgumentCaptor.forClass(Object.class);
verify(context.collector).emit(
eq(SingleTopicKafkaSpoutConfiguration.STREAM),
eq(new Values(SingleTopicKafkaSpoutConfiguration.TOPIC,
Integer.toString(i),
Integer.toString(i))),
messageId.capture());
context.spout.ack(messageId.getValue());
reset(context.collector);
};

for (Object item : context.spout.acked.values()) {
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(anyString(), anyList(), 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(anyString(), anyList(), 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
for (int i = 0; i < messageCount + 5; i++) {
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),
anyList(),
remainingMessageIds.capture());
for (Object id : remainingMessageIds.getAllValues()) {
context.spout.ack(id);
}

for (Object item : context.spout.acked.values()) {
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(anyString(), anyList(), messageIdToFail.capture());
reset(context.collector);

//play 2nd tuple
ArgumentCaptor<Object> messageIdToAck = ArgumentCaptor.forClass(Object.class);
context.spout.nextTuple();
verify(context.collector).emit(anyString(), anyList(), 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
for (int i = 0; i < messageCount + 5; i++) {
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),
anyList(),
remainingIds.capture());
for (Object id : remainingIds.getAllValues()) {
context.spout.ack(id);
};

for (Object item : context.spout.acked.values()) {
assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item);
};
}
}
Loading