Re: [DISCUSS] KIP-31 - Message format change proposal

2015-09-03 Thread Jiangjie Qin
Hey Ewen,

Thanks for the comments and they are really good questions. Please inline
replies.

On Thu, Sep 3, 2015 at 9:35 PM, Ewen Cheslack-Postava 
wrote:

> A few questions:
>
> 1. If we update the producers to only support V1, doesn't that mean people
> get stuck on the current version of the producer until they can be sure all
> their consumers have been upgraded? Is that going to be a problem for
> anyone, and does it potentially keep important fixes/enhancements (e.g. the
> upcoming client-side network request timeouts) because they have to wait
> for all the consumers to upgrade first?
>
This is a good point. I thought about this before, I and my initial
thinking is that we might need to add a config on producer to specify which
version you want to use to produce. But this seems to be a pretty ad-hoc
approach and I don't really like it. We are working on some general
protocol version control mechanism proposal and will have a separate KIP
for that.

>
> 2. Why minute granularity specifically? That's seems fairly specific to
> LI's example workload (which, by the way, is a very useful reference point
> to include in the KIP, thanks for that). If someone has the memory and can
> support it, why not let them go all the way to per-record timestamps (by
> making the interval configurable and they can set it really small/to 0)? It
> seems to me that some people might be willing to pay that cost for the
> application-side simplicity if they want to consume from specific
> timestamps. With the proposal as is, seeking to a timestamp still requires
> client-side logic to filter out messages that might have earlier timestamps
> due to the granularity chosen. The obvious tradeoff here is yet another
> config option -- I'm loath to add yet more configs, although this is one
> that we can choose a reasonable default for (like 1 minute) and people can
> easily change with no impact if they need to adjust it.
>
The searching granularity will be actually millisecond. The index
granularity only determines how close you will be to the actually message
with the timestamp you are looking for. For example, if you are looking for
a message with timestamp 10:00:15, a minute granularity will give you the
offset at 10:00:00, and it needs to go through the records from 10:00:00 to
10:00:15 to find the message. But with a second level granularity, it might
only need to go through the message produced in one second. So minute level
granularity index will take longer for search, but the precision will be
the same as second level index. That said, I am not objecting to adding the
granularity configuration but I am not sure how useful it would be to have
second level index because I think typically a consumer will be
long-running and only search for the timestamp at startup.
I will update the KIP page to clarify the precision.

>

3. Why not respect the timestamp passed in as long as it's not some
> sentinel value that indicates the broker should fill it in? When you do
> know they will be ordered, as they should be with mirror maker (which is
> specifically mentioned), this seems really useful. (More about this in
> questions below...)
>
Like what you mentioned in (4), having a log without monotonically
increasing timestamp is weird. To me it is even worse than having an empty
timestamp field in the inner message that will not be used except for log
compacted topic. I think the only way to solve this issue is to add another
CreateTime to the message. So far I am not sure how useful it is though
because arguably people can always put this timestamp in side the payload.
So I think this timestamp is more for server side usage instead of
application / client side usage.

>
> 4. I think one obvious answer to (3) is that accepting client's timestamps
> could break seeking-by-timestamp since they may not be properly ordered.
> However, I think this can break anyway under normal operations -- any
> imperfections in clock synchronization could result in older timestamps
> being applied to new messages on a new leader broker compared messages
> stored previously by the old leader. I think it's probably too common to
> just think that NTP will take care of it and then run into weird bugs
> because NTP isn't always perfect, sometimes does some unexpected things,
> and, of course, does what you tell it to and so is subject to user error.

This is a good point. Yes, NTP only guarantee limited synchronization
precision (several microseconds if you have low stratum and appropriate
PPS). My experience is that it actually is good and stable enough even for
some mission critical system such a core banking. Maybe this is more of an
implementation detail. The simple solution is that when leader append
messages to the log, it always take the max(lastAppendedTimestamp,
currentTimeMillis). Arguably we can play the same trick even if we let the
producer to fill in the timestamp. But that means the timestamp producer
set may ore may not be honored, which

[jira] [Updated] (KAFKA-1927) Replace requests in kafka.api with requests in org.apache.kafka.common.requests

2015-09-03 Thread jin xing (JIRA)

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

jin xing updated KAFKA-1927:

Assignee: Gwen Shapira

> Replace requests in kafka.api with requests in 
> org.apache.kafka.common.requests
> ---
>
> Key: KAFKA-1927
> URL: https://issues.apache.org/jira/browse/KAFKA-1927
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Jay Kreps
>Assignee: Gwen Shapira
> Attachments: KAFKA-1927.patch
>
>
> The common package introduced a better way of defining requests using a new 
> protocol definition DSL and also includes wrapper objects for these.
> We should switch KafkaApis over to use these request definitions and consider 
> the scala classes deprecated (we probably need to retain some of them for a 
> while for the scala clients).
> This will be a big improvement because
> 1. We will have each request now defined in only one place (Protocol.java)
> 2. We will have built-in support for multi-version requests
> 3. We will have much better error messages (no more cryptic underflow errors)



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


[jira] [Assigned] (KAFKA-1927) Replace requests in kafka.api with requests in org.apache.kafka.common.requests

2015-09-03 Thread jin xing (JIRA)

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

jin xing reassigned KAFKA-1927:
---

Assignee: jin xing  (was: Gwen Shapira)

> Replace requests in kafka.api with requests in 
> org.apache.kafka.common.requests
> ---
>
> Key: KAFKA-1927
> URL: https://issues.apache.org/jira/browse/KAFKA-1927
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Jay Kreps
>Assignee: jin xing
> Attachments: KAFKA-1927.patch
>
>
> The common package introduced a better way of defining requests using a new 
> protocol definition DSL and also includes wrapper objects for these.
> We should switch KafkaApis over to use these request definitions and consider 
> the scala classes deprecated (we probably need to retain some of them for a 
> while for the scala clients).
> This will be a big improvement because
> 1. We will have each request now defined in only one place (Protocol.java)
> 2. We will have built-in support for multi-version requests
> 3. We will have much better error messages (no more cryptic underflow errors)



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


[jira] [Updated] (KAFKA-1927) Replace requests in kafka.api with requests in org.apache.kafka.common.requests

2015-09-03 Thread jin xing (JIRA)

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

jin xing updated KAFKA-1927:

Assignee: (was: jin xing)

> Replace requests in kafka.api with requests in 
> org.apache.kafka.common.requests
> ---
>
> Key: KAFKA-1927
> URL: https://issues.apache.org/jira/browse/KAFKA-1927
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Jay Kreps
> Attachments: KAFKA-1927.patch
>
>
> The common package introduced a better way of defining requests using a new 
> protocol definition DSL and also includes wrapper objects for these.
> We should switch KafkaApis over to use these request definitions and consider 
> the scala classes deprecated (we probably need to retain some of them for a 
> while for the scala clients).
> This will be a big improvement because
> 1. We will have each request now defined in only one place (Protocol.java)
> 2. We will have built-in support for multi-version requests
> 3. We will have much better error messages (no more cryptic underflow errors)



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


[jira] [Commented] (KAFKA-2513) Checkstyle is not executed until gradle's tests phase

2015-09-03 Thread Ewen Cheslack-Postava (JIRA)

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

Ewen Cheslack-Postava commented on KAFKA-2513:
--

[~singhashish] Compile tasks don't generate something that can be run via the 
normal scripts. To use any of the normal tooling, you need to {{jar}}. I'm all 
for automated validation of code, but it should be a barrier to getting code 
checked in, not to me being able to run in-development code on my local machine.

I'm pretty vocal about stuff like this because it can have a huge impact on 
developer efficiency. It can actually take me significantly longer to clean up 
some crappy test/debugging code than it would to run the *system test* I'm 
trying to run to debug or validate some code. I really don't want a tool that's 
intended to make sure we don't *check in* bad code to get in the way of me 
iterating locally on dev code.

(If it's not obvious, there are a couple of rules that, by muscle memory, I 
consistently fail. They include:
1. for in for loops must have a space between "for" and the paranthesis
2. casts must have spaces after them as well
3. You're not allowed to put parentheses around ternary operators unless they 
are absolutely required. I actually think we should remove this one, it really 
hurts clarity in some cases.
)

> Checkstyle is not executed until gradle's tests phase
> -
>
> Key: KAFKA-2513
> URL: https://issues.apache.org/jira/browse/KAFKA-2513
> Project: Kafka
>  Issue Type: Bug
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
>Priority: Minor
>
> Checkstyle is added as a dependency to test and until someone runs test they 
> won't capture checkstyle issues. To me code style is more suited along with 
> compile than tests. This was also brought up on KAFKA-1893. May be we should 
> make checkstyle a dependency of jar, instead of test.



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


[jira] [Updated] (KAFKA-2489) System tests: update benchmark tests to run with new and old consumer

2015-09-03 Thread Ewen Cheslack-Postava (JIRA)

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

Ewen Cheslack-Postava updated KAFKA-2489:
-
Reviewer: Gwen Shapira  (was: Ewen Cheslack-Postava)
  Status: Patch Available  (was: Open)

> System tests: update benchmark tests to run with new and old consumer
> -
>
> Key: KAFKA-2489
> URL: https://issues.apache.org/jira/browse/KAFKA-2489
> Project: Kafka
>  Issue Type: Bug
>Reporter: Geoff Anderson
>Assignee: Geoff Anderson
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Update benchmark tests to run w/new consumer to help catch performance 
> regressions
> For context:
> https://www.mail-archive.com/dev@kafka.apache.org/msg33633.html
> The new consumer was previously reaching getting good performance. However, a 
> recent report on the mailing list indicates it's dropped significantly. After 
> evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
> compared to 600+MB/s previously. Before release, we should get the 
> performance 
> back on par.
> Some details about where the regression occurred from the mailing list 
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
>  :
> bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
> performance -> an error due to broker apparently not accepting the partition 
> assignment strategy. Since this commit seems to add heartbeats and the server 
> side code for partition assignment strategies, I assume we were missing 
> something on the client side and by filling in the server side, things 
> stopped 
> working.
> bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
> a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
> perf 
> test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, but 
> that's still far less than the 600MB/s I saw on the earlier commits.
> Ideally we would also at least have a system test in place for the new 
> consumer, even if regressions weren't automatically detected. It would at 
> least 
> allow for manually checking for regressions. This should not be difficult 
> since 
> there are already old consumer performance tests.



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


[jira] [Commented] (KAFKA-2516) Rename o.a.k.client.tools to o.a.k.tools

2015-09-03 Thread Ewen Cheslack-Postava (JIRA)

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

Ewen Cheslack-Postava commented on KAFKA-2516:
--

Compatibility czar checking in. Technically we've already released 
ProducerPerformance under o.a.k.clients.tools. Normally I'd say we need to take 
care to maintain some compatibility, but I think for this particular tool we 
can safely just move it. I doubt many people have that tool running in an 
automated way via the class name rather than the wrapper script, and in such a 
way that it would actually seriously break anything for them. 

However, I wanted to bring it up because we don't have an official policy 
around this and we've recently tried to be more careful about tools 
compatibility since they are a public API. Clearly the argument here is that 
some tools are not created equal and deciding as a project that benchmarking 
and simple testing tools are a separate class from normal operational tools and 
that we're not going to make compatibility guarantees for them. (If we're going 
that route, we could annotate them permanently with Unstable.) Also, this 
underscores the importance of getting the tools into the right location before 
they are released for the first time!

> Rename o.a.k.client.tools to o.a.k.tools
> 
>
> Key: KAFKA-2516
> URL: https://issues.apache.org/jira/browse/KAFKA-2516
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Currently our new performance tools are in o.a.k.client.tools but packaged in 
> kafka-tools not kafka-clients. This is a bit confusing.
> Since they deserve their own jar (you don't want our client tools packaged in 
> your app), lets give them a separate package and call it o.a.k.tools.



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


Re: [DISCUSS] KIP-31 - Message format change proposal

2015-09-03 Thread Ewen Cheslack-Postava
Heh, I guess in addition to my wall of text of questions, I should also say
that I think this provides useful functionality and fixes issues that we've
seen a bunch of questions and complaints about, so I'm in favor of a fix
and this looks like a pretty good approach :)

It might also be useful to say which release you're hoping to get this into.

-Ewen

On Thu, Sep 3, 2015 at 9:35 PM, Ewen Cheslack-Postava 
wrote:

> A few questions:
>
> 1. If we update the producers to only support V1, doesn't that mean people
> get stuck on the current version of the producer until they can be sure all
> their consumers have been upgraded? Is that going to be a problem for
> anyone, and does it potentially keep important fixes/enhancements (e.g. the
> upcoming client-side network request timeouts) because they have to wait
> for all the consumers to upgrade first?
>
> 2. Why minute granularity specifically? That's seems fairly specific to
> LI's example workload (which, by the way, is a very useful reference point
> to include in the KIP, thanks for that). If someone has the memory and can
> support it, why not let them go all the way to per-record timestamps (by
> making the interval configurable and they can set it really small/to 0)? It
> seems to me that some people might be willing to pay that cost for the
> application-side simplicity if they want to consume from specific
> timestamps. With the proposal as is, seeking to a timestamp still requires
> client-side logic to filter out messages that might have earlier timestamps
> due to the granularity chosen. The obvious tradeoff here is yet another
> config option -- I'm loath to add yet more configs, although this is one
> that we can choose a reasonable default for (like 1 minute) and people can
> easily change with no impact if they need to adjust it.
>
> 3. Why not respect the timestamp passed in as long as it's not some
> sentinel value that indicates the broker should fill it in? When you do
> know they will be ordered, as they should be with mirror maker (which is
> specifically mentioned), this seems really useful. (More about this in
> questions below...)
>
> 4. I think one obvious answer to (3) is that accepting client's timestamps
> could break seeking-by-timestamp since they may not be properly ordered.
> However, I think this can break anyway under normal operations -- any
> imperfections in clock synchronization could result in older timestamps
> being applied to new messages on a new leader broker compared messages
> stored previously by the old leader. I think it's probably too common to
> just think that NTP will take care of it and then run into weird bugs
> because NTP isn't always perfect, sometimes does some unexpected things,
> and, of course, does what you tell it to and so is subject to user error.
>
> It would be reasonable to argue that the leader change issue is much less
> likely of an issue than if you respect timestamps from producers, where, if
> applications actually filled it in, you'd receive a jumbled mess of
> timestamps and trying to do the binary search in the index wouldn't
> necessarily give you correct results. However, a) we could allow clients to
> fill that info in but discourage it where it might cause issues (i.e.
> normal applications) and it seems like a significant win for mirrormaker.
>
> I actually think not accepting timestamps is probably the better choice
> but a) it seems awkward in the protocol spec because we have to include the
> field in the produce requests since we don't want to have to fully decode
> them on the broker and b) losing that info during mirroring seems like it
> breaks the goal of fixing log retention (at least for mirrors) as well as
> the goal of improving searching by timestamp (at least for mirrors).
>
> 5. You never actually specified the granularity (or format, but I assume
> unix epoch) of the timestamp. Milliseconds? Microseconds? Nanoseconds? This
> definitely needs to eventually make it into the protocol docs.
>
> 6. Re: the rejected alternative. Are there any other options in changing
> the config format that might make it a bit lighter weight? For example, do
> we need a full int64? Could we do something relative instead that wouldn't
> require as many bytes? Without (5) being specified, it's actually difficult
> to evaluate some of these options.
>
> 7. Any plan to expose this in the client APIs? This is related to (4). If
> they are not exposed anywhere in the API as being associated with messages,
> then we can reasonably treat them as an internal implementation detail and
> be very clear about what looking up an offset for a timestamp means. If
> they are exposed, then they're more like message metadata that applications
> are probably going to want preserved, and thus will want the broker to
> respect the timestamp. For example, if I saw that consumers could get the
> timestamp, I'd want to be able to assign it at the producer so that even if
> I have significant batching I s

Re: [DISCUSS] KIP-31 - Message format change proposal

2015-09-03 Thread Ewen Cheslack-Postava
A few questions:

1. If we update the producers to only support V1, doesn't that mean people
get stuck on the current version of the producer until they can be sure all
their consumers have been upgraded? Is that going to be a problem for
anyone, and does it potentially keep important fixes/enhancements (e.g. the
upcoming client-side network request timeouts) because they have to wait
for all the consumers to upgrade first?

2. Why minute granularity specifically? That's seems fairly specific to
LI's example workload (which, by the way, is a very useful reference point
to include in the KIP, thanks for that). If someone has the memory and can
support it, why not let them go all the way to per-record timestamps (by
making the interval configurable and they can set it really small/to 0)? It
seems to me that some people might be willing to pay that cost for the
application-side simplicity if they want to consume from specific
timestamps. With the proposal as is, seeking to a timestamp still requires
client-side logic to filter out messages that might have earlier timestamps
due to the granularity chosen. The obvious tradeoff here is yet another
config option -- I'm loath to add yet more configs, although this is one
that we can choose a reasonable default for (like 1 minute) and people can
easily change with no impact if they need to adjust it.

3. Why not respect the timestamp passed in as long as it's not some
sentinel value that indicates the broker should fill it in? When you do
know they will be ordered, as they should be with mirror maker (which is
specifically mentioned), this seems really useful. (More about this in
questions below...)

4. I think one obvious answer to (3) is that accepting client's timestamps
could break seeking-by-timestamp since they may not be properly ordered.
However, I think this can break anyway under normal operations -- any
imperfections in clock synchronization could result in older timestamps
being applied to new messages on a new leader broker compared messages
stored previously by the old leader. I think it's probably too common to
just think that NTP will take care of it and then run into weird bugs
because NTP isn't always perfect, sometimes does some unexpected things,
and, of course, does what you tell it to and so is subject to user error.

It would be reasonable to argue that the leader change issue is much less
likely of an issue than if you respect timestamps from producers, where, if
applications actually filled it in, you'd receive a jumbled mess of
timestamps and trying to do the binary search in the index wouldn't
necessarily give you correct results. However, a) we could allow clients to
fill that info in but discourage it where it might cause issues (i.e.
normal applications) and it seems like a significant win for mirrormaker.

I actually think not accepting timestamps is probably the better choice but
a) it seems awkward in the protocol spec because we have to include the
field in the produce requests since we don't want to have to fully decode
them on the broker and b) losing that info during mirroring seems like it
breaks the goal of fixing log retention (at least for mirrors) as well as
the goal of improving searching by timestamp (at least for mirrors).

5. You never actually specified the granularity (or format, but I assume
unix epoch) of the timestamp. Milliseconds? Microseconds? Nanoseconds? This
definitely needs to eventually make it into the protocol docs.

6. Re: the rejected alternative. Are there any other options in changing
the config format that might make it a bit lighter weight? For example, do
we need a full int64? Could we do something relative instead that wouldn't
require as many bytes? Without (5) being specified, it's actually difficult
to evaluate some of these options.

7. Any plan to expose this in the client APIs? This is related to (4). If
they are not exposed anywhere in the API as being associated with messages,
then we can reasonably treat them as an internal implementation detail and
be very clear about what looking up an offset for a timestamp means. If
they are exposed, then they're more like message metadata that applications
are probably going to want preserved, and thus will want the broker to
respect the timestamp. For example, if I saw that consumers could get the
timestamp, I'd want to be able to assign it at the producer so that even if
I have significant batching I still get an accurate timestamp -- basically
I might replace some cases where I use a timestamp embedded in the message
with the timestamp provided by Kafka. (That said, we should consider the
impact that including it in the protocol can have; non-Java clients are
likely to expose it if it is available, whether it's actually a good idea
to or not.)

-Ewen


On Thu, Sep 3, 2015 at 3:47 PM, Jiangjie Qin 
wrote:

> Hi Guozhang,
>
> I checked the code again. Actually CRC check probably won't fail. The newly
> added timestamp field might be treated as keyLe

Re: Review Request 36858: Patch for KAFKA-2120

2015-09-03 Thread Mayuresh Gharat

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/36858/#review96965
---



clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java (line 141)


discussed offline



clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
(line 245)


As per the KIP, the replicationTimeout will be replaced by requestTimeout. 

"request.timeout.ms - the maximum time to wait for the response of a 
message AFTER the batch is ready,  including: actual network RTT, server 
replication time"


- Mayuresh Gharat


On Sept. 3, 2015, 10:12 p.m., Mayuresh Gharat wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/36858/
> ---
> 
> (Updated Sept. 3, 2015, 10:12 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-2120
> https://issues.apache.org/jira/browse/KAFKA-2120
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> Solved compile error
> 
> 
> Addressed Jason's comments for Kip-19
> 
> 
> Addressed Jun's comments
> 
> 
> Addressed Jason's comments about the default values for requestTimeout
> 
> 
> checkpoint
> 
> 
> Addressed Joel's concerns. Also tried to include Jun's feedback.
> 
> 
> Diffs
> -
> 
>   clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
> dc8f0f115bcda893c95d17c0a57be8d14518d034 
>   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
> 7d24c6f5dd2b63b96584f3aa8922a1d048dc1ae4 
>   clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java 
> 15d00d4e484bb5d51a9ae6857ed6e024a2cc1820 
>   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
> 7ab2503794ff3aab39df881bd9fbae6547827d3b 
>   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
> b31f7f1fbf93d29268b93811c9aad3e3c18e5312 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
> b9a2d4e2bc565f0ee72b27791afe5c894af262f1 
>   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
> 938981c23ec16dfaf81d1e647929a59e1572f40f 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
>  9517d9d0cd480d5ba1d12f1fde7963e60528d2f8 
>   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
> 804d569498396d431880641041fc9292076452cb 
>   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
> 06f00a99a73a288df9afa8c1d4abe3580fa968a6 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
>  4cb1e50d6c4ed55241aeaef1d3af09def5274103 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
>  a152bd7697dca55609a9ec4cfe0a82c10595fbc3 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
>  06182db1c3a5da85648199b4c0c98b80ea7c6c0c 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
> d2e64f7cd8bf56e433a210905b2874f71eee9ea0 
>   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
> f49d54cbc1915ac686ff70ac657f08e4c96489c1 
>   clients/src/test/java/org/apache/kafka/clients/MockClient.java 
> 9133d85342b11ba2c9888d4d2804d181831e7a8e 
>   clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java 
> 43238ceaad0322e39802b615bb805b895336a009 
>   
> clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java
>  2c693824fa53db1e38766b8c66a0ef42ef9d0f3a 
>   
> clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
>  5b2e4ffaeab7127648db608c179703b27b577414 
>   
> clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
>  aa44991777a855f4b7f4f7bf17107c69393ff8ff 
>   clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java 
> df1205c935bee9a30a50816dbade64d6014b1ef2 
>   clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 
> 3a684d98b05cadfb25c6f7f9a038ef1f6697edbf 
>   core/src/main/scala/kafka/tools/ProducerPerformance.scala 
> 0335cc64013ffe2cdf1c4879e86e11ec8c526712 
>   core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala 
> 1198df02ddd7727269e84a751ba99520f6d5584a 
>   core/src/test/scala/unit/kafka/utils/TestUtils.scala 
> 09b8444c2add87f0f70dbb182e892977a6b5c243 
> 
> Diff: https://reviews.apache.org/r/36858/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Mayuresh Gharat
> 
>



Huge Kafka Message size ( 386073344 ) in response

2015-09-03 Thread Qi Xu
Hi all,
I'm using the Kafka.Net library for implementing the Kafka Producer.
One issue I find out is that sometimes it reads the response from kafka
server, which indicates a huge message size 386073344. Apparently something
must be wrong.
But I'm not sure if it's a special flag that Kafka.net fails to handle or
it's a bug in Kafka Server side. Did you see this before?

Thanks,
Qi


[jira] [Commented] (KAFKA-2143) Replicas get ahead of leader and fail

2015-09-03 Thread Jiangjie Qin (JIRA)

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

Jiangjie Qin commented on KAFKA-2143:
-

[~junrao], you are right. I realized there should be unclean leader election in 
this case. I'll check the code further to see if there is any finding for 
KAFKA-2477.

> Replicas get ahead of leader and fail
> -
>
> Key: KAFKA-2143
> URL: https://issues.apache.org/jira/browse/KAFKA-2143
> Project: Kafka
>  Issue Type: Bug
>  Components: replication
>Affects Versions: 0.8.2.1
>Reporter: Evan Huus
>Assignee: Jiangjie Qin
>
> On a cluster of 6 nodes, we recently saw a case where a single 
> under-replicated partition suddenly appeared, replication lag spiked, and 
> network IO spiked. The cluster appeared to recover eventually on its own,
> Looking at the logs, the thing which failed was partition 7 of the topic 
> {{background_queue}}. It had an ISR of 1,4,3 and its leader at the time was 
> 3. Here are the interesting log lines:
> On node 3 (the leader):
> {noformat}
> [2015-04-23 16:50:05,879] ERROR [Replica Manager on Broker 3]: Error when 
> processing fetch request for partition [background_queue,7] offset 3722949957 
> from follower with correlation id 148185816. Possible cause: Request for 
> offset 3722949957 but we only have log segments in the range 3648049863 to 
> 3722949955. (kafka.server.ReplicaManager)
> [2015-04-23 16:50:05,879] ERROR [Replica Manager on Broker 3]: Error when 
> processing fetch request for partition [background_queue,7] offset 3722949957 
> from follower with correlation id 156007054. Possible cause: Request for 
> offset 3722949957 but we only have log segments in the range 3648049863 to 
> 3722949955. (kafka.server.ReplicaManager)
> [2015-04-23 16:50:13,960] INFO Partition [background_queue,7] on broker 3: 
> Shrinking ISR for partition [background_queue,7] from 1,4,3 to 3 
> (kafka.cluster.Partition)
> {noformat}
> Note that both replicas suddenly asked for an offset *ahead* of the available 
> offsets.
> And on nodes 1 and 4 (the replicas) many occurrences of the following:
> {noformat}
> [2015-04-23 16:50:05,935] INFO Scheduling log segment 3648049863 for log 
> background_queue-7 for deletion. (kafka.log.Log) (edited)
> {noformat}
> Based on my reading, this looks like the replicas somehow got *ahead* of the 
> leader, asked for an invalid offset, got confused, and re-replicated the 
> entire topic from scratch to recover (this matches our network graphs, which 
> show 3 sending a bunch of data to 1 and 4).
> Taking a stab in the dark at the cause, there appears to be a race condition 
> where replicas can receive a new offset before the leader has committed it 
> and is ready to replicate?



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


[jira] [Commented] (KAFKA-2143) Replicas get ahead of leader and fail

2015-09-03 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-2143:


[~becket_qin], since before step (3), both A and B are in ISR, the last 
committed offset in A can't be larger than 3000. So, in step (3), if A becomes 
a follower, it has to first truncate its log to last committed offset before 
fetching. So, at that point, A's fetch offset can't be larger than 3000 and 
therefore won't be out of range.

The following is a alternative scenario that can cause this.

1) Broker A (leader) receives messages to 5000
2) Broker B (follower) receives messages to 3000 (it is still in ISR because of 
replica.lag.max.messages)
3) For some reason, B is dropped out of ISR.
4) Broker A (the only one in ISR) commits messages to 5000.
5) For some reason, Broker A is considered dead and Broker B is live.
6) Broker B is selected as the new leader (unclean leader election) and is the 
only one in ISR.
7) Broker A is considered live again and starts fetching from 5000 (last 
committed offset) and gets OffsetOutOfRangeException.
8) In the mean time, B receives more messages to offset 6000.
9) Broker A tries to handle OffsetOutOfRangeException and finds out leader B's 
log end offset is now larger than its log end offset and truncates all its log.

