[DISCUSS] KIP-303: Add Dynamic Routing in Streams Sink

2018-05-14 Thread Guozhang Wang
Hello folks,

I'd like to start a discussion on adding dynamic routing functionality in
Streams sink node. I.e. users do not need to specify the topic name at
compilation time but can dynamically determine which topic to send to based
on each record's key value pairs. Please find a KIP here:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-303%3A+Add+Dynamic+Routing+in+Streams+Sink

Any feedbacks are highly appreciated.

Thanks!

-- Guozhang


Re: Exactly Once, Producer configuration

2018-05-14 Thread Matthias J. Sax
Hi,

I don't think that the `producer.properties` file exist once per broker,
because this file should not be at the broker machine at all. It should
be on the application side (ie, your "modules").

For transactions to work correctly, each `Producer` instance needs a
cluster wide unique transactional.id.

I hope this answers your questions.

Btw: you might also be interested in this two blog posts:

- https://www.confluent.io/blog/transactions-apache-kafka/
- https://www.confluent.io/blog/enabling-exactly-kafka-streams/


-Matthias

On 5/14/18 7:26 AM, thomas.k...@t-systems.com wrote:
> Hi,
> 
> the following article describes how to configure Apache Kafka system to 
> enforce exactly once approach:
> 
> https://www.confluent.io/blog/exactly-once-semantics-are-possible-heres-how-apache-kafka-does-it/
> 
> One recommendation states that parameter transactional.id should be set in 
> producer.properties file which usually exists only once per broker.
> 
> Now I have the following questions. Our system consists of 4 modules which 
> interact in a sequential way by exchanging messages using Kafka topics
> 
> 
> A --> B --> C --> D
> 
> I.e. modules A, B and C act as message producers and modules B, C and D act 
> as message consumers (and all modules do some application specific useful 
> processing).
> 
> 1)  Do we need separate producer/consumer configurations for these 
> modules and if so where and how do we have to configure them (as producer 
> configuration files are related to the number of brokers and not to the 
> number of modules)?
> 2)  Does transcational.id "only" cover technical aspects of Apache Kafka 
> or does it cover application aspects as well (i.e. is my assumption which led 
> to question 1 correct)?
> 3)  In case it is sufficient to configure transactional.id parameter in 
> producer.properties files and we have more than one broker (and with that 
> more than one producer.properties file). Do the values of the parameter 
> transactional.id have to be equal or do there have to be different (pairwise 
> disjoint) values?
> 
> I would be glad to get answers to these questions.
> 
> Kind regards,
> Thomas Kass
> 
> 
> 
> 
> 
> 



signature.asc
Description: OpenPGP digital signature


Re: Kafka system tests contribution

2018-05-14 Thread Matthias J. Sax
Hard to say why it fails. Note, that some system tests a flaky (we are
working constantly to stabilize them)

What helps debugging is to look into the ducktape log itself. You can
find it in the file `test_log.debug` that is written into
`resutls/`

Also inspect the actual test log files, as well as stdout and stderr files.

Unfortunately, I am not aware of a technique that allows you to set a
break point in a system test.


-Matthias


On 5/8/18 5:01 AM, Andriy Sorokhtey wrote:
> Hello Kafka team
> 
> I’d like to contribute to the Kafka system tests.
> 
> I’ve tried to execute system tests locally and I have some issues. Can
> anyone give me a hand to figure out what’s wrong?
> 
> So, I see that multiple system tests are failing when I try to run it with
> the docker or with vagrant.
> Maybe there is some way to debug it using PyCharm. For example, put some
> breakpoint and start debugging, when the test goes to the breakpoint I’d
> like to go to instances and check what’s going on there.
> I’ll be thankful for any advice.
> 
>  Here is an example of one test failure:
> 
>> [INFO:2018-05-03 06:37:19,861]: Triggering test 1 of 37...
>> [INFO:2018-05-03 06:37:19,870]: RunnerClient: Loading test {'directory':
>> '/opt/kafka-dev/tests/kafkatest/tests/streams', 'file_name':
>> 'streams_broker_compatibility_test.py', 'method_name':
>> 'test_compatible_brokers_eos_disabled', 'cls_name':
>> 'StreamsBrokerCompatibility', 'injected_args': {'broker_version':
>> '0.10.1.1'}}
>> [INFO:2018-05-03 06:37:19,874]: RunnerClient:
>> kafkatest.tests.streams.streams_broker_compatibility_test.StreamsBrokerCompatibility.test_compatible_brokers_eos_disabled.broker_version=
>> 0.10.1.1: Setting up...
>> [INFO:2018-05-03 06:37:22,484]: RunnerClient:
>> kafkatest.tests.streams.streams_broker_compatibility_test.StreamsBrokerCompatibility.test_compatible_brokers_eos_disabled.broker_version=
>> 0.10.1.1: Running...
>> [INFO:2018-05-03 06:38:34,129]: RunnerClient:
>> kafkatest.tests.streams.streams_broker_compatibility_test.StreamsBrokerCompatibility.test_compatible_brokers_eos_disabled.broker_version=
>> 0.10.1.1: FAIL: Never saw message indicating StreamsTest finished startup
>> on ducker@ducker05
>> Traceback (most recent call last):
>> File
>> "/usr/local/lib/python2.7/dist-packages/ducktape/tests/runner_client.py",
>> line 132, in run
>> data = self.run_test()
>> File
>> "/usr/local/lib/python2.7/dist-packages/ducktape/tests/runner_client.py",
>> line 185, in run_test
>> return self.test_context.function(self.test)
>> File "/usr/local/lib/python2.7/dist-packages/ducktape/mark/_mark.py", line
>> 324, in wrapper
>> return functools.partial(f, *args, **kwargs)(*w_args, **w_kwargs)
>> File
>> "/opt/kafka-dev/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py",
>> line 84, in test_compatible_brokers_eos_disabled
>> processor.start()
>> File
>> "/usr/local/lib/python2.7/dist-packages/ducktape/services/service.py", line
>> 234, in start
>> self.start_node(node)
>> File "/opt/kafka-dev/tests/kafkatest/services/streams.py", line 138, in
>> start_node
>> monitor.wait_until('StreamsTest instance started', timeout_sec=60,
>> err_msg="Never saw message indicating StreamsTest finished startup on " +
>> str(node.account))
>> File
>> "/usr/local/lib/python2.7/dist-packages/ducktape/cluster/remoteaccount.py",
>> line 668, in wait_until
>> allow_fail=True) == 0, **kwargs)
>> File "/usr/local/lib/python2.7/dist-packages/ducktape/utils/util.py", line
>> 36, in wait_until
>> raise TimeoutError(err_msg)
>> TimeoutError: Never saw message indicating StreamsTest finished startup on
>> ducker@ducker05
> 
> 
> If I figure out what's wrong I can try to fix other tests.
> 



signature.asc
Description: OpenPGP digital signature


Re: Can anyone take a look at this KIP and Jira?

2018-05-14 Thread Matthias J. Sax
I agree with Ted. This change does not require a KIP. If ok with you, I
can delete the KIP page in the Wiki for you.

-Matthias


On 5/10/18 11:59 AM, Ted Yu wrote:
> Since the change is internal to *SensorAccess class, looks like KIP is not
> required.*
> 
> On Thu, May 10, 2018 at 11:54 AM, qingjun wu  wrote:
> 
>> Dear Kafka Developers,
>>
>> I opened a KIP and also a Jira ticket related to this. Can you please take
>> a look? It should be simple change to Kafka, but it should improve the
>> performance a lot.
>>
>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-NEXT%3A+Get+rid+of+
>> unnecessary+read+lock
>>
>>
>> https://issues.apache.org/jira/browse/KAFKA-6722
>>
>>
>> --
>> Best Regards
>>  吴清俊|Wade Wu
>>
> 



signature.asc
Description: OpenPGP digital signature


Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Matthias J. Sax
+1 (binding)

Thanks a lot for the KIP!

-Matthias

On 5/14/18 10:17 AM, Guozhang Wang wrote:
> +1 from me
> 
> One more comment on the wiki: while reviewing the PR I realized that in `
> MockProcessorContext.java
> `
> we are also adding one additional API plus modifying the existing
> `setRecordMetadata` API. Since this class is part of the public test-utils
> package we should claim it in the wiki as well.
> 
> 
> Guozhang
> 
> On Mon, May 14, 2018 at 8:43 AM, Ted Yu  wrote:
> 
>> +1
>>
>> On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya <
>> quilcate.jo...@gmail.com> wrote:
>>
>>> Hi everyone,
>>>
>>> I would like to start a vote on KIP-244: Add Record Header support to
>> Kafka
>>> Streams
>>>
>>> KIP wiki page:
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 244%3A+Add+Record+Header+support+to+Kafka+Streams+Processor+API
>>>
>>> The discussion thread is here:
>>> http://mail-archives.apache.org/mod_mbox/kafka-dev/201805.
>>> mbox/%3CCAC3UcJvrgcBfe6%3DiW6%2BuTWsLB%2B4CsHgRmDx9TvCzJQrWvfg7_w%
>>> 40mail.gmail.com%3E
>>>
>>> Cheers,
>>> Jorge.
>>>
>>
> 
> 
> 



signature.asc
Description: OpenPGP digital signature


Re: [VOTE] KIP-255: OAuth Authentication via SASL/OAUTHBEARER

2018-05-14 Thread Ron Dagostino
Hi Jun.  Thanks for the +1 vote.

Regarding the first question about token claims, yes, you have it correct
about translating the OAuth token to a principle name via a JAAS module
option in the default unsecured case.  Specifically, the OAuth SASL Server
implementation is responsible for setting the authorization ID, and it gets
the authorization ID from the OAuthBearerToken's principalName() method.
The listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class
is responsible for handling an instance of OAuthBearerValidatorCallback to
accept a token compact serialization from the client and return an instance
of OAuthBearerToken (assuming the compact serialization validates), and in
the default unsecured case the builtin unsecured validator callback handler
defines the OAuthBearerToken.principalName() method to return the 'sub'
claim value by default (with the actual claim it uses being configurable
via the unsecuredValidatorPrincipalClaimName JAAS module option).  So that
is how we translate from a token to a principal name in the default
unsecured (out-of-the-box) case.

