Re: [DISCUSS] KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics

2017-10-03 Thread Stephane Maarek
Hi Randall,

Thanks for the KIP. I like it
What happens when the target topic is already created but the configs do not 
match? 
i.e. wrong RF, num partitions, or missing / additional configs? Will you 
attempt to apply the necessary changes or throw an error?

Thanks!
Stephane
 

On 24/5/17, 5:59 am, "Mathieu Fenniak"  wrote:

Ah, yes, I see you a highlighted part that should've made this clear
to me the first read. :-)  Much clearer now!

By the way, enjoyed your Debezium talk in NYC.

Looking forward to this Kafka Connect change; it will allow me to
remove a post-deployment tool that I hacked together for the purpose
of ensuring auto-created topics have the right config.

Mathieu


On Tue, May 23, 2017 at 11:38 AM, Randall Hauch  wrote:
> Thanks for the quick feedback, Mathieu. Yes, the first configuration rule
> whose regex matches will be applied, and no other rules will be used. I've
> updated the KIP to try to make this more clear, but let me know if it's
> still not clear.
>
> Best regards,
>
> Randall
>
> On Tue, May 23, 2017 at 10:07 AM, Mathieu Fenniak <
> mathieu.fenn...@replicon.com> wrote:
>
>> Hi Randall,
>>
>> Awesome, very much looking forward to this.
>>
>> It isn't 100% clear from the KIP how multiple config-based rules would
>> be applied; it looks like the first configuration rule whose regex
>> matches the topic name will be used, and no other rules will be
>> applied.  Is that correct?  (I wasn't sure if it might cascade
>> together multiple matching rules...)
>>
>> Looks great,
>>
>> Mathieu
>>
>>
>> On Mon, May 22, 2017 at 1:43 PM, Randall Hauch  wrote:
>> > Hi, all.
>> >
>> > We recently added the ability for Kafka Connect to create *internal*
>> topics
>> > using the new AdminClient, but it still would be great if Kafka Connect
>> > could do this for new topics that result from source connector records.
>> > I've outlined an approach to do this in "KIP-158 Kafka Connect should
>> allow
>> > source connectors to set topic-specific settings for new topics".
>> >
>> > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 158%3A+Kafka+Connect+should+allow+source+connectors+to+
>> set+topic-specific+settings+for+new+topics
>> > > 158%3A+Kafka+Connect+should+allow+source+connectors+to+
>> set+topic-specific+settings+for+new+topics>*
>> >
>> > Please take a look and provide feedback. Thanks!
>> >
>> > Best regards,
>> >
>> > Randall
>>





[GitHub] kafka pull request #4011: KAFKA-5903: Added Connect metrics to the worker an...

2017-10-03 Thread rhauch
GitHub user rhauch opened a pull request:

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

KAFKA-5903: Added Connect metrics to the worker and distributed herder

Added metrics to the Connect worker and rebalancing metrics to the 
distributed herder.

This is built on top of #3987, and I can rebase this PR once that is merged.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/rhauch/kafka kafka-5903

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/4011.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4011


commit d4afe3d2d0ca0c313e70266dca104caa0564d3f3
Author: Randall Hauch 
Date:   2017-09-28T19:12:21Z

KAFKA-5990 Enable generation of metrics docs for Connect

A new mechanism was added recently to the Metrics framework to make it 
easier to generate the documentation. It uses a registry with a 
MetricsNameTemplate for each metric, and then those templates are used when 
creating the actual metrics. The metrics framework provides utilities that can 
generate the HTML documentation from the registry of templates.

This change moves the recently-added Connect metrics over to use these 
templates and to then generate the metric documentation for Connect.

commit d7bab224800c4d56d754b7121abb209df9715ca1
Author: Randall Hauch 
Date:   2017-10-02T20:37:05Z

KAFKA-5903 Use string-valued metrics for connector name, type, version, and 
status

Changed from the indicator metrics for connector and task status that 
reported the various boolean states to a single string-valued metric for 
status. Also added string-valued metrics for connector name, version, and type.

commit 75ffb38f32f552dbf667553f998a9cde7af6465f
Author: Randall Hauch 
Date:   2017-10-03T01:05:44Z

KAFKA-5903 Added Connect worker metrics

commit 24efd5f29fe4b23da0c7f9949361b14fad8fbc4e
Author: Randall Hauch 
Date:   2017-10-04T05:15:19Z

KAFKA-5904 Added Connect rebalancing metrics to distributed herder

Added the rebalancing metrics to the distributed herder.




---


Build failed in Jenkins: kafka-trunk-jdk8 #2099

2017-10-03 Thread Apache Jenkins Server
See 


Changes:

[junrao] KAFKA-5864; ReplicaFetcherThread should not die due to replica in

--
[...truncated 3.34 MB...]
kafka.security.auth.SimpleAclAuthorizerTest > testAclInheritance PASSED

kafka.security.auth.SimpleAclAuthorizerTest > 
testDistributedConcurrentModificationOfResourceAcls STARTED

kafka.security.auth.SimpleAclAuthorizerTest > 
testDistributedConcurrentModificationOfResourceAcls PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testAclManagementAPIs STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testAclManagementAPIs PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testWildCardAcls STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testWildCardAcls PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testTopicAcl STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testTopicAcl PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testSuperUserHasAccess STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testSuperUserHasAccess PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testDenyTakesPrecedence STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testDenyTakesPrecedence PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testNoAclFoundOverride STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testNoAclFoundOverride PASSED

kafka.security.auth.SimpleAclAuthorizerTest > 
testHighConcurrencyModificationOfResourceAcls STARTED

kafka.security.auth.SimpleAclAuthorizerTest > 
testHighConcurrencyModificationOfResourceAcls PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache PASSED

kafka.integration.PrimitiveApiTest > testMultiProduce STARTED

