[jira] [Created] (KAFKA-1640) consumer will fail if encounter special offset

2014-09-18 Thread chenshangan (JIRA)
chenshangan created KAFKA-1640:
--

 Summary: consumer will fail if encounter special offset
 Key: KAFKA-1640
 URL: https://issues.apache.org/jira/browse/KAFKA-1640
 Project: Kafka
  Issue Type: Bug
  Components: consumer
Affects Versions: 0.8.1.1
Reporter: chenshangan
Assignee: Neha Narkhede


I'm running topology on storm which consume messages from kafka. Sometimes I 
encounter exceptions like offset out of range.  I did not pay much attention on 
this problem before and just restarted the topology to consume messages from 
latest offset because of various reasons. Recently, more services are dependent 
on kafka, and this problem still exists.  According to the invalid offset 
information, I did a test and finally found the problem. There exists an offset 
which consumer can never start from and can not skip over it unless you provide 
next valid offset to start from.  I can not figure out why.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1640) consumer will fail if encounter special offset

2014-09-18 Thread chenshangan (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1640?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14138733#comment-14138733
 ] 

chenshangan commented on KAFKA-1640:


The following is about the test:

offset info within each partition:

-Topic[org.mobile] Offsets--
[partition_id:0, earliest:1771669312, latest:1868984348, count:97315036]
[partition_id:1, earliest:1768183261, latest:1866445674, count:98262413]
[partition_id:2, earliest:1772497970, latest:1870503255, count:98005285]
[partition_id:3, earliest:1773379686, latest:1874249045, count:100869359]
[partition_id:4, earliest:1770481630, latest:1869468413, count:98986783]
[partition_id:5, earliest:1773971554, latest:1872383470, count:98411916]
[partition_id:6, earliest:1767987875, latest:1867386661, count:99398786]
[partition_id:7, earliest:1772896791, latest:1871889556, count:98992765]
[partition_id:8, earliest:1767845437, latest:1866253739, count:98408302]
[partition_id:9, earliest:1768493678, latest:1865977579, count:97483901]
[partition_id:10, earliest:1767675631, latest:1865135380, count:97459749]
[partition_id:11, earliest:1766688851, latest:1863677692, count:96988841]
[partition_id:12, earliest:1773892262, latest:1872170855, count:98278593]
[partition_id:13, earliest:1774271586, latest:1873379949, count:99108363]
[partition_id:14, earliest:1767316658, latest:1865490492, count:98173834]
[partition_id:15, earliest:1775086549, latest:1875130663, count:100044114]


invalid offset info:
partition_id:5,  offset:1826483041

offset between the earliest offset and invalid offset : 1826483041 - 1773971554 
= 52511487

so if I consume messages before offset 52511475 relative to the earliest offset 
and after offset 52511475, it's ok. If I consume messages contain offset 
52511475, it will fail without any error message and you can never consume 
messages starting from this offset.

> consumer will fail if encounter special offset
> --
>
> Key: KAFKA-1640
> URL: https://issues.apache.org/jira/browse/KAFKA-1640
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8.1.1
>Reporter: chenshangan
>Assignee: Neha Narkhede
>
> I'm running topology on storm which consume messages from kafka. Sometimes I 
> encounter exceptions like offset out of range.  I did not pay much attention 
> on this problem before and just restarted the topology to consume messages 
> from latest offset because of various reasons. Recently, more services are 
> dependent on kafka, and this problem still exists.  According to the invalid 
> offset information, I did a test and finally found the problem. There exists 
> an offset which consumer can never start from and can not skip over it unless 
> you provide next valid offset to start from.  I can not figure out why.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: Review Request 22131: Patch for KAFKA-1477

2014-09-18 Thread Ivan Lyutov

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/22131/
---

(Updated Sept. 18, 2014, 12:43 p.m.)


Review request for kafka.


Bugs: KAFKA-1477
https://issues.apache.org/jira/browse/KAFKA-1477


Repository: kafka


Description (updated)
---

Minor fixes, cleanup

Refactoring

Fixed tests compilation error.

Updated according to requested changes: refactoring, minor edits.


Added basic functionality for new producer.


bug fixes after rebase


bug fix after rebase


Diffs (updated)
-

  clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
f58b8508d3f813a51015abed772c704390887d7e 
  clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
f9de4af426449cceca12a8de9a9f54a6241d28d8 
  
