[jira] [Updated] (KAFKA-1812) Allow IpV6 in configuration with parseCsvMap

2014-12-12 Thread Joe Stein (JIRA)

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

Joe Stein updated KAFKA-1812:
-
Resolution: Fixed
Status: Resolved  (was: Patch Available)

+ 1 committed to trunk, thanks for the patch Jeff and the review Gwen!

  Allow IpV6 in configuration with parseCsvMap
 -

 Key: KAFKA-1812
 URL: https://issues.apache.org/jira/browse/KAFKA-1812
 Project: Kafka
  Issue Type: Bug
Reporter: Jeff Holoman
Assignee: Jeff Holoman
Priority: Minor
  Labels: newbie
 Fix For: 0.8.3

 Attachments: KAFKA-1812_2014-12-10_21:38:59.patch


 The current implementation of parseCsvMap in Utils expects k:v,k:v. This 
 modifies that function to accept a string with multiple : characters and 
 splitting on the last occurrence per pair. 
 This limitation is noted in the Reviewboard comments for KAFKA-1512



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


Build failed in Jenkins: Kafka-trunk #353

2014-12-12 Thread Apache Jenkins Server
See https://builds.apache.org/job/Kafka-trunk/353/changes

Changes:

[joe.stein] KAFKA-1812 Allow IpV6 in configuration with parseCsvMap patch by 
Jeff Holoman reviewed by Gwen Shapira and Joe Stein

--
[...truncated 1017 lines...]
kafka.producer.AsyncProducerTest  testNoBroker PASSED

kafka.producer.AsyncProducerTest  testIncompatibleEncoder PASSED

kafka.producer.AsyncProducerTest  testRandomPartitioner PASSED

kafka.producer.AsyncProducerTest  testFailedSendRetryLogic PASSED

kafka.producer.AsyncProducerTest  testJavaProducer PASSED

kafka.producer.AsyncProducerTest  testInvalidConfiguration PASSED

kafka.log.CleanerTest  testCleanSegments PASSED

kafka.log.CleanerTest  testCleaningWithDeletes PASSED

kafka.log.CleanerTest  testCleanSegmentsWithAbort PASSED

kafka.log.CleanerTest  testSegmentGrouping PASSED

kafka.log.CleanerTest  testBuildOffsetMap PASSED

kafka.log.LogManagerTest  testCreateLog PASSED

kafka.log.LogManagerTest  testGetNonExistentLog PASSED

kafka.log.LogManagerTest  testCleanupExpiredSegments PASSED

kafka.log.LogManagerTest  testCleanupSegmentsToMaintainSize PASSED

kafka.log.LogManagerTest  testTimeBasedFlush PASSED

kafka.log.LogManagerTest  testLeastLoadedAssignment PASSED

kafka.log.LogManagerTest  testTwoLogManagersUsingSameDirFails PASSED

kafka.log.LogManagerTest  testCheckpointRecoveryPoints PASSED

kafka.log.LogManagerTest  testRecoveryDirectoryMappingWithTrailingSlash PASSED

kafka.log.LogManagerTest  testRecoveryDirectoryMappingWithRelativeDirectory 
PASSED

kafka.log.LogConfigTest  testFromPropsDefaults PASSED

kafka.log.LogConfigTest  testFromPropsEmpty PASSED

kafka.log.LogConfigTest  testFromPropsToProps PASSED

kafka.log.LogConfigTest  testFromPropsInvalid PASSED

kafka.log.OffsetIndexTest  truncate PASSED

kafka.log.OffsetIndexTest  randomLookupTest PASSED

kafka.log.OffsetIndexTest  lookupExtremeCases PASSED

kafka.log.OffsetIndexTest  appendTooMany PASSED

kafka.log.OffsetIndexTest  appendOutOfOrder PASSED

kafka.log.OffsetIndexTest  testReopen PASSED

kafka.log.FileMessageSetTest  testWrittenEqualsRead PASSED

kafka.log.FileMessageSetTest  testIteratorIsConsistent PASSED

kafka.log.FileMessageSetTest  testSizeInBytes PASSED

kafka.log.FileMessageSetTest  testWriteTo PASSED

kafka.log.FileMessageSetTest  testFileSize PASSED

kafka.log.FileMessageSetTest  testIterationOverPartialAndTruncation PASSED

kafka.log.FileMessageSetTest  testIterationDoesntChangePosition PASSED

kafka.log.FileMessageSetTest  testRead PASSED

kafka.log.FileMessageSetTest  testSearch PASSED

