Re: [ANNOUNCE] Apache Kafka 2.4.1

2020-03-12 Thread Guozhang Wang
Thanks Bill for driving this. And Many thanks to all who've contributed to
this release!


Guozhang

On Thu, Mar 12, 2020 at 3:00 PM Matthias J. Sax  wrote:

> Thanks for driving the release Bill!
>
> -Matthias
>
> On 3/12/20 1:22 PM, Bill Bejeck wrote:
> > The Apache Kafka community is pleased to announce the release for Apache
> > Kafka 2.4.1
> >
> > This is a bug fix release and it includes fixes and improvements from 39
> > JIRAs, including a few critical bugs.
> >
> > All of the changes in this release can be found in the release notes:
> > https://www.apache.org/dist/kafka/2.4.1/RELEASE_NOTES.html
> >
> >
> > You can download the source and binary release (Scala 2.11, 2.12, and
> 2.13)
> > from:
> > https://kafka.apache.org/downloads#2.4.1
> >
> >
> ---
> >
> >
> > Apache Kafka is a distributed streaming platform with four core APIs:
> >
> >
> > ** The Producer API allows an application to publish a stream records to
> > one or more Kafka topics.
> >
> > ** The Consumer API allows an application to subscribe to one or more
> > topics and process the stream of records produced to them.
> >
> > ** The Streams API allows an application to act as a stream processor,
> > consuming an input stream from one or more topics and producing an
> > output stream to one or more output topics, effectively transforming the
> > input streams to output streams.
> >
> > ** The Connector API allows building and running reusable producers or
> > consumers that connect Kafka topics to existing applications or data
> > systems. For example, a connector to a relational database might
> > capture every change to a table.
> >
> >
> > With these APIs, Kafka can be used for two broad classes of application:
> >
> > ** Building real-time streaming data pipelines that reliably get data
> > between systems or applications.
> >
> > ** Building real-time streaming applications that transform or react
> > to the streams of data.
> >
> >
> > Apache Kafka is in use at large and small companies worldwide, including
> > Capital One, Goldman Sachs, ING, LinkedIn, Netflix, Pinterest, Rabobank,
> > Target, The New York Times, Uber, Yelp, and Zalando, among others.
> >
> > A big thank you for the following 35 contributors to this release!
> >
> > A. Sophie Blee-Goldman, Alex Kokachev, bill, Bill Bejeck, Boyang Chen,
> > Brian Bushree, Brian Byrne, Bruno Cadonna, Chia-Ping Tsai, Chris Egerton,
> > Colin Patrick McCabe, David Jacot, David Kim, David Mao, Dhruvil Shah,
> > Gunnar Morling, Guozhang Wang, huxi, Ismael Juma, Ivan Yurchenko, Jason
> > Gustafson, John Roesler, Konstantine Karantasis, Lev Zemlyanov, Manikumar
> > Reddy, Matthew Wong, Matthias J. Sax, Michael Gyarmathy, Michael Viamari,
> > Nigel Liang, Rajini Sivaram, Randall Hauch, Tomislav, Vikas Singh, Xin
> Wang
> >
> > We welcome your help and feedback. For more information on how to
> > report problems, and to get involved, visit the project website at
> > https://kafka.apache.org/
> >
> > Thank you!
> >
> >
> > Regards,
> >
> > Bill Bejeck
> >
>
>

-- 
-- Guozhang


Create KIP permission request

2020-03-12 Thread 67
hi there