clients/src/main/java/org/apache/kafka/clients/producer/internals/SSLSocketChannel.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/common/errors/UnknownKeyStoreException.java
 PRE-CREATION 
  clients/src/main/java/org/apache/kafka/common/network/Selector.java 
4dd2cdf773f7eb01a93d7f994383088960303dfc 
  
clients/src/main/java/org/apache/kafka/common/network/security/AuthConfig.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/common/network/security/KeyStores.java 
PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/common/network/security/SecureAuth.java 
PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/common/network/security/StoreInitializer.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/common/network/security/store/JKSInitializer.java
 PRE-CREATION 
  clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 
5c5e3d40819e41cab7b52a0eeaee5f2e7317b7b3 
  config/client.keystore PRE-CREATION 
  config/client.public-key PRE-CREATION 
  config/client.security.properties PRE-CREATION 
  config/consumer.properties 83847de30d10b6e78bb8de28e0bb925d7c0e6ca2 
  config/producer.properties 39d65d7c6c21f4fccd7af89be6ca12a088d5dd98 
  config/server.keystore PRE-CREATION 
  config/server.properties 5c0905a572b1f0d8b07bfca967a09cb856a6b09f 
  config/server.public-key PRE-CREATION 
  config/server.security.properties PRE-CREATION 
  core/src/main/scala/kafka/api/FetchRequest.scala 
51cdccf7f90eb530cc62b094ed822b8469d50b12 
  core/src/main/scala/kafka/client/ClientUtils.scala 
ce7ede3f6d60e756e252257bd8c6fedc21f21e1c 
  core/src/main/scala/kafka/cluster/Broker.scala 
9407ed21fbbd57edeecd888edc32bea6a05d95b3 
  core/src/main/scala/kafka/common/UnknownKeyStoreException.scala PRE-CREATION 
  core/src/main/scala/kafka/consumer/ConsumerConfig.scala 
9ebbee6c16dc83767297c729d2d74ebbd063a993 
  core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala 
b9e2bea7b442a19bcebd1b350d39541a8c9dd068 
  core/src/main/scala/kafka/consumer/SimpleConsumer.scala 
8db9203d164a4a54f94d8d289e070a0f61e03ff9 
  core/src/main/scala/kafka/controller/ControllerChannelManager.scala 
ecbfa0f328ba6a652a758ab20cacef324a8b2fb8 
  core/src/main/scala/kafka/network/BlockingChannel.scala 
eb7bb14d94cb3648c06d4de36a3b34aacbde4556 
  core/src/main/scala/kafka/network/SocketServer.scala 
d67899080c21e0b6db84657d6845c7ef23b59b0e 
  core/src/main/scala/kafka/network/security/AuthConfig.scala PRE-CREATION 
  core/src/main/scala/kafka/network/security/KeyStores.scala PRE-CREATION 
  core/src/main/scala/kafka/network/security/SSLSocketChannel.scala 
PRE-CREATION 
  core/src/main/scala/kafka/network/security/SecureAuth.scala PRE-CREATION 
  core/src/main/scala/kafka/network/security/store/JKSInitializer.scala 
PRE-CREATION 
  core/src/main/scala/kafka/producer/ProducerConfig.scala 
3cdf23dce3407f1770b9c6543e3a8ae8ab3ff255 
  core/src/main/scala/kafka/producer/ProducerPool.scala 
43df70bb461dd3e385e6b20396adef3c4016a3fc 
  core/src/main/scala/kafka/producer/SyncProducer.scala 
489f0077512d9a69be81649c490274964290fa40 
  core/src/main/scala/kafka/producer/SyncProducerConfig.scala 
69b2d0c11bb1412ce76d566f285333c806be301a 
  core/src/main/scala/kafka/server/AbstractFetcherThread.scala 
2e9532e820b5b5c63dfd55f5454b32866d084a37 
  core/src/main/scala/kafka/server/KafkaConfig.scala 
dce48db175d6ea379f848a7768de0b1c8e4b929f 
  core/src/main/scala/kafka/server/KafkaHealthcheck.scala 
4acdd70fe9c1ee78d6510741006c2ece65450671 
  core/src/main/scala/kafka/server/KafkaServer.scala 
28711182aaa70eaa623de858bc063cb2613b2a4d 
  core/src/main/scala/kafka/tools/ConsoleConsumer.scala 