Your patch reduces the amount of the data that Broker A needs to replicate in 
step 9, which is probably fine. However, we probably should first verify if 
this is indeed what's happening since it seems that it should happen rarely. 
Also, KAFKA-2477 reports a similar issue w/o any leadership change. So, may be 
there is something else that can cause this.

> Replicas get ahead of leader and fail
> -
>
> Key: KAFKA-2143
> URL: https://issues.apache.org/jira/browse/KAFKA-2143
> Project: Kafka
>  Issue Type: Bug
>  Components: replication
>Affects Versions: 0.8.2.1
>Reporter: Evan Huus
>Assignee: Jiangjie Qin
>
> On a cluster of 6 nodes, we recently saw a case where a single 
> under-replicated partition suddenly appeared, replication lag spiked, and 
> network IO spiked. The cluster appeared to recover eventually on its own,
> Looking at the logs, the thing which failed was partition 7 of the topic 
> {{background_queue}}. It had an ISR of 1,4,3 and its leader at the time was 
> 3. Here are the interesting log lines:
> On node 3 (the leader):
> {noformat}
> [2015-04-23 16:50:05,879] ERROR [Replica Manager on Broker 3]: Error when 
> processing fetch request for partition [background_queue,7] offset 3722949957 
> from follower with correlation id 148185816. Possible cause: Request for 
> offset 3722949957 but we only have log segments in the range 3648049863 to 
> 3722949955. (kafka.server.ReplicaManager)
> [2015-04-23 16:50:05,879] ERROR [Replica Manager on Broker 3]: Error when 
> processing fetch request for partition [background_queue,7] offset 3722949957 
> from follower with correlation id 156007054. Possible cause: Request for 
> offset 3722949957 but we only have log segments in the range 3648049863 to 
> 3722949955. (kafka.server.ReplicaManager)
> [2015-04-23 16:50:13,960] INFO Partition [background_queue,7] on broker 3: 
> Shrinking ISR for partition [background_queue,7] from 1,4,3 to 3 
> (kafka.cluster.Partition)
> {noformat}
> Note that both replicas suddenly asked for an offset *ahead* of the available 
> offsets.
> And on nodes 1 and 4 (the replicas) many occurrences of the following:
> {noformat}
> [2015-04-23 16:50:05,935] INFO Scheduling log segment 3648049863 for log 
> background_queue-7 for deletion. (kafka.log.Log) (edited)
> {noformat}
> Based on my reading, this looks like the replicas somehow got *ahead* of the 
> leader, asked for an invalid offset, got confused, and re-replicated the 
> entire topic from scratch to recover (this matches our network graphs, which 
> show 3 sending a bunch of data to 1 and 4).
> Taking a stab in the dark at the cause, there appears to be a race condition 
> where replicas can receive a new offset before the leader has committed it 
> and is ready to replicate?



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


[jira] [Commented] (KAFKA-2477) Replicas spuriously deleting all segments in partition

2015-09-03 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-2477:


Could you then try the following? In the above situation, go to broker 0's log 
dir for partition [log.event,3]. Get the name of the last log segment (the .log 
file). Then run the following
bin/kafka-run-class.sh kafka.tools.DumpLogSegments [logsegmentname]

This will print out the offset of each message. In the normal case, those 
offsets should be monotonically increasing. Could you check if there is any out 
of sequence offsets in the output especially close to 10349592109?


> Replicas spuriously deleting all segments in partition
> --
>
> Key: KAFKA-2477
> URL: https://issues.apache.org/jira/browse/KAFKA-2477
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: HÃ¥kon Hitland
> Attachments: kafka_log.txt
>
>
> We're seeing some strange behaviour in brokers: a replica will sometimes 
> schedule all segments in a partition for deletion, and then immediately start 
> replicating them back, triggering our check for under-replicating topics.
> This happens on average a couple of times a week, for different brokers and 
> topics.
> We have per-topic retention.ms and retention.bytes configuration, the topics 
> where we've seen this happen are hitting the size limit.



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


[jira] [Commented] (KAFKA-2516) Rename o.a.k.client.tools to o.a.k.tools

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-2516:


Good point.

> Rename o.a.k.client.tools to o.a.k.tools
> 
>
> Key: KAFKA-2516
> URL: https://issues.apache.org/jira/browse/KAFKA-2516
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Currently our new performance tools are in o.a.k.client.tools but packaged in 
> kafka-tools not kafka-clients. This is a bit confusing.
> Since they deserve their own jar (you don't want our client tools packaged in 
> your app), lets give them a separate package and call it o.a.k.tools.



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


[jira] [Created] (KAFKA-2516) Rename o.a.k.client.tools to o.a.k.tools

2015-09-03 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-2516:
---

 Summary: Rename o.a.k.client.tools to o.a.k.tools
 Key: KAFKA-2516
 URL: https://issues.apache.org/jira/browse/KAFKA-2516
 Project: Kafka
  Issue Type: Bug
Reporter: Gwen Shapira
Priority: Blocker
 Fix For: 0.8.3


Currently our new performance tools are in o.a.k.client.tools but packaged in 
kafka-tools not kafka-clients. This is a bit confusing.

Since they deserve their own jar (you don't want our client tools packaged in 
your app), lets give them a separate package and call it o.a.k.tools.



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


Build failed in Jenkins: Kafka-trunk #615

2015-09-03 Thread Apache Jenkins Server
See 

Changes:

[cshapi] MINOR: Fix trace/debug logs in RequestChannel

--
[...truncated 3203 lines...]
kafka.log.LogManagerTest > testTimeBasedFlush PASSED

kafka.log.LogManagerTest > testCreateLog PASSED

kafka.log.LogManagerTest > testRecoveryDirectoryMappingWithTrailingSlash PASSED

kafka.admin.DeleteTopicTest > testRecreateTopicAfterDeletion PASSED

kafka.api.ConsumerTest > testSeek PASSED

kafka.producer.ProducerTest > testUpdateBrokerPartitionInfo PASSED

kafka.api.ProducerSendTest > testCloseWithZeroTimeoutFromCallerThread PASSED

kafka.api.ProducerBounceTest > testBrokerFailure PASSED

kafka.log.LogTest > testParseTopicPartitionNameForMissingTopic PASSED

kafka.log.LogTest > testIndexRebuild PASSED

kafka.log.LogTest > testLogRolls PASSED

kafka.log.LogTest > testMessageSizeCheck PASSED

kafka.log.LogTest > testAsyncDelete PASSED

kafka.log.LogTest > testReadOutOfRange PASSED

kafka.log.LogTest > testReadAtLogGap PASSED

kafka.log.LogTest > testTimeBasedLogRoll PASSED

kafka.log.LogTest > testLoadEmptyLog PASSED

kafka.log.LogTest > testMessageSetSizeCheck PASSED

kafka.log.LogTest > testIndexResizingAtTruncation PASSED

kafka.log.LogTest > testCompactedTopicConstraints PASSED

kafka.log.LogTest > testThatGarbageCollectingSegmentsDoesntChangeOffset PASSED

kafka.log.LogTest > testAppendAndReadWithSequentialOffsets PASSED

kafka.log.LogTest > testParseTopicPartitionNameForNull PASSED

kafka.log.LogTest > testAppendAndReadWithNonSequentialOffsets PASSED

kafka.log.LogTest > testParseTopicPartitionNameForMissingSeparator PASSED

kafka.log.LogTest > testCorruptIndexRebuild PASSED

kafka.log.LogTest > testBogusIndexSegmentsAreRemoved PASSED

kafka.log.LogTest > testCompressedMessages PASSED

kafka.log.LogTest > testAppendMessageWithNullPayload PASSED

kafka.log.LogTest > testCorruptLog PASSED

kafka.log.LogTest > testLogRecoversToCorrectOffset PASSED

kafka.log.LogTest > testReopenThenTruncate PASSED

kafka.log.LogTest > testParseTopicPartitionNameForMissingPartition PASSED

kafka.log.LogTest > testParseTopicPartitionNameForEmptyName PASSED

kafka.log.LogTest > testOpenDeletesObsoleteFiles PASSED

kafka.log.LogTest > testSizeBasedLogRoll PASSED

kafka.log.LogTest > testTimeBasedLogRollJitter PASSED

kafka.log.LogTest > testParseTopicPartitionName PASSED

kafka.log.LogTest > testTruncateTo PASSED

kafka.log.LogTest > testCleanShutdownFile PASSED

kafka.integration.UncleanLeaderElectionTest > testUncleanLeaderElectionEnabled 
PASSED

kafka.admin.DeleteTopicTest > testAddPartitionDuringDeleteTopic PASSED

kafka.api.ConsumerTest > testPositionAndCommit PASSED

kafka.producer.ProducerTest > testSendWithDeadBroker PASSED

kafka.admin.DeleteTopicTest > testDeleteTopicWithAllAliveReplicas PASSED

kafka.api.ProducerFailureHandlingTest > testCannotSendToInternalTopic PASSED

kafka.api.ConsumerTest > testUnsubscribeTopic PASSED

kafka.admin.DeleteTopicTest > testDeleteTopicDuringAddPartition PASSED