I have a proposal which may change behavior of kafka-server-stop.sh, this 
script kill all instance on the host without any confirm,  someone may just 
want to kill the single one, so maybe a confirm when find multy instance is 
more safer.
already create a issue(https://issues.apache.org/jira/browse/KAFKA-9709) 
and pull request(https://github.com/apache/kafka/pull/8275), more detail can be 
found there.
a KIP is necessary too, so request for the permission to do so, my wiki 
id(login name) is 67


not native english speedker, forgive me if I have language problems.


best wishes



qiang liu






Re: [kafka-clients] Re: [VOTE] 2.5.0 RC1

2020-03-12 Thread Konstantine Karantasis
Hi David,

after some broader testing with Kafka Connect, we've discovered
https://issues.apache.org/jira/browse/KAFKA-9712 and I'd like to raise this
issue as a potential blocker for 2.5.0.
It seems to be a regression after a recent essential upgrade of the
reflections library. The jira contains all the details, and a PR is under
review.

Let us know what you think.
Konstantine

On Thu, Mar 12, 2020 at 8:21 AM Bill Bejeck  wrote:

> Hi David,
>
>
>1. Scanned the Javadoc, looks good
>2. Downloaded kafka_2.12-2.5.0 and ran the quickstart and streams
>quickstart
>3. Verified the signatures
>
> +1 (non-binding)
>
> Thanks for running the release David!
>
> -Bill
>
> On Tue, Mar 10, 2020 at 4:01 PM David Arthur 
> wrote:
>
> > Thanks for the test failure reports, Tom. Tracking (and fixing) these is
> > important and will make future release managers have an easier time :)
> >
> > -David
> >
> > On Tue, Mar 10, 2020 at 10:16 AM Tom Bentley 
> wrote:
> >
> >> Hi David,
> >>
> >> I verified signatures, built the tagged branch and ran unit and
> >> integration
> >> tests. I found some flaky tests, as follows:
> >>
> >> https://issues.apache.org/jira/browse/KAFKA-9691 (new)
> >> https://issues.apache.org/jira/browse/KAFKA-9692 (new)
> >> https://issues.apache.org/jira/browse/KAFKA-9283 (already reported)
> >>
> >> Many thanks,
> >>
> >> Tom
> >>
> >> On Tue, Mar 10, 2020 at 3:28 AM David Arthur  wrote:
> >>
> >> > Hello Kafka users, developers and client-developers,
> >> >
> >> > This is the second candidate for release of Apache Kafka 2.5.0. The
> >> first
> >> > release candidate included an erroneous NOTICE file, so another RC was
> >> > needed to fix that.
> >> >
> >> > This is a major release of Kafka which includes many new features,
> >> > improvements, and bug fixes including:
> >> >
> >> > * TLS 1.3 support (1.2 is now the default)
> >> > * Co-groups for Kafka Streams
> >> > * Incremental rebalance for Kafka Consumer
> >> > * New metrics for better operational insight
> >> > * Upgrade Zookeeper to 3.5.7
> >> > * Deprecate support for Scala 2.11
> >> >
> >> > Release notes for the 2.5.0 release:
> >> >
> https://home.apache.org/~davidarthur/kafka-2.5.0-rc1/RELEASE_NOTES.html
> >> >
> >> > *** Please download, test and vote by Monday, March 16th 2020 5pm 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/~davidarthur/kafka-2.5.0-rc1/
> >> >
> >> > * Maven artifacts to be voted upon:
> >> >
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >> >
> >> > * Javadoc:
> >> > https://home.apache.org/~davidarthur/kafka-2.5.0-rc1/javadoc/
> >> >
> >> > * Tag to be voted upon (off 2.5 branch) is the 2.5.0 tag:
> >> > https://github.com/apache/kafka/releases/tag/2.5.0-rc1
> >> >
> >> > * Documentation:
> >> > https://kafka.apache.org/25/documentation.html
> >> >
> >> > * Protocol:
> >> > https://kafka.apache.org/25/protocol.html
> >> >
> >> > * Links to successful Jenkins builds for the 2.5 branch to follow
> >> >
> >> > Thanks,
> >> > David Arthur
> >> >
> >>
> >
> >
> > --
> > -David
> >
> > --
> > 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/CAGCQu3rbtJgYBkmndKEj3DT2mUK9PfiuQP%3DiUa%2B1gZ%3D0DVVORA%40mail.gmail.com
> > <
> https://groups.google.com/d/msgid/kafka-clients/CAGCQu3rbtJgYBkmndKEj3DT2mUK9PfiuQP%3DiUa%2B1gZ%3D0DVVORA%40mail.gmail.com?utm_medium=email&utm_source=footer
> >
> > .
> >
>


[jira] [Created] (KAFKA-9715) TransactionStateManager: Eliminate unused reference to interBrokerProtocolVersion

2020-03-12 Thread Kowshik Prakasam (Jira)
Kowshik Prakasam created KAFKA-9715:
---

 Summary: TransactionStateManager: Eliminate unused reference to 
interBrokerProtocolVersion
 Key: KAFKA-9715
 URL: https://issues.apache.org/jira/browse/KAFKA-9715
 Project: Kafka
  Issue Type: Improvement
Reporter: Kowshik Prakasam


In TransactionStateManager, the attribute interBrokerProtocolVersion is unused. 
It can therefore be eliminated from the code.

 

[https://github.com/apache/kafka/blob/07db26c20fcbccbf758591607864f7fd4bd8975f/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala#L78]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Build failed in Jenkins: kafka-trunk-jdk11 #1232

2020-03-12 Thread Apache Jenkins Server
See 


Changes:

[github] KAFKA-9659: Add more log4j when updating static member mappings (#8269)


--
[...truncated 5.89 MB...]
org.apache.kafka.streams.internals.WindowStoreFacadeTest > shouldForwardFlush 
PASSED

org.apache.kafka.streams.internals.WindowStoreFacadeTest > shouldForwardInit 
STARTED

org.apache.kafka.streams.internals.WindowStoreFacadeTest > shouldForwardInit 
PASSED

org.apache.kafka.streams.TestTopicsTest > testNonUsedOutputTopic STARTED

org.apache.kafka.streams.TestTopicsTest > testNonUsedOutputTopic PASSED

org.apache.kafka.streams.TestTopicsTest > testEmptyTopic STARTED

org.apache.kafka.streams.TestTopicsTest > testEmptyTopic PASSED

org.apache.kafka.streams.TestTopicsTest > testStartTimestamp STARTED

org.apache.kafka.streams.TestTopicsTest > testStartTimestamp PASSED

org.apache.kafka.streams.TestTopicsTest > testNegativeAdvance STARTED

org.apache.kafka.streams.TestTopicsTest > testNegativeAdvance PASSED

org.apache.kafka.streams.TestTopicsTest > shouldNotAllowToCreateWithNullDriver 
STARTED

org.apache.kafka.streams.TestTopicsTest > shouldNotAllowToCreateWithNullDriver 
PASSED

org.apache.kafka.streams.TestTopicsTest > testDuration STARTED

org.apache.kafka.streams.TestTopicsTest > testDuration PASSED

org.apache.kafka.streams.TestTopicsTest > testOutputToString STARTED

org.apache.kafka.streams.TestTopicsTest > testOutputToString PASSED

org.apache.kafka.streams.TestTopicsTest > testValue STARTED

org.apache.kafka.streams.TestTopicsTest > testValue PASSED

org.apache.kafka.streams.TestTopicsTest > testTimestampAutoAdvance STARTED

org.apache.kafka.streams.TestTopicsTest > testTimestampAutoAdvance PASSED

org.apache.kafka.streams.TestTopicsTest > testOutputWrongSerde STARTED

org.apache.kafka.streams.TestTopicsTest > testOutputWrongSerde PASSED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateOutputTopicWithNullTopicName STARTED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateOutputTopicWithNullTopicName PASSED

org.apache.kafka.streams.TestTopicsTest > testWrongSerde STARTED

org.apache.kafka.streams.TestTopicsTest > testWrongSerde PASSED

org.apache.kafka.streams.TestTopicsTest > testKeyValuesToMapWithNull STARTED

org.apache.kafka.streams.TestTopicsTest > testKeyValuesToMapWithNull PASSED

org.apache.kafka.streams.TestTopicsTest > testNonExistingOutputTopic STARTED

org.apache.kafka.streams.TestTopicsTest > testNonExistingOutputTopic PASSED

org.apache.kafka.streams.TestTopicsTest > testMultipleTopics STARTED

org.apache.kafka.streams.TestTopicsTest > testMultipleTopics PASSED

org.apache.kafka.streams.TestTopicsTest > testKeyValueList STARTED

org.apache.kafka.streams.TestTopicsTest > testKeyValueList PASSED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateOutputWithNullDriver STARTED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateOutputWithNullDriver PASSED

org.apache.kafka.streams.TestTopicsTest > testValueList STARTED

org.apache.kafka.streams.TestTopicsTest > testValueList PASSED

org.apache.kafka.streams.TestTopicsTest > testRecordList STARTED

org.apache.kafka.streams.TestTopicsTest > testRecordList PASSED

org.apache.kafka.streams.TestTopicsTest > testNonExistingInputTopic STARTED

org.apache.kafka.streams.TestTopicsTest > testNonExistingInputTopic PASSED

org.apache.kafka.streams.TestTopicsTest > testKeyValuesToMap STARTED

org.apache.kafka.streams.TestTopicsTest > testKeyValuesToMap PASSED

org.apache.kafka.streams.TestTopicsTest > testRecordsToList STARTED

org.apache.kafka.streams.TestTopicsTest > testRecordsToList PASSED

org.apache.kafka.streams.TestTopicsTest > testKeyValueListDuration STARTED

org.apache.kafka.streams.TestTopicsTest > testKeyValueListDuration PASSED

org.apache.kafka.streams.TestTopicsTest > testInputToString STARTED

org.apache.kafka.streams.TestTopicsTest > testInputToString PASSED

org.apache.kafka.streams.TestTopicsTest > testTimestamp STARTED

org.apache.kafka.streams.TestTopicsTest > testTimestamp PASSED

org.apache.kafka.streams.TestTopicsTest > testWithHeaders STARTED

org.apache.kafka.streams.TestTopicsTest > testWithHeaders PASSED

org.apache.kafka.streams.TestTopicsTest > testKeyValue STARTED

org.apache.kafka.streams.TestTopicsTest > testKeyValue PASSED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateTopicWithNullTopicName STARTED

org.apache.kafka.streams.TestTopicsTest > 
shouldNotAllowToCreateTopicWithNullTopicName PASSED

> Task :streams:upgrade-system-tests-0100:compileJava NO-SOURCE
> Task :streams:upgrade-system-tests-0100:processResources NO-SOURCE
> Task :streams:upgrade-system-tests-0100:classes UP-TO-DATE
> Task :streams:upgrade-system-tests-0100:checkstyleMain NO-SOURCE
> Task :streams:upgrade-system-tests-0100:compileTestJava
> Task :streams:upgrade-system-tests-0100:processTestResources NO-SOURCE
> Task :st

[jira] [Created] (KAFKA-9714) Flaky Test SslTransportLayerTest#testTLSDefaults

2020-03-12 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-9714:
--

 Summary: Flaky Test SslTransportLayerTest#testTLSDefaults
 Key: KAFKA-9714
 URL: https://issues.apache.org/jira/browse/KAFKA-9714
 Project: Kafka
  Issue Type: Bug
  Components: core, security, unit tests
Reporter: Matthias J. Sax


[https://builds.apache.org/job/kafka-pr-jdk11-scala2.13/5145/testReport/junit/org.apache.kafka.common.network/SslTransportLayerTest/testTLSDefaults_tlsProtocol_TLSv1_2_/]
{quote}java.lang.AssertionError: Metric not updated failed-authentication-total 
expected:<0.0> but was:<1.0> expected:<0.0> but was:<1.0> at 
org.junit.Assert.fail(Assert.java:89) at 
org.junit.Assert.failNotEquals(Assert.java:835) at 
org.junit.Assert.assertEquals(Assert.java:555) at 
org.apache.kafka.common.network.NioEchoServer.waitForMetrics(NioEchoServer.java:194)
 at 
org.apache.kafka.common.network.NioEchoServer.verifyAuthenticationMetrics(NioEchoServer.java:156)
 at 
org.apache.kafka.common.network.SslTransportLayerTest.testTLSDefaults(SslTransportLayerTest.java:571){quote}
STDOUT
{quote}[2020-03-12 17:03:44,617] ERROR Modification time of key store could not 
be obtained: some.truststore.path 
(org.apache.kafka.common.security.ssl.SslEngineBuilder:300) 
java.nio.file.NoSuchFileException: some.truststore.path at 
java.base/sun.nio.fs.UnixException.translateToIOException(UnixException.java:92)
 at 
java.base/sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:111) 
at 
java.base/sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:116) 
at 
java.base/sun.nio.fs.UnixFileAttributeViews$Basic.readAttributes(UnixFileAttributeViews.java:55)
 at 
java.base/sun.nio.fs.UnixFileSystemProvider.readAttributes(UnixFileSystemProvider.java:149)
 at 
java.base/sun.nio.fs.LinuxFileSystemProvider.readAttributes(LinuxFileSystemProvider.java:99)
 at java.base/java.nio.file.Files.readAttributes(Files.java:1763) at 
java.base/java.nio.file.Files.getLastModifiedTime(Files.java:2314) at 
org.apache.kafka.common.security.ssl.SslEngineBuilder$SecurityStore.lastModifiedMs(SslEngineBuilder.java:298)
 at 
org.apache.kafka.common.security.ssl.SslEngineBuilder$SecurityStore.(SslEngineBuilder.java:275)
 at 
org.apache.kafka.common.security.ssl.SslEngineBuilder.createTruststore(SslEngineBuilder.java:182)
 at 
org.apache.kafka.common.security.ssl.SslEngineBuilder.(SslEngineBuilder.java:100)
 at 
org.apache.kafka.common.security.ssl.SslFactory.createNewSslEngineBuilder(SslFactory.java:140)
 at 
org.apache.kafka.common.security.ssl.SslFactory.validateReconfiguration(SslFactory.java:114)
 at 
org.apache.kafka.common.network.SslChannelBuilder.validateReconfiguration(SslChannelBuilder.java:85)
 at 
org.apache.kafka.common.network.SslTransportLayerTest.verifyInvalidReconfigure(SslTransportLayerTest.java:1123)
 at 
org.apache.kafka.common.network.SslTransportLayerTest.testServerTruststoreDynamicUpdate(SslTransportLayerTest.java:1113){quote}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-9644) incrementalAlterConfigs OpType.APPEND on unset property fails with NullPointerException

2020-03-12 Thread Manikumar (Jira)


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

Manikumar resolved KAFKA-9644.
--
Fix Version/s: 2.6.0
   Resolution: Fixed

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

> incrementalAlterConfigs OpType.APPEND on unset property fails with 
> NullPointerException
> ---
>
> Key: KAFKA-9644
> URL: https://issues.apache.org/jira/browse/KAFKA-9644
> Project: Kafka
>  Issue Type: Bug
>  Components: admin
>Affects Versions: 2.3.0
>Reporter: Steve Rodrigues
>Priority: Minor
> Fix For: 2.6.0
>
> Attachments: incrementalAlterTest.patch, 
> kafka.api.PlaintextAdminIntegrationTest.testValidIncrementalAlterConfigs.test.stdout
>
>
> Running incrementalAlterConfigs with an OpType.APPEND when the config 
> property doesn't already exist fails with a NullPointerException on the 
> broker.
> Attached is a patch to the PlaintextAdminIntegrationTest demonstrating this 
> failure and the test output showing the NPE.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Requesting Access to Submit a KIP

2020-03-12 Thread Matthias J. Sax
Done.

On 3/12/20 5:06 PM, Sanjana Kaundinya wrote:
> Hi,
> 
> I’d like to submit a KIP and am requesting access to the wiki page. My wiki 
> ID is skaundinya. Please let me know if anything else is needed from my side 
> - thanks!
> 
> Sincerely,
> Sanjana
> 
> 
> 



signature.asc
Description: OpenPGP digital signature


Re: Add a customized logo for Kafka Streams

2020-03-12 Thread Matthias J. Sax
I personally love it!

-Matthias

On 3/12/20 11:31 AM, Sophie Blee-Goldman wrote:
> How reasonable of it. Let's try this again:
> Streams Logo option 2
>

>
> On Thu, Mar 12, 2020 at 9:34 AM Guozhang Wang  wrote:
>
>> Hi Sophie,
>>
>> I cannot find the attachment from your previous email --- in fact, ASF
>> mailing list usually blocks all attachments for security reasons. If you
>> can share a link to the image (google drawings etc) in your email that
>> would be great.
>>
>> Guozhang
>>
>> On Wed, Mar 11, 2020 at 1:02 PM Sophie Blee-Goldman 
>> wrote:
>>
>>> Just to throw another proposal out there and inspire some debate, here's
>>> a similar-but-different
>>> idea (inspired by John + some sketches I found on google):
>>>
>>> *~~ See attachment, inlined image is too large for the mailing list ~~*
>>>
>>> This one's definitely more fun, my only concern is that it doesn't
really
>>> scale well. At the lower end
>>> of sizes the otters will be pretty difficult to see; and I had to
stretch
>>> out the Kafka circles even at
>>> the larger end just to fit them through.
>>>
>>> But maybe with a cleaner drawing and some color it'll still look
good and
>>> be recognizable + distinct
>>> enough when small.
>>>
>>> Any thoughts? Any binding and/or non-binding votes?
>>>
>>> On Sun, Mar 8, 2020 at 1:00 AM Sophie Blee-Goldman 
>>> wrote:
>>>
 Seems the mailing list may have filtered the inlined prototype logo,
 attaching it here instead

 On Sat, Mar 7, 2020 at 11:54 PM Sophie Blee-Goldman

 wrote:

> Matthias makes a good point about being careful not to position
Streams
> as
> outside of Apache Kafka. One obvious thing we could do it just include
> the
> Kafka logo as-is in the Streams logo, somehow.
>
> I have some unqualified opinions on what that might look like:
> A good logo is simple and clean, so incorporating the Kafka logo as a
> minor
> detail within a more complicated image is probably not the best way to
> get
> the quick and easy comprehension/recognition that we're going for.
>
> That said I'd throw out the idea of just attaching something to the
> Kafka logo,
> perhaps a stream-dwelling animal, perhaps a (river) otter? It could be
> "swimming" left of the Kafka logo, with its head touching the upper
> circle and
> its tail touching the bottom one. Like Streams, it starts with Kafka
> and ends
> with Kafka (ie reading input topics and writing to output topics).
>
> Without further ado, here's my very rough prototype for the Kafka
> Streams logo:
>
> [image: image.png]
> Obviously the real thing would be colored and presumably done by
someone
> with actual artist talent/experience (or at least photoshop ability).
>
> Thoughts?
>
> On Sat, Mar 7, 2020, 1:08 PM Matthias J. Sax  wrote:
>
> Boyang,
> 
> thanks for starting this discussion. I like the idea in general
> however we need to be a little careful IMHO -- as you mentioned Kafka
> is one project and thus we should avoid the impression that Kafka
> Streams is not part of Apache Kafka.
> 
> Besides this, many projects use animals that are often very adorable.
> Maybe we could find a cute Streams related mascot? :)
> 
> I would love to hear opinions especially from the PMC if having a logo
> for Kafka Streams is a viable thing to do.
> 
> 
> -Matthias
> 
> On 3/3/20 1:01 AM, Patrik Kleindl wrote:
 Hi Boyang Great idea, that would help in some discussions. To throw
 in a first idea: https://imgur.com/a/UowXaMk best regards Patrik

 On Mon, 2 Mar 2020 at 18:23, Boyang Chen
  wrote:

> Hey Apache Kafka committers and community folks,
>
> over the years Kafka Streams has been widely adopted and tons of
> blog posts and tech talks have been trying to introduce it to
> people with need of stream processing. As it is part of Apache
> Kafka project, there is always an awkward situation where Kafka
> Streams could not be campaigned as a standalone streaming engine,
> and makes people confused about its relation to Kafka.
>
> So, do we want to introduce a customized logo just for Streams?
> The immediate benefit is when people are making technical
> decisions, we could list Streams as a logo just like Flink and
> Spark Streaming, instead of putting Kafka logo there as it is not
> literally a legitimate comparison between processing framework
> and messaging system. Should we do a KIP for this?
>
> Boyang
>

>>
>
>>
>> --
>> -- Guozhang
>>
>



signature.asc
Description: OpenPGP digital signature


Re: 回复:回复:回复:[Vote] KIP-571: Add option to force remove members in StreamsResetter

2020-03-12 Thread Guozhang Wang
Hi Matthias,

About the AdminClient param API: that's a great point here. I think overall
if users want to just "remove all members" they should not need to first
get all the member.ids themselves, but instead internally the admin client
can first issue a describe-group request to get all the member.ids, and
then use them in the next issued leave-group request, all abstracted away
from the users. With that in mind, maybe in
RemoveMembersFromConsumerGroupOptions we can just introduce an overloaded
flag param besides "members" that indicate "remove all"?

Guozhang

On Thu, Mar 12, 2020 at 2:59 PM Matthias J. Sax  wrote:

> Feyman,
>
> some more comments/questions:
>
> The description of `LeaveGroupRequest` is clear but it's unclear how
> `MemberToRemove` should behave. Which parameter is required? Which is
> optional? What is the relationship between both.
>
> The `LeaveGroupRequest` description clearly states that specifying a
> `memberId` is optional if the `groupInstanceId` is provided. If
> `MemberToRemove` applies the same pattern, it must be explicitly defined
> in the KIP (and explained in the JavaDocs of `MemberToRemove`) because
> we cannot expect that an admin-client users knows that internally a
> `LeaveGroupRequest` is used nor what the semantics of a
> `LeaveGroupRequest` are.
>
>
> About Admin API:
>
> In general, I am also confused that we allow so specify a `memberId` at
> all, because the `memberId` is an internal id that is not really exposed
> to the user. Hence, from a AdminClient point of view, accepting a
> `memberId` as input seems questionable to me? Of course, `memberId` can
> be collected via `describeConsumerGroups()` but it will return the
> `memberId` of _all_ consumer in the group and thus how would a user know
> which member should be removed for a dynamic group (if an individual
> member should be removed)?
>
> Hence, how can any user get to know the `memberId` of an individual
> client in a programtic way?
>
> Also I am wondering in general, why the removal of single dynamic member
> is important? In general, I would expect a short `session.timeout` for
> dynamic groups and thus removing a specific member from the group seems
> not to be an important feature -- for static groups we expect a long
> `session.timeout` and a user can also identify individual clients via
> `groupInstandId`, hence the feature makes sense for this case and is
> straight forward to use.
>
>
> About StreamsResetter:
>
> For this case we just say "remove all members" and thus the
> `describeConsumerGroup` approach works. However, it seems to be a
> special case?
>
> Or, if we expected that the "remove all members" use case is the norm,
> why can't we make a change admin-client to directly accept a `group.id`?
> The admin-client can internal first do a `DescribeGroupRequest` and
> afterward corresponding `LeaveGroupRequest` -- i.e., instead of building
> this pattern in `StreamsResetter` we build it directly into `AdminClient`.
>
> Last, for static group the main use case seems to be to remove an
> individual member from the group but this feature is not covered by the
> KIP: I think using `--force` to remove all members makes sense, but an
> important second feature to remove an individual static member would
> require it's own flag to specify a single `group.instance.id`.
>
>
> Thoughts?
>
>
> -Matthias
>
>
>
>
>
> On 3/11/20 8:43 PM, feyman2009 wrote:
> > Hi, Sophie
> > For 1) Sorry, I found that my expression is kind of misleading,
> what I actually mean is: "if --force not specified, an exception saying
> there are still active members on broker side will be thrown and
> suggesting using StreamsResetter with --force", I just updated the KIP
> page.
> >
> > For 2)
> > I may also had some misleading expression previous, to clarify :
> >
> > Also, it's more efficient to just send a single "clear the group"
> request vs sending a LeaveGroup
> > request for every single member. What do you think?
> > => the comparison is to send a single "clear the group" request vs
> sending a "get members" + a "remove members" request since the
> adminClient.removeMembersFromConsumerGroup support batch removal. We
> don't need to send lots of leaveGroup requests for every single member.
> >
> >I can understand your point, but I think we could reuse the
> current
> > adminClient.removeMembersFromConsumerGroup interface effectively with
> the KIP.
> > What do you think?
> >
> > Thanks!
> >
> > Feyman
> >
> >
> > --
> > 发件人:Sophie Blee-Goldman 
> > 发送时间:2020年3月10日(星期二) 03:02
> > 收件人:dev ; feyman2009 
> > 主 题:Re: 回复:回复:[Vote] KIP-571: Add option to force remove
> members in StreamsResetter
> >
> > Hey Feyman,
> >
> > 1) Regarding point 2 in your last email, if I understand correctly you
> propose to change
> > the current behavior of the reset tool when --force is not specified,
> and wait for (up to)
> > the session timeou