For production use cases, the implementation associated with
listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class can
do whatever it wants.  As an example, I have written a class that wraps a
com.nimbusds.jwt.SignedJWT instance (see
https://connect2id.com/products/nimbus-jose-jwt/) and presents it as an
OAuthBearerToken:

public class NimbusSignedJwtOAuthBearerToken implements OAuthBearerToken {
private final SignedJWT signedJwt;
private final String principalName;
private final Set scope;
private final Long startTimeMs;
private final long lifetimeMs;

/**
 * Constructor
 *
 * @param signedJwt
 *the mandatory signed JWT
 * @param principalClaimName
 *the mandatory claim name identifying the claim from which
the
 *principal name will be extracted. The claim must exist
and must be
 *a String.
 * @param optionalScopeClaimName
 *the optional claim name identifying the claim from which
any scope
 *will be extracted. If specified and the claim exists then
the
 *value must be either a String or a String List.
 * @throws ParseException
 * if the principal claim does not exist or is not a
String; the
 * scope claim is neither a String nor a String List; the
'exp'
 * claim does not exist or is not a number; the 'iat' claim
exists
 * but is not a number; or the 'nbf' claim exists and is
not a
 * number.
 */
public NimbusSignedJwtOAuthBearerToken(SignedJWT signedJwt, String
principalClaimName,
String optionalScopeClaimName) throws ParseException {
// etc...
}

The callback handler runs the following code if the digital signature
validates:

callback.token(new NimbusSignedJwtOAuthBearerToken(signedJwt, "sub",
null));

I hope that answers the first question.  If not let me know what I
missed/misunderstood and I'll be glad to try to address it.

Regarding the second question, the classes OAuthBearerTokenCallback and
OAuthBearerValidatorCallback implement the Callback interface -- they are
the callbacks that the AuthenticateCallbackHandler implementations need to
handle.  Specifically, unless the unsecured functionality is what is
desired, the two configuration values [listener.name.sasl_ssl.oauthbearer.
]sasl.login.callback.handler.class and
listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class
define the callback handlers that need to handle OAuthBearerTokenCallback
and OAuthBearerValidatorCallback, respectively.

Regarding the third question, yes, I see your point that the way the spec
is worded could be taken to imply that the error code is a single
character: "A single ASCII..." (
https://tools.ietf.org/html/rfc6749#section-5.2).  However, it is not a
single character.  See the end of that section 5.2 for an example that
shows "error":"invalid_request" as the response.

Thanks again for the +1 vote, Jun, and please do let me know if I can cover
anything else.

Ron


On Mon, May 14, 2018 at 7:10 PM, Jun Rao  wrote:

> Hi, Ron,
>
> Thanks for the KIP. +1 from me. Just a few minor comments below.
>
> 1. It seems that we can translate an OAuth token to a principle name
> through the claim name configured in JASS. However, it's not clear to me
> how an OAuth token is mapped to a claim. Could you clarify that?
>
> 2. The wiki has the following code. It seems that OAuthBearerTokenCallback
> should implement AuthenticateCallbackHandler? Ditto
> for OAuthBearerValidatorCallback.
>
> public class OAuthBearerTokenCallback implements Callback
>
> 3. In OAuthBearerTokenCallback, we have the following method. The OAuth
> spec says the error code is a single ASCII. So, should we return a Char or
> a String?
>
> public String 

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-14 Thread Piyush Vijay
Colin,

createAcls take a AclBinding, however, instead of AclBindingFilter. What
are your thoughts here?

public abstract DescribeAclsResult describeAcls(AclBindingFilter
filter, DescribeAclsOptions options);

public abstract CreateAclsResult createAcls(Collection
acls, CreateAclsOptions options);

public abstract DeleteAclsResult
deleteAcls(Collection filters, DeleteAclsOptions
options);


Thanks

Piyush Vijay

On Mon, May 14, 2018 at 9:26 AM, Andy Coates  wrote:

> +1
>
> On 11 May 2018 at 17:14, Colin McCabe  wrote:
>
> > Hi Andy,
> >
> > I see what you mean.  I guess my thought here is that if the fields are
> > private, we can change it later if we need to.
> >
> > I definitely agree that we should use the scheme you describe for sending
> > ACLs over the wire (just the string + version number)
> >
> > cheers,
> > Colin
> >
> >
> > On Fri, May 11, 2018, at 09:39, Andy Coates wrote:
> > > i think I'm agreeing with you. I was merely suggesting that having an
> > > additional field that controls how the current field is interpreted is
> > more
> > > flexible / extensible in the future than using a 'union' style
> approach,
> > > where only one of several possible fields should be populated. But
> it's a
> > > minor thing.
> > >
> > >
> > >
> > >
> > >
> > >
> > > On 10 May 2018 at 09:29, Colin McCabe  wrote:
> > >
> > > > Hi Andy,
> > > >
> > > > The issue that I was trying to solve here is the Java API.  Right
> now,
> > > > someone can write "new ResourceFilter(ResourceType.TRANSACTIONAL_ID,
> > > > "foo*") and have a ResourceFilter that applies to a Transactional ID
> > named
> > > > "foo*".  This has to continue to work, or else we have broken
> > compatibility.
> > > >
> > > > I was proposing that there would be something like a new function
> like
> > > > ResourceFilter.fromPattern(ResourceType.TRANSACTIONAL_ID, "foo*")
> > which
> > > > would create a ResourceFilter that applied to transactional IDs
> > starting
> > > > with "foo", rather than transactional IDs named "foo*" specifically.
> > > >
> > > > I don't think it's important whether the Java class has an integer,
> an
> > > > enum, or two string fields.  The important thing is that there's a
> new
> > > > static function, or new constructor overload, etc. that works for
> > patterns
> > > > rather than literal strings.
> > > >
> > > > On Thu, May 10, 2018, at 03:30, Andy Coates wrote:
> > > > > Rather than having name and pattern fields on the ResourceFilter,
> > where
> > > > > it’s only valid for one to be set, and we want to restrict the
> > character
> > > > > set in case future enhancements need them, we could instead add a
> new
> > > > > integer ‘nameType’ field, and use constants to indicate how the
> name
> > > > > field should be interpreted, e.g. 0 = literal, 1 = wildcard. This
> > would
> > > > > be extendable, e.g we can later add 2 = regex, or what ever, and
> > > > > wouldn’t require any escaping.
> > > >
> > > > This is very user-unfriendly, though.  Users don't want to have to
> > > > explicitly supply a version number when using the API, which is what
> > this
> > > > would force them to do.  I don't think users are going to want to
> > memorize
> > > > that version 4 supprted "+", whereas version 3 only supported
> "[0-9]",
> > or
> > > > whatever.
> > > >
> > > > Just as an example, do you remember which versions of FetchRequest
> > added
> > > > which features?  I don't.  I always have to look at the code to
> > remember.
> > > >
> > > > Also, escaping is still required any time you overload a character to
> > mean
> > > > two things.  Escaping is required in the current proposal to be able
> to
> > > > create a pattern that matches only "foo*".  You have to type "foo\*"
> > It
> > > > would be required if we forced users to specify a version, as well.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > > >
> > > > > Sent from my iPhone
> > > > >
> > > > > > On 7 May 2018, at 05:16, Piyush Vijay 
> > wrote:
> > > > > >
> > > > > > Makes sense. I'll update the KIP.
> > > > > >
> > > > > > Does anyone have any other comments? :)
> > > > > >
> > > > > > Thanks
> > > > > >
> > > > > >
> > > > > > Piyush Vijay
> > > > > >
> > > > > >> On Thu, May 3, 2018 at 11:55 AM, Colin McCabe <
> cmcc...@apache.org
> > >
> > > > wrote:
> > > > > >>
> > > > > >> Yeah, I guess that's a good point.  It probably makes sense to
> > > > support the
> > > > > >> prefix scheme for consumer groups and transactional IDs as well
> as
> > > > topics.
> > > > > >>
> > > > > >> I agree that the current situation where anything goes in
> consumer
> > > > group
> > > > > >> names and transactional ID names is not ideal.  I wish we could
> > > > rewind the
> > > > > >> clock and impose restrictions on the names.  However, it doesn't
> > seem
> > > > > >> practical at the moment.  Adding new restrictions would break a
> > lot of
> > > > > >> existing users after an upgrade.  

Re: [VOTE] KIP-282: Add the listener name to the authentication context

2018-05-14 Thread Jun Rao
Hi, Mickael,

Thanks for the KIP. +1

Jun

On Wed, Apr 25, 2018 at 1:51 AM, Mickael Maison 
wrote:

> Hi,
>
> There has been no objections in the DISCUSS thread so I'd like to
> start a vote on KIP-282:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 282%3A+Add+the+listener+name+to+the+authentication+context
>
> Thanks
>


Jenkins build is back to normal : kafka-0.11.0-jdk7 #368

2018-05-14 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-6904) DynamicBrokerReconfigurationTest#testAdvertisedListenerUpdate is flaky

2018-05-14 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-6904:
-

 Summary: 
DynamicBrokerReconfigurationTest#testAdvertisedListenerUpdate is flaky
 Key: KAFKA-6904
 URL: https://issues.apache.org/jira/browse/KAFKA-6904
 Project: Kafka
  Issue Type: Test
Reporter: Ted Yu


>From 
>https://builds.apache.org/job/kafka-pr-jdk10-scala2.12/820/testReport/junit/kafka.server/DynamicBrokerReconfigurationTest/testAdvertisedListenerUpdate/
> :
{code}
kafka.server.DynamicBrokerReconfigurationTest.testAdvertisedListenerUpdate

Failing for the past 1 build (Since Failed#820 )
Took 21 sec.
Error Message
java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition.
Stacktrace
java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.NotLeaderForPartitionException: This server is 
not the leader for that topic-partition.
at 
org.apache.kafka.clients.producer.internals.FutureRecordMetadata.valueOrError(FutureRecordMetadata.java:94)
at 
org.apache.kafka.clients.producer.internals.FutureRecordMetadata.get(FutureRecordMetadata.java:77)
at 
org.apache.kafka.clients.producer.internals.FutureRecordMetadata.get(FutureRecordMetadata.java:29)
at 
kafka.server.DynamicBrokerReconfigurationTest.$anonfun$verifyProduceConsume$3(DynamicBrokerReconfigurationTest.scala:996)
at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:234)
at scala.collection.Iterator.foreach(Iterator.scala:944)
at scala.collection.Iterator.foreach$(Iterator.scala:944)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1432)
at scala.collection.IterableLike.foreach(IterableLike.scala:71)
at scala.collection.IterableLike.foreach$(IterableLike.scala:70)
at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
at scala.collection.TraversableLike.map(TraversableLike.scala:234)
at scala.collection.TraversableLike.map$(TraversableLike.scala:227)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at 
kafka.server.DynamicBrokerReconfigurationTest.verifyProduceConsume(DynamicBrokerReconfigurationTest.scala:996)
at 
kafka.server.DynamicBrokerReconfigurationTest.testAdvertisedListenerUpdate(DynamicBrokerReconfigurationTest.scala:742)
{code}
The above happened with jdk 10.





--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: [VOTE] KIP-255: OAuth Authentication via SASL/OAUTHBEARER

2018-05-14 Thread Jun Rao
Hi, Ron,

Thanks for the KIP. +1 from me. Just a few minor comments below.

1. It seems that we can translate an OAuth token to a principle name
through the claim name configured in JASS. However, it's not clear to me
how an OAuth token is mapped to a claim. Could you clarify that?

2. The wiki has the following code. It seems that OAuthBearerTokenCallback
should implement AuthenticateCallbackHandler? Ditto
for OAuthBearerValidatorCallback.

public class OAuthBearerTokenCallback implements Callback

3. In OAuthBearerTokenCallback, we have the following method. The OAuth
spec says the error code is a single ASCII. So, should we return a Char or
a String?

public String errorCode()

Jun


On Thu, May 3, 2018 at 8:55 PM, Ron Dagostino  wrote:

> Hi everyone.  I would like to start the vote for KIP-255:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=75968876
>
> This KIP proposes to add the following functionality related to
> SASL/OAUTHBEARER:
>
> 1) Allow clients (both brokers when SASL/OAUTHBEARER is the inter-broker
> protocol as well as non-broker clients) to flexibly retrieve an access
> token from an OAuth 2 authorization server based on the declaration of a
> custom login CallbackHandler implementation and have that access token
> transparently and automatically transmitted to a broker for authentication.
>
> 2) Allow brokers to flexibly validate provided access tokens when a client
> establishes a connection based on the declaration of a custom SASL Server
> CallbackHandler implementation.
>
> 3) Provide implementations of the above retrieval and validation features
> based on an unsecured JSON Web Token that function out-of-the-box with
> minimal configuration required (i.e. implementations of the two types of
> callback handlers mentioned above will be used by default with no need to
> explicitly declare them).
>
> 4) Allow clients (both brokers when SASL/OAUTHBEARER is the inter-broker
> protocol as well as non-broker clients) to transparently retrieve a new
> access token in the background before the existing access token expires in
> case the client has to open new connections.
>
> Thanks,
>
> Ron
>


[jira] [Resolved] (KAFKA-5907) Support aggregatedJavadoc in Java 9

2018-05-14 Thread Ismael Juma (JIRA)

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

Ismael Juma resolved KAFKA-5907.

Resolution: Fixed

[~omkreddy] It does seem to work now, so will mark it as resolved.

> Support aggregatedJavadoc in Java 9
> ---
>
> Key: KAFKA-5907
> URL: https://issues.apache.org/jira/browse/KAFKA-5907
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ismael Juma
>Priority: Major
> Fix For: 2.0.0
>
>
> The Java 9 Javadoc tool has some improvements including a search bar. 
> However, it currently fails with a number of errors like:
> {code}
> > Task :aggregatedJavadoc
> /Users/ijuma/src/kafka/streams/src/main/java/org/apache/kafka/streams/Topology.java:29:
>  error: package org.apache.kafka.streams.processor.internals does not exist
> import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
>^
> /Users/ijuma/src/kafka/streams/src/main/java/org/apache/kafka/streams/Topology.java:30:
>  error: package org.apache.kafka.streams.processor.internals does not exist
> import org.apache.kafka.streams.processor.internals.ProcessorNode;
>^
> /Users/ijuma/src/kafka/streams/src/main/java/org/apache/kafka/streams/Topology.java:31:
>  error: package org.apache.kafka.streams.processor.internals does not exist
> import org.apache.kafka.streams.processor.internals.ProcessorTopology;
>^
> /Users/ijuma/src/kafka/streams/src/main/java/org/apache/kafka/streams/Topology.java:32:
>  error: package org.apache.kafka.streams.processor.internals does not exist
> import org.apache.kafka.streams.processor.internals.SinkNode;
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Andy Coates
Hey all,

With 3 binding votes and 1 non-binding, this KIP has been accepted.

Thanks for everyones input and time.

Andy



On 14 May 2018 at 09:42, Guozhang Wang  wrote:

> Thanks Andy. That's fine, we can continue the vote thread.
>
> On Mon, May 14, 2018 at 9:25 AM, Andy Coates  wrote:
>
> > Hey All,
> >
> > I've updated the KIP to also cover the Scala API for completeness. (The
> PR
> > already covered this).
> >
> > Does this require another vote?
> >
> > On 11 May 2018 at 14:29, Matthias J. Sax  wrote:
> >
> > > +1 (binding)
> > >
> > >
> > > On 5/11/18 2:17 PM, Ted Yu wrote:
> > > > +1
> > > >
> > > > On Fri, May 11, 2018 at 2:16 PM, Guozhang Wang 
> > > wrote:
> > > >
> > > >> +1 (binding)
> > > >>
> > > >> Also slightly changed the title beyond changing [DISCUSS] to [VOTE]
> > > since
> > > >> Gmail will collapse these two threads into one, so people may not
> > > realize
> > > >> there is a voting started already.
> > > >>
> > > >>
> > > >> Guozhang
> > > >>
> > > >>
> > > >> On Fri, May 11, 2018 at 2:07 PM, Andy Coates 
> > wrote:
> > > >>
> > > >>> Hi all,
> > > >>>
> > > >>> I would like to start the vote on KIP-292: Add transformValues()
> > method
> > > >> to
> > > >>> KTable
> > > >>>
> > > >>> The link to this KIP is here:
> > > >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >>> 292%3A+Add+transformValues%28%29+method+to+KTable
> > > >>>
> > > >>> The discussion thread is here:
> > > >>> http://mail-archives.apache.org/mod_mbox/kafka-dev/201805.mbox/%
> > > >>> 3CCAO1RfpHyhEEwkwad5yxd0rLgPEK-PJ4y0%2B0y%3DwidHP0uOtToJQ%
> > > >>> 40mail.gmail.com%3E
> > > >>>
> > > >>> Thanks,
> > > >>>
> > > >>> Andy
> > > >>>
> > > >>
> > > >>
> > > >>
> > > >> --
> > > >> -- Guozhang
> > > >>
> > > >
> > >
> > >
> >
>
>
>
> --
> -- Guozhang
>


[jira] [Created] (KAFKA-6903) Improve KTable's sending old value behavior

2018-05-14 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6903:


 Summary: Improve KTable's sending old value behavior
 Key: KAFKA-6903
 URL: https://issues.apache.org/jira/browse/KAFKA-6903
 Project: Kafka
  Issue Type: Improvement
  Components: streams
Reporter: Guozhang Wang


Today in KTable's internal implementation, if old values are needed in the down 
stream (e.g. if there is an aggregation down stream so that old values need to 
be re-send to "subtract" its effects in addition to incorporate the effects of 
new values), we will re-compute the old values based on the parent's passed in 
old values. This behavior has two issues:

1) re-computing the values again means more cost: for each updated value, they 
are computed twice, once as the new value and once as the old value. This 
additional cost can ideally be saved.