kafka.integration.PrimitiveApiTest > testMultiProduce PASSED

kafka.integration.PrimitiveApiTest > testDefaultEncoderProducerAndFetch STARTED

kafka.integration.PrimitiveApiTest > testDefaultEncoderProducerAndFetch PASSED

kafka.integration.PrimitiveApiTest > testFetchRequestCanProperlySerialize 
STARTED

kafka.integration.PrimitiveApiTest > testFetchRequestCanProperlySerialize PASSED

kafka.integration.PrimitiveApiTest > testPipelinedProduceRequests STARTED

kafka.integration.PrimitiveApiTest > testPipelinedProduceRequests PASSED

kafka.integration.PrimitiveApiTest > testProduceAndMultiFetch STARTED

kafka.integration.PrimitiveApiTest > testProduceAndMultiFetch PASSED

kafka.integration.PrimitiveApiTest > 
testDefaultEncoderProducerAndFetchWithCompression STARTED

kafka.integration.PrimitiveApiTest > 
testDefaultEncoderProducerAndFetchWithCompression PASSED

kafka.integration.PrimitiveApiTest > testConsumerEmptyTopic STARTED

kafka.integration.PrimitiveApiTest > testConsumerEmptyTopic PASSED

kafka.integration.PrimitiveApiTest > testEmptyFetchRequest STARTED

kafka.integration.PrimitiveApiTest > testEmptyFetchRequest PASSED

kafka.integration.MetricsDuringTopicCreationDeletionTest > 
testMetricsDuringTopicCreateDelete STARTED

kafka.integration.MetricsDuringTopicCreationDeletionTest > 
testMetricsDuringTopicCreateDelete PASSED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooLow 
STARTED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooLow PASSED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooLow 
STARTED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooLow 
PASSED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooHigh 
STARTED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooHigh 
PASSED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooHigh 
STARTED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooHigh 
PASSED

kafka.integration.TopicMetadataTest > testIsrAfterBrokerShutDownAndJoinsBack 
STARTED

kafka.integration.TopicMetadataTest > testIsrAfterBrokerShutDownAndJoinsBack 
PASSED

kafka.integration.TopicMetadataTest > testAutoCreateTopicWithCollision STARTED

kafka.integration.TopicMetadataTest > testAutoCreateTopicWithCollision PASSED

kafka.integration.TopicMetadataTest > testAliveBrokerListWithNoTopics STARTED

kafka.integration.TopicMetadataTest > testAliveBrokerListWithNoTopics PASSED

kafka.integration.TopicMetadataTest > testAutoCreateTopic STARTED

kafka.integration.TopicMetadataTest > testAutoCreateTopic PASSED

kafka.integration.TopicMetadataTest > testGetAllTopicMetadata STARTED

kafka.integration.TopicMetadataTest > testGetAllTopicMetadata PASSED

kafka.integration.TopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterNewBrokerStartup STARTED

kafka.integration.TopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterNewBrokerStartup PASSED

kafka.integration.TopicMetadataTest > testBasicTopicMetadata STARTED

kafka.integration.TopicMetadataTest > testBasicTopicMetadata PASSED


Jenkins build is back to normal : kafka-trunk-jdk9 #86

2017-10-03 Thread Apache Jenkins Server
See 




[GitHub] kafka-site pull request #77: MINOR: Update verbiage on landing page https://...

2017-10-03 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka-site/pull/77


---


[GitHub] kafka-site issue #77: MINOR: Update verbiage on landing page https://kafka.a...

2017-10-03 Thread guozhangwang
Github user guozhangwang commented on the issue:

https://github.com/apache/kafka-site/pull/77
  
LGTM. Merged to asf-site.


---


[GitHub] kafka pull request #3820: KAFKA-5864; ReplicaFetcherThread should not die du...

2017-10-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---


Build failed in Jenkins: kafka-trunk-jdk8 #2098

2017-10-03 Thread Apache Jenkins Server
See 


Changes:

[me] KAFKA-5902: Added sink task metrics (KIP-196)

--
[...truncated 1.39 MB...]
org.apache.kafka.common.security.JaasContextTest > testControlFlag PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
noAuthorizationIdSpecified STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
noAuthorizationIdSpecified PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdEqualsAuthenticationId STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdEqualsAuthenticationId PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdNotEqualsAuthenticationId STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdNotEqualsAuthenticationId PASSED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testProducerWithInvalidCredentials STARTED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testProducerWithInvalidCredentials PASSED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testTransactionalProducerWithInvalidCredentials STARTED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testTransactionalProducerWithInvalidCredentials PASSED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testConsumerWithInvalidCredentials STARTED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testConsumerWithInvalidCredentials PASSED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testAdminClientWithInvalidCredentials STARTED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testAdminClientWithInvalidCredentials PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingUsernameSaslPlain STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingUsernameSaslPlain PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testValidSaslScramMechanisms STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testValidSaslScramMechanisms PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramSslServerWithoutSaslAuthenticateHeaderFailure STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramSslServerWithoutSaslAuthenticateHeaderFailure PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramPlaintextServerWithoutSaslAuthenticateHeaderFailure STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramPlaintextServerWithoutSaslAuthenticateHeaderFailure PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramPlaintextServerWithoutSaslAuthenticateHeader STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramPlaintextServerWithoutSaslAuthenticateHeader PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMechanismPluggability STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMechanismPluggability PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testScramUsernameWithSpecialCharacters STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testScramUsernameWithSpecialCharacters PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testApiVersionsRequestWithUnsupportedVersion STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testApiVersionsRequestWithUnsupportedVersion PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingPasswordSaslPlain STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingPasswordSaslPlain PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testInvalidLoginModule STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testInvalidLoginModule PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslPlainPlaintextClientWithoutSaslAuthenticateHeader STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslPlainPlaintextClientWithoutSaslAuthenticateHeader PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslPlainSslClientWithoutSaslAuthenticateHeader STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslPlainSslClientWithoutSaslAuthenticateHeader PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 