kafka.log.FileMessageSetTest  testIteratorWithLimits PASSED

kafka.log.FileMessageSetTest  testTruncate PASSED

kafka.log.LogCleanerIntegrationTest  cleanerTest PASSED

kafka.log.OffsetMapTest  testBasicValidation PASSED

kafka.log.OffsetMapTest  testClear PASSED

kafka.log.LogTest  testTimeBasedLogRoll PASSED

kafka.log.LogTest  testTimeBasedLogRollJitter PASSED

kafka.log.LogTest  testSizeBasedLogRoll PASSED

kafka.log.LogTest  testLoadEmptyLog PASSED

kafka.log.LogTest  testAppendAndReadWithSequentialOffsets PASSED

kafka.log.LogTest  testAppendAndReadWithNonSequentialOffsets PASSED

kafka.log.LogTest  testReadAtLogGap PASSED

kafka.log.LogTest  testReadOutOfRange PASSED

kafka.log.LogTest  testLogRolls PASSED

kafka.log.LogTest  testCompressedMessages PASSED

kafka.log.LogTest  testThatGarbageCollectingSegmentsDoesntChangeOffset PASSED

kafka.log.LogTest  testMessageSetSizeCheck PASSED

kafka.log.LogTest  testMessageSizeCheck PASSED

kafka.log.LogTest  testLogRecoversToCorrectOffset PASSED

kafka.log.LogTest  testIndexRebuild PASSED

kafka.log.LogTest  testTruncateTo PASSED

kafka.log.LogTest  testIndexResizingAtTruncation PASSED

kafka.log.LogTest  testBogusIndexSegmentsAreRemoved PASSED

kafka.log.LogTest  testReopenThenTruncate PASSED

kafka.log.LogTest  testAsyncDelete PASSED

kafka.log.LogTest  testOpenDeletesObsoleteFiles PASSED

kafka.log.LogTest  testAppendMessageWithNullPayload PASSED

kafka.log.LogTest  testCorruptLog PASSED

kafka.log.LogTest  testCleanShutdownFile PASSED

kafka.log.LogSegmentTest  testTruncate PASSED

kafka.log.LogSegmentTest  testReadOnEmptySegment PASSED

kafka.log.LogSegmentTest  testReadBeforeFirstOffset PASSED

kafka.log.LogSegmentTest  testMaxOffset PASSED

kafka.log.LogSegmentTest  testReadAfterLast PASSED

kafka.log.LogSegmentTest  testReadFromGap PASSED

kafka.log.LogSegmentTest  testTruncateFull PASSED

kafka.log.LogSegmentTest  testNextOffsetCalculation PASSED

kafka.log.LogSegmentTest  testChangeFileSuffixes PASSED

kafka.log.LogSegmentTest  testRecoveryFixesCorruptIndex PASSED

kafka.log.LogSegmentTest  testRecoveryWithCorruptMessage PASSED

kafka.api.ApiUtilsTest  testShortStringNonASCII PASSED

kafka.api.ApiUtilsTest  testShortStringASCII PASSED

kafka.api.RequestResponseSerializationTest  
testSerializationAndDeserialization PASSED

kafka.api.test.ProducerSendTest  testAutoCreateTopic PASSED


[jira] [Commented] (KAFKA-1811) ensuring registered broker host:port is unique

2014-12-12 Thread Dave Parfitt (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244175#comment-14244175
 ] 

Dave Parfitt commented on KAFKA-1811:
-

Hi Gwen -
  Thanks so much for taking the time to review my patch and pointing me in the 
right direction.

 2. We also like unit tests 
Absolutely. I wanted to make sure I was going down the correct path first.

 3. I'd consider pushing this check down to registerBrokerInZk. It seems like 
 a natural place to ensure uniqueness of registered brokers before 
 registering.
My thinking was that I wanted to detect non-unique host:port combinations 
before the SocketServer started, however it probably
makes more sense in registerBrokerInZk as you suggest.

 4. Another thing to consider is race conditions - what if new broker 
 registers while we are checking? Perhaps we can even use ZK itself to 
 enforce uniqueness?
I should have thought of this, is there any code in Kafka that you recommend I 
model this after?

Cheers -
Dave

 ensuring registered broker host:port is unique
 --

 Key: KAFKA-1811
 URL: https://issues.apache.org/jira/browse/KAFKA-1811
 Project: Kafka
  Issue Type: Improvement
Reporter: Jun Rao
  Labels: newbie
 Attachments: KAFKA_1811.patch


 Currently, we expect each of the registered broker to have a unique host:port 
 pair. However, we don't enforce that, which causes various weird problems. It 
 would be useful to ensure this during broker registration.



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


