[jira] [Work started] (KAFKA-3443) Support regex topics in addSource() and stream()

2016-05-07 Thread Bill Bejeck (JIRA)

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

Work on KAFKA-3443 started by Bill Bejeck.
--
> Support regex topics in addSource() and stream()
> 
>
> Key: KAFKA-3443
> URL: https://issues.apache.org/jira/browse/KAFKA-3443
> Project: Kafka
>  Issue Type: Sub-task
>  Components: streams
>Reporter: Guozhang Wang
>Assignee: Bill Bejeck
>  Labels: api
> Fix For: 0.10.1.0
>
>
> Currently Kafka Streams only support specific topics in creating source 
> streams, while we can leverage consumer's regex subscription to allow regex 
> topics as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-05-07 Thread Harsha
Ismael,
   Do we need to add old assign and subscribe that accepts List. It
   will get implicitly cast to collection with the new methods.
The only problem comes from the methods that accepts varargs.

-Harsha

On Sat, May 7, 2016, at 05:53 PM, Mark Grover wrote:
> Thanks Ismael, I agree with you, it makes sense to leave things the way
> they are in Kafka 0.10.
> 
> On Fri, May 6, 2016 at 5:27 PM, Ismael Juma  wrote:
> 
> > Hi Mark,
> >
> > Thanks for the email. First of all, I'd like to mention that the `Unstable`
> > annotation has been removed from the new Java consumer in 0.10, so you can
> > expect compatibility from now on. We definitely understand that
> > compatibility is important for widespread adoption.
> >
> > The current PR for KAFKA-3633 adds deprecated and overloaded methods for
> > `seekToBeginning`, `seekToEnd`, `pause` and `resume` each taking a varargs
> > parameter for backwards compatibility. If these methods solved the binary
> > compatibility issue, I'd be supportive of adding them.
> >
> > However, as I pointed out in my original message (and Jason elaborated
> > subsequently), something would also have to be done about `assign` and
> > `subscribe` in order to maintain binary compatibility between 0.9 and 0.10.
> > And a good solution for these methods is elusive.
> >
> > If we add deprecated and overloaded methods that take a `List` parameter,
> > then every existing user of the new consumer will be exposed to a
> > deprecated warning (or error if they have a warning as error build policy)
> > because everyone uses `subscribe`. Avoiding the warning involves using
> > `Set` instead of `List`, which is a bit weird and unintuitive (although we
> > could document it).
> >
> > We could add the overloaded methods without deprecating them. In this case,
> > we would be stuck with two methods for the same thing forever (for both
> > `subscribe` and `assign`). This makes the API more confusing and overloads
> > mean that type inference from lambdas would be less effective (if at all
> > effective).
> >
> > Or we could leave things as they are. The `subscribe` and `assign` changes
> > are source compatible so no source changes are needed by the common user
> > who just compiles against a particular version of the Kafka clients
> > library. It's also important to note that kafka-clients 0.9 works fine with
> > 0.10 brokers. Supporting both 0.9 and 0.10 clients from the same JAR will
> > be a bit annoying, but the ugly shim code for that is straightforward to
> > write for advanced users that need this.
> >
> > I should make it clear that this is my position, other committers may feel
> > differently.
> >
> > Ismael
> >
> > On Sat, May 7, 2016 at 12:38 AM, Mark Grover  wrote:
> >
> > > I understand and empathize with both sides of the story here. I spend
> > some
> > > of my time on Spark and Kafka integration and I have cc'ed Cody who's
> > been
> > > working on new Kafka consumer API with Spark Streaming.
> > > Spark hasn't merged the new Kafka consumer API integration, the PR is up
> > > and we, as a community, are deliberating
> > > <
> > >
> > https://issues.apache.org/jira/browse/SPARK-12177?focusedCommentId=15274910&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15274910
> > > >
> > > if now is the right time to put this in, given the flux in the API, the
> > > lack of delegation tokens support, etc.
> > >
> > > The proposed Spark integration with Kafka's new API relies on
> > > KafkaConsumer::pause() and KafkaConsumer::seekToEnd() and those methods
> > > break compatibility between 0.9 and 0.10 RC4 (since KAFKA-3633
> > >  remains unresolved).
> > >
> > > What this means is that if Spark supports both 0.9 and 0.10, we are going
> > > to have some complexity to compile against 0.9 and 0.10. So, Spark
> > > community, Cody and myself are all leaning towards not putting Kafka 0.9
> > > support in, and only supporting starting Kafka 0.10 (or, may be even
> > later)
> > > depending on the compatibility situation. The point I am trying to make
> > is
> > > that the new consumer API doesn't add much value for us just yet and
> > > breaking compatibility doesn't help in encouraging us to add support for
> > it
> > > in Spark.
> > >
> > > As far as this particular topic (KAFKA-3633) goes, I don't have a strong
> > > vote, since Spark isn't likely to support 0.9's new kafka consumer API
> > > anyways. However, I'd state the obvious that compatibility is important
> > if
> > > you'd like to encourage us to adopt the new API.
> > >
> > > Mark
> > >
> > > On Mon, May 2, 2016 at 10:45 AM, Guozhang Wang 
> > wrote:
> > >
> > > > Just my two cents here:
> > > >
> > > > I agree with Ewen and Grant on the indication of the "unstable"
> > > annotations
> > > > of being possible for backward incompatible. That means, users can
> > make a
> > > > call themselves of whether to start trying out the new APIs / libraries

Re: [VOTE] KIP-45: Standardize KafkaConsumer API to use Collection

2016-05-07 Thread Mark Grover
Thanks Ismael, I agree with you, it makes sense to leave things the way
they are in Kafka 0.10.

On Fri, May 6, 2016 at 5:27 PM, Ismael Juma  wrote:

