Re: [VOTE] 2.4.0 RC0

2019-11-19 Thread Manikumar
Hi All,

Thanks you for testing RC0. The following blocker issues were fixed after
RC0.

https://issues.apache.org/jira/browse/KAFKA-9196
https://github.com/apache/kafka/pull/7691

I am canceling RC0 VOTE and will create new RC soon.

Thanks,
Manikumar

On Thu, Nov 14, 2019 at 11:51 PM Manikumar 
wrote:

> Hello Kafka users, developers and client-developers,
>
> This is the first candidate for release of Apache Kafka 2.4.0.
> There is work in progress for couple blockers PRs. I am publishing RC0 to
> avoid further delays in testing the release.
>
> This release includes many new features, including:
> - Allow consumers to fetch from closest replica
> - Support for incremental cooperative rebalancing to the consumer
> rebalance protocol
> - MirrorMaker 2.0 (MM2), a new multi-cluster, cross-datacenter replication
> engine
> - New Java authorizer Interface
> - Support for  non-key joining in KTable
> - Administrative API for replica reassignment
> - Sticky partitioner
> - Return topic metadata and configs in CreateTopics response
> - Securing Internal connect REST endpoints
> - API to delete consumer offsets and expose it via the AdminClient.
>
> Release notes for the 2.4.0 release:
> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/RELEASE_NOTES.html
>
> *** Please download, test  by  Thursday, November 20, 9am PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> https://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/javadoc/
>
> * Tag to be voted upon (off 2.4 branch) is the 2.4.0 tag:
> https://github.com/apache/kafka/releases/tag/2.4.0-rc0
>
> * Documentation:
> https://kafka.apache.org/24/documentation.html
>
> * Protocol:
> https://kafka.apache.org/24/protocol.html
>
> Thanks,
> Manikumar
>


Re: [kafka-clients] [VOTE] 2.4.0 RC0

2019-11-19 Thread Eric Lalonde
Hi all,

Also seeing repeated failures of kafka.api.SaslPlainPlaintextConsumerTest > 
testCoordinatorFailover:

kafka.api.SaslPlainPlaintextConsumerTest > testCoordinatorFailover FAILED
java.lang.AssertionError: expected: but 
was:


Given the nature of the error, it may be a timing issue in the test itself, but 
it is repeatable, so I thought I’d raise it. 

To reproduce:

$ ./gradlew integrationTest



> On Nov 18, 2019, at 11:02 AM, Eric Lalonde  wrote:
> 
> This test has been failing when executed from the command line. I have not 
> run this test in an IDE. 
> 
>> On Nov 18, 2019, at 6:16 AM, Bruno Cadonna > > wrote:
>> 
>> Hi,
>> 
>> ClientMetricsTest.shouldAddCommitIdMetric should only fail if executed
>> from an IDE. The test fails because the test expects a file on the
>> class path which is not there when the test is executed from the IDE,
>> but is there when the test is executed from gradle. I will try to fix
>> the test so that it can also be executed from the IDE.
>> 
>> Best,
>> Bruno
>> 
>> On Mon, Nov 18, 2019 at 6:51 AM Vahid Hashemian
>> mailto:vahid.hashem...@gmail.com>> wrote:
>>> 
>>> Thanks Manikumar for managing this release. Looking forward to it.
>>> 
>>> I built binary from the source and was able to successfully run the 
>>> quickstarts.
>>> 
>>> However, this streams unit test also fails for me constantly:
>>> 
>>> ClientMetricsTest. shouldAddCommitIdMetric
>>> 
>>> java.lang.AssertionError:
>>>  Unexpected method call 
>>> StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The version 
>>> control commit ID of the Kafka Streams client", INFO, "unknown"):
>>>StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The 
>>> version control commit ID of the Kafka Streams client", INFO, 
>>> and(not("unknown"), notNull())): expected: 1, actual: 0
>>> at 
>>> org.easymock.internal.MockInvocationHandler.invoke(MockInvocationHandler.java:44)
>>> at 
>>> org.easymock.internal.ObjectMethodsFilter.invoke(ObjectMethodsFilter.java:101)
>>> at 
>>> org.easymock.internal.ClassProxyFactory$MockMethodInterceptor.intercept(ClassProxyFactory.java:97)
>>>...
>>> 
>>> Thanks,
>>> --Vahid
>>> 
>>> On Thu, Nov 14, 2019 at 10:21 AM Manikumar >> > wrote:
 
 Hello Kafka users, developers and client-developers,
 
 This is the first candidate for release of Apache Kafka 2.4.0.
 There is work in progress for couple blockers PRs. I am publishing RC0 to 
 avoid further delays in testing the release.
 
 This release includes many new features, including:
 - Allow consumers to fetch from closest replica
 - Support for incremental cooperative rebalancing to the consumer 
 rebalance protocol
 - MirrorMaker 2.0 (MM2), a new multi-cluster, cross-datacenter replication 
 engine
 - New Java authorizer Interface
 - Support for  non-key joining in KTable
 - Administrative API for replica reassignment
 - Sticky partitioner
 - Return topic metadata and configs in CreateTopics response
 - Securing Internal connect REST endpoints
 - API to delete consumer offsets and expose it via the AdminClient.
 
 Release notes for the 2.4.0 release:
 https://home.apache.org/~manikumar/kafka-2.4.0-rc0/RELEASE_NOTES.html 
 
 
 *** Please download, test  by  Thursday, November 20, 9am PT
 
 Kafka's KEYS file containing PGP keys we use to sign the release:
 https://kafka.apache.org/KEYS
 
 * Release artifacts to be voted upon (source and binary):
 https://home.apache.org/~manikumar/kafka-2.4.0-rc0/
 
 * Maven artifacts to be voted upon:
 https://repository.apache.org/content/groups/staging/org/apache/kafka/
 
 * Javadoc:
 https://home.apache.org/~manikumar/kafka-2.4.0-rc0/javadoc/
 
 * Tag to be voted upon (off 2.4 branch) is the 2.4.0 tag:
 https://github.com/apache/kafka/releases/tag/2.4.0-rc0
 
 * Documentation:
 https://kafka.apache.org/24/documentation.html
 
 * Protocol:
 https://kafka.apache.org/24/protocol.html
 
 Thanks,
 Manikumar
 
 --
 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+unsubscr...@googlegroups.com.
 To view this discussion on the web visit 
 https://groups.google.com/d/msgid/kafka-clients/CAMVt_Aw945uqcpisFjZHAR5m8Sidw6hW4ia%2B7%3DjxEfadmBPzcw%40mail.gmail.com.
>>> 
>>> 
>>> 
>>> --
>>> 
>>> Thanks!
>>> --Vahid
>>> 
>>> --
>>> 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+unsubscr...@googlegroups.com 
>>> 

Re: [kafka-clients] [VOTE] 2.4.0 RC0

2019-11-18 Thread Eric Lalonde
This test has been failing when executed from the command line. I have not run 
this test in an IDE. 