Requesting Access to Submit a KIP

2020-03-12 Thread Sanjana Kaundinya
Hi,

I’d like to submit a KIP and am requesting access to the wiki page. My wiki ID 
is skaundinya. Please let me know if anything else is needed from my side - 
thanks!

Sincerely,
Sanjana




[DISCUSS] KIP-577: Allow HTTP Response Headers Configured for Kafka Connect

2020-03-12 Thread Zhiguo Huang



Re: [VOTE] KIP-518: Allow listing consumer groups per state

2020-03-12 Thread Colin McCabe
Thanks, Mickael.  +1 (binding)

best,
Colin

On Fri, Mar 6, 2020, at 02:05, Mickael Maison wrote:
> Thanks David and Gwen for the votes
> Colin, I believe I've answered all your questions, can you take another look?
> 
> So far we have 1 binding and 5 non binding votes.
> 
> On Mon, Mar 2, 2020 at 4:56 PM Gwen Shapira  wrote:
> >
> > +1 (binding)
> >
> > Gwen Shapira
> > Engineering Manager | Confluent
> > 650.450.2760 | @gwenshap
> > Follow us: Twitter | blog
> >
> > On Mon, Mar 02, 2020 at 8:32 AM, David Jacot < dja...@confluent.io > wrote:
> >
> > >
> > >
> > >
> > > +1 (non-binding). Thanks for the KIP!
> > >
> > >
> > >
> > > David
> > >
> > >
> > >
> > > On Thu, Feb 6, 2020 at 10:45 PM Colin McCabe < cmccabe@ apache. org (
> > > cmcc...@apache.org ) > wrote:
> > >
> > >
> > >>
> > >>
> > >> Hi Mickael,
> > >>
> > >>
> > >>
> > >> Thanks for the KIP. I left a comment on the DISCUSS thread as well.
> > >>
> > >>
> > >>
> > >> best,
> > >> Colin
> > >>
> > >>
> > >>
> > >> On Thu, Feb 6, 2020, at 08:58, Mickael Maison wrote:
> > >>
> > >>
> > >>>
> > >>>
> > >>> Hi Manikumar,
> > >>>
> > >>>
> > >>>
> > >>> I believe I've answered David's comments in the DISCUSS thread. Thanks
> > >>>
> > >>>
> > >>>
> > >>> On Wed, Jan 15, 2020 at 10:15 AM Manikumar < manikumar. reddy@ gmail. 
> > >>> com (
> > >>> manikumar.re...@gmail.com ) >
> > >>>
> > >>>
> > >>
> > >>
> > >>
> > >> wrote:
> > >>
> > >>
> > >>>
> > 
> > 
> >  Hi Mickael,
> > 
> > 
> > 
> >  Thanks for the KIP. Can you respond to the comments from David on
> > 
> > 
> > >>>
> > >>>
> > >>
> > >>
> > >>
> > >> discuss
> > >>
> > >>
> > >>>
> > 
> > 
> >  thread?
> > 
> > 
> > 
> >  Thanks,
> > 
> > 
> > >>>
> > >>>
> > >>
> > >>
> > >
> > >
> > >
>


Re: [ANNOUNCE] Apache Kafka 2.4.1

2020-03-12 Thread Matthias J. Sax
Thanks for driving the release Bill!

-Matthias

On 3/12/20 1:22 PM, Bill Bejeck wrote:
> The Apache Kafka community is pleased to announce the release for Apache
> Kafka 2.4.1
> 
> This is a bug fix release and it includes fixes and improvements from 39
> JIRAs, including a few critical bugs.
> 
> All of the changes in this release can be found in the release notes:
> https://www.apache.org/dist/kafka/2.4.1/RELEASE_NOTES.html
> 
> 
> You can download the source and binary release (Scala 2.11, 2.12, and 2.13)
> from:
> https://kafka.apache.org/downloads#2.4.1
> 
> ---
> 
> 
> Apache Kafka is a distributed streaming platform with four core APIs:
> 
> 
> ** The Producer API allows an application to publish a stream records to
> one or more Kafka topics.
> 
> ** The Consumer API allows an application to subscribe to one or more
> topics and process the stream of records produced to them.
> 
> ** The Streams API allows an application to act as a stream processor,
> consuming an input stream from one or more topics and producing an
> output stream to one or more output topics, effectively transforming the
> input streams to output streams.
> 
> ** The Connector API allows building and running reusable producers or
> consumers that connect Kafka topics to existing applications or data
> systems. For example, a connector to a relational database might
> capture every change to a table.
> 
> 
> With these APIs, Kafka can be used for two broad classes of application:
> 
> ** Building real-time streaming data pipelines that reliably get data
> between systems or applications.
> 
> ** Building real-time streaming applications that transform or react
> to the streams of data.
> 
> 
> Apache Kafka is in use at large and small companies worldwide, including
> Capital One, Goldman Sachs, ING, LinkedIn, Netflix, Pinterest, Rabobank,
> Target, The New York Times, Uber, Yelp, and Zalando, among others.
> 
> A big thank you for the following 35 contributors to this release!
> 
> A. Sophie Blee-Goldman, Alex Kokachev, bill, Bill Bejeck, Boyang Chen,
> Brian Bushree, Brian Byrne, Bruno Cadonna, Chia-Ping Tsai, Chris Egerton,
> Colin Patrick McCabe, David Jacot, David Kim, David Mao, Dhruvil Shah,
> Gunnar Morling, Guozhang Wang, huxi, Ismael Juma, Ivan Yurchenko, Jason
> Gustafson, John Roesler, Konstantine Karantasis, Lev Zemlyanov, Manikumar
> Reddy, Matthew Wong, Matthias J. Sax, Michael Gyarmathy, Michael Viamari,
> Nigel Liang, Rajini Sivaram, Randall Hauch, Tomislav, Vikas Singh, Xin Wang
> 
> We welcome your help and feedback. For more information on how to
> report problems, and to get involved, visit the project website at
> https://kafka.apache.org/
> 
> Thank you!
> 
> 
> Regards,
> 
> Bill Bejeck
> 



signature.asc
Description: OpenPGP digital signature


Re: 回复:回复:回复:[Vote] KIP-571: Add option to force remove members in StreamsResetter

2020-03-12 Thread Matthias J. Sax
Feyman,

some more comments/questions:

The description of `LeaveGroupRequest` is clear but it's unclear how
`MemberToRemove` should behave. Which parameter is required? Which is
optional? What is the relationship between both.

The `LeaveGroupRequest` description clearly states that specifying a
`memberId` is optional if the `groupInstanceId` is provided. If
`MemberToRemove` applies the same pattern, it must be explicitly defined
in the KIP (and explained in the JavaDocs of `MemberToRemove`) because
we cannot expect that an admin-client users knows that internally a
`LeaveGroupRequest` is used nor what the semantics of a
`LeaveGroupRequest` are.


About Admin API:

In general, I am also confused that we allow so specify a `memberId` at
all, because the `memberId` is an internal id that is not really exposed
to the user. Hence, from a AdminClient point of view, accepting a
`memberId` as input seems questionable to me? Of course, `memberId` can
be collected via `describeConsumerGroups()` but it will return the
`memberId` of _all_ consumer in the group and thus how would a user know
which member should be removed for a dynamic group (if an individual
member should be removed)?

Hence, how can any user get to know the `memberId` of an individual
client in a programtic way?

Also I am wondering in general, why the removal of single dynamic member
is important? In general, I would expect a short `session.timeout` for
dynamic groups and thus removing a specific member from the group seems
not to be an important feature -- for static groups we expect a long
`session.timeout` and a user can also identify individual clients via
`groupInstandId`, hence the feature makes sense for this case and is
straight forward to use.


About StreamsResetter:

For this case we just say "remove all members" and thus the
`describeConsumerGroup` approach works. However, it seems to be a
special case?

Or, if we expected that the "remove all members" use case is the norm,
why can't we make a change admin-client to directly accept a `group.id`?
The admin-client can internal first do a `DescribeGroupRequest` and
afterward corresponding `LeaveGroupRequest` -- i.e., instead of building
this pattern in `StreamsResetter` we build it directly into `AdminClient`.

Last, for static group the main use case seems to be to remove an
individual member from the group but this feature is not covered by the
KIP: I think using `--force` to remove all members makes sense, but an
important second feature to remove an individual static member would
require it's own flag to specify a single `group.instance.id`.


Thoughts?


-Matthias