> Hi Mark,
>
> Thanks for the email. First of all, I'd like to mention that the `Unstable`
> annotation has been removed from the new Java consumer in 0.10, so you can
> expect compatibility from now on. We definitely understand that
> compatibility is important for widespread adoption.
>
> The current PR for KAFKA-3633 adds deprecated and overloaded methods for
> `seekToBeginning`, `seekToEnd`, `pause` and `resume` each taking a varargs
> parameter for backwards compatibility. If these methods solved the binary
> compatibility issue, I'd be supportive of adding them.
>
> However, as I pointed out in my original message (and Jason elaborated
> subsequently), something would also have to be done about `assign` and
> `subscribe` in order to maintain binary compatibility between 0.9 and 0.10.
> And a good solution for these methods is elusive.
>
> If we add deprecated and overloaded methods that take a `List` parameter,
> then every existing user of the new consumer will be exposed to a
> deprecated warning (or error if they have a warning as error build policy)
> because everyone uses `subscribe`. Avoiding the warning involves using
> `Set` instead of `List`, which is a bit weird and unintuitive (although we
> could document it).
>
> We could add the overloaded methods without deprecating them. In this case,
> we would be stuck with two methods for the same thing forever (for both
> `subscribe` and `assign`). This makes the API more confusing and overloads
> mean that type inference from lambdas would be less effective (if at all
> effective).
>
> Or we could leave things as they are. The `subscribe` and `assign` changes
> are source compatible so no source changes are needed by the common user
> who just compiles against a particular version of the Kafka clients
> library. It's also important to note that kafka-clients 0.9 works fine with
> 0.10 brokers. Supporting both 0.9 and 0.10 clients from the same JAR will
> be a bit annoying, but the ugly shim code for that is straightforward to
> write for advanced users that need this.
>
> I should make it clear that this is my position, other committers may feel
> differently.
>
> Ismael
>
> On Sat, May 7, 2016 at 12:38 AM, Mark Grover  wrote:
>
> > I understand and empathize with both sides of the story here. I spend
> some
> > of my time on Spark and Kafka integration and I have cc'ed Cody who's
> been
> > working on new Kafka consumer API with Spark Streaming.
> > Spark hasn't merged the new Kafka consumer API integration, the PR is up
> > and we, as a community, are deliberating
> > <
> >
> https://issues.apache.org/jira/browse/SPARK-12177?focusedCommentId=15274910&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15274910
> > >
> > if now is the right time to put this in, given the flux in the API, the
> > lack of delegation tokens support, etc.
> >
> > The proposed Spark integration with Kafka's new API relies on
> > KafkaConsumer::pause() and KafkaConsumer::seekToEnd() and those methods
> > break compatibility between 0.9 and 0.10 RC4 (since KAFKA-3633
> >  remains unresolved).
> >
> > What this means is that if Spark supports both 0.9 and 0.10, we are going
> > to have some complexity to compile against 0.9 and 0.10. So, Spark
> > community, Cody and myself are all leaning towards not putting Kafka 0.9
> > support in, and only supporting starting Kafka 0.10 (or, may be even
> later)
> > depending on the compatibility situation. The point I am trying to make
> is
> > that the new consumer API doesn't add much value for us just yet and
> > breaking compatibility doesn't help in encouraging us to add support for
> it
> > in Spark.
> >
> > As far as this particular topic (KAFKA-3633) goes, I don't have a strong
> > vote, since Spark isn't likely to support 0.9's new kafka consumer API
> > anyways. However, I'd state the obvious that compatibility is important
> if
> > you'd like to encourage us to adopt the new API.
> >
> > Mark
> >
> > On Mon, May 2, 2016 at 10:45 AM, Guozhang Wang 
> wrote:
> >
> > > Just my two cents here:
> > >
> > > I agree with Ewen and Grant on the indication of the "unstable"
> > annotations
> > > of being possible for backward incompatible. That means, users can
> make a
> > > call themselves of whether to start trying out the new APIs / libraries
> > > with the risk or changing code when it changes in a later release or
> just
> > > wait for it to be stable. Personally I don't think it would result in
> no
> > > one ever going to try out "unstable" new APIs, even in their production
> > > usages (for example at LI we used the LiKafkaClient to wrap the apache
> > > kafka clients and one motivation is to abstract any API backward
> > > incompatibility); for cases like the Storm integration scenarios, yes
> it
>

[jira] [Commented] (KAFKA-3565) Producer's throughput lower with compressed data after KIP-31/32

2016-05-07 Thread Jiangjie Qin (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15275431#comment-15275431
 ] 

Jiangjie Qin commented on KAFKA-3565:
-

[~junrao] I think I figured out the reason why 0.9 consumer has better 
performance than trunk. It is because the recompression on the broker side in 
0.9 is more efficient than the streaming compression on the producer side.

For the setting using snappy compression, message size 100B, valuebound 500, 
both trunk and 0.9 reports the same batch size on the producer side.
{noformat}
Producer_Test
Select_Rate:784.10  689.02
Batch_Size_Avg: 10625.7910204.10
Request_Size_Avg:   85144.3781771.16
Request_Latency_Avg:4.416.77
Request_Rate:   114.30  99.33
Records_Per_Request_Avg:801.00  801.00
Record_Queue_Time:  4.093.07
Compression_Rate_Avg:   0.790.81
92395.823709 records/sec (8.81 MB/sec), 6.52 ms avg latency, 436.00 ms max 
latency, 6 ms 50th, 9 ms 95th, 9 ms 99th, 17 ms 99.9th
79507.056251 records/sec (7.58 MB/sec), 8.43 ms avg latency, 220.00 ms max 
latency, 8 ms 50th, 11 ms 95th, 11 ms 99th, 18 ms 99.9th.
Consumer_Test
start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, 
nMsg.sec end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec
16:14:48:796, 16:15:07:793, 953.6743, 50.2013, 1000, 526398.9051 
16:17:17:637, 16:17:33:701, 953.6743, 59.3672, 1000, 622509.9602
--
max.in.flight.requests.per.connection=1, valueBound=500, linger.ms=10, 
messageSize=100, compression.type=snappy
{noformat}

But after I dump the log on the broker side, after recompression the shallow 
messages on 0.9 broker become ~8K but while the trunk broker still has ~10K 
shallow message. 

I ran the tests with lz4 as well. The results is updated in test run 16 and 17. 
I did not see the issue of snappy. Although after broker side recompression the 
sizes of the shallow messages change a little but they are roughly the same as 
the producer side batch size.

I did not see this problem when value bound is 5000. So it seems the batch 
compression on the broker side the better compression ratio of snappy for 
certain data pattern is the reason of the performance gap we saw in the test. I 
listed below the batch sizes before and after recompression for snappy with 
different settings:

{noformat}
Producer Batch Size Avg:   10204.49
Broker batch size:  ~8.0K
--
max.in.flight.requests.per.connection=1, valueBound=500, linger.ms=10, 
messageSize=100, compression.type=gzip

Producer Batch Size Avg:   9107.23
Broker batch size: ~6.6K
--
max.in.flight.requests.per.connection=1, valueBound=500, linger.ms=10, 
messageSize=1000, compression.type=snappy

Producer Batch Size Avg:   11457.56
Broker batch size: ~10.5K
--
max.in.flight.requests.per.connection=1, valueBound=5000, linger.ms=10, 
messageSize=100, compression.type=snappy

Producer Batch Size Avg:   10429.08
Broker batch size: ~9.4K
--
max.in.flight.requests.per.connection=1, valueBound=5000, linger.ms=10, 
messageSize=1000, compression.type=snappy
{noformat}



> Producer's throughput lower with compressed data after KIP-31/32
> 
>
> Key: KAFKA-3565
> URL: https://issues.apache.org/jira/browse/KAFKA-3565
> Project: Kafka
>  Issue Type: Bug
>Reporter: Ismael Juma
>Priority: Critical
> Fix For: 0.10.0.0
>
>
> Relative offsets were introduced by KIP-31 so that the broker does not have 
> to recompress data (this was previously required after offsets were 
> assigned). The implicit assumption is that reducing CPU usage required by 
> recompression would mean that producer throughput for compressed data would 
> increase.
> However, this doesn't seem to be the case:
> {code}
> Commit: eee95228fabe1643baa016a2d49fb0a9fe2c66bd (one before KIP-31/32)
> test_id:
> 2016-04-15--012.kafkatest.tests.benchmark_test.Benchmark.test_producer_throughput.topic=topic-replication-factor-three.security_protocol=PLAINTEXT.acks=1.message_size=100.compression_type=snappy
> status: PASS
> run time:   59.030 seconds
> {"records_per_sec": 519418.343653, "mb_per_sec": 49.54}
> {code}
> Full results: https://gist.github.com/ijuma/0afada4ff51ad6a5ac2125714d748292
> {code}
> Commit: fa594c811e4e329b6e7b897bce910c6772c46c0f (KIP-31/32)
> test_id:
> 2016-04-15--013.kafkatest.tests.benchmark_test.Benchmark.test_producer_throughput.topic=topic-replication-factor-three.security_protocol=PLAINTEXT.acks=1.message_size=100.compression_type=snappy
> status: PASS
> run time:   1 minute 0.243 seconds
> {"records_per_sec": 427308.818848, "mb_per_sec": 40.75}
> {code}
> Full res

[GitHub] kafka pull request: MINOR: Cleanup Admin Package

2016-05-07 Thread Ishiihara
GitHub user Ishiihara opened a pull request:

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

MINOR: Cleanup Admin Package



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

$ git pull https://github.com/Ishiihara/kafka clean-admin-package

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

https://github.com/apache/kafka/pull/1342.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 #1342


commit 0b11c453717dbd095025f289931ac3352f03bce6
Author: Liquan Pei 
Date:   2016-05-04T17:00:13Z

WIP




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


Re: KAFKA-3112

2016-05-07 Thread Raj Tanneru
Thanks Ismael and Tao. Appreciate it.

Sent from my iPhone

> On May 7, 2016, at 1:14 AM, Ismael Juma  wrote:
>
> Hi Raj and Tao,
>
> I just merged the KAFKA-3112 PR, so this issue will be fixed in 0.10.0.0.
>
> Thanks,
> Ismael
>
>> On Fri, May 6, 2016 at 7:47 PM, tao xiao  wrote:
>>
>> KAFKA-2657 is unresolved so you can safely assume it hasn't been fixed yet.
>>
>>> On Fri, 6 May 2016 at 07:38 Raj Tanneru  wrote:
>>>
>>> Yeah it is a duplicate of KAFKA-2657. The question is how to check / know
>>> if it is merged to 0.9.0.1 release. What are the options that I have if I
>>> need this fix. How can I get patch for this on 0.8.2.1?
>>>
>>> Sent from my iPhone
>>>
 On May 6, 2016, at 12:06 AM, tao xiao  wrote:

 It said this is a duplication. This is the
 https://issues.apache.org/jira/browse/KAFKA-2657 that KAKFA-3112
>>> duplicates
 to.

> On Thu, 5 May 2016 at 22:13 Raj Tanneru 
>> wrote:
>
>
> Hi All,
> Does anyone know if KAFKA-3112 is merged to 0.9.0.1? Is there a place
>> to
> check which version has this fix? Jira doesn’t show fix versions.
>
> https://issues.apache.org/jira/browse/KAFKA-3112
>
>
> Thanks,
> Raj Tanneru
> Information contained in this e-mail message is confidential. This
>>> e-mail
> message is intended only for the personal use of the recipient(s)
>> named
> above. If you are not an intended recipient, do not read, distribute
>> or
> reproduce this transmission (including any attachments). If you have
> received this email in error, please immediately notify the sender by
>>> email
> reply and delete the original message.
>>> Information contained in this e-mail message is confidential. This e-mail
>>> message is intended only for the personal use of the recipient(s) named
>>> above. If you are not an intended recipient, do not read, distribute or
>>> reproduce this transmission (including any attachments). If you have
>>> received this email in error, please immediately notify the sender by
>> email
>>> reply and delete the original message.
>>
Information contained in this e-mail message is confidential. This e-mail 
message is intended only for the personal use of the recipient(s) named above. 
If you are not an intended recipient, do not read, distribute or reproduce this 
transmission (including any attachments). If you have received this email in 
error, please immediately notify the sender by email reply and delete the 
original message.


[jira] [Work started] (KAFKA-3675) Add lz4 to parametrized `test_upgrade` system test

2016-05-07 Thread Ismael Juma (JIRA)

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

Work on KAFKA-3675 started by Ismael Juma.
--
> Add lz4 to parametrized `test_upgrade` system test
> --
>
> Key: KAFKA-3675
> URL: https://issues.apache.org/jira/browse/KAFKA-3675
> Project: Kafka
>  Issue Type: Test
>Reporter: Ismael Juma
>Assignee: Ismael Juma
> Fix For: 0.10.0.0
>
>
> KIP-57 fixes the LZ4 framing in message format 0.10.0 and we should verify 
> that this works correctly during upgrades.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (KAFKA-3675) Add lz4 to parametrized `test_upgrade` system test

2016-05-07 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-3675:
--

 Summary: Add lz4 to parametrized `test_upgrade` system test
 Key: KAFKA-3675
 URL: https://issues.apache.org/jira/browse/KAFKA-3675
 Project: Kafka
  Issue Type: Test
Reporter: Ismael Juma
Assignee: Ismael Juma
 Fix For: 0.10.0.0


KIP-57 fixes the LZ4 framing in message format 0.10.0 and we should verify that 
this works correctly during upgrades.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Build failed in Jenkins: kafka-0.10.0-jdk7 #63

2016-05-07 Thread Apache Jenkins Server
See 

Changes:

[ismael] KAFKA-3160; Fix LZ4 Framing

--
[...truncated 1675 lines...]

kafka.server.EdgeCaseRequestTest > testMalformedHeaderRequest PASSED

kafka.server.EdgeCaseRequestTest > testProduceRequestWithNullClientId PASSED

kafka.server.EdgeCaseRequestTest > testInvalidApiKeyRequest PASSED

kafka.server.EdgeCaseRequestTest > testHeaderOnlyRequest PASSED

kafka.server.MetadataRequestTest > testReplicaDownResponse PASSED

kafka.server.MetadataRequestTest > testRack PASSED

kafka.server.MetadataRequestTest > testIsInternal PASSED

kafka.server.MetadataRequestTest > testControllerId PASSED

kafka.server.MetadataRequestTest > testAllTopicsRequest PASSED

kafka.server.MetadataRequestTest > testNoTopicsRequest PASSED

kafka.server.HighwatermarkPersistenceTest > 
testHighWatermarkPersistenceMultiplePartitions PASSED

kafka.server.HighwatermarkPersistenceTest > 
testHighWatermarkPersistenceSinglePartition PASSED

kafka.server.LogRecoveryTest > testHWCheckpointNoFailuresMultipleLogSegments 
PASSED

kafka.server.LogRecoveryTest > testHWCheckpointWithFailuresMultipleLogSegments 
PASSED

kafka.server.LogRecoveryTest > testHWCheckpointNoFailuresSingleLogSegment PASSED

kafka.server.LogRecoveryTest > testHWCheckpointWithFailuresSingleLogSegment 
PASSED

kafka.integration.SaslPlaintextTopicMetadataTest > 
testIsrAfterBrokerShutDownAndJoinsBack PASSED

kafka.integration.SaslPlaintextTopicMetadataTest > 
testAutoCreateTopicWithCollision PASSED

kafka.integration.SaslPlaintextTopicMetadataTest > 
testAliveBrokerListWithNoTopics PASSED

kafka.integration.SaslPlaintextTopicMetadataTest > testAutoCreateTopic PASSED

kafka.integration.SaslPlaintextTopicMetadataTest > testGetAllTopicMetadata 
PASSED

kafka.integration.SaslPlaintextTopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterNewBrokerStartup PASSED

kafka.integration.SaslPlaintextTopicMetadataTest > testBasicTopicMetadata PASSED

kafka.integration.SaslPlaintextTopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterABrokerShutdown PASSED

kafka.integration.PrimitiveApiTest > testMultiProduce PASSED

kafka.integration.PrimitiveApiTest > testDefaultEncoderProducerAndFetch PASSED

kafka.integration.PrimitiveApiTest > testFetchRequestCanProperlySerialize PASSED

kafka.integration.PrimitiveApiTest > testPipelinedProduceRequests PASSED

kafka.integration.PrimitiveApiTest > testProduceAndMultiFetch PASSED

kafka.integration.PrimitiveApiTest > 
testDefaultEncoderProducerAndFetchWithCompression PASSED

kafka.integration.PrimitiveApiTest > testConsumerEmptyTopic PASSED

kafka.integration.PrimitiveApiTest > testEmptyFetchRequest PASSED

kafka.integration.PlaintextTopicMetadataTest > 
testIsrAfterBrokerShutDownAndJoinsBack PASSED

kafka.integration.PlaintextTopicMetadataTest > testAutoCreateTopicWithCollision 
PASSED

kafka.integration.PlaintextTopicMetadataTest > testAliveBrokerListWithNoTopics 
PASSED

kafka.integration.PlaintextTopicMetadataTest > testAutoCreateTopic PASSED

kafka.integration.PlaintextTopicMetadataTest > testGetAllTopicMetadata PASSED

kafka.integration.PlaintextTopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterNewBrokerStartup PASSED