2) if the computational logic is dependent on some state which could be updated 
over time, then calling the same applied function again may actually result in 
different values, due to the different state's snapshot.

We should consider how to improve this behavior to avoid the above issues.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: [VOTE] KIP-275 - Indicate "isClosing" in the SinkTaskContext

2018-05-14 Thread Matt Farmer
Bumping this thread.

For anyone who needs a refresher the discussion thread is here:
http://mail-archives.apache.org/mod_mbox/kafka-dev/201803.mbox/%3CCAM5dya9x---9M3uEf_wrJL5dw%2B6HLV4%3D5PfKKSTPE1vOHEWC_g%40mail.gmail.com%3E

And there's a work in progress PR open here:
https://github.com/apache/kafka/pull/5002

Thanks!

On Wed, Apr 25, 2018 at 1:04 PM, Matt Farmer  wrote:

> Bump!
>
> We're currently at 1 non-binding +1.
>
> Still soliciting votes here. =)
>
> On Wed, Apr 18, 2018 at 3:41 PM, Ted Yu  wrote:
>
>> +1
>>
>> On Wed, Apr 18, 2018 at 12:40 PM, Matt Farmer  wrote:
>>
>> > Good afternoon/evening/morning all:
>> >
>> > I'd like to start voting on KIP-275: Indicate "isClosing" in the
>> > SinkTaskContext
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?
>> pageId=75977607
>> >
>> > I'm going to start preparing the patch we've been using internally for
>> PR
>> > and get it up for review later this week.
>> >
>> > Thanks!
>> > Matt
>> >
>>
>
>


Re: [DISCUSS] KIP-298: Error Handling in Connect

2018-05-14 Thread Matt Farmer
Hi Arjun,

I'm following this very closely as better error handling in Connect is a
high priority
for MailChimp's Data Systems team.

A few thoughts (in no particular order):

For the dead letter queue configuration, could we use deadLetterQueue
instead of
dlq? Acronyms are notoriously hard to keep straight in everyone's head and
unless
there's a compelling reason it would be nice to use the characters and be
explicit.

Have you considered any behavior that would periodically attempt to restart
failed
tasks after a certain amount of time? To get around our issues internally
we've
deployed a tool that monitors for failed tasks and restarts the task by
hitting the
REST API after the failure. Such a config would allow us to get rid of this
tool.

Have you considered a config setting to allow-list additional classes as
retryable? In the situation we ran into, we were getting ConnectExceptions
that
were intermittent due to an unrelated service. With such a setting we could
have
deployed a config that temporarily whitelisted that Exception as
retry-worthy
and continued attempting to make progress while the other team worked
on mitigating the problem.

Thanks for the KIP!

On Wed, May 9, 2018 at 2:59 AM, Arjun Satish  wrote:

> All,
>
> I'd like to start a discussion on adding ways to handle and report record
> processing errors in Connect. Please find a KIP here:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 298%3A+Error+Handling+in+Connect
>
> Any feedback will be highly appreciated.
>
> Thanks very much,
> Arjun
>


[DISCUSS] KIP-301 Schema Inferencing in JsonConverters

2018-05-14 Thread Allen Tang
Hi,

I just opened a KIP to add Schema Inferencing in JsonConverters for
Kafka Connect.

The details of the proposal can be found here:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-301%3A+Schema+Inferencing+for+JsonConverter

Also, I have created a -

1.) JIRA ticket: https://issues.apache.org/jira/browse/KAFKA-6895

2.) Provisional PR with initial discussion:
https://github.com/apache/kafka/pull/5001

Looking forward to the community's feedback! Cheers!

-Allen


[jira] [Created] (KAFKA-6902) During replica reassignment if source broker is down the partition directory may not be removed from broker log.dirs

2018-05-14 Thread Koelli Mungee (JIRA)
Koelli Mungee created KAFKA-6902:


 Summary: During replica reassignment if source broker is down the 
partition directory may not be removed from broker log.dirs
 Key: KAFKA-6902
 URL: https://issues.apache.org/jira/browse/KAFKA-6902
 Project: Kafka
  Issue Type: Bug
  Components: core
Affects Versions: 0.11.0.2
Reporter: Koelli Mungee


During replica reassignment if source broker is down the partition directory 
may not be removed from the source broker's log.dirs location. This can lead to 
the partition directory still being there with zero length log data. This in 
turn leads to this partition showing up in JMX metric for the original broker: 
kafka.log.LogEndOffset even though this partition is no longer a part of the 
ISR list in that broker.

It would be good to have this cleaned up so the metric can be accurate at all 
times.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Guozhang Wang
+1 from me

One more comment on the wiki: while reviewing the PR I realized that in `
MockProcessorContext.java
`
we are also adding one additional API plus modifying the existing
`setRecordMetadata` API. Since this class is part of the public test-utils
package we should claim it in the wiki as well.


Guozhang

On Mon, May 14, 2018 at 8:43 AM, Ted Yu  wrote:

> +1
>
> On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya <
> quilcate.jo...@gmail.com> wrote:
>
> > Hi everyone,
> >
> > I would like to start a vote on KIP-244: Add Record Header support to
> Kafka
> > Streams
> >
> > KIP wiki page:
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 244%3A+Add+Record+Header+support+to+Kafka+Streams+Processor+API
> >
> > The discussion thread is here:
> > http://mail-archives.apache.org/mod_mbox/kafka-dev/201805.
> > mbox/%3CCAC3UcJvrgcBfe6%3DiW6%2BuTWsLB%2B4CsHgRmDx9TvCzJQrWvfg7_w%
> > 40mail.gmail.com%3E
> >
> > Cheers,
> > Jorge.
> >
>



-- 
-- Guozhang


Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Guozhang Wang
Thanks Andy. That's fine, we can continue the vote thread.

On Mon, May 14, 2018 at 9:25 AM, Andy Coates  wrote:

> Hey All,
>
> I've updated the KIP to also cover the Scala API for completeness. (The PR
> already covered this).
>
> Does this require another vote?
>
> On 11 May 2018 at 14:29, Matthias J. Sax  wrote:
>
> > +1 (binding)
> >
> >
> > On 5/11/18 2:17 PM, Ted Yu wrote:
> > > +1
> > >
> > > On Fri, May 11, 2018 at 2:16 PM, Guozhang Wang 
> > wrote:
> > >
> > >> +1 (binding)
> > >>
> > >> Also slightly changed the title beyond changing [DISCUSS] to [VOTE]
> > since
> > >> Gmail will collapse these two threads into one, so people may not
> > realize
> > >> there is a voting started already.
> > >>
> > >>
> > >> Guozhang
> > >>
> > >>
> > >> On Fri, May 11, 2018 at 2:07 PM, Andy Coates 
> wrote:
> > >>
> > >>> Hi all,
> > >>>
> > >>> I would like to start the vote on KIP-292: Add transformValues()
> method
> > >> to
> > >>> KTable
> > >>>
> > >>> The link to this KIP is here:
> > >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >>> 292%3A+Add+transformValues%28%29+method+to+KTable
> > >>>
> > >>> The discussion thread is here:
> > >>> http://mail-archives.apache.org/mod_mbox/kafka-dev/201805.mbox/%
> > >>> 3CCAO1RfpHyhEEwkwad5yxd0rLgPEK-PJ4y0%2B0y%3DwidHP0uOtToJQ%
> > >>> 40mail.gmail.com%3E
> > >>>
> > >>> Thanks,
> > >>>
> > >>> Andy
> > >>>
> > >>
> > >>
> > >>
> > >> --
> > >> -- Guozhang
> > >>
> > >
> >
> >
>



-- 
-- Guozhang


Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-14 Thread Andy Coates
+1

On 11 May 2018 at 17:14, Colin McCabe  wrote:

> Hi Andy,
>
> I see what you mean.  I guess my thought here is that if the fields are
> private, we can change it later if we need to.
>
> I definitely agree that we should use the scheme you describe for sending
> ACLs over the wire (just the string + version number)
>
> cheers,
> Colin
>
>
> On Fri, May 11, 2018, at 09:39, Andy Coates wrote:
> > i think I'm agreeing with you. I was merely suggesting that having an
> > additional field that controls how the current field is interpreted is
> more
> > flexible / extensible in the future than using a 'union' style approach,
> > where only one of several possible fields should be populated. But it's a
> > minor thing.
> >
> >
> >
> >
> >
> >
> > On 10 May 2018 at 09:29, Colin McCabe  wrote:
> >
> > > Hi Andy,
> > >
> > > The issue that I was trying to solve here is the Java API.  Right now,
> > > someone can write "new ResourceFilter(ResourceType.TRANSACTIONAL_ID,
> > > "foo*") and have a ResourceFilter that applies to a Transactional ID
> named
> > > "foo*".  This has to continue to work, or else we have broken
> compatibility.
> > >
> > > I was proposing that there would be something like a new function like
> > > ResourceFilter.fromPattern(ResourceType.TRANSACTIONAL_ID, "foo*")
> which
> > > would create a ResourceFilter that applied to transactional IDs
> starting
> > > with "foo", rather than transactional IDs named "foo*" specifically.
> > >
> > > I don't think it's important whether the Java class has an integer, an
> > > enum, or two string fields.  The important thing is that there's a new
> > > static function, or new constructor overload, etc. that works for
> patterns
> > > rather than literal strings.
> > >
> > > On Thu, May 10, 2018, at 03:30, Andy Coates wrote:
> > > > Rather than having name and pattern fields on the ResourceFilter,
> where
> > > > it’s only valid for one to be set, and we want to restrict the
> character
> > > > set in case future enhancements need them, we could instead add a new
> > > > integer ‘nameType’ field, and use constants to indicate how the name
> > > > field should be interpreted, e.g. 0 = literal, 1 = wildcard. This
> would
> > > > be extendable, e.g we can later add 2 = regex, or what ever, and
> > > > wouldn’t require any escaping.
> > >
> > > This is very user-unfriendly, though.  Users don't want to have to
> > > explicitly supply a version number when using the API, which is what
> this
> > > would force them to do.  I don't think users are going to want to
> memorize
> > > that version 4 supprted "+", whereas version 3 only supported "[0-9]",
> or
> > > whatever.
> > >
> > > Just as an example, do you remember which versions of FetchRequest
> added
> > > which features?  I don't.  I always have to look at the code to
> remember.
> > >
> > > Also, escaping is still required any time you overload a character to
> mean
> > > two things.  Escaping is required in the current proposal to be able to
> > > create a pattern that matches only "foo*".  You have to type "foo\*"
> It
> > > would be required if we forced users to specify a version, as well.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > Sent from my iPhone
> > > >
> > > > > On 7 May 2018, at 05:16, Piyush Vijay 
> wrote:
> > > > >
> > > > > Makes sense. I'll update the KIP.
> > > > >
> > > > > Does anyone have any other comments? :)
> > > > >
> > > > > Thanks
> > > > >
> > > > >
> > > > > Piyush Vijay
> > > > >
> > > > >> On Thu, May 3, 2018 at 11:55 AM, Colin McCabe  >
> > > wrote:
> > > > >>
> > > > >> Yeah, I guess that's a good point.  It probably makes sense to
> > > support the
> > > > >> prefix scheme for consumer groups and transactional IDs as well as
> > > topics.
> > > > >>
> > > > >> I agree that the current situation where anything goes in consumer
> > > group
> > > > >> names and transactional ID names is not ideal.  I wish we could
> > > rewind the
> > > > >> clock and impose restrictions on the names.  However, it doesn't
> seem
> > > > >> practical at the moment.  Adding new restrictions would break a
> lot of
> > > > >> existing users after an upgrade.  It would be a really bad upgrade
> > > > >> experience.
> > > > >>
> > > > >> However, I think we can support this in a compatible way.  From
> the
> > > > >> perspective of AdminClient, we just have to add a new field to
> > > > >> ResourceFilter.  Currently, it has two fields, resourceType and
> name:
> > > > >>
> > > > >>> /**
> > > > >>> * A filter which matches Resource objects.
> > > > >>> *
> > > > >>> * The API for this class is still evolving and we may break
> > > > >> compatibility in minor releases, if necessary.
> > > > >>> */
> > > > >>> @InterfaceStability.Evolving
> > > > >>> public class ResourceFilter {
> > > > >>>private final ResourceType resourceType;
> > > > >>>private final String name;
> > > > >>
> > > > >> We can 

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-14 Thread Edoardo Comar
Thanks Rajini

I still don't see the overlap between the two KIPS

KIP-235 allows an expansion of hostnames on the bootstrap list.

KIP-302 allows alternative IPs to be used to attempt a connection
(either at bootstrap and when processing the MetadataResponse) to a
given hostname.

A use case would be that of active/passive LB fronting the brokers.

Arguably, if Java honored the DNS-set TTL, and the TTL was low and on
subsequent requests, the order of IPs returned by the
InetAddress.getAllByName() was random, we may not need such an
enhancement.
In practice, a Java client can get stuck on a "bad" IP forever if it
only relies on the first IP returned.

HTH,
Edo

On 14 May 2018 at 16:23, Rajini Sivaram  wrote:
> Hi Edo,
>
> Thanks for the KIP. I think it will be good to include a diagram to make it
> easier to distinguish this scenario from that of KIP-235 without reading
> the PR.
>
> It may be worth considering if KIP-235 and this KIP could use a single
> config name with different values instead of two boolean configs:
>
> bootstrap.reverse.dns.lookup = true/false
> enable.all.dns.ips = true/false
>
> Not all values of (bootstrap.reverse.dns.lookup, enable.all.dns.ips) seem
> to make sense. And not all scenarios are handled. Even if we use multiple
> configs, it seems to me that we may want to name them differently.
>
> The possible combinations are:
>
> 1) Bootstrap
>
> a) No lookup
> b) Use all DNS entries with host name
> c) Use all DNS entries with canonical host name
>
> 2) Advertised listeners
>
> a) No lookup
> b) Use all DNS entries with host name
> c) Use all DNS entries with canonical host name
>
> The combinations that are enabled by the two boolean configs (
> bootstrap.reverse.dns.lookup, enable.all.dns.ips)  are:
>
>- (false, false) => (1a, 2a)
>- (true, false) => (1c, 2a)
>- (false, true) => (1b, 2b)
>- (true, true) => (??, 2b)
>
> It will be good if we can clearly identify which combinations we want to
> support and the scenarios where they may be useful. Perhaps (1a, 2a), (1c,
> 2a), (1b, 2b) and (1c, 2c) are useful?
>
>
> On Mon, May 14, 2018 at 2:58 PM, Skrzypek, Jonathan <
> jonathan.skrzy...@gs.com> wrote:
>
>> Ah, apologies didn't see there was already a decent amount of discussion
>> on this in the PR.
>>
>> This kind of sounds related to the environment you're running to me.
>> What is the rationale behind using the advertised listeners to do your
>> load balancing advertisement rather than a top level alias that has
>> everything ?
>>
>> It sounds like in your case there is a mismatch between bootstrap.servers
>> and advertised.listeners, and you want advertised.listeners to take
>> precedence and have the client iterate over what is returned by the broker.
>> So the extra parameter doesn't only have to do with DNS but it's also
>> appending from the broker, maybe the parameter name should reflect this ?
>>
>> Jonathan Skrzypek
>>
>>
>> -Original Message-
>> From: Skrzypek, Jonathan [Tech]
>> Sent: 14 May 2018 14:46
>> To: dev@kafka.apache.org
>> Subject: RE: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS
>> resolved IP addresses
>>
>> Hi,
>>
>> I see you noted the similarities with KIP-235.
>> But KIP-235 might also solve what this KIP is trying to achieve.
>>
>> When parsing bootstrap.servers, KIP-235 has the client add all underlying
>> hostnames and IPs.
>> And this happens before hitting the NetworkClient.
>>
>> So to me the client will try every single endpoint behind any
>> bootstrap.servers record.
>>
>> See https://github.com/apache/kafka/pull/4485/commits/24757eb7b0
>> 6bcf8c7d7649c85232c52b5d54f0e4#diff-89ef153462e64c250a21bd324ae1a851
>> which calls getAllByName like you suggested
>>
>> Jonathan Skrzypek
>>
>>
>> -Original Message-
>> From: Edoardo Comar [mailto:edoco...@gmail.com]
>> Sent: 14 May 2018 14:17
>> To: dev@kafka.apache.org
>> Subject: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved
>> IP addresses
>>
>> Hi all,
>>
>> We just opened a KIP to add support for the client to use all IPs returned
>> by DNS for the brokers
>>
>> The details are here -
>>
>> https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.a
>> pache.org_confluence_display_KAFKA_KIP-2D302-2B-2D-2BEnable-
>> 2BKafka-2Bclients-2Bto-2Buse-2Ball-2BDNS-2Bresolved-2BIP-
>> 2Baddresses=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxL
>> xfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafF
>> l1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=C-UZ6KUG7JFiPD_
>> CnHczDOVqH9-XC5f_OFkw4BTNrI4=
>>
>> The JIRA and provisional PR  (where the discussion lead to the creation of
>> this KIP) are :
>>
>> https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.
>> apache.org_jira_browse_KAFKA-2D6863=DwIBaQ=7563p3e2zaQw0
>> AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6
>> eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-
>> D2Dti4k=3Puqs5iYoPsw6hARQr6gvokdFE-H5USMiNVGOUtNkJI=
>>
>> 

Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Andy Coates
Hey All,