Re: [VOTE] KIP-206: Add support for UUID serialization and deserialization

2017-10-03 Thread Ted Yu
+1

 Original message From: Jakub Scholz  Date: 
10/3/17  2:24 PM  (GMT-08:00) To: dev@kafka.apache.org Subject: [VOTE] KIP-206: 
Add support for UUID serialization and deserialization 
Hi,

Since there were no further discussion points, I would like to start the
voting for KIP-206.

For more details about the KIP go to
https://cwiki.apache.org/confluence/display/KAFKA/KIP-206%3A+Add+support+for+UUID+serialization+and+deserialization

Thanks & Regards
Jakub


[VOTE] KIP-206: Add support for UUID serialization and deserialization

2017-10-03 Thread Jakub Scholz
Hi,

Since there were no further discussion points, I would like to start the
voting for KIP-206.

For more details about the KIP go to
https://cwiki.apache.org/confluence/display/KAFKA/KIP-206%3A+Add+support+for+UUID+serialization+and+deserialization

Thanks & Regards
Jakub


Build failed in Jenkins: kafka-trunk-jdk8 #2097

2017-10-03 Thread Apache Jenkins Server
See 


Changes:

[me] MINOR: Add / to connect docs, because HTML render doesn't respect

[jason] KAFKA-4108; Improve DumpLogSegments offsets-decoder output format

--
[...truncated 3.34 MB...]
kafka.security.auth.SimpleAclAuthorizerTest > testAclInheritance PASSED

kafka.security.auth.SimpleAclAuthorizerTest > 
testDistributedConcurrentModificationOfResourceAcls STARTED

kafka.security.auth.SimpleAclAuthorizerTest > 
testDistributedConcurrentModificationOfResourceAcls PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testAclManagementAPIs STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testAclManagementAPIs PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testWildCardAcls STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testWildCardAcls PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testTopicAcl STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testTopicAcl PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testSuperUserHasAccess STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testSuperUserHasAccess PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testDenyTakesPrecedence STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testDenyTakesPrecedence PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testNoAclFoundOverride STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testNoAclFoundOverride PASSED

kafka.security.auth.SimpleAclAuthorizerTest > 
testHighConcurrencyModificationOfResourceAcls STARTED

kafka.security.auth.SimpleAclAuthorizerTest > 
testHighConcurrencyModificationOfResourceAcls PASSED

kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache STARTED

kafka.security.auth.SimpleAclAuthorizerTest > testLoadCache PASSED

kafka.integration.PrimitiveApiTest > testMultiProduce STARTED

kafka.integration.PrimitiveApiTest > testMultiProduce PASSED

kafka.integration.PrimitiveApiTest > testDefaultEncoderProducerAndFetch STARTED

kafka.integration.PrimitiveApiTest > testDefaultEncoderProducerAndFetch PASSED

kafka.integration.PrimitiveApiTest > testFetchRequestCanProperlySerialize 
STARTED

kafka.integration.PrimitiveApiTest > testFetchRequestCanProperlySerialize PASSED

kafka.integration.PrimitiveApiTest > testPipelinedProduceRequests STARTED

kafka.integration.PrimitiveApiTest > testPipelinedProduceRequests PASSED

kafka.integration.PrimitiveApiTest > testProduceAndMultiFetch STARTED

kafka.integration.PrimitiveApiTest > testProduceAndMultiFetch PASSED

kafka.integration.PrimitiveApiTest > 
testDefaultEncoderProducerAndFetchWithCompression STARTED

kafka.integration.PrimitiveApiTest > 
testDefaultEncoderProducerAndFetchWithCompression PASSED

kafka.integration.PrimitiveApiTest > testConsumerEmptyTopic STARTED

kafka.integration.PrimitiveApiTest > testConsumerEmptyTopic PASSED

kafka.integration.PrimitiveApiTest > testEmptyFetchRequest STARTED

kafka.integration.PrimitiveApiTest > testEmptyFetchRequest PASSED

kafka.integration.MetricsDuringTopicCreationDeletionTest > 
testMetricsDuringTopicCreateDelete STARTED

kafka.integration.MetricsDuringTopicCreationDeletionTest > 
testMetricsDuringTopicCreateDelete PASSED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooLow 
STARTED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooLow PASSED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooLow 
STARTED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooLow 
PASSED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooHigh 
STARTED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooHigh 
PASSED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooHigh 
STARTED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooHigh 
PASSED

kafka.integration.TopicMetadataTest > testIsrAfterBrokerShutDownAndJoinsBack 
STARTED

kafka.integration.TopicMetadataTest > testIsrAfterBrokerShutDownAndJoinsBack 
PASSED

kafka.integration.TopicMetadataTest > testAutoCreateTopicWithCollision STARTED

kafka.integration.TopicMetadataTest > testAutoCreateTopicWithCollision PASSED

kafka.integration.TopicMetadataTest > testAliveBrokerListWithNoTopics STARTED

kafka.integration.TopicMetadataTest > testAliveBrokerListWithNoTopics PASSED

kafka.integration.TopicMetadataTest > testAutoCreateTopic STARTED

kafka.integration.TopicMetadataTest > testAutoCreateTopic PASSED

kafka.integration.TopicMetadataTest > testGetAllTopicMetadata STARTED

kafka.integration.TopicMetadataTest > testGetAllTopicMetadata PASSED

kafka.integration.TopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterNewBrokerStartup STARTED

kafka.integration.TopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterNewBrokerStartup PASSED

kafka.integration.TopicMetadataTest > testBasicTopicMetadata STARTED


Re: TimeoutException: Timed out waiting for a node assignment. when AdminClient attempts to create topics