kafka.integration.PlaintextTopicMetadataTest > testBasicTopicMetadata PASSED

kafka.integration.PlaintextTopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterABrokerShutdown PASSED

kafka.integration.SslTopicMetadataTest > testIsrAfterBrokerShutDownAndJoinsBack 
PASSED

kafka.integration.SslTopicMetadataTest > testAutoCreateTopicWithCollision PASSED

kafka.integration.SslTopicMetadataTest > testAliveBrokerListWithNoTopics PASSED

kafka.integration.SslTopicMetadataTest > testAutoCreateTopic PASSED

kafka.integration.SslTopicMetadataTest > testGetAllTopicMetadata PASSED

kafka.integration.SslTopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterNewBrokerStartup PASSED

kafka.integration.SslTopicMetadataTest > testBasicTopicMetadata PASSED

kafka.integration.SslTopicMetadataTest > 
testAliveBrokersListWithNoTopicsAfterABrokerShutdown PASSED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooLow PASSED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooLow 
PASSED

kafka.integration.AutoOffsetResetTest > testResetToEarliestWhenOffsetTooHigh 
PASSED

kafka.integration.AutoOffsetResetTest > testResetToLatestWhenOffsetTooHigh 
PASSED

kafka.integration.MinIsrConfigTest > testDefaultKafkaConfig PASSED

kafka.integration.UncleanLeaderElectionTest > testUncleanLeaderElectionEnabled 
PASSED

kafka.integration.UncleanLeaderElectionTest > 
testCleanLeaderElectionDisabledByTopicOverride PASSED

kafka.integration.UncleanLeaderElectionTest > testUncleanLeaderElectionDisabled 
PASSED

kafka.integration.UncleanLeaderElectionTest > 
testUncleanLeaderElectionInvalidTopicOverride PASSED
ERROR: Could not install GRADLE_2_4_RC_2_HOME
java.lang.NullPointerException
at 
hudso

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

2016-05-07 Thread Apache Jenkins Server
See 

Changes:

[ismael] KAFKA-3160; Fix LZ4 Framing

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on ubuntu-5 (docker Ubuntu ubuntu5 ubuntu yahoo-not-h2) in 
workspace 
 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git -c core.askpass=true fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/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 8fe2552239863f3a01d01708d55edf3c7082ff92 
(refs/remotes/origin/trunk)
 > git config core.sparsecheckout # timeout=10
 > git checkout -f 8fe2552239863f3a01d01708d55edf3c7082ff92
 > git rev-list c4bbf342432291220242ad4177fd72a959ddcb94 # timeout=10
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK1_8_0_66_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_66
[kafka-trunk-jdk8] $ /bin/bash -xe /tmp/hudson5446475224512205317.sh
+ 
/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2/bin/gradle
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
http://gradle.org/docs/2.4-rc-2/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:downloadWrapper

BUILD SUCCESSFUL

Total time: 29.841 secs
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK1_8_0_66_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk1.8.0_66
[kafka-trunk-jdk8] $ /bin/bash -xe /tmp/hudson6684806740560384241.sh
+ export GRADLE_OPTS=-Xmx1024m
+ GRADLE_OPTS=-Xmx1024m
+ ./gradlew -Dorg.gradle.project.maxParallelForks=1 clean jarAll testAll
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
https://docs.gradle.org/2.13/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
Build file ': 
line 230
useAnt has been deprecated and is scheduled to be removed in Gradle 3.0. The 
Ant-Based Scala compiler is deprecated, please see 
https://docs.gradle.org/current/userguide/scala_plugin.html.
:clean UP-TO-DATE
:clients:clean
:connect:clean UP-TO-DATE
:core:clean
:examples:clean
:log4j-appender:clean
:streams:clean
:tools:clean
:connect:api:clean
:connect:file:clean
:connect:json:clean
:connect:runtime:clean
:streams:examples:clean
:jar_core_2_10
Building project 'core' with Scala version 2.10.6
:kafka-trunk-jdk8:clients:compileJavawarning: [options] bootstrap class path 
not set in conjunction with -source 1.7
Note: Some input files use unchecked or unsafe operations.
Note: Recompile with -Xlint:unchecked for details.
1 warning

:kafka-trunk-jdk8:clients:processResources UP-TO-DATE
:kafka-trunk-jdk8:clients:classes
:kafka-trunk-jdk8:clients:determineCommitId UP-TO-DATE
:kafka-trunk-jdk8:clients:createVersionFile
:kafka-trunk-jdk8:clients:jar
:kafka-trunk-jdk8:core:compileJava UP-TO-DATE
:kafka-trunk-jdk8:core:compileScalaJava HotSpot(TM) 64-Bit Server VM warning: 
ignoring option MaxPermSize=512m; support was removed in 8.0

:79:
 value DEFAULT_TIMESTAMP in object OffsetCommitRequest is deprecated: see 
corresponding Javadoc for more information.

org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP
 ^
:36:
 value DEFAULT_TIMESTAMP in object OffsetCommitRequest is deprecated: see 
corresponding Javadoc for more information.
 commitTimestamp: Long = 
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP,

  ^
:37:
 value DEFAULT_TIMESTAMP in object OffsetCommitRequest is deprecated: see 
corresponding Javadoc for more information.
 expireTimestamp: Long = 
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) {
  

[jira] [Updated] (KAFKA-3160) Kafka LZ4 framing code miscalculates header checksum

2016-05-07 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-3160:
---
Resolution: Fixed
Status: Resolved  (was: Patch Available)

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

> Kafka LZ4 framing code miscalculates header checksum
> 
>
> Key: KAFKA-3160
> URL: https://issues.apache.org/jira/browse/KAFKA-3160
> Project: Kafka
>  Issue Type: Bug
>  Components: compression
>Affects Versions: 0.8.2.0, 0.8.2.1, 0.9.0.0, 0.8.2.2, 0.9.0.1
>Reporter: Dana Powers
>Assignee: Dana Powers
>Priority: Critical
>  Labels: compatibility, compression, lz4
> Fix For: 0.10.0.0
>
>
> KAFKA-1493 partially implements the LZ4 framing specification, but it 
> incorrectly calculates the header checksum. This causes 
> KafkaLZ4BlockInputStream to raise an error 
> [IOException(DESCRIPTOR_HASH_MISMATCH)] if a client sends *correctly* framed 
> LZ4 data. It also causes KafkaLZ4BlockOutputStream to generate incorrectly 
> framed LZ4 data, which means clients decoding LZ4 messages from kafka will 
> always receive incorrectly framed data.
> Specifically, the current implementation includes the 4-byte MagicNumber in 
> the checksum, which is incorrect.
> http://cyan4973.github.io/lz4/lz4_Frame_format.html
> Third-party clients that attempt to use off-the-shelf lz4 framing find that 
> brokers reject messages as having a corrupt checksum. So currently non-java 
> clients must 'fixup' lz4 packets to deal with the broken checksum.
> Magnus first identified this issue in librdkafka; kafka-python has the same 
> problem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-3160) Kafka LZ4 framing code miscalculates header checksum

2016-05-07 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-3160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15275336#comment-15275336
 ] 

ASF GitHub Bot commented on KAFKA-3160:
---

Github user asfgit closed the pull request at:

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


> Kafka LZ4 framing code miscalculates header checksum
> 
>
> Key: KAFKA-3160
> URL: https://issues.apache.org/jira/browse/KAFKA-3160
> Project: Kafka
>  Issue Type: Bug
>  Components: compression
>Affects Versions: 0.8.2.0, 0.8.2.1, 0.9.0.0, 0.8.2.2, 0.9.0.1
>Reporter: Dana Powers
>Assignee: Dana Powers
>Priority: Critical
>  Labels: compatibility, compression, lz4
> Fix For: 0.10.0.0
>
>
> KAFKA-1493 partially implements the LZ4 framing specification, but it 
> incorrectly calculates the header checksum. This causes 
> KafkaLZ4BlockInputStream to raise an error 
> [IOException(DESCRIPTOR_HASH_MISMATCH)] if a client sends *correctly* framed 
> LZ4 data. It also causes KafkaLZ4BlockOutputStream to generate incorrectly 
> framed LZ4 data, which means clients decoding LZ4 messages from kafka will 
> always receive incorrectly framed data.
> Specifically, the current implementation includes the 4-byte MagicNumber in 
> the checksum, which is incorrect.
> http://cyan4973.github.io/lz4/lz4_Frame_format.html
> Third-party clients that attempt to use off-the-shelf lz4 framing find that 
> brokers reject messages as having a corrupt checksum. So currently non-java 
> clients must 'fixup' lz4 packets to deal with the broken checksum.
> Magnus first identified this issue in librdkafka; kafka-python has the same 
> problem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] kafka pull request: KAFKA-3160: Fix LZ4 Framing

2016-05-07 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (KAFKA-3585) Shutdown slow when there is only one broker which is controller

2016-05-07 Thread Taiyuan Zhang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-3585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15275312#comment-15275312
 ] 

Taiyuan Zhang commented on KAFKA-3585:
--

By the way, I just noticed that I was using 0.10 for my local testing, then I 
switched to 0.9 but still couldn't reproduce it. 

1. When I closed server 0, server 1 said "now leader is server 1"
2. Then I closed server 1, server 2 said "now leader is server 2"
3. Then I closed server 2, it shut down cleanly without any error



> Shutdown slow when there is only one broker which is controller
> ---
>
> Key: KAFKA-3585
> URL: https://issues.apache.org/jira/browse/KAFKA-3585
> Project: Kafka
>  Issue Type: Improvement
>Affects Versions: 0.9.0.1
>Reporter: Pengwei
>Assignee: Taiyuan Zhang
>Priority: Minor
> Fix For: 0.10.0.1
>
>
> Reproducer Step:
> 1. Install 3 brokers's cluster
> 2. create a topic with 3 partition
> 3. shutdown the broker one by one , you will find the last one shutdown very 
> slow because of error:
> [2016-04-19 20:30:19,168] INFO [Kafka Server 1], Remaining partitions to 
> move: 
> __consumer_offsets-48,__consumer_offsets-13,__consumer_offsets-46,__consumer_offsets-11,__consumer_offsets-44,__consumer_offsets-42,__consumer_offsets-21,__consumer_offsets-19,__consumer_offsets-32,__consumer_offsets-30,__consumer_offsets-28,__consumer_offsets-26,__consumer_offsets-7,__consumer_offsets-40,__consumer_offsets-38,__consumer_offsets-36,__consumer_offsets-1,__consumer_offsets-34,__consumer_offsets-16,__consumer_offsets-45,__consumer_offsets-14,__consumer_offsets-12,__consumer_offsets-41,__consumer_offsets-10,__consumer_offsets-24,__consumer_offsets-22,__consumer_offsets-20,__consumer_offsets-49,__consumer_offsets-18,__consumer_offsets-31,__consumer_offsets-0,test2-0,__consumer_offsets-27,__consumer_offsets-39,__consumer_offsets-8,__consumer_offsets-37,__consumer_offsets-6,__consumer_offsets-4,__consumer_offsets-2
>  (kafka.server.KafkaServer)
> [2016-04-19 20:30:19,169] INFO [Kafka Server 1], Error code from controller: 
> 0 (kafka.server.KafkaServer)
> [2016-04-19 20:30:24,169] WARN [Kafka Server 1], Retrying controlled shutdown 
> after the previous attempt failed... (kafka.server.KafkaServer)
> [2016-04-19 20:30:24,171] WARN [Kafka Server 1], Proceeding to do an unclean 
> shutdown as all the controlled shutdown attempts failed 
> (kafka.server.KafkaServer)
> it is determined by :
> controlled.shutdown.retry.backoff.ms  = 5000
> controlled.shutdown.max.retries=3
> It slow because the last one can not elect the new leader for the remaining 
> partitions , the last one can improve to shutdown quickly, we can skip the 
> shutdown error when it is the last broker



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-3674) Connector target state changes not propagated to all workers

2016-05-07 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-3674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15275302#comment-15275302
 ] 

ASF GitHub Bot commented on KAFKA-3674:
---

GitHub user hachikuji opened a pull request:

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

KAFKA-3674: Ensure connector target state changes propagated to worker



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

$ git pull https://github.com/hachikuji/kafka KAFKA-3674

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

https://github.com/apache/kafka/pull/1341.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 #1341


commit cc6ea79edb2f7b734b0e277ac1b7d3cc231064fb
Author: Jason Gustafson 
Date:   2016-05-07T01:04:14Z

KAFKA-3674: Ensure connector target state changes propagated to worker




> Connector target state changes not propagated to all workers
> 
>
> Key: KAFKA-3674
> URL: https://issues.apache.org/jira/browse/KAFKA-3674
> Project: Kafka
>  Issue Type: Bug
>  Components: KafkaConnect
>Reporter: Jason Gustafson
>Assignee: Jason Gustafson
>Priority: Blocker
>
> Current handling of target state changes to a connector in DistributedHerder 
> seems dubious. We don't appear to propagate changes to the worker unless it 
> is running the connector itself, which means tasks running on separate 
> workers will not be notified of state changes. This should have been caught 
> with unit tests, but current coverage seems quite poor, so we should improve 
> that as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] kafka pull request: KAFKA-3674: Ensure connector target state chan...

2016-05-07 Thread hachikuji
GitHub user hachikuji opened a pull request:

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

KAFKA-3674: Ensure connector target state changes propagated to worker



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

$ git pull https://github.com/hachikuji/kafka KAFKA-3674

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

https://github.com/apache/kafka/pull/1341.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 #1341


commit cc6ea79edb2f7b734b0e277ac1b7d3cc231064fb
Author: Jason Gustafson 
Date:   2016-05-07T01:04:14Z

KAFKA-3674: Ensure connector target state changes propagated to worker




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] kafka pull request: HOTFIX: Ensure connector target state changes ...

2016-05-07 Thread hachikuji
Github user hachikuji closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Created] (KAFKA-3674) Connector target state changes not propagated to all workers

2016-05-07 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-3674:
--

 Summary: Connector target state changes not propagated to all 
workers
 Key: KAFKA-3674
 URL: https://issues.apache.org/jira/browse/KAFKA-3674
 Project: Kafka
  Issue Type: Bug
  Components: KafkaConnect
Reporter: Jason Gustafson
Assignee: Jason Gustafson
Priority: Blocker


Current handling of target state changes to a connector in DistributedHerder 
seems dubious. We don't appear to propagate changes to the worker unless it is 
running the connector itself, which means tasks running on separate workers 
will not be notified of state changes. This should have been caught with unit 
tests, but current coverage seems quite poor, so we should improve that as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: [VOTE] KIP-57: Interoperable LZ4 Framing

2016-05-07 Thread Dana Powers
Vote Passed! I will update the wiki.

-Dana
On May 7, 2016 3:48 AM, "Ismael Juma"  wrote:

> Dana, a long time has passed since the vote started and there are enough
> binding votes, so maybe it's time to declare that the vote has passed?
> Please mark the KIP as adopted in the KIP page and move it to the adopted
> table in the KIPs page once you do that.
>
> Ismael
> On 6 May 2016 22:16, "Ismael Juma"  wrote:
>
> +1 (assuming the changes I mentioned in the discuss thread are
> incorporated)
>
> Ismael
>
> On Thu, May 5, 2016 at 1:13 AM, Jun Rao  wrote:
>
> > Thanks for the response. +1 on the KIP.
> >
> > Jun
> >
> > On Thu, Apr 28, 2016 at 9:01 AM, Dana Powers 
> > wrote:
> >
> > > Sure thing. Yes, the substantive change is fixing the HC checksum.
> > >
> > > But to further improve interoperability, the kafka LZ4 class would no
> > > longer reject messages that have these optional header flags set. The
> > > flags might get set if the client/user chooses to use a non-java lz4
> > > compression library that includes them. In practice, naive support for
> > > the flags just means reading a few extra bytes in the header and/or
> > > footer of the payload. The KIP does not intend to use or validate this
> > > extra data.
> > >
> > > ContentSize is described as: "This field has no impact on decoding, it
> > > just informs the decoder how much data the frame holds (for example,
> > > to display it during decoding process, or for verification purpose).
> > > It can be safely skipped by a conformant decoder." We skip it.
> > >
> > > ContentChecksum is "Content Checksum validates the result, that all
> > > blocks were fully transmitted in the correct order and without error,
> > > and also that the encoding/decoding process itself generated no
> > > distortion." We skip it.
> > >
> > > -Dana
> > >
> > >
> > > On Thu, Apr 28, 2016 at 7:43 AM, Jun Rao  wrote:
> > > > Hi, Dana,
> > > >
> > > > Could you explain the following from the KIP a bit more? The KIP is
> > > > intended to just fix the HC checksum, but the following seems to
> > suggest
> > > > there are other format changes?
> > > >
> > > > KafkaLZ4* code:
> > > >
> > > >- add naive support for optional header flags (ContentSize,
> > > >ContentChecksum) to enable interoperability with off-the-shelf lz4
> > > libraries
> > > >- the only flag left unsupported is dependent-block compression,
> > which
> > > >our implementation does not currently support.
> > > >
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Apr 25, 2016 at 2:26 PM, Dana Powers 
> > > wrote:
> > > >
> > > >> Hi all,
> > > >>
> > > >> Initiating a vote thread because the KIP-57 proposal is specific to
> > > >> the 0.10 release.
> > > >>
> > > >> KIP-57 can be accessed here:
> > > >> <
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-57+-+Interoperable+LZ4+Framing
> > > >> >.
> > > >>
> > > >> The related JIRA is
> https://issues.apache.org/jira/browse/KAFKA-3160
> > > >> and working github PR at https://github.com/apache/kafka/pull/1212
> > > >>
> > > >> The vote will run for 72 hours.
> > > >>
> > > >> +1 (non-binding)
> > > >>
> > >
> >
>


Re: How to submit a KIP?

2016-05-07 Thread Ismael Juma
Jayesh, I have given you the required permissions. The process is outlined
in the following page (under the Process section):

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

In short, you should start a Discuss thread in the mailing list once the
KIP is created.

Thanks,
Ismael

On Sat, May 7, 2016 at 3:27 PM, Jayesh Thakrar  wrote:

> Thank you Ismael.
> My Apache id is jthakrar.
>
> Also, if you can let me know what the next step would be after creating
> the KIP.
>
> Thanks,
> Jayesh
>
>
> --
> *From:* Ismael Juma 
> *To:* dev@kafka.apache.org; Jayesh Thakrar 
> *Sent:* Saturday, May 7, 2016 2:57 AM
> *Subject:* Re: How to submit a KIP?
>
> Hi Jayesh,
>
> You need to create an Apache wiki account (if you haven't already) and
> then share your wiki id so that one of us can give you the required
> permissions.
>
> Thanks,
> Ismael
>
> On Sat, May 7, 2016 at 3:44 AM, Jayesh Thakrar <
> j_thak...@yahoo.com.invalid> wrote:
>
> Hi,
> I have created a Jira Issue -
> https://issues.apache.org/jira/browse/KAFKA-3663 and was wondering what
> is the next step?Should I create a KIP? If so, how do I do that, as I do
> not have permissions to do that.
> Thanks,Jayesh Thakrar
>
>
>
>
>


[jira] [Commented] (KAFKA-1981) Make log compaction point configurable

2016-05-07 Thread Ismael Juma (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15275276#comment-15275276
 ] 

Ismael Juma commented on KAFKA-1981:


Sorry, my bad, you should have the required permissions to create wiki pages 
now.

> Make log compaction point configurable
> --
>
> Key: KAFKA-1981
> URL: https://issues.apache.org/jira/browse/KAFKA-1981
> Project: Kafka
>  Issue Type: Improvement
>Affects Versions: 0.8.2.0
>Reporter: Jay Kreps
>  Labels: newbie++
> Attachments: KIP for Kafka Compaction Patch.md
>
>
> Currently if you enable log compaction the compactor will kick in whenever 
> you hit a certain "dirty ratio", i.e. when 50% of your data is uncompacted. 
> Other than this we don't give you fine-grained control over when compaction 
> occurs. In addition we never compact the active segment (since it is still 
> being written to).
> Other than this we don't really give you much control over when compaction 
> will happen. The result is that you can't really guarantee that a consumer 
> will get every update to a compacted topic--if the consumer falls behind a 
> bit it might just get the compacted version.
> This is usually fine, but it would be nice to make this more configurable so 
> you could set either a # messages, size, or time bound for compaction.
> This would let you say, for example, "any consumer that is no more than 1 
> hour behind will get every message."
> This should be relatively easy to implement since it just impacts the 
> end-point the compactor considers available for compaction. I think we 
> already have that concept, so this would just be some other overrides to add 
> in when calculating that.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: How to submit a KIP?

2016-05-07 Thread Jayesh Thakrar
Thank you Ismael.My Apache id is jthakrar.
Also, if you can let me know what the next step would be after creating the KIP.
Thanks,Jayesh

  From: Ismael Juma 
 To: dev@kafka.apache.org; Jayesh Thakrar  
 Sent: Saturday, May 7, 2016 2:57 AM
 Subject: Re: How to submit a KIP?
   
Hi Jayesh,
You need to create an Apache wiki account (if you haven't already) and then 
share your wiki id so that one of us can give you the required permissions.
Thanks,Ismael
On Sat, May 7, 2016 at 3:44 AM, Jayesh Thakrar  
wrote:

Hi,
I have created a Jira Issue - https://issues.apache.org/jira/browse/KAFKA-3663 
and was wondering what is the next step?Should I create a KIP? If so, how do I 
do that, as I do not have permissions to do that.
Thanks,Jayesh Thakrar




  

Re: [VOTE] KIP-57: Interoperable LZ4 Framing

2016-05-07 Thread Ismael Juma
Dana, a long time has passed since the vote started and there are enough
binding votes, so maybe it's time to declare that the vote has passed?
Please mark the KIP as adopted in the KIP page and move it to the adopted
table in the KIPs page once you do that.

Ismael
On 6 May 2016 22:16, "Ismael Juma"  wrote:

+1 (assuming the changes I mentioned in the discuss thread are incorporated)

Ismael

On Thu, May 5, 2016 at 1:13 AM, Jun Rao  wrote:

> Thanks for the response. +1 on the KIP.
>
> Jun
>
> On Thu, Apr 28, 2016 at 9:01 AM, Dana Powers 
> wrote:
>
> > Sure thing. Yes, the substantive change is fixing the HC checksum.
> >
> > But to further improve interoperability, the kafka LZ4 class would no
> > longer reject messages that have these optional header flags set. The
> > flags might get set if the client/user chooses to use a non-java lz4
> > compression library that includes them. In practice, naive support for
> > the flags just means reading a few extra bytes in the header and/or
> > footer of the payload. The KIP does not intend to use or validate this
> > extra data.
> >
> > ContentSize is described as: "This field has no impact on decoding, it
> > just informs the decoder how much data the frame holds (for example,
> > to display it during decoding process, or for verification purpose).
> > It can be safely skipped by a conformant decoder." We skip it.
> >
> > ContentChecksum is "Content Checksum validates the result, that all
> > blocks were fully transmitted in the correct order and without error,
> > and also that the encoding/decoding process itself generated no
> > distortion." We skip it.
> >
> > -Dana
> >
> >
> > On Thu, Apr 28, 2016 at 7:43 AM, Jun Rao  wrote:
> > > Hi, Dana,
> > >
> > > Could you explain the following from the KIP a bit more? The KIP is
> > > intended to just fix the HC checksum, but the following seems to
> suggest
> > > there are other format changes?
> > >
> > > KafkaLZ4* code:
> > >
> > >- add naive support for optional header flags (ContentSize,
> > >ContentChecksum) to enable interoperability with off-the-shelf lz4
> > libraries
> > >- the only flag left unsupported is dependent-block compression,
> which
> > >our implementation does not currently support.
> > >
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Apr 25, 2016 at 2:26 PM, Dana Powers 
> > wrote:
> > >
> > >> Hi all,
> > >>
> > >> Initiating a vote thread because the KIP-57 proposal is specific to
> > >> the 0.10 release.
> > >>
> > >> KIP-57 can be accessed here:
> > >> <
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-57+-+Interoperable+LZ4+Framing
> > >> >.
> > >>
> > >> The related JIRA is https://issues.apache.org/jira/browse/KAFKA-3160
> > >> and working github PR at https://github.com/apache/kafka/pull/1212
> > >>
> > >> The vote will run for 72 hours.
> > >>
> > >> +1 (non-binding)
> > >>
> >
>


[jira] [Commented] (KAFKA-3385) Need to log "Rejected connection" as WARNING message

2016-05-07 Thread Andrea Cosentino (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-3385?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15275187#comment-15275187
 ] 

Andrea Cosentino commented on KAFKA-3385:
-

Thanks [~ijuma] :-)

> Need to log "Rejected connection" as WARNING message
> 
>
> Key: KAFKA-3385
> URL: https://issues.apache.org/jira/browse/KAFKA-3385
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Xiaomin Zhang
>Assignee: Andrea Cosentino
>Priority: Minor
>
> We may found below INFO messages in the log due to inappropriate 
> configuration:
> INFO kafka.network. Acceptor: Rejected connection from /, address already 
> has the configured maximum of 10 connections.
> It will make more sense for Kafka to report above message as "WARN", not just 
> "INFO", as it truly indicates something need to check against. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-3385) Need to log "Rejected connection" as WARNING message

2016-05-07 Thread Ismael Juma (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-3385?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15275186#comment-15275186
 ] 

Ismael Juma commented on KAFKA-3385:


I think the question here is whether we think that going over the max number of 
connections is an expected event or whether it indicates a misconfiguration. 
The JIRA author thinks it's the latter. The people who wrote thought it was the 
former. It would be interesting to hear more opinions.

> Need to log "Rejected connection" as WARNING message
> 
>
> Key: KAFKA-3385
> URL: https://issues.apache.org/jira/browse/KAFKA-3385
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Xiaomin Zhang
>Assignee: Andrea Cosentino
>Priority: Minor
>
> We may found below INFO messages in the log due to inappropriate 
> configuration:
> INFO kafka.network. Acceptor: Rejected connection from /, address already 
> has the configured maximum of 10 connections.
> It will make more sense for Kafka to report above message as "WARN", not just 
> "INFO", as it truly indicates something need to check against. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Jenkins build is back to normal : kafka-trunk-jdk7 #1263

2016-05-07 Thread Apache Jenkins Server
See 



[jira] [Commented] (KAFKA-3385) Need to log "Rejected connection" as WARNING message

2016-05-07 Thread Andrea Cosentino (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-3385?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15275175#comment-15275175
 ] 

Andrea Cosentino commented on KAFKA-3385:
-

Is there any news on this PR? It is open since 1 month and half. I know it's 
not important, but if it's not useful I can close the PR.

> Need to log "Rejected connection" as WARNING message
> 
>
> Key: KAFKA-3385
> URL: https://issues.apache.org/jira/browse/KAFKA-3385
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Xiaomin Zhang
>Assignee: Andrea Cosentino
>Priority: Minor
>
> We may found below INFO messages in the log due to inappropriate 
> configuration:
> INFO kafka.network. Acceptor: Rejected connection from /, address already 
> has the configured maximum of 10 connections.
> It will make more sense for Kafka to report above message as "WARN", not just 
> "INFO", as it truly indicates something need to check against. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


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

2016-05-07 Thread Apache Jenkins Server
See 



Build failed in Jenkins: kafka-0.10.0-jdk7 #62

2016-05-07 Thread Apache Jenkins Server
See 

Changes:

[ismael] KAFKA-3112; Warn instead of error on unresolvable bootstrap server

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on H10 (docker Ubuntu ubuntu yahoo-not-h2) in workspace 

 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git -c core.askpass=true fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
 > git rev-parse refs/remotes/origin/0.10.0^{commit} # timeout=10
 > git rev-parse refs/remotes/origin/origin/0.10.0^{commit} # timeout=10
Checking out Revision 2277b3c43edc3b756c8262373846e93589998997 
(refs/remotes/origin/0.10.0)
 > git config core.sparsecheckout # timeout=10
 > git checkout -f 2277b3c43edc3b756c8262373846e93589998997
 > git rev-list b904a427c0e5e5683590528e54da33226c075c0b # timeout=10
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51
[kafka-0.10.0-jdk7] $ /bin/bash -xe /tmp/hudson7073909568751234982.sh
+ 
/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2/bin/gradle
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
http://gradle.org/docs/2.4-rc-2/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
:downloadWrapper

BUILD SUCCESSFUL

Total time: 26.03 secs
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51
[kafka-0.10.0-jdk7] $ /bin/bash -xe /tmp/hudson1850528906907802433.sh
+ export GRADLE_OPTS=-Xmx1024m
+ GRADLE_OPTS=-Xmx1024m
+ ./gradlew -Dorg.gradle.project.maxParallelForks=1 clean jarAll testAll
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
https://docs.gradle.org/2.13/userguide/gradle_daemon.html.
Building project 'core' with Scala version 2.10.6
Build file ': 
line 230
useAnt has been deprecated and is scheduled to be removed in Gradle 3.0. The 
Ant-Based Scala compiler is deprecated, please see 
https://docs.gradle.org/current/userguide/scala_plugin.html.
:clean UP-TO-DATE
:clients:clean UP-TO-DATE
:connect:clean UP-TO-DATE
:core:clean
:examples:clean UP-TO-DATE
:log4j-appender:clean UP-TO-DATE
:streams:clean UP-TO-DATE
:tools:clean UP-TO-DATE
:connect:api:clean UP-TO-DATE
:connect:file:clean UP-TO-DATE
:connect:json:clean UP-TO-DATE
:connect:runtime:clean UP-TO-DATE
:streams:examples:clean UP-TO-DATE
:jar_core_2_10
Building project 'core' with Scala version 2.10.6
:kafka-0.10.0-jdk7:clients:compileJava
:jar_core_2_10 FAILED

FAILURE: Build failed with an exception.

* What went wrong:
org.gradle.api.internal.changedetection.state.FileCollectionSnapshotImpl cannot 
be cast to 
org.gradle.api.internal.changedetection.state.OutputFilesCollectionSnapshotter$OutputFilesSnapshot

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug 
option to get more log output.

BUILD FAILED

Total time: 22.915 secs
Build step 'Execute shell' marked build as failure
Recording test results
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51
ERROR: Step ?Publish JUnit test result report? failed: No test report files 
were found. Configuration error?
Setting 
GRADLE_2_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_2.4-rc-2
Setting 
JDK_1_7U51_HOME=/home/jenkins/jenkins-slave/tools/hudson.model.JDK/jdk-1.7u51