kafka.api.ProducerFailureHandlingTest > testTooLargeRecordWithAckOne PASSED

kafka.api.ProducerSendTest > testCloseWithZeroTimeoutFromSenderThread PASSED

kafka.log.CleanerTest > testBuildOffsetMap PASSED

kafka.log.CleanerTest > testSegmentGrouping PASSED

kafka.log.CleanerTest > testCleanSegmentsWithAbort PASSED

kafka.log.CleanerTest > testSegmentGroupingWithSparseOffsets PASSED

kafka.log.CleanerTest > testRecoveryAfterCrash PASSED

kafka.log.CleanerTest > testCleaningWithDeletes PASSED

kafka.log.CleanerTest > testCleanSegments PASSED

kafka.log.CleanerTest > testCleaningWithUnkeyedMessages PASSED

kafka.api.ConsumerTest > testListTopics PASSED

kafka.api.ProducerFailureHandlingTest > testWrongBrokerList PASSED

kafka.api.ConsumerTest > testExpandingTopicSubscriptions PASSED

kafka.api.ProducerFailureHandlingTest > testNotEnoughReplicas PASSED

kafka.integration.UncleanLeaderElectionTest > 
testCleanLeaderElectionDisabledByTopicOverride PASSED

kafka.api.ConsumerTest > testGroupConsumption PASSED

kafka.api.ConsumerBounceTest > testSeekAndCommitWithBrokerFailures PASSED

kafka.api.ConsumerTest > testPartitionsFor PASSED

kafka.api.ProducerFailureHandlingTest > testNoResponse PASSED

kafka.api.ProducerFailureHandlingTest > testNonExistentTopic PASSED

kafka.api.ProducerFailureHandlingTest > testInvalidPartition PASSED

kafka.integration.UncleanLeaderElectionTest > testUncleanLeaderElectionDisabled 
PASSED

kafka.integration.UncleanLeaderElectionTest > 
testUncleanLeaderElectionInvalidTopicOverride PASSED

kafka.api.ProducerFailureHandlingTest > testSendAfterClosed PASSED

kafka.api.ProducerFailureHandlingTest > testTooLargeRecordWithAckZero PASSED

kafka.api.ConsumerTest > testSimpleConsumption PASSED

kafka.integration.UncleanLeaderElectionTest > 
testUncleanLeaderElectionEnabledByTopicOverride PASSED

kafka.api.ConsumerTest > testPartitionPauseAndResume PASSED

kafka.api.ConsumerBounceTest > testConsumptionWithBrokerFa

Re: [DISCUSS] KIP-31 - Message format change proposal

2015-09-03 Thread Jiangjie Qin
Hi Guozhang,

I checked the code again. Actually CRC check probably won't fail. The newly
added timestamp field might be treated as keyLength instead, so we are
likely to receive an IllegalArgumentException when try to read the key.
I'll update the KIP.

Thanks,

Jiangjie (Becket) Qin

On Thu, Sep 3, 2015 at 12:48 PM, Jiangjie Qin  wrote:

> Hi, Guozhang,
>
> Thanks for reading the KIP. By "old consumer", I meant the
> ZookeeperConsumerConnector in trunk now, i.e. without this bug fixed. If we
> fix the ZookeeperConsumerConnector then it will throw exception complaining
> about the unsupported version when it sees message format V1. What I was
> trying to say is that if we have some ZookeeperConsumerConnector running
> without the fix, the consumer will complain about CRC mismatch instead of
> unsupported version.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Thu, Sep 3, 2015 at 12:15 PM, Guozhang Wang  wrote:
>
>> Thanks for the write-up Jiangjie.
>>
>> One comment about migration plan: "For old consumers, if they see the new
>> protocol the CRC check will fail"..
>>
>> Do you mean this bug in the old consumer cannot be fixed in a
>> backward-compatible way?
>>
>> Guozhang
>>
>>
>> On Thu, Sep 3, 2015 at 8:35 AM, Jiangjie Qin 
>> wrote:
>>
>> > Hi,
>> >
>> > We just created KIP-31 to propose a message format change in Kafka.
>> >
>> >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-31+-+Message+format+change+proposal
>> >
>> > As a summary, the motivations are:
>> > 1. Avoid server side message re-compression
>> > 2. Honor time-based log roll and retention
>> > 3. Enable offset search by timestamp at a finer granularity.
>> >
>> > Feedback and comments are welcome!
>> >
>> > Thanks,
>> >
>> > Jiangjie (Becket) Qin
>> >
>>
>>
>>
>> --
>> -- Guozhang
>>
>
>


[jira] [Created] (KAFKA-2515) handle oversized messages properly in new consumer

2015-09-03 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-2515:
--

 Summary: handle oversized messages properly in new consumer
 Key: KAFKA-2515
 URL: https://issues.apache.org/jira/browse/KAFKA-2515
 Project: Kafka
  Issue Type: Sub-task
  Components: clients
Reporter: Jun Rao
 Fix For: 0.8.3


When there is an oversized message in the broker, it seems that the new 
consumer just silently gets stuck. We should at least log an error when this 
happens.



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


[jira] [Created] (KAFKA-2514) change default JVM options in kafka-run-class.sh

2015-09-03 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-2514:
--

 Summary: change default JVM options in kafka-run-class.sh
 Key: KAFKA-2514
 URL: https://issues.apache.org/jira/browse/KAFKA-2514
 Project: Kafka
  Issue Type: Improvement
  Components: config
Reporter: Jun Rao
Priority: Blocker
 Fix For: 0.8.3


Since we have stopped supporting java 6, we should improve the default jvm 
options. For example, we should probably change the following to G1 collectors.

if [ -z "$KAFKA_JVM_PERFORMANCE_OPTS" ]; then
  KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseParNewGC -XX:+UseConcMarkSweepGC 
-XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark 
-XX:+DisableExplicitGC -Djava.awt.headless=true"
fi




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


[jira] [Commented] (KAFKA-2513) Checkstyle is not executed until gradle's tests phase

2015-09-03 Thread Ashish K Singh (JIRA)

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

Ashish K Singh commented on KAFKA-2513:
---

I think if one wants to just compile, they can just choose to do {{compile}}. 
However, I think there is some value to have checkstyle execute as part of 
{{jar}}, but just my opinion.

> Checkstyle is not executed until gradle's tests phase
> -
>
> Key: KAFKA-2513
> URL: https://issues.apache.org/jira/browse/KAFKA-2513
> Project: Kafka
>  Issue Type: Bug
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
>Priority: Minor
>
> Checkstyle is added as a dependency to test and until someone runs test they 
> won't capture checkstyle issues. To me code style is more suited along with 
> compile than tests. This was also brought up on KAFKA-1893. May be we should 
> make checkstyle a dependency of jar, instead of test.



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


[jira] [Updated] (KAFKA-2512) Add version and magic byte verification on broker and clients.

2015-09-03 Thread Jiangjie Qin (JIRA)

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

Jiangjie Qin updated KAFKA-2512:

Description: Both new and old consumer do not verify the magic byte of a 
message when it consumes a message. This is needed for protocol upgrade so the 
consumer can provide correct information to user. Same check is needed on 
broker side and well.  (was: Both new and old consumer do not verify the magic 
byte of a message when it consumes a message. This is needed for protocol 
upgrade so the consumer can provide correct information to user.)

> Add version and magic byte verification on broker and clients.
> --
>
> Key: KAFKA-2512
> URL: https://issues.apache.org/jira/browse/KAFKA-2512
> Project: Kafka
>  Issue Type: Bug
>Reporter: Jiangjie Qin
>Assignee: Jiangjie Qin
>
> Both new and old consumer do not verify the magic byte of a message when it 
> consumes a message. This is needed for protocol upgrade so the consumer can 
> provide correct information to user. Same check is needed on broker side and 
> well.



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


[jira] [Commented] (KAFKA-2120) Add a request timeout to NetworkClient

2015-09-03 Thread Mayuresh Gharat (JIRA)

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

Mayuresh Gharat commented on KAFKA-2120:


Updated reviewboard https://reviews.apache.org/r/36858/diff/
 against branch origin/trunk

> Add a request timeout to NetworkClient
> --
>
> Key: KAFKA-2120
> URL: https://issues.apache.org/jira/browse/KAFKA-2120
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Jiangjie Qin
>Assignee: Mayuresh Gharat
>Priority: Blocker
> Fix For: 0.8.3
>
> Attachments: KAFKA-2120.patch, KAFKA-2120_2015-07-27_15:31:19.patch, 
> KAFKA-2120_2015-07-29_15:57:02.patch, KAFKA-2120_2015-08-10_19:55:18.patch, 
> KAFKA-2120_2015-08-12_10:59:09.patch, KAFKA-2120_2015-09-03_15:12:02.patch
>
>
> Currently NetworkClient does not have a timeout setting for requests. So if 
> no response is received for a request due to reasons such as broker is down, 
> the request will never be completed.
> Request timeout will also be used as implicit timeout for some methods such 
> as KafkaProducer.flush() and kafkaProducer.close().
> KIP-19 is created for this public interface change.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-19+-+Add+a+request+timeout+to+NetworkClient



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


Re: Review Request 36858: Patch for KAFKA-2120

2015-09-03 Thread Mayuresh Gharat

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/36858/
---

(Updated Sept. 3, 2015, 10:12 p.m.)


Review request for kafka.


Bugs: KAFKA-2120
https://issues.apache.org/jira/browse/KAFKA-2120


Repository: kafka


Description (updated)
---

Solved compile error


Addressed Jason's comments for Kip-19


Addressed Jun's comments


Addressed Jason's comments about the default values for requestTimeout


checkpoint


Addressed Joel's concerns. Also tried to include Jun's feedback.


Diffs (updated)
-

  clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
dc8f0f115bcda893c95d17c0a57be8d14518d034 
  clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
7d24c6f5dd2b63b96584f3aa8922a1d048dc1ae4 
  clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java 
15d00d4e484bb5d51a9ae6857ed6e024a2cc1820 
  clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
7ab2503794ff3aab39df881bd9fbae6547827d3b 
  clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
b31f7f1fbf93d29268b93811c9aad3e3c18e5312 
  clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
b9a2d4e2bc565f0ee72b27791afe5c894af262f1 
  clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
938981c23ec16dfaf81d1e647929a59e1572f40f 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
 9517d9d0cd480d5ba1d12f1fde7963e60528d2f8 
  clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
804d569498396d431880641041fc9292076452cb 
  clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
06f00a99a73a288df9afa8c1d4abe3580fa968a6 
  
clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
 4cb1e50d6c4ed55241aeaef1d3af09def5274103 
  
clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
 a152bd7697dca55609a9ec4cfe0a82c10595fbc3 
  
clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
 06182db1c3a5da85648199b4c0c98b80ea7c6c0c 
  clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
d2e64f7cd8bf56e433a210905b2874f71eee9ea0 
  clients/src/main/java/org/apache/kafka/common/network/Selector.java 
f49d54cbc1915ac686ff70ac657f08e4c96489c1 
  clients/src/test/java/org/apache/kafka/clients/MockClient.java 
9133d85342b11ba2c9888d4d2804d181831e7a8e 
  clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java 
43238ceaad0322e39802b615bb805b895336a009 
  
clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java
 2c693824fa53db1e38766b8c66a0ef42ef9d0f3a 
  
clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
 5b2e4ffaeab7127648db608c179703b27b577414 
  
clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
 aa44991777a855f4b7f4f7bf17107c69393ff8ff 
  clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java 
df1205c935bee9a30a50816dbade64d6014b1ef2 
  clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 
3a684d98b05cadfb25c6f7f9a038ef1f6697edbf 
  core/src/main/scala/kafka/tools/ProducerPerformance.scala 
0335cc64013ffe2cdf1c4879e86e11ec8c526712 
  core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala 
1198df02ddd7727269e84a751ba99520f6d5584a 
  core/src/test/scala/unit/kafka/utils/TestUtils.scala 
09b8444c2add87f0f70dbb182e892977a6b5c243 

Diff: https://reviews.apache.org/r/36858/diff/


Testing
---


Thanks,

Mayuresh Gharat



[jira] [Updated] (KAFKA-2120) Add a request timeout to NetworkClient

2015-09-03 Thread Mayuresh Gharat (JIRA)

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