> On Nov 18, 2019, at 6:16 AM, Bruno Cadonna  wrote:
> 
> Hi,
> 
> ClientMetricsTest.shouldAddCommitIdMetric should only fail if executed
> from an IDE. The test fails because the test expects a file on the
> class path which is not there when the test is executed from the IDE,
> but is there when the test is executed from gradle. I will try to fix
> the test so that it can also be executed from the IDE.
> 
> Best,
> Bruno
> 
> On Mon, Nov 18, 2019 at 6:51 AM Vahid Hashemian
> mailto:vahid.hashem...@gmail.com>> wrote:
>> 
>> Thanks Manikumar for managing this release. Looking forward to it.
>> 
>> I built binary from the source and was able to successfully run the 
>> quickstarts.
>> 
>> However, this streams unit test also fails for me constantly:
>> 
>> ClientMetricsTest. shouldAddCommitIdMetric
>> 
>> java.lang.AssertionError:
>>  Unexpected method call 
>> StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The version 
>> control commit ID of the Kafka Streams client", INFO, "unknown"):
>>StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The 
>> version control commit ID of the Kafka Streams client", INFO, 
>> and(not("unknown"), notNull())): expected: 1, actual: 0
>> at 
>> org.easymock.internal.MockInvocationHandler.invoke(MockInvocationHandler.java:44)
>> at 
>> org.easymock.internal.ObjectMethodsFilter.invoke(ObjectMethodsFilter.java:101)
>> at 
>> org.easymock.internal.ClassProxyFactory$MockMethodInterceptor.intercept(ClassProxyFactory.java:97)
>>...
>> 
>> Thanks,
>> --Vahid
>> 
>> On Thu, Nov 14, 2019 at 10:21 AM Manikumar  wrote:
>>> 
>>> Hello Kafka users, developers and client-developers,
>>> 
>>> This is the first candidate for release of Apache Kafka 2.4.0.
>>> There is work in progress for couple blockers PRs. I am publishing RC0 to 
>>> avoid further delays in testing the release.
>>> 
>>> This release includes many new features, including:
>>> - Allow consumers to fetch from closest replica
>>> - Support for incremental cooperative rebalancing to the consumer rebalance 
>>> protocol
>>> - MirrorMaker 2.0 (MM2), a new multi-cluster, cross-datacenter replication 
>>> engine
>>> - New Java authorizer Interface
>>> - Support for  non-key joining in KTable
>>> - Administrative API for replica reassignment
>>> - Sticky partitioner
>>> - Return topic metadata and configs in CreateTopics response
>>> - Securing Internal connect REST endpoints
>>> - API to delete consumer offsets and expose it via the AdminClient.
>>> 
>>> Release notes for the 2.4.0 release:
>>> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/RELEASE_NOTES.html
>>> 
>>> *** Please download, test  by  Thursday, November 20, 9am PT
>>> 
>>> Kafka's KEYS file containing PGP keys we use to sign the release:
>>> https://kafka.apache.org/KEYS
>>> 
>>> * Release artifacts to be voted upon (source and binary):
>>> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/
>>> 
>>> * Maven artifacts to be voted upon:
>>> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>>> 
>>> * Javadoc:
>>> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/javadoc/
>>> 
>>> * Tag to be voted upon (off 2.4 branch) is the 2.4.0 tag:
>>> https://github.com/apache/kafka/releases/tag/2.4.0-rc0
>>> 
>>> * Documentation:
>>> https://kafka.apache.org/24/documentation.html
>>> 
>>> * Protocol:
>>> https://kafka.apache.org/24/protocol.html
>>> 
>>> Thanks,
>>> Manikumar
>>> 
>>> --
>>> 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+unsubscr...@googlegroups.com.
>>> To view this discussion on the web visit 
>>> https://groups.google.com/d/msgid/kafka-clients/CAMVt_Aw945uqcpisFjZHAR5m8Sidw6hW4ia%2B7%3DjxEfadmBPzcw%40mail.gmail.com.
>> 
>> 
>> 
>> --
>> 
>> Thanks!
>> --Vahid
>> 
>> --
>> 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+unsubscr...@googlegroups.com 
>> .
>> To view this discussion on the web visit 
>> https://groups.google.com/d/msgid/kafka-clients/CAHR2v2mKJtHG6S9P%3Dmw08SxbWjQCowp8cpZNpzr9acW1EcdegQ%40mail.gmail.com
>>  
>> .



Re: [kafka-clients] [VOTE] 2.4.0 RC0

2019-11-18 Thread Vahid Hashemian
Thanks Bruno,

Just to clarify, I ran the tests from the command line: ./gradlew
streams:test

Regards,
--Vahid

On Mon, Nov 18, 2019 at 6:16 AM Bruno Cadonna  wrote:

> Hi,
>
> ClientMetricsTest.shouldAddCommitIdMetric should only fail if executed
> from an IDE. The test fails because the test expects a file on the
> class path which is not there when the test is executed from the IDE,
> but is there when the test is executed from gradle. I will try to fix
> the test so that it can also be executed from the IDE.
>
> Best,
> Bruno
>
> On Mon, Nov 18, 2019 at 6:51 AM Vahid Hashemian
>  wrote:
> >
> > Thanks Manikumar for managing this release. Looking forward to it.
> >
> > I built binary from the source and was able to successfully run the
> quickstarts.
> >
> > However, this streams unit test also fails for me constantly:
> >
> > ClientMetricsTest. shouldAddCommitIdMetric
> >
> > java.lang.AssertionError:
> >   Unexpected method call
> StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The version
> control commit ID of the Kafka Streams client", INFO, "unknown"):
> > StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The
> version control commit ID of the Kafka Streams client", INFO,
> and(not("unknown"), notNull())): expected: 1, actual: 0
> > at
> org.easymock.internal.MockInvocationHandler.invoke(MockInvocationHandler.java:44)
> > at
> org.easymock.internal.ObjectMethodsFilter.invoke(ObjectMethodsFilter.java:101)
> > at
> org.easymock.internal.ClassProxyFactory$MockMethodInterceptor.intercept(ClassProxyFactory.java:97)
> > ...
> >
> > Thanks,
> > --Vahid
> >
> > On Thu, Nov 14, 2019 at 10:21 AM Manikumar 
> wrote:
> >>
> >> Hello Kafka users, developers and client-developers,
> >>
> >> This is the first candidate for release of Apache Kafka 2.4.0.
> >> There is work in progress for couple blockers PRs. I am publishing RC0
> to avoid further delays in testing the release.
> >>
> >> This release includes many new features, including:
> >> - Allow consumers to fetch from closest replica
> >> - Support for incremental cooperative rebalancing to the consumer
> rebalance protocol
> >> - MirrorMaker 2.0 (MM2), a new multi-cluster, cross-datacenter
> replication engine
> >> - New Java authorizer Interface
> >> - Support for  non-key joining in KTable
> >> - Administrative API for replica reassignment
> >> - Sticky partitioner
> >> - Return topic metadata and configs in CreateTopics response
> >> - Securing Internal connect REST endpoints
> >> - API to delete consumer offsets and expose it via the AdminClient.
> >>
> >> Release notes for the 2.4.0 release:
> >> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/RELEASE_NOTES.html
> >>
> >> *** Please download, test  by  Thursday, November 20, 9am PT
> >>
> >> Kafka's KEYS file containing PGP keys we use to sign the release:
> >> https://kafka.apache.org/KEYS
> >>
> >> * Release artifacts to be voted upon (source and binary):
> >> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/
> >>
> >> * Maven artifacts to be voted upon:
> >> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >>
> >> * Javadoc:
> >> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/javadoc/
> >>
> >> * Tag to be voted upon (off 2.4 branch) is the 2.4.0 tag:
> >> https://github.com/apache/kafka/releases/tag/2.4.0-rc0
> >>
> >> * Documentation:
> >> https://kafka.apache.org/24/documentation.html
> >>
> >> * Protocol:
> >> https://kafka.apache.org/24/protocol.html
> >>
> >> Thanks,
> >> Manikumar
> >>
> >> --
> >> 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+unsubscr...@googlegroups.com.
> >> To view this discussion on the web visit
> https://groups.google.com/d/msgid/kafka-clients/CAMVt_Aw945uqcpisFjZHAR5m8Sidw6hW4ia%2B7%3DjxEfadmBPzcw%40mail.gmail.com
> .
> >
> >
> >
> > --
> >
> > Thanks!
> > --Vahid
> >
> > --
> > 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+unsubscr...@googlegroups.com.
> > To view this discussion on the web visit
> https://groups.google.com/d/msgid/kafka-clients/CAHR2v2mKJtHG6S9P%3Dmw08SxbWjQCowp8cpZNpzr9acW1EcdegQ%40mail.gmail.com
> .
>


-- 

Thanks!
--Vahid


Re: [kafka-clients] [VOTE] 2.4.0 RC0

2019-11-18 Thread Bruno Cadonna
Hi,

ClientMetricsTest.shouldAddCommitIdMetric should only fail if executed
from an IDE. The test fails because the test expects a file on the
class path which is not there when the test is executed from the IDE,
but is there when the test is executed from gradle. I will try to fix
the test so that it can also be executed from the IDE.

Best,
Bruno

On Mon, Nov 18, 2019 at 6:51 AM Vahid Hashemian
 wrote:
>
> Thanks Manikumar for managing this release. Looking forward to it.
>
> I built binary from the source and was able to successfully run the 
> quickstarts.
>
> However, this streams unit test also fails for me constantly:
>
> ClientMetricsTest. shouldAddCommitIdMetric
>
> java.lang.AssertionError:
>   Unexpected method call 
> StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The version 
> control commit ID of the Kafka Streams client", INFO, "unknown"):
> StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The 
> version control commit ID of the Kafka Streams client", INFO, 
> and(not("unknown"), notNull())): expected: 1, actual: 0
> at 
> org.easymock.internal.MockInvocationHandler.invoke(MockInvocationHandler.java:44)
> at 
> org.easymock.internal.ObjectMethodsFilter.invoke(ObjectMethodsFilter.java:101)
> at 
> org.easymock.internal.ClassProxyFactory$MockMethodInterceptor.intercept(ClassProxyFactory.java:97)
> ...
>
> Thanks,
> --Vahid
>
> On Thu, Nov 14, 2019 at 10:21 AM Manikumar  wrote:
>>
>> Hello Kafka users, developers and client-developers,
>>
>> This is the first candidate for release of Apache Kafka 2.4.0.
>> There is work in progress for couple blockers PRs. I am publishing RC0 to 
>> avoid further delays in testing the release.
>>
>> This release includes many new features, including:
>> - Allow consumers to fetch from closest replica
>> - Support for incremental cooperative rebalancing to the consumer rebalance 
>> protocol
>> - MirrorMaker 2.0 (MM2), a new multi-cluster, cross-datacenter replication 
>> engine
>> - New Java authorizer Interface
>> - Support for  non-key joining in KTable
>> - Administrative API for replica reassignment
>> - Sticky partitioner
>> - Return topic metadata and configs in CreateTopics response
>> - Securing Internal connect REST endpoints
>> - API to delete consumer offsets and expose it via the AdminClient.
>>
>> Release notes for the 2.4.0 release:
>> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/RELEASE_NOTES.html
>>
>> *** Please download, test  by  Thursday, November 20, 9am PT
>>
>> Kafka's KEYS file containing PGP keys we use to sign the release:
>> https://kafka.apache.org/KEYS
>>
>> * Release artifacts to be voted upon (source and binary):
>> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/
>>
>> * Maven artifacts to be voted upon:
>> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>>
>> * Javadoc:
>> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/javadoc/
>>
>> * Tag to be voted upon (off 2.4 branch) is the 2.4.0 tag:
>> https://github.com/apache/kafka/releases/tag/2.4.0-rc0
>>
>> * Documentation:
>> https://kafka.apache.org/24/documentation.html
>>
>> * Protocol:
>> https://kafka.apache.org/24/protocol.html
>>
>> Thanks,
>> Manikumar
>>
>> --
>> 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+unsubscr...@googlegroups.com.
>> To view this discussion on the web visit 
>> https://groups.google.com/d/msgid/kafka-clients/CAMVt_Aw945uqcpisFjZHAR5m8Sidw6hW4ia%2B7%3DjxEfadmBPzcw%40mail.gmail.com.
>
>
>
> --
>
> Thanks!
> --Vahid
>
> --
> 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+unsubscr...@googlegroups.com.
> To view this discussion on the web visit 
> https://groups.google.com/d/msgid/kafka-clients/CAHR2v2mKJtHG6S9P%3Dmw08SxbWjQCowp8cpZNpzr9acW1EcdegQ%40mail.gmail.com.


Re: [kafka-clients] [VOTE] 2.4.0 RC0

2019-11-17 Thread Vahid Hashemian
Thanks Manikumar for managing this release. Looking forward to it.

I built binary from the source and was able to successfully run the
quickstarts.

However, this streams unit test also fails for me constantly:

ClientMetricsTest. shouldAddCommitIdMetric

java.lang.AssertionError:
  Unexpected method call
StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The version
control commit ID of the Kafka Streams client", INFO, "unknown"):
StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The
version control commit ID of the Kafka Streams client", INFO,
and(not("unknown"), notNull())): expected: 1, actual: 0
at
org.easymock.internal.MockInvocationHandler.invoke(MockInvocationHandler.java:44)
at
org.easymock.internal.ObjectMethodsFilter.invoke(ObjectMethodsFilter.java:101)
at
org.easymock.internal.ClassProxyFactory$MockMethodInterceptor.intercept(ClassProxyFactory.java:97)
...

Thanks,
--Vahid

On Thu, Nov 14, 2019 at 10:21 AM Manikumar 
wrote:

> Hello Kafka users, developers and client-developers,
>
> This is the first candidate for release of Apache Kafka 2.4.0.
> There is work in progress for couple blockers PRs. I am publishing RC0 to
> avoid further delays in testing the release.
>
> This release includes many new features, including:
> - Allow consumers to fetch from closest replica
> - Support for incremental cooperative rebalancing to the consumer
> rebalance protocol
> - MirrorMaker 2.0 (MM2), a new multi-cluster, cross-datacenter replication
> engine
> - New Java authorizer Interface
> - Support for  non-key joining in KTable
> - Administrative API for replica reassignment
> - Sticky partitioner
> - Return topic metadata and configs in CreateTopics response
> - Securing Internal connect REST endpoints
> - API to delete consumer offsets and expose it via the AdminClient.
>
> Release notes for the 2.4.0 release:
> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/RELEASE_NOTES.html
>
> *** Please download, test  by  Thursday, November 20, 9am PT
>
> Kafka's KEYS file containing PGP keys we use to sign the release:
> https://kafka.apache.org/KEYS
>
> * Release artifacts to be voted upon (source and binary):
> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> https://home.apache.org/~manikumar/kafka-2.4.0-rc0/javadoc/
>
> * Tag to be voted upon (off 2.4 branch) is the 2.4.0 tag:
> https://github.com/apache/kafka/releases/tag/2.4.0-rc0
>
> * Documentation:
> https://kafka.apache.org/24/documentation.html
>
> * Protocol:
> https://kafka.apache.org/24/protocol.html
>
> Thanks,
> Manikumar
>
> --
> 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+unsubscr...@googlegroups.com.
> To view this discussion on the web visit
> https://groups.google.com/d/msgid/kafka-clients/CAMVt_Aw945uqcpisFjZHAR5m8Sidw6hW4ia%2B7%3DjxEfadmBPzcw%40mail.gmail.com
> 
> .
>