On 3/11/20 8:43 PM, feyman2009 wrote:
> Hi, Sophie
> For 1) Sorry, I found that my expression is kind of misleading,
what I actually mean is: "if --force not specified, an exception saying
there are still active members on broker side will be thrown and
suggesting using StreamsResetter with --force", I just updated the KIP page.
>
> For 2)
> I may also had some misleading expression previous, to clarify :
>
> Also, it's more efficient to just send a single "clear the group"
request vs sending a LeaveGroup
> request for every single member. What do you think?
> => the comparison is to send a single "clear the group" request vs
sending a "get members" + a "remove members" request since the
adminClient.removeMembersFromConsumerGroup support batch removal. We
don't need to send lots of leaveGroup requests for every single member.
>
>I can understand your point, but I think we could reuse the
current
> adminClient.removeMembersFromConsumerGroup interface effectively with
the KIP.
> What do you think?
>
> Thanks!
>
> Feyman
>
>
> --
> 发件人:Sophie Blee-Goldman 
> 发送时间:2020年3月10日(星期二) 03:02
> 收件人:dev ; feyman2009 
> 主 题:Re: 回复:回复:[Vote] KIP-571: Add option to force remove
members in StreamsResetter
>
> Hey Feyman,
>
> 1) Regarding point 2 in your last email, if I understand correctly you
propose to change
> the current behavior of the reset tool when --force is not specified,
and wait for (up to)
> the session timeout for all members to be removed. I'm not sure we
should change this,
> especially now that we have a better way to handle the case when the
group is not empty:
> we should continue to throw an exception and fail fast, but can print
a message suggesting
> to use the new --force option to remove remaining group members. Why
make users wait
> for the session timeout when we've just added a new feature that means
they don't have to?
>
> 2) Regarding Matthias' question:
>
>> I am really wondering, if for a static group, we should allow users
toremove individual members? For a dynamic group this feature would not
> make much sense IMHO, because the `memberId` is not know by the user.
>
> I think his point is similar to what I was trying to get at earlier,
with the proposal to add a new
> #removeAllMembers API rather than an

Build failed in Jenkins: kafka-trunk-jdk11 #1231

2020-03-12 Thread Apache Jenkins Server
See 


Changes:

[github] KAFKA-9675: Fix bug that prevents RocksDB metrics to be updated (#8256)


--
[...truncated 2.91 MB...]
org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnStreamsTime[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowIfInMemoryBuiltInStoreIsAccessedWithUntypedMethod[Eos enabled = 
false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowIfInMemoryBuiltInStoreIsAccessedWithUntypedMethod[Eos enabled = 
false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourcesThatMatchMultiplePattern[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourcesThatMatchMultiplePattern[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldPopulateGlobalStore[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldPopulateGlobalStore[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowIfPersistentBuiltInStoreIsAccessedWithUntypedMethod[Eos enabled = 
false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldThrowIfPersistentBuiltInStoreIsAccessedWithUntypedMethod[Eos enabled = 
false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldAllowPrePopulatingStatesStoresWithCachingEnabled[Eos enabled = false] 
STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldAllowPrePopulatingStatesStoresWithCachingEnabled[Eos enabled = false] 
PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnCorrectPersistentStoreTypeOnly[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnCorrectPersistentStoreTypeOnly[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldRespectTaskIdling[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldRespectTaskIdling[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSourceSpecificDeserializers[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSourceSpecificDeserializers[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldReturnAllStores[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldReturnAllStores[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotCreateStateDirectoryForStatelessTopology[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotCreateStateDirectoryForStatelessTopology[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldApplyGlobalUpdatesCorrectlyInRecursiveTopologies[Eos enabled = false] 
STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldApplyGlobalUpdatesCorrectlyInRecursiveTopologies[Eos enabled = false] 
PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnAllStoresNames[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnAllStoresNames[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessConsumerRecordList[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessConsumerRecordList[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSinkSpecificSerializers[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSinkSpecificSerializers[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldFlushStoreForFirstInput[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldFlushStoreForFirstInput[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourceThatMatchPattern[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourceThatMatchPattern[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUpdateStoreForNewKey[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUpdateStoreForNewKey[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldSendRecordViaCorrectSourceTopicDeprecated[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldSendRecordViaCorrectSourceTopicDeprecated[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnWallClockTime[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnWallClockTime[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldSetRecor

Re: jeffhuang0126 --- Need permission to create KIP, discuss thread etc. for working on new KIP

2020-03-12 Thread John Roesler
Hi Jeff,

I've granted you access to edit wiki pages. For discussions, you just send a 
message to this list (similar to the ones you can see from other KIPs).

Thanks in advance for your contribution!
-John

On Thu, Mar 12, 2020, at 15:25, Zhiguo Huang wrote:
> Hi,
> 
> My wikid jeffhuang0126
> I need permission to create KIP, discuss thread etc. for working on new KIP.
> 
> Jeff.
>


jeffhuang0126 --- Need permission to create KIP, discuss thread etc. for working on new KIP

2020-03-12 Thread Zhiguo Huang
Hi,

My wikid jeffhuang0126
I need permission to create KIP, discuss thread etc. for working on new KIP.

Jeff.


[ANNOUNCE] Apache Kafka 2.4.1

2020-03-12 Thread Bill Bejeck
The Apache Kafka community is pleased to announce the release for Apache
Kafka 2.4.1

This is a bug fix release and it includes fixes and improvements from 39
JIRAs, including a few critical bugs.

All of the changes in this release can be found in the release notes:
https://www.apache.org/dist/kafka/2.4.1/RELEASE_NOTES.html


You can download the source and binary release (Scala 2.11, 2.12, and 2.13)
from:
https://kafka.apache.org/downloads#2.4.1

---


Apache Kafka is a distributed streaming platform with four core APIs:


** The Producer API allows an application to publish a stream records to
one or more Kafka topics.

** The Consumer API allows an application to subscribe to one or more
topics and process the stream of records produced to them.

** The Streams API allows an application to act as a stream processor,
consuming an input stream from one or more topics and producing an
output stream to one or more output topics, effectively transforming the
input streams to output streams.

** The Connector API allows building and running reusable producers or
consumers that connect Kafka topics to existing applications or data
systems. For example, a connector to a relational database might
capture every change to a table.


With these APIs, Kafka can be used for two broad classes of application:

** Building real-time streaming data pipelines that reliably get data
between systems or applications.

** Building real-time streaming applications that transform or react
to the streams of data.


Apache Kafka is in use at large and small companies worldwide, including
Capital One, Goldman Sachs, ING, LinkedIn, Netflix, Pinterest, Rabobank,
Target, The New York Times, Uber, Yelp, and Zalando, among others.

A big thank you for the following 35 contributors to this release!

A. Sophie Blee-Goldman, Alex Kokachev, bill, Bill Bejeck, Boyang Chen,
Brian Bushree, Brian Byrne, Bruno Cadonna, Chia-Ping Tsai, Chris Egerton,
Colin Patrick McCabe, David Jacot, David Kim, David Mao, Dhruvil Shah,
Gunnar Morling, Guozhang Wang, huxi, Ismael Juma, Ivan Yurchenko, Jason
Gustafson, John Roesler, Konstantine Karantasis, Lev Zemlyanov, Manikumar
Reddy, Matthew Wong, Matthias J. Sax, Michael Gyarmathy, Michael Viamari,
Nigel Liang, Rajini Sivaram, Randall Hauch, Tomislav, Vikas Singh, Xin Wang

We welcome your help and feedback. For more information on how to
report problems, and to get involved, visit the project website at
https://kafka.apache.org/

Thank you!


Regards,

Bill Bejeck


[jira] [Created] (KAFKA-9713) Remove BufferExhausedException

2020-03-12 Thread Brian Byrne (Jira)
Brian Byrne created KAFKA-9713:
--

 Summary: Remove BufferExhausedException
 Key: KAFKA-9713
 URL: https://issues.apache.org/jira/browse/KAFKA-9713
 Project: Kafka
  Issue Type: Task
  Components: producer 
Reporter: Brian Byrne


BufferExhaustedException was deprecated in 0.9.0.0, and the corresponding 
block.on.buffer.full property has since been removed. The exception should 
follow.

{quote}Deprecations in 0.9.0.0

The producer config block.on.buffer.full has been deprecated and will be 
removed in future release. Currently its default value has been changed to 
false. The KafkaProducer will no longer throw BufferExhaustedException but 
instead will use max.block.ms value to block, after which it will throw a 
TimeoutException. If block.on.buffer.full property is set to true explicitly, 
it will set the max.block.ms to Long.MAX_VALUE and metadata.fetch.timeout.ms 
will not be honoured{quote}




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-9364) Fix misleading consumer logs on throttling

2020-03-12 Thread Colin McCabe (Jira)


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

Colin McCabe resolved KAFKA-9364.
-
Fix Version/s: 2.5.0
   Resolution: Fixed

> Fix misleading consumer logs on throttling
> --
>
> Key: KAFKA-9364
> URL: https://issues.apache.org/jira/browse/KAFKA-9364
> Project: Kafka
>  Issue Type: Bug
>Reporter: Colin McCabe
>Assignee: Colin McCabe
>Priority: Minor
> Fix For: 2.5.0
>
>
> Fix misleading consumer logs on throttling



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Add a customized logo for Kafka Streams

2020-03-12 Thread Sophie Blee-Goldman
How reasonable of it. Let's try this again:
Streams Logo option 2


On Thu, Mar 12, 2020 at 9:34 AM Guozhang Wang  wrote:

> Hi Sophie,
>
> I cannot find the attachment from your previous email --- in fact, ASF
> mailing list usually blocks all attachments for security reasons. If you
> can share a link to the image (google drawings etc) in your email that
> would be great.
>
> Guozhang
>
> On Wed, Mar 11, 2020 at 1:02 PM Sophie Blee-Goldman 
> wrote:
>
>> Just to throw another proposal out there and inspire some debate, here's
>> a similar-but-different
>> idea (inspired by John + some sketches I found on google):
>>
>> *~~ See attachment, inlined image is too large for the mailing list ~~*
>>
>> This one's definitely more fun, my only concern is that it doesn't really
>> scale well. At the lower end
>> of sizes the otters will be pretty difficult to see; and I had to stretch
>> out the Kafka circles even at
>> the larger end just to fit them through.
>>
>> But maybe with a cleaner drawing and some color it'll still look good and
>> be recognizable + distinct
>> enough when small.
>>
>> Any thoughts? Any binding and/or non-binding votes?
>>
>> On Sun, Mar 8, 2020 at 1:00 AM Sophie Blee-Goldman 
>> wrote:
>>
>>> Seems the mailing list may have filtered the inlined prototype logo,
>>> attaching it here instead
>>>
>>> On Sat, Mar 7, 2020 at 11:54 PM Sophie Blee-Goldman 
>>> wrote:
>>>
 Matthias makes a good point about being careful not to position Streams
 as
 outside of Apache Kafka. One obvious thing we could do it just include
 the
 Kafka logo as-is in the Streams logo, somehow.

 I have some unqualified opinions on what that might look like:
 A good logo is simple and clean, so incorporating the Kafka logo as a
 minor
 detail within a more complicated image is probably not the best way to
 get
 the quick and easy comprehension/recognition that we're going for.

 That said I'd throw out the idea of just attaching something to the
 Kafka logo,
 perhaps a stream-dwelling animal, perhaps a (river) otter? It could be
 "swimming" left of the Kafka logo, with its head touching the upper
 circle and
 its tail touching the bottom one. Like Streams, it starts with Kafka
 and ends
 with Kafka (ie reading input topics and writing to output topics).

 Without further ado, here's my very rough prototype for the Kafka
 Streams logo:

 [image: image.png]
 Obviously the real thing would be colored and presumably done by someone
 with actual artist talent/experience (or at least photoshop ability).

 Thoughts?

 On Sat, Mar 7, 2020, 1:08 PM Matthias J. Sax  wrote:

> -BEGIN PGP SIGNED MESSAGE-
> Hash: SHA512
>
> Boyang,
>
> thanks for starting this discussion. I like the idea in general
> however we need to be a little careful IMHO -- as you mentioned Kafka
> is one project and thus we should avoid the impression that Kafka
> Streams is not part of Apache Kafka.
>
> Besides this, many projects use animals that are often very adorable.
> Maybe we could find a cute Streams related mascot? :)
>
> I would love to hear opinions especially from the PMC if having a logo
> for Kafka Streams is a viable thing to do.
>
>
> - -Matthias
>
> On 3/3/20 1:01 AM, Patrik Kleindl wrote:
> > Hi Boyang Great idea, that would help in some discussions. To throw
> > in a first idea: https://imgur.com/a/UowXaMk best regards Patrik
> >
> > On Mon, 2 Mar 2020 at 18:23, Boyang Chen
> >  wrote:
> >
> >> Hey Apache Kafka committers and community folks,
> >>
> >> over the years Kafka Streams has been widely adopted and tons of
> >> blog posts and tech talks have been trying to introduce it to
> >> people with need of stream processing. As it is part of Apache
> >> Kafka project, there is always an awkward situation where Kafka
> >> Streams could not be campaigned as a standalone streaming engine,
> >> and makes people confused about its relation to Kafka.
> >>
> >> So, do we want to introduce a customized logo just for Streams?
> >> The immediate benefit is when people are making technical
> >> decisions, we could list Streams as a logo just like Flink and
> >> Spark Streaming, instead of putting Kafka logo there as it is not
> >> literally a legitimate comparison between processing framework
> >> and messaging system. Should we do a KIP for this?
> >>
> >> Boyang
> >>
> >
> -BEGIN PGP SIGNATURE-
>
> iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl5kDVQACgkQO4miYXKq
> /Og2Hg/+IOjjz3yrvHOz/p/Qbi9hXDK+GZT6Ixzk2HrrpMiLAAZs6WrNGQMwwI6a
> qrPdMRLKA45F53wwMXBqhLYe0O0vOIRSur3pef8UuTVEkbFstY6dNzduPjTXK4vV
> Ahzb+pu1lZ

[jira] [Created] (KAFKA-9712) Reflections library 0.9.12 introduced in 2.5 causes regression scanning for plugins on plugin_path

2020-03-12 Thread Nigel Liang (Jira)
Nigel Liang created KAFKA-9712:
--

 Summary: Reflections library 0.9.12 introduced in 2.5 causes 
regression scanning for plugins on plugin_path
 Key: KAFKA-9712
 URL: https://issues.apache.org/jira/browse/KAFKA-9712
 Project: Kafka
  Issue Type: Bug
Affects Versions: 2.5.0
Reporter: Nigel Liang


Reflections v0.9.12 was introduced in 2.5 branch to remove Guava dependency - 
https://issues.apache.org/jira/browse/KAFKA-3061

This version, however, contains a 
[bug|https://github.com/ronmamo/reflections/issues/273] that will cause 
regression when scanning for plugins if the `plugins_path` specified does not 
contain valid plugins. The case where we were able to repro this, the 
`plugins_path` was misconfigured to point to `~/.ssh` which contained unrelated 
files but no plugins. Further testing is needed to figure out if it will repro 
for other cases such as empty directory, combination of valid and invalid 
plugins in directory, just some types of plugins and not others in directory, 
etc.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Add a customized logo for Kafka Streams

2020-03-12 Thread Guozhang Wang
Hi Sophie,

I cannot find the attachment from your previous email --- in fact, ASF
mailing list usually blocks all attachments for security reasons. If you
can share a link to the image (google drawings etc) in your email that
would be great.

Guozhang

On Wed, Mar 11, 2020 at 1:02 PM Sophie Blee-Goldman 
wrote:

> Just to throw another proposal out there and inspire some debate, here's a
> similar-but-different
> idea (inspired by John + some sketches I found on google):
>
> *~~ See attachment, inlined image is too large for the mailing list ~~*
>
> This one's definitely more fun, my only concern is that it doesn't really
> scale well. At the lower end
> of sizes the otters will be pretty difficult to see; and I had to stretch
> out the Kafka circles even at
> the larger end just to fit them through.
>
> But maybe with a cleaner drawing and some color it'll still look good and
> be recognizable + distinct
> enough when small.
>
> Any thoughts? Any binding and/or non-binding votes?
>
> On Sun, Mar 8, 2020 at 1:00 AM Sophie Blee-Goldman 
> wrote:
>
>> Seems the mailing list may have filtered the inlined prototype logo,
>> attaching it here instead
>>
>> On Sat, Mar 7, 2020 at 11:54 PM Sophie Blee-Goldman 
>> wrote:
>>
>>> Matthias makes a good point about being careful not to position Streams
>>> as
>>> outside of Apache Kafka. One obvious thing we could do it just include
>>> the
>>> Kafka logo as-is in the Streams logo, somehow.
>>>
>>> I have some unqualified opinions on what that might look like:
>>> A good logo is simple and clean, so incorporating the Kafka logo as a
>>> minor
>>> detail within a more complicated image is probably not the best way to
>>> get
>>> the quick and easy comprehension/recognition that we're going for.
>>>
>>> That said I'd throw out the idea of just attaching something to the
>>> Kafka logo,
>>> perhaps a stream-dwelling animal, perhaps a (river) otter? It could be
>>> "swimming" left of the Kafka logo, with its head touching the upper
>>> circle and
>>> its tail touching the bottom one. Like Streams, it starts with Kafka and
>>> ends
>>> with Kafka (ie reading input topics and writing to output topics).
>>>
>>> Without further ado, here's my very rough prototype for the Kafka
>>> Streams logo:
>>>
>>> [image: image.png]
>>> Obviously the real thing would be colored and presumably done by someone
>>> with actual artist talent/experience (or at least photoshop ability).
>>>
>>> Thoughts?
>>>
>>> On Sat, Mar 7, 2020, 1:08 PM Matthias J. Sax  wrote:
>>>
 -BEGIN PGP SIGNED MESSAGE-
 Hash: SHA512

 Boyang,

 thanks for starting this discussion. I like the idea in general
 however we need to be a little careful IMHO -- as you mentioned Kafka
 is one project and thus we should avoid the impression that Kafka
 Streams is not part of Apache Kafka.

 Besides this, many projects use animals that are often very adorable.
 Maybe we could find a cute Streams related mascot? :)

 I would love to hear opinions especially from the PMC if having a logo
 for Kafka Streams is a viable thing to do.


 - -Matthias

 On 3/3/20 1:01 AM, Patrik Kleindl wrote:
 > Hi Boyang Great idea, that would help in some discussions. To throw
 > in a first idea: https://imgur.com/a/UowXaMk best regards Patrik
 >
 > On Mon, 2 Mar 2020 at 18:23, Boyang Chen
 >  wrote:
 >
 >> Hey Apache Kafka committers and community folks,
 >>
 >> over the years Kafka Streams has been widely adopted and tons of
 >> blog posts and tech talks have been trying to introduce it to
 >> people with need of stream processing. As it is part of Apache
 >> Kafka project, there is always an awkward situation where Kafka
 >> Streams could not be campaigned as a standalone streaming engine,
 >> and makes people confused about its relation to Kafka.
 >>
 >> So, do we want to introduce a customized logo just for Streams?
 >> The immediate benefit is when people are making technical
 >> decisions, we could list Streams as a logo just like Flink and
 >> Spark Streaming, instead of putting Kafka logo there as it is not
 >> literally a legitimate comparison between processing framework
 >> and messaging system. Should we do a KIP for this?
 >>
 >> Boyang
 >>
 >
 -BEGIN PGP SIGNATURE-

 iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl5kDVQACgkQO4miYXKq
 /Og2Hg/+IOjjz3yrvHOz/p/Qbi9hXDK+GZT6Ixzk2HrrpMiLAAZs6WrNGQMwwI6a
 qrPdMRLKA45F53wwMXBqhLYe0O0vOIRSur3pef8UuTVEkbFstY6dNzduPjTXK4vV
 Ahzb+pu1lZsF+n2DbMuXWvkmvMAsSrKQXbH58rA7I4qx7Zr6g1a/KL2b2oOoo3kI
 4x3PJfG6oLSnQHwkJxmM79ZjM7MLZh0d8cRqb7Oudy5MJiMzHY+Rm5aTu4nhPgRr
 cLxA8kz1PbGPboxjD9/ZGuZJMWfVnvY1wJcOp5UnOUs4kX5uYDyWw1sKIn3DcnW8
 YVzoto0syCHTAdbl89H2fxhJbtVp8JSxbBx9AW8mdgLOLsYRZGHZ1cbdJ3h4NkeE
 xTPTuTptQbhdcjbSVX6F0q+h1hiPCU5PKqcR12zGVTBI4rOGkhPIhDdnIti5Qp2e
 MQ1Urh/tWCfw

Re: Timestamp check in Kafka TransactionMetadata

2020-03-12 Thread Guozhang Wang
Hello,

We have one PR which I believe is addressing exactly your observed problem:
https://github.com/apache/kafka/pull/8278


Guozhang

On Wed, Mar 11, 2020 at 11:57 PM Guru C G 
wrote:

> We have come across an issue where in FATAL messages are logged in the
> broker.
>
> FATAL kafka.coordinator.transaction.TransactionMetadata:
> TransactionMetadata(transactionalId=tx-id-1, producerId=96011,
> producerEpoch=51, txnTimeoutMs=6, state=CompleteCommit,
> pendingState=Some(Ongoing), topicPartitions=Set(),
> txnStartTimestamp=1580894482199, txnLastUpdateTimestamp=1580894482292)'s
> transition to TxnTransitMetadata(producerId=96011, producerEpoch=51,
> txnTimeoutMs=6, txnState=Ongoing, topicPartitions=Set(topic1-0),
> txnStartTimestamp=1580894480766, txnLastUpdateTimestamp=1580894480766)
> failed: this should not happen
>
> On close inspection, we found the message is because the completed
> transaction has a newer timestamp(txnStartTimestamp=1580894482199) than the
> current timestamp of TxnTransitMetadata(txnStartTimestamp=1580894480766)
> and we also found the possibility of clocks in the broker being out of sync
> by a few seconds.
>
>
> https://github.com/apache/kafka/blob/b526528cafe4142b73df8c930473b0cddc84ca9d/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala#L382
>
>
> The scenario in general is acknowledged and partially addressed below.
> However, it does not cover the case where the startTime of Ongoing
> transaction is older than start time of completed/aborted.
>
> https://issues.apache.org/jira/browse/KAFKA-5415?focusedCommentId=16045170&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16045170
>
> Is this deliberate? Do we need that check there?
>
>

-- 
-- Guozhang


Re: Create KIP permissions

2020-03-12 Thread Bill Bejeck
Hi Paolo,

You're all set now.

Thanks,
Bill

On Thu, Mar 12, 2020 at 9:54 AM Paolo Moriello 
wrote:

> Hi,
>
> My name is Paolo Moriello, I'd like to get permissions to create KIPs. My
> wiki id is: paolomoriello.
>
> Thanks,
> Paolo
>


Jenkins build is back to normal : kafka-trunk-jdk11 #1230

2020-03-12 Thread Apache Jenkins Server
See 




Re: Create KIP permissions

2020-03-12 Thread Jun Rao
Hi, Paolo,

Thanks for your interest. Just gave you the wiki permissions.

Jun

On Thu, Mar 12, 2020 at 6:54 AM Paolo Moriello 
wrote:

> Hi,
>
> My name is Paolo Moriello, I'd like to get permissions to create KIPs. My
> wiki id is: paolomoriello.
>
> Thanks,
> Paolo
>


Re: [kafka-clients] Re: [VOTE] 2.5.0 RC1

2020-03-12 Thread Bill Bejeck
Hi David,


   1. Scanned the Javadoc, looks good
   2. Downloaded kafka_2.12-2.5.0 and ran the quickstart and streams
   quickstart
   3. Verified the signatures

+1 (non-binding)

Thanks for running the release David!

-Bill

On Tue, Mar 10, 2020 at 4:01 PM David Arthur 
wrote:

> Thanks for the test failure reports, Tom. Tracking (and fixing) these is
> important and will make future release managers have an easier time :)
>
> -David
>
> On Tue, Mar 10, 2020 at 10:16 AM Tom Bentley  wrote:
>
>> Hi David,
>>
>> I verified signatures, built the tagged branch and ran unit and
>> integration
>> tests. I found some flaky tests, as follows:
>>
>> https://issues.apache.org/jira/browse/KAFKA-9691 (new)
>> https://issues.apache.org/jira/browse/KAFKA-9692 (new)
>> https://issues.apache.org/jira/browse/KAFKA-9283 (already reported)
>>
>> Many thanks,
>>
>> Tom
>>
>> On Tue, Mar 10, 2020 at 3:28 AM David Arthur  wrote:
>>
>> > Hello Kafka users, developers and client-developers,
>> >
>> > This is the second candidate for release of Apache Kafka 2.5.0. The
>> first
>> > release candidate included an erroneous NOTICE file, so another RC was
>> > needed to fix that.
>> >
>> > This is a major release of Kafka which includes many new features,
>> > improvements, and bug fixes including:
>> >
>> > * TLS 1.3 support (1.2 is now the default)
>> > * Co-groups for Kafka Streams
>> > * Incremental rebalance for Kafka Consumer
>> > * New metrics for better operational insight
>> > * Upgrade Zookeeper to 3.5.7
>> > * Deprecate support for Scala 2.11
>> >
>> > Release notes for the 2.5.0 release:
>> > https://home.apache.org/~davidarthur/kafka-2.5.0-rc1/RELEASE_NOTES.html
>> >
>> > *** Please download, test and vote by Monday, March 16th 2020 5pm 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/~davidarthur/kafka-2.5.0-rc1/
>> >
>> > * Maven artifacts to be voted upon:
>> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
>> >
>> > * Javadoc:
>> > https://home.apache.org/~davidarthur/kafka-2.5.0-rc1/javadoc/
>> >
>> > * Tag to be voted upon (off 2.5 branch) is the 2.5.0 tag:
>> > https://github.com/apache/kafka/releases/tag/2.5.0-rc1
>> >
>> > * Documentation:
>> > https://kafka.apache.org/25/documentation.html
>> >
>> > * Protocol:
>> > https://kafka.apache.org/25/protocol.html
>> >
>> > * Links to successful Jenkins builds for the 2.5 branch to follow
>> >
>> > Thanks,
>> > David Arthur
>> >
>>
>
>
> --
> -David
>
> --
> 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/CAGCQu3rbtJgYBkmndKEj3DT2mUK9PfiuQP%3DiUa%2B1gZ%3D0DVVORA%40mail.gmail.com
> 
> .
>


[jira] [Created] (KAFKA-9711) The authentication failure caused by SSLEngine#beginHandshake is not properly caught and handled

2020-03-12 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-9711:
-

 Summary: The authentication failure caused by 
SSLEngine#beginHandshake is not properly caught and handled
 Key: KAFKA-9711
 URL: https://issues.apache.org/jira/browse/KAFKA-9711
 Project: Kafka
  Issue Type: Bug
Reporter: Chia-Ping Tsai
Assignee: Chia-Ping Tsai


{code:java}
@Override
public void handshake() throws IOException {
if (state == State.NOT_INITALIZED)
startHandshake(); // this line
if (ready())
throw renegotiationException();
if (state == State.CLOSING)
throw closingException();

{code}

SSLEngine#beginHandshake is possible to throw authentication failures (for 
example, no suitable cipher suites) so we ought to catch SSLException and then 
convert it to SslAuthenticationException so as to process authentication 
failures correctly.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Create KIP permissions

2020-03-12 Thread Paolo Moriello
Hi,

My name is Paolo Moriello, I'd like to get permissions to create KIPs. My
wiki id is: paolomoriello.

Thanks,
Paolo


[jira] [Resolved] (KAFKA-9695) AdminClient allows null topic configs, but broker throws NPE

2020-03-12 Thread Rajini Sivaram (Jira)


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

Rajini Sivaram resolved KAFKA-9695.
---
  Reviewer: Manikumar
Resolution: Fixed

Updated broker to throw InvalidRequestException for null config values in 
CreateTopics, AlterConfigs and IncrementalAlterConfigs.

> AdminClient allows null topic configs, but broker throws NPE
> 
>
> Key: KAFKA-9695
> URL: https://issues.apache.org/jira/browse/KAFKA-9695
> Project: Kafka
>  Issue Type: Bug
>Reporter: Rajini Sivaram
>Assignee: Rajini Sivaram
>Priority: Major
> Fix For: 2.6.0
>
>
> Config entries may contain null values, but broker's AdminManager throws NPE 
> resulting in UnknownServerException. We should handle null values in configs.
> {code:java}
> [2020-03-10 21:56:07,904] ERROR [Admin Manager on Broker 0]: Error processing 
> create topic request CreatableTopic(name='topic', numPartitions=2, 
> replicationFactor=3, assignments=[], 
> configs=[CreateableTopicConfig(name='message.format.version', value=null), 
> CreateableTopicConfig(name='compression.type', value='producer')]) 
> (kafka.server.AdminManager:76)
> java.lang.NullPointerException
>   at java.util.Hashtable.put(Hashtable.java:460)
>   at java.util.Properties.setProperty(Properties.java:166)
>   at 
> kafka.server.AdminManager.$anonfun$createTopics$3(AdminManager.scala:99)
>   at scala.collection.Iterator.foreach(Iterator.scala:941)
>   at scala.collection.Iterator.foreach$(Iterator.scala:941)
>   at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
>   at scala.collection.IterableLike.foreach(IterableLike.scala:74)
>   at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
>   at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
>   at 
> kafka.server.AdminManager.$anonfun$createTopics$2(AdminManager.scala:98)
>   at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
>   at 
> scala.collection.mutable.HashMap$$anon$2.$anonfun$foreach$3(HashMap.scala:158)
>   at scala.collection.mutable.HashTable.foreachEntry(HashTable.scala:237)
>   at scala.collection.mutable.HashTable.foreachEntry$(HashTable.scala:230)
>   at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:44)
>   at scala.collection.mutable.HashMap$$anon$2.foreach(HashMap.scala:158)
>   at scala.collection.TraversableLike.map(TraversableLike.scala:238)
>   at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
>   at scala.collection.AbstractTraversable.map(Traversable.scala:108)
>   at kafka.server.AdminManager.createTopics(AdminManager.scala:91)
>   at 
> kafka.server.KafkaApis.handleCreateTopicsRequest(KafkaApis.scala:1701)
>   at kafka.server.KafkaApis.handle(KafkaApis.scala:147)
>   at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:70)
>  {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [DISCUSS] KIP-552: Add interface to handle unused config

2020-03-12 Thread Patrik Kleindl
+1 to Matthias' proposal.
First reduce the false positives, then improve the various clients.
best regards
Patrik

On Thu, 12 Mar 2020 at 07:20, Matthias J. Sax  wrote:

> Personally, I think that adding a specify prefix for user configs would
> be the simplest fix. Kafka Streams and Kafka Connect could just use this
> prefix to avoid the false positive WARN logs.
>
> Hence, the KIP does not need to fix every false positive warning, it
> just changes the AbstractConfig about when a WARN is logged, and we can
> do follow up PRs to use this new feature in Connect and Streams.
>
>
> -Matthias
>
>
> On 3/11/20 7:55 AM, Artur Burtsev wrote:
> > Long time no see, was on holiday 🇳🇿.
> >
> > Thread got an interesting twist into reporting and fixing every single
> > warning individually, which is outside of my capacity. As I mentioned
> > in the KIP and also as highlighted by Patric - you don't have to do
> > anything extraordinary to get these warnings, you just use default
> > Kafka components (Streams, Connect) or Confluent schema registry. So
> > hopefully one day it will be fixed.
> >
> > I totally agree with Gwen, any misconfigs should be as visible as
> > possible, however the more false positives you have, the easier it is
> > to overlook misconfig.
> >
> > We don't worry about misconfigs as much as we do about warnings, so we
> > would mute easily, unfortunately very useful configuration dump is on
> > the same logger.
> >
> > Solution for as would be to wrap producer and consumer, log config
> > dumps from wrappers and mute original config loggers.
> >
> > I don't mind KIP to be canceled.
> >
> > Thanks,
> > Artur
> >
> > On Sat, Mar 7, 2020 at 10:11 PM Matthias J. Sax 
> wrote:
> >>
> > What is the status of this KIP?
> >
> >
> > -Matthias
> >
> > On 2/17/20 2:41 PM, John Roesler wrote:
>  Thanks Matthias,
> 
>  I got the impression this was considered and rejected in
>  KAFKA-7509, but I'm not sure why. Maybe it was never really
>  considered at all, just proposed and not-noticed? Perhaps Randall
>  or Sönke can comment. See:
> > https://issues.apache.org/jira/browse/KAFKA-7509?focusedCommentId=166608
> > 68&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpan
> > el#comment-16660868
> 
>  It would be good to know why that proposal didn't move forward.
> 
>  Thanks, -John
> 
> 
> 
>  On Mon, Feb 17, 2020, at 12:17, Matthias J. Sax wrote: I am just
>  getting aware of this KIP (not sure why I missed it).
> 
>  In Kafka Streams we have nested clients and need to "forward"
>  configs from outer layer to inner layers -- hence, we prefix some
>  configs to be able to know which inner nested clients needs this
>  config.
> 
>  I think the simplest approach is, to add a prefix (like
>  "userconfig."). All thus configs would be skipped in the
>  validation step to avoid the WARN log.
> 
>  When forwarding configs to inner classed (like nested clients in
>  KS, serializers etc) we would remove this prefix).
> 
>  Using a `RecordingMap` seem rather heavy weight and complex?
> 
>  Thoughts?
> 
>  -Matthias
> 
>  On 2/17/20 9:09 AM, John Roesler wrote:
> >>> Thanks Patrik,
> >>>
> >>> This seems to be a long and wandering issue. It seems that
> >>> KAFKA-7509 has followed a similar trajectory to
> >>> KAFKA-6793/KIP-552 , and 7509 is just recently closed in
> >>> favor of whatever we decide to do in KAFKA-6793.
> >>>
> >>> Considering (what I hope is) the whole history of this issue,
> >>> a few things emerge:
> >>>
> >>> 1. It's useful to get warned when you pass an invalid
> >>> configuration 2. It's not possible for the "top layer"
> >>> (Streams, Connect, etc.) to know up front which
> >>> configurations are applicable to pass down to the "second"
> >>> layer (Clients, RocksDB) because those layers themselves are
> >>> extensible (see below) 3. We should propose a change that
> >>> fixes this issue for the whole Kafka ecosystem at once.
> >>>
> >>> Elaboration on point 2: Users of Kafka libraries need to
> >>> register extra components like Processors, Interceptors,
> >>> RocksDBConfigSetters, RebalanceListeners, etc. They need to
> >>> pass configurations into these self-registered components.
> >>> Therefore, the outermost component (the one that you directly
> >>> pass a Properties to, and that instantiates other
> >>> Configurable components) _cannot_ know which configurations
> >>> are needed by the "extra" components inside the Configurable
> >>> components. Therefore, no approach that involves filtering
> >>> only the "needed" configurations up front, before
> >>> constructing a Configurable component, could work.
> >>>
> >>> Randall made an aside in this comment:
> >>>
> https://issues.apache.org/jira/browse/KAFKA-7509?focu

[jira] [Created] (KAFKA-9710) Windows kafka-server-start.bat script fails when Kafka is located in path with whitespace

2020-03-12 Thread Tomas Kovan (Jira)
Tomas Kovan created KAFKA-9710:
--

 Summary: Windows kafka-server-start.bat script fails when Kafka is 
located in path with whitespace 
 Key: KAFKA-9710
 URL: https://issues.apache.org/jira/browse/KAFKA-9710
 Project: Kafka
  Issue Type: Bug
  Components: tools
Affects Versions: 2.4.0
Reporter: Tomas Kovan
 Fix For: 2.4.0
 Attachments: kafka_image.png

*Steps to reproduce:*

On Windows when you place Kafka to directory path with space, for example 
"C:\kafka path with whitespace" in it and run 
".\bin\windows\kafka-server-start.bat", script will fail with error message 
"Error: Could not find or load main class path" (example in image). 

*Possible solution:*

Working solution is to remove double quotes around %CLASSPATH% variable in 
"kafka-run-class.bat" since all paths included in %CLASSPATH% variable already 
includes double quotes around them. So changing line 179 in file 
"kafka-run-class.bat" 
FROM:
{code:java}
set COMMAND=%JAVA% %KAFKA_HEAP_OPTS% %KAFKA_JVM_PERFORMANCE_OPTS% 
%KAFKA_JMX_OPTS% %KAFKA_LOG4J_OPTS% -cp "%CLASSPATH%" %KAFKA_OPTS% %*
{code}
TO:
{code:java}
set COMMAND=%JAVA% %KAFKA_HEAP_OPTS% %KAFKA_JVM_PERFORMANCE_OPTS% 
%KAFKA_JMX_OPTS% %KAFKA_LOG4J_OPTS% -cp %CLASSPATH% %KAFKA_OPTS% %*
{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)