Mayuresh Gharat updated KAFKA-2120:
---
Attachment: KAFKA-2120_2015-09-03_15:12:02.patch

> Add a request timeout to NetworkClient
> --
>
> Key: KAFKA-2120
> URL: https://issues.apache.org/jira/browse/KAFKA-2120
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Jiangjie Qin
>Assignee: Mayuresh Gharat
>Priority: Blocker
> Fix For: 0.8.3
>
> Attachments: KAFKA-2120.patch, KAFKA-2120_2015-07-27_15:31:19.patch, 
> KAFKA-2120_2015-07-29_15:57:02.patch, KAFKA-2120_2015-08-10_19:55:18.patch, 
> KAFKA-2120_2015-08-12_10:59:09.patch, KAFKA-2120_2015-09-03_15:12:02.patch
>
>
> Currently NetworkClient does not have a timeout setting for requests. So if 
> no response is received for a request due to reasons such as broker is down, 
> the request will never be completed.
> Request timeout will also be used as implicit timeout for some methods such 
> as KafkaProducer.flush() and kafkaProducer.close().
> KIP-19 is created for this public interface change.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-19+-+Add+a+request+timeout+to+NetworkClient



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


[jira] [Updated] (KAFKA-2120) Add a request timeout to NetworkClient

2015-09-03 Thread Mayuresh Gharat (JIRA)

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

Mayuresh Gharat updated KAFKA-2120:
---
Status: Patch Available  (was: In Progress)

> Add a request timeout to NetworkClient
> --
>
> Key: KAFKA-2120
> URL: https://issues.apache.org/jira/browse/KAFKA-2120
> Project: Kafka
>  Issue Type: New Feature
>Reporter: Jiangjie Qin
>Assignee: Mayuresh Gharat
>Priority: Blocker
> Fix For: 0.8.3
>
> Attachments: KAFKA-2120.patch, KAFKA-2120_2015-07-27_15:31:19.patch, 
> KAFKA-2120_2015-07-29_15:57:02.patch, KAFKA-2120_2015-08-10_19:55:18.patch, 
> KAFKA-2120_2015-08-12_10:59:09.patch, KAFKA-2120_2015-09-03_15:12:02.patch
>
>
> Currently NetworkClient does not have a timeout setting for requests. So if 
> no response is received for a request due to reasons such as broker is down, 
> the request will never be completed.
> Request timeout will also be used as implicit timeout for some methods such 
> as KafkaProducer.flush() and kafkaProducer.close().
> KIP-19 is created for this public interface change.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-19+-+Add+a+request+timeout+to+NetworkClient



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


[jira] [Updated] (KAFKA-2512) Add version and magic byte verification on broker and clients.

2015-09-03 Thread Jiangjie Qin (JIRA)

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

Jiangjie Qin updated KAFKA-2512:

Summary: Add version and magic byte verification on broker and clients.  
(was: KafkaConsumer and ZookeeperConsumerConnector should verify the magic 
byte.)

> Add version and magic byte verification on broker and clients.
> --
>
> Key: KAFKA-2512
> URL: https://issues.apache.org/jira/browse/KAFKA-2512
> Project: Kafka
>  Issue Type: Bug
>Reporter: Jiangjie Qin
>Assignee: Jiangjie Qin
>
> Both new and old consumer do not verify the magic byte of a message when it 
> consumes a message. This is needed for protocol upgrade so the consumer can 
> provide correct information to user.



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


[jira] [Commented] (KAFKA-2513) Checkstyle is not executed until gradle's tests phase

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-2513:


[~ewencp], the way this should work IMO is that there's a separate command that 
depends on the compile task and people can choose what they want. Are you 
saying that you would not want it to run for the `jar` command either?

> Checkstyle is not executed until gradle's tests phase
> -
>
> Key: KAFKA-2513
> URL: https://issues.apache.org/jira/browse/KAFKA-2513
> Project: Kafka
>  Issue Type: Bug
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
>Priority: Minor
>
> Checkstyle is added as a dependency to test and until someone runs test they 
> won't capture checkstyle issues. To me code style is more suited along with 
> compile than tests. This was also brought up on KAFKA-1893. May be we should 
> make checkstyle a dependency of jar, instead of test.



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


[jira] [Commented] (KAFKA-2513) Checkstyle is not executed until gradle's tests phase

2015-09-03 Thread Ewen Cheslack-Postava (JIRA)

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

Ewen Cheslack-Postava commented on KAFKA-2513:
--

I considered the same thing with the rat checks. It turns out to be really 
annoying to have those checks on for compile because it enforces rules you may 
not want to enforce during development. I really don't want to have to clean up 
the formatting of my ad hoc debugging code that I know I'm going to delete 
anyway...

> Checkstyle is not executed until gradle's tests phase
> -
>
> Key: KAFKA-2513
> URL: https://issues.apache.org/jira/browse/KAFKA-2513
> Project: Kafka
>  Issue Type: Bug
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
>Priority: Minor
>
> Checkstyle is added as a dependency to test and until someone runs test they 
> won't capture checkstyle issues. To me code style is more suited along with 
> compile than tests. This was also brought up on KAFKA-1893. May be we should 
> make checkstyle a dependency of jar, instead of test.



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


[jira] [Commented] (KAFKA-2397) leave group request

2015-09-03 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-2397:


[~becket_qin] I'm not sure anyone was suggesting replacing the session timeout 
if TCP disconnect was used to signal group departure. I think you need session 
timeout regardless of whether we have an explicit leave group request or we use 
the TCP disconnect. I also feel a little concern about #3, but I don't actually 
know of any cases where network issues will cause a disconnect. In general, my 
feeling is that the advantages of the TCP disconnect (in particular the ability 
to detect hard crashes more swiftly) are not worth the cost of exposing the 
lower level network layer in the consumer coordinator. At the moment, however, 
my main concern is more pragmatic: the window for a big change like that is 
starting to close.

> leave group request
> ---
>
> Key: KAFKA-2397
> URL: https://issues.apache.org/jira/browse/KAFKA-2397
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Onur Karaman
>Assignee: Onur Karaman
>Priority: Minor
> Fix For: 0.8.3
>
>
> Let's say every consumer in a group has session timeout s. Currently, if a 
> consumer leaves the group, the worst case time to stabilize the group is 2s 
> (s to detect the consumer failure + s for the rebalance window). If a 
> consumer instead can declare they are leaving the group, the worst case time 
> to stabilize the group would just be the s associated with the rebalance 
> window.
> This is a low priority optimization!



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


[jira] [Commented] (KAFKA-2397) leave group request

2015-09-03 Thread Jiangjie Qin (JIRA)

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

Jiangjie Qin commented on KAFKA-2397:
-

[~ewencp] [~hachikuji] Some thoughts on this. I agree with [~ewencp] that we 
should follow one protocol but not both. Personally I like explicit leave group 
request better.

The goals we want to achieve are:
1. When a consumer actually dies, we don't want to wait for too long before a 
rebalance.
2. When a consumer exits normally, we want to trigger a rebalance soon.
3. If there are some jitters or network issues, etc. We want to have some 
tolerance over that.

Using TCP connection to signify the liveliness will satisfy 2. 
For 1, if the TCP connection timeout is super long it won't work. That's why we 
introduced session timeout. 
For 3, using TCP connection to signify liveliness might cause problem.

Explicit leave group request is clear that a member will only be excluded from 
a group when it exit normally, or session is timeout. So all the three goals 
are met.

An important related scenario worth thinking about is bouncing a consumer. 
Without leave group request, it is possible to bounce a client without 
triggering rebalance as long as the consumer shuts down then come back before 
session timeout. If we send a leave group request explicitly, bouncing a 
consumer means there will be two rebalances (Which I think is the correct 
behavior). So making rebalance cheap and fast is very important.

> leave group request
> ---
>
> Key: KAFKA-2397
> URL: https://issues.apache.org/jira/browse/KAFKA-2397
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Onur Karaman
>Assignee: Onur Karaman
>Priority: Minor
> Fix For: 0.8.3
>
>
> Let's say every consumer in a group has session timeout s. Currently, if a 
> consumer leaves the group, the worst case time to stabilize the group is 2s 
> (s to detect the consumer failure + s for the rebalance window). If a 
> consumer instead can declare they are leaving the group, the worst case time 
> to stabilize the group would just be the s associated with the rebalance 
> window.
> This is a low priority optimization!



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


[jira] [Commented] (KAFKA-2513) Checkstyle is not executed until gradle's tests phase

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-2513:


I agree that it's more like compile than tests. Depending on jar by itself is 
not too good though because it won't be run when one runs the tests. I'd have 
to look at the build to have an actual proposal though.

> Checkstyle is not executed until gradle's tests phase
> -
>
> Key: KAFKA-2513
> URL: https://issues.apache.org/jira/browse/KAFKA-2513
> Project: Kafka
>  Issue Type: Bug
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
>Priority: Minor
>
> Checkstyle is added as a dependency to test and until someone runs test they 
> won't capture checkstyle issues. To me code style is more suited along with 
> compile than tests. This was also brought up on KAFKA-1893. May be we should 
> make checkstyle a dependency of jar, instead of test.



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


[jira] [Commented] (KAFKA-2513) Checkstyle is not executed until gradle's tests phase

2015-09-03 Thread Ashish K Singh (JIRA)

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

Ashish K Singh commented on KAFKA-2513:
---

[~ijuma], [~gwenshap] thoughts?

> Checkstyle is not executed until gradle's tests phase
> -
>
> Key: KAFKA-2513
> URL: https://issues.apache.org/jira/browse/KAFKA-2513
> Project: Kafka
>  Issue Type: Bug
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
>Priority: Minor
>
> Checkstyle is added as a dependency to test and until someone runs test they 
> won't capture checkstyle issues. To me code style is more suited along with 
> compile than tests. This was also brought up on KAFKA-1893. May be we should 
> make checkstyle a dependency of jar, instead of test.



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


[jira] [Created] (KAFKA-2513) Checkstyle is not executed until gradle's tests phase

2015-09-03 Thread Ashish K Singh (JIRA)
Ashish K Singh created KAFKA-2513:
-

 Summary: Checkstyle is not executed until gradle's tests phase
 Key: KAFKA-2513
 URL: https://issues.apache.org/jira/browse/KAFKA-2513
 Project: Kafka
  Issue Type: Bug
Reporter: Ashish K Singh
Assignee: Ashish K Singh
Priority: Minor


Checkstyle is added as a dependency to test and until someone runs test they 
won't capture checkstyle issues. To me code style is more suited along with 
compile than tests. This was also brought up on KAFKA-1893. May be we should 
make checkstyle a dependency of jar, instead of test.



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


[GitHub] kafka pull request: MINOR: Fix trace/debug logs in RequestChannel

2015-09-03 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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-2461) request logger no longer logs extra information in debug mode

2015-09-03 Thread Ashish K Singh (JIRA)

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

Ashish K Singh commented on KAFKA-2461:
---

[~gwenshap], [~ijuma], created https://github.com/apache/kafka/pull/192.

> request logger no longer logs extra information in debug mode
> -
>
> Key: KAFKA-2461
> URL: https://issues.apache.org/jira/browse/KAFKA-2461
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>Assignee: Ashish K Singh
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Currently request logging calls are identical for trace and debug:
> {code}
> if(requestLogger.isTraceEnabled)
> requestLogger.trace("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
> .format(requestDesc, connectionId, totalTime, 
> requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, 
> responseSendTime))
>   else if(requestLogger.isDebugEnabled)
> requestLogger.debug("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
>   .format(requestDesc, connectionId, totalTime, requestQueueTime, 
> apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime))
> {code}
> I think in the past (3 refactoring steps ago), we used to print more 
> information about specific topics and partitions in debug mode.



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


[GitHub] kafka pull request: MINOR: Fix trace/debug logs in RequestChannel

2015-09-03 Thread SinghAsDev
GitHub user SinghAsDev opened a pull request:

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

MINOR: Fix trace/debug logs in RequestChannel



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

$ git pull https://github.com/SinghAsDev/kafka KAFKA-2461-2

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

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


commit 0dd423549f4ea47729d7e5b42ee364b5020f9b2d
Author: Ashish Singh 
Date:   2015-09-03T20:20:25Z

MINOR: Fix trace/debug logs in RequestChannel




---
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-2512) KafkaConsumer and ZookeeperConsumerConnector should verify the magic byte.

