[jira] [Commented] (KAFKA-683) Fix correlation ids in all requests sent to kafka

2013-01-30 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13566563#comment-13566563
 ] 

Jun Rao commented on KAFKA-683:
---

Yes, the main changes were made by me in kafka-203. If we want to track the 
time per request, we have to somehow know the request type and potentially 
other fields (e.g., to distinguish a replica fetch request and a consumer fetch 
request). If there is a better way to do this, I'd be happy to make the change.

On second thought, is deserializing request in RequestChannel really bad? After 
all, it is supposed to be a request channel and putting request objects into it 
seems to makes sense too. This basically puts the deserialization overhead into 
the network threads instead of the API threads, which may not be ideal. 
However, the deserialization overhead should be small.

 Fix correlation ids in all requests sent to kafka
 -

 Key: KAFKA-683
 URL: https://issues.apache.org/jira/browse/KAFKA-683
 Project: Kafka
  Issue Type: Improvement
Affects Versions: 0.8
Reporter: Neha Narkhede
Assignee: Neha Narkhede
Priority: Critical
  Labels: improvement, replication
 Attachments: kafka-683-v1.patch, kafka-683-v2.patch, 
 kafka-683-v2-rebased.patch, kafka-683-v2-rebased-twice.patch, 
 kafka-683-v3.patch, kafka-683-v3-unit-test.patch


 We should fix the correlation ids in every request sent to Kafka and fix the 
 request log on the broker to specify not only the type of request and who 
 sent it, but also the correlation id. This will be very helpful while 
 troubleshooting problems in production.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-683) Fix correlation ids in all requests sent to kafka

2013-01-30 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13566599#comment-13566599
 ] 

Jay Kreps commented on KAFKA-683:
-

The design was that the network layer is fully generic and the interface it 
presents to the application code is basically a queue or channel to dequeue 
and enqueue requests and responses. All the network layer knows about requests 
and responses is that they are glorified byte arrays.

The details of how we handle request processing or the format of requests is 
not part of the network system. There is a specific layer, KafkaRequestHandler 
which is the  kafka part. Note how this bit happens in the *processing* thread 
pool not in the *network* thread pool.

The thought I had was to eventually factor more out of the KafkaApis layer and 
into the KafkaRequestHandler layer to make KafkaApis easier to test in the 
absence of a network server (i.e. you should just call new KafkaApis and then 
test the individual handle() methods). That was the layer that is meant to 
adopt our business logic in kafka apis to details of serialization, network, 
etc. Neha and I had briefly discussed this a few times.

I am not saying this plan is the right way to go, I just would have liked to 
discuss that approach. 

 Fix correlation ids in all requests sent to kafka
 -

 Key: KAFKA-683
 URL: https://issues.apache.org/jira/browse/KAFKA-683
 Project: Kafka
  Issue Type: Improvement
Affects Versions: 0.8
Reporter: Neha Narkhede
Assignee: Neha Narkhede
Priority: Critical
  Labels: improvement, replication
 Attachments: kafka-683-v1.patch, kafka-683-v2.patch, 
 kafka-683-v2-rebased.patch, kafka-683-v2-rebased-twice.patch, 
 kafka-683-v3.patch, kafka-683-v3-unit-test.patch


 We should fix the correlation ids in every request sent to Kafka and fix the 
 request log on the broker to specify not only the type of request and who 
 sent it, but also the correlation id. This will be very helpful while 
 troubleshooting problems in production.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-683) Fix correlation ids in all requests sent to kafka

2013-01-29 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13566234#comment-13566234
 ] 

Jay Kreps commented on KAFKA-683:
-

Yeah, I am on the fence about the metrics stuff, maybe it does make sense to 
measure request metrics based on entrance and exit from the queue though 
clearly that implementation is super super nasty (I think we could make request 
metrics a general concept of name/start/stop and track them up and down the 
stack without too much unpleasentness).

But I still see on 0.8 the thing I am complaining about which is
  val requestObj: RequestOrResponse = 
