[GitHub] storm pull request #2152: [STORM-2505] OffsetManager should account for offs...

2017-07-17 Thread askprasanna
Github user askprasanna commented on a diff in the pull request:

https://github.com/apache/storm/pull/2152#discussion_r127695589
  
--- Diff: 
external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
 ---
@@ -0,0 +1,146 @@
+/*
+ * Copyright 2017 The Apache Software Foundation.
+ *
+ * Licensed 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 static 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.*;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.InOrder;
+import org.mockito.MockitoAnnotations;
+
+public class KafkaSpoutCommitTest {
+
+private final long offsetCommitPeriodMs = 2_000;
+private final TopologyContext contextMock = 
mock(TopologyContext.class);
+private final SpoutOutputCollector collectorMock = 
mock(SpoutOutputCollector.class);
+private final Map conf = new HashMap<>();
+private final TopicPartition partition = new 
TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+private KafkaConsumer consumerMock;
+private KafkaSpout spout;
+private KafkaSpoutConfig spoutConfig;
+
+@Captor
+private ArgumentCaptor> 
commitCapture;
+
+private void setupSpout(Set assignedPartitions) {
+MockitoAnnotations.initMocks(this);
+spoutConfig = getKafkaSpoutConfigBuilder(-1)
+.setOffsetCommitPeriodMs(offsetCommitPeriodMs)
+.build();
+
+consumerMock = mock(KafkaConsumer.class);
+KafkaConsumerFactory consumerFactory = 
(kafkaSpoutConfig) -> consumerMock;
+
+//Set up a spout listening to 1 topic partition
+spout = new KafkaSpout<>(spoutConfig, consumerFactory);
+
+spout.open(conf, contextMock, collectorMock);
+spout.activate();
+
+ArgumentCaptor rebalanceListenerCapture 
= ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
+verify(consumerMock).subscribe(anyCollection(), 
rebalanceListenerCapture.capture());
+
+//Assign partitions to the spout
+ConsumerRebalanceListener consumerRebalanceListener = 
rebalanceListenerCapture.getValue();
+consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
+}
+
+@Test
+public void testCommitSuccessWithOffsetVoids() {
+//Verify that the commit logic can handle offset voids
+try (SimulatedTime simulatedTime = new SimulatedTime()) {
+setupSpout(Collections.singleton(partition));
+Map>> 
records = new HashMap<>();
+List> recordsForPartition = new 
ArrayList<>();
+// Offsets emitted are 0,1,2,3,4,...,8,9
--- End diff --

will tweak the comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #2152: [STORM-2505] OffsetManager should account for offs...

2017-07-14 Thread srdo
Github user srdo commented on a diff in the pull request:

https://github.com/apache/storm/pull/2152#discussion_r127538268
  
--- Diff: 
external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
 ---
@@ -0,0 +1,146 @@
+/*
+ * Copyright 2017 The Apache Software Foundation.
+ *
+ * Licensed 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 static 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.*;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.InOrder;
+import org.mockito.MockitoAnnotations;
+
+public class KafkaSpoutCommitTest {
+
+private final long offsetCommitPeriodMs = 2_000;
+private final TopologyContext contextMock = 
mock(TopologyContext.class);
+private final SpoutOutputCollector collectorMock = 
mock(SpoutOutputCollector.class);
+private final Map conf = new HashMap<>();
+private final TopicPartition partition = new 
TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+private KafkaConsumer consumerMock;
+private KafkaSpout spout;
+private KafkaSpoutConfig spoutConfig;
+
+@Captor
+private ArgumentCaptor> 
commitCapture;
+
+private void setupSpout(Set assignedPartitions) {
+MockitoAnnotations.initMocks(this);
+spoutConfig = getKafkaSpoutConfigBuilder(-1)
+.setOffsetCommitPeriodMs(offsetCommitPeriodMs)
+.build();
+
+consumerMock = mock(KafkaConsumer.class);
+KafkaConsumerFactory consumerFactory = 
(kafkaSpoutConfig) -> consumerMock;
+
+//Set up a spout listening to 1 topic partition
+spout = new KafkaSpout<>(spoutConfig, consumerFactory);
+
+spout.open(conf, contextMock, collectorMock);
+spout.activate();
+
+ArgumentCaptor rebalanceListenerCapture 
= ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
+verify(consumerMock).subscribe(anyCollection(), 
rebalanceListenerCapture.capture());
+
+//Assign partitions to the spout
+ConsumerRebalanceListener consumerRebalanceListener = 
rebalanceListenerCapture.getValue();
+consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
+}
+
+@Test
+public void testCommitSuccessWithOffsetVoids() {
+//Verify that the commit logic can handle offset voids
+try (SimulatedTime simulatedTime = new SimulatedTime()) {
+setupSpout(Collections.singleton(partition));
+Map>> 
records = new HashMap<>();
+List> recordsForPartition = new 
ArrayList<>();
+// Offsets emitted are 0,1,2,3,4,...,8,9
--- End diff --

Nit: This comment is a little confusing, the ... would usually imply that 
5,6,7 are included. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #2152: [STORM-2505] OffsetManager should account for offs...

2017-07-14 Thread srdo
Github user srdo commented on a diff in the pull request:

https://github.com/apache/storm/pull/2152#discussion_r127538620
  
--- Diff: 
external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
 ---
@@ -0,0 +1,146 @@
+/*
+ * Copyright 2017 The Apache Software Foundation.
+ *
+ * Licensed 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 static 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.*;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.InOrder;
+import org.mockito.MockitoAnnotations;
+
+public class KafkaSpoutCommitTest {
+
+private final long offsetCommitPeriodMs = 2_000;
+private final TopologyContext contextMock = 
mock(TopologyContext.class);
+private final SpoutOutputCollector collectorMock = 
mock(SpoutOutputCollector.class);
+private final Map conf = new HashMap<>();
+private final TopicPartition partition = new 
TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+private KafkaConsumer consumerMock;
+private KafkaSpout spout;
+private KafkaSpoutConfig spoutConfig;
+
+@Captor
+private ArgumentCaptor> 
commitCapture;
+
+private void setupSpout(Set assignedPartitions) {
+MockitoAnnotations.initMocks(this);
+spoutConfig = getKafkaSpoutConfigBuilder(-1)
+.setOffsetCommitPeriodMs(offsetCommitPeriodMs)
+.build();
+
+consumerMock = mock(KafkaConsumer.class);
+KafkaConsumerFactory consumerFactory = 
(kafkaSpoutConfig) -> consumerMock;
+
+//Set up a spout listening to 1 topic partition
+spout = new KafkaSpout<>(spoutConfig, consumerFactory);
+
+spout.open(conf, contextMock, collectorMock);
+spout.activate();
+
+ArgumentCaptor rebalanceListenerCapture 
= ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
+verify(consumerMock).subscribe(anyCollection(), 
rebalanceListenerCapture.capture());
+
+//Assign partitions to the spout
+ConsumerRebalanceListener consumerRebalanceListener = 
rebalanceListenerCapture.getValue();
+consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
+}
+
+@Test
+public void testCommitSuccessWithOffsetVoids() {
+//Verify that the commit logic can handle offset voids
+try (SimulatedTime simulatedTime = new SimulatedTime()) {
+setupSpout(Collections.singleton(partition));
+Map>> 
records = new HashMap<>();
+List> recordsForPartition = new 
ArrayList<>();
+// Offsets emitted are 0,1,2,3,4,...,8,9
+for (int i = 0; i < 5; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+for (int i = 8; i < 10; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+records.put(partition, recordsForPartition);
+
+when(consumerMock.poll(anyLong()))
+.thenReturn(new ConsumerRecords(records));
+
+for (int i = 0; i < r

[GitHub] storm pull request #2152: [STORM-2505] OffsetManager should account for offs...

2017-07-14 Thread srdo
Github user srdo commented on a diff in the pull request:

https://github.com/apache/storm/pull/2152#discussion_r127538410
  
--- Diff: 
external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
 ---
@@ -0,0 +1,146 @@
+/*
+ * Copyright 2017 The Apache Software Foundation.
+ *
+ * Licensed 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 static 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.*;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.InOrder;
+import org.mockito.MockitoAnnotations;
+
+public class KafkaSpoutCommitTest {
+
+private final long offsetCommitPeriodMs = 2_000;
+private final TopologyContext contextMock = 
mock(TopologyContext.class);
+private final SpoutOutputCollector collectorMock = 
mock(SpoutOutputCollector.class);
+private final Map conf = new HashMap<>();
+private final TopicPartition partition = new 
TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+private KafkaConsumer consumerMock;
+private KafkaSpout spout;
+private KafkaSpoutConfig spoutConfig;
+
+@Captor
+private ArgumentCaptor> 
commitCapture;
+
+private void setupSpout(Set assignedPartitions) {
+MockitoAnnotations.initMocks(this);
+spoutConfig = getKafkaSpoutConfigBuilder(-1)
+.setOffsetCommitPeriodMs(offsetCommitPeriodMs)
+.build();
+
+consumerMock = mock(KafkaConsumer.class);
+KafkaConsumerFactory consumerFactory = 
(kafkaSpoutConfig) -> consumerMock;
+
+//Set up a spout listening to 1 topic partition
+spout = new KafkaSpout<>(spoutConfig, consumerFactory);
+
+spout.open(conf, contextMock, collectorMock);
+spout.activate();
+
+ArgumentCaptor rebalanceListenerCapture 
= ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
+verify(consumerMock).subscribe(anyCollection(), 
rebalanceListenerCapture.capture());
+
+//Assign partitions to the spout
+ConsumerRebalanceListener consumerRebalanceListener = 
rebalanceListenerCapture.getValue();
+consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
+}
+
+@Test
+public void testCommitSuccessWithOffsetVoids() {
+//Verify that the commit logic can handle offset voids
+try (SimulatedTime simulatedTime = new SimulatedTime()) {
+setupSpout(Collections.singleton(partition));
+Map>> 
records = new HashMap<>();
+List> recordsForPartition = new 
ArrayList<>();
+// Offsets emitted are 0,1,2,3,4,...,8,9
+for (int i = 0; i < 5; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+for (int i = 8; i < 10; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+records.put(partition, recordsForPartition);
+
+when(consumerMock.poll(anyLong()))
+.thenReturn(new ConsumerRecords(records));
+
+for (int i = 0; i < r

[GitHub] storm pull request #2152: [STORM-2505] OffsetManager should account for offs...

2017-06-09 Thread askprasanna
Github user askprasanna commented on a diff in the pull request:

https://github.com/apache/storm/pull/2152#discussion_r121249114
  
--- Diff: 
external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
 ---
@@ -104,6 +104,10 @@ public KafkaSpout(KafkaSpoutConfig 
kafkaSpoutConfig) {
 this.kafkaSpoutConfig = kafkaSpoutConfig;
 }
 
+long getNumUncommittedOffsets() {
--- End diff --

Yes, the main purpose of the test is verifying commit behavior over offset 
gaps. We can remove this getter and the asserts on it if required.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #2152: [STORM-2505] OffsetManager should account for offs...

2017-06-09 Thread srdo
Github user srdo commented on a diff in the pull request:

https://github.com/apache/storm/pull/2152#discussion_r121217733
  
--- Diff: 
external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
 ---
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2017 The Apache Software Foundation.
+ *
+ * Licensed 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 static 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.*;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.InOrder;
+
+public class KafkaSpoutCommitTest {
+
+private final long offsetCommitPeriodMs = 2_000;
+private final TopologyContext contextMock = 
mock(TopologyContext.class);
+private final SpoutOutputCollector collectorMock = 
mock(SpoutOutputCollector.class);
+private final Map conf = new HashMap<>();
+private final TopicPartition partition = new 
TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+private KafkaConsumer consumerMock;
+private KafkaSpout spout;
+private KafkaSpoutConfig spoutConfig;
+
+private void setupSpout(Set assignedPartitions) {
+spoutConfig = getKafkaSpoutConfigBuilder(-1)
+.setOffsetCommitPeriodMs(offsetCommitPeriodMs)
+.build();
+
+consumerMock = mock(KafkaConsumer.class);
+KafkaConsumerFactory consumerFactory = 
(kafkaSpoutConfig) -> consumerMock;
+
+//Set up a spout listening to 1 topic partition
+spout = new KafkaSpout<>(spoutConfig, consumerFactory);
+
+spout.open(conf, contextMock, collectorMock);
+spout.activate();
+
+ArgumentCaptor rebalanceListenerCapture 
= ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
+verify(consumerMock).subscribe(anyCollection(), 
rebalanceListenerCapture.capture());
+
+//Assign partitions to the spout
+ConsumerRebalanceListener consumerRebalanceListener = 
rebalanceListenerCapture.getValue();
+consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
+}
+
+@Test
+public void testCommitSuccessWithOffsetVoids() {
+//Verify that the commit logic can handle offset voids
+try (SimulatedTime simulatedTime = new SimulatedTime()) {
+setupSpout(Collections.singleton(partition));
+Map>> 
records = new HashMap<>();
+List> recordsForPartition = new 
ArrayList<>();
+// Offsets emitted are 0,1,2,3,4,...,8,9
+for (int i = 0; i < 5; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+for (int i = 8; i < 10; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+records.put(partition, recordsForPartition);
+
+when(consumerMock.poll(anyLong()))
+.thenReturn(new ConsumerRecords(records));
+
+for (int i = 0; i < recordsForPartition.size(); i++) {
+spout.nextTuple();
+}
+
+ArgumentCaptor messageIds = 
ArgumentCaptor.forClass(KafkaSpoutMessageId.class);

[GitHub] storm pull request #2152: [STORM-2505] OffsetManager should account for offs...

2017-06-09 Thread srdo
Github user srdo commented on a diff in the pull request:

https://github.com/apache/storm/pull/2152#discussion_r121218000
  
--- Diff: 
external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
 ---
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2017 The Apache Software Foundation.
+ *
+ * Licensed 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 static 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.*;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.InOrder;
+
+public class KafkaSpoutCommitTest {
+
+private final long offsetCommitPeriodMs = 2_000;
+private final TopologyContext contextMock = 
mock(TopologyContext.class);
+private final SpoutOutputCollector collectorMock = 
mock(SpoutOutputCollector.class);
+private final Map conf = new HashMap<>();
+private final TopicPartition partition = new 
TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+private KafkaConsumer consumerMock;
+private KafkaSpout spout;
+private KafkaSpoutConfig spoutConfig;
+
+private void setupSpout(Set assignedPartitions) {
+spoutConfig = getKafkaSpoutConfigBuilder(-1)
+.setOffsetCommitPeriodMs(offsetCommitPeriodMs)
+.build();
+
+consumerMock = mock(KafkaConsumer.class);
+KafkaConsumerFactory consumerFactory = 
(kafkaSpoutConfig) -> consumerMock;
+
+//Set up a spout listening to 1 topic partition
+spout = new KafkaSpout<>(spoutConfig, consumerFactory);
+
+spout.open(conf, contextMock, collectorMock);
+spout.activate();
+
+ArgumentCaptor rebalanceListenerCapture 
= ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
+verify(consumerMock).subscribe(anyCollection(), 
rebalanceListenerCapture.capture());
+
+//Assign partitions to the spout
+ConsumerRebalanceListener consumerRebalanceListener = 
rebalanceListenerCapture.getValue();
+consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
+}
+
+@Test
+public void testCommitSuccessWithOffsetVoids() {
+//Verify that the commit logic can handle offset voids
+try (SimulatedTime simulatedTime = new SimulatedTime()) {
+setupSpout(Collections.singleton(partition));
+Map>> 
records = new HashMap<>();
+List> recordsForPartition = new 
ArrayList<>();
+// Offsets emitted are 0,1,2,3,4,...,8,9
+for (int i = 0; i < 5; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+for (int i = 8; i < 10; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+records.put(partition, recordsForPartition);
+
+when(consumerMock.poll(anyLong()))
+.thenReturn(new ConsumerRecords(records));
+
+for (int i = 0; i < recordsForPartition.size(); i++) {
+spout.nextTuple();
+}
+
+ArgumentCaptor messageIds = 
ArgumentCaptor.forClass(KafkaSpoutMessageId.class);

[GitHub] storm pull request #2152: [STORM-2505] OffsetManager should account for offs...

2017-06-09 Thread srdo
Github user srdo commented on a diff in the pull request:

https://github.com/apache/storm/pull/2152#discussion_r121218037
  
--- Diff: 
external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutCommitTest.java
 ---
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2017 The Apache Software Foundation.
+ *
+ * Licensed 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 static 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfigBuilder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.*;
+import org.apache.kafka.common.TopicPartition;
+import 
org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration;
+import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Time.SimulatedTime;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.InOrder;
+
+public class KafkaSpoutCommitTest {
+
+private final long offsetCommitPeriodMs = 2_000;
+private final TopologyContext contextMock = 
mock(TopologyContext.class);
+private final SpoutOutputCollector collectorMock = 
mock(SpoutOutputCollector.class);
+private final Map conf = new HashMap<>();
+private final TopicPartition partition = new 
TopicPartition(SingleTopicKafkaSpoutConfiguration.TOPIC, 1);
+private KafkaConsumer consumerMock;
+private KafkaSpout spout;
+private KafkaSpoutConfig spoutConfig;
+
+private void setupSpout(Set assignedPartitions) {
+spoutConfig = getKafkaSpoutConfigBuilder(-1)
+.setOffsetCommitPeriodMs(offsetCommitPeriodMs)
+.build();
+
+consumerMock = mock(KafkaConsumer.class);
+KafkaConsumerFactory consumerFactory = 
(kafkaSpoutConfig) -> consumerMock;
+
+//Set up a spout listening to 1 topic partition
+spout = new KafkaSpout<>(spoutConfig, consumerFactory);
+
+spout.open(conf, contextMock, collectorMock);
+spout.activate();
+
+ArgumentCaptor rebalanceListenerCapture 
= ArgumentCaptor.forClass(ConsumerRebalanceListener.class);
+verify(consumerMock).subscribe(anyCollection(), 
rebalanceListenerCapture.capture());
+
+//Assign partitions to the spout
+ConsumerRebalanceListener consumerRebalanceListener = 
rebalanceListenerCapture.getValue();
+consumerRebalanceListener.onPartitionsAssigned(assignedPartitions);
+}
+
+@Test
+public void testCommitSuccessWithOffsetVoids() {
+//Verify that the commit logic can handle offset voids
+try (SimulatedTime simulatedTime = new SimulatedTime()) {
+setupSpout(Collections.singleton(partition));
+Map>> 
records = new HashMap<>();
+List> recordsForPartition = new 
ArrayList<>();
+// Offsets emitted are 0,1,2,3,4,...,8,9
+for (int i = 0; i < 5; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+for (int i = 8; i < 10; i++) {
+recordsForPartition.add(new 
ConsumerRecord(partition.topic(), partition.partition(), i, "key", "value"));
+}
+records.put(partition, recordsForPartition);
+
+when(consumerMock.poll(anyLong()))
+.thenReturn(new ConsumerRecords(records));
+
+for (int i = 0; i < recordsForPartition.size(); i++) {
+spout.nextTuple();
+}
+
+ArgumentCaptor messageIds = 
ArgumentCaptor.forClass(KafkaSpoutMessageId.class);

[GitHub] storm pull request #2152: [STORM-2505] OffsetManager should account for offs...

2017-06-09 Thread srdo
Github user srdo commented on a diff in the pull request:

https://github.com/apache/storm/pull/2152#discussion_r121221199
  
--- Diff: 
external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java
 ---
@@ -104,6 +104,10 @@ public KafkaSpout(KafkaSpoutConfig 
kafkaSpoutConfig) {
 this.kafkaSpoutConfig = kafkaSpoutConfig;
 }
 
+long getNumUncommittedOffsets() {
--- End diff --

I'm wondering if we can avoid inspecting this directly in the test by unit 
testing OffsetManager instead of indirectly testing it via the KafkaSpout? 
Testing that the spout will commit past gaps in the tuple stream is a good 
idea, but the check of this field is actually testing that OffsetManager.commit 
isn't buggy, which I feel belongs in a test for that class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---