2017-10-03 Thread Elliot Metsger
I always do this: post a detailed explanation and then find some
discrepancy: the earlier message contains log snippits from another test
run and so the ports don't match up with my text.  Here's log snippets from
the posted files:


Queuing the call:
13:52:40.968 [  main] DEBUG
[KafkaAdminClient$AdminClientRu] - adminclient-1: queueing
Call(callName=createTopics, deadlineMs=1507053280963) with a timeout 12
ms from now.

Encountering connection refused:
13:52:41.024 [-client-thread | adminclient-1] TRACE [
 NetworkClient] - Found least loaded node 127.0.0.1:55383 (id: -1 rack:
null)
13:52:41.024 [-client-thread | adminclient-1] DEBUG
[NetworkClient$DefaultMetadataU] - Initialize connection to node -1 for
sending metadata request
13:52:41.025 [-client-thread | adminclient-1] DEBUG [
 NetworkClient] - Initiating connection to node -1 at 127.0.0.1:55383.
13:52:41.024 [nerName(PLAINTEXT)-PLAINTEXT-0] TRACE [
 Logging$class] - Socket server received response to send, registering for
write and sending data: Response(Request(0,127.0.0.1:55388-127.0.0.1:55390
,Session(User:ANONYMOUS,/127.0.0.1
),null,142764461178823,ListenerName(PLAINTEXT),PLAINTEXT),Some(org.apache.kafka.common.network.NetworkSend@54b50f3c
),SendAction)
13:52:41.025 [-client-thread | adminclient-1] DEBUG [
 Metrics] - Added sensor with name node--1.bytes-sent
13:52:41.027 [-client-thread | adminclient-1] DEBUG [
 Metrics] - Added sensor with name node--1.bytes-received
13:52:41.028 [-client-thread | adminclient-1] DEBUG [
 Metrics] - Added sensor with name node--1.latency
13:52:41.029 [ller-0-to-broker-0-send-thread] TRACE [
 NetworkClient] - Completed receive from node 0, for key 6, received
{error_code=0}
13:52:41.032 [-client-thread | adminclient-1] DEBUG [
Selector] - Connection with /127.0.0.1 disconnected
java.net.ConnectException: Connection refused
at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
at
sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
at
org.apache.kafka.common.network.PlaintextTransportLayer.finishConnect(PlaintextTransportLayer.java:50)
at
org.apache.kafka.common.network.KafkaChannel.finishConnect(KafkaChannel.java:95)
at
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:359)
at org.apache.kafka.common.network.Selector.poll(Selector.java:326)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:432)
at
org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.run(KafkaAdminClient.java:927)

And finally the TimeoutException:

13:54:40.964 [-client-thread | adminclient-1] DEBUG [
 KafkaAdminClient$Call] - Call(callName=createTopics,
deadlineMs=1507053280963) timed out at 1507053280964 after 1 attempt(s)
java.lang.Exception: TimeoutException: Timed out waiting for a node
assignment.
at
org.apache.kafka.clients.admin.KafkaAdminClient$Call.fail(KafkaAdminClient.java:475)
at
org.apache.kafka.clients.admin.KafkaAdminClient$TimeoutProcessor.handleTimeouts(KafkaAdminClient.java:591)
at
org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.timeoutNewCalls(KafkaAdminClient.java:663)
at
org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.run(KafkaAdminClient.java:908)
at java.lang.Thread.run(Thread.java:748)


Full trace:

13:54:40.966 [  main] ERROR [
TestContextManager] - Caught exception while allowing TestExecutionListener
[org.springframework.test.context.support.DependencyInjectionTestExecutionListener@72ccd81a]
to prepare test instance
[info.rmapproject.core.model.impl.openrdf.ORMapEventCreationTest@6d8792db]
java.lang.IllegalStateException: Failed to load ApplicationContext
at
org.springframework.test.context.cache.DefaultCacheAwareContextLoaderDelegate.loadContext(DefaultCacheAwareContextLoaderDelegate.java:125)
at
org.springframework.test.context.support.DefaultTestContext.getApplicationContext(DefaultTestContext.java:107)
at
org.springframework.test.context.support.DependencyInjectionTestExecutionListener.injectDependencies(DependencyInjectionTestExecutionListener.java:117)
at
org.springframework.test.context.support.DependencyInjectionTestExecutionListener.prepareTestInstance(DependencyInjectionTestExecutionListener.java:83)
at
org.springframework.test.context.TestContextManager.prepareTestInstance(TestContextManager.java:242)
at
org.springframework.test.context.junit4.SpringJUnit4ClassRunner.createTest(SpringJUnit4ClassRunner.java:227)
at
org.springframework.test.context.junit4.SpringJUnit4ClassRunner$1.runReflectiveCall(SpringJUnit4ClassRunner.java:289)
at
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at
org.springframework.test.context.junit4.SpringJUnit4ClassRunner.methodBlock(SpringJUnit4ClassRunner.java:291)
at
org.springframework.test.context.junit4.SpringJUnit4ClassRunner.runChild(SpringJUnit4ClassRunner.java:246)
at

Build failed in Jenkins: kafka-trunk-jdk9 #85

2017-10-03 Thread Apache Jenkins Server
See 


Changes:

[me] KAFKA-5902: Added sink task metrics (KIP-196)

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on H26 (couchdbtest ubuntu xenial) in workspace 

 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url https://github.com/apache/kafka.git # timeout=10
Fetching upstream changes from https://github.com/apache/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress https://github.com/apache/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
 > git rev-parse refs/remotes/origin/trunk^{commit} # timeout=10
 > git rev-parse refs/remotes/origin/origin/trunk^{commit} # timeout=10