323fc8566d974acc4e5c7d7c2a065794f3b5df4a 
  core/src/main/scala/kafka/tools/ConsoleProducer.scala 
da4dad405c8d8f26a64cda78a292e1f5bfbdcc22 
  core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 
d1e7c434e77859d746b8dc68dd5d5a3740425e79 
  core/src/main/scala/kafka/tools/GetOffsetShell.scala 
9c6064e201eebbcd5b276a0dedd02937439edc94 
  core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 
af4783646803e58714770c21f8c3352370f26854

[jira] [Updated] (KAFKA-1477) add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication

2014-09-18 Thread Ivan Lyutov (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ivan Lyutov updated KAFKA-1477:
---
Attachment: KAFKA-1477_trunk.patch

Updated patch to current trunk

> add authentication layer and initial JKS x509 implementation for brokers, 
> producers and consumer for network communication
> --
>
> Key: KAFKA-1477
> URL: https://issues.apache.org/jira/browse/KAFKA-1477
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Joe Stein
>Assignee: Ivan Lyutov
> Fix For: 0.9.0
>
> Attachments: KAFKA-1477-binary.patch, KAFKA-1477.patch, 
> KAFKA-1477_2014-06-02_16:59:40.patch, KAFKA-1477_2014-06-02_17:24:26.patch, 
> KAFKA-1477_2014-06-03_13:46:17.patch, KAFKA-1477_trunk.patch
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (KAFKA-1419) cross build for scala 2.11

2014-09-18 Thread Ivan Lyutov (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ivan Lyutov updated KAFKA-1419:
---
Attachment: KAFKA-1419_cross_build_for_scala_2_11_for_0_8_1_branch.patch

Added the same patch for 0.8.1 branch

> cross build for scala 2.11
> --
>
> Key: KAFKA-1419
> URL: https://issues.apache.org/jira/browse/KAFKA-1419
> Project: Kafka
>  Issue Type: Improvement
>  Components: clients
>Affects Versions: 0.8.1
>Reporter: Scott Clasen
>Assignee: Ivan Lyutov
>Priority: Blocker
> Fix For: 0.8.2, 0.8.1.2
>
> Attachments: KAFKA-1419-scalaBinaryVersion.patch, 
> KAFKA-1419-scalaBinaryVersion.patch, KAFKA-1419.patch, KAFKA-1419.patch, 
> KAFKA-1419_2014-07-28_15:05:16.patch, KAFKA-1419_2014-07-29_15:13:43.patch, 
> KAFKA-1419_2014-08-04_14:43:26.patch, KAFKA-1419_2014-08-05_12:51:16.patch, 
> KAFKA-1419_2014-08-07_10:17:34.patch, KAFKA-1419_2014-08-07_10:52:18.patch, 
> KAFKA-1419_cross_build_for_scala_2_11_for_0_8_1_branch.patch
>
>
> Please publish builds for scala 2.11, hopefully just needs a small tweak to 
> the gradle conf?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-18 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14138972#comment-14138972
 ] 

Neha Narkhede commented on KAFKA-1282:
--

Thanks for the updated patch. Overall, looks great. Few comments -
1. Can you rename initialNextIdleCloseCheckTimeValue to 
nextIdleCloseCheckTimeValue?
2. It will be easier to understand the code if we rename currentTime to 
currentTimeNanos.


> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1558) AdminUtils.deleteTopic does not work

2014-09-18 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14138979#comment-14138979
 ] 

Neha Narkhede commented on KAFKA-1558:
--

[~sriharsha] Delete topic shouldn't be affected by the replica fetcher issue. 
Have you let it stay as is and see if delete topic eventually completes? If 
not, have you taken a thread dump to see where it halts, if at all? 

For each of the tests above, an important scenario is a large number of topics 
being actively written to and read from. For the tests, it is important to 
create, say a 1000 topics, and run producers in parallel that write to those 
1000 topics, consumers that consume and then introduce each of the failures 
above. Delete topic should succeed even if there are writes/reads happening to 
the topic.

