Re: Possible StreamingConfig Bug

2015-11-28 Thread Bill Bejeck
Yes, that is the case from my usage.

FWIW I have a Jira ticket (KAFKA-2902) and have submitted a PR (#596)
to address the issue.

Thanks,
Bill

> On Nov 28, 2015, at 1:27 AM, Guozhang Wang  wrote:
>
> I think the problem is that "getConsumerConfigs" called in StreamThread.
> createConsumer() triggers getRestoreConsumerConfigs instead of
> getBaseConsumerConfigs, which seems a bug to me.
>
> Guozhang
>
> On Fri, Nov 27, 2015 at 8:01 PM, Yasuhiro Matsuda <
> yasuhiro.mats...@gmail.com> wrote:
>
>> The group id is removed from the restore consumer config because the
>> restore consumer should not participate in the specified consumer group. I
>> don't know why it is failing.
>>
>> On Fri, Nov 27, 2015 at 12:37 PM, Guozhang Wang 
>> wrote:
>>
>>> Hello Bill,
>>>
>>> Thanks for reporting it, this is a valid issue, could you create a
>> ticket?
>>>
>>> Guozhang
>>>
 On Fri, Nov 27, 2015 at 6:19 AM, Bill Bejeck  wrote:

 All,

 When starting KafkaStreaming I'm getting the following error (even when
 explicitly setting the groupId with props.put("group.id
 ","test-consumer-group")
 );

 Exception in thread "StreamThread-1"
 org.apache.kafka.common.KafkaException:
 org.apache.kafka.common.errors.ApiException: The configured groupId is
 invalid
 at
>> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:309)
 at
>> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:198)
 Caused by: org.apache.kafka.common.errors.ApiException: The configured
 groupId is invalid

 I've traced the source of the issue to the
 StreamingConfig.getConsumerConfigs method as it calls
 getRestoreConsumerConfigs (which explicitly removes the groupId
>> property)
 vs using getBaseConsumerConfigs which returns the passed in configs
 unaltered.

 When I switched the method call, KafkaStreaming starts up fine.

 If you agree with this change/fix, I'll create a Jira ticket and put in
>>> the
 PR, yada yada yada..

 Thanks,
 Bill
>>>
>>>
>>>
>>> --
>>> -- Guozhang
>
>
>
> --
> -- Guozhang


[jira] [Commented] (KAFKA-2891) Gaps in messages delivered by new consumer after Kafka restart

2015-11-28 Thread Ben Stopford (JIRA)

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

Ben Stopford commented on KAFKA-2891:
-

Sorry [~hachikuji]- typo - should have said "implies the problem should not be 
consumer side". now changed. 

> Gaps in messages delivered by new consumer after Kafka restart
> --
>
> Key: KAFKA-2891
> URL: https://issues.apache.org/jira/browse/KAFKA-2891
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.9.0.0
>Reporter: Rajini Sivaram
>Priority: Critical
>
> Replication tests when run with the new consumer with SSL/SASL were failing 
> very often because messages were not being consumed from some topics after a 
> Kafka restart. The fix in KAFKA-2877 has made this a lot better. But I am 
> still seeing some failures (less often now) because a small set of messages 
> are not received after Kafka restart. This failure looks slightly different 
> from the one before the fix for KAFKA-2877 was applied, hence the new defect. 
> The test fails because not all acked messages are received by the consumer, 
> and the number of messages missing are quite small.
> [~benstopford] Are the upgrade tests working reliably with KAFKA-2877 now?
> Not sure if any of these log entries are important:
> {quote}
> [2015-11-25 14:41:12,342] INFO SyncGroup for group test-consumer-group failed 
> due to NOT_COORDINATOR_FOR_GROUP, will find new coordinator and rejoin 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-25 14:41:12,342] INFO Marking the coordinator 2147483644 dead. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-25 14:41:12,958] INFO Attempt to join group test-consumer-group 
> failed due to unknown member id, resetting and retrying. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-25 14:41:42,437] INFO Fetch offset null is out of range, resetting 
> offset (org.apache.kafka.clients.consumer.internals.Fetcher)
> {quote}



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


[jira] [Comment Edited] (KAFKA-2891) Gaps in messages delivered by new consumer after Kafka restart

2015-11-28 Thread Ben Stopford (JIRA)

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

Ben Stopford edited comment on KAFKA-2891 at 11/28/15 10:32 AM:


One more bit of info - when this problem occurs the missing messages are not in 
the server data files. This implies the problem should not be consumer side. 
However we don't seem to see this when the old consumer is used. 


was (Author: benstopford):
One more bit of info - when this problem occurs the missing messages are not in 
the server data files. This implies the problem should be on the consumer side. 
However we don't seem to see this when the old consumer is used. 

> Gaps in messages delivered by new consumer after Kafka restart
> --
>
> Key: KAFKA-2891
> URL: https://issues.apache.org/jira/browse/KAFKA-2891
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.9.0.0
>Reporter: Rajini Sivaram
>Priority: Critical
>
> Replication tests when run with the new consumer with SSL/SASL were failing 
> very often because messages were not being consumed from some topics after a 
> Kafka restart. The fix in KAFKA-2877 has made this a lot better. But I am 
> still seeing some failures (less often now) because a small set of messages 
> are not received after Kafka restart. This failure looks slightly different 
> from the one before the fix for KAFKA-2877 was applied, hence the new defect. 
> The test fails because not all acked messages are received by the consumer, 
> and the number of messages missing are quite small.
> [~benstopford] Are the upgrade tests working reliably with KAFKA-2877 now?
> Not sure if any of these log entries are important:
> {quote}
> [2015-11-25 14:41:12,342] INFO SyncGroup for group test-consumer-group failed 
> due to NOT_COORDINATOR_FOR_GROUP, will find new coordinator and rejoin 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-25 14:41:12,342] INFO Marking the coordinator 2147483644 dead. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-25 14:41:12,958] INFO Attempt to join group test-consumer-group 
> failed due to unknown member id, resetting and retrying. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-25 14:41:42,437] INFO Fetch offset null is out of range, resetting 
> offset (org.apache.kafka.clients.consumer.internals.Fetcher)
> {quote}



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


[jira] [Created] (KAFKA-2904) Consumer Fails to Reconnect after 30s post restarts

2015-11-28 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-2904:
---

 Summary: Consumer Fails to Reconnect after 30s post restarts
 Key: KAFKA-2904
 URL: https://issues.apache.org/jira/browse/KAFKA-2904
 Project: Kafka
  Issue Type: Bug
Reporter: Ben Stopford
Assignee: Ben Stopford


This problem occurs in around 1 in 20 executions of the security rolling 
upgrade test. 

Test scenario is a rolling upgrade where each of the three servers are 
restarted in turn whilst producer and consumers run. A ten second sleep between 
start and stop of each node has been added to ensure there is time for failover 
to occur (re KAFKA-2827). 

Failure results in no consumed messages after the failure point. 

Periodically the consumer does not reconnect for its 30s timeout. The 
consumer’s log at this point is at the bottom of this jira.

ISR's appear normal at the time of the failure.

The producer is able to produce throughout this period. 

*TIMELINE:*

{quote}
20:39:23 - Test starts Consumer and Producer
20:39:27 - Consumer starts consuming produced messages
20:39:30 - Node 1 shutdown complete
20:39:45 - Node 1 restarts
20:39:59 - Node 2 shutdown complete
20:40:14 - Node 2 restarts 
20:40:27 - Consumer stops consuming
20:40:28 - Node 2 becomes controller
20:40:28 - Node 3 shutdown complete
20:40:34 - GroupCoordinator 2: Preparing to restabilize group 
unique-test-group...
20:40:42 - Node 3 restarts
*20:41:03 - Consumer times out*
20:41:03 - GroupCoordinator 2: Stabilized group unique-test-group...
20:41:03 - GroupCoordinator 2: Assignment received from leader for group 
unique-test-group...
20:41:03 - GroupCoordinator 2: Preparing to restabilize group 
unique-test-group...
20:41:03 - GroupCoordinator 2: Group unique-test-group... is dead and removed 
20:41:53 - Producer shuts down
{quote}


Consumer log at time of failure:


{quote}
[2015-11-27 20:40:27,268] INFO Current consumption count is 10100 
(kafka.tools.ConsoleConsumer$)
[2015-11-27 20:40:27,321] ERROR Error ILLEGAL_GENERATION occurred while 
committing offsets for group unique-test-group-0.952644842527 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2015-11-27 20:40:27,321] WARN Auto offset commit failed: Commit cannot be 
completed due to group rebalance 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2015-11-27 20:40:27,322] ERROR Error ILLEGAL_GENERATION occurred while 
committing offsets for group unique-test-group-0.952644842527 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2015-11-27 20:40:27,322] WARN Auto offset commit failed:  
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2015-11-27 20:40:27,329] INFO Attempt to join group 
unique-test-group-0.952644842527 failed due to unknown member id, resetting and 
retrying. (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2015-11-27 20:40:27,347] INFO SyncGroup for group 
unique-test-group-0.952644842527 failed due to UNKNOWN_MEMBER_ID, rejoining the 
group (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2015-11-27 20:40:27,357] INFO SyncGroup for group 
unique-test-group-0.952644842527 failed due to NOT_COORDINATOR_FOR_GROUP, will 
find new coordinator and rejoin 
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2015-11-27 20:40:27,357] INFO Marking the coordinator 2147483644 dead. 
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2015-11-27 20:40:28,097] INFO Attempt to join group 
unique-test-group-0.952644842527 failed due to unknown member id, resetting and 
retrying. (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2015-11-27 20:40:33,627] INFO Marking the coordinator 2147483646 dead. 
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2015-11-27 20:40:33,627] INFO Attempt to join group 
unique-test-group-0.952644842527 failed due to obsolete coordinator 
information, retrying. 
(org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2015-11-27 20:41:03,704] ERROR Error processing message, terminating consumer 
process:  (kafka.tools.ConsoleConsumer$)
kafka.consumer.ConsumerTimeoutException
at kafka.consumer.NewShinyConsumer.receive(BaseConsumer.scala:59)
at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:112)
at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:69)
at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:47)
at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala)
[2015-11-27 20:41:03,737] WARN TGT renewal thread has been interrupted and will 
exit. (org.apache.kafka.common.security.kerberos.Login)
{quote}




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


[jira] [Created] (KAFKA-2905) System test for rolling upgrade to enable ZooKeeper ACLs with SASL

2015-11-28 Thread Flavio Junqueira (JIRA)
Flavio Junqueira created KAFKA-2905:
---

 Summary: System test for rolling upgrade to enable ZooKeeper ACLs 
with SASL
 Key: KAFKA-2905
 URL: https://issues.apache.org/jira/browse/KAFKA-2905
 Project: Kafka
  Issue Type: Test
Affects Versions: 0.9.0.0
Reporter: Flavio Junqueira
Assignee: Flavio Junqueira


Write a ducktape test to verify the ability of performing a rolling upgrade to 
enable the use of secure ACLs and SASL with ZooKeeper.



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


[jira] [Updated] (KAFKA-2904) Consumer Fails to Reconnect after 30s post restarts

2015-11-28 Thread Ben Stopford (JIRA)

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

Ben Stopford updated KAFKA-2904:

Attachment: 2015-11-27--001 (1).tar.gz

> Consumer Fails to Reconnect after 30s post restarts
> ---
>
> Key: KAFKA-2904
> URL: https://issues.apache.org/jira/browse/KAFKA-2904
> Project: Kafka
>  Issue Type: Bug
>Reporter: Ben Stopford
>Assignee: Ben Stopford
> Attachments: 2015-11-27--001 (1).tar.gz
>
>
> This problem occurs in around 1 in 20 executions of the security rolling 
> upgrade test. 
> Test scenario is a rolling upgrade where each of the three servers are 
> restarted in turn whilst producer and consumers run. A ten second sleep 
> between start and stop of each node has been added to ensure there is time 
> for failover to occur (re KAFKA-2827). 
> Failure results in no consumed messages after the failure point. 
> Periodically the consumer does not reconnect for its 30s timeout. The 
> consumer’s log at this point is at the bottom of this jira.
> ISR's appear normal at the time of the failure.
> The producer is able to produce throughout this period. 
> *TIMELINE:*
> {quote}
> 20:39:23 - Test starts Consumer and Producer
> 20:39:27 - Consumer starts consuming produced messages
> 20:39:30 - Node 1 shutdown complete
> 20:39:45 - Node 1 restarts
> 20:39:59 - Node 2 shutdown complete
> 20:40:14 - Node 2 restarts 
> 20:40:27 - Consumer stops consuming
> 20:40:28 - Node 2 becomes controller
> 20:40:28 - Node 3 shutdown complete
> 20:40:34 - GroupCoordinator 2: Preparing to restabilize group 
> unique-test-group...
> 20:40:42 - Node 3 restarts
> *20:41:03 - Consumer times out*
> 20:41:03 - GroupCoordinator 2: Stabilized group unique-test-group...
> 20:41:03 - GroupCoordinator 2: Assignment received from leader for group 
> unique-test-group...
> 20:41:03 - GroupCoordinator 2: Preparing to restabilize group 
> unique-test-group...
> 20:41:03 - GroupCoordinator 2: Group unique-test-group... is dead and removed 
> 20:41:53 - Producer shuts down
> {quote}
> Consumer log at time of failure:
> {quote}
> [2015-11-27 20:40:27,268] INFO Current consumption count is 10100 
> (kafka.tools.ConsoleConsumer$)
> [2015-11-27 20:40:27,321] ERROR Error ILLEGAL_GENERATION occurred while 
> committing offsets for group unique-test-group-0.952644842527 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> [2015-11-27 20:40:27,321] WARN Auto offset commit failed: Commit cannot be 
> completed due to group rebalance 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> [2015-11-27 20:40:27,322] ERROR Error ILLEGAL_GENERATION occurred while 
> committing offsets for group unique-test-group-0.952644842527 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> [2015-11-27 20:40:27,322] WARN Auto offset commit failed:  
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> [2015-11-27 20:40:27,329] INFO Attempt to join group 
> unique-test-group-0.952644842527 failed due to unknown member id, resetting 
> and retrying. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-27 20:40:27,347] INFO SyncGroup for group 
> unique-test-group-0.952644842527 failed due to UNKNOWN_MEMBER_ID, rejoining 
> the group (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-27 20:40:27,357] INFO SyncGroup for group 
> unique-test-group-0.952644842527 failed due to NOT_COORDINATOR_FOR_GROUP, 
> will find new coordinator and rejoin 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-27 20:40:27,357] INFO Marking the coordinator 2147483644 dead. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-27 20:40:28,097] INFO Attempt to join group 
> unique-test-group-0.952644842527 failed due to unknown member id, resetting 
> and retrying. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-27 20:40:33,627] INFO Marking the coordinator 2147483646 dead. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-27 20:40:33,627] INFO Attempt to join group 
> unique-test-group-0.952644842527 failed due to obsolete coordinator 
> information, retrying. 
> (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2015-11-27 20:41:03,704] ERROR Error processing message, terminating 
> consumer process:  (kafka.tools.ConsoleConsumer$)
> kafka.consumer.ConsumerTimeoutException
>   at kafka.consumer.NewShinyConsumer.receive(BaseConsumer.scala:59)
>   at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:112)
>   at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:69)
>   at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:47)
>   at 

[jira] [Commented] (KAFKA-2897) Class NIOServerCnxn$Factory not found due to mismatch in dependencies

2015-11-28 Thread Yuval Steinberg (JIRA)

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

Yuval Steinberg commented on KAFKA-2897:


Then how come not only that is seems to work fine for us (except an error in 
the logs), but the community is working with it? Could it be that just the 
Factory is not a critical / commonly used?

> Class NIOServerCnxn$Factory not found due to mismatch in dependencies
> -
>
> Key: KAFKA-2897
> URL: https://issues.apache.org/jira/browse/KAFKA-2897
> Project: Kafka
>  Issue Type: Bug
>Reporter: Yuval Steinberg
>Priority: Trivial
>
> It seems that there is a mismatch between zookeeper & zkclient in kafka 
> pom.xml causes this error upon initialization:
> The pom file (e.g. of kafka_2.10:0.8.2.1) requires zookeeper:3.4.6 & 
> zkclient:0.3, However:
> * While zookeeper:3.3.x had a class NIOServerCnxn$Factory, in 3.4.x it became 
> an independent class (NIOServerCnxnFactory)
> * ZkServer class of zkclient:0.3 is still using (importing) 
> NIOServerCnxn.Factory and only in 0.5 changes to use NIOServerCnxnFactory.
> It seems that the version of zkclient in the pom file should be updated



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


[jira] [Commented] (KAFKA-2732) Add test cases with ZK Auth, SASL and SSL

2015-11-28 Thread ASF GitHub Bot (JIRA)

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

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

Github user asfgit closed the pull request at:

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


> Add test cases with ZK Auth, SASL and SSL
> -
>
> Key: KAFKA-2732
> URL: https://issues.apache.org/jira/browse/KAFKA-2732
> Project: Kafka
>  Issue Type: Test
>  Components: security
>Affects Versions: 0.9.0.0
>Reporter: Flavio Junqueira
>Assignee: Flavio Junqueira
> Fix For: 0.9.1.0
>
>
> Add test cases to verify the security functionality being added in 0.9. 



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


[jira] [Resolved] (KAFKA-2732) Add test cases with ZK Auth, SASL and SSL

2015-11-28 Thread Jun Rao (JIRA)

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

Jun Rao resolved KAFKA-2732.

Resolution: Fixed

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

> Add test cases with ZK Auth, SASL and SSL
> -
>
> Key: KAFKA-2732
> URL: https://issues.apache.org/jira/browse/KAFKA-2732
> Project: Kafka
>  Issue Type: Test
>  Components: security
>Affects Versions: 0.9.0.0
>Reporter: Flavio Junqueira
>Assignee: Flavio Junqueira
> Fix For: 0.9.1.0
>
>
> Add test cases to verify the security functionality being added in 0.9. 



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


[GitHub] kafka pull request: KAFKA-2732: Add class for ZK Auth.

2015-11-28 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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-jdk8 #191

2015-11-28 Thread Apache Jenkins Server
See 

Changes:

[junrao] KAFKA-2732; Add class for ZK Auth.

--
[...truncated 4461 lines...]
org.apache.kafka.connect.json.JsonConverterTest > timeToJson PASSED

org.apache.kafka.connect.json.JsonConverterTest > structToJson PASSED

org.apache.kafka.connect.json.JsonConverterTest > 
testConnectSchemaMetadataTranslation PASSED

org.apache.kafka.connect.json.JsonConverterTest > shortToJson PASSED

org.apache.kafka.connect.json.JsonConverterTest > dateToConnect PASSED

org.apache.kafka.connect.json.JsonConverterTest > doubleToJson PASSED

org.apache.kafka.connect.json.JsonConverterTest > timeToConnect PASSED

org.apache.kafka.connect.json.JsonConverterTest > mapToConnectStringKeys PASSED

org.apache.kafka.connect.json.JsonConverterTest > floatToConnect PASSED

org.apache.kafka.connect.json.JsonConverterTest > decimalToJson PASSED

org.apache.kafka.connect.json.JsonConverterTest > arrayToConnect PASSED

org.apache.kafka.connect.json.JsonConverterTest > 
testCacheSchemaToConnectConversion PASSED

org.apache.kafka.connect.json.JsonConverterTest > booleanToJson PASSED

org.apache.kafka.connect.json.JsonConverterTest > bytesToConnect PASSED

org.apache.kafka.connect.json.JsonConverterTest > doubleToConnect PASSED
:connect:runtime:checkstyleMain
:connect:runtime:compileTestJavawarning: [options] bootstrap class path not set 
in conjunction with -source 1.7
Note: Some input files use unchecked or unsafe operations.
Note: Recompile with -Xlint:unchecked for details.
1 warning

:connect:runtime:processTestResources
:connect:runtime:testClasses
:connect:runtime:checkstyleTest
:connect:runtime:test

org.apache.kafka.connect.util.KafkaBasedLogTest > testStartStop PASSED

org.apache.kafka.connect.util.KafkaBasedLogTest > testReloadOnStart PASSED

org.apache.kafka.connect.util.KafkaBasedLogTest > testSendAndReadToEnd FAILED
org.junit.ComparisonFailure: expected: but was:
at org.junit.Assert.assertEquals(Assert.java:115)
at org.junit.Assert.assertEquals(Assert.java:144)
at 
org.apache.kafka.connect.util.KafkaBasedLogTest.testSendAndReadToEnd(KafkaBasedLogTest.java:312)

org.apache.kafka.connect.util.KafkaBasedLogTest > testConsumerError PASSED

org.apache.kafka.connect.util.KafkaBasedLogTest > testProducerError PASSED

org.apache.kafka.connect.util.ShutdownableThreadTest > testGracefulShutdown 
PASSED

org.apache.kafka.connect.util.ShutdownableThreadTest > testForcibleShutdown 
PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testCreateConnector PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testListConnectors PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testGetConnectorConfigConnectorNotFound PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testPutConnectorConfig PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testGetConnectorTaskConfigs PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testGetConnectorTaskConfigsConnectorNotFound PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testPutConnectorTaskConfigs PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testPutConnectorTaskConfigsConnectorNotFound PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testDeleteConnectorNotLeader PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testDeleteConnectorNotFound PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testGetConnector PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testGetConnectorConfig PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testListConnectorsNotLeader PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testCreateConnectorNotLeader PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testCreateConnectorExists PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testDeleteConnector PASSED

org.apache.kafka.connect.runtime.rest.resources.ConnectorsResourceTest > 
testListConnectorsNotSynced PASSED

org.apache.kafka.connect.runtime.WorkerSinkTaskThreadedTest > 
testPollsInBackground PASSED

org.apache.kafka.connect.runtime.WorkerSinkTaskThreadedTest > testCommit PASSED

org.apache.kafka.connect.runtime.WorkerSinkTaskThreadedTest > 
testCommitTaskFlushFailure PASSED

org.apache.kafka.connect.runtime.WorkerSinkTaskThreadedTest > 
testCommitTaskSuccessAndFlushFailure PASSED

org.apache.kafka.connect.runtime.WorkerSinkTaskThreadedTest > 
testCommitConsumerFailure PASSED

org.apache.kafka.connect.runtime.WorkerSinkTaskThreadedTest > testCommitTimeout 
PASSED


Doubt on Kafka storage and Consuming messages

2015-11-28 Thread Goutam Chowdhury
Hi ,
I am new in Kafka . I went through some docments to understand Kafka. I
have some questions. Please help me to understand
1> Kafka storage:
What is the physical existing of segment file ?
What do you mean by "flushing segment file to disk"?
One of the document mentioned
-- "A message is only exposed to the consumers after it is flushed"
That means consumer has to wait ?

1> Consuming the message
>From where consumer read the messages? from segment file or from .log file
or from some cache?
2> If it is from cache then how and who is storing into such cache?



Thanks And Regards
Goutam Chowdhury


Re: Release source doesn't contain .gitignore

2015-11-28 Thread Grant Henke
Hi Edward,

I am not sure what the best practice is. Since it is checked in as part of
the project, I considered it effectively part of its source. I suspect the
method for collecting and taring the source files excluded it by accident
(perhaps by many of these projects) and is something that could be fixed.

Thanks,
Grant

On Thu, Nov 26, 2015 at 5:33 AM, Edward Ribeiro 
wrote:

> Hi Grant,
>
> Sorry for my stupid question, but it is usually a best practice to include
> such file in the src tar balls? I am certainly okay with including it, of
> course. :)
>
> But I didn't see it in ZooKeeper 3.4.7, Hadoop 2.4.6 or Cassandra 3.0
> sources, for example. OTOH, I see the .gitattributes file in Hadoop src
> folder with the content as below:
>
> # Auto detect text files and perform LF normalization
> *text=auto
>
> *.cs text diff=csharp
> *.java   text diff=java
> *.html   text diff=html
> *.py text diff=python
> *.pl text diff=perl
> *.pm text diff=perl
> *.csstext
> *.js text
> *.sqltext
>
> *.sh text eol=lf
>
> *.battext eol=crlf
> *.cmdtext eol=crlf
> *.vcxproj text merge=union eol=crlf
> *.csproj text merge=union eol=crlf
> *.slntext merge=union eol=crlf
>
>
> Cheers,
> Eddie
>
>
> On Wed, Nov 25, 2015 at 5:30 PM, Grant Henke  wrote:
>
> > The recent release source does not contain the repo's .gitignore file.
> This
> > is definitely not a major issue, but I wanted to bring it up so it could
> be
> > added in future releases.
> >
> > I downloaded the source from:
> >
> >
> https://www.apache.org/dyn/closer.cgi?path=/kafka/0.9.0.0/kafka-0.9.0.0-src.tgz
> >
> > Thanks,
> > Grant
> > --
> > Grant Henke
> > Software Engineer | Cloudera
> > gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
> >
>



-- 
Grant Henke
Software Engineer | Cloudera
gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke


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

2015-11-28 Thread Apache Jenkins Server
See 

Changes:

[junrao] KAFKA-2732; Add class for ZK Auth.

--
[...truncated 2795 lines...]

kafka.log.LogTest > testTruncateTo PASSED

kafka.log.LogTest > testCleanShutdownFile PASSED

kafka.log.OffsetIndexTest > lookupExtremeCases PASSED

kafka.log.OffsetIndexTest > appendTooMany PASSED

kafka.log.OffsetIndexTest > randomLookupTest PASSED

kafka.log.OffsetIndexTest > testReopen PASSED

kafka.log.OffsetIndexTest > appendOutOfOrder PASSED

kafka.log.OffsetIndexTest > truncate PASSED

kafka.log.LogSegmentTest > testRecoveryWithCorruptMessage PASSED

kafka.log.LogSegmentTest > testRecoveryFixesCorruptIndex PASSED

kafka.log.LogSegmentTest > testReadFromGap PASSED

kafka.log.LogSegmentTest > testTruncate PASSED

kafka.log.LogSegmentTest > testReadBeforeFirstOffset PASSED

kafka.log.LogSegmentTest > testCreateWithInitFileSizeAppendMessage PASSED

kafka.log.LogSegmentTest > testChangeFileSuffixes PASSED

kafka.log.LogSegmentTest > testMaxOffset PASSED

kafka.log.LogSegmentTest > testNextOffsetCalculation PASSED

kafka.log.LogSegmentTest > testReadOnEmptySegment PASSED

kafka.log.LogSegmentTest > testReadAfterLast PASSED

kafka.log.LogSegmentTest > testCreateWithInitFileSizeClearShutdown PASSED

kafka.log.LogSegmentTest > testTruncateFull PASSED

kafka.log.CleanerTest > testBuildOffsetMap PASSED

kafka.log.CleanerTest > testSegmentGrouping PASSED

kafka.log.CleanerTest > testCleanSegmentsWithAbort PASSED

kafka.log.CleanerTest > testSegmentGroupingWithSparseOffsets PASSED

kafka.log.CleanerTest > testRecoveryAfterCrash PASSED

kafka.log.CleanerTest > testLogToClean PASSED

kafka.log.CleanerTest > testCleaningWithDeletes PASSED

kafka.log.CleanerTest > testCleanSegments PASSED

kafka.log.CleanerTest > testCleaningWithUnkeyedMessages PASSED

kafka.log.OffsetMapTest > testClear PASSED

kafka.log.OffsetMapTest > testBasicValidation PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testHeartbeatWrongCoordinator 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testDescribeGroupStable PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testHeartbeatIllegalGeneration 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testDescribeGroupWrongCoordinator PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testDescribeGroupRebalancing 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testLeaderFailureInSyncGroup 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testGenerationIdIncrementsOnRebalance PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testSyncGroupFromIllegalGeneration PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testInvalidGroupId PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testHeartbeatUnknownGroup 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testListGroupsIncludesStableGroups PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testHeartbeatDuringRebalanceCausesRebalanceInProgress PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testJoinGroupInconsistentGroupProtocol PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testJoinGroupSessionTimeoutTooLarge PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testJoinGroupSessionTimeoutTooSmall PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testSyncGroupEmptyAssignment 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testCommitOffsetWithDefaultGeneration PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testJoinGroupFromUnchangedLeaderShouldRebalance PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testHeartbeatRebalanceInProgress PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testLeaveGroupUnknownGroup 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testListGroupsIncludesRebalancingGroups PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testSyncGroupFollowerAfterLeader PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testCommitOffsetInAwaitingSync 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testJoinGroupWrongCoordinator 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testJoinGroupUnknownConsumerExistingGroup PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testSyncGroupFromUnknownGroup 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testJoinGroupInconsistentProtocolType PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testCommitOffsetFromUnknownGroup PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testLeaveGroupWrongCoordinator 
PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testLeaveGroupUnknownConsumerExistingGroup PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testJoinGroupUnknownConsumerNewGroup PASSED

kafka.coordinator.GroupCoordinatorResponseTest > 
testJoinGroupFromUnchangedFollowerDoesNotRebalance PASSED

kafka.coordinator.GroupCoordinatorResponseTest > testValidJoinGroup