I've updated the KIP to also cover the Scala API for completeness. (The PR
already covered this).

Does this require another vote?

On 11 May 2018 at 14:29, Matthias J. Sax  wrote:

> +1 (binding)
>
>
> On 5/11/18 2:17 PM, Ted Yu wrote:
> > +1
> >
> > On Fri, May 11, 2018 at 2:16 PM, Guozhang Wang 
> wrote:
> >
> >> +1 (binding)
> >>
> >> Also slightly changed the title beyond changing [DISCUSS] to [VOTE]
> since
> >> Gmail will collapse these two threads into one, so people may not
> realize
> >> there is a voting started already.
> >>
> >>
> >> Guozhang
> >>
> >>
> >> On Fri, May 11, 2018 at 2:07 PM, Andy Coates  wrote:
> >>
> >>> Hi all,
> >>>
> >>> I would like to start the vote on KIP-292: Add transformValues() method
> >> to
> >>> KTable
> >>>
> >>> The link to this KIP is here:
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 292%3A+Add+transformValues%28%29+method+to+KTable
> >>>
> >>> The discussion thread is here:
> >>> http://mail-archives.apache.org/mod_mbox/kafka-dev/201805.mbox/%
> >>> 3CCAO1RfpHyhEEwkwad5yxd0rLgPEK-PJ4y0%2B0y%3DwidHP0uOtToJQ%
> >>> 40mail.gmail.com%3E
> >>>
> >>> Thanks,
> >>>
> >>> Andy
> >>>
> >>
> >>
> >>
> >> --
> >> -- Guozhang
> >>
> >
>
>


Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Jorge Esteban Quilcate Otoya
Yes, I've one already created: https://github.com/apache/kafka/pull/4955

On Mon, 14 May 2018, 17:55 Guozhang Wang,  wrote:

> Thanks Jorge, that sounds good to me.
>
> Also please feel free to send out the PR for reviews while the KIP is being
> voted on.
>
>
> Guozhang
>
>
> On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya <
> quilcate.jo...@gmail.com> wrote:
>
> > Thanks for your feedback everyone!
> >
> > If there is no more comments on this KIP, I think we can open the VOTE
> > thread.
> >
> > Cheers,
> > Jorge.
> >
> > El sáb., 12 may. 2018 a las 2:02, Guozhang Wang ()
> > escribió:
> >
> > > Yeah I'm only talking about the DSL part (i.e. how stateful / stateless
> > > operators default inheritance protocol would be promised) to be managed
> > > with KIP-159.
> > >
> > > For allowing users to override the default behavior in PAPI, that would
> > be
> > > in a different KIP.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Fri, May 11, 2018 at 10:41 AM, Matthias J. Sax <
> matth...@confluent.io
> > >
> > > wrote:
> > >
> > > > I am actually not sure about this. Because it's about the semantics
> at
> > > > PAPI level, but KIP-159 targets the DSL, it might actually be better
> to
> > > > have a separate KIP?
> > > >
> > > > -Matthias
> > > >
> > > > On 5/11/18 9:26 AM, Guozhang Wang wrote:
> > > > > That's a good question. I think we can manage this in KIP-159. I
> will
> > > go
> > > > > ahead and try to augment that KIP together with the original author
> > > > Jeyhun.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Fri, May 11, 2018 at 12:45 AM, Jorge Esteban Quilcate Otoya <
> > > > > quilcate.jo...@gmail.com> wrote:
> > > > >
> > > > >> Thanks Guozhang and Matthias! I do also agree with this way of
> > > handling
> > > > >> headers inheritance. I will add them to the KIP doc.
> > > > >>
> > > > >>> We can discuss about extending the current protocol and how to
> > enable
> > > > >> users
> > > > >>> override those rule, and how to expose them in the DSL layer in a
> > > > future
> > > > >>> KIP.
> > > > >>
> > > > >> About this, should this be managed on KIP-159 or a new one?
> > > > >>
> > > > >> El jue., 10 may. 2018 a las 17:46, Matthias J. Sax (<
> > > > matth...@confluent.io
> > > > >>> )
> > > > >> escribió:
> > > > >>
> > > > >>> Thanks Guozhang! Sounds good to me!
> > > > >>>
> > > > >>> -Matthias
> > > > >>>
> > > > >>> On 5/10/18 7:55 AM, Guozhang Wang wrote:
> > > >  Thanks for your thoughts Matthias. I think if we do want to
> bring
> > > > >> KIP-244
> > > >  into 2.0 then we need to keep its scope small and well defined.
> > For
> > > > >> that
> > > >  I'm proposing:
> > > > 
> > > >  1. Make the inheritance implementation of headers consistent
> with
> > > what
> > > > >> we
> > > >  had with other record context fields. I.e. pass through the
> record
> > > > >>> context
> > > >  in `context.forward()`. Note that within a processor node, users
> > can
> > > >  already manipulate the Headers with the given APIs, so at the
> time
> > > of
> > > >  forwarding, the library can just copy what-ever is left /
> updated
> > to
> > > > >> the
> > > >  next processor node.
> > > > 
> > > >  2. In the sink node, where a record is being sent to the Kafka
> > > topic,
> > > > >> we
> > > >  should consider the following:
> > > > 
> > > >  a. For sink topics, we will set the headers into the producer
> > > record.
> > > >  b. For repartition topics, we will the headers into the producer
> > > > >> record.
> > > >  c. For changelog topics, we will drop the headers in the produce
> > > > record
> > > >  since they will not be used in restoration and not stored in the
> > > state
> > > >  store either.
> > > > 
> > > > 
> > > >  We can discuss about extending the current protocol and how to
> > > enable
> > > > >>> users
> > > >  override those rule, and how to expose them in the DSL layer in
> a
> > > > >> future
> > > >  KIP.
> > > > 
> > > > 
> > > > 
> > > >  Guozhang
> > > > 
> > > > 
> > > >  On Mon, May 7, 2018 at 5:49 PM, Matthias J. Sax <
> > > > matth...@confluent.io
> > > > >>>
> > > >  wrote:
> > > > 
> > > > > Guozhang,
> > > > >
> > > > > if you advocate to forward headers by default, it might be a
> > better
> > > > > default strategy do forward the headers for all operators
> > (similar
> > > to
> > > > > topic/partition/offset metadata). It's usually harder for users
> > to
> > > > > reason about different cases and thus I would prefer to have
> > > > >> consistent
> > > > > behavior, ie, only one default strategy instead of introducing
> > > > >> different
> > > > > cases.
> > > > >
> > > > > Btw: My argument about dropping headers by default only
> implies,
> > > that
> > > > > users need to copy the headers explicitly to the output records

Re: [VOTE] KIP-292: Add KTable#transformValues() method in Kafka Streams DSL

2018-05-14 Thread Damian Guy
+1

On Fri, 11 May 2018 at 14:30 Matthias J. Sax  wrote:

> +1 (binding)
>
>
> On 5/11/18 2:17 PM, Ted Yu wrote:
> > +1
> >
> > On Fri, May 11, 2018 at 2:16 PM, Guozhang Wang 
> wrote:
> >
> >> +1 (binding)
> >>
> >> Also slightly changed the title beyond changing [DISCUSS] to [VOTE]
> since
> >> Gmail will collapse these two threads into one, so people may not
> realize
> >> there is a voting started already.
> >>
> >>
> >> Guozhang
> >>
> >>
> >> On Fri, May 11, 2018 at 2:07 PM, Andy Coates  wrote:
> >>
> >>> Hi all,
> >>>
> >>> I would like to start the vote on KIP-292: Add transformValues() method
> >> to
> >>> KTable
> >>>
> >>> The link to this KIP is here:
> >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 292%3A+Add+transformValues%28%29+method+to+KTable
> >>>
> >>> The discussion thread is here:
> >>> http://mail-archives.apache.org/mod_mbox/kafka-dev/201805.mbox/%
> >>> 3CCAO1RfpHyhEEwkwad5yxd0rLgPEK-PJ4y0%2B0y%3DwidHP0uOtToJQ%
> >>> 40mail.gmail.com%3E
> >>>
> >>> Thanks,
> >>>
> >>> Andy
> >>>
> >>
> >>
> >>
> >> --
> >> -- Guozhang
> >>
> >
>
>


59 Kafka stream tests fails with one error...

2018-05-14 Thread Wabi SA
Want to fix this error but need community help because I am not java man.
Thanks in advance, see logs please.

Tests fails with next error:
[INFO:2018-05-14 18:48:51,676]: starting test run with session id
2018-05-14--008...
[INFO:2018-05-14 18:48:51,676]: running 1 tests...
[INFO:2018-05-14 18:48:51,676]: Triggering test 1 of 1...
[INFO:2018-05-14 18:48:51,687]: RunnerClient: Loading test {'directory':
'/home/vmukha/proj/kafka-1.0.1.redhat-7/tests/kafkatest/tests/streams',
'file_name': 'streams_smoke_test.py', 'method_name': 'test_streams',
'cls_name': 'StreamsSmokeTest', 'injected_args': None}
[INFO:2018-05-14 18:48:51,693]: RunnerClient:
kafkatest.tests.streams.streams_smoke_test.StreamsSmokeTest.test_streams:
Setting up...
[INFO:2018-05-14 18:50:11,903]: RunnerClient:
kafkatest.tests.streams.streams_smoke_test.StreamsSmokeTest.test_streams:
Running...
[INFO:2018-05-14 18:51:12,925]: RunnerClient:
kafkatest.tests.streams.streams_smoke_test.StreamsSmokeTest.test_streams:
FAIL: Never saw message indicating StreamsTest finished startup on
vagrant@worker5
Traceback (most recent call last):
  File
"/home/vmukha/proj/kafka-1.0.1.redhat-7/tests/evenv/local/lib/python2.7/site-packages/ducktape/tests/runner_client.py",
line 132, in run
data = self.run_test()
  File
"/home/vmukha/proj/kafka-1.0.1.redhat-7/tests/evenv/local/lib/python2.7/site-packages/ducktape/tests/runner_client.py",
line 189, in run_test
return self.test_context.function(self.test)
  File
"/home/vmukha/proj/kafka-1.0.1.redhat-7/tests/kafkatest/tests/streams/streams_smoke_test.py",
line 56, in test_streams
self.driver.start()
  File
"/home/vmukha/proj/kafka-1.0.1.redhat-7/tests/evenv/local/lib/python2.7/site-packages/ducktape/services/service.py",
line 234, in start
self.start_node(node)
  File
"/home/vmukha/proj/kafka-1.0.1.redhat-7/tests/kafkatest/services/streams.py",
line 138, in start_node
monitor.wait_until('StreamsTest instance started', timeout_sec=60,
err_msg="Never saw message indicating StreamsTest finished startup on " +
str(node.account))
  File