RequestKeys.deserializerForKey(requestId)(buffer)
which if I understand correctly moves api deserialization into the network 
layer and into the network threads. This was not meant to be the definition of 
the network layer. It hard-codes our api set and format so you can no longer 
test that layer independently.

But basically yes, I was doing the code review for the other bug and noticed 
that indeed a virtually everyone on the team has come and done something mean 
to my beautiful little request queue (only Swapnil is innocent!). Each of these 
things was done as part of a useful feature but since it was done by someone 
who doesn't care much about this layer it was done without thinking about 
whether a request channel is a thing that instantiates producer requests, 
contains the list of all the metrics we track, parses requests, or whatever. I 
think each person was just trying to get their thing done and thought, ah, here 
is a nice place to shove my stuff. Probably everyone after the first person 
thought, this code is real crap so no sense trying to hard because once you 
have one or two WTFs in there no one else really puts in much effort.

 Fix correlation ids in all requests sent to kafka
 -

 Key: KAFKA-683
 URL: https://issues.apache.org/jira/browse/KAFKA-683
 Project: Kafka
  Issue Type: Improvement
Affects Versions: 0.8
Reporter: Neha Narkhede
Assignee: Neha Narkhede
Priority: Critical
  Labels: improvement, replication
 Attachments: kafka-683-v1.patch, kafka-683-v2.patch, 
 kafka-683-v2-rebased.patch, kafka-683-v2-rebased-twice.patch, 
 kafka-683-v3.patch, kafka-683-v3-unit-test.patch


 We should fix the correlation ids in every request sent to Kafka and fix the 
 request log on the broker to specify not only the type of request and who 
 sent it, but also the correlation id. This will be very helpful while 
 troubleshooting problems in production.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-683) Fix correlation ids in all requests sent to kafka

2013-01-11 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13551550#comment-13551550
 ] 

Jun Rao commented on KAFKA-683:
---

The changes in core/src/test/resources/log4j.properties are not intended, 
right? Other than that, +1 on the latest patch. We can have another jira to 
clean up the request string format.

 Fix correlation ids in all requests sent to kafka
 -

 Key: KAFKA-683
 URL: https://issues.apache.org/jira/browse/KAFKA-683
 Project: Kafka
  Issue Type: Improvement
Affects Versions: 0.8
Reporter: Neha Narkhede
Assignee: Neha Narkhede
Priority: Critical
  Labels: improvement, replication
 Attachments: kafka-683-v1.patch, kafka-683-v2.patch, 
 kafka-683-v2-rebased.patch, kafka-683-v2-rebased-twice.patch, 
 kafka-683-v3.patch, kafka-683-v3-unit-test.patch


 We should fix the correlation ids in every request sent to Kafka and fix the 
 request log on the broker to specify not only the type of request and who 
 sent it, but also the correlation id. This will be very helpful while 
 troubleshooting problems in production.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-683) Fix correlation ids in all requests sent to kafka

2013-01-10 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13549792#comment-13549792
 ] 

Jun Rao commented on KAFKA-683:
---

Patch v2 doesn't seem to apply on 0.8. Could you rebase?

For 6, I didn't mean to remove the trace logging in RequestChannel. What I 
meant is that we already print out requestObj which includes every field in a 
request. So, there is no need to explicitly print out clientid, correlationid 
and versionid.

 Fix correlation ids in all requests sent to kafka
 -

 Key: KAFKA-683
 URL: https://issues.apache.org/jira/browse/KAFKA-683
 Project: Kafka
  Issue Type: Improvement
Affects Versions: 0.8
Reporter: Neha Narkhede
Assignee: Neha Narkhede
Priority: Critical
  Labels: improvement, replication
 Attachments: kafka-683-v1.patch, kafka-683-v2.patch


 We should fix the correlation ids in every request sent to Kafka and fix the 
 request log on the broker to specify not only the type of request and who 
 sent it, but also the correlation id. This will be very helpful while 
 troubleshooting problems in production.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-683) Fix correlation ids in all requests sent to kafka