[jira] [Commented] (KAFKA-1811) ensuring registered broker host:port is unique

2014-12-12 Thread Gwen Shapira (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244220#comment-14244220
 ] 

Gwen Shapira commented on KAFKA-1811:
-

#3 - I think the main issue here is with the ZK registration. If the 
SocketServer fails to start, there will be no registration and therefore no 
uniqueness issue. Perhaps [~junrao] can confirm.

#4 - Perhaps check how we maintain broker ID uniqueness? I think that just 
creating a znode in ZK with the id as the name will force ZK to guarantee 
uniqueness for us (by failing to write if the znode exists), but perhaps there 
is more involved. Anyway, may be a good place to start.
[~nehanarkhede] may have better suggestions here.


 ensuring registered broker host:port is unique
 --

 Key: KAFKA-1811
 URL: https://issues.apache.org/jira/browse/KAFKA-1811
 Project: Kafka
  Issue Type: Improvement
Reporter: Jun Rao
  Labels: newbie
 Attachments: KAFKA_1811.patch


 Currently, we expect each of the registered broker to have a unique host:port 
 pair. However, we don't enforce that, which causes various weird problems. It 
 would be useful to ensure this during broker registration.



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


[jira] [Updated] (KAFKA-1815) ServerShutdownTest fails in trunk.

2014-12-12 Thread Joe Stein (JIRA)

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

Joe Stein updated KAFKA-1815:
-
Fix Version/s: 0.8.3

CI says it is still broken https://builds.apache.org/view/All/job/Kafka-trunk/ 
and it was broken for me when I did that commit... I didn't see this ticket 
until just now will look through it later when I have some time towards fixing 
this, I mentioned KAFKA-1650 about it also

 ServerShutdownTest fails in trunk.
 --

 Key: KAFKA-1815
 URL: https://issues.apache.org/jira/browse/KAFKA-1815
 Project: Kafka
  Issue Type: Bug
Reporter: Anatoly Fayngelerin
Priority: Minor
 Fix For: 0.8.3

 Attachments: shutdown_test_fix.patch


 I ran into these failures consistently when trying to build Kafka locally:
 kafka.server.ServerShutdownTest  testCleanShutdown FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdown(ServerShutdownTest.scala:101)
 kafka.server.ServerShutdownTest  testCleanShutdownWithDeleteTopicEnabled 
 FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdownWithDeleteTopicEnabled(ServerShutdownTest.scala:114)
 kafka.server.ServerShutdownTest  testCleanShutdownAfterFailedStartup FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdownAfterFailedStartup(ServerShutdownTest.scala:141)
 It looks like Jenkins also had issues with these tests:
 https://builds.apache.org/job/Kafka-trunk/351/console
 I would like to provide a patch that fixes this.



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


[jira] [Commented] (KAFKA-1815) ServerShutdownTest fails in trunk.

2014-12-12 Thread Chris Cope (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1815?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244393#comment-14244393
 ] 

Chris Cope commented on KAFKA-1815:
---

I spoke too soon. I'm not getting different test results after running this a 
bunch of times on our test farm. Sometimes the 3 tests on this ticket fail. 
Sometimes testMetricsLeak fails. Sometimes all 4 fail together.

 ServerShutdownTest fails in trunk.
 --

 Key: KAFKA-1815
 URL: https://issues.apache.org/jira/browse/KAFKA-1815
 Project: Kafka
  Issue Type: Bug
Reporter: Anatoly Fayngelerin
Priority: Minor
 Fix For: 0.8.3

 Attachments: shutdown_test_fix.patch


 I ran into these failures consistently when trying to build Kafka locally:
 kafka.server.ServerShutdownTest  testCleanShutdown FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdown(ServerShutdownTest.scala:101)
 kafka.server.ServerShutdownTest  testCleanShutdownWithDeleteTopicEnabled 
 FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdownWithDeleteTopicEnabled(ServerShutdownTest.scala:114)
 kafka.server.ServerShutdownTest  testCleanShutdownAfterFailedStartup FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdownAfterFailedStartup(ServerShutdownTest.scala:141)
 It looks like Jenkins also had issues with these tests:
 https://builds.apache.org/job/Kafka-trunk/351/console
 I would like to provide a patch that fixes this.



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


[jira] [Comment Edited] (KAFKA-1815) ServerShutdownTest fails in trunk.

2014-12-12 Thread Chris Cope (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1815?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244393#comment-14244393
 ] 