Checking out Revision 05357b7030c784a6548453f533d3c00e19548ba2 
(refs/remotes/origin/trunk)
Commit message: "KAFKA-5902: Added sink task metrics (KIP-196)"
 > git config core.sparsecheckout # timeout=10
 > git checkout -f 05357b7030c784a6548453f533d3c00e19548ba2
 > git rev-list 5663f51edfeb8f642fe74955021d6f38e9d9abe4 # timeout=10
Setting 
GRADLE_3_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_3.4-rc-2
[kafka-trunk-jdk9] $ /bin/bash -xe /tmp/jenkins5985609650175284353.sh
+ rm -rf 
+ 
/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_3.4-rc-2/bin/gradle

ERROR: JAVA_HOME is set to an invalid directory: 
/home/jenkins/tools/java/latest1.9

Please set the JAVA_HOME variable in your environment to match the
location of your Java installation.

Build step 'Execute shell' marked build as failure
Recording test results
Setting 
GRADLE_3_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_3.4-rc-2
ERROR: Step ?Publish JUnit test result report? failed: Test reports were found 
but none of them are new. Did tests run? 
For example, 

 is 11 days old

Setting 
GRADLE_3_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_3.4-rc-2


TimeoutException: Timed out waiting for a node assignment. when AdminClient attempts to create topics

2017-10-03 Thread Elliot Metsger
Hi everyone,

New user to Kafka, and am psyched to get it working in our environment!
I'm on 0.11.0.0, and using spring-kafka 2.0.0.rc1.  I'm struggling to get
things working in our unit/integration tests.

I don't want to overload everyone with the context of the issues, but this
is the stacktrace I'm seeing (from the tail end of the Maven build):
https://gist.github.com/emetsger/05ef47ce15ca54b2735a65627992a2d0

The full build output is here: https://goo.gl/pwM8Lw

It appears that the AdminClient times out trying to create the topic for
the unit test:

13:54:40.964 [-client-thread | adminclient-1] DEBUG [
 KafkaAdminClient$Call] - Call(callName=createTopics,
deadlineMs=1507053280963) timed out at 1507053280964 after 1 attempt(s)
java.lang.Exception: TimeoutException: Timed out waiting for a node
assignment.
at
org.apache.kafka.clients.admin.KafkaAdminClient$Call.fail(KafkaAdminClient.java:475)
at
org.apache.kafka.clients.admin.KafkaAdminClient$TimeoutProcessor.handleTimeouts(KafkaAdminClient.java:591)
at
org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.timeoutNewCalls(KafkaAdminClient.java:663)
at
org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.run(KafkaAdminClient.java:908)
at java.lang.Thread.run(Thread.java:748)

The unit test uses the maven-buildhelper-plugin to reserve a port for the
Kakfa broker, in this case, port 55383.  If you look at the gist, I've
added the properties used by the AdminClient and the Broker, and it seems
that they are configured properly (that is, the AC is configured to talk to
127.0.0.1:55383, which is where the broker is listening).

Another relevant note is that I'm using spring-kafka, and Spring is
managing all the beans related to Kafka: the producer, consumer, and
broker.  The broker is being started using the @EmbeddedKafka annotation,
which will create an embedded instance of Zookeeper, and an EmbeddedKafka
instance:

@TestPropertySource(locations = { "classpath:/rmapcore.properties",
"classpath:/kafka-broker.properties" })
@EmbeddedKafka(topics = { "rmap-event-topic" }, brokerProperties = {
"log.dir=${kafka.broker.logs-dir}", "port=${kafka.broker.port}" })

Kafka should be automatically creating the topic (`rmap-event-topic` in
this case), and it looks like the AdminClient tries, but ultimately fails,
timing out trying to connect to the broker:

15:21:34.977 [  main] DEBUG
[KafkaAdminClient$AdminClientRu] - adminclient-1: queueing
Call(callName=createTopics, deadlineMs=1507058614976) with a timeout 12
ms from now.
...
15:21:35.047 [-client-thread | adminclient-1] TRACE [
 NetworkClient] - Found least loaded node 127.0.0.1:58230 (id: -1 rack:
null)
15:21:35.047 [-client-thread | adminclient-1] DEBUG
[NetworkClient$DefaultMetadataU] - Initialize connection to node -1 for
sending metadata request
15:21:35.047 [-client-thread | adminclient-1] DEBUG [
 NetworkClient] - Initiating connection to node -1 at 127.0.0.1:58230.
15:21:35.048 [-client-thread | adminclient-1] DEBUG [
 Metrics] - Added sensor with name node--1.bytes-sent
15:21:35.048 [-client-thread | adminclient-1] DEBUG [
 Metrics] - Added sensor with name node--1.bytes-received
15:21:35.049 [-client-thread | adminclient-1] DEBUG [
 Metrics] - Added sensor with name node--1.latency
15:21:35.055 [-client-thread | adminclient-1] DEBUG [
Selector] - Connection with /127.0.0.1 disconnected
java.net.ConnectException: Connection refused
at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
at
org.apache.kafka.common.network.PlaintextTransportLayer.finishConnect(PlaintextTransportLayer.java:50)
at
org.apache.kafka.common.network.KafkaChannel.finishConnect(KafkaChannel.java:95)
at
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:359)
at org.apache.kafka.common.network.Selector.poll(Selector.java:326)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:432)
at
org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.run(KafkaAdminClient.java:927)
at java.lang.Thread.run(Thread.java:748)

Polling continues, and ultimately times out:

15:23:34.981 [-client-thread | adminclient-1] DEBUG [
 KafkaAdminClient$Call] - Call(callName=createTopics,
deadlineMs=1507058614976) timed out at 1507058614981 after 1 attempt(s)
java.lang.Exception: TimeoutException: Timed out waiting for a node
assignment.
at
org.apache.kafka.clients.admin.KafkaAdminClient$Call.fail(KafkaAdminClient.java:475)
at
org.apache.kafka.clients.admin.KafkaAdminClient$TimeoutProcessor.handleTimeouts(KafkaAdminClient.java:591)
at
org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.timeoutNewCalls(KafkaAdminClient.java:663)
at
org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.run(KafkaAdminClient.java:908)
at java.lang.Thread.run(Thread.java:748)