"/home/vmukha/proj/kafka-1.0.1.redhat-7/tests/evenv/local/lib/python2.7/site-packages/ducktape/cluster/remoteaccount.py",
line 680, in wait_until
allow_fail=True) == 0, **kwargs)
  File
"/home/vmukha/proj/kafka-1.0.1.redhat-7/tests/evenv/local/lib/python2.7/site-packages/ducktape/utils/util.py",
line 41, in wait_until
raise TimeoutError(err_msg() if callable(err_msg) else err_msg)
TimeoutError: Never saw message indicating StreamsTest finished startup on
vagrant@worker5

If looks into the log on vagrant@worker5 mnt/streams/streams.stderr we
could see:

Error: Could not find or load main class
org.apache.kafka.streams.tests.StreamsSmokeTest
^^
Any idea how to fix this error?

p.s. with this error now fails 59 tests from Kafka test suite, so might be
one root for all of them.

BR And Thanks,
Viacheslav


Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-14 Thread Edoardo Comar
Hi Jonathan
thanks for your comments

> This kind of sounds related to the environment you're running to me.
> What is the rationale behind using the advertised listeners to do your load 
> balancing advertisement rather than a top level alias that has everything ?
>
> It sounds like in your case there is a mismatch between bootstrap.servers and 
> advertised.listeners, and you want advertised.listeners to take precedence 
> and have the client iterate over what is returned by the broker.
> So the extra parameter doesn't only have to do with DNS but it's also 
> appending from the broker, maybe the parameter name should reflect this ?

There is no mismatch for us.

Our KIP-302 and associated PR allow the network client - both at
bootstrap and when using the cluster's metadata response - to connect
to the brokers using alternative IPs in case the first one fails.
This KIP changes the client behaviour only if the DNS is configured to
resolve a single name with multiple valid IP addresses.

KIP-235 use case seems targeted to enrich the bootstrap list with more
hostnames as long as they all resolve. And I understand it to be
Kerberos-motivated.
I can't see a similar list expansion being applied to the hostnames
returned in a metadata response.

KIP-235 doesn't seem to resolve our use case, and KIP-302 doesn't seem
to help with yours.
I keep seeing as addressing different concerns, the only commonality
is ... they have DNS in their titles :-)

Edoardo

> Jonathan Skrzypek
>
>
> -Original Message-
> From: Skrzypek, Jonathan [Tech]
> Sent: 14 May 2018 14:46
> To: dev@kafka.apache.org
> Subject: RE: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved 
> IP addresses
>
> Hi,
>
> I see you noted the similarities with KIP-235.
> But KIP-235 might also solve what this KIP is trying to achieve.
>
> When parsing bootstrap.servers, KIP-235 has the client add all underlying 
> hostnames and IPs.
> And this happens before hitting the NetworkClient.
>
> So to me the client will try every single endpoint behind any 
> bootstrap.servers record.
>
> See 
> https://github.com/apache/kafka/pull/4485/commits/24757eb7b06bcf8c7d7649c85232c52b5d54f0e4#diff-89ef153462e64c250a21bd324ae1a851
> which calls getAllByName like you suggested
>
> Jonathan Skrzypek
>
>
> -Original Message-
> From: Edoardo Comar [mailto:edoco...@gmail.com]
> Sent: 14 May 2018 14:17
> To: dev@kafka.apache.org
> Subject: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP 
> addresses
>
> Hi all,
>
> We just opened a KIP to add support for the client to use all IPs returned
> by DNS for the brokers
>
> The details are here -
>
> https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.apache.org_confluence_display_KAFKA_KIP-2D302-2B-2D-2BEnable-2BKafka-2Bclients-2Bto-2Buse-2Ball-2BDNS-2Bresolved-2BIP-2Baddresses=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=C-UZ6KUG7JFiPD_CnHczDOVqH9-XC5f_OFkw4BTNrI4=
>
> The JIRA and provisional PR  (where the discussion lead to the creation of
> this KIP) are :
>
> https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_KAFKA-2D6863=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=3Puqs5iYoPsw6hARQr6gvokdFE-H5USMiNVGOUtNkJI=
>
> https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_kafka_pull_4987=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=Hqn5dOgQy4-MHTIJLE49O8bNomry3SoGq9OVoHU-CRA=
>
> Looking forward to the community's feedback.
> It would be amazing to have it voted by May 22nd :-) :-)
>
> Edoardo & Mickael




-- 
"When the people fear their government, there is tyranny; when the
government fears the people, there is liberty." [Thomas Jefferson]


Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Guozhang Wang
Thanks Jorge, that sounds good to me.

Also please feel free to send out the PR for reviews while the KIP is being
voted on.


Guozhang


On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya <
quilcate.jo...@gmail.com> wrote:

> Thanks for your feedback everyone!
>
> If there is no more comments on this KIP, I think we can open the VOTE
> thread.
>
> Cheers,
> Jorge.
>
> El sáb., 12 may. 2018 a las 2:02, Guozhang Wang ()
> escribió:
>
> > Yeah I'm only talking about the DSL part (i.e. how stateful / stateless
> > operators default inheritance protocol would be promised) to be managed
> > with KIP-159.
> >
> > For allowing users to override the default behavior in PAPI, that would
> be
> > in a different KIP.
> >
> >
> > Guozhang
> >
> >
> > On Fri, May 11, 2018 at 10:41 AM, Matthias J. Sax  >
> > wrote:
> >
> > > I am actually not sure about this. Because it's about the semantics at
> > > PAPI level, but KIP-159 targets the DSL, it might actually be better to
> > > have a separate KIP?
> > >
> > > -Matthias
> > >
> > > On 5/11/18 9:26 AM, Guozhang Wang wrote:
> > > > That's a good question. I think we can manage this in KIP-159. I will
> > go
> > > > ahead and try to augment that KIP together with the original author
> > > Jeyhun.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > > On Fri, May 11, 2018 at 12:45 AM, Jorge Esteban Quilcate Otoya <
> > > > quilcate.jo...@gmail.com> wrote:
> > > >
> > > >> Thanks Guozhang and Matthias! I do also agree with this way of
> > handling
> > > >> headers inheritance. I will add them to the KIP doc.
> > > >>
> > > >>> We can discuss about extending the current protocol and how to
> enable
> > > >> users
> > > >>> override those rule, and how to expose them in the DSL layer in a
> > > future
> > > >>> KIP.
> > > >>
> > > >> About this, should this be managed on KIP-159 or a new one?
> > > >>
> > > >> El jue., 10 may. 2018 a las 17:46, Matthias J. Sax (<
> > > matth...@confluent.io
> > > >>> )
> > > >> escribió:
> > > >>
> > > >>> Thanks Guozhang! Sounds good to me!
> > > >>>
> > > >>> -Matthias
> > > >>>
> > > >>> On 5/10/18 7:55 AM, Guozhang Wang wrote:
> > >  Thanks for your thoughts Matthias. I think if we do want to bring
> > > >> KIP-244
> > >  into 2.0 then we need to keep its scope small and well defined.
> For
> > > >> that
> > >  I'm proposing:
> > > 
> > >  1. Make the inheritance implementation of headers consistent with
> > what
> > > >> we
> > >  had with other record context fields. I.e. pass through the record
> > > >>> context
> > >  in `context.forward()`. Note that within a processor node, users
> can
> > >  already manipulate the Headers with the given APIs, so at the time
> > of
> > >  forwarding, the library can just copy what-ever is left / updated
> to
> > > >> the
> > >  next processor node.
> > > 
> > >  2. In the sink node, where a record is being sent to the Kafka
> > topic,
> > > >> we
> > >  should consider the following:
> > > 
> > >  a. For sink topics, we will set the headers into the producer
> > record.
> > >  b. For repartition topics, we will the headers into the producer
> > > >> record.
> > >  c. For changelog topics, we will drop the headers in the produce
> > > record
> > >  since they will not be used in restoration and not stored in the
> > state
> > >  store either.
> > > 
> > > 
> > >  We can discuss about extending the current protocol and how to
> > enable
> > > >>> users
> > >  override those rule, and how to expose them in the DSL layer in a
> > > >> future
> > >  KIP.
> > > 
> > > 
> > > 
> > >  Guozhang
> > > 
> > > 
> > >  On Mon, May 7, 2018 at 5:49 PM, Matthias J. Sax <
> > > matth...@confluent.io
> > > >>>
> > >  wrote:
> > > 
> > > > Guozhang,
> > > >
> > > > if you advocate to forward headers by default, it might be a
> better
> > > > default strategy do forward the headers for all operators
> (similar
> > to
> > > > topic/partition/offset metadata). It's usually harder for users
> to
> > > > reason about different cases and thus I would prefer to have
> > > >> consistent
> > > > behavior, ie, only one default strategy instead of introducing
> > > >> different
> > > > cases.
> > > >
> > > > Btw: My argument about dropping headers by default only implies,
> > that
> > > > users need to copy the headers explicitly to the output records
> in
> > > >> there
> > > > code of they want to inspect them later -- it does not imply that
> > > > headers cannot be forwarded downstream. (Not sure if this was
> > clear).
> > > >
> > > > I am also ok with copying be default thought (for me, it's a
> 51/49
> > > > preference for dropping by default only).
> > > >
> > > >
> > > > -Matthias
> > > >
> > > > On 5/7/18 4:52 PM, Guozhang Wang wrote:
> > > >> Hi Matthias,

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Ted Yu
+1

On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya <
quilcate.jo...@gmail.com> wrote:

> Hi everyone,
>
> I would like to start a vote on KIP-244: Add Record Header support to Kafka
> Streams
>
> KIP wiki page:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 244%3A+Add+Record+Header+support+to+Kafka+Streams+Processor+API
>
> The discussion thread is here:
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201805.
> mbox/%3CCAC3UcJvrgcBfe6%3DiW6%2BuTWsLB%2B4CsHgRmDx9TvCzJQrWvfg7_w%
> 40mail.gmail.com%3E
>
> Cheers,
> Jorge.
>


[VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Jorge Esteban Quilcate Otoya
Hi everyone,

I would like to start a vote on KIP-244: Add Record Header support to Kafka
Streams

KIP wiki page:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-244%3A+Add+Record+Header+support+to+Kafka+Streams+Processor+API

The discussion thread is here:
http://mail-archives.apache.org/mod_mbox/kafka-dev/201805.mbox/%3CCAC3UcJvrgcBfe6%3DiW6%2BuTWsLB%2B4CsHgRmDx9TvCzJQrWvfg7_w%40mail.gmail.com%3E

Cheers,
Jorge.


Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Jorge Esteban Quilcate Otoya
Thanks for your feedback everyone!

If there is no more comments on this KIP, I think we can open the VOTE
thread.

Cheers,
Jorge.

El sáb., 12 may. 2018 a las 2:02, Guozhang Wang ()
escribió:

> Yeah I'm only talking about the DSL part (i.e. how stateful / stateless
> operators default inheritance protocol would be promised) to be managed
> with KIP-159.
>
> For allowing users to override the default behavior in PAPI, that would be
> in a different KIP.
>
>
> Guozhang
>
>
> On Fri, May 11, 2018 at 10:41 AM, Matthias J. Sax 
> wrote:
>
> > I am actually not sure about this. Because it's about the semantics at
> > PAPI level, but KIP-159 targets the DSL, it might actually be better to
> > have a separate KIP?
> >
> > -Matthias
> >
> > On 5/11/18 9:26 AM, Guozhang Wang wrote:
> > > That's a good question. I think we can manage this in KIP-159. I will
> go
> > > ahead and try to augment that KIP together with the original author
> > Jeyhun.
> > >
> > >
> > > Guozhang
> > >
> > > On Fri, May 11, 2018 at 12:45 AM, Jorge Esteban Quilcate Otoya <
> > > quilcate.jo...@gmail.com> wrote:
> > >
> > >> Thanks Guozhang and Matthias! I do also agree with this way of
> handling
> > >> headers inheritance. I will add them to the KIP doc.
> > >>
> > >>> We can discuss about extending the current protocol and how to enable
> > >> users
> > >>> override those rule, and how to expose them in the DSL layer in a
> > future
> > >>> KIP.
> > >>
> > >> About this, should this be managed on KIP-159 or a new one?
> > >>
> > >> El jue., 10 may. 2018 a las 17:46, Matthias J. Sax (<
> > matth...@confluent.io
> > >>> )
> > >> escribió:
> > >>
> > >>> Thanks Guozhang! Sounds good to me!
> > >>>
> > >>> -Matthias
> > >>>
> > >>> On 5/10/18 7:55 AM, Guozhang Wang wrote:
> >  Thanks for your thoughts Matthias. I think if we do want to bring
> > >> KIP-244
> >  into 2.0 then we need to keep its scope small and well defined. For
> > >> that
> >  I'm proposing:
> > 
> >  1. Make the inheritance implementation of headers consistent with
> what
> > >> we
> >  had with other record context fields. I.e. pass through the record
> > >>> context
> >  in `context.forward()`. Note that within a processor node, users can
> >  already manipulate the Headers with the given APIs, so at the time
> of
> >  forwarding, the library can just copy what-ever is left / updated to
> > >> the
> >  next processor node.
> > 
> >  2. In the sink node, where a record is being sent to the Kafka
> topic,
> > >> we
> >  should consider the following:
> > 
> >  a. For sink topics, we will set the headers into the producer
> record.
> >  b. For repartition topics, we will the headers into the producer
> > >> record.
> >  c. For changelog topics, we will drop the headers in the produce
> > record
> >  since they will not be used in restoration and not stored in the
> state
> >  store either.
> > 
> > 
> >  We can discuss about extending the current protocol and how to
> enable
> > >>> users
> >  override those rule, and how to expose them in the DSL layer in a
> > >> future
> >  KIP.
> > 
> > 
> > 
> >  Guozhang
> > 
> > 
> >  On Mon, May 7, 2018 at 5:49 PM, Matthias J. Sax <
> > matth...@confluent.io
> > >>>
> >  wrote:
> > 
> > > Guozhang,
> > >
> > > if you advocate to forward headers by default, it might be a better
> > > default strategy do forward the headers for all operators (similar
> to
> > > topic/partition/offset metadata). It's usually harder for users to
> > > reason about different cases and thus I would prefer to have
> > >> consistent
> > > behavior, ie, only one default strategy instead of introducing
> > >> different
> > > cases.
> > >
> > > Btw: My argument about dropping headers by default only implies,
> that
> > > users need to copy the headers explicitly to the output records in
> > >> there
> > > code of they want to inspect them later -- it does not imply that
> > > headers cannot be forwarded downstream. (Not sure if this was
> clear).
> > >
> > > I am also ok with copying be default thought (for me, it's a 51/49
> > > preference for dropping by default only).
> > >
> > >
> > > -Matthias
> > >
> > > On 5/7/18 4:52 PM, Guozhang Wang wrote:
> > >> Hi Matthias,
> > >>
> > >> My concern of setting `null` in all cases is that it would make
> > >> headers
> > > not
> > >> very useful in KIP-244 then, because headers will only be
> available
> > >> at
> > > the
> > >> source stream / table, but not in any of the following instances.
> In
> > >> practice users may be more likely to look into the headers later
> in
> > >> the
> > >> pipeline. Personally I'd suggest we pass the headers for all
> > >> stateless
> > >> operators in DSL and everywhere in PAPI's 

Re: [VOTE] KIP-278: Add version option to Kafka's commands

2018-05-14 Thread Sasaki Toru

Rajini,

Thank you for your appropriate advice, I misunderstood some non-binding votes.
I get this KIP back to the state waiting to vote.


Thanks,
Sasaki


2018-05-14 20:03 GMT+09:00 Mickael Maison :


+1 (non binding)
Thanks for the KIP

On Mon, May 14, 2018 at 10:36 AM, Rajini Sivaram
 wrote:

Sasaki,

I believe this has 2 binding votes and six non-binding votes, so needs
another binding vote to pass.

On Mon, May 14, 2018 at 9:00 AM, Sasaki Toru 
wrote:


Hi all,

This KIP would be accepted with 4 binding and 4 non-binding votes.

Thank you for participating.


Thanks,
Sasaki.

2018-05-12 9:31 GMT+09:00 Colin McCabe :

+1 (non-binding)

Colin

On Fri, May 11, 2018, at 12:35, Attila Sasvári wrote:


+1 (non-binding)

Thomas Crayford  ezt írta (időpont: 2018.
máj.
11., P 18:20):

+1 (non-binding)

On Fri, May 11, 2018 at 5:17 PM, Guozhang Wang 


wrote:
Thanks Toru-san, +1 (binding)

On Fri, May 11, 2018 at 8:30 AM, Jason Gustafson <

ja...@confluent.io

wrote:

Thanks for the KIP! +1 (binding)

On Fri, May 11, 2018 at 12:35 AM, Manikumar <


manikumar.re...@gmail.com

wrote:

+1 (non-binding)

Thanks for the KIP.

On Fri, May 11, 2018 at 12:56 PM, zhenya Sun 


wrote:

+1 building

在 2018年5月11日,上午9:51,Ted Yu  写道:

+1

On Thu, May 10, 2018 at 6:42 PM, Sasaki Toru <


sasaki...@oss.nttdata.com>
wrote:

Hi all,

I would like to start the vote on KIP-278: Add version


option to

Kafka's

commands.

The link to this KIP is here:


The discussion thread is here:
<


https://www.mail-archive.com/dev@kafka.apache.org/msg86688.

html>

Many thanks,
Sasaki

--
Sasaki Toru(sasaki...@oss.nttdata.com) NTT DATA CORPORATION




--
-- Guozhang



--
Sasaki Toru(sasaki...@oss.nttdata.com) NTT DATA CORPORATION




--
Sasaki Toru(sasaki...@oss.nttdata.com) NTT DATA CORPORATION



KAFKA-6864

2018-05-14 Thread git

Hi there,

could anyone please have a look at
 * https://issues.apache.org/jira/browse/KAFKA-6864 and the pull request 
as well as

 * https://github.com/apache/kafka/pull/3873 ?

Thanks,
Mathias


[jira] [Created] (KAFKA-6901) Kafka crashes when trying to delete segment when retetention time is reached

2018-05-14 Thread JIRA
Grégory R. created KAFKA-6901:
-

 Summary: Kafka crashes when trying to delete segment when 
retetention time is reached 
 Key: KAFKA-6901
 URL: https://issues.apache.org/jira/browse/KAFKA-6901
 Project: Kafka
  Issue Type: Bug
  Components: core, log
Affects Versions: 1.0.0
 Environment: OS: Windows Server 2012 R2
Reporter: Grégory R.


Following the parameter
{code:java}
log.retention.hours = 16{code}
kafka tries to delete segments.

This action crashes the server with following log:

 
{code:java}

[2018-05-11 15:17:58,036] INFO Found deletable segments with base offsets [0] 
due to retention time 60480ms breach (kafka.log.Log)
[2018-05-11 15:17:58,068] INFO Rolled new log segment for 'event-0' in 12 ms. 
(kafka.log.Log)
[2018-05-11 15:17:58,068] INFO Scheduling log segment 0 for log event-0 for 
deletion. (kafka.log.Log)
[2018-05-11 15:17:58,068] ERROR Error while deleting segments for event-0 in 
dir C:\App\VISBridge\kafka_2.12-1.0.0\kafka-log (kafka.server.L
ogDirFailureChannel)
java.nio.file.FileSystemException: 
C:\App\VISBridge\kafka_2.12-1.0.0\kafka-log\event-0\.log -> 
C:\App\VISBridge\kafka_2.
12-1.0.0\kafka-log\event-0\.log.deleted: Le processus ne 
peut pas accÚder au fichier car ce fichier est utilisÚ par un a
utre processus.

    at 
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)
    at 
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)
    at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:387)
    at 
sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287)
    at java.nio.file.Files.move(Files.java:1395)
    at 
org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:682)
    at 
org.apache.kafka.common.record.FileRecords.renameTo(FileRecords.java:212)
    at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:398)
    at kafka.log.Log.asyncDeleteSegment(Log.scala:1592)
    at kafka.log.Log.deleteSegment(Log.scala:1579)
    at kafka.log.Log.$anonfun$deleteSegments$3(Log.scala:1152)
    at kafka.log.Log.$anonfun$deleteSegments$3$adapted(Log.scala:1152)
    at 
scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:59)
    at 
scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:52)
    at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
    at kafka.log.Log.$anonfun$deleteSegments$2(Log.scala:1152)
    at 
scala.runtime.java8.JFunction0$mcI$sp.apply(JFunction0$mcI$sp.java:12)
    at kafka.log.Log.maybeHandleIOException(Log.scala:1669)
    at kafka.log.Log.deleteSegments(Log.scala:1143)
    at kafka.log.Log.deleteOldSegments(Log.scala:1138)
    at kafka.log.Log.deleteRetentionMsBreachedSegments(Log.scala:1211)
    at kafka.log.Log.deleteOldSegments(Log.scala:1204)
    at kafka.log.LogManager.$anonfun$cleanupLogs$3(LogManager.scala:715)
    at 
kafka.log.LogManager.$anonfun$cleanupLogs$3$adapted(LogManager.scala:713)
    at 
scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:789)
    at scala.collection.Iterator.foreach(Iterator.scala:929)
    at scala.collection.Iterator.foreach$(Iterator.scala:929)
    at scala.collection.AbstractIterator.foreach(Iterator.scala:1417)
    at scala.collection.IterableLike.foreach(IterableLike.scala:71)
    at scala.collection.IterableLike.foreach$(IterableLike.scala:70)
    at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
    at 
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:788)
    at kafka.log.LogManager.cleanupLogs(LogManager.scala:713)
    at kafka.log.LogManager.$anonfun$startup$2(LogManager.scala:341)
    at 
kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:110)
    at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:61)
    at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
    at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
    at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
    at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
    at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:748)
    Suppressed: java.nio.file.FileSystemException: 
C:\App\VISBridge\kafka_2.12-1.0.0\kafka-log\event-0\.log -> 
C:\Ap
p\VISBridge\kafka_2.12-1.0.0\kafka-log\event-0\.log.deleted:
 Le processus ne peut 

Re: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-14 Thread Rajini Sivaram
Hi Edo,

Thanks for the KIP. I think it will be good to include a diagram to make it
easier to distinguish this scenario from that of KIP-235 without reading
the PR.

It may be worth considering if KIP-235 and this KIP could use a single
config name with different values instead of two boolean configs:

bootstrap.reverse.dns.lookup = true/false
enable.all.dns.ips = true/false

Not all values of (bootstrap.reverse.dns.lookup, enable.all.dns.ips) seem
to make sense. And not all scenarios are handled. Even if we use multiple
configs, it seems to me that we may want to name them differently.

The possible combinations are:

1) Bootstrap

a) No lookup
b) Use all DNS entries with host name
c) Use all DNS entries with canonical host name

2) Advertised listeners

a) No lookup
b) Use all DNS entries with host name
c) Use all DNS entries with canonical host name

The combinations that are enabled by the two boolean configs (
bootstrap.reverse.dns.lookup, enable.all.dns.ips)  are:

   - (false, false) => (1a, 2a)
   - (true, false) => (1c, 2a)
   - (false, true) => (1b, 2b)
   - (true, true) => (??, 2b)

It will be good if we can clearly identify which combinations we want to
support and the scenarios where they may be useful. Perhaps (1a, 2a), (1c,
2a), (1b, 2b) and (1c, 2c) are useful?


On Mon, May 14, 2018 at 2:58 PM, Skrzypek, Jonathan <
jonathan.skrzy...@gs.com> wrote:

> Ah, apologies didn't see there was already a decent amount of discussion
> on this in the PR.
>
> This kind of sounds related to the environment you're running to me.
> What is the rationale behind using the advertised listeners to do your
> load balancing advertisement rather than a top level alias that has
> everything ?
>
> It sounds like in your case there is a mismatch between bootstrap.servers
> and advertised.listeners, and you want advertised.listeners to take
> precedence and have the client iterate over what is returned by the broker.
> So the extra parameter doesn't only have to do with DNS but it's also
> appending from the broker, maybe the parameter name should reflect this ?
>
> Jonathan Skrzypek
>
>
> -Original Message-
> From: Skrzypek, Jonathan [Tech]
> Sent: 14 May 2018 14:46
> To: dev@kafka.apache.org
> Subject: RE: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS
> resolved IP addresses
>
> Hi,
>
> I see you noted the similarities with KIP-235.
> But KIP-235 might also solve what this KIP is trying to achieve.
>
> When parsing bootstrap.servers, KIP-235 has the client add all underlying
> hostnames and IPs.
> And this happens before hitting the NetworkClient.
>
> So to me the client will try every single endpoint behind any
> bootstrap.servers record.
>
> See https://github.com/apache/kafka/pull/4485/commits/24757eb7b0
> 6bcf8c7d7649c85232c52b5d54f0e4#diff-89ef153462e64c250a21bd324ae1a851
> which calls getAllByName like you suggested
>
> Jonathan Skrzypek
>
>
> -Original Message-
> From: Edoardo Comar [mailto:edoco...@gmail.com]
> Sent: 14 May 2018 14:17
> To: dev@kafka.apache.org
> Subject: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved
> IP addresses
>
> Hi all,
>
> We just opened a KIP to add support for the client to use all IPs returned
> by DNS for the brokers
>
> The details are here -
>
> https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.a
> pache.org_confluence_display_KAFKA_KIP-2D302-2B-2D-2BEnable-
> 2BKafka-2Bclients-2Bto-2Buse-2Ball-2BDNS-2Bresolved-2BIP-
> 2Baddresses=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxL
> xfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafF
> l1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=C-UZ6KUG7JFiPD_
> CnHczDOVqH9-XC5f_OFkw4BTNrI4=
>
> The JIRA and provisional PR  (where the discussion lead to the creation of
> this KIP) are :
>
> https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.
> apache.org_jira_browse_KAFKA-2D6863=DwIBaQ=7563p3e2zaQw0
> AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6
> eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-
> D2Dti4k=3Puqs5iYoPsw6hARQr6gvokdFE-H5USMiNVGOUtNkJI=
>
> https://urldefense.proofpoint.com/v2/url?u=https-3A__github.
> com_apache_kafka_pull_4987=DwIBaQ=7563p3e2zaQw0AB1wrFVgy
> agb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaC
> bPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k&
> s=Hqn5dOgQy4-MHTIJLE49O8bNomry3SoGq9OVoHU-CRA=
>
> Looking forward to the community's feedback.
> It would be amazing to have it voted by May 22nd :-) :-)
>
> Edoardo & Mickael
>


[jira] [Created] (KAFKA-6900) Add thenCompose to KafkaFuture

2018-05-14 Thread Richard Tjerngren (JIRA)
Richard Tjerngren created KAFKA-6900:


 Summary: Add thenCompose to KafkaFuture
 Key: KAFKA-6900
 URL: https://issues.apache.org/jira/browse/KAFKA-6900
 Project: Kafka
  Issue Type: Improvement
  Components: clients
Affects Versions: 1.1.0
Reporter: Richard Tjerngren


KafkaFuture supports Future chaining via the thenApply method just like 
CompletableFuture, however, thenApply is not intended to be used for lambdas 
that in turn return a future:

 
{code:java}
KafkaFutureImpl future = new KafkaFutureImpl<>();
KafkaFuture nestedFuture = future.thenApply(result -> 
methodThatReturnsFuture(result));
{code}
Completable future has a method called thenCompose 
[javadoc|https://docs.oracle.com/javase/10/docs/api/java/util/concurrent/CompletionStage.html#thenCompose(java.util.function.Function)]

The would be:
{code:java}
public KafkaFuture thenCompose(Function();
KafkaFuture nestedFuture = future.thenCompose(result -> 
methodThatReturnsFuture(result));
{code}
This would enable developers to chain asynchronous calls in a more natural way 
and it also makes KafkaFuture behave more similar to Javas CompletableFuture 
and Javascripts Promise

 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Exactly Once, Producer configuration

2018-05-14 Thread Thomas.Kass
Hi,

the following article describes how to configure Apache Kafka system to enforce 
exactly once approach:

https://www.confluent.io/blog/exactly-once-semantics-are-possible-heres-how-apache-kafka-does-it/

One recommendation states that parameter transactional.id should be set in 
producer.properties file which usually exists only once per broker.

Now I have the following questions. Our system consists of 4 modules which 
interact in a sequential way by exchanging messages using Kafka topics


A --> B --> C --> D

I.e. modules A, B and C act as message producers and modules B, C and D act as 
message consumers (and all modules do some application specific useful 
processing).

1)  Do we need separate producer/consumer configurations for these modules 
and if so where and how do we have to configure them (as producer configuration 
files are related to the number of brokers and not to the number of modules)?
2)  Does transcational.id "only" cover technical aspects of Apache Kafka or 
does it cover application aspects as well (i.e. is my assumption which led to 
question 1 correct)?
3)  In case it is sufficient to configure transactional.id parameter in 
producer.properties files and we have more than one broker (and with that more 
than one producer.properties file). Do the values of the parameter 
transactional.id have to be equal or do there have to be different (pairwise 
disjoint) values?

I would be glad to get answers to these questions.

Kind regards,
Thomas Kass







RE: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-14 Thread Skrzypek, Jonathan
Ah, apologies didn't see there was already a decent amount of discussion on 
this in the PR.

This kind of sounds related to the environment you're running to me.
What is the rationale behind using the advertised listeners to do your load 
balancing advertisement rather than a top level alias that has everything ?

It sounds like in your case there is a mismatch between bootstrap.servers and 
advertised.listeners, and you want advertised.listeners to take precedence and 
have the client iterate over what is returned by the broker.
So the extra parameter doesn't only have to do with DNS but it's also appending 
from the broker, maybe the parameter name should reflect this ?

Jonathan Skrzypek 


-Original Message-
From: Skrzypek, Jonathan [Tech] 
Sent: 14 May 2018 14:46
To: dev@kafka.apache.org
Subject: RE: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved 
IP addresses

Hi,

I see you noted the similarities with KIP-235.
But KIP-235 might also solve what this KIP is trying to achieve.

When parsing bootstrap.servers, KIP-235 has the client add all underlying 
hostnames and IPs.
And this happens before hitting the NetworkClient.

So to me the client will try every single endpoint behind any bootstrap.servers 
record.

See 
https://github.com/apache/kafka/pull/4485/commits/24757eb7b06bcf8c7d7649c85232c52b5d54f0e4#diff-89ef153462e64c250a21bd324ae1a851
which calls getAllByName like you suggested

Jonathan Skrzypek 


-Original Message-
From: Edoardo Comar [mailto:edoco...@gmail.com] 
Sent: 14 May 2018 14:17
To: dev@kafka.apache.org
Subject: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP 
addresses

Hi all,

We just opened a KIP to add support for the client to use all IPs returned
by DNS for the brokers

The details are here -

https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.apache.org_confluence_display_KAFKA_KIP-2D302-2B-2D-2BEnable-2BKafka-2Bclients-2Bto-2Buse-2Ball-2BDNS-2Bresolved-2BIP-2Baddresses=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=C-UZ6KUG7JFiPD_CnHczDOVqH9-XC5f_OFkw4BTNrI4=
 

The JIRA and provisional PR  (where the discussion lead to the creation of
this KIP) are :

https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_KAFKA-2D6863=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=3Puqs5iYoPsw6hARQr6gvokdFE-H5USMiNVGOUtNkJI=
 

https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_kafka_pull_4987=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=Hqn5dOgQy4-MHTIJLE49O8bNomry3SoGq9OVoHU-CRA=
 

Looking forward to the community's feedback.
It would be amazing to have it voted by May 22nd :-) :-)

Edoardo & Mickael


RE: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-14 Thread Skrzypek, Jonathan
Hi,

I see you noted the similarities with KIP-235.
But KIP-235 might also solve what this KIP is trying to achieve.

When parsing bootstrap.servers, KIP-235 has the client add all underlying 
hostnames and IPs.
And this happens before hitting the NetworkClient.

So to me the client will try every single endpoint behind any bootstrap.servers 
record.

See 
https://github.com/apache/kafka/pull/4485/commits/24757eb7b06bcf8c7d7649c85232c52b5d54f0e4#diff-89ef153462e64c250a21bd324ae1a851
which calls getAllByName like you suggested

Jonathan Skrzypek 


-Original Message-
From: Edoardo Comar [mailto:edoco...@gmail.com] 
Sent: 14 May 2018 14:17
To: dev@kafka.apache.org
Subject: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP 
addresses

Hi all,

We just opened a KIP to add support for the client to use all IPs returned
by DNS for the brokers

The details are here -

https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.apache.org_confluence_display_KAFKA_KIP-2D302-2B-2D-2BEnable-2BKafka-2Bclients-2Bto-2Buse-2Ball-2BDNS-2Bresolved-2BIP-2Baddresses=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=C-UZ6KUG7JFiPD_CnHczDOVqH9-XC5f_OFkw4BTNrI4=
 

The JIRA and provisional PR  (where the discussion lead to the creation of
this KIP) are :

https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_KAFKA-2D6863=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=3Puqs5iYoPsw6hARQr6gvokdFE-H5USMiNVGOUtNkJI=
 

https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_kafka_pull_4987=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k=Hqn5dOgQy4-MHTIJLE49O8bNomry3SoGq9OVoHU-CRA=
 

Looking forward to the community's feedback.
It would be amazing to have it voted by May 22nd :-) :-)

Edoardo & Mickael


[DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses

2018-05-14 Thread Edoardo Comar
Hi all,

We just opened a KIP to add support for the client to use all IPs returned
by DNS for the brokers

The details are here -

https://cwiki.apache.org/confluence/display/KAFKA/KIP-302+-+Enable+Kafka+clients+to+use+all+DNS+resolved+IP+addresses

The JIRA and provisional PR  (where the discussion lead to the creation of
this KIP) are :

https://issues.apache.org/jira/browse/KAFKA-6863

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

Looking forward to the community's feedback.
It would be amazing to have it voted by May 22nd :-) :-)

Edoardo & Mickael


RE: [VOTE] KIP-235 Add DNS alias support for secured connection

2018-05-14 Thread Skrzypek, Jonathan
Sure, I modified the KIP to add more details 

https://cwiki.apache.org/confluence/display/KAFKA/KIP-235%3A+Add+DNS+alias+support+for+secured+connection


Jonathan Skrzypek 


-Original Message-
From: Ismael Juma [mailto:ism...@juma.me.uk] 
Sent: 14 May 2018 11:53
To: dev
Subject: Re: [VOTE] KIP-235 Add DNS alias support for secured connection

Thanks for the KIP, Jonathan. It would be helpful to have more detail on
how SSL authentication could be broken if the new behaviour is the default.
I know this was discussed in the mailing list thread, but it's important to
include it in the KIP since it's the main reason why a new config is needed
(and configs should be avoided whenever we can just do the right thing).

Ismael

On Fri, Mar 23, 2018 at 12:05 PM Skrzypek, Jonathan <
jonathan.skrzy...@gs.com> wrote:

> Hi,
>
> I would like to start a vote for KIP-235
>
>
> https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.apache.org_confluence_display_KAFKA_KIP-2D235-253A-2BAdd-2BDNS-2Balias-2Bsupport-2Bfor-2Bsecured-2Bconnection=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=FM_uCHnnO2dqxWC0bi7_QOJKfKmQI80-Xduvb-URWOw=RpGkijfK-WHcU0s8ZtMXEkIr69QraJhYKaGSC9V_rnI=
>  
>
> This is a proposition to add an option for reverse dns lookup of
> bootstrap.servers hosts, allowing the use of dns aliases on clusters using
> SASL authentication.
>
>
>
>


Re: [VOTE] KIP-278: Add version option to Kafka's commands

2018-05-14 Thread Mickael Maison
+1 (non binding)
Thanks for the KIP

On Mon, May 14, 2018 at 10:36 AM, Rajini Sivaram
 wrote:
> Sasaki,
>
> I believe this has 2 binding votes and six non-binding votes, so needs
> another binding vote to pass.
>
> On Mon, May 14, 2018 at 9:00 AM, Sasaki Toru 
> wrote:
>
>> Hi all,
>>
>> This KIP would be accepted with 4 binding and 4 non-binding votes.
>>
>> Thank you for participating.
>>
>>
>> Thanks,
>> Sasaki.
>>
>> 2018-05-12 9:31 GMT+09:00 Colin McCabe :
>>>
>>> +1 (non-binding)

 Colin

 On Fri, May 11, 2018, at 12:35, Attila Sasvári wrote:

> +1 (non-binding)
>
> Thomas Crayford  ezt írta (időpont: 2018.
> máj.
> 11., P 18:20):
>
> +1 (non-binding)
>>
>> On Fri, May 11, 2018 at 5:17 PM, Guozhang Wang 
>>
> wrote:

> Thanks Toru-san, +1 (binding)
>>>
>>> On Fri, May 11, 2018 at 8:30 AM, Jason Gustafson >> wrote:
>>>
>>> Thanks for the KIP! +1 (binding)

 On Fri, May 11, 2018 at 12:35 AM, Manikumar <

>>> manikumar.re...@gmail.com

> wrote:

 +1 (non-binding)
>
> Thanks for the KIP.
>
> On Fri, May 11, 2018 at 12:56 PM, zhenya Sun 
>
 wrote:
>>
>>> +1 building
>>
>>> 在 2018年5月11日,上午9:51,Ted Yu  写道:
>>>
>>> +1
>>>
>>> On Thu, May 10, 2018 at 6:42 PM, Sasaki Toru <
>>>
>> sasaki...@oss.nttdata.com>
>
>> wrote:
>>>
>>> Hi all,

 I would like to start the vote on KIP-278: Add version

>>> option to

> Kafka's
>
>> commands.

 The link to this KIP is here:
 >> 278+-

> +Add+version+option+to+Kafka%27s+commands>

 The discussion thread is here:
 <

>>> https://www.mail-archive.com/dev@kafka.apache.org/msg86688.html>
>>
>>>
 Many thanks,
 Sasaki

 --
 Sasaki Toru(sasaki...@oss.nttdata.com) NTT DATA CORPORATION



>>
>>>
>>> --
>>> -- Guozhang
>>>
>>>
>> --
>> Sasaki Toru(sasaki...@oss.nttdata.com) NTT DATA CORPORATION
>>
>>


Re: [VOTE] KIP-235 Add DNS alias support for secured connection

2018-05-14 Thread Ismael Juma
Thanks for the KIP, Jonathan. It would be helpful to have more detail on
how SSL authentication could be broken if the new behaviour is the default.
I know this was discussed in the mailing list thread, but it's important to
include it in the KIP since it's the main reason why a new config is needed
(and configs should be avoided whenever we can just do the right thing).

Ismael

On Fri, Mar 23, 2018 at 12:05 PM Skrzypek, Jonathan <
jonathan.skrzy...@gs.com> wrote:

> Hi,
>
> I would like to start a vote for KIP-235
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-235%3A+Add+DNS+alias+support+for+secured+connection
>
> This is a proposition to add an option for reverse dns lookup of
> bootstrap.servers hosts, allowing the use of dns aliases on clusters using
> SASL authentication.
>
>
>
>


Re: [VOTE] KIP-278: Add version option to Kafka's commands

2018-05-14 Thread Rajini Sivaram
Sasaki,

I believe this has 2 binding votes and six non-binding votes, so needs
another binding vote to pass.

On Mon, May 14, 2018 at 9:00 AM, Sasaki Toru 
wrote:

> Hi all,
>
> This KIP would be accepted with 4 binding and 4 non-binding votes.
>
> Thank you for participating.
>
>
> Thanks,
> Sasaki.
>
> 2018-05-12 9:31 GMT+09:00 Colin McCabe :
>>
>> +1 (non-binding)
>>>
>>> Colin
>>>
>>> On Fri, May 11, 2018, at 12:35, Attila Sasvári wrote:
>>>
 +1 (non-binding)

 Thomas Crayford  ezt írta (időpont: 2018.
 máj.
 11., P 18:20):

 +1 (non-binding)
>
> On Fri, May 11, 2018 at 5:17 PM, Guozhang Wang 
>
 wrote:
>>>
 Thanks Toru-san, +1 (binding)
>>
>> On Fri, May 11, 2018 at 8:30 AM, Jason Gustafson > wrote:
>>
>> Thanks for the KIP! +1 (binding)
>>>
>>> On Fri, May 11, 2018 at 12:35 AM, Manikumar <
>>>
>> manikumar.re...@gmail.com
>>>
 wrote:
>>>
>>> +1 (non-binding)

 Thanks for the KIP.

 On Fri, May 11, 2018 at 12:56 PM, zhenya Sun 

>>> wrote:
>
>> +1 building
>
>> 在 2018年5月11日,上午9:51,Ted Yu  写道:
>>
>> +1
>>
>> On Thu, May 10, 2018 at 6:42 PM, Sasaki Toru <
>>
> sasaki...@oss.nttdata.com>

> wrote:
>>
>> Hi all,
>>>
>>> I would like to start the vote on KIP-278: Add version
>>>
>> option to
>>>
 Kafka's

> commands.
>>>
>>> The link to this KIP is here:
>>> >>
>> 278+-
>>>
 +Add+version+option+to+Kafka%27s+commands>
>>>
>>> The discussion thread is here:
>>> <
>>>
>> https://www.mail-archive.com/dev@kafka.apache.org/msg86688.html>
>
>>
>>> Many thanks,
>>> Sasaki
>>>
>>> --
>>> Sasaki Toru(sasaki...@oss.nttdata.com) NTT DATA CORPORATION
>>>
>>>
>>>
>
>>
>> --
>> -- Guozhang
>>
>>
> --
> Sasaki Toru(sasaki...@oss.nttdata.com) NTT DATA CORPORATION
>
>


