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 <
> vahidhashem...@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 <isma...@gmail.com>
> > To:     Vahid S Hashemian <vahidhashem...@us.ibm.com>
> > Cc:     d...@kafka.apache.org, kafka-clients
> > <kafka-clie...@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 <
> > vahidhashem...@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 <ism...@juma.me.uk>
> > > To:        d...@kafka.apache.org, Kafka Users <users@kafka.apache.org>,
> > > kafka-clients <kafka-clie...@googlegroups.com>
> > > Date:        06/22/2017 06:16 PM
> > > Subject:        [VOTE] 0.11.0.0 RC2
> > > Sent by:        isma...@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
> > >
> > >
> > >
> > >
> >
> >
> >
> >
> >
>

Reply via email to