Any help or advice is appreciated!

Thanks,
Elliot

Feel free to follow the steps to reproduce 

[GitHub] kafka pull request #3975: KAFKA-5902: Added sink task metrics

2017-10-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---


[jira] [Resolved] (KAFKA-5902) Create Connect metrics for sink tasks

2017-10-03 Thread Ewen Cheslack-Postava (JIRA)

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

Ewen Cheslack-Postava resolved KAFKA-5902.
--
Resolution: Fixed

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

> Create Connect metrics for sink tasks
> -
>
> Key: KAFKA-5902
> URL: https://issues.apache.org/jira/browse/KAFKA-5902
> Project: Kafka
>  Issue Type: Sub-task
>  Components: KafkaConnect
>Affects Versions: 0.11.0.0
>Reporter: Randall Hauch
>Assignee: Randall Hauch
>Priority: Blocker
> Fix For: 1.0.0
>
>
> See KAFKA-2376 for parent task and 
> [KIP-196|https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework]
>  for the details on the metrics. This subtask is to create the "Sink Task 
> Metrics".



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[DISCUSS] URIs on Producer and Consumer

2017-10-03 Thread Clebert Suconic
I believe I need write access to the WIKI Page:


https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals

As I don't see the KIP Template..


If anyone could please include me the to the group please?




On Tue, Oct 3, 2017 at 9:23 AM, Christopher Shannon
 wrote:
> I think this would be useful as a secondary way to configure.  If others
> agree then you can write up a KIP and it can be discussed in more detail.
>
> On Tue, Oct 3, 2017 at 8:56 AM, Clebert Suconic 
> wrote:
>
>> Maybe I didn't make the message clear enough...
>>
>> Would using an URI to the constructor (in addition to the properties)
>> help the API, or anyone see a reason to not do it?
>>
>> KafkaConsumer consumer = new
>> KafkaConsumer<>("tcp://localhost:?receive.buffer.bytes=-2", new
>> ByteArrayDeserializer(), new ByteArrayDeserializer());
>>
>> I could send a Pull Request for that. The framework I would write
>> would validate if the parameters are valid or not.
>>
>>
>> Thanks in advance
>>
>>
>> On Mon, Oct 2, 2017 at 9:14 AM, Clebert Suconic
>>  wrote:
>> > At ActiveMQ and ActiveMQ Artemis, ConnectionFactories have an
>> > interesting feature where you can pass parameters through an URI.
>> >
>> > I was looking at Producer and Consumer APIs, and these two classes are
>> > using a method that I considered old for Artemis resembling HornetQ:
>> >
>> > Instead of passing a Properties (aka HashMaps), users would be able to
>> > create a Consumer or Producer by simply doing:
>> >
>> > new Consumer("tcp::/host:port?properties=values;properties=
>> values...etc");
>> >
>> > Example:
>> >
>> >
>> > Instead of the following:
>> >
>> > Map config = new HashMap<>();
>> > config.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:");
>> > config.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, -2);
>> > new KafkaConsumer<>(config, new ByteArrayDeserializer(), new
>> > ByteArrayDeserializer());
>> >
>> >
>> >
>> > Someone could do
>> >
>> > new KafkaConsumer<>("tcp://localhost:?receive.buffer.bytes=-2",
>> > new ByteArrayDeserializer(), new ByteArrayDeserializer());
>> >
>> >
>> >
>> > I don't know if that little API improvement would be welcomed? I would be
>> > able to send a Pull Request but I don't want to do it if that wouldn't
>> > be welcomed in the first place:
>> >
>> >
>> > Just an idea...  let me know if that is welcomed or not.
>> >
>> > If so I can forward the discussion into how I would implement it.
>>
>>
>>
>> --
>> Clebert Suconic
>>



-- 
Clebert Suconic


Jenkins build is back to normal : kafka-trunk-jdk8 #2096

2017-10-03 Thread Apache Jenkins Server
See 




[GitHub] kafka-site issue #77: MINOR: Update verbiage on landing page https://kafka.a...

2017-10-03 Thread joel-hamill
Github user joel-hamill commented on the issue:

https://github.com/apache/kafka-site/pull/77
  
ping @guozhangwang for merge


---


[GitHub] kafka pull request #1937: KAFKA-4108: Improve DumpLogSegments offsets-decode...

2017-10-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---


[GitHub] kafka pull request #4008: MINOR: Add / to connect docs, because HTML ...

2017-10-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---


[GitHub] kafka pull request #4010: KAFKA-5909; Removed the source jars from classpath

2017-10-03 Thread Kamal15
GitHub user Kamal15 opened a pull request:

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

KAFKA-5909; Removed the source jars from classpath

Executing CLI tools don't require source jars so removed it from the 
classpath.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/Kamal15/kafka patch-1

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/4010.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4010


commit e52eceb29420124dc6f3f31a6d026cb05ff1bd82
Author: Kamal C 
Date:   2017-10-03T16:56:37Z

KAFKA-5909; Removed the source jars from classpath

Executing CLI tools don't require source jars so removed it from the 
classpath.




---


[GitHub] kafka pull request #3871: KAFKA-5909; Removed the source jars from classpath...

2017-10-03 Thread Kamal15
Github user Kamal15 closed the pull request at:

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


---


Build failed in Jenkins: kafka-trunk-jdk8 #2095

2017-10-03 Thread Apache Jenkins Server
See 


Changes:

[damian.guy] MINOR: fix JavaDocs warnings