Chris Cope edited comment on KAFKA-1815 at 12/12/14 4:20 PM:
-

I spoke too soon. I'm now getting different test results after running this a 
bunch of times on our test farm. Sometimes the 3 tests on this ticket fail. 
Sometimes testMetricsLeak fails. Sometimes all 4 fail together.


was (Author: copester):
I spoke too soon. I'm not getting different test results after running this a 
bunch of times on our test farm. Sometimes the 3 tests on this ticket fail. 
Sometimes testMetricsLeak fails. Sometimes all 4 fail together.

 ServerShutdownTest fails in trunk.
 --

 Key: KAFKA-1815
 URL: https://issues.apache.org/jira/browse/KAFKA-1815
 Project: Kafka
  Issue Type: Bug
Reporter: Anatoly Fayngelerin
Priority: Minor
 Fix For: 0.8.3

 Attachments: shutdown_test_fix.patch


 I ran into these failures consistently when trying to build Kafka locally:
 kafka.server.ServerShutdownTest  testCleanShutdown FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdown(ServerShutdownTest.scala:101)
 kafka.server.ServerShutdownTest  testCleanShutdownWithDeleteTopicEnabled 
 FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdownWithDeleteTopicEnabled(ServerShutdownTest.scala:114)
 kafka.server.ServerShutdownTest  testCleanShutdownAfterFailedStartup FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdownAfterFailedStartup(ServerShutdownTest.scala:141)
 It looks like Jenkins also had issues with these tests:
 https://builds.apache.org/job/Kafka-trunk/351/console
 I would like to provide a patch that fixes this.



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


[jira] [Commented] (KAFKA-1273) Brokers should make sure replica.fetch.max.bytes = message.max.bytes

2014-12-12 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244556#comment-14244556
 ] 

Jun Rao commented on KAFKA-1273:


I was thinking that one way is to just disallow customizing max.message.size 
per topic. Such customization may break downstream consumers like MirrorMaker. 
I am not sure if there is a strong use case for the per topic customization.

 Brokers should make sure replica.fetch.max.bytes = message.max.bytes
 -

 Key: KAFKA-1273
 URL: https://issues.apache.org/jira/browse/KAFKA-1273
 Project: Kafka
  Issue Type: Bug
  Components: replication
Affects Versions: 0.8.0
Reporter: Dong Zhong
Assignee: Sriharsha Chintalapani
  Labels: newbie

 If message.max.bytes is larger than replica.fetch.max.bytes,followers can't 
 fetch data from the leader and will incur endless retry. And this may cause 
 high network traffic between followers and leaders.
 Brokers should make sure replica.fetch.max.bytes = message.max.bytes by 
 adding a sanity check, or throw an exception.



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


[jira] [Commented] (KAFKA-1512) Limit the maximum number of connections per ip address

2014-12-12 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1512?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244564#comment-14244564
 ] 

Jay Kreps commented on KAFKA-1512:
--

Ah, it looks like I didn't fully wire through the overrides from the config. 
That is a bug.

 Limit the maximum number of connections per ip address
 --

 Key: KAFKA-1512
 URL: https://issues.apache.org/jira/browse/KAFKA-1512
 Project: Kafka
  Issue Type: New Feature
Reporter: Jay Kreps
Assignee: Jay Kreps
 Fix For: 0.8.2

 Attachments: KAFKA-1512.patch, KAFKA-1512.patch, 
 KAFKA-1512_2014-07-03_15:17:55.patch, KAFKA-1512_2014-07-14_13:28:15.patch


 To protect against client connection leaks add a new configuration
   max.connections.per.ip
 that causes the SocketServer to enforce a limit on the maximum number of 
 connections from each InetAddress instance. For backwards compatibility this 
 will default to 2 billion.



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


[jira] [Commented] (KAFKA-727) broker can still expose uncommitted data to a consumer

2014-12-12 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244581#comment-14244581
 ] 

Jun Rao commented on KAFKA-727:
---

Is there an easy way to reproduce this issue? Thanks,

 broker can still expose uncommitted data to a consumer
 --

 Key: KAFKA-727
 URL: https://issues.apache.org/jira/browse/KAFKA-727
 Project: Kafka
  Issue Type: Bug
  Components: core
Affects Versions: 0.8.0
Reporter: Jun Rao
Assignee: Jay Kreps
Priority: Blocker
  Labels: p1
 Attachments: KAFKA-727-v1.patch


 Even after kafka-698 is fixed, we still see consumer clients occasionally see 
 uncommitted data. The following is how this can happen.
 1. In Log.read(), we pass in startOffset  HW and maxOffset = HW.
 2. Then we call LogSegment.read(), in which we call translateOffset on the 
 maxOffset. The offset doesn't exist and translateOffset returns null.
 3. Continue in LogSegment.read(), we then call messageSet.sizeInBytes() to 
 fetch and return the data.
 What can happen is that between step 2 and step 3, a new message is appended 
 to the log and is not committed yet. Now, we have exposed uncommitted data to 
 the client.



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


[jira] [Commented] (KAFKA-1811) ensuring registered broker host:port is unique

2014-12-12 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1811?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244603#comment-14244603
 ] 

Neha Narkhede commented on KAFKA-1811:
--

As [~gwenshap] points out correctly, the issue is during the zookeeper 
registration. While a broker checks for uniqueness, another broker could sneak 
in and register itself. The correct way of doing this is a little complicated. 
Basically, all brokers would have to register in a 2-step process - 1. Acquire 
a lock (zk path) in order to register itself. 2. If the lock is successfully 
acquired, proceed with registration if the host:port is unique. 

 ensuring registered broker host:port is unique
 --

 Key: KAFKA-1811
 URL: https://issues.apache.org/jira/browse/KAFKA-1811
 Project: Kafka
  Issue Type: Improvement
Reporter: Jun Rao
  Labels: newbie
 Attachments: KAFKA_1811.patch


 Currently, we expect each of the registered broker to have a unique host:port 
 pair. However, we don't enforce that, which causes various weird problems. It 
 would be useful to ensure this during broker registration.



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


Re: Review Request 27735: Patch for KAFKA-1173

2014-12-12 Thread Jun Rao

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



vagrant/README.md
https://reviews.apache.org/r/27735/#comment107788

Will need to add an Apache license header.


- Jun Rao


On Nov. 19, 2014, 12:01 a.m., Ewen Cheslack-Postava wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/27735/
 ---
 
 (Updated Nov. 19, 2014, 12:01 a.m.)
 
 
 Review request for kafka.
 
 
 Bugs: KAFKA-1173
 https://issues.apache.org/jira/browse/KAFKA-1173
 
 
 Repository: kafka
 
 
 Description
 ---
 
 Add basic EC2 support, cleaner Vagrantfile, README cleanup, etc.
 
 
 Better naming, hostmanager for routable VM names, vagrant-cachier to reduce 
 startup cost, cleanup provisioning scripts, initial support for multiple 
 zookeepers, general cleanup.
 
 
 Don't sync a few directories that aren't actually required on the server.
 
 
 Add generic worker node support.
 
 
 Default # of workers should be 0
 
 
 Add support for Zookeeper clusters.
 
 This requires us to split up allocating VMs and provisioning because Vagrant
 will run the provisioner for the first node before all nodes are allocated. 
 This
 leaves the first node running Zookeeper with unroutable peer hostnames which 
 it,
 for some reason, caches as unroutable. The cluster never properly finishes
 forming since the nodes are unable to open connections to nodes booted later
 than they were. The simple solution is to make sure all nodes are booted 
 before
 starting configuration so we have all the addresses and hostnames available 
 and
 routable.
 
 Fix AWS provider commands in Vagrant README.
 
 
 Addressing Joe's comments.
 
 
 Add support for EC2 VPC settings.
 
 
 Update Vagrant README to use --no-parallel when using EC2.
 
 There's an issue that causes Vagrant to hang when running in
 parallel. The last message is from vagrant-hostmanager, but it's not
 clear if it is the actual cause.
 
 Only setup private static IP addresses under VirtualBox.
 
 
 Diffs
 -
 
   .gitignore 99b32a6770e3da59bc0167d77d45ca339ac3dbbd 
   README.md 9aca90664b2a80a37125775ddbdea06ba6c53644 
   Vagrantfile PRE-CREATION 
   vagrant/README.md PRE-CREATION 
   vagrant/base.sh PRE-CREATION 
   vagrant/broker.sh PRE-CREATION 
   vagrant/zk.sh PRE-CREATION 
 
 Diff: https://reviews.apache.org/r/27735/diff/
 
 
 Testing
 ---
 
 
 Thanks,
 
 Ewen Cheslack-Postava
 




[jira] [Updated] (KAFKA-1807) Improve accuracy of ProducerPerformance target throughput

2014-12-12 Thread Jun Rao (JIRA)

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