-- 

Thanks!
--Vahid


Re: [VOTE] 2.4.0 RC0

2019-11-15 Thread Eric Lalonde
Hello,

- all PGP signatures are good
- All md5, sha1sums and sha512sums pass

2648 unit test pass, 1 failure (ran twice)

ClientsMetricsTest.shouldAddCommitIdMetric Failed:

Java.lang.AssertionError: 
  Unexpected method call 
StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The version 
control commit ID of the Kafka Streams client", INFO, "unknown"):
StreamsMetricsImpl.addClientLevelImmutableMetric("commit-id", "The version 
control commit ID of the Kafka Streams client", INFO, and(not("unknown"), 
notNull())): expected: 1, actual: 0
at 
org.easymock.internal.MockInvocationHandler.invoke(MockInvocationHandler.java:44)
at 
org.easymock.internal.ObjectMethodsFilter.invoke(ObjectMethodsFilter.java:101)
at 
org.easymock.internal.ClassProxyFactory$MockMethodInterceptor.intercept(ClassProxyFactory.java:97)
at 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl$$EnhancerByCGLIB$$43750e14.addClientLevelImmutableMetric()
at 
org.apache.kafka.streams.internals.metrics.ClientMetrics.addCommitIdMetric(ClientMetrics.java:80)
at 
org.apache.kafka.streams.internals.metrics.ClientMetricsTest.setUpAndVerifyMetricOneParam(ClientMetricsTest.java:116)
at 
org.apache.kafka.streams.internals.metrics.ClientMetricsTest.shouldAddCommitIdMetric(ClientMetricsTest.java:65)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:305)
at 
org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:365)
at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
at 
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
at org.junit.runners.ParentRunner$4.run(ParentRunner.java:330)
at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:78)
at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:328)
at org.junit.runners.ParentRunner.access$100(ParentRunner.java:65)
at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:292)
at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:305)
at org.junit.runners.ParentRunner.run(ParentRunner.java:412)
at 
org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
at 
org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
at 
org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
at 
org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
at 
org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
at 
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
at 
org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:33)
at 
org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:94)
at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
at 
org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
at sun.reflect.GeneratedMethodAccessor10.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:36)
at 
org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
at 
org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$D

[VOTE] 2.4.0 RC0

2019-11-14 Thread Manikumar
Hello Kafka users, developers and client-developers,

This is the first candidate for release of Apache Kafka 2.4.0.
There is work in progress for couple blockers PRs. I am publishing RC0 to
avoid further delays in testing the release.

This release includes many new features, including:
- Allow consumers to fetch from closest replica
- Support for incremental cooperative rebalancing to the consumer rebalance
protocol
- MirrorMaker 2.0 (MM2), a new multi-cluster, cross-datacenter replication
engine
- New Java authorizer Interface
- Support for  non-key joining in KTable
- Administrative API for replica reassignment
- Sticky partitioner
- Return topic metadata and configs in CreateTopics response
- Securing Internal connect REST endpoints
- API to delete consumer offsets and expose it via the AdminClient.

Release notes for the 2.4.0 release:
https://home.apache.org/~manikumar/kafka-2.4.0-rc0/RELEASE_NOTES.html

*** Please download, test  by  Thursday, November 20, 9am PT

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

* Release artifacts to be voted upon (source and binary):
https://home.apache.org/~manikumar/kafka-2.4.0-rc0/

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

* Javadoc:
https://home.apache.org/~manikumar/kafka-2.4.0-rc0/javadoc/

* Tag to be voted upon (off 2.4 branch) is the 2.4.0 tag:
https://github.com/apache/kafka/releases/tag/2.4.0-rc0

* Documentation:
https://kafka.apache.org/24/documentation.html

* Protocol:
https://kafka.apache.org/24/protocol.html

Thanks,
Manikumar