2015-09-03 Thread Jiangjie Qin (JIRA)
Jiangjie Qin created KAFKA-2512:
---

 Summary: KafkaConsumer and ZookeeperConsumerConnector should 
verify the magic byte.
 Key: KAFKA-2512
 URL: https://issues.apache.org/jira/browse/KAFKA-2512
 Project: Kafka
  Issue Type: Bug
Reporter: Jiangjie Qin
Assignee: Jiangjie Qin


Both new and old consumer do not verify the magic byte of a message when it 
consumes a message. This is needed for protocol upgrade so the consumer can 
provide correct information to user.



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


Re: [DISCUSS] KIP-31 - Message format change proposal

2015-09-03 Thread Jiangjie Qin
Hi, Guozhang,

Thanks for reading the KIP. By "old consumer", I meant the
ZookeeperConsumerConnector in trunk now, i.e. without this bug fixed. If we
fix the ZookeeperConsumerConnector then it will throw exception complaining
about the unsupported version when it sees message format V1. What I was
trying to say is that if we have some ZookeeperConsumerConnector running
without the fix, the consumer will complain about CRC mismatch instead of
unsupported version.

Thanks,

Jiangjie (Becket) Qin

On Thu, Sep 3, 2015 at 12:15 PM, Guozhang Wang  wrote:

> Thanks for the write-up Jiangjie.
>
> One comment about migration plan: "For old consumers, if they see the new
> protocol the CRC check will fail"..
>
> Do you mean this bug in the old consumer cannot be fixed in a
> backward-compatible way?
>
> Guozhang
>
>
> On Thu, Sep 3, 2015 at 8:35 AM, Jiangjie Qin 
> wrote:
>
> > Hi,
> >
> > We just created KIP-31 to propose a message format change in Kafka.
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-31+-+Message+format+change+proposal
> >
> > As a summary, the motivations are:
> > 1. Avoid server side message re-compression
> > 2. Honor time-based log roll and retention
> > 3. Enable offset search by timestamp at a finer granularity.
> >
> > Feedback and comments are welcome!
> >
> > Thanks,
> >
> > Jiangjie (Becket) Qin
> >
>
>
>
> --
> -- Guozhang
>


Re: Review Request 36578: Patch for KAFKA-2338

2015-09-03 Thread Jun Rao

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/36578/#review97668
---


Thanks for the patch. A couple of comments below.


core/src/main/scala/kafka/admin/TopicCommand.scala (line 92)


Could we log what broker and consumer properties that the user need to 
change? Also, we should say what value the user needs to change to. This is a 
bit tricky with keys. So, the fetch size should be >= (max message size + key 
size + overhead).



core/src/main/scala/kafka/server/AbstractFetcherThread.scala (lines 176 - 178)


As Ewen pointed out earlier, this doesn't quite work. The broker never 
returns ErrorMapping.MessageSizeTooLargeCode. It's the responsibility of the 
consumer to check oversized messages. The way you check that is if 
messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0, then there is an 
oversided message. Also, we should probably do the logging in 
ReplicaFetcherThread.processPartitionData() since for regular consumers, we 
already check oversized messages in ConsumerIterator.makeNext().


- Jun Rao