2013-01-10 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13549967#comment-13549967
 ] 

Neha Narkhede commented on KAFKA-683:
-

That is a good suggestion. However, that still doesn't get rid of the 3 special 
fields from RequestChannel since we don't have access to the individual request 
objects there unless we cast/convert, which I thought was unnecessary.

 Fix correlation ids in all requests sent to kafka
 -

 Key: KAFKA-683
 URL: https://issues.apache.org/jira/browse/KAFKA-683
 Project: Kafka
  Issue Type: Improvement
Affects Versions: 0.8
Reporter: Neha Narkhede
Assignee: Neha Narkhede
Priority: Critical
  Labels: improvement, replication
 Attachments: kafka-683-v1.patch, kafka-683-v2.patch, 
 kafka-683-v2-rebased.patch


 We should fix the correlation ids in every request sent to Kafka and fix the 
 request log on the broker to specify not only the type of request and who 
 sent it, but also the correlation id. This will be very helpful while 
 troubleshooting problems in production.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-683) Fix correlation ids in all requests sent to kafka

2013-01-04 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13544506#comment-13544506
 ] 

Jun Rao commented on KAFKA-683:
---

Thanks for the patch. Some comments:

1. DefaultEventHandler.send(): Instead of
  val currentCorrelationId = correlationId.get()-1
it's probably better to
  val currentCorrelationId = correlationId.getAndIncrement()
at the beginning and then reuse it when needed.

2. FileMessageSet: If initChannelPositionToEnd is true, we could be either 
creating a new segment or loading an existing segment during startup. So, we 
should rephrase the info message a bit.

3. KafkaMigrationTool: Currently, the tool requires exactly one of whitelist or 
blacklist. So, we will not be able to use the default value of whitelist. We 
can probably leave whitelist as a required argument, but put in the description 
of how to specify all topics correctly (ie, .*).

4. Log:
4.1 maybeRoll(): Rechecking the condition has the problem that time-based 
condition may not return the same value. We can probably check each condition 
once and if the condition is true, log the cause and set a boolean var 
shouldRoll to true.
4.2 markDeletedWhile(): For the new logging, should we somehow indicate that 
those logs are from this function? Also, it seems that we log whether all 
current index/data files exist. Should we log the name of the index/data files 
too so that we know which ones are missing?

5. javaapi.TopicMetadataRequest: The scala optional parameter for correlationId 
won't work for java. We will have to manually create two constructors, one with 
correlation id and the other without.

6. RequestChannel: We are already logging the whole request which includes 
clientid, correlationid and versionid. So, there is no need to log them 
explicitly.

7. config/log4j.properties: All scripts in bin/ currently uses this file. The 
changes are really intended for Kafka broker. Perhaps we can create a new log4j 
file just for the broker and change the kafka broker scripts accordingly. Also, 
for kafka broker, should we log to both file and console? Finally, I got the 
following warning when running the kafka server startup script.
log4j:WARN No such property [maxBackupIndex] in org.apache.log4j.FileAppender.
log4j:WARN No such property [maxFileSize] in org.apache.log4j.FileAppender.
log4j:WARN No such property [maxBackupIndex] in org.apache.log4j.FileAppender.
log4j:WARN No such property [maxFileSize] in org.apache.log4j.FileAppender.

8. Was the state change log added? I didn't see the change in the scala code or 
the log4j property file.


 Fix correlation ids in all requests sent to kafka
 -

 Key: KAFKA-683
 URL: https://issues.apache.org/jira/browse/KAFKA-683
 Project: Kafka
  Issue Type: Improvement
Affects Versions: 0.8
Reporter: Neha Narkhede
Assignee: Neha Narkhede
Priority: Critical
  Labels: improvement, replication
 Attachments: kafka-683-v1.patch


 We should fix the correlation ids in every request sent to Kafka and fix the 
 request log on the broker to specify not only the type of request and who 
 sent it, but also the correlation id. This will be very helpful while 
 troubleshooting problems in production.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira