You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Ismael Juma <is...@juma.me.uk> on 2017/06/23 01:16:10 UTC

[VOTE] 0.11.0.0 RC2

Hello Kafka users, developers and client-developers,

This is the third candidate for release of Apache Kafka 0.11.0.0.

This is a major version release of Apache Kafka. It includes 32 new KIPs.
See the release notes and release plan (
https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
for more details. A few feature highlights:

* Exactly-once delivery and transactional messaging
* Streams exactly-once semantics
* Admin client with support for topic, ACLs and config management
* Record headers
* Request rate quotas
* Improved resiliency: replication protocol improvement and single-threaded
controller
* Richer and more efficient message format

Release notes for the 0.11.0.0 release:
http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html

*** Please download, test and vote by Tuesday, June 27, 6pm PT

Kafka's KEYS file containing PGP keys we use to sign the release:
http://kafka.apache.org/KEYS

* Release artifacts to be voted upon (source and binary):
http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/

* Maven artifacts to be voted upon:
https://repository.apache.org/content/groups/staging/org/apache/kafka/

* Javadoc:
http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/

* Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=8698fa1f41102f1664b05baa4d6953fc9564d91e

* Documentation:
http://kafka.apache.org/0110/documentation.html

* Protocol:
http://kafka.apache.org/0110/protocol.html

* Successful Jenkins builds for the 0.11.0 branch:
Unit/integration tests: https://builds.apache.org/job/kafka-0.11.0-jdk7/187/
System tests: pending (will send an update tomorrow)

/**************************************

Thanks,
Ismael

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks Jeff, that's helpful. To be clear, this should not affect the Java
client (max.request.size = 1 MB) or librdkafka-based clients
(message.max.bytes = 1MB) with the default settings. I'm a bit surprised
that Sarama doesn't have a similar mechanism. Seems like we'll have to live
with that.

Ismael

On Mon, Jun 26, 2017 at 9:51 PM, Jeff Chao <jc...@heroku.com> wrote:

> Hi,
>
> Heroku has been doing additional performance testing on (1) log compaction
> and, separately (2) Go clients with older message format against 0.11-rc2
> brokers with new message format.
>
> For log compaction, we've tested with messages using a single key, messages
> using unique keys, and messages with a bounded key range. There were no
> notable negative performance impacts.
>
> For client testing with old format vs new format, we had Sarama Go async
> producer clients speaking their older client protocol versions and had
> messages producing in a tight loop. This resulted in a high percentage of
> errors, though some messages went through:
>
> Failed to produce message kafka: Failed to produce message to topic
> rc2-topic: kafka server: Message was too large, server rejected it to avoid
> allocation error.
>
> Although this is to be expected as mentioned in the docs (
> http://kafka.apache.org/0110/documentation.html#upgrade_11_message_format)
> where in aggregate messages may become larger than max.message.bytes from
> the broker, we'd like to point out that this might be confusing for users
> running older clients against 0.11. That said, users can however work
> around this issue by tuning their request size to be less than
> max.message.bytes.
>
> This, along with the testing previously mentioned by Tom wraps up our
> performance testing. Overall, we're a +1 (non-binding) for this release,
> but wanted to point out the client issue above.
>
> Thanks,
> Jeff
>
> On Mon, Jun 26, 2017 at 12:41 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > To answer your questions:
> >
> > 1. Yes, the issues exists in trunk too.
> >
> > 2. I haven't checked with Cygwin, but I can give it a try.
> >
> > And thanks for addressing this issue. I can confirm with your PR I no
> > longer see it.
> > But now that the tests progress I see quite a few errors like this in
> > core:
> >
> > kafka.server.ReplicaFetchTest > classMethod FAILED
> >     java.lang.AssertionError: Found unexpected threads,
> > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> > ProcessThread(sid:0 cport:56565):, metrics-mete
> > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> > Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> > cport:59720):, ZkClie
> > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> > producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> > /127.0.0.1:54926 w
> > orkers Thread 2, Test worker, SyncThread:0,
> > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> > 2 to /127.0.0.1:54926 workers Thread 3,
> > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> > alizer, metrics-meter-tick-thread-1)
> >
> > I tested on a VM and a physical machine, and both give me a lot of errors
> > like this.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> >
> > From:   Ismael Juma <is...@gmail.com>
> > To:     Vahid S Hashemian <va...@us.ibm.com>
> > Cc:     dev@kafka.apache.org, kafka-clients
> > <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> > Date:   06/26/2017 03:53 AM
> > Subject:        Re: [VOTE] 0.11.0.0 RC2
> >
> >
> >
> > Hi Vahid,
> >
> > Sorry for not replying to the previous email, I had missed it. A couple
> of
> > questions:
> >
> > 1. Is this also happening in trunk? Seems like it should be the case for
> > months and seemingly no-one reported it until the RC stage.
> > 2. Is it correct that this only happens when compiling on Windows without
> > Cygwin?
> >
> > I believe the following PR should fix it, please verify:
> >
> > https://github.com/apache/kafka/pull/3431
> >
> > Ismael
> >
> > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> > vahidhashemian@us.ibm.com> wrote:
> >
> > > Hi Ismael,
> > >
> > > Not sure if my response on RC1 was lost or this issue is not a
> > > show-stopper:
> > >
> > > I checked again and with RC2, tests still fail in my Windown 64 bit
> > > environment.
> > >
> > > :clients:checkstyleMain
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > > [ApiExceptionBuilder, BrokerNotAvailableException,
> > > ClusterAuthorizationException, ConcurrentTransactionsException,
> > > ControllerMovedException, CoordinatorLoadInProgressException,
> > > CoordinatorNotAvailableException, CorruptRecordException,
> > > DuplicateSequenceNumberException, GroupAuthorizationException,
> > > IllegalGenerationException, IllegalSaslStateException,
> > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > > InvalidConfigurationException, InvalidFetchSizeException,
> > > InvalidGroupIdException, InvalidPartitionsException,
> > > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > > InvalidReplicationFactorException, InvalidRequestException,
> > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > > InvalidTimestampException, InvalidTopicException,
> > InvalidTxnStateException,
> > > InvalidTxnTimeoutException, LeaderNotAvailableException,
> > NetworkException,
> > > NotControllerException, NotCoordinatorException,
> > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > > OffsetOutOfRangeException, OperationNotAttemptedException,
> > > OutOfOrderSequenceException, PolicyViolationException,
> > > ProducerFencedException, RebalanceInProgressException,
> > > RecordBatchTooLargeException, RecordTooLargeException,
> > > ReplicaNotAvailableException, SecurityDisabledException,
> > TimeoutException,
> > > TopicAuthorizationException, TopicExistsException,
> > > TransactionCoordinatorFencedException,
> > TransactionalIdAuthorizationException,
> > > UnknownMemberIdException, UnknownServerException,
> > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
> eption,
> > > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > > [ClassDataAbstractionCoupling]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Fan-Out Complexity is 60 (max allowed is 40).
> > [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > :clients:checkstyleMain FAILED
> > >
> > > FAILURE: Build failed with an exception.
> > >
> > > Thanks.
> > > --Vahid
> > >
> > >
> > >
> > > From:        Ismael Juma <is...@juma.me.uk>
> > > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > > kafka-clients <ka...@googlegroups.com>
> > > Date:        06/22/2017 06:16 PM
> > > Subject:        [VOTE] 0.11.0.0 RC2
> > > Sent by:        ismaelj@gmail.com
> > > ------------------------------
> > >
> > >
> > >
> > > Hello Kafka users, developers and client-developers,
> > >
> > > This is the third candidate for release of Apache Kafka 0.11.0.0.
> > >
> > > This is a major version release of Apache Kafka. It includes 32 new
> > KIPs.
> > > See the release notes and release plan (
> > > https://cwiki.apache.org/confluence/display/KAFKA/Release+
> Plan+0.11.0.0)
> > > for more details. A few feature highlights:
> > >
> > > * Exactly-once delivery and transactional messaging
> > > * Streams exactly-once semantics
> > > * Admin client with support for topic, ACLs and config management
> > > * Record headers
> > > * Request rate quotas
> > > * Improved resiliency: replication protocol improvement and
> > single-threaded
> > > controller
> > > * Richer and more efficient message format
> > >
> > > Release notes for the 0.11.0.0 release:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> > >
> > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> > >
> > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > http://kafka.apache.org/KEYS
> > >
> > > * Release artifacts to be voted upon (source and binary):
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> > >
> > > * Maven artifacts to be voted upon:
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >
> > > * Javadoc:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> > >
> > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> > >
> > > * Documentation:
> > > http://kafka.apache.org/0110/documentation.html
> > >
> > > * Protocol:
> > > http://kafka.apache.org/0110/protocol.html
> > >
> > > * Successful Jenkins builds for the 0.11.0 branch:
> > > Unit/integration tests: https://builds.apache.org/job/
> > > kafka-0.11.0-jdk7/187/
> > > System tests: pending (will send an update tomorrow)
> > >
> > > /**************************************
> > >
> > > Thanks,
> > > Ismael
> > >
> > >
> > >
> > >
> >
> >
> >
> >
> >
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks Jeff, that's helpful. To be clear, this should not affect the Java
client (max.request.size = 1 MB) or librdkafka-based clients
(message.max.bytes = 1MB) with the default settings. I'm a bit surprised
that Sarama doesn't have a similar mechanism. Seems like we'll have to live
with that.

Ismael

On Mon, Jun 26, 2017 at 9:51 PM, Jeff Chao <jc...@heroku.com> wrote:

> Hi,
>
> Heroku has been doing additional performance testing on (1) log compaction
> and, separately (2) Go clients with older message format against 0.11-rc2
> brokers with new message format.
>
> For log compaction, we've tested with messages using a single key, messages
> using unique keys, and messages with a bounded key range. There were no
> notable negative performance impacts.
>
> For client testing with old format vs new format, we had Sarama Go async
> producer clients speaking their older client protocol versions and had
> messages producing in a tight loop. This resulted in a high percentage of
> errors, though some messages went through:
>
> Failed to produce message kafka: Failed to produce message to topic
> rc2-topic: kafka server: Message was too large, server rejected it to avoid
> allocation error.
>
> Although this is to be expected as mentioned in the docs (
> http://kafka.apache.org/0110/documentation.html#upgrade_11_message_format)
> where in aggregate messages may become larger than max.message.bytes from
> the broker, we'd like to point out that this might be confusing for users
> running older clients against 0.11. That said, users can however work
> around this issue by tuning their request size to be less than
> max.message.bytes.
>
> This, along with the testing previously mentioned by Tom wraps up our
> performance testing. Overall, we're a +1 (non-binding) for this release,
> but wanted to point out the client issue above.
>
> Thanks,
> Jeff
>
> On Mon, Jun 26, 2017 at 12:41 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > To answer your questions:
> >
> > 1. Yes, the issues exists in trunk too.
> >
> > 2. I haven't checked with Cygwin, but I can give it a try.
> >
> > And thanks for addressing this issue. I can confirm with your PR I no
> > longer see it.
> > But now that the tests progress I see quite a few errors like this in
> > core:
> >
> > kafka.server.ReplicaFetchTest > classMethod FAILED
> >     java.lang.AssertionError: Found unexpected threads,
> > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> > ProcessThread(sid:0 cport:56565):, metrics-mete
> > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> > Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> > cport:59720):, ZkClie
> > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> > producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> > /127.0.0.1:54926 w
> > orkers Thread 2, Test worker, SyncThread:0,
> > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> > 2 to /127.0.0.1:54926 workers Thread 3,
> > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> > alizer, metrics-meter-tick-thread-1)
> >
> > I tested on a VM and a physical machine, and both give me a lot of errors
> > like this.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> >
> > From:   Ismael Juma <is...@gmail.com>
> > To:     Vahid S Hashemian <va...@us.ibm.com>
> > Cc:     dev@kafka.apache.org, kafka-clients
> > <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> > Date:   06/26/2017 03:53 AM
> > Subject:        Re: [VOTE] 0.11.0.0 RC2
> >
> >
> >
> > Hi Vahid,
> >
> > Sorry for not replying to the previous email, I had missed it. A couple
> of
> > questions:
> >
> > 1. Is this also happening in trunk? Seems like it should be the case for
> > months and seemingly no-one reported it until the RC stage.
> > 2. Is it correct that this only happens when compiling on Windows without
> > Cygwin?
> >
> > I believe the following PR should fix it, please verify:
> >
> > https://github.com/apache/kafka/pull/3431
> >
> > Ismael
> >
> > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> > vahidhashemian@us.ibm.com> wrote:
> >
> > > Hi Ismael,
> > >
> > > Not sure if my response on RC1 was lost or this issue is not a
> > > show-stopper:
> > >
> > > I checked again and with RC2, tests still fail in my Windown 64 bit
> > > environment.
> > >
> > > :clients:checkstyleMain
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > > [ApiExceptionBuilder, BrokerNotAvailableException,
> > > ClusterAuthorizationException, ConcurrentTransactionsException,
> > > ControllerMovedException, CoordinatorLoadInProgressException,
> > > CoordinatorNotAvailableException, CorruptRecordException,
> > > DuplicateSequenceNumberException, GroupAuthorizationException,
> > > IllegalGenerationException, IllegalSaslStateException,
> > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > > InvalidConfigurationException, InvalidFetchSizeException,
> > > InvalidGroupIdException, InvalidPartitionsException,
> > > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > > InvalidReplicationFactorException, InvalidRequestException,
> > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > > InvalidTimestampException, InvalidTopicException,
> > InvalidTxnStateException,
> > > InvalidTxnTimeoutException, LeaderNotAvailableException,
> > NetworkException,
> > > NotControllerException, NotCoordinatorException,
> > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > > OffsetOutOfRangeException, OperationNotAttemptedException,
> > > OutOfOrderSequenceException, PolicyViolationException,
> > > ProducerFencedException, RebalanceInProgressException,
> > > RecordBatchTooLargeException, RecordTooLargeException,
> > > ReplicaNotAvailableException, SecurityDisabledException,
> > TimeoutException,
> > > TopicAuthorizationException, TopicExistsException,
> > > TransactionCoordinatorFencedException,
> > TransactionalIdAuthorizationException,
> > > UnknownMemberIdException, UnknownServerException,
> > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
> eption,
> > > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > > [ClassDataAbstractionCoupling]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Fan-Out Complexity is 60 (max allowed is 40).
> > [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > :clients:checkstyleMain FAILED
> > >
> > > FAILURE: Build failed with an exception.
> > >
> > > Thanks.
> > > --Vahid
> > >
> > >
> > >
> > > From:        Ismael Juma <is...@juma.me.uk>
> > > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > > kafka-clients <ka...@googlegroups.com>
> > > Date:        06/22/2017 06:16 PM
> > > Subject:        [VOTE] 0.11.0.0 RC2
> > > Sent by:        ismaelj@gmail.com
> > > ------------------------------
> > >
> > >
> > >
> > > Hello Kafka users, developers and client-developers,
> > >
> > > This is the third candidate for release of Apache Kafka 0.11.0.0.
> > >
> > > This is a major version release of Apache Kafka. It includes 32 new
> > KIPs.
> > > See the release notes and release plan (
> > > https://cwiki.apache.org/confluence/display/KAFKA/Release+
> Plan+0.11.0.0)
> > > for more details. A few feature highlights:
> > >
> > > * Exactly-once delivery and transactional messaging
> > > * Streams exactly-once semantics
> > > * Admin client with support for topic, ACLs and config management
> > > * Record headers
> > > * Request rate quotas
> > > * Improved resiliency: replication protocol improvement and
> > single-threaded
> > > controller
> > > * Richer and more efficient message format
> > >
> > > Release notes for the 0.11.0.0 release:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> > >
> > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> > >
> > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > http://kafka.apache.org/KEYS
> > >
> > > * Release artifacts to be voted upon (source and binary):
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> > >
> > > * Maven artifacts to be voted upon:
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >
> > > * Javadoc:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> > >
> > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> > >
> > > * Documentation:
> > > http://kafka.apache.org/0110/documentation.html
> > >
> > > * Protocol:
> > > http://kafka.apache.org/0110/protocol.html
> > >
> > > * Successful Jenkins builds for the 0.11.0 branch:
> > > Unit/integration tests: https://builds.apache.org/job/
> > > kafka-0.11.0-jdk7/187/
> > > System tests: pending (will send an update tomorrow)
> > >
> > > /**************************************
> > >
> > > Thanks,
> > > Ismael
> > >
> > >
> > >
> > >
> >
> >
> >
> >
> >
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Jeff Chao <jc...@heroku.com>.
Hi,

Heroku has been doing additional performance testing on (1) log compaction
and, separately (2) Go clients with older message format against 0.11-rc2
brokers with new message format.

For log compaction, we've tested with messages using a single key, messages
using unique keys, and messages with a bounded key range. There were no
notable negative performance impacts.

For client testing with old format vs new format, we had Sarama Go async
producer clients speaking their older client protocol versions and had
messages producing in a tight loop. This resulted in a high percentage of
errors, though some messages went through:

Failed to produce message kafka: Failed to produce message to topic
rc2-topic: kafka server: Message was too large, server rejected it to avoid
allocation error.

Although this is to be expected as mentioned in the docs (
http://kafka.apache.org/0110/documentation.html#upgrade_11_message_format)
where in aggregate messages may become larger than max.message.bytes from
the broker, we'd like to point out that this might be confusing for users
running older clients against 0.11. That said, users can however work
around this issue by tuning their request size to be less than
max.message.bytes.

This, along with the testing previously mentioned by Tom wraps up our
performance testing. Overall, we're a +1 (non-binding) for this release,
but wanted to point out the client issue above.

Thanks,
Jeff

On Mon, Jun 26, 2017 at 12:41 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> To answer your questions:
>
> 1. Yes, the issues exists in trunk too.
>
> 2. I haven't checked with Cygwin, but I can give it a try.
>
> And thanks for addressing this issue. I can confirm with your PR I no
> longer see it.
> But now that the tests progress I see quite a few errors like this in
> core:
>
> kafka.server.ReplicaFetchTest > classMethod FAILED
>     java.lang.AssertionError: Found unexpected threads,
> allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> ProcessThread(sid:0 cport:56565):, metrics-mete
> r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> cport:59720):, ZkClie
> nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> /127.0.0.1:54926 w
> orkers Thread 2, Test worker, SyncThread:0,
> NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> 2 to /127.0.0.1:54926 workers Thread 3,
> ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> alizer, metrics-meter-tick-thread-1)
>
> I tested on a VM and a physical machine, and both give me a lot of errors
> like this.
>
> Thanks.
> --Vahid
>
>
>
>
> From:   Ismael Juma <is...@gmail.com>
> To:     Vahid S Hashemian <va...@us.ibm.com>
> Cc:     dev@kafka.apache.org, kafka-clients
> <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> Date:   06/26/2017 03:53 AM
> Subject:        Re: [VOTE] 0.11.0.0 RC2
>
>
>
> Hi Vahid,
>
> Sorry for not replying to the previous email, I had missed it. A couple of
> questions:
>
> 1. Is this also happening in trunk? Seems like it should be the case for
> months and seemingly no-one reported it until the RC stage.
> 2. Is it correct that this only happens when compiling on Windows without
> Cygwin?
>
> I believe the following PR should fix it, please verify:
>
> https://github.com/apache/kafka/pull/3431
>
> Ismael
>
> On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > Not sure if my response on RC1 was lost or this issue is not a
> > show-stopper:
> >
> > I checked again and with RC2, tests still fail in my Windown 64 bit
> > environment.
> >
> > :clients:checkstyleMain
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > [ApiExceptionBuilder, BrokerNotAvailableException,
> > ClusterAuthorizationException, ConcurrentTransactionsException,
> > ControllerMovedException, CoordinatorLoadInProgressException,
> > CoordinatorNotAvailableException, CorruptRecordException,
> > DuplicateSequenceNumberException, GroupAuthorizationException,
> > IllegalGenerationException, IllegalSaslStateException,
> > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > InvalidConfigurationException, InvalidFetchSizeException,
> > InvalidGroupIdException, InvalidPartitionsException,
> > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > InvalidReplicationFactorException, InvalidRequestException,
> > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > InvalidTimestampException, InvalidTopicException,
> InvalidTxnStateException,
> > InvalidTxnTimeoutException, LeaderNotAvailableException,
> NetworkException,
> > NotControllerException, NotCoordinatorException,
> > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > OffsetOutOfRangeException, OperationNotAttemptedException,
> > OutOfOrderSequenceException, PolicyViolationException,
> > ProducerFencedException, RebalanceInProgressException,
> > RecordBatchTooLargeException, RecordTooLargeException,
> > ReplicaNotAvailableException, SecurityDisabledException,
> TimeoutException,
> > TopicAuthorizationException, TopicExistsException,
> > TransactionCoordinatorFencedException,
> TransactionalIdAuthorizationException,
> > UnknownMemberIdException, UnknownServerException,
> > UnknownTopicOrPartitionException, UnsupportedForMessageFormatException,
> > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > [ClassDataAbstractionCoupling]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Fan-Out Complexity is 60 (max allowed is 40).
> [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 (max
> > allowed is 40). [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 (max
> > allowed is 40). [ClassFanOutComplexity]
> > :clients:checkstyleMain FAILED
> >
> > FAILURE: Build failed with an exception.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> > From:        Ismael Juma <is...@juma.me.uk>
> > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > kafka-clients <ka...@googlegroups.com>
> > Date:        06/22/2017 06:16 PM
> > Subject:        [VOTE] 0.11.0.0 RC2
> > Sent by:        ismaelj@gmail.com
> > ------------------------------
> >
> >
> >
> > Hello Kafka users, developers and client-developers,
> >
> > This is the third candidate for release of Apache Kafka 0.11.0.0.
> >
> > This is a major version release of Apache Kafka. It includes 32 new
> KIPs.
> > See the release notes and release plan (
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> > for more details. A few feature highlights:
> >
> > * Exactly-once delivery and transactional messaging
> > * Streams exactly-once semantics
> > * Admin client with support for topic, ACLs and config management
> > * Record headers
> > * Request rate quotas
> > * Improved resiliency: replication protocol improvement and
> single-threaded
> > controller
> > * Richer and more efficient message format
> >
> > Release notes for the 0.11.0.0 release:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > http://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> >
> > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> >
> > * Documentation:
> > http://kafka.apache.org/0110/documentation.html
> >
> > * Protocol:
> > http://kafka.apache.org/0110/protocol.html
> >
> > * Successful Jenkins builds for the 0.11.0 branch:
> > Unit/integration tests: https://builds.apache.org/job/
> > kafka-0.11.0-jdk7/187/
> > System tests: pending (will send an update tomorrow)
> >
> > /**************************************
> >
> > Thanks,
> > Ismael
> >
> >
> >
> >
>
>
>
>
>

Re: [kafka-clients] Re: [VOTE] 0.11.0.0 RC2

Posted by Jun Rao <ju...@confluent.io>.
Hi, Ismael,

Thanks for running the release. +1. Verified quickstart on the 2.11 binary.

Jun

On Mon, Jun 26, 2017 at 3:53 PM, Ismael Juma <is...@gmail.com> wrote:

> Hi Vahid,
>
> There are a few known issues when running Kafka on Windows. A PR with some
> fixes is: https://github.com/apache/kafka/pull/3283. The fact that the
> index cannot be accessed indicates that it may be a similar issue. I
> suggest we move this discussion to the relevant JIRAs instead of the
> release thread.
>
> Ismael
>
> On Mon, Jun 26, 2017 at 11:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
>> Hi Ismael,
>>
>> This is the output of core tests from the start until the first failed
>> test.
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithUnevenRacks
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAware
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithUnevenReplicas
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlreadyAssigned
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAware PASSED
>>
>> kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAwareWith6Partitions
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRac
>> kAwareWith6PartitionsAnd3Brokers PASSED
>>
>> kafka.admin.AdminRackAwareTest > testGetRackAlternatedBrokerListAndAssignReplicasToBrokers
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testMoreReplicasThanRacks PASSED
>>
>> kafka.admin.AdminRackAwareTest > testSingleRack PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithRandomStartIndex
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testLargeNumberPartitionsAssignment
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testLessReplicasThanRacks PASSED
>>
>> kafka.admin.AclCommandTest > testInvalidAuthorizerProperty PASSED
>>
>> kafka.admin.ConfigCommandTest > testScramCredentials PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldParseArgumentsForTopicsEntityType
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > testUserClientQuotaOpts PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldAddTopicConfig PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldAddClientConfig PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldDeleteBrokerConfig PASSED
>>
>> kafka.admin.DeleteConsumerGroupTest > testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > testQuotaConfigEntity PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedBracketConfig
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldFailIfUnrecognisedEntityType PASSED
>>
>> kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted
>> PASSED
>>
>> kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED
>>
>> kafka.admin.BrokerApiVersionsCommandTest > checkBrokerApiVersionCommandOutput
>> PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldRemoveThrottleReplicaListBasedOnProposedAssignment PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindMovingReplicasMultipleTopics PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldNotOverwriteExistingPropertiesWhenLimitIsAdded PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindMovingReplicasMultipleTopicsAndPartitions PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldRemoveThrottleLimitFromAllBrokers PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicas
>> PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindMovingReplicasMultiplePartitions PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest > shouldSetQuotaLimit PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindMovingReplicasWhenProposedIsSubsetOfExisting PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest > shouldUpdateQuotaLimit PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindTwoMovingReplicasInSamePartition PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedEntityName
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldSupportCommaSeparatedValues PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedConfig
>> PASSED
>>
>> kafka.admin.DeleteConsumerGroupTest > testGroupTopicWideDeleteInZKDo
>> esNothingForActiveGroupConsumingMultipleTopics PASSED
>>
>> kafka.admin.AddPartitionsTest > testReplicaPlacementAllServers PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldParseArgumentsForBrokersEntityType
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldAddBrokerConfig PASSED
>>
>> kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED
>>
>> kafka.admin.ConfigCommandTest > testQuotaDescribeEntities PASSED
>>
>> kafka.admin.AdminTest > testGetBrokerMetadatas PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldParseArgumentsForClientsEntityType
>> PASSED
>>
>> kafka.admin.AclCommandTest > testAclCli PASSED
>>
>> kafka.admin.ReassignPartitionsIntegrationTest > testRackAwareReassign
>> PASSED
>>
>> kafka.admin.AdminTest > testBootstrapClientIdConfig PASSED
>>
>> kafka.admin.ReassignPartitionsClusterTest >
>> shouldExecuteThrottledReassignment FAILED
>>     java.nio.file.FileSystemException: C:\Users\IBM_AD~1\AppData\Loca
>> l\Temp\kafka-719085320148197500\my-topic-0\00000000000000000000.index:
>> The process cannot access the file because it is being used by another
>> process.
>>
>>
>> From the error message, it sounds like one of the prior tests does not do
>> a proper clean-up?!
>>
>> Thanks.
>> --Vahid
>>
>>
>>
>>
>> From:        Ismael Juma <is...@juma.me.uk>
>> To:        dev@kafka.apache.org
>> Cc:        kafka-clients <ka...@googlegroups.com>, Kafka Users <
>> users@kafka.apache.org>
>> Date:        06/26/2017 01:54 PM
>> Subject:        Re: [VOTE] 0.11.0.0 RC2
>> Sent by:        ismaelj@gmail.com
>> ------------------------------
>>
>>
>>
>> Hi Vahid,
>>
>> Can you please check which test fails first? The errors you mentioned can
>> happen if a test fails and doesn't clean-up properly.
>>
>> Ismael
>>
>> On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
>> vahidhashemian@us.ibm.com> wrote:
>>
>> > Hi Ismael,
>> >
>> > To answer your questions:
>> >
>> > 1. Yes, the issues exists in trunk too.
>> >
>> > 2. I haven't checked with Cygwin, but I can give it a try.
>> >
>> > And thanks for addressing this issue. I can confirm with your PR I no
>> > longer see it.
>> > But now that the tests progress I see quite a few errors like this in
>> > core:
>> >
>> > kafka.server.ReplicaFetchTest > classMethod FAILED
>> >     java.lang.AssertionError: Found unexpected threads,
>> > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
>> > ProcessThread(sid:0 cport:56565):, metrics-mete
>> > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
>> > Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
>> > cport:59720):, ZkClie
>> > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
>> > producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942
>> to
>> > /127.0.0.1:54926 w
>> > orkers Thread 2, Test worker, SyncThread:0,
>> > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
>> > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
>> > 2 to /127.0.0.1:54926 workers Thread 3,
>> > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
>> > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
>> > alizer, metrics-meter-tick-thread-1)
>> >
>> > I tested on a VM and a physical machine, and both give me a lot of
>> errors
>> > like this.
>> >
>> > Thanks.
>> > --Vahid
>> >
>> >
>> >
>> >
>> > From:   Ismael Juma <is...@gmail.com>
>> > To:     Vahid S Hashemian <va...@us.ibm.com>
>> > Cc:     dev@kafka.apache.org, kafka-clients
>> > <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
>> > Date:   06/26/2017 03:53 AM
>> > Subject:        Re: [VOTE] 0.11.0.0 RC2
>> >
>> >
>> >
>> > Hi Vahid,
>> >
>> > Sorry for not replying to the previous email, I had missed it. A couple
>> of
>> > questions:
>> >
>> > 1. Is this also happening in trunk? Seems like it should be the case for
>> > months and seemingly no-one reported it until the RC stage.
>> > 2. Is it correct that this only happens when compiling on Windows
>> without
>> > Cygwin?
>> >
>> > I believe the following PR should fix it, please verify:
>> >
>> > https://github.com/apache/kafka/pull/3431
>> >
>> > Ismael
>> >
>> > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
>> > vahidhashemian@us.ibm.com> wrote:
>> >
>> > > Hi Ismael,
>> > >
>> > > Not sure if my response on RC1 was lost or this issue is not a
>> > > show-stopper:
>> > >
>> > > I checked again and with RC2, tests still fail in my Windown 64 bit
>> > > environment.
>> > >
>> > > :clients:checkstyleMain
>> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
>> > >
>> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
>> > protocol\Errors.java:89:1:
>> > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
>> > > [ApiExceptionBuilder, BrokerNotAvailableException,
>> > > ClusterAuthorizationException, ConcurrentTransactionsException,
>> > > ControllerMovedException, CoordinatorLoadInProgressException,
>> > > CoordinatorNotAvailableException, CorruptRecordException,
>> > > DuplicateSequenceNumberException, GroupAuthorizationException,
>> > > IllegalGenerationException, IllegalSaslStateException,
>> > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
>> > > InvalidConfigurationException, InvalidFetchSizeException,
>> > > InvalidGroupIdException, InvalidPartitionsException,
>> > > InvalidPidMappingException, InvalidReplicaAssignmentException,
>> > > InvalidReplicationFactorException, InvalidRequestException,
>> > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
>> > > InvalidTimestampException, InvalidTopicException,
>> > InvalidTxnStateException,
>> > > InvalidTxnTimeoutException, LeaderNotAvailableException,
>> > NetworkException,
>> > > NotControllerException, NotCoordinatorException,
>> > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
>> > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
>> > > OffsetOutOfRangeException, OperationNotAttemptedException,
>> > > OutOfOrderSequenceException, PolicyViolationException,
>> > > ProducerFencedException, RebalanceInProgressException,
>> > > RecordBatchTooLargeException, RecordTooLargeException,
>> > > ReplicaNotAvailableException, SecurityDisabledException,
>> > TimeoutException,
>> > > TopicAuthorizationException, TopicExistsException,
>> > > TransactionCoordinatorFencedException,
>> > TransactionalIdAuthorizationException,
>> > > UnknownMemberIdException, UnknownServerException,
>> > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
>> eption,
>> > > UnsupportedSaslMechanismException, UnsupportedVersionException].
>> > > [ClassDataAbstractionCoupling]
>> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
>> > >
>> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
>> > protocol\Errors.java:89:1:
>> > > Class Fan-Out Complexity is 60 (max allowed is 40).
>> > [ClassFanOutComplexity]
>> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
>> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
>> > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
>> (max
>> > > allowed is 40). [ClassFanOutComplexity]
>> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
>> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
>> > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42
>> (max
>> > > allowed is 40). [ClassFanOutComplexity]
>> > > :clients:checkstyleMain FAILED
>> > >
>> > > FAILURE: Build failed with an exception.
>> > >
>> > > Thanks.
>> > > --Vahid
>> > >
>> > >
>> > >
>> > > From:        Ismael Juma <is...@juma.me.uk>
>> > > To:        dev@kafka.apache.org, Kafka Users <users@kafka.apache.org
>> >,
>> > > kafka-clients <ka...@googlegroups.com>
>> > > Date:        06/22/2017 06:16 PM
>> > > Subject:        [VOTE] 0.11.0.0 RC2
>> > > Sent by:        ismaelj@gmail.com
>> > > ------------------------------
>> > >
>> > >
>> > >
>> > > Hello Kafka users, developers and client-developers,
>> > >
>> > > This is the third candidate for release of Apache Kafka 0.11.0.0.
>> > >
>> > > This is a major version release of Apache Kafka. It includes 32 new
>> > KIPs.
>> > > See the release notes and release plan (
>> > > https://cwiki.apache.org/confluence/display/KAFKA/Release+
>> Plan+0.11.0.0)
>> > > for more details. A few feature highlights:
>> > >
>> > > * Exactly-once delivery and transactional messaging
>> > > * Streams exactly-once semantics
>> > > * Admin client with support for topic, ACLs and config management
>> > > * Record headers
>> > > * Request rate quotas
>> > > * Improved resiliency: replication protocol improvement and
>> > single-threaded
>> > > controller
>> > > * Richer and more efficient message format
>> > >
>> > > Release notes for the 0.11.0.0 release:
>> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>> > >
>> > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
>> > >
>> > > Kafka's KEYS file containing PGP keys we use to sign the release:
>> > > http://kafka.apache.org/KEYS
>> > >
>> > > * Release artifacts to be voted upon (source and binary):
>> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>> > >
>> > > * Maven artifacts to be voted upon:
>> > > https://repository.apache.org/content/groups/staging/org/apa
>> che/kafka/
>> > >
>> > > * Javadoc:
>> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>> > >
>> > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
>> > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
>> > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
>> > >
>> > > * Documentation:
>> > > http://kafka.apache.org/0110/documentation.html
>> > >
>> > > * Protocol:
>> > > http://kafka.apache.org/0110/protocol.html
>> > >
>> > > * Successful Jenkins builds for the 0.11.0 branch:
>> > > Unit/integration tests: https://builds.apache.org/job/
>> > > kafka-0.11.0-jdk7/187/
>> > > System tests: pending (will send an update tomorrow)
>> > >
>> > > /**************************************
>> > >
>> > > Thanks,
>> > > Ismael
>> > >
>> > >
>> > >
>> > >
>> >
>> >
>> >
>> >
>> >
>>
>>
>>
>>
> --
> You received this message because you are subscribed to the Google Groups
> "kafka-clients" group.
> To unsubscribe from this group and stop receiving emails from it, send an
> email to kafka-clients+unsubscribe@googlegroups.com.
> To post to this group, send email to kafka-clients@googlegroups.com.
> Visit this group at https://groups.google.com/group/kafka-clients.
> To view this discussion on the web visit https://groups.google.com/d/
> msgid/kafka-clients/CAD5tkZYey%3DLSri2NpGnh1RNGfCTtZRJyCNMOU7
> cF0nZW5Ec38g%40mail.gmail.com
> <https://groups.google.com/d/msgid/kafka-clients/CAD5tkZYey%3DLSri2NpGnh1RNGfCTtZRJyCNMOU7cF0nZW5Ec38g%40mail.gmail.com?utm_medium=email&utm_source=footer>
> .
>
> For more options, visit https://groups.google.com/d/optout.
>

Re: [kafka-clients] Re: [VOTE] 0.11.0.0 RC2

Posted by Jun Rao <ju...@confluent.io>.
Hi, Ismael,

Thanks for running the release. +1. Verified quickstart on the 2.11 binary.

Jun

On Mon, Jun 26, 2017 at 3:53 PM, Ismael Juma <is...@gmail.com> wrote:

> Hi Vahid,
>
> There are a few known issues when running Kafka on Windows. A PR with some
> fixes is: https://github.com/apache/kafka/pull/3283. The fact that the
> index cannot be accessed indicates that it may be a similar issue. I
> suggest we move this discussion to the relevant JIRAs instead of the
> release thread.
>
> Ismael
>
> On Mon, Jun 26, 2017 at 11:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
>> Hi Ismael,
>>
>> This is the output of core tests from the start until the first failed
>> test.
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithUnevenRacks
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAware
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithUnevenReplicas
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlreadyAssigned
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAware PASSED
>>
>> kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAwareWith6Partitions
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRac
>> kAwareWith6PartitionsAnd3Brokers PASSED
>>
>> kafka.admin.AdminRackAwareTest > testGetRackAlternatedBrokerListAndAssignReplicasToBrokers
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testMoreReplicasThanRacks PASSED
>>
>> kafka.admin.AdminRackAwareTest > testSingleRack PASSED
>>
>> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithRandomStartIndex
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testLargeNumberPartitionsAssignment
>> PASSED
>>
>> kafka.admin.AdminRackAwareTest > testLessReplicasThanRacks PASSED
>>
>> kafka.admin.AclCommandTest > testInvalidAuthorizerProperty PASSED
>>
>> kafka.admin.ConfigCommandTest > testScramCredentials PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldParseArgumentsForTopicsEntityType
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > testUserClientQuotaOpts PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldAddTopicConfig PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldAddClientConfig PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldDeleteBrokerConfig PASSED
>>
>> kafka.admin.DeleteConsumerGroupTest > testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > testQuotaConfigEntity PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedBracketConfig
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldFailIfUnrecognisedEntityType PASSED
>>
>> kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted
>> PASSED
>>
>> kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED
>>
>> kafka.admin.BrokerApiVersionsCommandTest > checkBrokerApiVersionCommandOutput
>> PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldRemoveThrottleReplicaListBasedOnProposedAssignment PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindMovingReplicasMultipleTopics PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldNotOverwriteExistingPropertiesWhenLimitIsAdded PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindMovingReplicasMultipleTopicsAndPartitions PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldRemoveThrottleLimitFromAllBrokers PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicas
>> PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindMovingReplicasMultiplePartitions PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest > shouldSetQuotaLimit PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindMovingReplicasWhenProposedIsSubsetOfExisting PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest > shouldUpdateQuotaLimit PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldFindTwoMovingReplicasInSamePartition PASSED
>>
>> kafka.admin.ReassignPartitionsCommandTest >
>> shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedEntityName
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldSupportCommaSeparatedValues PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedConfig
>> PASSED
>>
>> kafka.admin.DeleteConsumerGroupTest > testGroupTopicWideDeleteInZKDo
>> esNothingForActiveGroupConsumingMultipleTopics PASSED
>>
>> kafka.admin.AddPartitionsTest > testReplicaPlacementAllServers PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldParseArgumentsForBrokersEntityType
>> PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldAddBrokerConfig PASSED
>>
>> kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED
>>
>> kafka.admin.ConfigCommandTest > testQuotaDescribeEntities PASSED
>>
>> kafka.admin.AdminTest > testGetBrokerMetadatas PASSED
>>
>> kafka.admin.ConfigCommandTest > shouldParseArgumentsForClientsEntityType
>> PASSED
>>
>> kafka.admin.AclCommandTest > testAclCli PASSED
>>
>> kafka.admin.ReassignPartitionsIntegrationTest > testRackAwareReassign
>> PASSED
>>
>> kafka.admin.AdminTest > testBootstrapClientIdConfig PASSED
>>
>> kafka.admin.ReassignPartitionsClusterTest >
>> shouldExecuteThrottledReassignment FAILED
>>     java.nio.file.FileSystemException: C:\Users\IBM_AD~1\AppData\Loca
>> l\Temp\kafka-719085320148197500\my-topic-0\00000000000000000000.index:
>> The process cannot access the file because it is being used by another
>> process.
>>
>>
>> From the error message, it sounds like one of the prior tests does not do
>> a proper clean-up?!
>>
>> Thanks.
>> --Vahid
>>
>>
>>
>>
>> From:        Ismael Juma <is...@juma.me.uk>
>> To:        dev@kafka.apache.org
>> Cc:        kafka-clients <ka...@googlegroups.com>, Kafka Users <
>> users@kafka.apache.org>
>> Date:        06/26/2017 01:54 PM
>> Subject:        Re: [VOTE] 0.11.0.0 RC2
>> Sent by:        ismaelj@gmail.com
>> ------------------------------
>>
>>
>>
>> Hi Vahid,
>>
>> Can you please check which test fails first? The errors you mentioned can
>> happen if a test fails and doesn't clean-up properly.
>>
>> Ismael
>>
>> On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
>> vahidhashemian@us.ibm.com> wrote:
>>
>> > Hi Ismael,
>> >
>> > To answer your questions:
>> >
>> > 1. Yes, the issues exists in trunk too.
>> >
>> > 2. I haven't checked with Cygwin, but I can give it a try.
>> >
>> > And thanks for addressing this issue. I can confirm with your PR I no
>> > longer see it.
>> > But now that the tests progress I see quite a few errors like this in
>> > core:
>> >
>> > kafka.server.ReplicaFetchTest > classMethod FAILED
>> >     java.lang.AssertionError: Found unexpected threads,
>> > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
>> > ProcessThread(sid:0 cport:56565):, metrics-mete
>> > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
>> > Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
>> > cport:59720):, ZkClie
>> > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
>> > producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942
>> to
>> > /127.0.0.1:54926 w
>> > orkers Thread 2, Test worker, SyncThread:0,
>> > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
>> > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
>> > 2 to /127.0.0.1:54926 workers Thread 3,
>> > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
>> > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
>> > alizer, metrics-meter-tick-thread-1)
>> >
>> > I tested on a VM and a physical machine, and both give me a lot of
>> errors
>> > like this.
>> >
>> > Thanks.
>> > --Vahid
>> >
>> >
>> >
>> >
>> > From:   Ismael Juma <is...@gmail.com>
>> > To:     Vahid S Hashemian <va...@us.ibm.com>
>> > Cc:     dev@kafka.apache.org, kafka-clients
>> > <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
>> > Date:   06/26/2017 03:53 AM
>> > Subject:        Re: [VOTE] 0.11.0.0 RC2
>> >
>> >
>> >
>> > Hi Vahid,
>> >
>> > Sorry for not replying to the previous email, I had missed it. A couple
>> of
>> > questions:
>> >
>> > 1. Is this also happening in trunk? Seems like it should be the case for
>> > months and seemingly no-one reported it until the RC stage.
>> > 2. Is it correct that this only happens when compiling on Windows
>> without
>> > Cygwin?
>> >
>> > I believe the following PR should fix it, please verify:
>> >
>> > https://github.com/apache/kafka/pull/3431
>> >
>> > Ismael
>> >
>> > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
>> > vahidhashemian@us.ibm.com> wrote:
>> >
>> > > Hi Ismael,
>> > >
>> > > Not sure if my response on RC1 was lost or this issue is not a
>> > > show-stopper:
>> > >
>> > > I checked again and with RC2, tests still fail in my Windown 64 bit
>> > > environment.
>> > >
>> > > :clients:checkstyleMain
>> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
>> > >
>> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
>> > protocol\Errors.java:89:1:
>> > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
>> > > [ApiExceptionBuilder, BrokerNotAvailableException,
>> > > ClusterAuthorizationException, ConcurrentTransactionsException,
>> > > ControllerMovedException, CoordinatorLoadInProgressException,
>> > > CoordinatorNotAvailableException, CorruptRecordException,
>> > > DuplicateSequenceNumberException, GroupAuthorizationException,
>> > > IllegalGenerationException, IllegalSaslStateException,
>> > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
>> > > InvalidConfigurationException, InvalidFetchSizeException,
>> > > InvalidGroupIdException, InvalidPartitionsException,
>> > > InvalidPidMappingException, InvalidReplicaAssignmentException,
>> > > InvalidReplicationFactorException, InvalidRequestException,
>> > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
>> > > InvalidTimestampException, InvalidTopicException,
>> > InvalidTxnStateException,
>> > > InvalidTxnTimeoutException, LeaderNotAvailableException,
>> > NetworkException,
>> > > NotControllerException, NotCoordinatorException,
>> > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
>> > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
>> > > OffsetOutOfRangeException, OperationNotAttemptedException,
>> > > OutOfOrderSequenceException, PolicyViolationException,
>> > > ProducerFencedException, RebalanceInProgressException,
>> > > RecordBatchTooLargeException, RecordTooLargeException,
>> > > ReplicaNotAvailableException, SecurityDisabledException,
>> > TimeoutException,
>> > > TopicAuthorizationException, TopicExistsException,
>> > > TransactionCoordinatorFencedException,
>> > TransactionalIdAuthorizationException,
>> > > UnknownMemberIdException, UnknownServerException,
>> > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
>> eption,
>> > > UnsupportedSaslMechanismException, UnsupportedVersionException].
>> > > [ClassDataAbstractionCoupling]
>> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
>> > >
>> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
>> > protocol\Errors.java:89:1:
>> > > Class Fan-Out Complexity is 60 (max allowed is 40).
>> > [ClassFanOutComplexity]
>> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
>> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
>> > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
>> (max
>> > > allowed is 40). [ClassFanOutComplexity]
>> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
>> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
>> > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42
>> (max
>> > > allowed is 40). [ClassFanOutComplexity]
>> > > :clients:checkstyleMain FAILED
>> > >
>> > > FAILURE: Build failed with an exception.
>> > >
>> > > Thanks.
>> > > --Vahid
>> > >
>> > >
>> > >
>> > > From:        Ismael Juma <is...@juma.me.uk>
>> > > To:        dev@kafka.apache.org, Kafka Users <users@kafka.apache.org
>> >,
>> > > kafka-clients <ka...@googlegroups.com>
>> > > Date:        06/22/2017 06:16 PM
>> > > Subject:        [VOTE] 0.11.0.0 RC2
>> > > Sent by:        ismaelj@gmail.com
>> > > ------------------------------
>> > >
>> > >
>> > >
>> > > Hello Kafka users, developers and client-developers,
>> > >
>> > > This is the third candidate for release of Apache Kafka 0.11.0.0.
>> > >
>> > > This is a major version release of Apache Kafka. It includes 32 new
>> > KIPs.
>> > > See the release notes and release plan (
>> > > https://cwiki.apache.org/confluence/display/KAFKA/Release+
>> Plan+0.11.0.0)
>> > > for more details. A few feature highlights:
>> > >
>> > > * Exactly-once delivery and transactional messaging
>> > > * Streams exactly-once semantics
>> > > * Admin client with support for topic, ACLs and config management
>> > > * Record headers
>> > > * Request rate quotas
>> > > * Improved resiliency: replication protocol improvement and
>> > single-threaded
>> > > controller
>> > > * Richer and more efficient message format
>> > >
>> > > Release notes for the 0.11.0.0 release:
>> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>> > >
>> > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
>> > >
>> > > Kafka's KEYS file containing PGP keys we use to sign the release:
>> > > http://kafka.apache.org/KEYS
>> > >
>> > > * Release artifacts to be voted upon (source and binary):
>> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>> > >
>> > > * Maven artifacts to be voted upon:
>> > > https://repository.apache.org/content/groups/staging/org/apa
>> che/kafka/
>> > >
>> > > * Javadoc:
>> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>> > >
>> > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
>> > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
>> > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
>> > >
>> > > * Documentation:
>> > > http://kafka.apache.org/0110/documentation.html
>> > >
>> > > * Protocol:
>> > > http://kafka.apache.org/0110/protocol.html
>> > >
>> > > * Successful Jenkins builds for the 0.11.0 branch:
>> > > Unit/integration tests: https://builds.apache.org/job/
>> > > kafka-0.11.0-jdk7/187/
>> > > System tests: pending (will send an update tomorrow)
>> > >
>> > > /**************************************
>> > >
>> > > Thanks,
>> > > Ismael
>> > >
>> > >
>> > >
>> > >
>> >
>> >
>> >
>> >
>> >
>>
>>
>>
>>
> --
> You received this message because you are subscribed to the Google Groups
> "kafka-clients" group.
> To unsubscribe from this group and stop receiving emails from it, send an
> email to kafka-clients+unsubscribe@googlegroups.com.
> To post to this group, send email to kafka-clients@googlegroups.com.
> Visit this group at https://groups.google.com/group/kafka-clients.
> To view this discussion on the web visit https://groups.google.com/d/
> msgid/kafka-clients/CAD5tkZYey%3DLSri2NpGnh1RNGfCTtZRJyCNMOU7
> cF0nZW5Ec38g%40mail.gmail.com
> <https://groups.google.com/d/msgid/kafka-clients/CAD5tkZYey%3DLSri2NpGnh1RNGfCTtZRJyCNMOU7cF0nZW5Ec38g%40mail.gmail.com?utm_medium=email&utm_source=footer>
> .
>
> For more options, visit https://groups.google.com/d/optout.
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@juma.me.uk>.
Hi Gwen,

Thanks for verifying the release candidate.

Regarding KAFKA-4815, the thought did occur to me. However, I was not keen
on moving all the subtasks that are not completed to another issue. Since
you asked, I have done so though. :)

Ismael

On Tue, Jun 27, 2017 at 6:40 AM, Gwen Shapira <gw...@confluent.io> wrote:

> Hi,
>
> One super minor issue (that can be fixed without a new RC): The big
> exactly-once stuff (KIP-98) doesn't actually show up as new features in the
> release notes. Most chunks appear as sub-tasks, but the new feature itself
> (KAFKA-4815) is marked as 0.11.1.0 so this is missing. I get that this is
> cosmetic, but having the biggest feature of the release missing from the
> release notes seems like a big deal to me :)
>
> Other than that...
> Validated signatures, ran quickstart, ran tests and everything looks good.
>
> +1 (binding).
>
>
> On Mon, Jun 26, 2017 at 6:54 PM Ismael Juma <is...@gmail.com> wrote:
>
> > Hi Vahid,
> >
> > There are a few known issues when running Kafka on Windows. A PR with
> some
> > fixes is: https://github.com/apache/kafka/pull/3283. The fact that the
> > index cannot be accessed indicates that it may be a similar issue. I
> > suggest we move this discussion to the relevant JIRAs instead of the
> > release thread.
> >
> > Ismael
> >
> > On Mon, Jun 26, 2017 at 11:25 PM, Vahid S Hashemian <
> > vahidhashemian@us.ibm.com> wrote:
> >
> > > Hi Ismael,
> > >
> > > This is the output of core tests from the start until the first failed
> > > test.
> > >
> > > kafka.admin.AdminRackAwareTest >
> > testAssignmentWithRackAwareWithUnevenRacks
> > > PASSED
> > >
> > > kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAware
> > > PASSED
> > >
> > > kafka.admin.AdminRackAwareTest >
> > testAssignmentWithRackAwareWithUnevenReplicas
> > > PASSED
> > >
> > > kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlrea
> dyAssigned
> > > PASSED
> > >
> > > kafka.admin.AdminRackAwareTest > testAssignmentWithRackAware PASSED
> > >
> > > kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED
> > >
> > > kafka.admin.AdminRackAwareTest >
> > testAssignmentWith2ReplicasRackAwareWith6Partitions
> > > PASSED
> > >
> > > kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRac
> > > kAwareWith6PartitionsAnd3Brokers PASSED
> > >
> > > kafka.admin.AdminRackAwareTest >
> > testGetRackAlternatedBrokerListAndAssignReplicasToBrokers
> > > PASSED
> > >
> > > kafka.admin.AdminRackAwareTest > testMoreReplicasThanRacks PASSED
> > >
> > > kafka.admin.AdminRackAwareTest > testSingleRack PASSED
> > >
> > > kafka.admin.AdminRackAwareTest >
> > testAssignmentWithRackAwareWithRandomStartIndex
> > > PASSED
> > >
> > > kafka.admin.AdminRackAwareTest > testLargeNumberPartitionsAssignment
> > > PASSED
> > >
> > > kafka.admin.AdminRackAwareTest > testLessReplicasThanRacks PASSED
> > >
> > > kafka.admin.AclCommandTest > testInvalidAuthorizerProperty PASSED
> > >
> > > kafka.admin.ConfigCommandTest > testScramCredentials PASSED
> > >
> > > kafka.admin.ConfigCommandTest > shouldParseArgumentsForTopicsE
> ntityType
> > > PASSED
> > >
> > > kafka.admin.ConfigCommandTest > testUserClientQuotaOpts PASSED
> > >
> > > kafka.admin.ConfigCommandTest > shouldAddTopicConfig PASSED
> > >
> > > kafka.admin.ConfigCommandTest > shouldAddClientConfig PASSED
> > >
> > > kafka.admin.ConfigCommandTest > shouldDeleteBrokerConfig PASSED
> > >
> > > kafka.admin.DeleteConsumerGroupTest >
> > testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup
> > > PASSED
> > >
> > > kafka.admin.ConfigCommandTest > testQuotaConfigEntity PASSED
> > >
> > > kafka.admin.ConfigCommandTest >
> > shouldNotUpdateBrokerConfigIfMalformedBracketConfig
> > > PASSED
> > >
> > > kafka.admin.ConfigCommandTest > shouldFailIfUnrecognisedEntityType
> PASSED
> > >
> > > kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED
> > >
> > > kafka.admin.ConfigCommandTest >
> > shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted
> > > PASSED
> > >
> > > kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED
> > >
> > > kafka.admin.BrokerApiVersionsCommandTest >
> > checkBrokerApiVersionCommandOutput
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest >
> > shouldRemoveThrottleReplicaListBasedOnProposedAssignment
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest >
> > shouldFindMovingReplicasMultipleTopics
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest >
> > shouldNotOverwriteExistingPropertiesWhenLimitIsAdded
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest >
> > shouldFindMovingReplicasMultipleTopicsAndPartitions
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest >
> > shouldRemoveThrottleLimitFromAllBrokers
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicas
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest >
> > shouldFindMovingReplicasMultiplePartitions
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest > shouldSetQuotaLimit PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest >
> > shouldFindMovingReplicasWhenProposedIsSubsetOfExisting
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest > shouldUpdateQuotaLimit
> PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest >
> > shouldFindTwoMovingReplicasInSamePartition
> > > PASSED
> > >
> > > kafka.admin.ReassignPartitionsCommandTest >
> > shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas
> > > PASSED
> > >
> > > kafka.admin.ConfigCommandTest >
> > shouldNotUpdateBrokerConfigIfMalformedEntityName
> > > PASSED
> > >
> > > kafka.admin.ConfigCommandTest > shouldSupportCommaSeparatedValues
> PASSED
> > >
> > > kafka.admin.ConfigCommandTest >
> > shouldNotUpdateBrokerConfigIfMalformedConfig
> > > PASSED
> > >
> > > kafka.admin.DeleteConsumerGroupTest > testGroupTopicWideDeleteInZKDo
> > > esNothingForActiveGroupConsumingMultipleTopics PASSED
> > >
> > > kafka.admin.AddPartitionsTest > testReplicaPlacementAllServers PASSED
> > >
> > > kafka.admin.ConfigCommandTest > shouldParseArgumentsForBrokers
> EntityType
> > > PASSED
> > >
> > > kafka.admin.ConfigCommandTest > shouldAddBrokerConfig PASSED
> > >
> > > kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED
> > >
> > > kafka.admin.ConfigCommandTest > testQuotaDescribeEntities PASSED
> > >
> > > kafka.admin.AdminTest > testGetBrokerMetadatas PASSED
> > >
> > > kafka.admin.ConfigCommandTest > shouldParseArgumentsForClients
> EntityType
> > > PASSED
> > >
> > > kafka.admin.AclCommandTest > testAclCli PASSED
> > >
> > > kafka.admin.ReassignPartitionsIntegrationTest > testRackAwareReassign
> > > PASSED
> > >
> > > kafka.admin.AdminTest > testBootstrapClientIdConfig PASSED
> > >
> > > kafka.admin.ReassignPartitionsClusterTest >
> > shouldExecuteThrottledReassignment
> > > FAILED
> > >     java.nio.file.FileSystemException: C:\Users\IBM_AD~1\AppData\
> > >
> > Local\Temp\kafka-719085320148197500\my-topic-0\
> 00000000000000000000.index:
> > > The process cannot access the file because it is being used by another
> > > process.
> > >
> > >
> > > From the error message, it sounds like one of the prior tests does not
> do
> > > a proper clean-up?!
> > >
> > > Thanks.
> > > --Vahid
> > >
> > >
> > >
> > >
> > > From:        Ismael Juma <is...@juma.me.uk>
> > > To:        dev@kafka.apache.org
> > > Cc:        kafka-clients <ka...@googlegroups.com>, Kafka
> Users <
> > > users@kafka.apache.org>
> > > Date:        06/26/2017 01:54 PM
> > > Subject:        Re: [VOTE] 0.11.0.0 RC2
> > > Sent by:        ismaelj@gmail.com
> > > ------------------------------
> > >
> > >
> > >
> > > Hi Vahid,
> > >
> > > Can you please check which test fails first? The errors you mentioned
> can
> > > happen if a test fails and doesn't clean-up properly.
> > >
> > > Ismael
> > >
> > > On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
> > > vahidhashemian@us.ibm.com> wrote:
> > >
> > > > Hi Ismael,
> > > >
> > > > To answer your questions:
> > > >
> > > > 1. Yes, the issues exists in trunk too.
> > > >
> > > > 2. I haven't checked with Cygwin, but I can give it a try.
> > > >
> > > > And thanks for addressing this issue. I can confirm with your PR I no
> > > > longer see it.
> > > > But now that the tests progress I see quite a few errors like this in
> > > > core:
> > > >
> > > > kafka.server.ReplicaFetchTest > classMethod FAILED
> > > >     java.lang.AssertionError: Found unexpected threads,
> > > > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> > > > ProcessThread(sid:0 cport:56565):, metrics-mete
> > > > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> > > > Handler, ForkJoinPool-1-worker-1, Attach Listener,
> ProcessThread(sid:0
> > > > cport:59720):, ZkClie
> > > > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> > > > producer-1, Test worker-SendThread(127.0.0.1:56565), /
> 127.0.0.1:54942
> > to
> > > > /127.0.0.1:54926 w
> > > > orkers Thread 2, Test worker, SyncThread:0,
> > > > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> > > > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> > > > 2 to /127.0.0.1:54926 workers Thread 3,
> > > > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> > > > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> > > > alizer, metrics-meter-tick-thread-1)
> > > >
> > > > I tested on a VM and a physical machine, and both give me a lot of
> > errors
> > > > like this.
> > > >
> > > > Thanks.
> > > > --Vahid
> > > >
> > > >
> > > >
> > > >
> > > > From:   Ismael Juma <is...@gmail.com>
> > > > To:     Vahid S Hashemian <va...@us.ibm.com>
> > > > Cc:     dev@kafka.apache.org, kafka-clients
> > > > <ka...@googlegroups.com>, Kafka Users <
> users@kafka.apache.org>
> > > > Date:   06/26/2017 03:53 AM
> > > > Subject:        Re: [VOTE] 0.11.0.0 RC2
> > > >
> > > >
> > > >
> > > > Hi Vahid,
> > > >
> > > > Sorry for not replying to the previous email, I had missed it. A
> couple
> > > of
> > > > questions:
> > > >
> > > > 1. Is this also happening in trunk? Seems like it should be the case
> > for
> > > > months and seemingly no-one reported it until the RC stage.
> > > > 2. Is it correct that this only happens when compiling on Windows
> > without
> > > > Cygwin?
> > > >
> > > > I believe the following PR should fix it, please verify:
> > > >
> > > > https://github.com/apache/kafka/pull/3431
> > > >
> > > > Ismael
> > > >
> > > > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> > > > vahidhashemian@us.ibm.com> wrote:
> > > >
> > > > > Hi Ismael,
> > > > >
> > > > > Not sure if my response on RC1 was lost or this issue is not a
> > > > > show-stopper:
> > > > >
> > > > > I checked again and with RC2, tests still fail in my Windown 64 bit
> > > > > environment.
> > > > >
> > > > > :clients:checkstyleMain
> > > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > > >
> > > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > > protocol\Errors.java:89:1:
> > > > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > > > > [ApiExceptionBuilder, BrokerNotAvailableException,
> > > > > ClusterAuthorizationException, ConcurrentTransactionsException,
> > > > > ControllerMovedException, CoordinatorLoadInProgressException,
> > > > > CoordinatorNotAvailableException, CorruptRecordException,
> > > > > DuplicateSequenceNumberException, GroupAuthorizationException,
> > > > > IllegalGenerationException, IllegalSaslStateException,
> > > > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeExcepti
> on,
> > > > > InvalidConfigurationException, InvalidFetchSizeException,
> > > > > InvalidGroupIdException, InvalidPartitionsException,
> > > > > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > > > > InvalidReplicationFactorException, InvalidRequestException,
> > > > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > > > > InvalidTimestampException, InvalidTopicException,
> > > > InvalidTxnStateException,
> > > > > InvalidTxnTimeoutException, LeaderNotAvailableException,
> > > > NetworkException,
> > > > > NotControllerException, NotCoordinatorException,
> > > > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > > > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > > > > OffsetOutOfRangeException, OperationNotAttemptedException,
> > > > > OutOfOrderSequenceException, PolicyViolationException,
> > > > > ProducerFencedException, RebalanceInProgressException,
> > > > > RecordBatchTooLargeException, RecordTooLargeException,
> > > > > ReplicaNotAvailableException, SecurityDisabledException,
> > > > TimeoutException,
> > > > > TopicAuthorizationException, TopicExistsException,
> > > > > TransactionCoordinatorFencedException,
> > > > TransactionalIdAuthorizationException,
> > > > > UnknownMemberIdException, UnknownServerException,
> > > > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
> > > eption,
> > > > > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > > > > [ClassDataAbstractionCoupling]
> > > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > > >
> > > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > > protocol\Errors.java:89:1:
> > > > > Class Fan-Out Complexity is 60 (max allowed is 40).
> > > > [ClassFanOutComplexity]
> > > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
> > > (max
> > > > > allowed is 40). [ClassFanOutComplexity]
> > > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is
> 42
> > > (max
> > > > > allowed is 40). [ClassFanOutComplexity]
> > > > > :clients:checkstyleMain FAILED
> > > > >
> > > > > FAILURE: Build failed with an exception.
> > > > >
> > > > > Thanks.
> > > > > --Vahid
> > > > >
> > > > >
> > > > >
> > > > > From:        Ismael Juma <is...@juma.me.uk>
> > > > > To:        dev@kafka.apache.org, Kafka Users <
> users@kafka.apache.org
> > >,
> > > > > kafka-clients <ka...@googlegroups.com>
> > > > > Date:        06/22/2017 06:16 PM
> > > > > Subject:        [VOTE] 0.11.0.0 RC2
> > > > > Sent by:        ismaelj@gmail.com
> > > > > ------------------------------
> > > > >
> > > > >
> > > > >
> > > > > Hello Kafka users, developers and client-developers,
> > > > >
> > > > > This is the third candidate for release of Apache Kafka 0.11.0.0.
> > > > >
> > > > > This is a major version release of Apache Kafka. It includes 32 new
> > > > KIPs.
> > > > > See the release notes and release plan (
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > Release+Plan+0.11.0.0)
> > > > > for more details. A few feature highlights:
> > > > >
> > > > > * Exactly-once delivery and transactional messaging
> > > > > * Streams exactly-once semantics
> > > > > * Admin client with support for topic, ACLs and config management
> > > > > * Record headers
> > > > > * Request rate quotas
> > > > > * Improved resiliency: replication protocol improvement and
> > > > single-threaded
> > > > > controller
> > > > > * Richer and more efficient message format
> > > > >
> > > > > Release notes for the 0.11.0.0 release:
> > > > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_
> NOTES.html
> > > > >
> > > > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> > > > >
> > > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > > http://kafka.apache.org/KEYS
> > > > >
> > > > > * Release artifacts to be voted upon (source and binary):
> > > > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> > > > >
> > > > > * Maven artifacts to be voted upon:
> > > > >
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > > >
> > > > > * Javadoc:
> > > > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> > > > >
> > > > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > > > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > > > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> > > > >
> > > > > * Documentation:
> > > > > http://kafka.apache.org/0110/documentation.html
> > > > >
> > > > > * Protocol:
> > > > > http://kafka.apache.org/0110/protocol.html
> > > > >
> > > > > * Successful Jenkins builds for the 0.11.0 branch:
> > > > > Unit/integration tests: https://builds.apache.org/job/
> > > > > kafka-0.11.0-jdk7/187/
> > > > > System tests: pending (will send an update tomorrow)
> > > > >
> > > > > /**************************************
> > > > >
> > > > > Thanks,
> > > > > Ismael
> > > > >
> > > > >
> > > > >
> > > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > >
> > >
> > >
> > >
> >
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Gwen Shapira <gw...@confluent.io>.
Hi,

One super minor issue (that can be fixed without a new RC): The big
exactly-once stuff (KIP-98) doesn't actually show up as new features in the
release notes. Most chunks appear as sub-tasks, but the new feature itself
(KAFKA-4815) is marked as 0.11.1.0 so this is missing. I get that this is
cosmetic, but having the biggest feature of the release missing from the
release notes seems like a big deal to me :)

Other than that...
Validated signatures, ran quickstart, ran tests and everything looks good.

+1 (binding).


On Mon, Jun 26, 2017 at 6:54 PM Ismael Juma <is...@gmail.com> wrote:

> Hi Vahid,
>
> There are a few known issues when running Kafka on Windows. A PR with some
> fixes is: https://github.com/apache/kafka/pull/3283. The fact that the
> index cannot be accessed indicates that it may be a similar issue. I
> suggest we move this discussion to the relevant JIRAs instead of the
> release thread.
>
> Ismael
>
> On Mon, Jun 26, 2017 at 11:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > This is the output of core tests from the start until the first failed
> > test.
> >
> > kafka.admin.AdminRackAwareTest >
> testAssignmentWithRackAwareWithUnevenRacks
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAware
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest >
> testAssignmentWithRackAwareWithUnevenReplicas
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlreadyAssigned
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testAssignmentWithRackAware PASSED
> >
> > kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED
> >
> > kafka.admin.AdminRackAwareTest >
> testAssignmentWith2ReplicasRackAwareWith6Partitions
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRac
> > kAwareWith6PartitionsAnd3Brokers PASSED
> >
> > kafka.admin.AdminRackAwareTest >
> testGetRackAlternatedBrokerListAndAssignReplicasToBrokers
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testMoreReplicasThanRacks PASSED
> >
> > kafka.admin.AdminRackAwareTest > testSingleRack PASSED
> >
> > kafka.admin.AdminRackAwareTest >
> testAssignmentWithRackAwareWithRandomStartIndex
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testLargeNumberPartitionsAssignment
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testLessReplicasThanRacks PASSED
> >
> > kafka.admin.AclCommandTest > testInvalidAuthorizerProperty PASSED
> >
> > kafka.admin.ConfigCommandTest > testScramCredentials PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldParseArgumentsForTopicsEntityType
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > testUserClientQuotaOpts PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldAddTopicConfig PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldAddClientConfig PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldDeleteBrokerConfig PASSED
> >
> > kafka.admin.DeleteConsumerGroupTest >
> testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > testQuotaConfigEntity PASSED
> >
> > kafka.admin.ConfigCommandTest >
> shouldNotUpdateBrokerConfigIfMalformedBracketConfig
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldFailIfUnrecognisedEntityType PASSED
> >
> > kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED
> >
> > kafka.admin.ConfigCommandTest >
> shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted
> > PASSED
> >
> > kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED
> >
> > kafka.admin.BrokerApiVersionsCommandTest >
> checkBrokerApiVersionCommandOutput
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldRemoveThrottleReplicaListBasedOnProposedAssignment
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindMovingReplicasMultipleTopics
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldNotOverwriteExistingPropertiesWhenLimitIsAdded
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindMovingReplicasMultipleTopicsAndPartitions
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldRemoveThrottleLimitFromAllBrokers
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicas
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindMovingReplicasMultiplePartitions
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest > shouldSetQuotaLimit PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindMovingReplicasWhenProposedIsSubsetOfExisting
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest > shouldUpdateQuotaLimit PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindTwoMovingReplicasInSamePartition
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas
> > PASSED
> >
> > kafka.admin.ConfigCommandTest >
> shouldNotUpdateBrokerConfigIfMalformedEntityName
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldSupportCommaSeparatedValues PASSED
> >
> > kafka.admin.ConfigCommandTest >
> shouldNotUpdateBrokerConfigIfMalformedConfig
> > PASSED
> >
> > kafka.admin.DeleteConsumerGroupTest > testGroupTopicWideDeleteInZKDo
> > esNothingForActiveGroupConsumingMultipleTopics PASSED
> >
> > kafka.admin.AddPartitionsTest > testReplicaPlacementAllServers PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldParseArgumentsForBrokersEntityType
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldAddBrokerConfig PASSED
> >
> > kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED
> >
> > kafka.admin.ConfigCommandTest > testQuotaDescribeEntities PASSED
> >
> > kafka.admin.AdminTest > testGetBrokerMetadatas PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldParseArgumentsForClientsEntityType
> > PASSED
> >
> > kafka.admin.AclCommandTest > testAclCli PASSED
> >
> > kafka.admin.ReassignPartitionsIntegrationTest > testRackAwareReassign
> > PASSED
> >
> > kafka.admin.AdminTest > testBootstrapClientIdConfig PASSED
> >
> > kafka.admin.ReassignPartitionsClusterTest >
> shouldExecuteThrottledReassignment
> > FAILED
> >     java.nio.file.FileSystemException: C:\Users\IBM_AD~1\AppData\
> >
> Local\Temp\kafka-719085320148197500\my-topic-0\00000000000000000000.index:
> > The process cannot access the file because it is being used by another
> > process.
> >
> >
> > From the error message, it sounds like one of the prior tests does not do
> > a proper clean-up?!
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> >
> > From:        Ismael Juma <is...@juma.me.uk>
> > To:        dev@kafka.apache.org
> > Cc:        kafka-clients <ka...@googlegroups.com>, Kafka Users <
> > users@kafka.apache.org>
> > Date:        06/26/2017 01:54 PM
> > Subject:        Re: [VOTE] 0.11.0.0 RC2
> > Sent by:        ismaelj@gmail.com
> > ------------------------------
> >
> >
> >
> > Hi Vahid,
> >
> > Can you please check which test fails first? The errors you mentioned can
> > happen if a test fails and doesn't clean-up properly.
> >
> > Ismael
> >
> > On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
> > vahidhashemian@us.ibm.com> wrote:
> >
> > > Hi Ismael,
> > >
> > > To answer your questions:
> > >
> > > 1. Yes, the issues exists in trunk too.
> > >
> > > 2. I haven't checked with Cygwin, but I can give it a try.
> > >
> > > And thanks for addressing this issue. I can confirm with your PR I no
> > > longer see it.
> > > But now that the tests progress I see quite a few errors like this in
> > > core:
> > >
> > > kafka.server.ReplicaFetchTest > classMethod FAILED
> > >     java.lang.AssertionError: Found unexpected threads,
> > > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> > > ProcessThread(sid:0 cport:56565):, metrics-mete
> > > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> > > Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> > > cport:59720):, ZkClie
> > > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> > > producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942
> to
> > > /127.0.0.1:54926 w
> > > orkers Thread 2, Test worker, SyncThread:0,
> > > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> > > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> > > 2 to /127.0.0.1:54926 workers Thread 3,
> > > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> > > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> > > alizer, metrics-meter-tick-thread-1)
> > >
> > > I tested on a VM and a physical machine, and both give me a lot of
> errors
> > > like this.
> > >
> > > Thanks.
> > > --Vahid
> > >
> > >
> > >
> > >
> > > From:   Ismael Juma <is...@gmail.com>
> > > To:     Vahid S Hashemian <va...@us.ibm.com>
> > > Cc:     dev@kafka.apache.org, kafka-clients
> > > <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> > > Date:   06/26/2017 03:53 AM
> > > Subject:        Re: [VOTE] 0.11.0.0 RC2
> > >
> > >
> > >
> > > Hi Vahid,
> > >
> > > Sorry for not replying to the previous email, I had missed it. A couple
> > of
> > > questions:
> > >
> > > 1. Is this also happening in trunk? Seems like it should be the case
> for
> > > months and seemingly no-one reported it until the RC stage.
> > > 2. Is it correct that this only happens when compiling on Windows
> without
> > > Cygwin?
> > >
> > > I believe the following PR should fix it, please verify:
> > >
> > > https://github.com/apache/kafka/pull/3431
> > >
> > > Ismael
> > >
> > > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> > > vahidhashemian@us.ibm.com> wrote:
> > >
> > > > Hi Ismael,
> > > >
> > > > Not sure if my response on RC1 was lost or this issue is not a
> > > > show-stopper:
> > > >
> > > > I checked again and with RC2, tests still fail in my Windown 64 bit
> > > > environment.
> > > >
> > > > :clients:checkstyleMain
> > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > >
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > protocol\Errors.java:89:1:
> > > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > > > [ApiExceptionBuilder, BrokerNotAvailableException,
> > > > ClusterAuthorizationException, ConcurrentTransactionsException,
> > > > ControllerMovedException, CoordinatorLoadInProgressException,
> > > > CoordinatorNotAvailableException, CorruptRecordException,
> > > > DuplicateSequenceNumberException, GroupAuthorizationException,
> > > > IllegalGenerationException, IllegalSaslStateException,
> > > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > > > InvalidConfigurationException, InvalidFetchSizeException,
> > > > InvalidGroupIdException, InvalidPartitionsException,
> > > > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > > > InvalidReplicationFactorException, InvalidRequestException,
> > > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > > > InvalidTimestampException, InvalidTopicException,
> > > InvalidTxnStateException,
> > > > InvalidTxnTimeoutException, LeaderNotAvailableException,
> > > NetworkException,
> > > > NotControllerException, NotCoordinatorException,
> > > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > > > OffsetOutOfRangeException, OperationNotAttemptedException,
> > > > OutOfOrderSequenceException, PolicyViolationException,
> > > > ProducerFencedException, RebalanceInProgressException,
> > > > RecordBatchTooLargeException, RecordTooLargeException,
> > > > ReplicaNotAvailableException, SecurityDisabledException,
> > > TimeoutException,
> > > > TopicAuthorizationException, TopicExistsException,
> > > > TransactionCoordinatorFencedException,
> > > TransactionalIdAuthorizationException,
> > > > UnknownMemberIdException, UnknownServerException,
> > > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
> > eption,
> > > > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > > > [ClassDataAbstractionCoupling]
> > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > >
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > protocol\Errors.java:89:1:
> > > > Class Fan-Out Complexity is 60 (max allowed is 40).
> > > [ClassFanOutComplexity]
> > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
> > (max
> > > > allowed is 40). [ClassFanOutComplexity]
> > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42
> > (max
> > > > allowed is 40). [ClassFanOutComplexity]
> > > > :clients:checkstyleMain FAILED
> > > >
> > > > FAILURE: Build failed with an exception.
> > > >
> > > > Thanks.
> > > > --Vahid
> > > >
> > > >
> > > >
> > > > From:        Ismael Juma <is...@juma.me.uk>
> > > > To:        dev@kafka.apache.org, Kafka Users <users@kafka.apache.org
> >,
> > > > kafka-clients <ka...@googlegroups.com>
> > > > Date:        06/22/2017 06:16 PM
> > > > Subject:        [VOTE] 0.11.0.0 RC2
> > > > Sent by:        ismaelj@gmail.com
> > > > ------------------------------
> > > >
> > > >
> > > >
> > > > Hello Kafka users, developers and client-developers,
> > > >
> > > > This is the third candidate for release of Apache Kafka 0.11.0.0.
> > > >
> > > > This is a major version release of Apache Kafka. It includes 32 new
> > > KIPs.
> > > > See the release notes and release plan (
> > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > Release+Plan+0.11.0.0)
> > > > for more details. A few feature highlights:
> > > >
> > > > * Exactly-once delivery and transactional messaging
> > > > * Streams exactly-once semantics
> > > > * Admin client with support for topic, ACLs and config management
> > > > * Record headers
> > > > * Request rate quotas
> > > > * Improved resiliency: replication protocol improvement and
> > > single-threaded
> > > > controller
> > > > * Richer and more efficient message format
> > > >
> > > > Release notes for the 0.11.0.0 release:
> > > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> > > >
> > > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> > > >
> > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > http://kafka.apache.org/KEYS
> > > >
> > > > * Release artifacts to be voted upon (source and binary):
> > > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> > > >
> > > > * Maven artifacts to be voted upon:
> > > >
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > >
> > > > * Javadoc:
> > > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> > > >
> > > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> > > >
> > > > * Documentation:
> > > > http://kafka.apache.org/0110/documentation.html
> > > >
> > > > * Protocol:
> > > > http://kafka.apache.org/0110/protocol.html
> > > >
> > > > * Successful Jenkins builds for the 0.11.0 branch:
> > > > Unit/integration tests: https://builds.apache.org/job/
> > > > kafka-0.11.0-jdk7/187/
> > > > System tests: pending (will send an update tomorrow)
> > > >
> > > > /**************************************
> > > >
> > > > Thanks,
> > > > Ismael
> > > >
> > > >
> > > >
> > > >
> > >
> > >
> > >
> > >
> > >
> >
> >
> >
> >
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Gwen Shapira <gw...@confluent.io>.
Hi,

One super minor issue (that can be fixed without a new RC): The big
exactly-once stuff (KIP-98) doesn't actually show up as new features in the
release notes. Most chunks appear as sub-tasks, but the new feature itself
(KAFKA-4815) is marked as 0.11.1.0 so this is missing. I get that this is
cosmetic, but having the biggest feature of the release missing from the
release notes seems like a big deal to me :)

Other than that...
Validated signatures, ran quickstart, ran tests and everything looks good.

+1 (binding).


On Mon, Jun 26, 2017 at 6:54 PM Ismael Juma <is...@gmail.com> wrote:

> Hi Vahid,
>
> There are a few known issues when running Kafka on Windows. A PR with some
> fixes is: https://github.com/apache/kafka/pull/3283. The fact that the
> index cannot be accessed indicates that it may be a similar issue. I
> suggest we move this discussion to the relevant JIRAs instead of the
> release thread.
>
> Ismael
>
> On Mon, Jun 26, 2017 at 11:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > This is the output of core tests from the start until the first failed
> > test.
> >
> > kafka.admin.AdminRackAwareTest >
> testAssignmentWithRackAwareWithUnevenRacks
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAware
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest >
> testAssignmentWithRackAwareWithUnevenReplicas
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlreadyAssigned
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testAssignmentWithRackAware PASSED
> >
> > kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED
> >
> > kafka.admin.AdminRackAwareTest >
> testAssignmentWith2ReplicasRackAwareWith6Partitions
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRac
> > kAwareWith6PartitionsAnd3Brokers PASSED
> >
> > kafka.admin.AdminRackAwareTest >
> testGetRackAlternatedBrokerListAndAssignReplicasToBrokers
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testMoreReplicasThanRacks PASSED
> >
> > kafka.admin.AdminRackAwareTest > testSingleRack PASSED
> >
> > kafka.admin.AdminRackAwareTest >
> testAssignmentWithRackAwareWithRandomStartIndex
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testLargeNumberPartitionsAssignment
> > PASSED
> >
> > kafka.admin.AdminRackAwareTest > testLessReplicasThanRacks PASSED
> >
> > kafka.admin.AclCommandTest > testInvalidAuthorizerProperty PASSED
> >
> > kafka.admin.ConfigCommandTest > testScramCredentials PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldParseArgumentsForTopicsEntityType
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > testUserClientQuotaOpts PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldAddTopicConfig PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldAddClientConfig PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldDeleteBrokerConfig PASSED
> >
> > kafka.admin.DeleteConsumerGroupTest >
> testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > testQuotaConfigEntity PASSED
> >
> > kafka.admin.ConfigCommandTest >
> shouldNotUpdateBrokerConfigIfMalformedBracketConfig
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldFailIfUnrecognisedEntityType PASSED
> >
> > kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED
> >
> > kafka.admin.ConfigCommandTest >
> shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted
> > PASSED
> >
> > kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED
> >
> > kafka.admin.BrokerApiVersionsCommandTest >
> checkBrokerApiVersionCommandOutput
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldRemoveThrottleReplicaListBasedOnProposedAssignment
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindMovingReplicasMultipleTopics
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldNotOverwriteExistingPropertiesWhenLimitIsAdded
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindMovingReplicasMultipleTopicsAndPartitions
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldRemoveThrottleLimitFromAllBrokers
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicas
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindMovingReplicasMultiplePartitions
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest > shouldSetQuotaLimit PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindMovingReplicasWhenProposedIsSubsetOfExisting
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest > shouldUpdateQuotaLimit PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldFindTwoMovingReplicasInSamePartition
> > PASSED
> >
> > kafka.admin.ReassignPartitionsCommandTest >
> shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas
> > PASSED
> >
> > kafka.admin.ConfigCommandTest >
> shouldNotUpdateBrokerConfigIfMalformedEntityName
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldSupportCommaSeparatedValues PASSED
> >
> > kafka.admin.ConfigCommandTest >
> shouldNotUpdateBrokerConfigIfMalformedConfig
> > PASSED
> >
> > kafka.admin.DeleteConsumerGroupTest > testGroupTopicWideDeleteInZKDo
> > esNothingForActiveGroupConsumingMultipleTopics PASSED
> >
> > kafka.admin.AddPartitionsTest > testReplicaPlacementAllServers PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldParseArgumentsForBrokersEntityType
> > PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldAddBrokerConfig PASSED
> >
> > kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED
> >
> > kafka.admin.ConfigCommandTest > testQuotaDescribeEntities PASSED
> >
> > kafka.admin.AdminTest > testGetBrokerMetadatas PASSED
> >
> > kafka.admin.ConfigCommandTest > shouldParseArgumentsForClientsEntityType
> > PASSED
> >
> > kafka.admin.AclCommandTest > testAclCli PASSED
> >
> > kafka.admin.ReassignPartitionsIntegrationTest > testRackAwareReassign
> > PASSED
> >
> > kafka.admin.AdminTest > testBootstrapClientIdConfig PASSED
> >
> > kafka.admin.ReassignPartitionsClusterTest >
> shouldExecuteThrottledReassignment
> > FAILED
> >     java.nio.file.FileSystemException: C:\Users\IBM_AD~1\AppData\
> >
> Local\Temp\kafka-719085320148197500\my-topic-0\00000000000000000000.index:
> > The process cannot access the file because it is being used by another
> > process.
> >
> >
> > From the error message, it sounds like one of the prior tests does not do
> > a proper clean-up?!
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> >
> > From:        Ismael Juma <is...@juma.me.uk>
> > To:        dev@kafka.apache.org
> > Cc:        kafka-clients <ka...@googlegroups.com>, Kafka Users <
> > users@kafka.apache.org>
> > Date:        06/26/2017 01:54 PM
> > Subject:        Re: [VOTE] 0.11.0.0 RC2
> > Sent by:        ismaelj@gmail.com
> > ------------------------------
> >
> >
> >
> > Hi Vahid,
> >
> > Can you please check which test fails first? The errors you mentioned can
> > happen if a test fails and doesn't clean-up properly.
> >
> > Ismael
> >
> > On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
> > vahidhashemian@us.ibm.com> wrote:
> >
> > > Hi Ismael,
> > >
> > > To answer your questions:
> > >
> > > 1. Yes, the issues exists in trunk too.
> > >
> > > 2. I haven't checked with Cygwin, but I can give it a try.
> > >
> > > And thanks for addressing this issue. I can confirm with your PR I no
> > > longer see it.
> > > But now that the tests progress I see quite a few errors like this in
> > > core:
> > >
> > > kafka.server.ReplicaFetchTest > classMethod FAILED
> > >     java.lang.AssertionError: Found unexpected threads,
> > > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> > > ProcessThread(sid:0 cport:56565):, metrics-mete
> > > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> > > Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> > > cport:59720):, ZkClie
> > > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> > > producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942
> to
> > > /127.0.0.1:54926 w
> > > orkers Thread 2, Test worker, SyncThread:0,
> > > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> > > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> > > 2 to /127.0.0.1:54926 workers Thread 3,
> > > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> > > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> > > alizer, metrics-meter-tick-thread-1)
> > >
> > > I tested on a VM and a physical machine, and both give me a lot of
> errors
> > > like this.
> > >
> > > Thanks.
> > > --Vahid
> > >
> > >
> > >
> > >
> > > From:   Ismael Juma <is...@gmail.com>
> > > To:     Vahid S Hashemian <va...@us.ibm.com>
> > > Cc:     dev@kafka.apache.org, kafka-clients
> > > <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> > > Date:   06/26/2017 03:53 AM
> > > Subject:        Re: [VOTE] 0.11.0.0 RC2
> > >
> > >
> > >
> > > Hi Vahid,
> > >
> > > Sorry for not replying to the previous email, I had missed it. A couple
> > of
> > > questions:
> > >
> > > 1. Is this also happening in trunk? Seems like it should be the case
> for
> > > months and seemingly no-one reported it until the RC stage.
> > > 2. Is it correct that this only happens when compiling on Windows
> without
> > > Cygwin?
> > >
> > > I believe the following PR should fix it, please verify:
> > >
> > > https://github.com/apache/kafka/pull/3431
> > >
> > > Ismael
> > >
> > > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> > > vahidhashemian@us.ibm.com> wrote:
> > >
> > > > Hi Ismael,
> > > >
> > > > Not sure if my response on RC1 was lost or this issue is not a
> > > > show-stopper:
> > > >
> > > > I checked again and with RC2, tests still fail in my Windown 64 bit
> > > > environment.
> > > >
> > > > :clients:checkstyleMain
> > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > >
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > protocol\Errors.java:89:1:
> > > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > > > [ApiExceptionBuilder, BrokerNotAvailableException,
> > > > ClusterAuthorizationException, ConcurrentTransactionsException,
> > > > ControllerMovedException, CoordinatorLoadInProgressException,
> > > > CoordinatorNotAvailableException, CorruptRecordException,
> > > > DuplicateSequenceNumberException, GroupAuthorizationException,
> > > > IllegalGenerationException, IllegalSaslStateException,
> > > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > > > InvalidConfigurationException, InvalidFetchSizeException,
> > > > InvalidGroupIdException, InvalidPartitionsException,
> > > > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > > > InvalidReplicationFactorException, InvalidRequestException,
> > > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > > > InvalidTimestampException, InvalidTopicException,
> > > InvalidTxnStateException,
> > > > InvalidTxnTimeoutException, LeaderNotAvailableException,
> > > NetworkException,
> > > > NotControllerException, NotCoordinatorException,
> > > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > > > OffsetOutOfRangeException, OperationNotAttemptedException,
> > > > OutOfOrderSequenceException, PolicyViolationException,
> > > > ProducerFencedException, RebalanceInProgressException,
> > > > RecordBatchTooLargeException, RecordTooLargeException,
> > > > ReplicaNotAvailableException, SecurityDisabledException,
> > > TimeoutException,
> > > > TopicAuthorizationException, TopicExistsException,
> > > > TransactionCoordinatorFencedException,
> > > TransactionalIdAuthorizationException,
> > > > UnknownMemberIdException, UnknownServerException,
> > > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
> > eption,
> > > > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > > > [ClassDataAbstractionCoupling]
> > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > >
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > protocol\Errors.java:89:1:
> > > > Class Fan-Out Complexity is 60 (max allowed is 40).
> > > [ClassFanOutComplexity]
> > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
> > (max
> > > > allowed is 40). [ClassFanOutComplexity]
> > > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42
> > (max
> > > > allowed is 40). [ClassFanOutComplexity]
> > > > :clients:checkstyleMain FAILED
> > > >
> > > > FAILURE: Build failed with an exception.
> > > >
> > > > Thanks.
> > > > --Vahid
> > > >
> > > >
> > > >
> > > > From:        Ismael Juma <is...@juma.me.uk>
> > > > To:        dev@kafka.apache.org, Kafka Users <users@kafka.apache.org
> >,
> > > > kafka-clients <ka...@googlegroups.com>
> > > > Date:        06/22/2017 06:16 PM
> > > > Subject:        [VOTE] 0.11.0.0 RC2
> > > > Sent by:        ismaelj@gmail.com
> > > > ------------------------------
> > > >
> > > >
> > > >
> > > > Hello Kafka users, developers and client-developers,
> > > >
> > > > This is the third candidate for release of Apache Kafka 0.11.0.0.
> > > >
> > > > This is a major version release of Apache Kafka. It includes 32 new
> > > KIPs.
> > > > See the release notes and release plan (
> > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > Release+Plan+0.11.0.0)
> > > > for more details. A few feature highlights:
> > > >
> > > > * Exactly-once delivery and transactional messaging
> > > > * Streams exactly-once semantics
> > > > * Admin client with support for topic, ACLs and config management
> > > > * Record headers
> > > > * Request rate quotas
> > > > * Improved resiliency: replication protocol improvement and
> > > single-threaded
> > > > controller
> > > > * Richer and more efficient message format
> > > >
> > > > Release notes for the 0.11.0.0 release:
> > > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> > > >
> > > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> > > >
> > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > http://kafka.apache.org/KEYS
> > > >
> > > > * Release artifacts to be voted upon (source and binary):
> > > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> > > >
> > > > * Maven artifacts to be voted upon:
> > > >
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > >
> > > > * Javadoc:
> > > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> > > >
> > > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> > > >
> > > > * Documentation:
> > > > http://kafka.apache.org/0110/documentation.html
> > > >
> > > > * Protocol:
> > > > http://kafka.apache.org/0110/protocol.html
> > > >
> > > > * Successful Jenkins builds for the 0.11.0 branch:
> > > > Unit/integration tests: https://builds.apache.org/job/
> > > > kafka-0.11.0-jdk7/187/
> > > > System tests: pending (will send an update tomorrow)
> > > >
> > > > /**************************************
> > > >
> > > > Thanks,
> > > > Ismael
> > > >
> > > >
> > > >
> > > >
> > >
> > >
> > >
> > >
> > >
> >
> >
> >
> >
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@gmail.com>.
Hi Vahid,

There are a few known issues when running Kafka on Windows. A PR with some
fixes is: https://github.com/apache/kafka/pull/3283. The fact that the
index cannot be accessed indicates that it may be a similar issue. I
suggest we move this discussion to the relevant JIRAs instead of the
release thread.

Ismael

On Mon, Jun 26, 2017 at 11:25 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> This is the output of core tests from the start until the first failed
> test.
>
> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithUnevenRacks
> PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAware
> PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithUnevenReplicas
> PASSED
>
> kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlreadyAssigned
> PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAware PASSED
>
> kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAwareWith6Partitions
> PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRac
> kAwareWith6PartitionsAnd3Brokers PASSED
>
> kafka.admin.AdminRackAwareTest > testGetRackAlternatedBrokerListAndAssignReplicasToBrokers
> PASSED
>
> kafka.admin.AdminRackAwareTest > testMoreReplicasThanRacks PASSED
>
> kafka.admin.AdminRackAwareTest > testSingleRack PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithRandomStartIndex
> PASSED
>
> kafka.admin.AdminRackAwareTest > testLargeNumberPartitionsAssignment
> PASSED
>
> kafka.admin.AdminRackAwareTest > testLessReplicasThanRacks PASSED
>
> kafka.admin.AclCommandTest > testInvalidAuthorizerProperty PASSED
>
> kafka.admin.ConfigCommandTest > testScramCredentials PASSED
>
> kafka.admin.ConfigCommandTest > shouldParseArgumentsForTopicsEntityType
> PASSED
>
> kafka.admin.ConfigCommandTest > testUserClientQuotaOpts PASSED
>
> kafka.admin.ConfigCommandTest > shouldAddTopicConfig PASSED
>
> kafka.admin.ConfigCommandTest > shouldAddClientConfig PASSED
>
> kafka.admin.ConfigCommandTest > shouldDeleteBrokerConfig PASSED
>
> kafka.admin.DeleteConsumerGroupTest > testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup
> PASSED
>
> kafka.admin.ConfigCommandTest > testQuotaConfigEntity PASSED
>
> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedBracketConfig
> PASSED
>
> kafka.admin.ConfigCommandTest > shouldFailIfUnrecognisedEntityType PASSED
>
> kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED
>
> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted
> PASSED
>
> kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED
>
> kafka.admin.BrokerApiVersionsCommandTest > checkBrokerApiVersionCommandOutput
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldRemoveThrottleReplicaListBasedOnProposedAssignment
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicasMultipleTopics
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldNotOverwriteExistingPropertiesWhenLimitIsAdded
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicasMultipleTopicsAndPartitions
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldRemoveThrottleLimitFromAllBrokers
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicas
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicasMultiplePartitions
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldSetQuotaLimit PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicasWhenProposedIsSubsetOfExisting
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldUpdateQuotaLimit PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindTwoMovingReplicasInSamePartition
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas
> PASSED
>
> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedEntityName
> PASSED
>
> kafka.admin.ConfigCommandTest > shouldSupportCommaSeparatedValues PASSED
>
> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedConfig
> PASSED
>
> kafka.admin.DeleteConsumerGroupTest > testGroupTopicWideDeleteInZKDo
> esNothingForActiveGroupConsumingMultipleTopics PASSED
>
> kafka.admin.AddPartitionsTest > testReplicaPlacementAllServers PASSED
>
> kafka.admin.ConfigCommandTest > shouldParseArgumentsForBrokersEntityType
> PASSED
>
> kafka.admin.ConfigCommandTest > shouldAddBrokerConfig PASSED
>
> kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED
>
> kafka.admin.ConfigCommandTest > testQuotaDescribeEntities PASSED
>
> kafka.admin.AdminTest > testGetBrokerMetadatas PASSED
>
> kafka.admin.ConfigCommandTest > shouldParseArgumentsForClientsEntityType
> PASSED
>
> kafka.admin.AclCommandTest > testAclCli PASSED
>
> kafka.admin.ReassignPartitionsIntegrationTest > testRackAwareReassign
> PASSED
>
> kafka.admin.AdminTest > testBootstrapClientIdConfig PASSED
>
> kafka.admin.ReassignPartitionsClusterTest > shouldExecuteThrottledReassignment
> FAILED
>     java.nio.file.FileSystemException: C:\Users\IBM_AD~1\AppData\
> Local\Temp\kafka-719085320148197500\my-topic-0\00000000000000000000.index:
> The process cannot access the file because it is being used by another
> process.
>
>
> From the error message, it sounds like one of the prior tests does not do
> a proper clean-up?!
>
> Thanks.
> --Vahid
>
>
>
>
> From:        Ismael Juma <is...@juma.me.uk>
> To:        dev@kafka.apache.org
> Cc:        kafka-clients <ka...@googlegroups.com>, Kafka Users <
> users@kafka.apache.org>
> Date:        06/26/2017 01:54 PM
> Subject:        Re: [VOTE] 0.11.0.0 RC2
> Sent by:        ismaelj@gmail.com
> ------------------------------
>
>
>
> Hi Vahid,
>
> Can you please check which test fails first? The errors you mentioned can
> happen if a test fails and doesn't clean-up properly.
>
> Ismael
>
> On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > To answer your questions:
> >
> > 1. Yes, the issues exists in trunk too.
> >
> > 2. I haven't checked with Cygwin, but I can give it a try.
> >
> > And thanks for addressing this issue. I can confirm with your PR I no
> > longer see it.
> > But now that the tests progress I see quite a few errors like this in
> > core:
> >
> > kafka.server.ReplicaFetchTest > classMethod FAILED
> >     java.lang.AssertionError: Found unexpected threads,
> > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> > ProcessThread(sid:0 cport:56565):, metrics-mete
> > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> > Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> > cport:59720):, ZkClie
> > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> > producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> > /127.0.0.1:54926 w
> > orkers Thread 2, Test worker, SyncThread:0,
> > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> > 2 to /127.0.0.1:54926 workers Thread 3,
> > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> > alizer, metrics-meter-tick-thread-1)
> >
> > I tested on a VM and a physical machine, and both give me a lot of errors
> > like this.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> >
> > From:   Ismael Juma <is...@gmail.com>
> > To:     Vahid S Hashemian <va...@us.ibm.com>
> > Cc:     dev@kafka.apache.org, kafka-clients
> > <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> > Date:   06/26/2017 03:53 AM
> > Subject:        Re: [VOTE] 0.11.0.0 RC2
> >
> >
> >
> > Hi Vahid,
> >
> > Sorry for not replying to the previous email, I had missed it. A couple
> of
> > questions:
> >
> > 1. Is this also happening in trunk? Seems like it should be the case for
> > months and seemingly no-one reported it until the RC stage.
> > 2. Is it correct that this only happens when compiling on Windows without
> > Cygwin?
> >
> > I believe the following PR should fix it, please verify:
> >
> > https://github.com/apache/kafka/pull/3431
> >
> > Ismael
> >
> > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> > vahidhashemian@us.ibm.com> wrote:
> >
> > > Hi Ismael,
> > >
> > > Not sure if my response on RC1 was lost or this issue is not a
> > > show-stopper:
> > >
> > > I checked again and with RC2, tests still fail in my Windown 64 bit
> > > environment.
> > >
> > > :clients:checkstyleMain
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > > [ApiExceptionBuilder, BrokerNotAvailableException,
> > > ClusterAuthorizationException, ConcurrentTransactionsException,
> > > ControllerMovedException, CoordinatorLoadInProgressException,
> > > CoordinatorNotAvailableException, CorruptRecordException,
> > > DuplicateSequenceNumberException, GroupAuthorizationException,
> > > IllegalGenerationException, IllegalSaslStateException,
> > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > > InvalidConfigurationException, InvalidFetchSizeException,
> > > InvalidGroupIdException, InvalidPartitionsException,
> > > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > > InvalidReplicationFactorException, InvalidRequestException,
> > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > > InvalidTimestampException, InvalidTopicException,
> > InvalidTxnStateException,
> > > InvalidTxnTimeoutException, LeaderNotAvailableException,
> > NetworkException,
> > > NotControllerException, NotCoordinatorException,
> > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > > OffsetOutOfRangeException, OperationNotAttemptedException,
> > > OutOfOrderSequenceException, PolicyViolationException,
> > > ProducerFencedException, RebalanceInProgressException,
> > > RecordBatchTooLargeException, RecordTooLargeException,
> > > ReplicaNotAvailableException, SecurityDisabledException,
> > TimeoutException,
> > > TopicAuthorizationException, TopicExistsException,
> > > TransactionCoordinatorFencedException,
> > TransactionalIdAuthorizationException,
> > > UnknownMemberIdException, UnknownServerException,
> > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
> eption,
> > > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > > [ClassDataAbstractionCoupling]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Fan-Out Complexity is 60 (max allowed is 40).
> > [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > :clients:checkstyleMain FAILED
> > >
> > > FAILURE: Build failed with an exception.
> > >
> > > Thanks.
> > > --Vahid
> > >
> > >
> > >
> > > From:        Ismael Juma <is...@juma.me.uk>
> > > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > > kafka-clients <ka...@googlegroups.com>
> > > Date:        06/22/2017 06:16 PM
> > > Subject:        [VOTE] 0.11.0.0 RC2
> > > Sent by:        ismaelj@gmail.com
> > > ------------------------------
> > >
> > >
> > >
> > > Hello Kafka users, developers and client-developers,
> > >
> > > This is the third candidate for release of Apache Kafka 0.11.0.0.
> > >
> > > This is a major version release of Apache Kafka. It includes 32 new
> > KIPs.
> > > See the release notes and release plan (
> > > https://cwiki.apache.org/confluence/display/KAFKA/
> Release+Plan+0.11.0.0)
> > > for more details. A few feature highlights:
> > >
> > > * Exactly-once delivery and transactional messaging
> > > * Streams exactly-once semantics
> > > * Admin client with support for topic, ACLs and config management
> > > * Record headers
> > > * Request rate quotas
> > > * Improved resiliency: replication protocol improvement and
> > single-threaded
> > > controller
> > > * Richer and more efficient message format
> > >
> > > Release notes for the 0.11.0.0 release:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> > >
> > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> > >
> > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > http://kafka.apache.org/KEYS
> > >
> > > * Release artifacts to be voted upon (source and binary):
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> > >
> > > * Maven artifacts to be voted upon:
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >
> > > * Javadoc:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> > >
> > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> > >
> > > * Documentation:
> > > http://kafka.apache.org/0110/documentation.html
> > >
> > > * Protocol:
> > > http://kafka.apache.org/0110/protocol.html
> > >
> > > * Successful Jenkins builds for the 0.11.0 branch:
> > > Unit/integration tests: https://builds.apache.org/job/
> > > kafka-0.11.0-jdk7/187/
> > > System tests: pending (will send an update tomorrow)
> > >
> > > /**************************************
> > >
> > > Thanks,
> > > Ismael
> > >
> > >
> > >
> > >
> >
> >
> >
> >
> >
>
>
>
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@gmail.com>.
Hi Vahid,

There are a few known issues when running Kafka on Windows. A PR with some
fixes is: https://github.com/apache/kafka/pull/3283. The fact that the
index cannot be accessed indicates that it may be a similar issue. I
suggest we move this discussion to the relevant JIRAs instead of the
release thread.

Ismael

On Mon, Jun 26, 2017 at 11:25 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> This is the output of core tests from the start until the first failed
> test.
>
> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithUnevenRacks
> PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAware
> PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithUnevenReplicas
> PASSED
>
> kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlreadyAssigned
> PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAware PASSED
>
> kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAwareWith6Partitions
> PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRac
> kAwareWith6PartitionsAnd3Brokers PASSED
>
> kafka.admin.AdminRackAwareTest > testGetRackAlternatedBrokerListAndAssignReplicasToBrokers
> PASSED
>
> kafka.admin.AdminRackAwareTest > testMoreReplicasThanRacks PASSED
>
> kafka.admin.AdminRackAwareTest > testSingleRack PASSED
>
> kafka.admin.AdminRackAwareTest > testAssignmentWithRackAwareWithRandomStartIndex
> PASSED
>
> kafka.admin.AdminRackAwareTest > testLargeNumberPartitionsAssignment
> PASSED
>
> kafka.admin.AdminRackAwareTest > testLessReplicasThanRacks PASSED
>
> kafka.admin.AclCommandTest > testInvalidAuthorizerProperty PASSED
>
> kafka.admin.ConfigCommandTest > testScramCredentials PASSED
>
> kafka.admin.ConfigCommandTest > shouldParseArgumentsForTopicsEntityType
> PASSED
>
> kafka.admin.ConfigCommandTest > testUserClientQuotaOpts PASSED
>
> kafka.admin.ConfigCommandTest > shouldAddTopicConfig PASSED
>
> kafka.admin.ConfigCommandTest > shouldAddClientConfig PASSED
>
> kafka.admin.ConfigCommandTest > shouldDeleteBrokerConfig PASSED
>
> kafka.admin.DeleteConsumerGroupTest > testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup
> PASSED
>
> kafka.admin.ConfigCommandTest > testQuotaConfigEntity PASSED
>
> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedBracketConfig
> PASSED
>
> kafka.admin.ConfigCommandTest > shouldFailIfUnrecognisedEntityType PASSED
>
> kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED
>
> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted
> PASSED
>
> kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED
>
> kafka.admin.BrokerApiVersionsCommandTest > checkBrokerApiVersionCommandOutput
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldRemoveThrottleReplicaListBasedOnProposedAssignment
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicasMultipleTopics
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldNotOverwriteExistingPropertiesWhenLimitIsAdded
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicasMultipleTopicsAndPartitions
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldRemoveThrottleLimitFromAllBrokers
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicas
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicasMultiplePartitions
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldSetQuotaLimit PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicasWhenProposedIsSubsetOfExisting
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldUpdateQuotaLimit PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldFindTwoMovingReplicasInSamePartition
> PASSED
>
> kafka.admin.ReassignPartitionsCommandTest > shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas
> PASSED
>
> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedEntityName
> PASSED
>
> kafka.admin.ConfigCommandTest > shouldSupportCommaSeparatedValues PASSED
>
> kafka.admin.ConfigCommandTest > shouldNotUpdateBrokerConfigIfMalformedConfig
> PASSED
>
> kafka.admin.DeleteConsumerGroupTest > testGroupTopicWideDeleteInZKDo
> esNothingForActiveGroupConsumingMultipleTopics PASSED
>
> kafka.admin.AddPartitionsTest > testReplicaPlacementAllServers PASSED
>
> kafka.admin.ConfigCommandTest > shouldParseArgumentsForBrokersEntityType
> PASSED
>
> kafka.admin.ConfigCommandTest > shouldAddBrokerConfig PASSED
>
> kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED
>
> kafka.admin.ConfigCommandTest > testQuotaDescribeEntities PASSED
>
> kafka.admin.AdminTest > testGetBrokerMetadatas PASSED
>
> kafka.admin.ConfigCommandTest > shouldParseArgumentsForClientsEntityType
> PASSED
>
> kafka.admin.AclCommandTest > testAclCli PASSED
>
> kafka.admin.ReassignPartitionsIntegrationTest > testRackAwareReassign
> PASSED
>
> kafka.admin.AdminTest > testBootstrapClientIdConfig PASSED
>
> kafka.admin.ReassignPartitionsClusterTest > shouldExecuteThrottledReassignment
> FAILED
>     java.nio.file.FileSystemException: C:\Users\IBM_AD~1\AppData\
> Local\Temp\kafka-719085320148197500\my-topic-0\00000000000000000000.index:
> The process cannot access the file because it is being used by another
> process.
>
>
> From the error message, it sounds like one of the prior tests does not do
> a proper clean-up?!
>
> Thanks.
> --Vahid
>
>
>
>
> From:        Ismael Juma <is...@juma.me.uk>
> To:        dev@kafka.apache.org
> Cc:        kafka-clients <ka...@googlegroups.com>, Kafka Users <
> users@kafka.apache.org>
> Date:        06/26/2017 01:54 PM
> Subject:        Re: [VOTE] 0.11.0.0 RC2
> Sent by:        ismaelj@gmail.com
> ------------------------------
>
>
>
> Hi Vahid,
>
> Can you please check which test fails first? The errors you mentioned can
> happen if a test fails and doesn't clean-up properly.
>
> Ismael
>
> On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > To answer your questions:
> >
> > 1. Yes, the issues exists in trunk too.
> >
> > 2. I haven't checked with Cygwin, but I can give it a try.
> >
> > And thanks for addressing this issue. I can confirm with your PR I no
> > longer see it.
> > But now that the tests progress I see quite a few errors like this in
> > core:
> >
> > kafka.server.ReplicaFetchTest > classMethod FAILED
> >     java.lang.AssertionError: Found unexpected threads,
> > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> > ProcessThread(sid:0 cport:56565):, metrics-mete
> > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> > Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> > cport:59720):, ZkClie
> > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> > producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> > /127.0.0.1:54926 w
> > orkers Thread 2, Test worker, SyncThread:0,
> > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> > 2 to /127.0.0.1:54926 workers Thread 3,
> > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> > alizer, metrics-meter-tick-thread-1)
> >
> > I tested on a VM and a physical machine, and both give me a lot of errors
> > like this.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> >
> > From:   Ismael Juma <is...@gmail.com>
> > To:     Vahid S Hashemian <va...@us.ibm.com>
> > Cc:     dev@kafka.apache.org, kafka-clients
> > <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> > Date:   06/26/2017 03:53 AM
> > Subject:        Re: [VOTE] 0.11.0.0 RC2
> >
> >
> >
> > Hi Vahid,
> >
> > Sorry for not replying to the previous email, I had missed it. A couple
> of
> > questions:
> >
> > 1. Is this also happening in trunk? Seems like it should be the case for
> > months and seemingly no-one reported it until the RC stage.
> > 2. Is it correct that this only happens when compiling on Windows without
> > Cygwin?
> >
> > I believe the following PR should fix it, please verify:
> >
> > https://github.com/apache/kafka/pull/3431
> >
> > Ismael
> >
> > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> > vahidhashemian@us.ibm.com> wrote:
> >
> > > Hi Ismael,
> > >
> > > Not sure if my response on RC1 was lost or this issue is not a
> > > show-stopper:
> > >
> > > I checked again and with RC2, tests still fail in my Windown 64 bit
> > > environment.
> > >
> > > :clients:checkstyleMain
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > > [ApiExceptionBuilder, BrokerNotAvailableException,
> > > ClusterAuthorizationException, ConcurrentTransactionsException,
> > > ControllerMovedException, CoordinatorLoadInProgressException,
> > > CoordinatorNotAvailableException, CorruptRecordException,
> > > DuplicateSequenceNumberException, GroupAuthorizationException,
> > > IllegalGenerationException, IllegalSaslStateException,
> > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > > InvalidConfigurationException, InvalidFetchSizeException,
> > > InvalidGroupIdException, InvalidPartitionsException,
> > > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > > InvalidReplicationFactorException, InvalidRequestException,
> > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > > InvalidTimestampException, InvalidTopicException,
> > InvalidTxnStateException,
> > > InvalidTxnTimeoutException, LeaderNotAvailableException,
> > NetworkException,
> > > NotControllerException, NotCoordinatorException,
> > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > > OffsetOutOfRangeException, OperationNotAttemptedException,
> > > OutOfOrderSequenceException, PolicyViolationException,
> > > ProducerFencedException, RebalanceInProgressException,
> > > RecordBatchTooLargeException, RecordTooLargeException,
> > > ReplicaNotAvailableException, SecurityDisabledException,
> > TimeoutException,
> > > TopicAuthorizationException, TopicExistsException,
> > > TransactionCoordinatorFencedException,
> > TransactionalIdAuthorizationException,
> > > UnknownMemberIdException, UnknownServerException,
> > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
> eption,
> > > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > > [ClassDataAbstractionCoupling]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Fan-Out Complexity is 60 (max allowed is 40).
> > [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > :clients:checkstyleMain FAILED
> > >
> > > FAILURE: Build failed with an exception.
> > >
> > > Thanks.
> > > --Vahid
> > >
> > >
> > >
> > > From:        Ismael Juma <is...@juma.me.uk>
> > > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > > kafka-clients <ka...@googlegroups.com>
> > > Date:        06/22/2017 06:16 PM
> > > Subject:        [VOTE] 0.11.0.0 RC2
> > > Sent by:        ismaelj@gmail.com
> > > ------------------------------
> > >
> > >
> > >
> > > Hello Kafka users, developers and client-developers,
> > >
> > > This is the third candidate for release of Apache Kafka 0.11.0.0.
> > >
> > > This is a major version release of Apache Kafka. It includes 32 new
> > KIPs.
> > > See the release notes and release plan (
> > > https://cwiki.apache.org/confluence/display/KAFKA/
> Release+Plan+0.11.0.0)
> > > for more details. A few feature highlights:
> > >
> > > * Exactly-once delivery and transactional messaging
> > > * Streams exactly-once semantics
> > > * Admin client with support for topic, ACLs and config management
> > > * Record headers
> > > * Request rate quotas
> > > * Improved resiliency: replication protocol improvement and
> > single-threaded
> > > controller
> > > * Richer and more efficient message format
> > >
> > > Release notes for the 0.11.0.0 release:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> > >
> > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> > >
> > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > http://kafka.apache.org/KEYS
> > >
> > > * Release artifacts to be voted upon (source and binary):
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> > >
> > > * Maven artifacts to be voted upon:
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >
> > > * Javadoc:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> > >
> > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> > >
> > > * Documentation:
> > > http://kafka.apache.org/0110/documentation.html
> > >
> > > * Protocol:
> > > http://kafka.apache.org/0110/protocol.html
> > >
> > > * Successful Jenkins builds for the 0.11.0 branch:
> > > Unit/integration tests: https://builds.apache.org/job/
> > > kafka-0.11.0-jdk7/187/
> > > System tests: pending (will send an update tomorrow)
> > >
> > > /**************************************
> > >
> > > Thanks,
> > > Ismael
> > >
> > >
> > >
> > >
> >
> >
> >
> >
> >
>
>
>
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Vahid S Hashemian <va...@us.ibm.com>.
Hi Ismael,

This is the output of core tests from the start until the first failed 
test.

kafka.admin.AdminRackAwareTest > 
testAssignmentWithRackAwareWithUnevenRacks PASSED

kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAware 
PASSED

kafka.admin.AdminRackAwareTest > 
testAssignmentWithRackAwareWithUnevenReplicas PASSED

kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlreadyAssigned 
PASSED

kafka.admin.AdminRackAwareTest > testAssignmentWithRackAware PASSED

kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED

kafka.admin.AdminRackAwareTest > 
testAssignmentWith2ReplicasRackAwareWith6Partitions PASSED

kafka.admin.AdminRackAwareTest > 
testAssignmentWith2ReplicasRackAwareWith6PartitionsAnd3Brokers PASSED

kafka.admin.AdminRackAwareTest > 
testGetRackAlternatedBrokerListAndAssignReplicasToBrokers PASSED

kafka.admin.AdminRackAwareTest > testMoreReplicasThanRacks PASSED

kafka.admin.AdminRackAwareTest > testSingleRack PASSED

kafka.admin.AdminRackAwareTest > 
testAssignmentWithRackAwareWithRandomStartIndex PASSED

kafka.admin.AdminRackAwareTest > testLargeNumberPartitionsAssignment 
PASSED

kafka.admin.AdminRackAwareTest > testLessReplicasThanRacks PASSED

kafka.admin.AclCommandTest > testInvalidAuthorizerProperty PASSED

kafka.admin.ConfigCommandTest > testScramCredentials PASSED

kafka.admin.ConfigCommandTest > shouldParseArgumentsForTopicsEntityType 
PASSED

kafka.admin.ConfigCommandTest > testUserClientQuotaOpts PASSED

kafka.admin.ConfigCommandTest > shouldAddTopicConfig PASSED

kafka.admin.ConfigCommandTest > shouldAddClientConfig PASSED

kafka.admin.ConfigCommandTest > shouldDeleteBrokerConfig PASSED

kafka.admin.DeleteConsumerGroupTest > 
testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup PASSED

kafka.admin.ConfigCommandTest > testQuotaConfigEntity PASSED

kafka.admin.ConfigCommandTest > 
shouldNotUpdateBrokerConfigIfMalformedBracketConfig PASSED

kafka.admin.ConfigCommandTest > shouldFailIfUnrecognisedEntityType PASSED

kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED

kafka.admin.ConfigCommandTest > 
shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted PASSED

kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED

kafka.admin.BrokerApiVersionsCommandTest > 
checkBrokerApiVersionCommandOutput PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldRemoveThrottleReplicaListBasedOnProposedAssignment PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindMovingReplicasMultipleTopics PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldNotOverwriteExistingPropertiesWhenLimitIsAdded PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindMovingReplicasMultipleTopicsAndPartitions PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldRemoveThrottleLimitFromAllBrokers PASSED

kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicas 
PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindMovingReplicasMultiplePartitions PASSED

kafka.admin.ReassignPartitionsCommandTest > shouldSetQuotaLimit PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindMovingReplicasWhenProposedIsSubsetOfExisting PASSED

kafka.admin.ReassignPartitionsCommandTest > shouldUpdateQuotaLimit PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindTwoMovingReplicasInSamePartition PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas PASSED

kafka.admin.ConfigCommandTest > 
shouldNotUpdateBrokerConfigIfMalformedEntityName PASSED

kafka.admin.ConfigCommandTest > shouldSupportCommaSeparatedValues PASSED

kafka.admin.ConfigCommandTest > 
shouldNotUpdateBrokerConfigIfMalformedConfig PASSED

kafka.admin.DeleteConsumerGroupTest > 
testGroupTopicWideDeleteInZKDoesNothingForActiveGroupConsumingMultipleTopics 
PASSED

kafka.admin.AddPartitionsTest > testReplicaPlacementAllServers PASSED

kafka.admin.ConfigCommandTest > shouldParseArgumentsForBrokersEntityType 
PASSED

kafka.admin.ConfigCommandTest > shouldAddBrokerConfig PASSED

kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED

kafka.admin.ConfigCommandTest > testQuotaDescribeEntities PASSED

kafka.admin.AdminTest > testGetBrokerMetadatas PASSED

kafka.admin.ConfigCommandTest > shouldParseArgumentsForClientsEntityType 
PASSED

kafka.admin.AclCommandTest > testAclCli PASSED

kafka.admin.ReassignPartitionsIntegrationTest > testRackAwareReassign 
PASSED

kafka.admin.AdminTest > testBootstrapClientIdConfig PASSED

kafka.admin.ReassignPartitionsClusterTest > 
shouldExecuteThrottledReassignment FAILED
    java.nio.file.FileSystemException: 
C:\Users\IBM_AD~1\AppData\Local\Temp\kafka-719085320148197500\my-topic-0\00000000000000000000.index: 
The process cannot access the file because it is being used by another 
process.


From the error message, it sounds like one of the prior tests does not do 
a proper clean-up?!

Thanks.
--Vahid
 



From:   Ismael Juma <is...@juma.me.uk>
To:     dev@kafka.apache.org
Cc:     kafka-clients <ka...@googlegroups.com>, Kafka Users 
<us...@kafka.apache.org>
Date:   06/26/2017 01:54 PM
Subject:        Re: [VOTE] 0.11.0.0 RC2
Sent by:        ismaelj@gmail.com



Hi Vahid,

Can you please check which test fails first? The errors you mentioned can
happen if a test fails and doesn't clean-up properly.

Ismael

On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> To answer your questions:
>
> 1. Yes, the issues exists in trunk too.
>
> 2. I haven't checked with Cygwin, but I can give it a try.
>
> And thanks for addressing this issue. I can confirm with your PR I no
> longer see it.
> But now that the tests progress I see quite a few errors like this in
> core:
>
> kafka.server.ReplicaFetchTest > classMethod FAILED
>     java.lang.AssertionError: Found unexpected threads,
> allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> ProcessThread(sid:0 cport:56565):, metrics-mete
> r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> cport:59720):, ZkClie
> nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> /127.0.0.1:54926 w
> orkers Thread 2, Test worker, SyncThread:0,
> NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> 2 to /127.0.0.1:54926 workers Thread 3,
> ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> alizer, metrics-meter-tick-thread-1)
>
> I tested on a VM and a physical machine, and both give me a lot of 
errors
> like this.
>
> Thanks.
> --Vahid
>
>
>
>
> From:   Ismael Juma <is...@gmail.com>
> To:     Vahid S Hashemian <va...@us.ibm.com>
> Cc:     dev@kafka.apache.org, kafka-clients
> <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> Date:   06/26/2017 03:53 AM
> Subject:        Re: [VOTE] 0.11.0.0 RC2
>
>
>
> Hi Vahid,
>
> Sorry for not replying to the previous email, I had missed it. A couple 
of
> questions:
>
> 1. Is this also happening in trunk? Seems like it should be the case for
> months and seemingly no-one reported it until the RC stage.
> 2. Is it correct that this only happens when compiling on Windows 
without
> Cygwin?
>
> I believe the following PR should fix it, please verify:
>
> https://github.com/apache/kafka/pull/3431
>
> Ismael
>
> On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > Not sure if my response on RC1 was lost or this issue is not a
> > show-stopper:
> >
> > I checked again and with RC2, tests still fail in my Windown 64 bit
> > environment.
> >
> > :clients:checkstyleMain
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > [ApiExceptionBuilder, BrokerNotAvailableException,
> > ClusterAuthorizationException, ConcurrentTransactionsException,
> > ControllerMovedException, CoordinatorLoadInProgressException,
> > CoordinatorNotAvailableException, CorruptRecordException,
> > DuplicateSequenceNumberException, GroupAuthorizationException,
> > IllegalGenerationException, IllegalSaslStateException,
> > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > InvalidConfigurationException, InvalidFetchSizeException,
> > InvalidGroupIdException, InvalidPartitionsException,
> > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > InvalidReplicationFactorException, InvalidRequestException,
> > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > InvalidTimestampException, InvalidTopicException,
> InvalidTxnStateException,
> > InvalidTxnTimeoutException, LeaderNotAvailableException,
> NetworkException,
> > NotControllerException, NotCoordinatorException,
> > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > OffsetOutOfRangeException, OperationNotAttemptedException,
> > OutOfOrderSequenceException, PolicyViolationException,
> > ProducerFencedException, RebalanceInProgressException,
> > RecordBatchTooLargeException, RecordTooLargeException,
> > ReplicaNotAvailableException, SecurityDisabledException,
> TimeoutException,
> > TopicAuthorizationException, TopicExistsException,
> > TransactionCoordinatorFencedException,
> TransactionalIdAuthorizationException,
> > UnknownMemberIdException, UnknownServerException,
> > UnknownTopicOrPartitionException, 
UnsupportedForMessageFormatException,
> > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > [ClassDataAbstractionCoupling]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Fan-Out Complexity is 60 (max allowed is 40).
> [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 
(max
> > allowed is 40). [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 
(max
> > allowed is 40). [ClassFanOutComplexity]
> > :clients:checkstyleMain FAILED
> >
> > FAILURE: Build failed with an exception.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> > From:        Ismael Juma <is...@juma.me.uk>
> > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > kafka-clients <ka...@googlegroups.com>
> > Date:        06/22/2017 06:16 PM
> > Subject:        [VOTE] 0.11.0.0 RC2
> > Sent by:        ismaelj@gmail.com
> > ------------------------------
> >
> >
> >
> > Hello Kafka users, developers and client-developers,
> >
> > This is the third candidate for release of Apache Kafka 0.11.0.0.
> >
> > This is a major version release of Apache Kafka. It includes 32 new
> KIPs.
> > See the release notes and release plan (
> > 
https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> > for more details. A few feature highlights:
> >
> > * Exactly-once delivery and transactional messaging
> > * Streams exactly-once semantics
> > * Admin client with support for topic, ACLs and config management
> > * Record headers
> > * Request rate quotas
> > * Improved resiliency: replication protocol improvement and
> single-threaded
> > controller
> > * Richer and more efficient message format
> >
> > Release notes for the 0.11.0.0 release:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > http://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> >
> > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> >
> > * Documentation:
> > http://kafka.apache.org/0110/documentation.html
> >
> > * Protocol:
> > http://kafka.apache.org/0110/protocol.html
> >
> > * Successful Jenkins builds for the 0.11.0 branch:
> > Unit/integration tests: https://builds.apache.org/job/
> > kafka-0.11.0-jdk7/187/
> > System tests: pending (will send an update tomorrow)
> >
> > /**************************************
> >
> > Thanks,
> > Ismael
> >
> >
> >
> >
>
>
>
>
>





Re: [VOTE] 0.11.0.0 RC2

Posted by Jason Gustafson <ja...@confluent.io>.
+1 from me. I verified the artifacts and tested basic producing and
consuming. I also verified the procedure for upgrading to the new message
format.

-Jason

On Mon, Jun 26, 2017 at 1:53 PM, Ismael Juma <is...@juma.me.uk> wrote:

> Hi Vahid,
>
> Can you please check which test fails first? The errors you mentioned can
> happen if a test fails and doesn't clean-up properly.
>
> Ismael
>
> On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > To answer your questions:
> >
> > 1. Yes, the issues exists in trunk too.
> >
> > 2. I haven't checked with Cygwin, but I can give it a try.
> >
> > And thanks for addressing this issue. I can confirm with your PR I no
> > longer see it.
> > But now that the tests progress I see quite a few errors like this in
> > core:
> >
> > kafka.server.ReplicaFetchTest > classMethod FAILED
> >     java.lang.AssertionError: Found unexpected threads,
> > allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> > ProcessThread(sid:0 cport:56565):, metrics-mete
> > r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> > Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> > cport:59720):, ZkClie
> > nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> > producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> > /127.0.0.1:54926 w
> > orkers Thread 2, Test worker, SyncThread:0,
> > NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> > 2 to /127.0.0.1:54926 workers Thread 3,
> > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> > alizer, metrics-meter-tick-thread-1)
> >
> > I tested on a VM and a physical machine, and both give me a lot of errors
> > like this.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> >
> > From:   Ismael Juma <is...@gmail.com>
> > To:     Vahid S Hashemian <va...@us.ibm.com>
> > Cc:     dev@kafka.apache.org, kafka-clients
> > <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> > Date:   06/26/2017 03:53 AM
> > Subject:        Re: [VOTE] 0.11.0.0 RC2
> >
> >
> >
> > Hi Vahid,
> >
> > Sorry for not replying to the previous email, I had missed it. A couple
> of
> > questions:
> >
> > 1. Is this also happening in trunk? Seems like it should be the case for
> > months and seemingly no-one reported it until the RC stage.
> > 2. Is it correct that this only happens when compiling on Windows without
> > Cygwin?
> >
> > I believe the following PR should fix it, please verify:
> >
> > https://github.com/apache/kafka/pull/3431
> >
> > Ismael
> >
> > On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> > vahidhashemian@us.ibm.com> wrote:
> >
> > > Hi Ismael,
> > >
> > > Not sure if my response on RC1 was lost or this issue is not a
> > > show-stopper:
> > >
> > > I checked again and with RC2, tests still fail in my Windown 64 bit
> > > environment.
> > >
> > > :clients:checkstyleMain
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > > [ApiExceptionBuilder, BrokerNotAvailableException,
> > > ClusterAuthorizationException, ConcurrentTransactionsException,
> > > ControllerMovedException, CoordinatorLoadInProgressException,
> > > CoordinatorNotAvailableException, CorruptRecordException,
> > > DuplicateSequenceNumberException, GroupAuthorizationException,
> > > IllegalGenerationException, IllegalSaslStateException,
> > > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > > InvalidConfigurationException, InvalidFetchSizeException,
> > > InvalidGroupIdException, InvalidPartitionsException,
> > > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > > InvalidReplicationFactorException, InvalidRequestException,
> > > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > > InvalidTimestampException, InvalidTopicException,
> > InvalidTxnStateException,
> > > InvalidTxnTimeoutException, LeaderNotAvailableException,
> > NetworkException,
> > > NotControllerException, NotCoordinatorException,
> > > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > > OffsetOutOfRangeException, OperationNotAttemptedException,
> > > OutOfOrderSequenceException, PolicyViolationException,
> > > ProducerFencedException, RebalanceInProgressException,
> > > RecordBatchTooLargeException, RecordTooLargeException,
> > > ReplicaNotAvailableException, SecurityDisabledException,
> > TimeoutException,
> > > TopicAuthorizationException, TopicExistsException,
> > > TransactionCoordinatorFencedException,
> > TransactionalIdAuthorizationException,
> > > UnknownMemberIdException, UnknownServerException,
> > > UnknownTopicOrPartitionException, UnsupportedForMessageFormatExc
> eption,
> > > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > > [ClassDataAbstractionCoupling]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > >
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > protocol\Errors.java:89:1:
> > > Class Fan-Out Complexity is 60 (max allowed is 40).
> > [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42
> (max
> > > allowed is 40). [ClassFanOutComplexity]
> > > :clients:checkstyleMain FAILED
> > >
> > > FAILURE: Build failed with an exception.
> > >
> > > Thanks.
> > > --Vahid
> > >
> > >
> > >
> > > From:        Ismael Juma <is...@juma.me.uk>
> > > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > > kafka-clients <ka...@googlegroups.com>
> > > Date:        06/22/2017 06:16 PM
> > > Subject:        [VOTE] 0.11.0.0 RC2
> > > Sent by:        ismaelj@gmail.com
> > > ------------------------------
> > >
> > >
> > >
> > > Hello Kafka users, developers and client-developers,
> > >
> > > This is the third candidate for release of Apache Kafka 0.11.0.0.
> > >
> > > This is a major version release of Apache Kafka. It includes 32 new
> > KIPs.
> > > See the release notes and release plan (
> > > https://cwiki.apache.org/confluence/display/KAFKA/
> Release+Plan+0.11.0.0)
> > > for more details. A few feature highlights:
> > >
> > > * Exactly-once delivery and transactional messaging
> > > * Streams exactly-once semantics
> > > * Admin client with support for topic, ACLs and config management
> > > * Record headers
> > > * Request rate quotas
> > > * Improved resiliency: replication protocol improvement and
> > single-threaded
> > > controller
> > > * Richer and more efficient message format
> > >
> > > Release notes for the 0.11.0.0 release:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> > >
> > > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> > >
> > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > http://kafka.apache.org/KEYS
> > >
> > > * Release artifacts to be voted upon (source and binary):
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> > >
> > > * Maven artifacts to be voted upon:
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >
> > > * Javadoc:
> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> > >
> > > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> > >
> > > * Documentation:
> > > http://kafka.apache.org/0110/documentation.html
> > >
> > > * Protocol:
> > > http://kafka.apache.org/0110/protocol.html
> > >
> > > * Successful Jenkins builds for the 0.11.0 branch:
> > > Unit/integration tests: https://builds.apache.org/job/
> > > kafka-0.11.0-jdk7/187/
> > > System tests: pending (will send an update tomorrow)
> > >
> > > /**************************************
> > >
> > > Thanks,
> > > Ismael
> > >
> > >
> > >
> > >
> >
> >
> >
> >
> >
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Vahid S Hashemian <va...@us.ibm.com>.
Hi Ismael,

This is the output of core tests from the start until the first failed 
test.

kafka.admin.AdminRackAwareTest > 
testAssignmentWithRackAwareWithUnevenRacks PASSED

kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAware 
PASSED

kafka.admin.AdminRackAwareTest > 
testAssignmentWithRackAwareWithUnevenReplicas PASSED

kafka.admin.AdminRackAwareTest > testSkipBrokerWithReplicaAlreadyAssigned 
PASSED

kafka.admin.AdminRackAwareTest > testAssignmentWithRackAware PASSED

kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED

kafka.admin.AdminRackAwareTest > 
testAssignmentWith2ReplicasRackAwareWith6Partitions PASSED

kafka.admin.AdminRackAwareTest > 
testAssignmentWith2ReplicasRackAwareWith6PartitionsAnd3Brokers PASSED

kafka.admin.AdminRackAwareTest > 
testGetRackAlternatedBrokerListAndAssignReplicasToBrokers PASSED

kafka.admin.AdminRackAwareTest > testMoreReplicasThanRacks PASSED

kafka.admin.AdminRackAwareTest > testSingleRack PASSED

kafka.admin.AdminRackAwareTest > 
testAssignmentWithRackAwareWithRandomStartIndex PASSED

kafka.admin.AdminRackAwareTest > testLargeNumberPartitionsAssignment 
PASSED

kafka.admin.AdminRackAwareTest > testLessReplicasThanRacks PASSED

kafka.admin.AclCommandTest > testInvalidAuthorizerProperty PASSED

kafka.admin.ConfigCommandTest > testScramCredentials PASSED

kafka.admin.ConfigCommandTest > shouldParseArgumentsForTopicsEntityType 
PASSED

kafka.admin.ConfigCommandTest > testUserClientQuotaOpts PASSED

kafka.admin.ConfigCommandTest > shouldAddTopicConfig PASSED

kafka.admin.ConfigCommandTest > shouldAddClientConfig PASSED

kafka.admin.ConfigCommandTest > shouldDeleteBrokerConfig PASSED

kafka.admin.DeleteConsumerGroupTest > 
testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup PASSED

kafka.admin.ConfigCommandTest > testQuotaConfigEntity PASSED

kafka.admin.ConfigCommandTest > 
shouldNotUpdateBrokerConfigIfMalformedBracketConfig PASSED

kafka.admin.ConfigCommandTest > shouldFailIfUnrecognisedEntityType PASSED

kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED

kafka.admin.ConfigCommandTest > 
shouldNotUpdateBrokerConfigIfNonExistingConfigIsDeleted PASSED

kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED

kafka.admin.BrokerApiVersionsCommandTest > 
checkBrokerApiVersionCommandOutput PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldRemoveThrottleReplicaListBasedOnProposedAssignment PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindMovingReplicasMultipleTopics PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldNotOverwriteExistingPropertiesWhenLimitIsAdded PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindMovingReplicasMultipleTopicsAndPartitions PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldRemoveThrottleLimitFromAllBrokers PASSED

kafka.admin.ReassignPartitionsCommandTest > shouldFindMovingReplicas 
PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindMovingReplicasMultiplePartitions PASSED

kafka.admin.ReassignPartitionsCommandTest > shouldSetQuotaLimit PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindMovingReplicasWhenProposedIsSubsetOfExisting PASSED

kafka.admin.ReassignPartitionsCommandTest > shouldUpdateQuotaLimit PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldFindTwoMovingReplicasInSamePartition PASSED

kafka.admin.ReassignPartitionsCommandTest > 
shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas PASSED

kafka.admin.ConfigCommandTest > 
shouldNotUpdateBrokerConfigIfMalformedEntityName PASSED

kafka.admin.ConfigCommandTest > shouldSupportCommaSeparatedValues PASSED

kafka.admin.ConfigCommandTest > 
shouldNotUpdateBrokerConfigIfMalformedConfig PASSED

kafka.admin.DeleteConsumerGroupTest > 
testGroupTopicWideDeleteInZKDoesNothingForActiveGroupConsumingMultipleTopics 
PASSED

kafka.admin.AddPartitionsTest > testReplicaPlacementAllServers PASSED

kafka.admin.ConfigCommandTest > shouldParseArgumentsForBrokersEntityType 
PASSED

kafka.admin.ConfigCommandTest > shouldAddBrokerConfig PASSED

kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED

kafka.admin.ConfigCommandTest > testQuotaDescribeEntities PASSED

kafka.admin.AdminTest > testGetBrokerMetadatas PASSED

kafka.admin.ConfigCommandTest > shouldParseArgumentsForClientsEntityType 
PASSED

kafka.admin.AclCommandTest > testAclCli PASSED

kafka.admin.ReassignPartitionsIntegrationTest > testRackAwareReassign 
PASSED

kafka.admin.AdminTest > testBootstrapClientIdConfig PASSED

kafka.admin.ReassignPartitionsClusterTest > 
shouldExecuteThrottledReassignment FAILED
    java.nio.file.FileSystemException: 
C:\Users\IBM_AD~1\AppData\Local\Temp\kafka-719085320148197500\my-topic-0\00000000000000000000.index: 
The process cannot access the file because it is being used by another 
process.


From the error message, it sounds like one of the prior tests does not do 
a proper clean-up?!

Thanks.
--Vahid
 



From:   Ismael Juma <is...@juma.me.uk>
To:     dev@kafka.apache.org
Cc:     kafka-clients <ka...@googlegroups.com>, Kafka Users 
<us...@kafka.apache.org>
Date:   06/26/2017 01:54 PM
Subject:        Re: [VOTE] 0.11.0.0 RC2
Sent by:        ismaelj@gmail.com



Hi Vahid,

Can you please check which test fails first? The errors you mentioned can
happen if a test fails and doesn't clean-up properly.

Ismael

On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> To answer your questions:
>
> 1. Yes, the issues exists in trunk too.
>
> 2. I haven't checked with Cygwin, but I can give it a try.
>
> And thanks for addressing this issue. I can confirm with your PR I no
> longer see it.
> But now that the tests progress I see quite a few errors like this in
> core:
>
> kafka.server.ReplicaFetchTest > classMethod FAILED
>     java.lang.AssertionError: Found unexpected threads,
> allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> ProcessThread(sid:0 cport:56565):, metrics-mete
> r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> cport:59720):, ZkClie
> nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> /127.0.0.1:54926 w
> orkers Thread 2, Test worker, SyncThread:0,
> NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> 2 to /127.0.0.1:54926 workers Thread 3,
> ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> alizer, metrics-meter-tick-thread-1)
>
> I tested on a VM and a physical machine, and both give me a lot of 
errors
> like this.
>
> Thanks.
> --Vahid
>
>
>
>
> From:   Ismael Juma <is...@gmail.com>
> To:     Vahid S Hashemian <va...@us.ibm.com>
> Cc:     dev@kafka.apache.org, kafka-clients
> <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> Date:   06/26/2017 03:53 AM
> Subject:        Re: [VOTE] 0.11.0.0 RC2
>
>
>
> Hi Vahid,
>
> Sorry for not replying to the previous email, I had missed it. A couple 
of
> questions:
>
> 1. Is this also happening in trunk? Seems like it should be the case for
> months and seemingly no-one reported it until the RC stage.
> 2. Is it correct that this only happens when compiling on Windows 
without
> Cygwin?
>
> I believe the following PR should fix it, please verify:
>
> https://github.com/apache/kafka/pull/3431
>
> Ismael
>
> On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > Not sure if my response on RC1 was lost or this issue is not a
> > show-stopper:
> >
> > I checked again and with RC2, tests still fail in my Windown 64 bit
> > environment.
> >
> > :clients:checkstyleMain
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > [ApiExceptionBuilder, BrokerNotAvailableException,
> > ClusterAuthorizationException, ConcurrentTransactionsException,
> > ControllerMovedException, CoordinatorLoadInProgressException,
> > CoordinatorNotAvailableException, CorruptRecordException,
> > DuplicateSequenceNumberException, GroupAuthorizationException,
> > IllegalGenerationException, IllegalSaslStateException,
> > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > InvalidConfigurationException, InvalidFetchSizeException,
> > InvalidGroupIdException, InvalidPartitionsException,
> > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > InvalidReplicationFactorException, InvalidRequestException,
> > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > InvalidTimestampException, InvalidTopicException,
> InvalidTxnStateException,
> > InvalidTxnTimeoutException, LeaderNotAvailableException,
> NetworkException,
> > NotControllerException, NotCoordinatorException,
> > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > OffsetOutOfRangeException, OperationNotAttemptedException,
> > OutOfOrderSequenceException, PolicyViolationException,
> > ProducerFencedException, RebalanceInProgressException,
> > RecordBatchTooLargeException, RecordTooLargeException,
> > ReplicaNotAvailableException, SecurityDisabledException,
> TimeoutException,
> > TopicAuthorizationException, TopicExistsException,
> > TransactionCoordinatorFencedException,
> TransactionalIdAuthorizationException,
> > UnknownMemberIdException, UnknownServerException,
> > UnknownTopicOrPartitionException, 
UnsupportedForMessageFormatException,
> > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > [ClassDataAbstractionCoupling]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Fan-Out Complexity is 60 (max allowed is 40).
> [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 
(max
> > allowed is 40). [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 
(max
> > allowed is 40). [ClassFanOutComplexity]
> > :clients:checkstyleMain FAILED
> >
> > FAILURE: Build failed with an exception.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> > From:        Ismael Juma <is...@juma.me.uk>
> > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > kafka-clients <ka...@googlegroups.com>
> > Date:        06/22/2017 06:16 PM
> > Subject:        [VOTE] 0.11.0.0 RC2
> > Sent by:        ismaelj@gmail.com
> > ------------------------------
> >
> >
> >
> > Hello Kafka users, developers and client-developers,
> >
> > This is the third candidate for release of Apache Kafka 0.11.0.0.
> >
> > This is a major version release of Apache Kafka. It includes 32 new
> KIPs.
> > See the release notes and release plan (
> > 
https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> > for more details. A few feature highlights:
> >
> > * Exactly-once delivery and transactional messaging
> > * Streams exactly-once semantics
> > * Admin client with support for topic, ACLs and config management
> > * Record headers
> > * Request rate quotas
> > * Improved resiliency: replication protocol improvement and
> single-threaded
> > controller
> > * Richer and more efficient message format
> >
> > Release notes for the 0.11.0.0 release:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > http://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> >
> > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> >
> > * Documentation:
> > http://kafka.apache.org/0110/documentation.html
> >
> > * Protocol:
> > http://kafka.apache.org/0110/protocol.html
> >
> > * Successful Jenkins builds for the 0.11.0 branch:
> > Unit/integration tests: https://builds.apache.org/job/
> > kafka-0.11.0-jdk7/187/
> > System tests: pending (will send an update tomorrow)
> >
> > /**************************************
> >
> > Thanks,
> > Ismael
> >
> >
> >
> >
>
>
>
>
>





Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@juma.me.uk>.
Hi Vahid,

Can you please check which test fails first? The errors you mentioned can
happen if a test fails and doesn't clean-up properly.

Ismael

On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> To answer your questions:
>
> 1. Yes, the issues exists in trunk too.
>
> 2. I haven't checked with Cygwin, but I can give it a try.
>
> And thanks for addressing this issue. I can confirm with your PR I no
> longer see it.
> But now that the tests progress I see quite a few errors like this in
> core:
>
> kafka.server.ReplicaFetchTest > classMethod FAILED
>     java.lang.AssertionError: Found unexpected threads,
> allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> ProcessThread(sid:0 cport:56565):, metrics-mete
> r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> cport:59720):, ZkClie
> nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> /127.0.0.1:54926 w
> orkers Thread 2, Test worker, SyncThread:0,
> NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> 2 to /127.0.0.1:54926 workers Thread 3,
> ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> alizer, metrics-meter-tick-thread-1)
>
> I tested on a VM and a physical machine, and both give me a lot of errors
> like this.
>
> Thanks.
> --Vahid
>
>
>
>
> From:   Ismael Juma <is...@gmail.com>
> To:     Vahid S Hashemian <va...@us.ibm.com>
> Cc:     dev@kafka.apache.org, kafka-clients
> <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> Date:   06/26/2017 03:53 AM
> Subject:        Re: [VOTE] 0.11.0.0 RC2
>
>
>
> Hi Vahid,
>
> Sorry for not replying to the previous email, I had missed it. A couple of
> questions:
>
> 1. Is this also happening in trunk? Seems like it should be the case for
> months and seemingly no-one reported it until the RC stage.
> 2. Is it correct that this only happens when compiling on Windows without
> Cygwin?
>
> I believe the following PR should fix it, please verify:
>
> https://github.com/apache/kafka/pull/3431
>
> Ismael
>
> On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > Not sure if my response on RC1 was lost or this issue is not a
> > show-stopper:
> >
> > I checked again and with RC2, tests still fail in my Windown 64 bit
> > environment.
> >
> > :clients:checkstyleMain
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > [ApiExceptionBuilder, BrokerNotAvailableException,
> > ClusterAuthorizationException, ConcurrentTransactionsException,
> > ControllerMovedException, CoordinatorLoadInProgressException,
> > CoordinatorNotAvailableException, CorruptRecordException,
> > DuplicateSequenceNumberException, GroupAuthorizationException,
> > IllegalGenerationException, IllegalSaslStateException,
> > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > InvalidConfigurationException, InvalidFetchSizeException,
> > InvalidGroupIdException, InvalidPartitionsException,
> > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > InvalidReplicationFactorException, InvalidRequestException,
> > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > InvalidTimestampException, InvalidTopicException,
> InvalidTxnStateException,
> > InvalidTxnTimeoutException, LeaderNotAvailableException,
> NetworkException,
> > NotControllerException, NotCoordinatorException,
> > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > OffsetOutOfRangeException, OperationNotAttemptedException,
> > OutOfOrderSequenceException, PolicyViolationException,
> > ProducerFencedException, RebalanceInProgressException,
> > RecordBatchTooLargeException, RecordTooLargeException,
> > ReplicaNotAvailableException, SecurityDisabledException,
> TimeoutException,
> > TopicAuthorizationException, TopicExistsException,
> > TransactionCoordinatorFencedException,
> TransactionalIdAuthorizationException,
> > UnknownMemberIdException, UnknownServerException,
> > UnknownTopicOrPartitionException, UnsupportedForMessageFormatException,
> > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > [ClassDataAbstractionCoupling]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Fan-Out Complexity is 60 (max allowed is 40).
> [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 (max
> > allowed is 40). [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 (max
> > allowed is 40). [ClassFanOutComplexity]
> > :clients:checkstyleMain FAILED
> >
> > FAILURE: Build failed with an exception.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> > From:        Ismael Juma <is...@juma.me.uk>
> > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > kafka-clients <ka...@googlegroups.com>
> > Date:        06/22/2017 06:16 PM
> > Subject:        [VOTE] 0.11.0.0 RC2
> > Sent by:        ismaelj@gmail.com
> > ------------------------------
> >
> >
> >
> > Hello Kafka users, developers and client-developers,
> >
> > This is the third candidate for release of Apache Kafka 0.11.0.0.
> >
> > This is a major version release of Apache Kafka. It includes 32 new
> KIPs.
> > See the release notes and release plan (
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> > for more details. A few feature highlights:
> >
> > * Exactly-once delivery and transactional messaging
> > * Streams exactly-once semantics
> > * Admin client with support for topic, ACLs and config management
> > * Record headers
> > * Request rate quotas
> > * Improved resiliency: replication protocol improvement and
> single-threaded
> > controller
> > * Richer and more efficient message format
> >
> > Release notes for the 0.11.0.0 release:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > http://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> >
> > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> >
> > * Documentation:
> > http://kafka.apache.org/0110/documentation.html
> >
> > * Protocol:
> > http://kafka.apache.org/0110/protocol.html
> >
> > * Successful Jenkins builds for the 0.11.0 branch:
> > Unit/integration tests: https://builds.apache.org/job/
> > kafka-0.11.0-jdk7/187/
> > System tests: pending (will send an update tomorrow)
> >
> > /**************************************
> >
> > Thanks,
> > Ismael
> >
> >
> >
> >
>
>
>
>
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Jeff Chao <jc...@heroku.com>.
Hi,

Heroku has been doing additional performance testing on (1) log compaction
and, separately (2) Go clients with older message format against 0.11-rc2
brokers with new message format.

For log compaction, we've tested with messages using a single key, messages
using unique keys, and messages with a bounded key range. There were no
notable negative performance impacts.

For client testing with old format vs new format, we had Sarama Go async
producer clients speaking their older client protocol versions and had
messages producing in a tight loop. This resulted in a high percentage of
errors, though some messages went through:

Failed to produce message kafka: Failed to produce message to topic
rc2-topic: kafka server: Message was too large, server rejected it to avoid
allocation error.

Although this is to be expected as mentioned in the docs (
http://kafka.apache.org/0110/documentation.html#upgrade_11_message_format)
where in aggregate messages may become larger than max.message.bytes from
the broker, we'd like to point out that this might be confusing for users
running older clients against 0.11. That said, users can however work
around this issue by tuning their request size to be less than
max.message.bytes.

This, along with the testing previously mentioned by Tom wraps up our
performance testing. Overall, we're a +1 (non-binding) for this release,
but wanted to point out the client issue above.

Thanks,
Jeff

On Mon, Jun 26, 2017 at 12:41 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> To answer your questions:
>
> 1. Yes, the issues exists in trunk too.
>
> 2. I haven't checked with Cygwin, but I can give it a try.
>
> And thanks for addressing this issue. I can confirm with your PR I no
> longer see it.
> But now that the tests progress I see quite a few errors like this in
> core:
>
> kafka.server.ReplicaFetchTest > classMethod FAILED
>     java.lang.AssertionError: Found unexpected threads,
> allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> ProcessThread(sid:0 cport:56565):, metrics-mete
> r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> cport:59720):, ZkClie
> nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> /127.0.0.1:54926 w
> orkers Thread 2, Test worker, SyncThread:0,
> NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> 2 to /127.0.0.1:54926 workers Thread 3,
> ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> alizer, metrics-meter-tick-thread-1)
>
> I tested on a VM and a physical machine, and both give me a lot of errors
> like this.
>
> Thanks.
> --Vahid
>
>
>
>
> From:   Ismael Juma <is...@gmail.com>
> To:     Vahid S Hashemian <va...@us.ibm.com>
> Cc:     dev@kafka.apache.org, kafka-clients
> <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> Date:   06/26/2017 03:53 AM
> Subject:        Re: [VOTE] 0.11.0.0 RC2
>
>
>
> Hi Vahid,
>
> Sorry for not replying to the previous email, I had missed it. A couple of
> questions:
>
> 1. Is this also happening in trunk? Seems like it should be the case for
> months and seemingly no-one reported it until the RC stage.
> 2. Is it correct that this only happens when compiling on Windows without
> Cygwin?
>
> I believe the following PR should fix it, please verify:
>
> https://github.com/apache/kafka/pull/3431
>
> Ismael
>
> On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > Not sure if my response on RC1 was lost or this issue is not a
> > show-stopper:
> >
> > I checked again and with RC2, tests still fail in my Windown 64 bit
> > environment.
> >
> > :clients:checkstyleMain
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > [ApiExceptionBuilder, BrokerNotAvailableException,
> > ClusterAuthorizationException, ConcurrentTransactionsException,
> > ControllerMovedException, CoordinatorLoadInProgressException,
> > CoordinatorNotAvailableException, CorruptRecordException,
> > DuplicateSequenceNumberException, GroupAuthorizationException,
> > IllegalGenerationException, IllegalSaslStateException,
> > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > InvalidConfigurationException, InvalidFetchSizeException,
> > InvalidGroupIdException, InvalidPartitionsException,
> > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > InvalidReplicationFactorException, InvalidRequestException,
> > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > InvalidTimestampException, InvalidTopicException,
> InvalidTxnStateException,
> > InvalidTxnTimeoutException, LeaderNotAvailableException,
> NetworkException,
> > NotControllerException, NotCoordinatorException,
> > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > OffsetOutOfRangeException, OperationNotAttemptedException,
> > OutOfOrderSequenceException, PolicyViolationException,
> > ProducerFencedException, RebalanceInProgressException,
> > RecordBatchTooLargeException, RecordTooLargeException,
> > ReplicaNotAvailableException, SecurityDisabledException,
> TimeoutException,
> > TopicAuthorizationException, TopicExistsException,
> > TransactionCoordinatorFencedException,
> TransactionalIdAuthorizationException,
> > UnknownMemberIdException, UnknownServerException,
> > UnknownTopicOrPartitionException, UnsupportedForMessageFormatException,
> > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > [ClassDataAbstractionCoupling]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Fan-Out Complexity is 60 (max allowed is 40).
> [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 (max
> > allowed is 40). [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 (max
> > allowed is 40). [ClassFanOutComplexity]
> > :clients:checkstyleMain FAILED
> >
> > FAILURE: Build failed with an exception.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> > From:        Ismael Juma <is...@juma.me.uk>
> > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > kafka-clients <ka...@googlegroups.com>
> > Date:        06/22/2017 06:16 PM
> > Subject:        [VOTE] 0.11.0.0 RC2
> > Sent by:        ismaelj@gmail.com
> > ------------------------------
> >
> >
> >
> > Hello Kafka users, developers and client-developers,
> >
> > This is the third candidate for release of Apache Kafka 0.11.0.0.
> >
> > This is a major version release of Apache Kafka. It includes 32 new
> KIPs.
> > See the release notes and release plan (
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> > for more details. A few feature highlights:
> >
> > * Exactly-once delivery and transactional messaging
> > * Streams exactly-once semantics
> > * Admin client with support for topic, ACLs and config management
> > * Record headers
> > * Request rate quotas
> > * Improved resiliency: replication protocol improvement and
> single-threaded
> > controller
> > * Richer and more efficient message format
> >
> > Release notes for the 0.11.0.0 release:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > http://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> >
> > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> >
> > * Documentation:
> > http://kafka.apache.org/0110/documentation.html
> >
> > * Protocol:
> > http://kafka.apache.org/0110/protocol.html
> >
> > * Successful Jenkins builds for the 0.11.0 branch:
> > Unit/integration tests: https://builds.apache.org/job/
> > kafka-0.11.0-jdk7/187/
> > System tests: pending (will send an update tomorrow)
> >
> > /**************************************
> >
> > Thanks,
> > Ismael
> >
> >
> >
> >
>
>
>
>
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@juma.me.uk>.
Hi Vahid,

Can you please check which test fails first? The errors you mentioned can
happen if a test fails and doesn't clean-up properly.

Ismael

On Mon, Jun 26, 2017 at 8:41 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> To answer your questions:
>
> 1. Yes, the issues exists in trunk too.
>
> 2. I haven't checked with Cygwin, but I can give it a try.
>
> And thanks for addressing this issue. I can confirm with your PR I no
> longer see it.
> But now that the tests progress I see quite a few errors like this in
> core:
>
> kafka.server.ReplicaFetchTest > classMethod FAILED
>     java.lang.AssertionError: Found unexpected threads,
> allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565,
> ProcessThread(sid:0 cport:56565):, metrics-mete
> r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference
> Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0
> cport:59720):, ZkClie
> nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread |
> producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to
> /127.0.0.1:54926 w
> orkers Thread 2, Test worker, SyncThread:0,
> NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test
> worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
> 2 to /127.0.0.1:54926 workers Thread 3,
> ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0
> cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
> alizer, metrics-meter-tick-thread-1)
>
> I tested on a VM and a physical machine, and both give me a lot of errors
> like this.
>
> Thanks.
> --Vahid
>
>
>
>
> From:   Ismael Juma <is...@gmail.com>
> To:     Vahid S Hashemian <va...@us.ibm.com>
> Cc:     dev@kafka.apache.org, kafka-clients
> <ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
> Date:   06/26/2017 03:53 AM
> Subject:        Re: [VOTE] 0.11.0.0 RC2
>
>
>
> Hi Vahid,
>
> Sorry for not replying to the previous email, I had missed it. A couple of
> questions:
>
> 1. Is this also happening in trunk? Seems like it should be the case for
> months and seemingly no-one reported it until the RC stage.
> 2. Is it correct that this only happens when compiling on Windows without
> Cygwin?
>
> I believe the following PR should fix it, please verify:
>
> https://github.com/apache/kafka/pull/3431
>
> Ismael
>
> On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
> vahidhashemian@us.ibm.com> wrote:
>
> > Hi Ismael,
> >
> > Not sure if my response on RC1 was lost or this issue is not a
> > show-stopper:
> >
> > I checked again and with RC2, tests still fail in my Windown 64 bit
> > environment.
> >
> > :clients:checkstyleMain
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> > [ApiExceptionBuilder, BrokerNotAvailableException,
> > ClusterAuthorizationException, ConcurrentTransactionsException,
> > ControllerMovedException, CoordinatorLoadInProgressException,
> > CoordinatorNotAvailableException, CorruptRecordException,
> > DuplicateSequenceNumberException, GroupAuthorizationException,
> > IllegalGenerationException, IllegalSaslStateException,
> > InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> > InvalidConfigurationException, InvalidFetchSizeException,
> > InvalidGroupIdException, InvalidPartitionsException,
> > InvalidPidMappingException, InvalidReplicaAssignmentException,
> > InvalidReplicationFactorException, InvalidRequestException,
> > InvalidRequiredAcksException, InvalidSessionTimeoutException,
> > InvalidTimestampException, InvalidTopicException,
> InvalidTxnStateException,
> > InvalidTxnTimeoutException, LeaderNotAvailableException,
> NetworkException,
> > NotControllerException, NotCoordinatorException,
> > NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> > NotLeaderForPartitionException, OffsetMetadataTooLarge,
> > OffsetOutOfRangeException, OperationNotAttemptedException,
> > OutOfOrderSequenceException, PolicyViolationException,
> > ProducerFencedException, RebalanceInProgressException,
> > RecordBatchTooLargeException, RecordTooLargeException,
> > ReplicaNotAvailableException, SecurityDisabledException,
> TimeoutException,
> > TopicAuthorizationException, TopicExistsException,
> > TransactionCoordinatorFencedException,
> TransactionalIdAuthorizationException,
> > UnknownMemberIdException, UnknownServerException,
> > UnknownTopicOrPartitionException, UnsupportedForMessageFormatException,
> > UnsupportedSaslMechanismException, UnsupportedVersionException].
> > [ClassDataAbstractionCoupling]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> >
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> protocol\Errors.java:89:1:
> > Class Fan-Out Complexity is 60 (max allowed is 40).
> [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 (max
> > allowed is 40). [ClassFanOutComplexity]
> > [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 (max
> > allowed is 40). [ClassFanOutComplexity]
> > :clients:checkstyleMain FAILED
> >
> > FAILURE: Build failed with an exception.
> >
> > Thanks.
> > --Vahid
> >
> >
> >
> > From:        Ismael Juma <is...@juma.me.uk>
> > To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> > kafka-clients <ka...@googlegroups.com>
> > Date:        06/22/2017 06:16 PM
> > Subject:        [VOTE] 0.11.0.0 RC2
> > Sent by:        ismaelj@gmail.com
> > ------------------------------
> >
> >
> >
> > Hello Kafka users, developers and client-developers,
> >
> > This is the third candidate for release of Apache Kafka 0.11.0.0.
> >
> > This is a major version release of Apache Kafka. It includes 32 new
> KIPs.
> > See the release notes and release plan (
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> > for more details. A few feature highlights:
> >
> > * Exactly-once delivery and transactional messaging
> > * Streams exactly-once semantics
> > * Admin client with support for topic, ACLs and config management
> > * Record headers
> > * Request rate quotas
> > * Improved resiliency: replication protocol improvement and
> single-threaded
> > controller
> > * Richer and more efficient message format
> >
> > Release notes for the 0.11.0.0 release:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > http://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> >
> > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> >
> > * Documentation:
> > http://kafka.apache.org/0110/documentation.html
> >
> > * Protocol:
> > http://kafka.apache.org/0110/protocol.html
> >
> > * Successful Jenkins builds for the 0.11.0 branch:
> > Unit/integration tests: https://builds.apache.org/job/
> > kafka-0.11.0-jdk7/187/
> > System tests: pending (will send an update tomorrow)
> >
> > /**************************************
> >
> > Thanks,
> > Ismael
> >
> >
> >
> >
>
>
>
>
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Vahid S Hashemian <va...@us.ibm.com>.
Hi Ismael,

To answer your questions:

1. Yes, the issues exists in trunk too.

2. I haven't checked with Cygwin, but I can give it a try.

And thanks for addressing this issue. I can confirm with your PR I no 
longer see it.
But now that the tests progress I see quite a few errors like this in 
core:

kafka.server.ReplicaFetchTest > classMethod FAILED
    java.lang.AssertionError: Found unexpected threads, 
allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565, 
ProcessThread(sid:0 cport:56565):, metrics-mete
r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference 
Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0 
cport:59720):, ZkClie
nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread | 
producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to 
/127.0.0.1:54926 w
orkers Thread 2, Test worker, SyncThread:0, 
NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test 
worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
2 to /127.0.0.1:54926 workers Thread 3, 
ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0 
cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
alizer, metrics-meter-tick-thread-1)

I tested on a VM and a physical machine, and both give me a lot of errors 
like this.

Thanks.
--Vahid




From:   Ismael Juma <is...@gmail.com>
To:     Vahid S Hashemian <va...@us.ibm.com>
Cc:     dev@kafka.apache.org, kafka-clients 
<ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
Date:   06/26/2017 03:53 AM
Subject:        Re: [VOTE] 0.11.0.0 RC2



Hi Vahid,

Sorry for not replying to the previous email, I had missed it. A couple of
questions:

1. Is this also happening in trunk? Seems like it should be the case for
months and seemingly no-one reported it until the RC stage.
2. Is it correct that this only happens when compiling on Windows without
Cygwin?

I believe the following PR should fix it, please verify:

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

Ismael

On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> Not sure if my response on RC1 was lost or this issue is not a
> show-stopper:
>
> I checked again and with RC2, tests still fail in my Windown 64 bit
> environment.
>
> :clients:checkstyleMain
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 
0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1:
> Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> [ApiExceptionBuilder, BrokerNotAvailableException,
> ClusterAuthorizationException, ConcurrentTransactionsException,
> ControllerMovedException, CoordinatorLoadInProgressException,
> CoordinatorNotAvailableException, CorruptRecordException,
> DuplicateSequenceNumberException, GroupAuthorizationException,
> IllegalGenerationException, IllegalSaslStateException,
> InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> InvalidConfigurationException, InvalidFetchSizeException,
> InvalidGroupIdException, InvalidPartitionsException,
> InvalidPidMappingException, InvalidReplicaAssignmentException,
> InvalidReplicationFactorException, InvalidRequestException,
> InvalidRequiredAcksException, InvalidSessionTimeoutException,
> InvalidTimestampException, InvalidTopicException, 
InvalidTxnStateException,
> InvalidTxnTimeoutException, LeaderNotAvailableException, 
NetworkException,
> NotControllerException, NotCoordinatorException,
> NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> NotLeaderForPartitionException, OffsetMetadataTooLarge,
> OffsetOutOfRangeException, OperationNotAttemptedException,
> OutOfOrderSequenceException, PolicyViolationException,
> ProducerFencedException, RebalanceInProgressException,
> RecordBatchTooLargeException, RecordTooLargeException,
> ReplicaNotAvailableException, SecurityDisabledException, 
TimeoutException,
> TopicAuthorizationException, TopicExistsException,
> TransactionCoordinatorFencedException, 
TransactionalIdAuthorizationException,
> UnknownMemberIdException, UnknownServerException,
> UnknownTopicOrPartitionException, UnsupportedForMessageFormatException,
> UnsupportedSaslMechanismException, UnsupportedVersionException].
> [ClassDataAbstractionCoupling]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 
0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1:
> Class Fan-Out Complexity is 60 (max allowed is 40). 
[ClassFanOutComplexity]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 (max
> allowed is 40). [ClassFanOutComplexity]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 (max
> allowed is 40). [ClassFanOutComplexity]
> :clients:checkstyleMain FAILED
>
> FAILURE: Build failed with an exception.
>
> Thanks.
> --Vahid
>
>
>
> From:        Ismael Juma <is...@juma.me.uk>
> To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> kafka-clients <ka...@googlegroups.com>
> Date:        06/22/2017 06:16 PM
> Subject:        [VOTE] 0.11.0.0 RC2
> Sent by:        ismaelj@gmail.com
> ------------------------------
>
>
>
> Hello Kafka users, developers and client-developers,
>
> This is the third candidate for release of Apache Kafka 0.11.0.0.
>
> This is a major version release of Apache Kafka. It includes 32 new 
KIPs.
> See the release notes and release plan (
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> for more details. A few feature highlights:
>
> * Exactly-once delivery and transactional messaging
> * Streams exactly-once semantics
> * Admin client with support for topic, ACLs and config management
> * Record headers
> * Request rate quotas
> * Improved resiliency: replication protocol improvement and 
single-threaded
> controller
> * Richer and more efficient message format
>
> Release notes for the 0.11.0.0 release:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>
> *** Please download, test and vote by Tuesday, June 27, 6pm PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> http://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>
> * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> 8698fa1f41102f1664b05baa4d6953fc9564d91e
>
> * Documentation:
> http://kafka.apache.org/0110/documentation.html
>
> * Protocol:
> http://kafka.apache.org/0110/protocol.html
>
> * Successful Jenkins builds for the 0.11.0 branch:
> Unit/integration tests: https://builds.apache.org/job/
> kafka-0.11.0-jdk7/187/
> System tests: pending (will send an update tomorrow)
>
> /**************************************
>
> Thanks,
> Ismael
>
>
>
>





Re: [VOTE] 0.11.0.0 RC2

Posted by Vahid S Hashemian <va...@us.ibm.com>.
Hi Ismael,

To answer your questions:

1. Yes, the issues exists in trunk too.

2. I haven't checked with Cygwin, but I can give it a try.

And thanks for addressing this issue. I can confirm with your PR I no 
longer see it.
But now that the tests progress I see quite a few errors like this in 
core:

kafka.server.ReplicaFetchTest > classMethod FAILED
    java.lang.AssertionError: Found unexpected threads, 
allThreads=Set(ZkClient-EventThread-268-127.0.0.1:56565, 
ProcessThread(sid:0 cport:56565):, metrics-mete
r-tick-thread-2, SessionTracker, Signal Dispatcher, main, Reference 
Handler, ForkJoinPool-1-worker-1, Attach Listener, ProcessThread(sid:0 
cport:59720):, ZkClie
nt-EventThread-1347-127.0.0.1:59720, kafka-producer-network-thread | 
producer-1, Test worker-SendThread(127.0.0.1:56565), /127.0.0.1:54942 to 
/127.0.0.1:54926 w
orkers Thread 2, Test worker, SyncThread:0, 
NIOServerCxn.Factory:/127.0.0.1:0, Test worker-EventThread, Test 
worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494
2 to /127.0.0.1:54926 workers Thread 3, 
ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0 
cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin
alizer, metrics-meter-tick-thread-1)

I tested on a VM and a physical machine, and both give me a lot of errors 
like this.

Thanks.
--Vahid




From:   Ismael Juma <is...@gmail.com>
To:     Vahid S Hashemian <va...@us.ibm.com>
Cc:     dev@kafka.apache.org, kafka-clients 
<ka...@googlegroups.com>, Kafka Users <us...@kafka.apache.org>
Date:   06/26/2017 03:53 AM
Subject:        Re: [VOTE] 0.11.0.0 RC2



Hi Vahid,

Sorry for not replying to the previous email, I had missed it. A couple of
questions:

1. Is this also happening in trunk? Seems like it should be the case for
months and seemingly no-one reported it until the RC stage.
2. Is it correct that this only happens when compiling on Windows without
Cygwin?

I believe the following PR should fix it, please verify:

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

Ismael

On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> Not sure if my response on RC1 was lost or this issue is not a
> show-stopper:
>
> I checked again and with RC2, tests still fail in my Windown 64 bit
> environment.
>
> :clients:checkstyleMain
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 
0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1:
> Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> [ApiExceptionBuilder, BrokerNotAvailableException,
> ClusterAuthorizationException, ConcurrentTransactionsException,
> ControllerMovedException, CoordinatorLoadInProgressException,
> CoordinatorNotAvailableException, CorruptRecordException,
> DuplicateSequenceNumberException, GroupAuthorizationException,
> IllegalGenerationException, IllegalSaslStateException,
> InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> InvalidConfigurationException, InvalidFetchSizeException,
> InvalidGroupIdException, InvalidPartitionsException,
> InvalidPidMappingException, InvalidReplicaAssignmentException,
> InvalidReplicationFactorException, InvalidRequestException,
> InvalidRequiredAcksException, InvalidSessionTimeoutException,
> InvalidTimestampException, InvalidTopicException, 
InvalidTxnStateException,
> InvalidTxnTimeoutException, LeaderNotAvailableException, 
NetworkException,
> NotControllerException, NotCoordinatorException,
> NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> NotLeaderForPartitionException, OffsetMetadataTooLarge,
> OffsetOutOfRangeException, OperationNotAttemptedException,
> OutOfOrderSequenceException, PolicyViolationException,
> ProducerFencedException, RebalanceInProgressException,
> RecordBatchTooLargeException, RecordTooLargeException,
> ReplicaNotAvailableException, SecurityDisabledException, 
TimeoutException,
> TopicAuthorizationException, TopicExistsException,
> TransactionCoordinatorFencedException, 
TransactionalIdAuthorizationException,
> UnknownMemberIdException, UnknownServerException,
> UnknownTopicOrPartitionException, UnsupportedForMessageFormatException,
> UnsupportedSaslMechanismException, UnsupportedVersionException].
> [ClassDataAbstractionCoupling]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 
0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1:
> Class Fan-Out Complexity is 60 (max allowed is 40). 
[ClassFanOutComplexity]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 (max
> allowed is 40). [ClassFanOutComplexity]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 (max
> allowed is 40). [ClassFanOutComplexity]
> :clients:checkstyleMain FAILED
>
> FAILURE: Build failed with an exception.
>
> Thanks.
> --Vahid
>
>
>
> From:        Ismael Juma <is...@juma.me.uk>
> To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> kafka-clients <ka...@googlegroups.com>
> Date:        06/22/2017 06:16 PM
> Subject:        [VOTE] 0.11.0.0 RC2
> Sent by:        ismaelj@gmail.com
> ------------------------------
>
>
>
> Hello Kafka users, developers and client-developers,
>
> This is the third candidate for release of Apache Kafka 0.11.0.0.
>
> This is a major version release of Apache Kafka. It includes 32 new 
KIPs.
> See the release notes and release plan (
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> for more details. A few feature highlights:
>
> * Exactly-once delivery and transactional messaging
> * Streams exactly-once semantics
> * Admin client with support for topic, ACLs and config management
> * Record headers
> * Request rate quotas
> * Improved resiliency: replication protocol improvement and 
single-threaded
> controller
> * Richer and more efficient message format
>
> Release notes for the 0.11.0.0 release:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>
> *** Please download, test and vote by Tuesday, June 27, 6pm PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> http://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>
> * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> 8698fa1f41102f1664b05baa4d6953fc9564d91e
>
> * Documentation:
> http://kafka.apache.org/0110/documentation.html
>
> * Protocol:
> http://kafka.apache.org/0110/protocol.html
>
> * Successful Jenkins builds for the 0.11.0 branch:
> Unit/integration tests: https://builds.apache.org/job/
> kafka-0.11.0-jdk7/187/
> System tests: pending (will send an update tomorrow)
>
> /**************************************
>
> Thanks,
> Ismael
>
>
>
>





Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@gmail.com>.
Hi Vahid,

Sorry for not replying to the previous email, I had missed it. A couple of
questions:

1. Is this also happening in trunk? Seems like it should be the case for
months and seemingly no-one reported it until the RC stage.
2. Is it correct that this only happens when compiling on Windows without
Cygwin?

I believe the following PR should fix it, please verify:

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

Ismael

On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> Not sure if my response on RC1 was lost or this issue is not a
> show-stopper:
>
> I checked again and with RC2, tests still fail in my Windown 64 bit
> environment.
>
> :clients:checkstyleMain
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1:
> Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> [ApiExceptionBuilder, BrokerNotAvailableException,
> ClusterAuthorizationException, ConcurrentTransactionsException,
> ControllerMovedException, CoordinatorLoadInProgressException,
> CoordinatorNotAvailableException, CorruptRecordException,
> DuplicateSequenceNumberException, GroupAuthorizationException,
> IllegalGenerationException, IllegalSaslStateException,
> InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> InvalidConfigurationException, InvalidFetchSizeException,
> InvalidGroupIdException, InvalidPartitionsException,
> InvalidPidMappingException, InvalidReplicaAssignmentException,
> InvalidReplicationFactorException, InvalidRequestException,
> InvalidRequiredAcksException, InvalidSessionTimeoutException,
> InvalidTimestampException, InvalidTopicException, InvalidTxnStateException,
> InvalidTxnTimeoutException, LeaderNotAvailableException, NetworkException,
> NotControllerException, NotCoordinatorException,
> NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> NotLeaderForPartitionException, OffsetMetadataTooLarge,
> OffsetOutOfRangeException, OperationNotAttemptedException,
> OutOfOrderSequenceException, PolicyViolationException,
> ProducerFencedException, RebalanceInProgressException,
> RecordBatchTooLargeException, RecordTooLargeException,
> ReplicaNotAvailableException, SecurityDisabledException, TimeoutException,
> TopicAuthorizationException, TopicExistsException,
> TransactionCoordinatorFencedException, TransactionalIdAuthorizationException,
> UnknownMemberIdException, UnknownServerException,
> UnknownTopicOrPartitionException, UnsupportedForMessageFormatException,
> UnsupportedSaslMechanismException, UnsupportedVersionException].
> [ClassDataAbstractionCoupling]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1:
> Class Fan-Out Complexity is 60 (max allowed is 40). [ClassFanOutComplexity]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 (max
> allowed is 40). [ClassFanOutComplexity]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 (max
> allowed is 40). [ClassFanOutComplexity]
> :clients:checkstyleMain FAILED
>
> FAILURE: Build failed with an exception.
>
> Thanks.
> --Vahid
>
>
>
> From:        Ismael Juma <is...@juma.me.uk>
> To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> kafka-clients <ka...@googlegroups.com>
> Date:        06/22/2017 06:16 PM
> Subject:        [VOTE] 0.11.0.0 RC2
> Sent by:        ismaelj@gmail.com
> ------------------------------
>
>
>
> Hello Kafka users, developers and client-developers,
>
> This is the third candidate for release of Apache Kafka 0.11.0.0.
>
> This is a major version release of Apache Kafka. It includes 32 new KIPs.
> See the release notes and release plan (
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> for more details. A few feature highlights:
>
> * Exactly-once delivery and transactional messaging
> * Streams exactly-once semantics
> * Admin client with support for topic, ACLs and config management
> * Record headers
> * Request rate quotas
> * Improved resiliency: replication protocol improvement and single-threaded
> controller
> * Richer and more efficient message format
>
> Release notes for the 0.11.0.0 release:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>
> *** Please download, test and vote by Tuesday, June 27, 6pm PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> http://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>
> * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> 8698fa1f41102f1664b05baa4d6953fc9564d91e
>
> * Documentation:
> http://kafka.apache.org/0110/documentation.html
>
> * Protocol:
> http://kafka.apache.org/0110/protocol.html
>
> * Successful Jenkins builds for the 0.11.0 branch:
> Unit/integration tests: https://builds.apache.org/job/
> kafka-0.11.0-jdk7/187/
> System tests: pending (will send an update tomorrow)
>
> /**************************************
>
> Thanks,
> Ismael
>
>
>
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@gmail.com>.
Hi Vahid,

Sorry for not replying to the previous email, I had missed it. A couple of
questions:

1. Is this also happening in trunk? Seems like it should be the case for
months and seemingly no-one reported it until the RC stage.
2. Is it correct that this only happens when compiling on Windows without
Cygwin?

I believe the following PR should fix it, please verify:

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

Ismael

On Fri, Jun 23, 2017 at 8:25 PM, Vahid S Hashemian <
vahidhashemian@us.ibm.com> wrote:

> Hi Ismael,
>
> Not sure if my response on RC1 was lost or this issue is not a
> show-stopper:
>
> I checked again and with RC2, tests still fail in my Windown 64 bit
> environment.
>
> :clients:checkstyleMain
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1:
> Class Data Abstraction Coupling is 57 (max allowed is 20) classes
> [ApiExceptionBuilder, BrokerNotAvailableException,
> ClusterAuthorizationException, ConcurrentTransactionsException,
> ControllerMovedException, CoordinatorLoadInProgressException,
> CoordinatorNotAvailableException, CorruptRecordException,
> DuplicateSequenceNumberException, GroupAuthorizationException,
> IllegalGenerationException, IllegalSaslStateException,
> InconsistentGroupProtocolException, InvalidCommitOffsetSizeException,
> InvalidConfigurationException, InvalidFetchSizeException,
> InvalidGroupIdException, InvalidPartitionsException,
> InvalidPidMappingException, InvalidReplicaAssignmentException,
> InvalidReplicationFactorException, InvalidRequestException,
> InvalidRequiredAcksException, InvalidSessionTimeoutException,
> InvalidTimestampException, InvalidTopicException, InvalidTxnStateException,
> InvalidTxnTimeoutException, LeaderNotAvailableException, NetworkException,
> NotControllerException, NotCoordinatorException,
> NotEnoughReplicasAfterAppendException, NotEnoughReplicasException,
> NotLeaderForPartitionException, OffsetMetadataTooLarge,
> OffsetOutOfRangeException, OperationNotAttemptedException,
> OutOfOrderSequenceException, PolicyViolationException,
> ProducerFencedException, RebalanceInProgressException,
> RecordBatchTooLargeException, RecordTooLargeException,
> ReplicaNotAvailableException, SecurityDisabledException, TimeoutException,
> TopicAuthorizationException, TopicExistsException,
> TransactionCoordinatorFencedException, TransactionalIdAuthorizationException,
> UnknownMemberIdException, UnknownServerException,
> UnknownTopicOrPartitionException, UnsupportedForMessageFormatException,
> UnsupportedSaslMechanismException, UnsupportedVersionException].
> [ClassDataAbstractionCoupling]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1:
> Class Fan-Out Complexity is 60 (max allowed is 40). [ClassFanOutComplexity]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> requests\AbstractRequest.java:26:1: Class Fan-Out Complexity is 43 (max
> allowed is 40). [ClassFanOutComplexity]
> [ant:checkstyle] [ERROR] C:\Users\User\Downloads\kafka-
> 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\
> requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 (max
> allowed is 40). [ClassFanOutComplexity]
> :clients:checkstyleMain FAILED
>
> FAILURE: Build failed with an exception.
>
> Thanks.
> --Vahid
>
>
>
> From:        Ismael Juma <is...@juma.me.uk>
> To:        dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>,
> kafka-clients <ka...@googlegroups.com>
> Date:        06/22/2017 06:16 PM
> Subject:        [VOTE] 0.11.0.0 RC2
> Sent by:        ismaelj@gmail.com
> ------------------------------
>
>
>
> Hello Kafka users, developers and client-developers,
>
> This is the third candidate for release of Apache Kafka 0.11.0.0.
>
> This is a major version release of Apache Kafka. It includes 32 new KIPs.
> See the release notes and release plan (
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
> for more details. A few feature highlights:
>
> * Exactly-once delivery and transactional messaging
> * Streams exactly-once semantics
> * Admin client with support for topic, ACLs and config management
> * Record headers
> * Request rate quotas
> * Improved resiliency: replication protocol improvement and single-threaded
> controller
> * Richer and more efficient message format
>
> Release notes for the 0.11.0.0 release:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>
> *** Please download, test and vote by Tuesday, June 27, 6pm PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> http://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>
> * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> 8698fa1f41102f1664b05baa4d6953fc9564d91e
>
> * Documentation:
> http://kafka.apache.org/0110/documentation.html
>
> * Protocol:
> http://kafka.apache.org/0110/protocol.html
>
> * Successful Jenkins builds for the 0.11.0 branch:
> Unit/integration tests: https://builds.apache.org/job/
> kafka-0.11.0-jdk7/187/
> System tests: pending (will send an update tomorrow)
>
> /**************************************
>
> Thanks,
> Ismael
>
>
>
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Vahid S Hashemian <va...@us.ibm.com>.
Hi Ismael,

Not sure if my response on RC1 was lost or this issue is not a 
show-stopper:

I checked again and with RC2, tests still fail in my Windown 64 bit 
environment.

:clients:checkstyleMain
[ant:checkstyle] [ERROR] 
C:\Users\User\Downloads\kafka-0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1: 
Class Data Abstraction Coupling is 57 (max allowed is 20) classes 
[ApiExceptionBuilder, BrokerNotAvailableException, 
ClusterAuthorizationException, ConcurrentTransactionsException, 
ControllerMovedException, CoordinatorLoadInProgressException, 
CoordinatorNotAvailableException, CorruptRecordException, 
DuplicateSequenceNumberException, GroupAuthorizationException, 
IllegalGenerationException, IllegalSaslStateException, 
InconsistentGroupProtocolException, InvalidCommitOffsetSizeException, 
InvalidConfigurationException, InvalidFetchSizeException, 
InvalidGroupIdException, InvalidPartitionsException, 
InvalidPidMappingException, InvalidReplicaAssignmentException, 
InvalidReplicationFactorException, InvalidRequestException, 
InvalidRequiredAcksException, InvalidSessionTimeoutException, 
InvalidTimestampException, InvalidTopicException, 
InvalidTxnStateException, InvalidTxnTimeoutException, 
LeaderNotAvailableException, NetworkException, NotControllerException, 
NotCoordinatorException, NotEnoughReplicasAfterAppendException, 
NotEnoughReplicasException, NotLeaderForPartitionException, 
OffsetMetadataTooLarge, OffsetOutOfRangeException, 
OperationNotAttemptedException, OutOfOrderSequenceException, 
PolicyViolationException, ProducerFencedException, 
RebalanceInProgressException, RecordBatchTooLargeException, 
RecordTooLargeException, ReplicaNotAvailableException, 
SecurityDisabledException, TimeoutException, TopicAuthorizationException, 
TopicExistsException, TransactionCoordinatorFencedException, 
TransactionalIdAuthorizationException, UnknownMemberIdException, 
UnknownServerException, UnknownTopicOrPartitionException, 
UnsupportedForMessageFormatException, UnsupportedSaslMechanismException, 
UnsupportedVersionException]. [ClassDataAbstractionCoupling]
[ant:checkstyle] [ERROR] 
C:\Users\User\Downloads\kafka-0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1: 
Class Fan-Out Complexity is 60 (max allowed is 40). 
[ClassFanOutComplexity]
[ant:checkstyle] [ERROR] 
C:\Users\User\Downloads\kafka-0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\requests\AbstractRequest.java:26:1: 
Class Fan-Out Complexity is 43 (max allowed is 40). 
[ClassFanOutComplexity]
[ant:checkstyle] [ERROR] 
C:\Users\User\Downloads\kafka-0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\requests\AbstractResponse.java:26:1: 
Class Fan-Out Complexity is 42 (max allowed is 40). 
[ClassFanOutComplexity]
:clients:checkstyleMain FAILED

FAILURE: Build failed with an exception.

Thanks.
--Vahid



From:   Ismael Juma <is...@juma.me.uk>
To:     dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>, 
kafka-clients <ka...@googlegroups.com>
Date:   06/22/2017 06:16 PM
Subject:        [VOTE] 0.11.0.0 RC2
Sent by:        ismaelj@gmail.com



Hello Kafka users, developers and client-developers,

This is the third candidate for release of Apache Kafka 0.11.0.0.

This is a major version release of Apache Kafka. It includes 32 new KIPs.
See the release notes and release plan (
https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
for more details. A few feature highlights:

* Exactly-once delivery and transactional messaging
* Streams exactly-once semantics
* Admin client with support for topic, ACLs and config management
* Record headers
* Request rate quotas
* Improved resiliency: replication protocol improvement and 
single-threaded
controller
* Richer and more efficient message format

Release notes for the 0.11.0.0 release:
http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html

*** Please download, test and vote by Tuesday, June 27, 6pm PT

Kafka's KEYS file containing PGP keys we use to sign the release:
http://kafka.apache.org/KEYS

* Release artifacts to be voted upon (source and binary):
http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/

* Maven artifacts to be voted upon:
https://repository.apache.org/content/groups/staging/org/apache/kafka/

* Javadoc:
http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/

* Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=8698fa1f41102f1664b05baa4d6953fc9564d91e


* Documentation:
http://kafka.apache.org/0110/documentation.html

* Protocol:
http://kafka.apache.org/0110/protocol.html

* Successful Jenkins builds for the 0.11.0 branch:
Unit/integration tests: 
https://builds.apache.org/job/kafka-0.11.0-jdk7/187/
System tests: pending (will send an update tomorrow)

/**************************************

Thanks,
Ismael





Re: [VOTE] 0.11.0.0 RC2

Posted by Tom Crayford <tc...@heroku.com>.
Hi,

As previously, Heroku has been doing performance testing of 0.11.0.0 RC2.
Now that the ProducerPerformance.java tool supports it, we've even been
doing testing with the new transactional/idempotence features in KIP-98.

We've tested with idempotency and read_committed consumers and note no
notable performance impact there from our perspective. That isn't to say
that there is no impact, just that our testing can't see enough of one to
be obvious.

The new tooling supports `transaction-duration-ms` as a flag, which gives
the number of milliseconds between transaction commits in the producer.
This setting has vast (and expected) impacts on performance from our
perspective.

Tuned down to 1ms, we see throughput around 23x lower than without
transactions enabled (but with idempotency and read_committed consumers).

Tuned to 10ms, we see throughput at about 70% of performance without
transactions enabled

At 100ms and 1000ms transaction duration, there is no notable impact to
throughput with usual operating conditions as far as we can tell. That
isn't to say that there is no impact, just that our testing can't see
enough of one to be obvious.

We have also performed some failure testing with 0.11 transactions enabled,
and notice nothing out of the ordinary there. Note that we have not tested
the correctness of the feature (as in, does it deliver what is promised),
just if the producers or consumers or brokers do anything outwardly strange
through the logs etc.

All in all, this is shaping up to be a great release. We're going to
continue some further testing, but right now are heading towards a +1.

Thanks

Tom Crayford
Heroku Kafka

On Fri, Jun 23, 2017 at 2:36 AM, Ismael Juma <is...@juma.me.uk> wrote:

> A quick note on notable changes since rc1:
>
> 1. A significant performance improvement if transactions are enabled:
> https://github.com/apache/kafka/commit/f239f1f839f8bcbd80cce2a4a8643e
> 15d340be8e
> 2. Fixed a controller regression if many brokers are started
> simultaneously:
> https://github.com/apache/kafka/commit/c0033b0e0b9e56242752c82f15c638
> 8d041914a1
> 3. Fixed a couple of Connect regressions:
> https://github.com/apache/kafka/commit/c029960bf4ae2cd79b22886f4ee519
> c4af0bcc8b
> and
> https://github.com/apache/kafka/commit/1d65f15f2b656b7817eeaf6ee1d36e
> b3e2cf063f
> 4. Fixed an import log cleaner issue:
> https://github.com/apache/kafka/commit/186e3d5efc79ed803f0915d472ace7
> 7cbec88694
>
> Full diff:
> https://github.com/apache/kafka/compare/5b351216621f52a471c21826d0dec3
> ce3187e697...0.11.0.0-rc2
>
> Ismael
>
> On Fri, Jun 23, 2017 at 2:16 AM, Ismael Juma <is...@juma.me.uk> wrote:
>
> > Hello Kafka users, developers and client-developers,
> >
> > This is the third candidate for release of Apache Kafka 0.11.0.0.
> >
> > This is a major version release of Apache Kafka. It includes 32 new KIPs.
> > See the release notes and release plan (https://cwiki.apache.org/
> > confluence/display/KAFKA/Release+Plan+0.11.0.0) for more details. A few
> > feature highlights:
> >
> > * Exactly-once delivery and transactional messaging
> > * Streams exactly-once semantics
> > * Admin client with support for topic, ACLs and config management
> > * Record headers
> > * Request rate quotas
> > * Improved resiliency: replication protocol improvement and
> > single-threaded controller
> > * Richer and more efficient message format
> >
> > Release notes for the 0.11.0.0 release:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > http://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> >
> > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> >
> > * Documentation:
> > http://kafka.apache.org/0110/documentation.html
> >
> > * Protocol:
> > http://kafka.apache.org/0110/protocol.html
> >
> > * Successful Jenkins builds for the 0.11.0 branch:
> > Unit/integration tests: https://builds.apache.org/job/
> > kafka-0.11.0-jdk7/187/
> > System tests: pending (will send an update tomorrow)
> >
> > /**************************************
> >
> > Thanks,
> > Ismael
> >
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Tom Crayford <tc...@heroku.com>.
Hi,

As previously, Heroku has been doing performance testing of 0.11.0.0 RC2.
Now that the ProducerPerformance.java tool supports it, we've even been
doing testing with the new transactional/idempotence features in KIP-98.

We've tested with idempotency and read_committed consumers and note no
notable performance impact there from our perspective. That isn't to say
that there is no impact, just that our testing can't see enough of one to
be obvious.

The new tooling supports `transaction-duration-ms` as a flag, which gives
the number of milliseconds between transaction commits in the producer.
This setting has vast (and expected) impacts on performance from our
perspective.

Tuned down to 1ms, we see throughput around 23x lower than without
transactions enabled (but with idempotency and read_committed consumers).

Tuned to 10ms, we see throughput at about 70% of performance without
transactions enabled

At 100ms and 1000ms transaction duration, there is no notable impact to
throughput with usual operating conditions as far as we can tell. That
isn't to say that there is no impact, just that our testing can't see
enough of one to be obvious.

We have also performed some failure testing with 0.11 transactions enabled,
and notice nothing out of the ordinary there. Note that we have not tested
the correctness of the feature (as in, does it deliver what is promised),
just if the producers or consumers or brokers do anything outwardly strange
through the logs etc.

All in all, this is shaping up to be a great release. We're going to
continue some further testing, but right now are heading towards a +1.

Thanks

Tom Crayford
Heroku Kafka

On Fri, Jun 23, 2017 at 2:36 AM, Ismael Juma <is...@juma.me.uk> wrote:

> A quick note on notable changes since rc1:
>
> 1. A significant performance improvement if transactions are enabled:
> https://github.com/apache/kafka/commit/f239f1f839f8bcbd80cce2a4a8643e
> 15d340be8e
> 2. Fixed a controller regression if many brokers are started
> simultaneously:
> https://github.com/apache/kafka/commit/c0033b0e0b9e56242752c82f15c638
> 8d041914a1
> 3. Fixed a couple of Connect regressions:
> https://github.com/apache/kafka/commit/c029960bf4ae2cd79b22886f4ee519
> c4af0bcc8b
> and
> https://github.com/apache/kafka/commit/1d65f15f2b656b7817eeaf6ee1d36e
> b3e2cf063f
> 4. Fixed an import log cleaner issue:
> https://github.com/apache/kafka/commit/186e3d5efc79ed803f0915d472ace7
> 7cbec88694
>
> Full diff:
> https://github.com/apache/kafka/compare/5b351216621f52a471c21826d0dec3
> ce3187e697...0.11.0.0-rc2
>
> Ismael
>
> On Fri, Jun 23, 2017 at 2:16 AM, Ismael Juma <is...@juma.me.uk> wrote:
>
> > Hello Kafka users, developers and client-developers,
> >
> > This is the third candidate for release of Apache Kafka 0.11.0.0.
> >
> > This is a major version release of Apache Kafka. It includes 32 new KIPs.
> > See the release notes and release plan (https://cwiki.apache.org/
> > confluence/display/KAFKA/Release+Plan+0.11.0.0) for more details. A few
> > feature highlights:
> >
> > * Exactly-once delivery and transactional messaging
> > * Streams exactly-once semantics
> > * Admin client with support for topic, ACLs and config management
> > * Record headers
> > * Request rate quotas
> > * Improved resiliency: replication protocol improvement and
> > single-threaded controller
> > * Richer and more efficient message format
> >
> > Release notes for the 0.11.0.0 release:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Tuesday, June 27, 6pm PT
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > http://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
> >
> > * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> > https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> > 8698fa1f41102f1664b05baa4d6953fc9564d91e
> >
> > * Documentation:
> > http://kafka.apache.org/0110/documentation.html
> >
> > * Protocol:
> > http://kafka.apache.org/0110/protocol.html
> >
> > * Successful Jenkins builds for the 0.11.0 branch:
> > Unit/integration tests: https://builds.apache.org/job/
> > kafka-0.11.0-jdk7/187/
> > System tests: pending (will send an update tomorrow)
> >
> > /**************************************
> >
> > Thanks,
> > Ismael
> >
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@juma.me.uk>.
A quick note on notable changes since rc1:

1. A significant performance improvement if transactions are enabled:
https://github.com/apache/kafka/commit/f239f1f839f8bcbd80cce2a4a8643e15d340be8e
2. Fixed a controller regression if many brokers are started
simultaneously:
https://github.com/apache/kafka/commit/c0033b0e0b9e56242752c82f15c6388d041914a1
3. Fixed a couple of Connect regressions:
https://github.com/apache/kafka/commit/c029960bf4ae2cd79b22886f4ee519c4af0bcc8b
and
https://github.com/apache/kafka/commit/1d65f15f2b656b7817eeaf6ee1d36eb3e2cf063f
4. Fixed an import log cleaner issue:
https://github.com/apache/kafka/commit/186e3d5efc79ed803f0915d472ace77cbec88694

Full diff:
https://github.com/apache/kafka/compare/5b351216621f52a471c21826d0dec3ce3187e697...0.11.0.0-rc2

Ismael

On Fri, Jun 23, 2017 at 2:16 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Hello Kafka users, developers and client-developers,
>
> This is the third candidate for release of Apache Kafka 0.11.0.0.
>
> This is a major version release of Apache Kafka. It includes 32 new KIPs.
> See the release notes and release plan (https://cwiki.apache.org/
> confluence/display/KAFKA/Release+Plan+0.11.0.0) for more details. A few
> feature highlights:
>
> * Exactly-once delivery and transactional messaging
> * Streams exactly-once semantics
> * Admin client with support for topic, ACLs and config management
> * Record headers
> * Request rate quotas
> * Improved resiliency: replication protocol improvement and
> single-threaded controller
> * Richer and more efficient message format
>
> Release notes for the 0.11.0.0 release:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>
> *** Please download, test and vote by Tuesday, June 27, 6pm PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> http://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>
> * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> 8698fa1f41102f1664b05baa4d6953fc9564d91e
>
> * Documentation:
> http://kafka.apache.org/0110/documentation.html
>
> * Protocol:
> http://kafka.apache.org/0110/protocol.html
>
> * Successful Jenkins builds for the 0.11.0 branch:
> Unit/integration tests: https://builds.apache.org/job/
> kafka-0.11.0-jdk7/187/
> System tests: pending (will send an update tomorrow)
>
> /**************************************
>
> Thanks,
> Ismael
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@juma.me.uk>.
A quick note on notable changes since rc1:

1. A significant performance improvement if transactions are enabled:
https://github.com/apache/kafka/commit/f239f1f839f8bcbd80cce2a4a8643e15d340be8e
2. Fixed a controller regression if many brokers are started
simultaneously:
https://github.com/apache/kafka/commit/c0033b0e0b9e56242752c82f15c6388d041914a1
3. Fixed a couple of Connect regressions:
https://github.com/apache/kafka/commit/c029960bf4ae2cd79b22886f4ee519c4af0bcc8b
and
https://github.com/apache/kafka/commit/1d65f15f2b656b7817eeaf6ee1d36eb3e2cf063f
4. Fixed an import log cleaner issue:
https://github.com/apache/kafka/commit/186e3d5efc79ed803f0915d472ace77cbec88694

Full diff:
https://github.com/apache/kafka/compare/5b351216621f52a471c21826d0dec3ce3187e697...0.11.0.0-rc2

Ismael

On Fri, Jun 23, 2017 at 2:16 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Hello Kafka users, developers and client-developers,
>
> This is the third candidate for release of Apache Kafka 0.11.0.0.
>
> This is a major version release of Apache Kafka. It includes 32 new KIPs.
> See the release notes and release plan (https://cwiki.apache.org/
> confluence/display/KAFKA/Release+Plan+0.11.0.0) for more details. A few
> feature highlights:
>
> * Exactly-once delivery and transactional messaging
> * Streams exactly-once semantics
> * Admin client with support for topic, ACLs and config management
> * Record headers
> * Request rate quotas
> * Improved resiliency: replication protocol improvement and
> single-threaded controller
> * Richer and more efficient message format
>
> Release notes for the 0.11.0.0 release:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>
> *** Please download, test and vote by Tuesday, June 27, 6pm PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> http://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>
> * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> 8698fa1f41102f1664b05baa4d6953fc9564d91e
>
> * Documentation:
> http://kafka.apache.org/0110/documentation.html
>
> * Protocol:
> http://kafka.apache.org/0110/protocol.html
>
> * Successful Jenkins builds for the 0.11.0 branch:
> Unit/integration tests: https://builds.apache.org/job/
> kafka-0.11.0-jdk7/187/
> System tests: pending (will send an update tomorrow)
>
> /**************************************
>
> Thanks,
> Ismael
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Vahid S Hashemian <va...@us.ibm.com>.
Hi Ismael,

Not sure if my response on RC1 was lost or this issue is not a 
show-stopper:

I checked again and with RC2, tests still fail in my Windown 64 bit 
environment.

:clients:checkstyleMain
[ant:checkstyle] [ERROR] 
C:\Users\User\Downloads\kafka-0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1: 
Class Data Abstraction Coupling is 57 (max allowed is 20) classes 
[ApiExceptionBuilder, BrokerNotAvailableException, 
ClusterAuthorizationException, ConcurrentTransactionsException, 
ControllerMovedException, CoordinatorLoadInProgressException, 
CoordinatorNotAvailableException, CorruptRecordException, 
DuplicateSequenceNumberException, GroupAuthorizationException, 
IllegalGenerationException, IllegalSaslStateException, 
InconsistentGroupProtocolException, InvalidCommitOffsetSizeException, 
InvalidConfigurationException, InvalidFetchSizeException, 
InvalidGroupIdException, InvalidPartitionsException, 
InvalidPidMappingException, InvalidReplicaAssignmentException, 
InvalidReplicationFactorException, InvalidRequestException, 
InvalidRequiredAcksException, InvalidSessionTimeoutException, 
InvalidTimestampException, InvalidTopicException, 
InvalidTxnStateException, InvalidTxnTimeoutException, 
LeaderNotAvailableException, NetworkException, NotControllerException, 
NotCoordinatorException, NotEnoughReplicasAfterAppendException, 
NotEnoughReplicasException, NotLeaderForPartitionException, 
OffsetMetadataTooLarge, OffsetOutOfRangeException, 
OperationNotAttemptedException, OutOfOrderSequenceException, 
PolicyViolationException, ProducerFencedException, 
RebalanceInProgressException, RecordBatchTooLargeException, 
RecordTooLargeException, ReplicaNotAvailableException, 
SecurityDisabledException, TimeoutException, TopicAuthorizationException, 
TopicExistsException, TransactionCoordinatorFencedException, 
TransactionalIdAuthorizationException, UnknownMemberIdException, 
UnknownServerException, UnknownTopicOrPartitionException, 
UnsupportedForMessageFormatException, UnsupportedSaslMechanismException, 
UnsupportedVersionException]. [ClassDataAbstractionCoupling]
[ant:checkstyle] [ERROR] 
C:\Users\User\Downloads\kafka-0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\protocol\Errors.java:89:1: 
Class Fan-Out Complexity is 60 (max allowed is 40). 
[ClassFanOutComplexity]
[ant:checkstyle] [ERROR] 
C:\Users\User\Downloads\kafka-0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\requests\AbstractRequest.java:26:1: 
Class Fan-Out Complexity is 43 (max allowed is 40). 
[ClassFanOutComplexity]
[ant:checkstyle] [ERROR] 
C:\Users\User\Downloads\kafka-0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\requests\AbstractResponse.java:26:1: 
Class Fan-Out Complexity is 42 (max allowed is 40). 
[ClassFanOutComplexity]
:clients:checkstyleMain FAILED

FAILURE: Build failed with an exception.

Thanks.
--Vahid



From:   Ismael Juma <is...@juma.me.uk>
To:     dev@kafka.apache.org, Kafka Users <us...@kafka.apache.org>, 
kafka-clients <ka...@googlegroups.com>
Date:   06/22/2017 06:16 PM
Subject:        [VOTE] 0.11.0.0 RC2
Sent by:        ismaelj@gmail.com



Hello Kafka users, developers and client-developers,

This is the third candidate for release of Apache Kafka 0.11.0.0.

This is a major version release of Apache Kafka. It includes 32 new KIPs.
See the release notes and release plan (
https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0)
for more details. A few feature highlights:

* Exactly-once delivery and transactional messaging
* Streams exactly-once semantics
* Admin client with support for topic, ACLs and config management
* Record headers
* Request rate quotas
* Improved resiliency: replication protocol improvement and 
single-threaded
controller
* Richer and more efficient message format

Release notes for the 0.11.0.0 release:
http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html

*** Please download, test and vote by Tuesday, June 27, 6pm PT

Kafka's KEYS file containing PGP keys we use to sign the release:
http://kafka.apache.org/KEYS

* Release artifacts to be voted upon (source and binary):
http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/

* Maven artifacts to be voted upon:
https://repository.apache.org/content/groups/staging/org/apache/kafka/

* Javadoc:
http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/

* Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=8698fa1f41102f1664b05baa4d6953fc9564d91e


* Documentation:
http://kafka.apache.org/0110/documentation.html

* Protocol:
http://kafka.apache.org/0110/protocol.html

* Successful Jenkins builds for the 0.11.0 branch:
Unit/integration tests: 
https://builds.apache.org/job/kafka-0.11.0-jdk7/187/
System tests: pending (will send an update tomorrow)

/**************************************

Thanks,
Ismael





Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@juma.me.uk>.
This vote passes with 7 +1 votes (3 binding) and no 0 or -1 votes.

+1 votes
PMC Members: Jun, Gwen, Guozhang
Committers: Jason, Ismael
Community: Tom Crayford, Jeff Chao

0 votes: none

-1 votes: none

Thanks to everyone who tested, voted and/or contributed to the release.

I'll continue with the release process and the release announcement will follow
in the next few days.

Ismael

P.S. Guozhang's vote was in a separate thread due to some delivery issues
when replying to this thread.

On Fri, Jun 23, 2017 at 2:16 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Hello Kafka users, developers and client-developers,
>
> This is the third candidate for release of Apache Kafka 0.11.0.0.
>
> This is a major version release of Apache Kafka. It includes 32 new KIPs.
> See the release notes and release plan (https://cwiki.apache.org/
> confluence/display/KAFKA/Release+Plan+0.11.0.0) for more details. A few
> feature highlights:
>
> * Exactly-once delivery and transactional messaging
> * Streams exactly-once semantics
> * Admin client with support for topic, ACLs and config management
> * Record headers
> * Request rate quotas
> * Improved resiliency: replication protocol improvement and
> single-threaded controller
> * Richer and more efficient message format
>
> Release notes for the 0.11.0.0 release:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>
> *** Please download, test and vote by Tuesday, June 27, 6pm PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> http://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>
> * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> 8698fa1f41102f1664b05baa4d6953fc9564d91e
>
> * Documentation:
> http://kafka.apache.org/0110/documentation.html
>
> * Protocol:
> http://kafka.apache.org/0110/protocol.html
>
> * Successful Jenkins builds for the 0.11.0 branch:
> Unit/integration tests: https://builds.apache.org/job/
> kafka-0.11.0-jdk7/187/
> System tests: pending (will send an update tomorrow)
>
> /**************************************
>
> Thanks,
> Ismael
>

Re: [VOTE] 0.11.0.0 RC2

Posted by Ismael Juma <is...@juma.me.uk>.
This vote passes with 7 +1 votes (3 binding) and no 0 or -1 votes.

+1 votes
PMC Members: Jun, Gwen, Guozhang
Committers: Jason, Ismael
Community: Tom Crayford, Jeff Chao

0 votes: none

-1 votes: none

Thanks to everyone who tested, voted and/or contributed to the release.

I'll continue with the release process and the release announcement will follow
in the next few days.

Ismael

P.S. Guozhang's vote was in a separate thread due to some delivery issues
when replying to this thread.

On Fri, Jun 23, 2017 at 2:16 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Hello Kafka users, developers and client-developers,
>
> This is the third candidate for release of Apache Kafka 0.11.0.0.
>
> This is a major version release of Apache Kafka. It includes 32 new KIPs.
> See the release notes and release plan (https://cwiki.apache.org/
> confluence/display/KAFKA/Release+Plan+0.11.0.0) for more details. A few
> feature highlights:
>
> * Exactly-once delivery and transactional messaging
> * Streams exactly-once semantics
> * Admin client with support for topic, ACLs and config management
> * Record headers
> * Request rate quotas
> * Improved resiliency: replication protocol improvement and
> single-threaded controller
> * Richer and more efficient message format
>
> Release notes for the 0.11.0.0 release:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/RELEASE_NOTES.html
>
> *** Please download, test and vote by Tuesday, June 27, 6pm PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> http://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> http://home.apache.org/~ijuma/kafka-0.11.0.0-rc2/javadoc/
>
> * Tag to be voted upon (off 0.11.0 branch) is the 0.11.0.0 tag:
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=
> 8698fa1f41102f1664b05baa4d6953fc9564d91e
>
> * Documentation:
> http://kafka.apache.org/0110/documentation.html
>
> * Protocol:
> http://kafka.apache.org/0110/protocol.html
>
> * Successful Jenkins builds for the 0.11.0 branch:
> Unit/integration tests: https://builds.apache.org/job/
> kafka-0.11.0-jdk7/187/
> System tests: pending (will send an update tomorrow)
>
> /**************************************
>
> Thanks,
> Ismael
>