> AdminUtils.deleteTopic does not work
> 
>
> Key: KAFKA-1558
> URL: https://issues.apache.org/jira/browse/KAFKA-1558
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1.1
>Reporter: Henning Schmiedehausen
>Assignee: Sriharsha Chintalapani
>Priority: Blocker
> Fix For: 0.8.2
>
>
> the AdminUtils:.deleteTopic method is implemented as
> {code}
> def deleteTopic(zkClient: ZkClient, topic: String) {
> ZkUtils.createPersistentPath(zkClient, 
> ZkUtils.getDeleteTopicPath(topic))
> }
> {code}
> but the DeleteTopicCommand actually does
> {code}
> zkClient = new ZkClient(zkConnect, 3, 3, ZKStringSerializer)
> zkClient.deleteRecursive(ZkUtils.getTopicPath(topic))
> {code}
> so I guess, that the 'createPersistentPath' above should actually be 
> {code}
> def deleteTopic(zkClient: ZkClient, topic: String) {
> ZkUtils.deletePathRecursive(zkClient, ZkUtils.getTopicPath(topic))
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1558) AdminUtils.deleteTopic does not work

2014-09-18 Thread Sriharsha Chintalapani (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14139025#comment-14139025
 ] 

Sriharsha Chintalapani commented on KAFKA-1558:
---

[~nehanarkhede] Should we also consider the topic's log size. I'll work on 
running the tests with 1000 topics and post results today or early tomorrow. 
Thanks.

> AdminUtils.deleteTopic does not work
> 
>
> Key: KAFKA-1558
> URL: https://issues.apache.org/jira/browse/KAFKA-1558
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.1.1
>Reporter: Henning Schmiedehausen
>Assignee: Sriharsha Chintalapani
>Priority: Blocker
> Fix For: 0.8.2
>
>
> the AdminUtils:.deleteTopic method is implemented as
> {code}
> def deleteTopic(zkClient: ZkClient, topic: String) {
> ZkUtils.createPersistentPath(zkClient, 
> ZkUtils.getDeleteTopicPath(topic))
> }
> {code}
> but the DeleteTopicCommand actually does
> {code}
> zkClient = new ZkClient(zkConnect, 3, 3, ZKStringSerializer)
> zkClient.deleteRecursive(ZkUtils.getTopicPath(topic))
> {code}
> so I guess, that the 'createPersistentPath' above should actually be 
> {code}
> def deleteTopic(zkClient: ZkClient, topic: String) {
> ZkUtils.deletePathRecursive(zkClient, ZkUtils.getTopicPath(topic))
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1618) Exception thrown when running console producer with no port number for the broker

2014-09-18 Thread BalajiSeshadri (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1618?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14139038#comment-14139038
 ] 

BalajiSeshadri commented on KAFKA-1618:
---

[~nehanarkhede] I submitted patch(KAFKA-1618-ALL.patch) for the tools listed by 
you.

Please review and let me know if you have more to fix.



> Exception thrown when running console producer with no port number for the 
> broker
> -
>
> Key: KAFKA-1618
> URL: https://issues.apache.org/jira/browse/KAFKA-1618
> Project: Kafka
>  Issue Type: Improvement
>Affects Versions: 0.8.1.1
>Reporter: Gwen Shapira
>Assignee: Balaji Seshadri
>  Labels: newbie
> Fix For: 0.8.2
>
> Attachments: KAFKA-1618-ALL.patch, KAFKA-1618.patch
>
>
> When running console producer with just "localhost" as the broker list, I get 
> ArrayIndexOutOfBounds exception.
> I expect either a clearer error about arguments or for the producer to 
> "guess" a default port.
> [root@shapira-1 bin]# ./kafka-console-producer.sh  --topic rufus1 
> --broker-list localhost
> java.lang.ArrayIndexOutOfBoundsException: 1
>   at 
> kafka.client.ClientUtils$$anonfun$parseBrokerList$1.apply(ClientUtils.scala:102)
>   at 
> kafka.client.ClientUtils$$anonfun$parseBrokerList$1.apply(ClientUtils.scala:97)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
>   at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
>   at 
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
>   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
>   at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:105)
>   at kafka.client.ClientUtils$.parseBrokerList(ClientUtils.scala:97)
>   at 
> kafka.producer.BrokerPartitionInfo.(BrokerPartitionInfo.scala:32)
>   at 
> kafka.producer.async.DefaultEventHandler.(DefaultEventHandler.scala:41)
>   at kafka.producer.Producer.(Producer.scala:59)
>   at kafka.producer.ConsoleProducer$.main(ConsoleProducer.scala:158)
>   at kafka.producer.ConsoleProducer.main(ConsoleProducer.scala)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1640) consumer will fail if encounter special offset

2014-09-18 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1640?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14139142#comment-14139142
 ] 

Guozhang Wang commented on KAFKA-1640:
--

Can you turn on log4j on the consumer is see what error / exception was thrown 
internally?

> consumer will fail if encounter special offset
> --
>
> Key: KAFKA-1640
> URL: https://issues.apache.org/jira/browse/KAFKA-1640
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8.1.1
>Reporter: chenshangan
>Assignee: Neha Narkhede
>
> I'm running topology on storm which consume messages from kafka. Sometimes I 
> encounter exceptions like offset out of range.  I did not pay much attention 
> on this problem before and just restarted the topology to consume messages 
> from latest offset because of various reasons. Recently, more services are 
> dependent on kafka, and this problem still exists.  According to the invalid 
> offset information, I did a test and finally found the problem. There exists 
> an offset which consumer can never start from and can not skip over it unless 
> you provide next valid offset to start from.  I can not figure out why.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1591) Clean-up Unnecessary stack trace in error/warn logs

2014-09-18 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14139192#comment-14139192
 ] 

Guozhang Wang commented on KAFKA-1591:
--

Thanks for the patch, a few more comments below:

2.0. The patch cannot be applied to latest trunk, could you rebase?

2.1. In ClientUtils, could you also print the e.toString?

2.2. In in ReplicaManager, it may be useful to keep the stack trace for "Error 
on broker %d while processing LeaderAndIsr ...". The reasons is that this error 
may be thrown from different underlying modules, or even a non-Kafka exception 
(we are catching Throwable here). And the upper level KafkaApis will not log 
the stack trace either when it catches this re-thrown error.

> Clean-up Unnecessary stack trace in error/warn logs
> ---
>
> Key: KAFKA-1591
> URL: https://issues.apache.org/jira/browse/KAFKA-1591
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>Assignee: Abhishek Sharma
>  Labels: newbie
> Fix For: 0.9.0
>
> Attachments: Jira-1591- Complete Changes Patch.patch, 
> Jira-1591-SocketConnection-Warning.patch, 
> Jira1591-SendProducerRequest-Warning.patch
>
>
> Some of the unnecessary stack traces in error / warning log entries can 
> easily pollute the log files. Examples include KAFKA-1066, etc.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (KAFKA-1592) Some INFO level logging needs to be DEBUG

2014-09-18 Thread Mayuresh Gharat (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Mayuresh Gharat resolved KAFKA-1592.

Resolution: Duplicate
  Reviewer: Guozhang Wang

Closing a duplicate as the changes were taken care of in 1591

> Some INFO level logging needs to be DEBUG
> -
>
> Key: KAFKA-1592
> URL: https://issues.apache.org/jira/browse/KAFKA-1592
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>Assignee: Mayuresh Gharat
>  Labels: newbie
> Fix For: 0.9.0
>
>
> Some of the INFO level log4j entries are not really useful, for example in 
> SocketServer.Processor, due to metadata requests that reply on a separate and 
> short-lived socket, the following log can be constantly printed:
> info("Closing socket connection to 
> %s.".format(channelFor(key).socket.getInetAddress)) 
> We'd better move them to DEBUG if they are expected in normal state.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-18 Thread nicu marasoiu (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

nicu marasoiu updated KAFKA-1282:
-
Attachment: 1282_brush.patch

attached, renamed time and for the "initial/reset value of the nextIdleCheck", 
i just inlined the function, the code is more clear like this i think

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brush.patch, 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-18 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14139513#comment-14139513
 ] 

Jun Rao commented on KAFKA-1282:


Do you think you can reproduce that data loss issue in 1 out of your 7 tests? 
With ack=1 and retries, this shouldn't happen. Perhaps it's useful to enable 
the trace logging in the producer to see what's exactly happening there.

Could you also do the same test by enabling the new producer in console 
producer?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brush.patch, 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1123) Broker IPv6 addresses parsed incorrectly

2014-09-18 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14139537#comment-14139537
 ] 

Jun Rao commented on KAFKA-1123:


Thanks for the latest patch. In Broker, you used the simple string interpolator 
like the following. So far, we haven't adopted this syntax. Perhaps we can 
leave out those changes for now for consistency?

s"Broker id $id does not exist"

> Broker IPv6 addresses parsed incorrectly
> 
>
> Key: KAFKA-1123
> URL: https://issues.apache.org/jira/browse/KAFKA-1123
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Andrew Otto
>  Labels: newbie
> Attachments: KAFKA-1123.patch, KAFKA-1123_v2.patch, 
> KAFKA-1123_v3.patch
>
>
> It seems that broker addresses are parsed incorrectly when IPv6 addresses are 
> supplied.  IPv6 addresses have colons in them, and Kafka seems to be 
> interpreting the first : as the address:port separator.
> I have only tried this with the console-producer --broker-list option, so I 
> don't know if this affects anything deeper than the CLI.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (KAFKA-1123) Broker IPv6 addresses parsed incorrectly

2014-09-18 Thread JIRA

 [ 
https://issues.apache.org/jira/browse/KAFKA-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Krzysztof Szafrański updated KAFKA-1123:

Attachment: KAFKA-1123_v4.patch

I have now removed the string interpolation syntax.

> Broker IPv6 addresses parsed incorrectly
> 
>
> Key: KAFKA-1123
> URL: https://issues.apache.org/jira/browse/KAFKA-1123
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Andrew Otto
>  Labels: newbie
> Attachments: KAFKA-1123.patch, KAFKA-1123_v2.patch, 
> KAFKA-1123_v3.patch, KAFKA-1123_v4.patch
>
>
> It seems that broker addresses are parsed incorrectly when IPv6 addresses are 
> supplied.  IPv6 addresses have colons in them, and Kafka seems to be 
> interpreting the first : as the address:port separator.
> I have only tried this with the console-producer --broker-list option, so I 
> don't know if this affects anything deeper than the CLI.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (KAFKA-1123) Broker IPv6 addresses parsed incorrectly

2014-09-18 Thread Jun Rao (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jun Rao updated KAFKA-1123:
---
   Resolution: Fixed
Fix Version/s: 0.8.2
 Assignee: Krzysztof Szafrański
   Status: Resolved  (was: Patch Available)

Thanks for patch v4. +1 and committed to trunk.

> Broker IPv6 addresses parsed incorrectly
> 
>
> Key: KAFKA-1123
> URL: https://issues.apache.org/jira/browse/KAFKA-1123
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Andrew Otto
>Assignee: Krzysztof Szafrański
>  Labels: newbie
> Fix For: 0.8.2
>
> Attachments: KAFKA-1123.patch, KAFKA-1123_v2.patch, 
> KAFKA-1123_v3.patch, KAFKA-1123_v4.patch
>
>
> It seems that broker addresses are parsed incorrectly when IPv6 addresses are 
> supplied.  IPv6 addresses have colons in them, and Kafka seems to be 
> interpreting the first : as the address:port separator.
> I have only tried this with the console-producer --broker-list option, so I 
> don't know if this affects anything deeper than the CLI.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (KAFKA-1641) Log cleaner exits if last cleaned offset is lower than earliest offset

2014-09-18 Thread Joel Koshy (JIRA)
Joel Koshy created KAFKA-1641:
-

 Summary: Log cleaner exits if last cleaned offset is lower than 
earliest offset
 Key: KAFKA-1641
 URL: https://issues.apache.org/jira/browse/KAFKA-1641
 Project: Kafka
  Issue Type: Bug
Affects Versions: 0.8.1.1
Reporter: Joel Koshy


Encountered this recently: the log cleaner exited a while ago (I think because 
the topic had compressed messages). That issue was subsequently addressed by 
having the producer only send uncompressed. However, on a subsequent restart of 
the broker we see this:

In this scenario I think it is reasonable to just emit a warning and have the 
cleaner round up its first dirty offset to the base offset of the first segment.

{code}
[kafka-server] [] [kafka-log-cleaner-thread-0], Error due to 
java.lang.IllegalArgumentException: requirement failed: Last clean offset is 
54770438 but segment base offset is 382844024 for log testtopic-0.
at scala.Predef$.require(Predef.scala:145)
at kafka.log.Cleaner.buildOffsetMap(LogCleaner.scala:491)
at kafka.log.Cleaner.clean(LogCleaner.scala:288)
at kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:202)
at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:187)
at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51)
{code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Jenkins build is back to normal : Kafka-trunk #269

2014-09-18 Thread Apache Jenkins Server
See 



[jira] [Commented] (KAFKA-1640) consumer will fail if encounter special offset

2014-09-18 Thread chenshangan (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1640?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14139928#comment-14139928
 ] 

chenshangan commented on KAFKA-1640:


[~guozhang] thanks for you suggestion,  it's because the size of that single 
message is bigger than the max buffer size. I will close this issue. 
btw, what do you mean of turning on log4j on the consumer and how? I use log4j 
from the whole client project, I think it should affect the whole client, but 
why not ?

> consumer will fail if encounter special offset
> --
>
> Key: KAFKA-1640
> URL: https://issues.apache.org/jira/browse/KAFKA-1640
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8.1.1
>Reporter: chenshangan
>Assignee: Neha Narkhede
>
> I'm running topology on storm which consume messages from kafka. Sometimes I 
> encounter exceptions like offset out of range.  I did not pay much attention 
> on this problem before and just restarted the topology to consume messages 
> from latest offset because of various reasons. Recently, more services are 
> dependent on kafka, and this problem still exists.  According to the invalid 
> offset information, I did a test and finally found the problem. There exists 
> an offset which consumer can never start from and can not skip over it unless 
> you provide next valid offset to start from.  I can not figure out why.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (KAFKA-1640) consumer will fail if encounter special offset

2014-09-18 Thread chenshangan (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1640?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

chenshangan resolved KAFKA-1640.

Resolution: Not a Problem

> consumer will fail if encounter special offset
> --
>
> Key: KAFKA-1640
> URL: https://issues.apache.org/jira/browse/KAFKA-1640
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8.1.1
>Reporter: chenshangan
>Assignee: Neha Narkhede
>
> I'm running topology on storm which consume messages from kafka. Sometimes I 
> encounter exceptions like offset out of range.  I did not pay much attention 
> on this problem before and just restarted the topology to consume messages 
> from latest offset because of various reasons. Recently, more services are 
> dependent on kafka, and this problem still exists.  According to the invalid 
> offset information, I did a test and finally found the problem. There exists 
> an offset which consumer can never start from and can not skip over it unless 
> you provide next valid offset to start from.  I can not figure out why.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (KAFKA-1642) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost

2014-09-18 Thread Bhavesh Mistry (JIRA)
Bhavesh Mistry created KAFKA-1642:
-

 Summary: [Java New Producer Kafka Trunk] CPU Usage Spike to 100% 
when network connection is lost
 Key: KAFKA-1642
 URL: https://issues.apache.org/jira/browse/KAFKA-1642
 Project: Kafka
  Issue Type: Bug
  Components: producer 
Affects Versions: 0.8.2
Reporter: Bhavesh Mistry
Assignee: Jun Rao


I see my CPU spike to 100% when network connection is lost for while.  It seems 
network  IO thread are very busy logging following error message.  Is this 
expected behavior ?
2014-09-17 14:06:16.830 [kafka-producer-network-thread] ERROR 
org.apache.kafka.clients.producer.internals.Sender - Uncaught error in kafka 
producer I/O thread: 

java.lang.IllegalStateException: No entry found for node -2

at 
org.apache.kafka.clients.ClusterConnectionStates.nodeState(ClusterConnectionStates.java:110)

at 
org.apache.kafka.clients.ClusterConnectionStates.disconnected(ClusterConnectionStates.java:99)

at 
org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:394)

at 
org.apache.kafka.clients.NetworkClient.maybeUpdateMetadata(NetworkClient.java:380)

at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:174)

at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:175)

at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:115)

at java.lang.Thread.run(Thread.java:744)

Thanks,

Bhavesh



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (KAFKA-1591) Clean-up Unnecessary stack trace in error/warn logs

2014-09-18 Thread Abhishek Sharma (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1591?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Abhishek Sharma updated KAFKA-1591:
---
Attachment: Jira-1591- As per Guozhang latest comment.patch

Contains combined one patch file - As per Guozhang latest comments.


> Clean-up Unnecessary stack trace in error/warn logs
> ---
>
> Key: KAFKA-1591
> URL: https://issues.apache.org/jira/browse/KAFKA-1591
> Project: Kafka
>  Issue Type: Bug
>Reporter: Guozhang Wang
>Assignee: Abhishek Sharma
>  Labels: newbie
> Fix For: 0.9.0
>
> Attachments: Jira-1591- As per Guozhang latest comment.patch, 
> Jira-1591- Complete Changes Patch.patch, 
> Jira-1591-SocketConnection-Warning.patch, 
> Jira1591-SendProducerRequest-Warning.patch
>
>
> Some of the unnecessary stack traces in error / warning log entries can 
> easily pollute the log files. Examples include KAFKA-1066, etc.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)