Jun Rao updated KAFKA-1807:
---
Resolution: Fixed
Status: Resolved  (was: Patch Available)

 Improve accuracy of ProducerPerformance target throughput
 -

 Key: KAFKA-1807
 URL: https://issues.apache.org/jira/browse/KAFKA-1807
 Project: Kafka
  Issue Type: Improvement
  Components: clients, tools
Affects Versions: 0.8.1.1
Reporter: Ewen Cheslack-Postava
Assignee: Ewen Cheslack-Postava
Priority: Minor
 Fix For: 0.8.3

 Attachments: KAFKA-1807.patch


 The code in ProducerPerformance that tries to match a target throughput is 
 very inaccurate because it doesn't account for time spent sending messages. 
 Since we have to get the current time to timestamp the messages, we can be 
 much more accurate by computing the current rate over the entire run and only 
 add to the sleep deficit if we're above the target rate.



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


[jira] [Updated] (KAFKA-1813) Build fails for scala 2.9.2

2014-12-12 Thread Jun Rao (JIRA)

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

Jun Rao updated KAFKA-1813:
---
   Resolution: Fixed
Fix Version/s: 0.8.3
 Assignee: Anatoly Fayngelerin
   Status: Resolved  (was: Patch Available)

Thanks for the patch. The issue was introduced by KAFKA-1650. +1 and committed 
to trunk.

 Build fails for scala 2.9.2
 ---

 Key: KAFKA-1813
 URL: https://issues.apache.org/jira/browse/KAFKA-1813
 Project: Kafka
  Issue Type: Bug
  Components: build
Reporter: Anatoly Fayngelerin
Assignee: Anatoly Fayngelerin
Priority: Minor
 Fix For: 0.8.3

 Attachments: fix_2_9_2_build.patch


 Currently, in trunk, the 2.9.2 build fails with the following error:
 MirrorMaker.scala:507 overloaded method value commitOffsets with alternatives:
   (isAutoCommit: Boolean,topicPartitionOffsets: 
 scala.collection.immutable.Map[kafka.common.TopicAndPartition,kafka.common.OffsetAndMetadata])Unit
  and
   (isAutoCommit: Boolean)Unit and
   = Unit
  cannot be applied to (isAutoCommit: Boolean, 
 scala.collection.immutable.Map[kafka.common.TopicAndPartition,kafka.common.OffsetAndMetadata])
 connector.commitOffsets(isAutoCommit = false, offsetsToCommit)
 It looks like the 2.9.2 compiler cannot resolve an overloaded method when 
 mixing named and ordered parameters.
 I ran into this when I cloned the repo and ran ./gradlew test.



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


[jira] [Assigned] (KAFKA-1512) Limit the maximum number of connections per ip address

2014-12-12 Thread Jeff Holoman (JIRA)

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

Jeff Holoman reassigned KAFKA-1512:
---

Assignee: Jeff Holoman

 Limit the maximum number of connections per ip address
 --

 Key: KAFKA-1512
 URL: https://issues.apache.org/jira/browse/KAFKA-1512
 Project: Kafka
  Issue Type: New Feature
Reporter: Jay Kreps
Assignee: Jeff Holoman
 Fix For: 0.8.2

 Attachments: KAFKA-1512.patch, KAFKA-1512.patch, 
 KAFKA-1512_2014-07-03_15:17:55.patch, KAFKA-1512_2014-07-14_13:28:15.patch


 To protect against client connection leaks add a new configuration
   max.connections.per.ip
 that causes the SocketServer to enforce a limit on the maximum number of 
 connections from each InetAddress instance. For backwards compatibility this 
 will default to 2 billion.



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


[jira] [Commented] (KAFKA-1512) Limit the maximum number of connections per ip address

2014-12-12 Thread Jeff Holoman (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1512?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244729#comment-14244729
 ] 

Jeff Holoman commented on KAFKA-1512:
-

[~jkreps] I can fix this if you're ok with that.

 Limit the maximum number of connections per ip address
 --

 Key: KAFKA-1512
 URL: https://issues.apache.org/jira/browse/KAFKA-1512
 Project: Kafka
  Issue Type: New Feature
Reporter: Jay Kreps
 Fix For: 0.8.2

 Attachments: KAFKA-1512.patch, KAFKA-1512.patch, 
 KAFKA-1512_2014-07-03_15:17:55.patch, KAFKA-1512_2014-07-14_13:28:15.patch


 To protect against client connection leaks add a new configuration
   max.connections.per.ip
 that causes the SocketServer to enforce a limit on the maximum number of 
 connections from each InetAddress instance. For backwards compatibility this 
 will default to 2 billion.



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


[jira] [Created] (KAFKA-1817) AdminUtils.createTopic vs kafka-topics.sh --create with partitions

2014-12-12 Thread Jason Kania (JIRA)
Jason Kania created KAFKA-1817:
--

 Summary: AdminUtils.createTopic vs kafka-topics.sh --create with 
partitions
 Key: KAFKA-1817
 URL: https://issues.apache.org/jira/browse/KAFKA-1817
 Project: Kafka
  Issue Type: Bug
Affects Versions: 0.8.2
 Environment: debian linux current version  up to date
Reporter: Jason Kania


When topics are created using AdminUtils.createTopic in code, no partitions 
folder is created The zookeeper shell shows this.

ls /brokers/topics/foshizzle
[]

However, when kafka-topics.sh --create is run, the partitions folder is created:

ls /brokers/topics/foshizzle
[partitions]

The unfortunately useless error message KeeperErrorCode = NoNode for 
/brokers/topics/periodicReading/partitions makes it unclear what to do. When 
the topics are listed via kafka-topics.sh, they appear to have been created 
fine. It would be good if the exception was wrapped by Kafka to suggested 
looking in the zookeeper shell so a person didn't have to dig around to 
understand what the meaning of this path is...



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


[jira] [Commented] (KAFKA-1694) kafka command line and centralized operations

2014-12-12 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1694?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14245091#comment-14245091
 ] 

Jun Rao commented on KAFKA-1694:


Thanks for the patch. A few comments.

10. I still don't think having a single admin request/response is a good idea. 
Since the args for different sub admin requests are quite different, it's very 
hard to reason about what the format in args is. Ideally, we should be able to 
figure out the request format just from the protocol definition. It seems to me 
it's cleaner to just create the following standalone requests/responses.

1. create topic
2. alter topic
3. delete topic
4. list topic
5. describe topic (this will be used to replace TopicMetadataRequest eventually)
6. describe cluster (return all brokers and the controller)

11. We can reuse the java request objects in the scala request (see 
HeartbeatRequestAndHeader.scala as an example).

12. describe topic : I am not sure that we need to include the following 
options in the request. We can just return the info (replicas, isrs, etc) of 
all partitions and let the client decide what to do with them.
reportUnderReplicatedPartitions
reportUnavailablePartitions

Since this is a large patch, could you submit an RB using the patch review tool?

 kafka command line and centralized operations
 -

 Key: KAFKA-1694
 URL: https://issues.apache.org/jira/browse/KAFKA-1694
 Project: Kafka
  Issue Type: Bug
Reporter: Joe Stein
Priority: Critical
 Fix For: 0.8.3

 Attachments: KAFKA-1772_1802_1775_1774_v2.patch


 https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Command+Line+and+Related+Improvements



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


A plugin: Kafka Sink and Source for flume

2014-12-12 Thread Jack Ye
Does this work for kafka 0.8.1? thank you

https://github.com/baniuyao/flume-kafka



Re: A plugin: Kafka Sink and Source for flume

2014-12-12 Thread Gwen Shapira
I'd use the one in Flume trunk (it was far better tested) and we also
have Kafka-channel :)
And it works with 0.8.1.

https://github.com/apache/flume/tree/trunk/flume-ng-sources/flume-kafka-source
https://github.com/apache/flume/tree/trunk/flume-ng-sinks/flume-ng-kafka-sink
https://github.com/apache/flume/tree/trunk/flume-ng-channels/flume-kafka-channel

Gwen

On Fri, Dec 12, 2014 at 5:33 PM, Jack Ye jack...@jasper.com wrote:
 Does this work for kafka 0.8.1? thank you

 https://github.com/baniuyao/flume-kafka



Re: A plugin: Kafka Sink and Source for flume

2014-12-12 Thread Gwen Shapira
OK, I can't resist a bit more detail about my favorite project :)

Here's a good blog post on how to use the Flume-Kafka source, sink and channel:
http://blog.cloudera.com/blog/2014/11/flafka-apache-flume-meets-apache-kafka-for-event-processing/

One of the main differences from Baniuyao's version is that we support
committing offsets to zookeeper on every batch (rather than
automatically) and batch sizes are configurable so you can tune
throughput / latency / cpu utilization / zookeeper load, etc.

Gwen

On Fri, Dec 12, 2014 at 6:10 PM, Gwen Shapira gshap...@cloudera.com wrote:
 I'd use the one in Flume trunk (it was far better tested) and we also
 have Kafka-channel :)
 And it works with 0.8.1.

 https://github.com/apache/flume/tree/trunk/flume-ng-sources/flume-kafka-source
 https://github.com/apache/flume/tree/trunk/flume-ng-sinks/flume-ng-kafka-sink
 https://github.com/apache/flume/tree/trunk/flume-ng-channels/flume-kafka-channel

 Gwen

 On Fri, Dec 12, 2014 at 5:33 PM, Jack Ye jack...@jasper.com wrote:
 Does this work for kafka 0.8.1? thank you

 https://github.com/baniuyao/flume-kafka



[jira] [Commented] (KAFKA-1815) ServerShutdownTest fails in trunk.

2014-12-12 Thread Chris Cope (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1815?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14245115#comment-14245115
 ] 

Chris Cope commented on KAFKA-1815:
---

Thanks, [~junrao], though that was actually [~fanatoly]'s patch. In terms of 
current state of tests passing since the latest commit 
523b36589e942cb99a95debd2c45e795ae533d08 for KAFKA-1813, and I'm seeing 
consistent passing of all the tests except for the occasional KAFKA-1501 
failures which continue to haunt me. Thanks!

 ServerShutdownTest fails in trunk.
 --

 Key: KAFKA-1815
 URL: https://issues.apache.org/jira/browse/KAFKA-1815
 Project: Kafka
  Issue Type: Bug
Reporter: Anatoly Fayngelerin
Assignee: Chris Cope
Priority: Minor
 Fix For: 0.8.3

 Attachments: shutdown_test_fix.patch


 I ran into these failures consistently when trying to build Kafka locally:
 kafka.server.ServerShutdownTest  testCleanShutdown FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdown(ServerShutdownTest.scala:101)
 kafka.server.ServerShutdownTest  testCleanShutdownWithDeleteTopicEnabled 
 FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdownWithDeleteTopicEnabled(ServerShutdownTest.scala:114)
 kafka.server.ServerShutdownTest  testCleanShutdownAfterFailedStartup FAILED
 java.lang.NullPointerException
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest$$anonfun$verifyNonDaemonThreadsStatus$2.apply(ServerShutdownTest.scala:147)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:114)
 at 
 scala.collection.TraversableOnce$$anonfun$count$1.apply(TraversableOnce.scala:113)
 at 
 scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
 at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
 at 
 scala.collection.TraversableOnce$class.count(TraversableOnce.scala:113)
 at scala.collection.mutable.ArrayOps$ofRef.count(ArrayOps.scala:105)
 at 
 kafka.server.ServerShutdownTest.verifyNonDaemonThreadsStatus(ServerShutdownTest.scala:147)
 at 
 kafka.server.ServerShutdownTest.testCleanShutdownAfterFailedStartup(ServerShutdownTest.scala:141)
 It looks like Jenkins also had issues with these tests:
 https://builds.apache.org/job/Kafka-trunk/351/console
 I would like to provide a patch that fixes this.



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


[jira] [Commented] (KAFKA-1813) Build fails for scala 2.9.2

2014-12-12 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1813?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14245122#comment-14245122
 ] 

Guozhang Wang commented on KAFKA-1813:
--

Thanks Anatoly.

When I ran the tests / builds before checking in KAFKA-1650 I did not see 
either this issue or KAFKA-1815, which is a bit wired. Will do some local 
testing again.

 Build fails for scala 2.9.2
 ---

 Key: KAFKA-1813
 URL: https://issues.apache.org/jira/browse/KAFKA-1813
 Project: Kafka
  Issue Type: Bug
  Components: build
Reporter: Anatoly Fayngelerin
Assignee: Anatoly Fayngelerin
Priority: Minor
 Fix For: 0.8.3

 Attachments: fix_2_9_2_build.patch


 Currently, in trunk, the 2.9.2 build fails with the following error:
 MirrorMaker.scala:507 overloaded method value commitOffsets with alternatives:
   (isAutoCommit: Boolean,topicPartitionOffsets: 
 scala.collection.immutable.Map[kafka.common.TopicAndPartition,kafka.common.OffsetAndMetadata])Unit
  and
   (isAutoCommit: Boolean)Unit and
   = Unit
  cannot be applied to (isAutoCommit: Boolean, 
 scala.collection.immutable.Map[kafka.common.TopicAndPartition,kafka.common.OffsetAndMetadata])
 connector.commitOffsets(isAutoCommit = false, offsetsToCommit)
 It looks like the 2.9.2 compiler cannot resolve an overloaded method when 
 mixing named and ordered parameters.
 I ran into this when I cloned the repo and ran ./gradlew test.



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