--
[...truncated 2.95 MB...]
org.apache.kafka.common.security.JaasContextTest > testControlFlag PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
noAuthorizationIdSpecified STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
noAuthorizationIdSpecified PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdEqualsAuthenticationId STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdEqualsAuthenticationId PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdNotEqualsAuthenticationId STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdNotEqualsAuthenticationId PASSED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testProducerWithInvalidCredentials STARTED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testProducerWithInvalidCredentials PASSED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testTransactionalProducerWithInvalidCredentials STARTED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testTransactionalProducerWithInvalidCredentials PASSED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testConsumerWithInvalidCredentials STARTED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testConsumerWithInvalidCredentials PASSED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testAdminClientWithInvalidCredentials STARTED

org.apache.kafka.common.security.authenticator.ClientAuthenticationFailureTest 
> testAdminClientWithInvalidCredentials PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingUsernameSaslPlain STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingUsernameSaslPlain PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testValidSaslScramMechanisms STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testValidSaslScramMechanisms PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramSslServerWithoutSaslAuthenticateHeaderFailure STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramSslServerWithoutSaslAuthenticateHeaderFailure PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramPlaintextServerWithoutSaslAuthenticateHeaderFailure STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramPlaintextServerWithoutSaslAuthenticateHeaderFailure PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramPlaintextServerWithoutSaslAuthenticateHeader STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslScramPlaintextServerWithoutSaslAuthenticateHeader PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMechanismPluggability STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMechanismPluggability PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testScramUsernameWithSpecialCharacters STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testScramUsernameWithSpecialCharacters PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testApiVersionsRequestWithUnsupportedVersion STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testApiVersionsRequestWithUnsupportedVersion PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingPasswordSaslPlain STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingPasswordSaslPlain PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testInvalidLoginModule STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testInvalidLoginModule PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslPlainPlaintextClientWithoutSaslAuthenticateHeader STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslPlainPlaintextClientWithoutSaslAuthenticateHeader PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslPlainSslClientWithoutSaslAuthenticateHeader STARTED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
oldSaslPlainSslClientWithoutSaslAuthenticateHeader PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 

[GitHub] kafka pull request #3957: KAFKA-6005: Reject JoinGroup request from first me...

2017-10-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---


[jira] [Resolved] (KAFKA-6005) Reject JoinGroup request from first member with empty protocol type/protocol list

2017-10-03 Thread Jason Gustafson (JIRA)

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

Jason Gustafson resolved KAFKA-6005.

Resolution: Fixed

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

> Reject JoinGroup request from first member with empty protocol type/protocol 
> list
> -
>
> Key: KAFKA-6005
> URL: https://issues.apache.org/jira/browse/KAFKA-6005
> Project: Kafka
>  Issue Type: Bug
>Reporter: Manikumar
>Assignee: Manikumar
>Priority: Minor
> Fix For: 1.0.0
>
>
> Currently, if the first group member joins with empty 
> partition.assignment.strategy, then the group won't allow any other members 
> with valid protocols.  This JIRA is to add validation to reject JoinGroup 
> requests from the first member with empty protocol type or empty protocol list
> Also, add consumer-side validations to check at least one partition assigner 
> class name is configured while using subscribe APIs.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[GitHub] kafka-site issue #77: MINOR: Update verbiage on landing page https://kafka.a...

2017-10-03 Thread joel-hamill
Github user joel-hamill commented on the issue:

https://github.com/apache/kafka-site/pull/77
  
@miguno done 
https://github.com/apache/kafka-site/pull/77#issuecomment-333752239


---


[GitHub] kafka pull request #4005: MINOR: fix JavaDocs warnings

2017-10-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---


[jira] [Created] (KAFKA-6006) Default replication factor should be set in server.properties

2017-10-03 Thread Martin M (JIRA)
Martin M created KAFKA-6006:
---

 Summary: Default replication factor should be set in 
server.properties
 Key: KAFKA-6006
 URL: https://issues.apache.org/jira/browse/KAFKA-6006
 Project: Kafka
  Issue Type: Improvement
  Components: core
Affects Versions: 0.11.0.0
Reporter: Martin M


The replication factor should be set globally in server.properties such that it 
can be applied by default to all topics/partitions when topic is not 
auto-created. Having the configuration globally will allow consumers/producers 
have the same setting. 
One such situation is when consumers are distributed on different servers.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[GitHub] kafka pull request #4008: MINOR: Add / to connect docs, because HTML ...

2017-10-03 Thread tombentley
GitHub user tombentley opened a pull request:

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

MINOR: Add / to connect docs, because HTML render doesn't respect 
blank lines

This just adds / to paragraphs in Kafak Connect docs.

@ewencp are you a good person to review this?

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/tombentley/kafka MINOR-connect-docs-paragraph

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/4008.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4008


commit f1cd37902a38636354c440338c2aba84a803576c
Author: Tom Bentley 
Date:   2017-10-03T13:34:10Z

MINOR: Add / to connect docs, because HTML render doesn't respect 
blank lines




---


Re: [DISCUSS] URIs on Producer and Consumer

2017-10-03 Thread Christopher Shannon
I think this would be useful as a secondary way to configure.  If others
agree then you can write up a KIP and it can be discussed in more detail.

On Tue, Oct 3, 2017 at 8:56 AM, Clebert Suconic 
wrote:

> Maybe I didn't make the message clear enough...
>
> Would using an URI to the constructor (in addition to the properties)
> help the API, or anyone see a reason to not do it?
>
> KafkaConsumer consumer = new
> KafkaConsumer<>("tcp://localhost:?receive.buffer.bytes=-2", new
> ByteArrayDeserializer(), new ByteArrayDeserializer());
>
> I could send a Pull Request for that. The framework I would write
> would validate if the parameters are valid or not.
>
>
> Thanks in advance
>
>
> On Mon, Oct 2, 2017 at 9:14 AM, Clebert Suconic
>  wrote:
> > At ActiveMQ and ActiveMQ Artemis, ConnectionFactories have an
> > interesting feature where you can pass parameters through an URI.
> >
> > I was looking at Producer and Consumer APIs, and these two classes are
> > using a method that I considered old for Artemis resembling HornetQ:
> >
> > Instead of passing a Properties (aka HashMaps), users would be able to
> > create a Consumer or Producer by simply doing:
> >
> > new Consumer("tcp::/host:port?properties=values;properties=
> values...etc");
> >
> > Example:
> >
> >
> > Instead of the following:
> >
> > Map config = new HashMap<>();
> > config.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:");
> > config.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, -2);
> > new KafkaConsumer<>(config, new ByteArrayDeserializer(), new
> > ByteArrayDeserializer());
> >
> >
> >
> > Someone could do
> >
> > new KafkaConsumer<>("tcp://localhost:?receive.buffer.bytes=-2",
> > new ByteArrayDeserializer(), new ByteArrayDeserializer());
> >
> >
> >
> > I don't know if that little API improvement would be welcomed? I would be
> > able to send a Pull Request but I don't want to do it if that wouldn't
> > be welcomed in the first place:
> >
> >
> > Just an idea...  let me know if that is welcomed or not.
> >
> > If so I can forward the discussion into how I would implement it.
>
>
>
> --
> Clebert Suconic
>


[GitHub] kafka pull request #4007: MINOR: Java 9 version handling improvements

2017-10-03 Thread ijuma
GitHub user ijuma opened a pull request:

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

MINOR: Java 9 version handling improvements

- Upgrade Gradle to 4.2.1, which handles Azul Zulu 9's version
correctly.
- Add tests to our Java version handling code
- Refactor the code to make it possible to add tests
- Rename `isIBMJdk` method to use consistent naming
convention.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/ijuma/kafka 
java-9-version-handling-improvements

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/4007.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4007


commit 1ef63b35c7b1fb5b568f0a71924f3a3483c6eb1b
Author: Ismael Juma 
Date:   2017-10-03T11:16:01Z

Add tests for java version compatibility methods

commit 236bffc910bd46601715ed1086f5986229b74cb9
Author: Ismael Juma 
Date:   2017-10-03T11:16:33Z

Rename `Java.isIBMJdk` method to use consistent convention

commit c82ca6b7c522efa4be432ab3f2c67145aa0870d8
Author: Ismael Juma 
Date:   2017-10-03T11:17:01Z

Update to Gradle 4.2.1




---


[GitHub] kafka pull request #3797: KAFKA-5843; Mx4jLoader.maybeLoad should only be ex...

2017-10-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---


[jira] [Resolved] (KAFKA-4946) Evaluation of kafka_mx4jenable is inverted

2017-10-03 Thread Ismael Juma (JIRA)

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

Ismael Juma resolved KAFKA-4946.

Resolution: Duplicate

Marking as duplicate of KAFKA-5843 as that had a PR. Credited Ralph as well in 
the merged commit:

https://github.com/apache/kafka/commit/716330a5b2f629e6bb0316b5f036837660916a0e

> Evaluation of kafka_mx4jenable is inverted
> --
>
> Key: KAFKA-4946
> URL: https://issues.apache.org/jira/browse/KAFKA-4946
> Project: Kafka
>  Issue Type: Bug
>  Components: core, metrics
>Affects Versions: 0.10.2.0
>Reporter: Ralph Weires
>Priority: Minor
> Attachments: mx4j-enable.patch
>
>
> Mx4j stopped working for me after upgrade from 0.8.2 -> 0.10.2.0
> I had the "-Dkafka_mx4jenable=true" flag set in a custom start script, which 
> however now actually prevents MX4J from being loaded in 0.10.2.0
> The flag is only taken into account since the patch from KAFKA-2002 it seems 
> to me - we should however skip trying to load MX4J if the flag is set to 
> false, rather than if it's set to true (as it's currently done).
> Trivial patch attached, check should be negated.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (KAFKA-6005) Reject JoinGroup request from first member with empty protocol type/protocol list

2017-10-03 Thread Manikumar (JIRA)
Manikumar created KAFKA-6005:


 Summary: Reject JoinGroup request from first member with empty 
protocol type/protocol list
 Key: KAFKA-6005
 URL: https://issues.apache.org/jira/browse/KAFKA-6005
 Project: Kafka
  Issue Type: Bug
Reporter: Manikumar
Assignee: Manikumar
Priority: Minor
 Fix For: 1.0.0


Currently, if the first group member joins with empty 
partition.assignment.strategy, then the group won't allow any other members 
with valid protocols.  This JIRA is to add validation to reject JoinGroup 
requests from the first member with empty protocol type or empty protocol list

Also, add consumer-side validations to check at least one partition assigner 
class name is configured while using subscribe APIs.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[GitHub] kafka pull request #4006: MINOR: JavaDoc improvements for new state store AP...

2017-10-03 Thread mjsax
GitHub user mjsax opened a pull request:

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

MINOR: JavaDoc improvements for new state store API



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/mjsax/kafka 
minor-javadoc-improvments-for-stores

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/4006.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4006


commit fa1ea8498f6be2676ed15f34de1c328ff76b28e8
Author: Matthias J. Sax 
Date:   2017-10-03T07:49:28Z

MINOR: JavaDoc improvments for new state store API




---


Jenkins build is back to normal : kafka-trunk-jdk8 #2093

2017-10-03 Thread Apache Jenkins Server
See 




[GitHub] kafka-site issue #77: MINOR: Add streams child topics to left-hand nav

2017-10-03 Thread miguno
Github user miguno commented on the issue:

https://github.com/apache/kafka-site/pull/77
  
Could you please edit/update the PR title then?

I understand that this PR is now a change to the content pane of the main 
page (kafka.apache.org) only?


---