RE: [VOTE] KIP-235 Add DNS alias support for secured connection

2018-05-14 Thread Skrzypek, Jonathan
Up :)
Anyone for a binding vote here ?

Jonathan Skrzypek 

-Original Message-
From: Rajini Sivaram [mailto:rajinisiva...@gmail.com] 
Sent: 10 May 2018 13:17
To: dev
Subject: Re: [VOTE] KIP-235 Add DNS alias support for secured connection

Thanks Jonathan. You have binding votes from me and Gwen. One more binding
vote is required for this KIP to be approved.

On Thu, May 10, 2018 at 1:14 PM, Skrzypek, Jonathan <
jonathan.skrzy...@gs.com> wrote:

> Hi,
>
> Have implemented the changes discussed.
> bootstrap.reverse.dns.lookup is disabled by default.
> When enabled, the client will perform reverse dns lookup regardless of the
> security protocol used.
>
> https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_kafka_pull_4485=DwIBaQ=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E=X8udiS6RLS6dhJElpufCtnaJoeGWVp7TAjcW1o7HYRI=x8aeZlBx-fTv7gYq8qnfX1I3_rQC8-1b4lBUn36b2nU=
>  
>
>
> Jonathan Skrzypek
>
>
> -Original Message-
> From: Skrzypek, Jonathan [Tech]
> Sent: 01 May 2018 17:17
> To: dev
> Subject: RE: [VOTE] KIP-235 Add DNS alias support for secured connection
>
> Oops, yes indeed that makes sense, got confused between SASL_SSL and SSL.
>
> Updated the KIP.
>
>
>
> Jonathan Skrzypek
>
>
> -Original Message-
> From: Rajini Sivaram [mailto:rajinisiva...@gmail.com]
> Sent: 01 May 2018 11:08
> To: dev
> Subject: Re: [VOTE] KIP-235 Add DNS alias support for secured connection
>
> Jonathan,
>
> Not doing the reverse lookup for SASL_SSL limits the usability of this KIP
> since it can no longer be used in a secure environment where Kerberos is
> used with TLS. Perhaps the best option is to do the lookup if the option is
> explicitly enabled regardless of what the security protocol is. If there is
> a SSL handshake failure with this option enabled, the error message can be
> updated to indicate that it could be because a reverse lookup was used. Can
> you state in the KIP that the default value of
> bootstrap.reverse.dns.lookup will
> be false and hence there is no backwards compatibility issue.
>
> On Mon, Apr 30, 2018 at 1:41 PM, Skrzypek, Jonathan <
> jonathan.skrzy...@gs.com> wrote:
>
> > Thanks for your comments.
> > Have updated the KIP.
> >
> > I agree SSL and SASL_SSL will face similar issues and should behave the
> > same.
> > Thinking about this further,  I'm wondering whether setting
> > bootstrap.reverse.dns.lookup to true whilst using any of those protocols
> > should throw a critical error and stop, or at least log a warning stating
> > that the lookup won't be performed.
> > This sounds better than silently ignoring and leave users with the
> > impression they can use SSL and bootstrap server aliases.
> > Abruptly stopping the client sounds a bit extreme so I'm leaning towards
> a
> > warning.
> >
> > Thoughts ?
> >
> > I'm not sure about checking whether the list has IP addresses.
> > There could be cases where the list has a mix of FQDNs and IPs, so I
> would
> > rather perform the lookup regardless of the case when the parameter is
> > enabled.
> >
> > On the security aspects, I am by no means a security or SASL expert so
> > commented the KIP with what I believe to be the case.
> >
> > Jonathan Skrzypek
> >
> > -Original Message-
> > From: Rajini Sivaram [mailto:rajinisiva...@gmail.com]
> > Sent: 29 April 2018 15:38
> > To: dev
> > Subject: Re: [VOTE] KIP-235 Add DNS alias support for secured connection
> >
> > Hi Jonathan,
> >
> > Thanks for the KIP.
> >
> > +1 (binding) with a couple comments below to add more detail to the KIP.
> >
> >1. Make it clearer when the new option `bootstrap.reverse.dns.lookup`
> >should or shouldn't be used. Document security considerations as well
> as
> >other system configurations that may have an impact.
> >2. The PR currently disables the new code path for security protocol
> >SSL. But this doesn't address SASL_SSL which could also do hostname
> >verification. Do we even want to do reverse lookup if bootstrap list
> >contains IP addresses? If we do, we should handle SSL and SASL_SSL in
> > the
> >same way (which basically means handling all protocols in the same
> way).
> >
> >
> > On Thu, Apr 26, 2018 at 2:16 PM, Stephane Maarek <
> > steph...@simplemachines.com.au> wrote:
> >
> > > +1 as a user
> > > BUT
> > >
> > > I am no security expert. I have experienced that issue while setting
> up a
> > > cluster and while I would have liked a feature like that (I opened a
> JIRA
> > > at the time), I always guessed that the reason was because of some
> > security
> > > protection.
> > >
> > > Now from a setup point of view this helps a ton, but I really want to
> > make
> > > sure this doesn't introduce any security risk by relaxing a constraint.
> > >
> > > Is there a security assessment possible by someone accredited ?
> > >
> > > Sorry for raising these questions just want to make sure it's addressed
> > >
> > > On Thu., 26 Apr. 2018, 5:32 pm Gwen 

Re: [VOTE] KIP-294 - Enable TLS hostname verification by default

2018-05-14 Thread Rajini Sivaram
The vote has passed with three binding (Jun, Ismael, me) and five
non-binding (Ted, Michael, Manikumar, Edoardo, Jakub) votes. I will update
the KIP page.

Thanks everyone!

Regards,

Rajini

On Fri, May 11, 2018 at 2:28 PM, Rajini Sivaram 
wrote:

> Hi Jun,
>
> I have updated the KIP with examples on setting ssl.endpoint.
> identification.algorithm to an empty string. It turns out I had to update
> ConfigCommand to do this for dynamic configs, I have updated the PR as well.
>
> Thanks for pointing this out!
>
> Regards,
>
> Rajini
>
>
> On Fri, May 11, 2018 at 12:34 AM, Ismael Juma  wrote:
>
>> Thanks for the KIP, +1 (binding) from me.
>>
>> Ismael
>>
>> On Wed, May 9, 2018 at 8:29 AM Rajini Sivaram 
>> wrote:
>>
>> > Hi all,
>> >
>> > Since there have been no objections on this straightforward KIP, I would
>> > like to initiate the voting process. KIP-294 proposes to use a secure
>> > default value for endpoint identification when using SSL as the security
>> > protocol. The KIP Is here:
>> >
>> >
>> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-294+-+
>> Enable+TLS+hostname+verification+by+default
>> >
>> > If there are any concerns, please add them to this thread or the
>> discussion
>> > thread (https://www.mail-archive.com/dev@kafka.apache.org/msg87549.html
>> )
>> >
>> > Regards,
>> >
>> > Rajini
>> >
>>
>
>


Re: [VOTE] KIP-278: Add version option to Kafka's commands

2018-05-14 Thread Sasaki Toru

Hi all,

This KIP would be accepted with 4 binding and 4 non-binding votes.

Thank you for participating.


Thanks,
Sasaki.

2018-05-12 9:31 GMT+09:00 Colin McCabe :


+1 (non-binding)

Colin

On Fri, May 11, 2018, at 12:35, Attila Sasvári wrote:

+1 (non-binding)

Thomas Crayford  ezt írta (időpont: 2018. máj.
11., P 18:20):


+1 (non-binding)

On Fri, May 11, 2018 at 5:17 PM, Guozhang Wang 

wrote:

Thanks Toru-san, +1 (binding)

On Fri, May 11, 2018 at 8:30 AM, Jason Gustafson 

wrote:

+1 building

在 2018年5月11日,上午9:51,Ted Yu  写道:

+1

On Thu, May 10, 2018 at 6:42 PM, Sasaki Toru <

sasaki...@oss.nttdata.com>

wrote:


Hi all,

I would like to start the vote on KIP-278: Add version

option to

Kafka's

commands.

The link to this KIP is here:


The discussion thread is here:
<

https://www.mail-archive.com/dev@kafka.apache.org/msg86688.html>


Many thanks,
Sasaki

--
Sasaki Toru(sasaki...@oss.nttdata.com) NTT DATA CORPORATION







--
-- Guozhang



--
Sasaki Toru(sasaki...@oss.nttdata.com) NTT DATA CORPORATION



Re: [DISCUSS] KIP-273 Kafka to support using ETCD beside Zookeeper

2018-05-14 Thread Molnár Bálint
Hi Colin,

Do you have a rough estimate on that?

Thanks,
Balint

Colin McCabe  ezt írta (időpont: 2018. máj. 9., Sze,
19:53):

> Hi Molnar,
>
> The points Ismael brought up earlier (and that were brought up on KIP-30)
> are still relevant here.  As Ismael said, the goal is to get rid of
> external dependencies here.   We're going to post more about this soon
> (sorry for the delay)
>
> thanks,
> Colin
>
>
> On Wed, May 9, 2018, at 07:29, Molnár Bálint wrote:
> > Hi,
> > I just rebased the Etcd implementation proposal on trunk. Pinging to see
> if
> > anyone has feedback on my questions from my previous email.
> >
> > Molnár Bálint  ezt írta (időpont: 2018. ápr. 4.,
> > Sze, 10:08):
> >
> > > Hi,
> > > Thanks again for the feedback.
> > >
> > > Is there already ongoing work for having an own consensus
> implementation
> > > within Kafka?
> > > If that work haven't started yet, we think there is value in having an
> > > interim solution, that allows the use of another consensus system
> besides
> > > Zookeeper.
> > >
> > > We ask the community to take a look at the Etcd implementation proposal
> > > we created and provide feedback on that.
> > > This helps to asses rather this approach is viable at all.
> > >
> > > We are open to collaborate on integrating our proposed Etcd
> implementation
> > > into any integration test system, to certify that all use cases works
> as
> > > expected.
> > >
> > > Balint
> > >
> > > 2018-03-30 22:21 GMT+02:00 Gwen Shapira :
> > >
> > >> Hi,
> > >>
> > >> I had an offline discussion with Ismael and wanted to summarize the
> > >> comments and questions he raised so we are all on the same page.
> > >>
> > >> The core issue is that this change adds a new public API. Since we
> already
> > >> know that the goal for the next 1-2 years is to get rid of ZK
> completely.
> > >> Do we want to go to the effort of adding (and discussing and
> reviewing) a
> > >> new public API, knowing that it will be completely removed in a year?
> And
> > >> since building and testing a plugin also involves effort, will anyone
> do
> > >> it
> > >> for something that is going to be temporary by design?
> > >>
> > >> Ismael, correct me if this isn't a fair representation of your
> concerns.
> > >>
> > >> Gwen
> > >>
> > >>
> > >>
> > >> On Thu, Mar 29, 2018 at 9:33 AM, Gwen Shapira 
> wrote:
> > >>
> > >> > Few other concerns that were raised in the previous discussion were
> > >> around
> > >> > the challenges both to maintainers and users in making this API
> > >> pluggable
> > >> > and how does making the interface pluggable aligns with future
> goals for
> > >> > the project. At the time this was difficult to discuss because there
> > >> wasn't
> > >> > a concrete proposal. I want to discuss these points in the context
> of
> > >> this
> > >> > specific proposal:
> > >> >
> > >> > 1. Problem: Pluggable APIs mean larger surface testing area and
> multiple
> > >> > implementations to cover.
> > >> > In this case: At the time, the Kafka project didn't have much
> > >> > experience with pluggable APIs and components, so the concerns were
> very
> > >> > valid. Right now Kafka has multiple pluggable components -
> Connectors,
> > >> > converters, transformations, authentication protocols, authorization
> > >> > database, coordination protocol, serializers, etc. I think that as a
> > >> > community we gotten better at testing the interface, testing the
> very
> > >> few
> > >> > implementations that are included in Apache Kafka itself and
> allowing
> > >> the
> > >> > community to innovate and validate outside of the Kafka project. I
> don't
> > >> > recall major issues either from lack of testing or from usability
> > >> > perspective.
> > >> >
> > >> > 2. Problem: Users don't want to choose a consensus implementation,
> they
> > >> > just don't want ZK.
> > >> > In this case: I agree that users don't actually want to spend
> time
> > >> > choosing consensus implementation and a simpler deployment model
> would
> > >> > serve them better. IMO, if Apache Kafka ships with our well-tested
> ZK
> > >> > implementation, 99% of the users will choose to use that (a vast
> > >> majority
> > >> > uses our less-than-amazing authorization plugin), and the few that
> > >> really
> > >> > need something else for whatever reason, will be able to get what
> they
> > >> > need. As Jake said, we need to face the fact that development
> > >> trajectory of
> > >> > ZK isn't amazing at the moment, that it is lacking features our
> users
> > >> need
> > >> > (SSL) and it will be good to allow the user community to explore
> > >> > alternatives.
> > >> >
> > >> > 3. Problem: Why got to the effort of refactoring if we know we want
> to
> > >> get
> > >> > rid of ZK.
> > >> > In this case: This change isn't huge, it doesn't rewrite large
> > >> > portions of Kafka and it does not make the future direction any more
> > >> > 

[jira] [Resolved] (KAFKA-5909) Remove source jars from classpath while executing CLI tools

2018-05-14 Thread Kamal Chandraprakash (JIRA)

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

Kamal Chandraprakash resolved KAFKA-5909.
-
Resolution: Not A Problem

> Remove source jars from classpath while executing CLI tools
> ---
>
> Key: KAFKA-5909
> URL: https://issues.apache.org/jira/browse/KAFKA-5909
> Project: Kafka
>  Issue Type: Bug
>  Components: tools
>Reporter: Kamal Chandraprakash
>Assignee: Kamal Chandraprakash
>Priority: Minor
>  Labels: newbie
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)