On Sept. 2, 2015, 10:30 p.m., Edward Ribeiro wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/36578/
> ---
> 
> (Updated Sept. 2, 2015, 10:30 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-2338
> https://issues.apache.org/jira/browse/KAFKA-2338
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> KAFKA-2338 Warn users if they change max.message.bytes that they also need to 
> update broker and consumer settings
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/admin/TopicCommand.scala 
> f1405a5b2961bc826caa22507db8ba39ce1cf4d3 
>   core/src/main/scala/kafka/server/AbstractFetcherThread.scala 
> dca975ca1bf3e560b9d9817e7f2a511ef4296e70 
> 
> Diff: https://reviews.apache.org/r/36578/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Edward Ribeiro
> 
>



Re: [DISCUSS] KIP-31 - Message format change proposal

2015-09-03 Thread Guozhang Wang
Thanks for the write-up Jiangjie.

One comment about migration plan: "For old consumers, if they see the new
protocol the CRC check will fail"..

Do you mean this bug in the old consumer cannot be fixed in a
backward-compatible way?

Guozhang


On Thu, Sep 3, 2015 at 8:35 AM, Jiangjie Qin 
wrote:

> Hi,
>
> We just created KIP-31 to propose a message format change in Kafka.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-31+-+Message+format+change+proposal
>
> As a summary, the motivations are:
> 1. Avoid server side message re-compression
> 2. Honor time-based log roll and retention
> 3. Enable offset search by timestamp at a finer granularity.
>
> Feedback and comments are welcome!
>
> Thanks,
>
> Jiangjie (Becket) Qin
>



-- 
-- Guozhang


[jira] [Commented] (KAFKA-2461) request logger no longer logs extra information in debug mode

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-2461:


[~singhashish], I'll let [~gwenshap] comment on whether it should be available 
on both debug and trace or just trace. I don't have a strong opinion either way.

> request logger no longer logs extra information in debug mode
> -
>
> Key: KAFKA-2461
> URL: https://issues.apache.org/jira/browse/KAFKA-2461
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>Assignee: Ashish K Singh
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Currently request logging calls are identical for trace and debug:
> {code}
> if(requestLogger.isTraceEnabled)
> requestLogger.trace("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
> .format(requestDesc, connectionId, totalTime, 
> requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, 
> responseSendTime))
>   else if(requestLogger.isDebugEnabled)
> requestLogger.debug("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
>   .format(requestDesc, connectionId, totalTime, requestQueueTime, 
> apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime))
> {code}
> I think in the past (3 refactoring steps ago), we used to print more 
> information about specific topics and partitions in debug mode.



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


[jira] [Commented] (KAFKA-2461) request logger no longer logs extra information in debug mode

2015-09-03 Thread Gwen Shapira (JIRA)

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

Gwen Shapira commented on KAFKA-2461:
-

Well, the reason we have two different log statements (one for trace and one 
for debug) is that the partition information can be HUGE on large clusters, and 
not always needed when diagnosing performance issues (the main use of this 
log). So we need one level with details (TRACE) and one level without (DEBUG).

> request logger no longer logs extra information in debug mode
> -
>
> Key: KAFKA-2461
> URL: https://issues.apache.org/jira/browse/KAFKA-2461
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>Assignee: Ashish K Singh
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Currently request logging calls are identical for trace and debug:
> {code}
> if(requestLogger.isTraceEnabled)
> requestLogger.trace("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
> .format(requestDesc, connectionId, totalTime, 
> requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, 
> responseSendTime))
>   else if(requestLogger.isDebugEnabled)
> requestLogger.debug("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
>   .format(requestDesc, connectionId, totalTime, requestQueueTime, 
> apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime))
> {code}
> I think in the past (3 refactoring steps ago), we used to print more 
> information about specific topics and partitions in debug mode.



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


[jira] [Commented] (KAFKA-2461) request logger no longer logs extra information in debug mode

2015-09-03 Thread Ashish K Singh (JIRA)

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

Ashish K Singh commented on KAFKA-2461:
---

Should I keep the detailed info in debug level as well? I am kind of in favor 
of that.

> request logger no longer logs extra information in debug mode
> -
>
> Key: KAFKA-2461
> URL: https://issues.apache.org/jira/browse/KAFKA-2461
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>Assignee: Ashish K Singh
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Currently request logging calls are identical for trace and debug:
> {code}
> if(requestLogger.isTraceEnabled)
> requestLogger.trace("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
> .format(requestDesc, connectionId, totalTime, 
> requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, 
> responseSendTime))
>   else if(requestLogger.isDebugEnabled)
> requestLogger.debug("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
>   .format(requestDesc, connectionId, totalTime, requestQueueTime, 
> apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime))
> {code}
> I think in the past (3 refactoring steps ago), we used to print more 
> information about specific topics and partitions in debug mode.



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


[jira] [Commented] (KAFKA-2461) request logger no longer logs extra information in debug mode

2015-09-03 Thread Gwen Shapira (JIRA)

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

Gwen Shapira commented on KAFKA-2461:
-

TRIVIAL pull request that doesn't require a JIRA makes sense.


> request logger no longer logs extra information in debug mode
> -
>
> Key: KAFKA-2461
> URL: https://issues.apache.org/jira/browse/KAFKA-2461
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>Assignee: Ashish K Singh
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Currently request logging calls are identical for trace and debug:
> {code}
> if(requestLogger.isTraceEnabled)
> requestLogger.trace("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
> .format(requestDesc, connectionId, totalTime, 
> requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, 
> responseSendTime))
>   else if(requestLogger.isDebugEnabled)
> requestLogger.debug("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
>   .format(requestDesc, connectionId, totalTime, requestQueueTime, 
> apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime))
> {code}
> I think in the past (3 refactoring steps ago), we used to print more 
> information about specific topics and partitions in debug mode.



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


[DISCUSS] KIP-31 - Message format change proposal

2015-09-03 Thread Jiangjie Qin
Hi,

We just created KIP-31 to propose a message format change in Kafka.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-31+-+Message+format+change+proposal

As a summary, the motivations are:
1. Avoid server side message re-compression
2. Honor time-based log roll and retention
3. Enable offset search by timestamp at a finer granularity.

Feedback and comments are welcome!

Thanks,

Jiangjie (Becket) Qin


[jira] [Commented] (KAFKA-2461) request logger no longer logs extra information in debug mode

2015-09-03 Thread Ashish K Singh (JIRA)

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

Ashish K Singh commented on KAFKA-2461:
---

[~ijuma] Oops, I got it reversed. That's my bad. However, I feel that info 
actually fits at debug level as well. What do you think? Would you suggest I 
fix it as part of a separate JIRA or same JIRA. Thanks for pointing it out.

> request logger no longer logs extra information in debug mode
> -
>
> Key: KAFKA-2461
> URL: https://issues.apache.org/jira/browse/KAFKA-2461
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>Assignee: Ashish K Singh
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Currently request logging calls are identical for trace and debug:
> {code}
> if(requestLogger.isTraceEnabled)
> requestLogger.trace("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
> .format(requestDesc, connectionId, totalTime, 
> requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, 
> responseSendTime))
>   else if(requestLogger.isDebugEnabled)
> requestLogger.debug("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
>   .format(requestDesc, connectionId, totalTime, requestQueueTime, 
> apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime))
> {code}
> I think in the past (3 refactoring steps ago), we used to print more 
> information about specific topics and partitions in debug mode.



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


[jira] [Created] (KAFKA-2511) KIP-31 message format change

2015-09-03 Thread Jiangjie Qin (JIRA)
Jiangjie Qin created KAFKA-2511:
---

 Summary: KIP-31 message format change
 Key: KAFKA-2511
 URL: https://issues.apache.org/jira/browse/KAFKA-2511
 Project: Kafka
  Issue Type: Improvement
Reporter: Jiangjie Qin
Assignee: Jiangjie Qin


This ticket is created for KIP 31. Please refer to the KIP for details.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-31+-+Message+format+change+proposal



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


[jira] [Resolved] (KAFKA-2506) Refactor KafkaConsumer.partitionsFor(topic) to get metadata of topic without modifying state of Metadata

2015-09-03 Thread Ashish K Singh (JIRA)

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

Ashish K Singh resolved KAFKA-2506.
---
Resolution: Duplicate

> Refactor KafkaConsumer.partitionsFor(topic) to get metadata of topic without 
> modifying state of Metadata
> 
>
> Key: KAFKA-2506
> URL: https://issues.apache.org/jira/browse/KAFKA-2506
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
>
> While working on KAFKA-1893, we realized that it will be good to refactor 
> KafkaConsumer.partitionsFor(topic) to get metadata of topic without modifying 
> state of Metadata. It can follow an approach similar to listTopics().



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


[jira] [Commented] (KAFKA-2506) Refactor KafkaConsumer.partitionsFor(topic) to get metadata of topic without modifying state of Metadata

2015-09-03 Thread Ashish K Singh (JIRA)

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

Ashish K Singh commented on KAFKA-2506:
---

I think KAFKA-1893 does not depend on this, we can wait. Closing this JIRA. 
Thanks!

> Refactor KafkaConsumer.partitionsFor(topic) to get metadata of topic without 
> modifying state of Metadata
> 
>
> Key: KAFKA-2506
> URL: https://issues.apache.org/jira/browse/KAFKA-2506
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
>
> While working on KAFKA-1893, we realized that it will be good to refactor 
> KafkaConsumer.partitionsFor(topic) to get metadata of topic without modifying 
> state of Metadata. It can follow an approach similar to listTopics().



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


[jira] [Updated] (KAFKA-1935) Consumer should use a separate socket for Coordinator connection

2015-09-03 Thread Jason Gustafson (JIRA)

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

Jason Gustafson updated KAFKA-1935:
---
Issue Type: Sub-task  (was: Bug)
Parent: KAFKA-1326

> Consumer should use a separate socket for Coordinator connection
> 
>
> Key: KAFKA-1935
> URL: https://issues.apache.org/jira/browse/KAFKA-1935
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Guozhang Wang
> Fix For: 0.9.0
>
>
> KAFKA-1925 is just a quick-fix of this issue, we need to let consumer to be 
> able to create separate sockets for the same server for coordinator / broker 
> roles.



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


[jira] [Commented] (KAFKA-2510) Prevent broker from re-replicating / losing data due to disk misconfiguration

2015-09-03 Thread Gwen Shapira (JIRA)

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

Gwen Shapira commented on KAFKA-2510:
-

Note that if you accidentally manage to set an entire cluster to the wrong 
directory (easy when Chef or similar manages your configuration), you also lose 
the consumer offsets - so only clients that use external offset store will even 
notice that the data is gone.

Losing ALL data in the cluster without any errors is a huge problem.

> Prevent broker from re-replicating / losing data due to disk misconfiguration
> -
>
> Key: KAFKA-2510
> URL: https://issues.apache.org/jira/browse/KAFKA-2510
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>
> Currently Kafka assumes that whatever it sees in the data directory is the 
> correct state of the data.
> This means that if an admin mistakenly configures Chef to use wrong data 
> directory, one of the following can happen:
> 1. The broker will replicate a bunch of partitions and take over the network
> 2. If you did this to enough brokers, you can lose entire topics and 
> partitions.
> We have information about existing topics, partitions and their ISR in 
> zookeeper.
> We need a mode in which if a broker starts, is in ISR for a partition and 
> doesn't have any data or directory for the partition, the broker will issue a 
> huge ERROR in the log and refuse to do anything for the partition.
> [~fpj] worked on the problem for ZK and had some ideas on what is required 
> here. 



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


[jira] [Created] (KAFKA-2510) Prevent broker from re-replicating / losing data due to disk misconfiguration

2015-09-03 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-2510:
---

 Summary: Prevent broker from re-replicating / losing data due to 
disk misconfiguration
 Key: KAFKA-2510
 URL: https://issues.apache.org/jira/browse/KAFKA-2510
 Project: Kafka
  Issue Type: Bug
Reporter: Gwen Shapira


Currently Kafka assumes that whatever it sees in the data directory is the 
correct state of the data.

This means that if an admin mistakenly configures Chef to use wrong data 
directory, one of the following can happen:
1. The broker will replicate a bunch of partitions and take over the network
2. If you did this to enough brokers, you can lose entire topics and partitions.

We have information about existing topics, partitions and their ISR in 
zookeeper.

We need a mode in which if a broker starts, is in ISR for a partition and 
doesn't have any data or directory for the partition, the broker will issue a 
huge ERROR in the log and refuse to do anything for the partition.

[~fpj] worked on the problem for ZK and had some ideas on what is required 
here. 



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


[jira] [Commented] (KAFKA-2506) Refactor KafkaConsumer.partitionsFor(topic) to get metadata of topic without modifying state of Metadata

2015-09-03 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-2506:


[~singhashish] I realized that the partitionsFor implementation also broke my 
patch, so I made a quick fix for it there. If you wanted to have this change 
before KAFKA-1893 is committed, then maybe we just do it here since it's small 
and I'm hoping to have KAFKA-1893 before my patch is committed. Otherwise, we 
can wait as Ewen suggests.

> Refactor KafkaConsumer.partitionsFor(topic) to get metadata of topic without 
> modifying state of Metadata
> 
>
> Key: KAFKA-2506
> URL: https://issues.apache.org/jira/browse/KAFKA-2506
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Ashish K Singh
>Assignee: Ashish K Singh
>
> While working on KAFKA-1893, we realized that it will be good to refactor 
> KafkaConsumer.partitionsFor(topic) to get metadata of topic without modifying 
> state of Metadata. It can follow an approach similar to listTopics().



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


[VOTE] 0.8.2.2 Candidate 1

2015-09-03 Thread Jun Rao
This is the first candidate for release of Apache Kafka 0.8.2.2. This only
fixes two critical issues (KAFKA-2189 and KAFKA-2308) related to snappy in
0.8.2.1.

Release Notes for the 0.8.2.2 release
https://people.apache.org/~junrao/kafka-0.8.2.2-candidate1/RELEASE_NOTES.html

*** Please download, test and vote by Tuesday, Sep 8, 7pm PT

Kafka's KEYS file containing PGP keys we use to sign the release:
http://kafka.apache.org/KEYS in addition to the md5, sha1
and sha2 (SHA256) checksum.

* Release artifacts to be voted upon (source and binary):
https://people.apache.org/~junrao/kafka-0.8.2.2-candidate1/

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

* scala-doc
https://people.apache.org/~junrao/kafka-0.8.2.2-candidate1/scaladoc/

* java-doc
https://people.apache.org/~junrao/kafka-0.8.2.2-candidate1/javadoc/

* The tag to be voted upon (off the 0.8.2 branch) is the 0.8.2.2 tag
https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=d01226cfdcb3d9daad8465234750fa515a1e7e4a

/***

Thanks,

Jun


Re: Kafka New Consumer Performance Test ThroughPut Degradation

2015-09-03 Thread Poorna Chandra Tejashvi Reddy
Hi Jason,

Thanks for the update, I will be watching KAFKA-2489.

Thanks,
-Poorna

On Wed, Sep 2, 2015 at 5:31 PM, Jason Gustafson  wrote:

> Hi Poorna,
>
> We've been looking at the same issue as part of KAFKA-2489. I suspect the
> initial rebalance is causing the test to timeout, but we're still
> investigating.
>
> Thanks,
> Jason
>
> On Wed, Sep 2, 2015 at 2:03 PM, Poorna Chandra Tejashvi Reddy <
> pctre...@gmail.com> wrote:
>
> > Hi,
> >
> > I have checked out the latest code out of
> https://github.com/apache/kafka
> > based
> > on commit id e582447adb4708731aff74aa294e7ce2b30b0a41. Looks like the
> > performance test on the new-consumer is broken.
> >
> > in/kafka-consumer-perf-test.sh --zookeeper zkip:2181 --broker-list
> > brokerIp:9092 --topic test --messages 5 --new-consumer
> >
> > The test does not return any response. Is this expected and is there a
> > better way to test the new-consumer.
> >
> >
> > Thanks,
> >
> > -Poorna
> >
> >
> > On Thu, Aug 27, 2015 at 2:25 PM, Poorna Chandra Tejashvi Reddy <
> > pctre...@gmail.com> wrote:
> >
> > > Hi,
> > >
> > > We have built the latest kafka from https://github.com/apache/kafka
> > based
> > > on this commit id 436b7ddc386eb688ba0f12836710f5e4bcaa06c8 .
> > > We ran the performance test on a 3 node kafka cluster. There is a huge
> > > throughput degradation using the new-consumer compared to the regular
> > > consumer. Below are the numbers that explain the same.
> > >
> > > bin/kafka-consumer-perf-test.sh --zookeeper zkIp:2181 --broker-list
> > > brokerIp:9092 --topics test --messages 500 : gives a throughput of
> > 693 K
> > >
> > > bin/kafka-consumer-perf-test.sh --zookeeper zkIp:2181 --broker-list
> > > brokerIp:9092 --topics test --messages 500 --new-consumer : gives a
> > > throughput of  51k
> > >
> > > The whole set up is based on ec2, Kafka brokers running on r3.2x large.
> > >
> > > Are you guys aware of this performance degradation , do you have a JIRA
> > > for this, which can be used to track the resolution.
> > >
> > >
> > > Thanks,
> > >
> > > -Poorna
> > >
> >
>


Re: Hello!

2015-09-03 Thread Sudhanshu
Thank you Prabhjot.

I will go through the wiki for setup. Sure :).

On Thu, Sep 3, 2015 at 5:27 PM, Prabhjot Bharaj 
wrote:

> Hi Sudhanshu,
>
> You can go through the details mentioned here:
> http://kafka.apache.org/contributing.html
>
> Also, in case you need to setup your IDE, you can refer to these links:-
>
>
> http://www.lewuathe.com/blog/2014/10/16/build-apache-kafka-with-intellij-idea/
>
> https://cwiki.apache.org/confluence/display/KAFKA/Eclipse-Scala-Gradle-Git+Developement+Environment+Setup
>
> These links were very useful for me
>
> Also, I am in Bangalore. if you want, we can together start on the
> understanding part
>
> Regards,
> Prabhjot
>
> On Thu, Sep 3, 2015 at 4:50 PM, Sudhanshu 
> wrote:
>
> > Hello Developers,
> >
> > I am Sudhanshu Gupta, software engineer working in Bangalore, India. We
> are
> > using kafka in production and its fantastic.
> >
> > I am willing to contribute to the project. I have forked and cloned the
> > kafka repo in my local. I want to get start on it. It would be great if
> > anyone from the community can help to understand the codebase and let me
> > fix some issues.
> >
> > My Jira id: sudhanshu-gupta
> >
> > --
> > Regards,
> > Sudhanshu
> >
>
>
>
> --
> -
> "There are only 10 types of people in the world: Those who understand
> binary, and those who don't"
>



-- 
Regards,
Sudhanshu


Re: Hello!

2015-09-03 Thread Prabhjot Bharaj
Hi Sudhanshu,

You can go through the details mentioned here:
http://kafka.apache.org/contributing.html

Also, in case you need to setup your IDE, you can refer to these links:-

http://www.lewuathe.com/blog/2014/10/16/build-apache-kafka-with-intellij-idea/
https://cwiki.apache.org/confluence/display/KAFKA/Eclipse-Scala-Gradle-Git+Developement+Environment+Setup

These links were very useful for me

Also, I am in Bangalore. if you want, we can together start on the
understanding part

Regards,
Prabhjot

On Thu, Sep 3, 2015 at 4:50 PM, Sudhanshu 
wrote:

> Hello Developers,
>
> I am Sudhanshu Gupta, software engineer working in Bangalore, India. We are
> using kafka in production and its fantastic.
>
> I am willing to contribute to the project. I have forked and cloned the
> kafka repo in my local. I want to get start on it. It would be great if
> anyone from the community can help to understand the codebase and let me
> fix some issues.
>
> My Jira id: sudhanshu-gupta
>
> --
> Regards,
> Sudhanshu
>



-- 
-
"There are only 10 types of people in the world: Those who understand
binary, and those who don't"


Hello!

2015-09-03 Thread Sudhanshu
Hello Developers,

I am Sudhanshu Gupta, software engineer working in Bangalore, India. We are
using kafka in production and its fantastic.

I am willing to contribute to the project. I have forked and cloned the
kafka repo in my local. I want to get start on it. It would be great if
anyone from the community can help to understand the codebase and let me
fix some issues.

My Jira id: sudhanshu-gupta

-- 
Regards,
Sudhanshu


[jira] [Commented] (KAFKA-1101) Need better java api for embedding kafkaserver in a java container app

2015-09-03 Thread Dhaval M Patel (JIRA)

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

Dhaval M Patel commented on KAFKA-1101:
---

Has support for single constructor been added to apache kafka 0.8.2.1 jar?

> Need better java api for embedding kafkaserver in a java container app
> --
>
> Key: KAFKA-1101
> URL: https://issues.apache.org/jira/browse/KAFKA-1101
> Project: Kafka
>  Issue Type: Bug
>Reporter: Jason Rosenberg
>
> We embed the KafkaServer inside a java service container framework, which 
> makes it easy to deploy and monitor within our infrastructure.  When 
> upgrading to kafka 0.8 from 0.7.2, I ran into an issue with not being able to 
> pass the needed constructor arg (SystemTime), since there doesn't appear to 
> be an easy way to instantiate that from java.  So, I ended up with this janky 
> solution using SystemTime$.MODULE$.
> Could a default constructor be added which assumes a default SystemTime, 
> rather than requiring that here?
> Note, I need to construct a KafkaServer directly, since I need to manage the 
> lifecycle more directly than can be done with KafkaServerStartable.
> {code}
>   // Need to do janky scala MODULE$ dereferencing, in order to get a 
> default value in ctor
>   server = new kafka.server.KafkaServer(kafkaConfig, SystemTime$.MODULE$);
>   server.startup();
> {code}



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


[jira] [Commented] (KAFKA-2461) request logger no longer logs extra information in debug mode

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-2461:


I'm a bit unclear about this. Normally, trace is the noisiest of all log 
levels, but in this case debug is more detailed than trace? Can you please 
share the reasoning?

> request logger no longer logs extra information in debug mode
> -
>
> Key: KAFKA-2461
> URL: https://issues.apache.org/jira/browse/KAFKA-2461
> Project: Kafka
>  Issue Type: Bug
>Reporter: Gwen Shapira
>Assignee: Ashish K Singh
>Priority: Blocker
> Fix For: 0.8.3
>
>
> Currently request logging calls are identical for trace and debug:
> {code}
> if(requestLogger.isTraceEnabled)
> requestLogger.trace("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
> .format(requestDesc, connectionId, totalTime, 
> requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, 
> responseSendTime))
>   else if(requestLogger.isDebugEnabled)
> requestLogger.debug("Completed request:%s from connection 
> %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d"
>   .format(requestDesc, connectionId, totalTime, requestQueueTime, 
> apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime))
> {code}
> I think in the past (3 refactoring steps ago), we used to print more 
> information about specific topics and partitions in debug mode.



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


[jira] [Commented] (KAFKA-2072) Add StopReplica request/response to o.a.k.common.requests and replace the usage in core module

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-2072:


[~dajac], what I did for the other JIRAs like KAFKA-2067 is to split it into 
two so that I could resolve the part that was already done while tracking what 
was left to do in a separate one.

> Add StopReplica request/response to o.a.k.common.requests and replace the 
> usage in core module
> --
>
> Key: KAFKA-2072
> URL: https://issues.apache.org/jira/browse/KAFKA-2072
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: David Jacot
>
> Add StopReplica request/response to o.a.k.common.requests and replace the 
> usage in core module



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


[jira] [Resolved] (KAFKA-2074) Add UpdateMetadata request/response to o.a.k.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma resolved KAFKA-2074.

Resolution: Fixed

Done as part of KAFKA-2411, filed KAFKA-2508 for replacing usage in core.

> Add UpdateMetadata request/response to o.a.k.common.requests
> 
>
> Key: KAFKA-2074
> URL: https://issues.apache.org/jira/browse/KAFKA-2074
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
>
> Add UpdateMetadata request/response to o.a.k.common.requests



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


[jira] [Updated] (KAFKA-2074) Add UpdateMetadata request/response to o.a.k.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-2074:
---
Description: Add UpdateMetadata request/response to o.a.k.common.requests  
(was: Add UpdateMetadata request/response to o.a.k.common.requests and replace 
its use in core module)

> Add UpdateMetadata request/response to o.a.k.common.requests
> 
>
> Key: KAFKA-2074
> URL: https://issues.apache.org/jira/browse/KAFKA-2074
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
>
> Add UpdateMetadata request/response to o.a.k.common.requests



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


[jira] [Updated] (KAFKA-2074) Add UpdateMetadata request/response to o.a.k.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-2074:
---
Summary: Add UpdateMetadata request/response to o.a.k.common.requests  
(was: Add UpdateMetadata request/response to o.a.k.common.requests and replace 
its use in core module)

> Add UpdateMetadata request/response to o.a.k.common.requests
> 
>
> Key: KAFKA-2074
> URL: https://issues.apache.org/jira/browse/KAFKA-2074
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
>
> Add UpdateMetadata request/response to o.a.k.common.requests and replace its 
> use in core module



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


[jira] [Updated] (KAFKA-2067) Add LeaderAndISR request/response to org.apache.kafka.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-2067:
---
   Resolution: Fixed
Fix Version/s: 0.8.3
   Status: Resolved  (was: Patch Available)

This was done as part of KAFKA-2411, filed KAFKA-2509 for replacing the usage 
of the old classes.

> Add LeaderAndISR request/response to org.apache.kafka.common.requests
> -
>
> Key: KAFKA-2067
> URL: https://issues.apache.org/jira/browse/KAFKA-2067
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
> Fix For: 0.8.3
>
>
> Add LeaderAndISR request/response to org.apache.kafka.common.requests.
> Note that this will require adding a bunch of new objects to o.a.k.common - 
> LeaderAndISR, LeaderISRAndEpoch and possibly others.
> It may be nice to have a scala implicit to translate those objects from their 
> old (core) implementation to the o.a.k.common implementation.



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


[jira] [Updated] (KAFKA-2067) Add LeaderAndISR request/response to org.apache.kafka.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-2067:
---
Description: 
Add LeaderAndISR request/response to org.apache.kafka.common.requests.

Note that this will require adding a bunch of new objects to o.a.k.common - 
LeaderAndISR, LeaderISRAndEpoch and possibly others.

It may be nice to have a scala implicit to translate those objects from their 
old (core) implementation to the o.a.k.common implementation.

  was:
Add LeaderAndISR request/response to org.apache.kafka.common.requests and 
replace usage in core module.

Note that this will require adding a bunch of new objects to o.a.k.common - 
LeaderAndISR, LeaderISRAndEpoch and possibly others.

It may be nice to have a scala implicit to translate those objects from their 
old (core) implementation to the o.a.k.common implementation.


> Add LeaderAndISR request/response to org.apache.kafka.common.requests
> -
>
> Key: KAFKA-2067
> URL: https://issues.apache.org/jira/browse/KAFKA-2067
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
>
> Add LeaderAndISR request/response to org.apache.kafka.common.requests.
> Note that this will require adding a bunch of new objects to o.a.k.common - 
> LeaderAndISR, LeaderISRAndEpoch and possibly others.
> It may be nice to have a scala implicit to translate those objects from their 
> old (core) implementation to the o.a.k.common implementation.



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


[jira] [Created] (KAFKA-2509) Replace LeaderAndIsr{Request,Response} with org.apache.kafka.common.network.requests equivalent

2015-09-03 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-2509:
--

 Summary: Replace LeaderAndIsr{Request,Response} with 
org.apache.kafka.common.network.requests equivalent
 Key: KAFKA-2509
 URL: https://issues.apache.org/jira/browse/KAFKA-2509
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






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


[jira] [Updated] (KAFKA-2067) Add LeaderAndISR request/response to org.apache.kafka.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-2067:
---
Summary: Add LeaderAndISR request/response to 
org.apache.kafka.common.requests  (was: Add LeaderAndISR request/response to 
org.apache.kafka.common.requests and replace usage in core module)

> Add LeaderAndISR request/response to org.apache.kafka.common.requests
> -
>
> Key: KAFKA-2067
> URL: https://issues.apache.org/jira/browse/KAFKA-2067
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
>
> Add LeaderAndISR request/response to org.apache.kafka.common.requests and 
> replace usage in core module.
> Note that this will require adding a bunch of new objects to o.a.k.common - 
> LeaderAndISR, LeaderISRAndEpoch and possibly others.
> It may be nice to have a scala implicit to translate those objects from their 
> old (core) implementation to the o.a.k.common implementation.



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


[jira] [Updated] (KAFKA-2508) Replace UpdateMetadata{Request,Response} with org.apache.kafka.common.requests equivalent

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-2508:
---
Summary: Replace UpdateMetadata{Request,Response} with 
org.apache.kafka.common.requests equivalent  (was: Replace 
UpdateMetadata{Request,Response} with org.apache.common.requests equivalent)

> Replace UpdateMetadata{Request,Response} with 
> org.apache.kafka.common.requests equivalent
> -
>
> Key: KAFKA-2508
> URL: https://issues.apache.org/jira/browse/KAFKA-2508
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Ismael Juma
>




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


[jira] [Created] (KAFKA-2508) Replace UpdateMetadata{Request,Response} with org.apache.common.requests equivalent

2015-09-03 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-2508:
--

 Summary: Replace UpdateMetadata{Request,Response} with 
org.apache.common.requests equivalent
 Key: KAFKA-2508
 URL: https://issues.apache.org/jira/browse/KAFKA-2508
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






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


[jira] [Updated] (KAFKA-2065) Add ControlledShutdown to org.apache.kafka.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

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

Done via KAFKA-2411, I filed KAFKA-2507 for replacing the usage in core.

> Add ControlledShutdown to  org.apache.kafka.common.requests
> ---
>
> Key: KAFKA-2065
> URL: https://issues.apache.org/jira/browse/KAFKA-2065
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
>
> Add ControlledShutdown to  org.apache.kafka.common.requests



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


[jira] [Updated] (KAFKA-2065) Add ControlledShutdown to org.apache.kafka.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-2065:
---
Fix Version/s: 0.8.3

> Add ControlledShutdown to  org.apache.kafka.common.requests
> ---
>
> Key: KAFKA-2065
> URL: https://issues.apache.org/jira/browse/KAFKA-2065
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
> Fix For: 0.8.3
>
>
> Add ControlledShutdown to  org.apache.kafka.common.requests



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


[jira] [Updated] (KAFKA-2065) Add ControlledShutdown to org.apache.kafka.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-2065:
---
Summary: Add ControlledShutdown to  org.apache.kafka.common.requests  (was: 
Add ControlledShutdown to  org.apache.kafka.common.requests and replace current 
use in core module)

> Add ControlledShutdown to  org.apache.kafka.common.requests
> ---
>
> Key: KAFKA-2065
> URL: https://issues.apache.org/jira/browse/KAFKA-2065
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
>
> Add ControlledShutdown to  org.apache.kafka.common.requests and replace 
> current use in core module



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


[jira] [Updated] (KAFKA-2065) Add ControlledShutdown to org.apache.kafka.common.requests

2015-09-03 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-2065:
---
Description: Add ControlledShutdown to  org.apache.kafka.common.requests  
(was: Add ControlledShutdown to  org.apache.kafka.common.requests and replace 
current use in core module)

> Add ControlledShutdown to  org.apache.kafka.common.requests
> ---
>
> Key: KAFKA-2065
> URL: https://issues.apache.org/jira/browse/KAFKA-2065
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Gwen Shapira
>Assignee: Ismael Juma
>
> Add ControlledShutdown to  org.apache.kafka.common.requests



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


[jira] [Created] (KAFKA-2507) Replace ControlledShutdown{Request,Response} with org.apache.kafka.common.requests equivalent

2015-09-03 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-2507:
--

 Summary: Replace ControlledShutdown{Request,Response} with 
org.apache.kafka.common.requests equivalent
 Key: KAFKA-2507
 URL: https://issues.apache.org/jira/browse/KAFKA-2507
 Project: Kafka
  Issue Type: Sub-task
Reporter: Ismael Juma






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


[jira] [Commented] (KAFKA-2114) Unable to change min.insync.replicas default

2015-09-03 Thread Stevo Slavic (JIRA)

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

Stevo Slavic commented on KAFKA-2114:
-

I guess affected version should be 0.8.2, correct?

> Unable to change min.insync.replicas default
> 
>
> Key: KAFKA-2114
> URL: https://issues.apache.org/jira/browse/KAFKA-2114
> Project: Kafka
>  Issue Type: Bug
>Reporter: Bryan Baugher
>Assignee: Gwen Shapira
> Fix For: 0.8.3
>
> Attachments: KAFKA-2114.patch
>
>
> Following the comment here[1] I was unable to change the min.insync.replicas 
> default value. I tested this by setting up a 3 node cluster, wrote to a topic 
> with a replication factor of 3, using request.required.acks=-1 and setting 
> min.insync.replicas=2 on the broker's server.properties. I then shutdown 2 
> brokers but I was still able to write successfully. Only after running the 
> alter topic command setting min.insync.replicas=2 on the topic did I see 
> write failures.
> [1] - 
> http://mail-archives.apache.org/mod_mbox/kafka-users/201504.mbox/%3CCANZ-JHF71yqKE6%2BKKhWe2EGUJv6R3bTpoJnYck3u1-M35sobgg%40mail.gmail.com%3E



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