[jira] [Commented] (KAFKA-2985) Consumer group stuck in rebalancing state

2016-01-23 Thread Federico Fissore (JIRA)

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

Federico Fissore commented on KAFKA-2985:
-

Jason, no, I used released 0.9.0.0: I still haven't set up a container with 
latest 0.9.0 branch. I'll do that later this Monday
(and it would be great if you could provide a nightly of Kafka: that way I'd 
just have to download it and try it)

> Consumer group stuck in rebalancing state
> -
>
> Key: KAFKA-2985
> URL: https://issues.apache.org/jira/browse/KAFKA-2985
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.9.0.0
> Environment: Kafka 0.9.0.0.
> Kafka Java consumer 0.9.0.0
> 2 Java producers.
> 3 Java consumers using the new consumer API.
> 2 kafka brokers.
>Reporter: Jens Rantil
>Assignee: Jason Gustafson
>
> We've doing some load testing on Kafka. _After_ the load test when our 
> consumers and have two times now seen Kafka become stuck in consumer group 
> rebalancing. This is after all our consumers are done consuming and 
> essentially polling periodically without getting any records.
> The brokers list the consumer group (named "default"), but I can't query the 
> offsets:
> {noformat}
> jrantil@queue-0:/srv/kafka/kafka$ ./bin/kafka-consumer-groups.sh 
> --new-consumer --bootstrap-server localhost:9092 --list
> default
> jrantil@queue-0:/srv/kafka/kafka$ ./bin/kafka-consumer-groups.sh 
> --new-consumer --bootstrap-server localhost:9092 --describe --group 
> default|sort
> Consumer group `default` does not exist or is rebalancing.
> {noformat}
> Retrying to query the offsets for 15 minutes or so still said it was 
> rebalancing. After restarting our first broker, the group immediately started 
> rebalancing. That broker was logging this before restart:
> {noformat}
> [2015-12-12 13:09:48,517] INFO [Group Metadata Manager on Broker 0]: Removed 
> 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager)
> [2015-12-12 13:10:16,139] INFO [GroupCoordinator 0]: Stabilized group default 
> generation 16 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:10:16,141] INFO [GroupCoordinator 0]: Assignment received from 
> leader for group default for generation 16 
> (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:10:16,575] INFO [GroupCoordinator 0]: Preparing to restabilize 
> group default with old generation 16 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:11:15,141] INFO [GroupCoordinator 0]: Stabilized group default 
> generation 17 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:11:15,143] INFO [GroupCoordinator 0]: Assignment received from 
> leader for group default for generation 17 
> (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:11:15,314] INFO [GroupCoordinator 0]: Preparing to restabilize 
> group default with old generation 17 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:12:14,144] INFO [GroupCoordinator 0]: Stabilized group default 
> generation 18 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:12:14,145] INFO [GroupCoordinator 0]: Assignment received from 
> leader for group default for generation 18 
> (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:12:14,340] INFO [GroupCoordinator 0]: Preparing to restabilize 
> group default with old generation 18 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:13:13,146] INFO [GroupCoordinator 0]: Stabilized group default 
> generation 19 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:13:13,148] INFO [GroupCoordinator 0]: Assignment received from 
> leader for group default for generation 19 
> (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:13:13,238] INFO [GroupCoordinator 0]: Preparing to restabilize 
> group default with old generation 19 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:14:12,148] INFO [GroupCoordinator 0]: Stabilized group default 
> generation 20 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:14:12,149] INFO [GroupCoordinator 0]: Assignment received from 
> leader for group default for generation 20 
> (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:14:12,360] INFO [GroupCoordinator 0]: Preparing to restabilize 
> group default with old generation 20 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:15:11,150] INFO [GroupCoordinator 0]: Stabilized group default 
> generation 21 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:15:11,152] INFO [GroupCoordinator 0]: Assignment received from 
> leader for group default for generation 21 
> (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:15:11,217] INFO [GroupCoordinator 0]: Preparing to restabilize 
> group default with old generation 21 (kafka.coordinator.GroupCoordinator)
> [2015-12-12 13:16:10,152] INFO [GroupCoordinator 0]: Stabilized 

[jira] [Commented] (KAFKA-2426) A Kafka node tries to connect to itself through its advertised hostname

2016-01-23 Thread JIRA

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

Mikaël Cluseau commented on KAFKA-2426:
---

Hi Alex,

there are good chances that I was testing the docker case with a version before 
they introduced libnetwork in docker (which added hairpin support as I 
understand it). As of Kubernetes, the issue was found while working on the 
iptables proxy mode, and solved by the hairpin support.

This means that any topology where the Kafka nodes advertise a public IP that 
isn't their own will break if the "firewall" uses a standard Linux bridge that 
doesn't have hairpin enabled. This also forces SNAT/MASQUERADE of the 
nodeA-to-nodeA traffic, so that the node's IP is hidden from itself (otherwise 
TCP will fail to reconcile the reply packets with the connection). There's also 
this annoying useless roundtrip on the network but that's minor :-)

So, as of today, this issue cannot be reproduced anymore with Kubernetes 
(userland proxy only in 1.0, iptables in 1.1). People doing manual/other things 
may hit it, thought.

It would still be nice if we could avoid some extra latency+badnwidth by having 
a distinction between the public IPs from where the clients come and the 
cluster IPs used for inter-broker communications (like replication). I assume 
here that the leader broker is responsible of sending the messages to its 
followers, and that this will be done through the public IPs.

> A Kafka node tries to connect to itself through its advertised hostname
> ---
>
> Key: KAFKA-2426
> URL: https://issues.apache.org/jira/browse/KAFKA-2426
> Project: Kafka
>  Issue Type: Bug
>  Components: network
>Affects Versions: 0.8.2.1
> Environment: Docker https://github.com/wurstmeister/kafka-docker, 
> managed by a Kubernetes cluster, with an "iptables proxy".
>Reporter: Mikaël Cluseau
>Assignee: Jun Rao
>
> Hi,
> when used behind a firewall, Apache Kafka nodes are trying to connect to 
> themselves using their advertised hostnames. This means that if you have a 
> service IP managed by the docker's host using *only* iptables DNAT rules, the 
> node's connection to "itself" times out.
> This is the case in any setup where a host will DNAT the service IP to the 
> instance's IP, and send the packet back on the same interface other a Linux 
> Bridge port not configured in "hairpin" mode. It's because of this: 
> https://git.kernel.org/cgit/linux/kernel/git/torvalds/linux.git/tree/net/bridge/br_forward.c#n30
> The specific part of the kubernetes issue is here: 
> https://github.com/BenTheElder/kubernetes/issues/3#issuecomment-123925060 .
> The timeout involves that the even if partition's leader is elected, it then 
> fails to accept writes from the other members, causing a write lock. and 
> generating very heavy logs (as fast as Kafka usualy is, but through log4j 
> this time ;)).
> This also means that the normal docker case work by going through the 
> userspace-proxy, which necessarily impacts the performance.
> The workaround for us was to add a "127.0.0.2 advertised-hostname" to 
> /etc/hosts in the container startup script.



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


[jira] [Created] (KAFKA-3142) Improve error message in kstreams

2016-01-23 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-3142:


 Summary: Improve error message in kstreams
 Key: KAFKA-3142
 URL: https://issues.apache.org/jira/browse/KAFKA-3142
 Project: Kafka
  Issue Type: Improvement
Reporter: Jay Kreps


If you have a bug in your kstream code the error message could be slightly 
better.

1. Not sure what the second error about it already being closed is.
2. I recommend we avoid wrapping the exception (at least for runtime 
exceptions) so it is more clear to the user that the error is coming from their 
code not ours.

{code}
[2016-01-23 14:40:59,776] ERROR Uncaught error during processing in thread 
[StreamThread-1]:  
(org.apache.kafka.streams.processor.internals.StreamThread:222)
org.apache.kafka.common.KafkaException: java.lang.NullPointerException
at 
org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:327)
at 
org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:220)
Caused by: java.lang.NullPointerException
at TestKstreamsApi.lambda$0(TestKstreamsApi.java:27)
at 
org.apache.kafka.streams.kstream.internals.KStreamMap$KStreamMapProcessor.process(KStreamMap.java:42)
at 
org.apache.kafka.streams.processor.internals.ProcessorNode.process(ProcessorNode.java:68)
at 
org.apache.kafka.streams.processor.internals.StreamTask.forward(StreamTask.java:324)
at 
org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:175)
at 
org.apache.kafka.streams.processor.internals.SourceNode.process(SourceNode.java:56)
at 
org.apache.kafka.streams.processor.internals.StreamTask.process(StreamTask.java:170)
at 
org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:310)
... 1 more
[2016-01-23 14:40:59,803] ERROR Failed to close a StreamTask #0_0 in thread 
[StreamThread-1]:  
(org.apache.kafka.streams.processor.internals.StreamThread:586)
java.lang.IllegalStateException: This consumer has already been closed.
at 
org.apache.kafka.clients.consumer.KafkaConsumer.ensureNotClosed(KafkaConsumer.java:1281)
at 
org.apache.kafka.clients.consumer.KafkaConsumer.acquire(KafkaConsumer.java:1292)
at 
org.apache.kafka.clients.consumer.KafkaConsumer.assign(KafkaConsumer.java:779)
at 
org.apache.kafka.streams.processor.internals.ProcessorStateManager.close(ProcessorStateManager.java:339)
at 
org.apache.kafka.streams.processor.internals.AbstractTask.close(AbstractTask.java:95)
at 
org.apache.kafka.streams.processor.internals.StreamTask.close(StreamTask.java:306)
at 
org.apache.kafka.streams.processor.internals.StreamThread.closeOne(StreamThread.java:584)
at 
org.apache.kafka.streams.processor.internals.StreamThread.removeStreamTasks(StreamThread.java:571)
at 
org.apache.kafka.streams.processor.internals.StreamThread.shutdown(StreamThread.java:265)
at 
org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:225)
{code}



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


[jira] [Updated] (KAFKA-3142) Improve error message in kstreams

2016-01-23 Thread Guozhang Wang (JIRA)

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

Guozhang Wang updated KAFKA-3142:
-
Issue Type: Sub-task  (was: Improvement)
Parent: KAFKA-2590

> Improve error message in kstreams
> -
>
> Key: KAFKA-3142
> URL: https://issues.apache.org/jira/browse/KAFKA-3142
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Jay Kreps
>
> If you have a bug in your kstream code the error message could be slightly 
> better.
> 1. Not sure what the second error about it already being closed is.
> 2. I recommend we avoid wrapping the exception (at least for runtime 
> exceptions) so it is more clear to the user that the error is coming from 
> their code not ours.
> {code}
> [2016-01-23 14:40:59,776] ERROR Uncaught error during processing in thread 
> [StreamThread-1]:  
> (org.apache.kafka.streams.processor.internals.StreamThread:222)
> org.apache.kafka.common.KafkaException: java.lang.NullPointerException
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:327)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:220)
> Caused by: java.lang.NullPointerException
>   at TestKstreamsApi.lambda$0(TestKstreamsApi.java:27)
>   at 
> org.apache.kafka.streams.kstream.internals.KStreamMap$KStreamMapProcessor.process(KStreamMap.java:42)
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorNode.process(ProcessorNode.java:68)
>   at 
> org.apache.kafka.streams.processor.internals.StreamTask.forward(StreamTask.java:324)
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:175)
>   at 
> org.apache.kafka.streams.processor.internals.SourceNode.process(SourceNode.java:56)
>   at 
> org.apache.kafka.streams.processor.internals.StreamTask.process(StreamTask.java:170)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:310)
>   ... 1 more
> [2016-01-23 14:40:59,803] ERROR Failed to close a StreamTask #0_0 in thread 
> [StreamThread-1]:  
> (org.apache.kafka.streams.processor.internals.StreamThread:586)
> java.lang.IllegalStateException: This consumer has already been closed.
>   at 
> org.apache.kafka.clients.consumer.KafkaConsumer.ensureNotClosed(KafkaConsumer.java:1281)
>   at 
> org.apache.kafka.clients.consumer.KafkaConsumer.acquire(KafkaConsumer.java:1292)
>   at 
> org.apache.kafka.clients.consumer.KafkaConsumer.assign(KafkaConsumer.java:779)
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorStateManager.close(ProcessorStateManager.java:339)
>   at 
> org.apache.kafka.streams.processor.internals.AbstractTask.close(AbstractTask.java:95)
>   at 
> org.apache.kafka.streams.processor.internals.StreamTask.close(StreamTask.java:306)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.closeOne(StreamThread.java:584)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.removeStreamTasks(StreamThread.java:571)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.shutdown(StreamThread.java:265)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:225)
> {code}



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


