Re: [DISCUSS] KIP-17 - Add HighwaterMarkOffset to OffsetFetchResponse

2015-03-26 Thread Todd Palino
I agree with Jun here, that it would make it easier to do lag checking.
However, for individual checks it's really not that much trouble to do the
second request. If you're doing a lot of lag checking (like every consumer
and every topic) where the scale would start to make a difference, I would
argue that you should not be using individual OffsetFetchRequests to do it.
You should instead consume the __consumer_offsets topic to get the
committed offsets, in which case you're still back to getting the high
watermark another way (either through an OffsetRequest or through JMX).

-Todd


On Thu, Mar 26, 2015 at 7:54 AM, Jun Rao j...@confluent.io wrote:

 Grant,

 In addition to FetchRequest, currently we have another way to get the high
 watermark through OffsetRequest (

 https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetRequest
 ).
 OffsetRequest is a read-only request and is much lighter than FetchRequest.
 This is what monitoring tools like ConsumerOffsetChecker is using now.

 By returning the high watermark in the OffsetFetchRequest, we can implement
 tools like ConsumerOffsetChecker a bit simpler: instead of making two
 requests, the tool just needs to make one request. However, I am not sure
 if it makes a big difference.

 Thanks,

 Jun


 On Tue, Mar 24, 2015 at 8:13 PM, Grant Henke ghe...@cloudera.com wrote:

  Here is an initial proposal to add HighwaterMarkOffset to the
  OffsetFetchResponse:
 
 
 https://cwiki.apache.org/confluence/display/KAFKA/KIP-17+-+Add+HighwaterMarkOffset+to+OffsetFetchResponse
 
  I can add a jira and more implementation details if the
  initial proposal has interest.
 
  Thanks,
  Grant
  --
  Grant Henke
  Solutions Consultant | Cloudera
  ghe...@cloudera.com | 920-980-8979
  twitter.com/ghenke http://twitter.com/gchenke |
  linkedin.com/in/granthenke
 



Re: [DISCUSS] KIP-17 - Add HighwaterMarkOffset to OffsetFetchResponse

2015-03-26 Thread Jun Rao
Grant,

In addition to FetchRequest, currently we have another way to get the high
watermark through OffsetRequest (
https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetRequest).
OffsetRequest is a read-only request and is much lighter than FetchRequest.
This is what monitoring tools like ConsumerOffsetChecker is using now.

By returning the high watermark in the OffsetFetchRequest, we can implement
tools like ConsumerOffsetChecker a bit simpler: instead of making two
requests, the tool just needs to make one request. However, I am not sure
if it makes a big difference.

Thanks,

Jun


On Tue, Mar 24, 2015 at 8:13 PM, Grant Henke ghe...@cloudera.com wrote:

 Here is an initial proposal to add HighwaterMarkOffset to the
 OffsetFetchResponse:

 https://cwiki.apache.org/confluence/display/KAFKA/KIP-17+-+Add+HighwaterMarkOffset+to+OffsetFetchResponse

 I can add a jira and more implementation details if the
 initial proposal has interest.

 Thanks,
 Grant
 --
 Grant Henke
 Solutions Consultant | Cloudera
 ghe...@cloudera.com | 920-980-8979
 twitter.com/ghenke http://twitter.com/gchenke |
 linkedin.com/in/granthenke