That's fair, and nice find with the transaction performance improvement! Once the RC is out, we'll do a final round of performance testing with the new ProducerPerformance changes enabled.
I think it's fair that this shouldn't delay the release. Is there an official stance on what should and shouldn't delay a release documented somewhere? Thanks Tom Crayford Heroku Kafka On Thu, Jun 22, 2017 at 4:45 PM, Ismael Juma <isma...@gmail.com> wrote: > Hi Tom, > > We are going to do another RC to include Apurva's significant performance > improvement when transactions are enabled: > > https://github.com/apache/kafka/commit/f239f1f839f8bcbd80cce2a4a8643e > 15d340be8e > > Given that, we can also include the ProducerPerformance changes that > Apurva did to find and fix the performance issue. > > In my opinion, the ProducerPerformance change alone would not be enough > reason for another RC as users can run the tool from trunk to test older > releases. In any case, this is hypothetical at this point. :) > > And thanks for continuing your testing, it's very much appreciated! > > Ismael > > On Wed, Jun 21, 2017 at 8:03 PM, Tom Crayford <tcrayf...@heroku.com> > wrote: > >> That looks better than mine, nice! I think the tooling matters a lot to >> the usability of the product we're shipping, being able to test out Kafka's >> features on your own hardware/setup is very important to knowing if it can >> work. >> >> On Wed, Jun 21, 2017 at 8:01 PM, Apurva Mehta <apu...@confluent.io> >> wrote: >> >>> Hi Tom, >>> >>> I actually made modifications to the produce performance tool to do real >>> transactions earlier this week as part of our benchmarking (results >>> published here: bit.ly/kafka-eos-perf). I just submitted that patch >>> here: >>> https://github.com/apache/kafka/pull/3400/files >>> >>> I think my version is more complete since it runs the full gamut of APIs: >>> initTransactions, beginTransaction, commitTransaction. Also, it is the >>> version used for our published benchmarks. >>> >>> I am not sure that this tool is a blocker for the release though, since >>> it >>> doesn't really affect the usability of the feature any way. >>> >>> Thanks, >>> Apurva >>> >>> On Wed, Jun 21, 2017 at 11:12 AM, Tom Crayford <tcrayf...@heroku.com> >>> wrote: >>> >>> > Hi there, >>> > >>> > I'm -1 (non-binding) on shipping this RC. >>> > >>> > Heroku has carried on performance testing with 0.11 RC1. We have >>> updated >>> > our test setup to use 0.11.0.0 RC1 client libraries. Without any of the >>> > transactional features enabled, we get slightly better performance than >>> > 0.10.2.1 with 10.2.1 client libraries. >>> > >>> > However, we attempted to run a performance test today with >>> transactions, >>> > idempotence and consumer read_committed enabled, but couldn't, because >>> > enabling transactions requires the producer to call `initTransactions` >>> > before starting to send messages, and the producer performance tool >>> doesn't >>> > allow for that. >>> > >>> > I'm -1 (non-binding) on shipping this RC in this state, because users >>> > expect to be able to use the inbuilt performance testing tools, and >>> > preventing them from testing the impact of the new features using the >>> > inbuilt tools isn't great. I made a PR for this: >>> > https://github.com/apache/kafka/pull/3398 (the change is very small). >>> > Happy >>> > to make a jira as well, if that makes sense. >>> > >>> > Thanks >>> > >>> > Tom Crayford >>> > Heroku Kafka >>> > >>> > On Tue, Jun 20, 2017 at 8:32 PM, Vahid S Hashemian < >>> > vahidhashem...@us.ibm.com> wrote: >>> > >>> > > Hi Ismael, >>> > > >>> > > Thanks for running the release. >>> > > >>> > > Running tests ('gradlew.bat test') on my Windows 64-bit VM results in >>> > > these checkstyle errors: >>> > > >>> > > :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, >>> > > UnsupportedForMessageFormatException, UnsupportedSaslMechanismExcept >>> ion, >>> > > 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 >>> > > >>> > > I wonder if there is an issue with my VM since I don't get similar >>> errors >>> > > on Ubuntu or Mac. >>> > > >>> > > --Vahid >>> > > >>> > > >>> > > >>> > > >>> > > From: Ismael Juma <ism...@juma.me.uk> >>> > > To: d...@kafka.apache.org, Kafka Users <users@kafka.apache.org>, >>> > > kafka-clients <kafka-clie...@googlegroups.com> >>> > > Date: 06/18/2017 03:32 PM >>> > > Subject: [VOTE] 0.11.0.0 RC1 >>> > > Sent by: isma...@gmail.com >>> > > >>> > > >>> > > >>> > > Hello Kafka users, developers and client-developers, >>> > > >>> > > This is the second 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/conf >>> > > luence/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 >>> > > >>> > > A number of issues have been resolved since RC0 and there are no >>> known >>> > > blockers remaining. >>> > > >>> > > Release notes for the 0.11.0.0 release: >>> > > http://home.apache.org/~ijuma/kafka-0.11.0.0-rc1/RELEASE_NOTES.html >>> > > >>> > > *** Please download, test and vote by Thursday, June 22, 9am 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-rc1/ >>> > > >>> > > * 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-rc1/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= >>> > > 4818d4e1cbef1a8e9c027100fef317077fb3fb99 >>> > > >>> > > >>> > > * 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/167/ >>> > > System tests: >>> > > https://jenkins.confluent.io/job/system-test-kafka-0.11.0/16/ >>> > > (all 274 tests passed, the reported failure was not related to the >>> tests) >>> > > >>> > > /************************************** >>> > > >>> > > Thanks, >>> > > Ismael >>> > > >>> > > >>> > > >>> > > >>> > > >>> > >>> >> >> >