[GitHub] kafka pull request: KAFKA-2846: Add Ducktape test for kafka-consum...

2016-01-23 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/555


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


[jira] [Resolved] (KAFKA-2846) Add Ducktape test for kafka-consumer-groups

2016-01-23 Thread Ewen Cheslack-Postava (JIRA)

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

Ewen Cheslack-Postava resolved KAFKA-2846.
--
Resolution: Fixed

Issue resolved by pull request 555
[https://github.com/apache/kafka/pull/555]

> Add Ducktape test for kafka-consumer-groups
> ---
>
> Key: KAFKA-2846
> URL: https://issues.apache.org/jira/browse/KAFKA-2846
> Project: Kafka
>  Issue Type: Test
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
> Fix For: 0.9.1.0
>
>
> kafka-consumer-groups is a user facing tool. Having system tests will make 
> sure that we are not changing its behavior unintentionally.



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


[jira] [Commented] (KAFKA-2846) Add Ducktape test for kafka-consumer-groups

2016-01-23 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on KAFKA-2846:
---

Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/555


> Add Ducktape test for kafka-consumer-groups
> ---
>
> Key: KAFKA-2846
> URL: https://issues.apache.org/jira/browse/KAFKA-2846
> Project: Kafka
>  Issue Type: Test
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
> Fix For: 0.9.1.0
>
>
> kafka-consumer-groups is a user facing tool. Having system tests will make 
> sure that we are not changing its behavior unintentionally.



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


Build failed in Jenkins: kafka-trunk-jdk8 #309

2016-01-23 Thread Apache Jenkins Server
See 

Changes:

[me] KAFKA-2846: Add Ducktape test for kafka-consumer-groups

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on H11 (docker Ubuntu ubuntu yahoo-not-h2) in workspace 

 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git -c core.askpass=true fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
 > git rev-parse refs/remotes/origin/trunk^{commit} # timeout=10
 > git rev-parse refs/remotes/origin/origin/trunk^{commit} # timeout=10
Checking out Revision 4f39b5bc5b2104fb39ab5b0c087fe84a71205a74 
(refs/remotes/origin/trunk)
 > git config core.sparsecheckout # timeout=10
 > git checkout -f 4f39b5bc5b2104fb39ab5b0c087fe84a71205a74
 > git rev-list bc9237701b06768c119e954ddb4cd2e61c24e305 # timeout=10
Setting 
JDK1_8_0_45_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_45
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
[kafka-trunk-jdk8] $ /bin/bash -xe /tmp/hudson5894136376158890533.sh
+ 
/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2/bin/gradle
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
http://gradle.org/docs/2.4-rc-2/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:downloadWrapper

BUILD SUCCESSFUL

Total time: 16.511 secs
Setting 
JDK1_8_0_45_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_45
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
[kafka-trunk-jdk8] $ /bin/bash -xe /tmp/hudson9197031999886613459.sh
+ export GRADLE_OPTS=-Xmx1024m
+ GRADLE_OPTS=-Xmx1024m
+ ./gradlew -Dorg.gradle.project.maxParallelForks=1 clean jarAll testAll
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
https://docs.gradle.org/2.10/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:clean UP-TO-DATE
:clients:clean UP-TO-DATE
:connect:clean UP-TO-DATE
:core:clean UP-TO-DATE
:examples:clean UP-TO-DATE
:log4j-appender:clean UP-TO-DATE
:streams:clean UP-TO-DATE
:tools:clean UP-TO-DATE
:connect:api:clean UP-TO-DATE
:connect:file:clean UP-TO-DATE
:connect:json:clean UP-TO-DATE
:connect:runtime:clean UP-TO-DATE
:streams:examples:clean UP-TO-DATE
:jar_core_2_10
Building project 'core' with Scala version 2.10.6
:kafka-trunk-jdk8:clients:compileJava
:jar_core_2_10 FAILED

FAILURE: Build failed with an exception.

* What went wrong:
Failed to capture snapshot of input files for task 'compileJava' during 
up-to-date check.  See stacktrace for details.
> Could not add entry 
> '/home/jenkins/.gradle/caches/modules-2/files-2.1/net.jpountz.lz4/lz4/1.3.0/c708bb2590c0652a642236ef45d9f99ff842a2ce/lz4-1.3.0.jar'
>  to cache fileHashes.bin 
> (

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug 
option to get more log output.

BUILD FAILED

Total time: 15.859 secs
Build step 'Execute shell' marked build as failure
Recording test results
Setting 
JDK1_8_0_45_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_45
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
ERROR: Publisher 'Publish JUnit test result report' failed: No test report 
files were found. Configuration error?
Setting 
JDK1_8_0_45_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_45
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2


Build failed in Jenkins: kafka-trunk-jdk7 #983

2016-01-23 Thread Apache Jenkins Server
See 

Changes:

[me] KAFKA-2846: Add Ducktape test for kafka-consumer-groups

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on H10 (docker Ubuntu ubuntu yahoo-not-h2) in workspace 

 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git -c core.askpass=true fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
 > git rev-parse refs/remotes/origin/trunk^{commit} # timeout=10
 > git rev-parse refs/remotes/origin/origin/trunk^{commit} # timeout=10
Checking out Revision 4f39b5bc5b2104fb39ab5b0c087fe84a71205a74 
(refs/remotes/origin/trunk)
 > git config core.sparsecheckout # timeout=10
 > git checkout -f 4f39b5bc5b2104fb39ab5b0c087fe84a71205a74
 > git rev-list bc9237701b06768c119e954ddb4cd2e61c24e305 # timeout=10
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51
[kafka-trunk-jdk7] $ /bin/bash -xe /tmp/hudson5429145019005329684.sh
+ 
/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2/bin/gradle
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
http://gradle.org/docs/2.4-rc-2/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:downloadWrapper

BUILD SUCCESSFUL

Total time: 15.307 secs
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51
[kafka-trunk-jdk7] $ /bin/bash -xe /tmp/hudson6475453563200803216.sh
+ export GRADLE_OPTS=-Xmx1024m
+ GRADLE_OPTS=-Xmx1024m
+ ./gradlew -Dorg.gradle.project.maxParallelForks=1 clean jarAll testAll
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
https://docs.gradle.org/2.10/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:clean UP-TO-DATE
:clients:clean
:connect:clean UP-TO-DATE
:core:clean
:examples:clean UP-TO-DATE
:log4j-appender:clean UP-TO-DATE
:streams:clean UP-TO-DATE
:tools:clean UP-TO-DATE
:connect:api:clean UP-TO-DATE
:connect:file:clean UP-TO-DATE
:connect:json:clean UP-TO-DATE
:connect:runtime:clean UP-TO-DATE
:streams:examples:clean UP-TO-DATE
:jar_core_2_10
Building project 'core' with Scala version 2.10.6
:kafka-trunk-jdk7:clients:compileJavaNote: 

 uses or overrides a deprecated API.
Note: Recompile with -Xlint:deprecation for details.
Note: Some input files use unchecked or unsafe operations.
Note: Recompile with -Xlint:unchecked for details.

:kafka-trunk-jdk7:clients:processResources UP-TO-DATE
:kafka-trunk-jdk7:clients:classes
:kafka-trunk-jdk7:clients:determineCommitId UP-TO-DATE
:kafka-trunk-jdk7:clients:createVersionFile
:kafka-trunk-jdk7:clients:jar
:kafka-trunk-jdk7:core:compileJava UP-TO-DATE
:kafka-trunk-jdk7:core:compileScala
:79:
 value DEFAULT_TIMESTAMP in object OffsetCommitRequest is deprecated: see 
corresponding Javadoc for more information.

org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP
 ^
:36:
 value DEFAULT_TIMESTAMP in object OffsetCommitRequest is deprecated: see 
corresponding Javadoc for more information.
 commitTimestamp: Long = 
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP,

  ^
:37:
 value DEFAULT_TIMESTAMP in object OffsetCommitRequest is deprecated: see 
corresponding Javadoc for more information.
 expireTimestamp: Long = 
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) {

  ^


[GitHub] kafka pull request: MINOR: Improve Kafka documentation

2016-01-23 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/778


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


Build failed in Jenkins: kafka-trunk-jdk7 #984

2016-01-23 Thread Apache Jenkins Server
See 

Changes:

[me] MINOR: Improve Kafka documentation

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on H11 (docker Ubuntu ubuntu yahoo-not-h2) in workspace 

 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git -c core.askpass=true fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
 > git rev-parse refs/remotes/origin/trunk^{commit} # timeout=10
 > git rev-parse refs/remotes/origin/origin/trunk^{commit} # timeout=10
Checking out Revision d00cf520fb0b36c7c705250b1773db2f242d5f44 
(refs/remotes/origin/trunk)
 > git config core.sparsecheckout # timeout=10
 > git checkout -f d00cf520fb0b36c7c705250b1773db2f242d5f44
 > git rev-list 4f39b5bc5b2104fb39ab5b0c087fe84a71205a74 # timeout=10
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51
[kafka-trunk-jdk7] $ /bin/bash -xe /tmp/hudson8949545311291782290.sh
+ 
/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2/bin/gradle
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
http://gradle.org/docs/2.4-rc-2/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:downloadWrapper

BUILD SUCCESSFUL

Total time: 14.897 secs
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51
[kafka-trunk-jdk7] $ /bin/bash -xe /tmp/hudson1871951440625157709.sh
+ export GRADLE_OPTS=-Xmx1024m
+ GRADLE_OPTS=-Xmx1024m
+ ./gradlew -Dorg.gradle.project.maxParallelForks=1 clean jarAll testAll
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
https://docs.gradle.org/2.10/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:clean UP-TO-DATE
:clients:clean UP-TO-DATE
:connect:clean UP-TO-DATE
:core:clean UP-TO-DATE
:examples:clean UP-TO-DATE
:log4j-appender:clean UP-TO-DATE
:streams:clean UP-TO-DATE
:tools:clean UP-TO-DATE
:connect:api:clean UP-TO-DATE
:connect:file:clean UP-TO-DATE
:connect:json:clean UP-TO-DATE
:connect:runtime:clean UP-TO-DATE
:streams:examples:clean UP-TO-DATE
:jar_core_2_10
Building project 'core' with Scala version 2.10.6
:kafka-trunk-jdk7:clients:compileJava
:jar_core_2_10 FAILED

FAILURE: Build failed with an exception.

* What went wrong:
Failed to capture snapshot of input files for task 'compileJava' during 
up-to-date check.  See stacktrace for details.
> Could not add entry 
> '/home/jenkins/.gradle/caches/modules-2/files-2.1/net.jpountz.lz4/lz4/1.3.0/c708bb2590c0652a642236ef45d9f99ff842a2ce/lz4-1.3.0.jar'
>  to cache fileHashes.bin 
> (

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug 
option to get more log output.

BUILD FAILED

Total time: 18.149 secs
Build step 'Execute shell' marked build as failure
Recording test results
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51
ERROR: Publisher 'Publish JUnit test result report' failed: No test report 
files were found. Configuration error?
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51


[GitHub] kafka pull request: KAFKA-3138: 0.9.0 docs still say that log comp...

2016-01-23 Thread wushujames
GitHub user wushujames opened a pull request:

https://github.com/apache/kafka/pull/807

KAFKA-3138: 0.9.0 docs still say that log compaction doesn't work on 
compressed topics.

Log compaction is supported on compressed topics as of 0.9.0, so update the 
docs to reflect that.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/wushujames/kafka KAFKA-3138

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/807.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #807


commit 3ef3ef157544474c6c37e0453b8e362cd936f02e
Author: James Cheng 
Date:   2016-01-24T06:48:21Z

Log compaction is supported on compressed topics as of 0.9.0, so update the 
docs to reflect that.




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


[jira] [Commented] (KAFKA-3138) 0.9.0 docs still say that log compaction doesn't work on compressed topics.

2016-01-23 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on KAFKA-3138:
---

GitHub user wushujames opened a pull request:

https://github.com/apache/kafka/pull/807

KAFKA-3138: 0.9.0 docs still say that log compaction doesn't work on 
compressed topics.

Log compaction is supported on compressed topics as of 0.9.0, so update the 
docs to reflect that.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/wushujames/kafka KAFKA-3138

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/807.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #807


commit 3ef3ef157544474c6c37e0453b8e362cd936f02e
Author: James Cheng 
Date:   2016-01-24T06:48:21Z

Log compaction is supported on compressed topics as of 0.9.0, so update the 
docs to reflect that.




> 0.9.0 docs still say that log compaction doesn't work on compressed topics.
> ---
>
> Key: KAFKA-3138
> URL: https://issues.apache.org/jira/browse/KAFKA-3138
> Project: Kafka
>  Issue Type: Bug
>Reporter: James Cheng
>
> The 0.9.0 docs say "Log compaction is not yet compatible with compressed 
> topics.". But I believe that was fixed in 0.9.0.
> Is the fix to simply remove that line from the docs? It sounds newbie level. 
> If so, I would like to work on this JIRA.



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


Build failed in Jenkins: kafka-trunk-jdk8 #310

2016-01-23 Thread Apache Jenkins Server
See 

Changes:

[me] MINOR: Improve Kafka documentation

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on H11 (docker Ubuntu ubuntu yahoo-not-h2) in workspace 

 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git -c core.askpass=true fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
 > git rev-parse refs/remotes/origin/trunk^{commit} # timeout=10
 > git rev-parse refs/remotes/origin/origin/trunk^{commit} # timeout=10
Checking out Revision d00cf520fb0b36c7c705250b1773db2f242d5f44 
(refs/remotes/origin/trunk)
 > git config core.sparsecheckout # timeout=10
 > git checkout -f d00cf520fb0b36c7c705250b1773db2f242d5f44
 > git rev-list 4f39b5bc5b2104fb39ab5b0c087fe84a71205a74 # timeout=10
Setting 
JDK1_8_0_45_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_45
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
[kafka-trunk-jdk8] $ /bin/bash -xe /tmp/hudson8981097962683057057.sh
+ 
/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2/bin/gradle
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
http://gradle.org/docs/2.4-rc-2/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:downloadWrapper

BUILD SUCCESSFUL

Total time: 11.773 secs
Setting 
JDK1_8_0_45_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_45
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
[kafka-trunk-jdk8] $ /bin/bash -xe /tmp/hudson7233861946163356963.sh
+ export GRADLE_OPTS=-Xmx1024m
+ GRADLE_OPTS=-Xmx1024m
+ ./gradlew -Dorg.gradle.project.maxParallelForks=1 clean jarAll testAll
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
https://docs.gradle.org/2.10/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:clean UP-TO-DATE
:clients:clean UP-TO-DATE
:connect:clean UP-TO-DATE
:core:clean UP-TO-DATE
:examples:clean UP-TO-DATE
:log4j-appender:clean UP-TO-DATE
:streams:clean UP-TO-DATE
:tools:clean UP-TO-DATE
:connect:api:clean UP-TO-DATE
:connect:file:clean UP-TO-DATE
:connect:json:clean UP-TO-DATE
:connect:runtime:clean UP-TO-DATE
:streams:examples:clean UP-TO-DATE
:jar_core_2_10
Building project 'core' with Scala version 2.10.6
:kafka-trunk-jdk8:clients:compileJava
:jar_core_2_10 FAILED

FAILURE: Build failed with an exception.

* What went wrong:
Failed to capture snapshot of input files for task 'compileJava' during 
up-to-date check.  See stacktrace for details.
> Could not add entry 
> '/home/jenkins/.gradle/caches/modules-2/files-2.1/net.jpountz.lz4/lz4/1.3.0/c708bb2590c0652a642236ef45d9f99ff842a2ce/lz4-1.3.0.jar'
>  to cache fileHashes.bin 
> (

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug 
option to get more log output.

BUILD FAILED

Total time: 14.446 secs
Build step 'Execute shell' marked build as failure
Recording test results
Setting 
JDK1_8_0_45_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_45
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
ERROR: Publisher 'Publish JUnit test result report' failed: No test report 
files were found. Configuration error?
Setting 
JDK1_8_0_45_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_45
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2