Re: KAFKA-3112

2016-05-07 Thread Ismael Juma
Hi Raj and Tao,

I just merged the KAFKA-3112 PR, so this issue will be fixed in 0.10.0.0.

Thanks,
Ismael

On Fri, May 6, 2016 at 7:47 PM, tao xiao  wrote:

> KAFKA-2657 is unresolved so you can safely assume it hasn't been fixed yet.
>
> On Fri, 6 May 2016 at 07:38 Raj Tanneru  wrote:
>
> > Yeah it is a duplicate of KAFKA-2657. The question is how to check / know
> > if it is merged to 0.9.0.1 release. What are the options that I have if I
> > need this fix. How can I get patch for this on 0.8.2.1?
> >
> > Sent from my iPhone
> >
> > > On May 6, 2016, at 12:06 AM, tao xiao  wrote:
> > >
> > > It said this is a duplication. This is the
> > > https://issues.apache.org/jira/browse/KAFKA-2657 that KAKFA-3112
> > duplicates
> > > to.
> > >
> > >> On Thu, 5 May 2016 at 22:13 Raj Tanneru 
> wrote:
> > >>
> > >>
> > >> Hi All,
> > >> Does anyone know if KAFKA-3112 is merged to 0.9.0.1? Is there a place
> to
> > >> check which version has this fix? Jira doesn’t show fix versions.
> > >>
> > >> https://issues.apache.org/jira/browse/KAFKA-3112
> > >>
> > >>
> > >> Thanks,
> > >> Raj Tanneru
> > >> Information contained in this e-mail message is confidential. This
> > e-mail
> > >> message is intended only for the personal use of the recipient(s)
> named
> > >> above. If you are not an intended recipient, do not read, distribute
> or
> > >> reproduce this transmission (including any attachments). If you have
> > >> received this email in error, please immediately notify the sender by
> > email
> > >> reply and delete the original message.
> > >>
> > Information contained in this e-mail message is confidential. This e-mail
> > message is intended only for the personal use of the recipient(s) named
> > above. If you are not an intended recipient, do not read, distribute or
> > reproduce this transmission (including any attachments). If you have
> > received this email in error, please immediately notify the sender by
> email
> > reply and delete the original message.
> >
>


[jira] [Resolved] (KAFKA-2657) Kafka clients fail to start if one of broker isn't resolved by DNS

2016-05-07 Thread Ismael Juma (JIRA)

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

Ismael Juma resolved KAFKA-2657.

Resolution: Duplicate

This was fixed via KAFKA-3112.

> Kafka clients fail to start if one of broker isn't resolved by DNS 
> ---
>
> Key: KAFKA-2657
> URL: https://issues.apache.org/jira/browse/KAFKA-2657
> Project: Kafka
>  Issue Type: Bug
>Reporter: Alexander Pakulov
>Priority: Minor
>
> During org.apache.kafka.clients.producer.KafkaProducer and 
> org.apache.kafka.clients.consumer.KafkaConsumer object creation constructors 
> invoke org.apache.kafka.common.utils.ClientUtils#parseAndValidateAddresses
> which potentially could throw an exception if one the nodes hasn't been 
> resolved by DNS. As a result of that - object hasn't been created and you 
> aren't able to use Kafka clients.
> I personally think that Kafka should be able to operate with cluster with 
> quorum number of instances.
> {code:java}
> try {
> InetSocketAddress address = new InetSocketAddress(host, port);
> if (address.isUnresolved())
> throw new ConfigException("DNS resolution failed for url in " + 
> ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
> addresses.add(address);
> } catch (NumberFormatException e) {
> throw new ConfigException("Invalid port in " + 
> ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + ": " + url);
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (KAFKA-3112) One incorrect bootstrap server will prevent Kafka producer from opening

2016-05-07 Thread Ismael Juma (JIRA)

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

Ismael Juma resolved KAFKA-3112.

   Resolution: Fixed
 Reviewer: Ismael Juma
Fix Version/s: 0.10.0.0

> One incorrect bootstrap server will prevent Kafka producer from opening
> ---
>
> Key: KAFKA-3112
> URL: https://issues.apache.org/jira/browse/KAFKA-3112
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.8.2.1
>Reporter: Jonathan Bond
>Assignee: Jonathan Bond
>Priority: Critical
> Fix For: 0.10.0.0
>
>
> If any of the servers specified in bootstrap.servers are not resolvable 
> through DNS the configuration is taken as an error, and the client won't 
> start up. We pass in 30 possible servers, and one had an issue so the client 
> wouldn't start. 
> It would be better if the client will attempt to start if there is at least 
> one server available from DNS.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Reopened] (KAFKA-3112) One incorrect bootstrap server will prevent Kafka producer from opening

2016-05-07 Thread Ismael Juma (JIRA)

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

Ismael Juma reopened KAFKA-3112:


> One incorrect bootstrap server will prevent Kafka producer from opening
> ---
>
> Key: KAFKA-3112
> URL: https://issues.apache.org/jira/browse/KAFKA-3112
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.8.2.1
>Reporter: Jonathan Bond
>Assignee: Jonathan Bond
>Priority: Critical
>
> If any of the servers specified in bootstrap.servers are not resolvable 
> through DNS the configuration is taken as an error, and the client won't 
> start up. We pass in 30 possible servers, and one had an issue so the client 
> wouldn't start. 
> It would be better if the client will attempt to start if there is at least 
> one server available from DNS.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (KAFKA-3112) One incorrect bootstrap server will prevent Kafka producer from opening

2016-05-07 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-3112:
---
Assignee: Jonathan Bond

> One incorrect bootstrap server will prevent Kafka producer from opening
> ---
>
> Key: KAFKA-3112
> URL: https://issues.apache.org/jira/browse/KAFKA-3112
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.8.2.1
>Reporter: Jonathan Bond
>Assignee: Jonathan Bond
>Priority: Critical
>
> If any of the servers specified in bootstrap.servers are not resolvable 
> through DNS the configuration is taken as an error, and the client won't 
> start up. We pass in 30 possible servers, and one had an issue so the client 
> wouldn't start. 
> It would be better if the client will attempt to start if there is at least 
> one server available from DNS.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-3112) One incorrect bootstrap server will prevent Kafka producer from opening

2016-05-07 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-3112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15275148#comment-15275148
 ] 

ASF GitHub Bot commented on KAFKA-3112:
---

Github user asfgit closed the pull request at:

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


> One incorrect bootstrap server will prevent Kafka producer from opening
> ---
>
> Key: KAFKA-3112
> URL: https://issues.apache.org/jira/browse/KAFKA-3112
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.8.2.1
>Reporter: Jonathan Bond
>Priority: Critical
>
> If any of the servers specified in bootstrap.servers are not resolvable 
> through DNS the configuration is taken as an error, and the client won't 
> start up. We pass in 30 possible servers, and one had an issue so the client 
> wouldn't start. 
> It would be better if the client will attempt to start if there is at least 
> one server available from DNS.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[GitHub] kafka pull request: KAFKA-3112: Change verification of bootstrap s...

2016-05-07 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


Re: How to submit a KIP?

2016-05-07 Thread Ismael Juma
Hi Jayesh,

You need to create an Apache wiki account (if you haven't already) and then
share your wiki id so that one of us can give you the required permissions.

Thanks,
Ismael

On Sat, May 7, 2016 at 3:44 AM, Jayesh Thakrar 
wrote:

> Hi,
> I have created a Jira Issue -
> https://issues.apache.org/jira/browse/KAFKA-3663 and was wondering what
> is the next step?Should I create a KIP? If so, how do I do that, as I do
> not have permissions to do that.
> Thanks,Jayesh Thakrar
>