That sounds good, although is that the only change (sorry I have not
done a careful review of that patch and would like to before it gets
checked in).

On Fri, Oct 31, 2014 at 10:42:13AM -0700, Jun Rao wrote:
> To circle back on this thread. The patch in kafka-1482 is almost ready. To
> make the mbean names more meaningful and easier to parse, the patch will
> use explicit key/value pairs in the mbean name for things like clientId and
> topic, and will get rid of the quotes.
> 
> So, instead of
>    "kafka.server":type="BrokerTopicMetrics",name="topic-1-BytesInPerSec"
> we will have
>    kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec,topic=topic-1
> 
> Any objection to committing this to the 0.8.2 branch?
> 
> Thanks,
> 
> Jun
> 
> On Fri, Oct 17, 2014 at 11:54 AM, Jun Rao <jun...@gmail.com> wrote:
> 
> > Hi, everyone,
> >
> > We are fixing the mbean names in kafka-1482, by adding separate explicit
> > tags in the name for things like clientId and topic. Another thing that
> > some people have complained before is that we use quotes in the jmx name.
> > Should we also just get rid of the quotes as part of kafka-1482? So,
> > instead of
> >    "kafka.server":type="BrokerTopicMetrics",name="topic-1-BytesInPerSec"
> > we will have
> >    kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec,topic=topic-1
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Oct 9, 2014 at 11:12 AM, Neha Narkhede <neha.narkh...@gmail.com>
> > wrote:
> >
> >> I am going to vote for 1482 to be included in 0.8.2, if we have a patch
> >> submitted in a week. I think we've had this JIRA opened for too long and
> >> we
> >> held people back so it's only fair to release this.
> >>
> >> On Wed, Oct 8, 2014 at 9:40 PM, Jun Rao <jun...@gmail.com> wrote:
> >>
> >> > Otis,
> >> >
> >> > Just have the patch ready asap. We can make a call then.
> >> >
> >> > Thanks,
> >> >
> >> > Jun
> >> >
> >> > On Wed, Oct 8, 2014 at 6:13 AM, Otis Gospodnetic <
> >> > otis.gospodne...@gmail.com
> >> > > wrote:
> >> >
> >> > > Hi Jun,
> >> > >
> >> > > Would by the end of next week be acceptable for 0.8.2?
> >> > >
> >> > > Thanks,
> >> > > Otis
> >> > > --
> >> > > Monitoring * Alerting * Anomaly Detection * Centralized Log Management
> >> > > Solr & Elasticsearch Support * http://sematext.com/
> >> > >
> >> > >
> >> > > On Tue, Oct 7, 2014 at 4:04 PM, Jun Rao <jun...@gmail.com> wrote:
> >> > >
> >> > > > Otis,
> >> > > >
> >> > > > Yes, if you guys can help provide a patch in a few days, we can
> >> > probably
> >> > > > get it to the 0.8.2 release.
> >> > > >
> >> > > > Thanks,
> >> > > >
> >> > > > Jun
> >> > > >
> >> > > > On Tue, Oct 7, 2014 at 12:10 PM, Otis Gospodnetic <
> >> > > > otis.gospodne...@gmail.com> wrote:
> >> > > >
> >> > > > > Hi Jun,
> >> > > > >
> >> > > > > I think your MBean renaming approach will work.  I see
> >> > > > > https://issues.apache.org/jira/browse/KAFKA-1481 has Fix Version
> >> > > 0.8.2,
> >> > > > > but
> >> > > > > is not marked as a Blocker.  We'd love to get the MBeans fixed so
> >> > this
> >> > > > > makes it in 0.8.2 release.  Do you know if this is on anyone's
> >> plate
> >> > > (the
> >> > > > > issue is currently Unassigned)?  If not, should we provide a new
> >> > patch
> >> > > > that
> >> > > > > uses your approach?
> >> > > > >
> >> > > > > Thanks,
> >> > > > > Otis
> >> > > > > --
> >> > > > > Monitoring * Alerting * Anomaly Detection * Centralized Log
> >> > Management
> >> > > > > Solr & Elasticsearch Support * http://sematext.com/
> >> > > > >
> >> > > > >
> >> > > > > On Thu, Sep 18, 2014 at 4:49 PM, Jun Rao <jun...@gmail.com>
> >> wrote:
> >> > > > >
> >> > > > > > Otis,
> >> > > > > >
> >> > > > > > In kafka-1481, we will have to change the mbean names (at least
> >> the
> >> > > > ones
> >> > > > > > with clientid and topic) anyway. Using the name/value pair in
> >> the
> >> > > mbean
> >> > > > > > name allows us to do this in a cleaner way. Yes, "," is not
> >> allowed
> >> > > in
> >> > > > > > clientid or topic.
> >> > > > > >
> >> > > > > > Bhavesh,
> >> > > > > >
> >> > > > > > Yes, I was thinking of making changes in the new metrics
> >> package.
> >> > > > > Something
> >> > > > > > like allowing the sensor names to have name/value pairs. The jmx
> >> > > names
> >> > > > > will
> >> > > > > > just follow accordingly. This is probably cleaner than doing the
> >> > > > > escaping.
> >> > > > > > Also, the metric names are more intuitive (otherwise, you have
> >> to
> >> > > know
> >> > > > > > which part is the clientid and which part is the topic).
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > >
> >> > > > > > Jun
> >> > > > > >
> >> > > > > > On Wed, Sep 17, 2014 at 2:32 PM, Otis Gospodnetic <
> >> > > > > > otis.gospodne...@gmail.com> wrote:
> >> > > > > >
> >> > > > > > > Hi Jun,
> >> > > > > > >
> >> > > > > > > On Wed, Sep 17, 2014 at 12:35 PM, Jun Rao <jun...@gmail.com>
> >> > > wrote:
> >> > > > > > >
> >> > > > > > > > Bhavesh,
> >> > > > > > > >
> >> > > > > > > > Yes, allowing dot in clientId and topic makes it a bit
> >> harder
> >> > to
> >> > > > > define
> >> > > > > > > the
> >> > > > > > > > JMX bean names. I see a couple of solutions here.
> >> > > > > > > >
> >> > > > > > > > 1. Disable dot in clientId and topic names. The issue is
> >> that
> >> > dot
> >> > > > may
> >> > > > > > > > already be used in existing deployment.
> >> > > > > > > >
> >> > > > > > > > 2. We can represent the JMX bean name differently in the new
> >> > > > > producer.
> >> > > > > > > > Instead of
> >> > > > > > > >   kafka.producer.myclientid:type=mytopic
> >> > > > > > > > we could change it to
> >> > > > > > > >   kafka.producer:clientId=myclientid,topic=mytopic
> >> > > > > > > >
> >> > > > > > > > I felt that option 2 is probably better since it doesn't
> >> affect
> >> > > > > > existing
> >> > > > > > > > users.
> >> > > > > > > >
> >> > > > > > >
> >> > > > > > > If it doesn't affect existing users, great.
> >> > > > > > >
> >> > > > > > > If you are saying that each "piece" of MBean name could be
> >> > > expressed
> >> > > > as
> >> > > > > > > name=value pair, with something like "," (forbidden in host
> >> > names,
> >> > > > > topic
> >> > > > > > > names, client IDs, etc. I assume?) then yes, I think this
> >> would
> >> > be
> >> > > > > easier
> >> > > > > > > to parse and it would be easier for people to understand what
> >> is
> >> > > > what.
> >> > > > > > >
> >> > > > > > > Otis
> >> > > > > > > --
> >> > > > > > > Monitoring * Alerting * Anomaly Detection * Centralized Log
> >> > > > Management
> >> > > > > > > Solr & Elasticsearch Support * http://sematext.com/
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > >
> >> > > > > > > > Otis,
> >> > > > > > > >
> >> > > > > > > > We probably can also use option 2 to address KAFKA-1481. For
> >> > > > > > > topic/clientid
> >> > > > > > > > specific metrics, we could explicitly specify the metric
> >> name
> >> > so
> >> > > > that
> >> > > > > > it
> >> > > > > > > > contains "topic=mytopic,clientid=myclientid". That seems to
> >> be
> >> > a
> >> > > > much
> >> > > > > > > > cleaner way than having all parts included in a single
> >> string
> >> > > > > separated
> >> > > > > > > by
> >> > > > > > > > '|'.
> >> > > > > > > >
> >> > > > > > > > Thanks,
> >> > > > > > > >
> >> > > > > > > > Jun
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > On Tue, Sep 16, 2014 at 5:15 PM, Bhavesh Mistry <
> >> > > > > > > > mistry.p.bhav...@gmail.com>
> >> > > > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > HI Otis,
> >> > > > > > > > >
> >> > > > > > > > > What is migration path ?  If topic with special chars
> >> exists
> >> > > > > already(
> >> > > > > > > > > ".","-","|" etc)  in previous version of
> >> producer/consumer of
> >> > > > > Kafka,
> >> > > > > > > what
> >> > > > > > > > > happens after the upgrade new producer or consumer (kafka
> >> > > > version)
> >> > > > > ?
> >> > > > > > > > Also,
> >> > > > > > > > > in new producer API (Kafka Trunk), does this enforce the
> >> rule
> >> > > > about
> >> > > > > > > > client
> >> > > > > > > > > id as well ?
> >> > > > > > > > >
> >> > > > > > > > > Thanks,
> >> > > > > > > > >
> >> > > > > > > > > Bhavesh
> >> > > > > > > > >
> >> > > > > > > > > On Tue, Sep 16, 2014 at 2:09 PM, Otis Gospodnetic <
> >> > > > > > > > > otis.gospodne...@gmail.com> wrote:
> >> > > > > > > > >
> >> > > > > > > > > > Hi,
> >> > > > > > > > > >
> >> > > > > > > > > > So maybe I should I should have asked the Q explicitly:
> >> > > > > > > > > > Could we commit the patch from
> >> > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-1481 now
> >> > that, I
> >> > > > > hope,
> >> > > > > > > > it's
> >> > > > > > > > > > clear what problems the current MBean names can cause?
> >> > > > > > > > > >
> >> > > > > > > > > > Thanks,
> >> > > > > > > > > > Otis
> >> > > > > > > > > > --
> >> > > > > > > > > > Monitoring * Alerting * Anomaly Detection * Centralized
> >> Log
> >> > > > > > > Management
> >> > > > > > > > > > Solr & Elasticsearch Support * http://sematext.com/
> >> > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > > > On Mon, Sep 15, 2014 at 10:40 PM, Otis Gospodnetic <
> >> > > > > > > > > > otis.gospodne...@gmail.com> wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > Hi,
> >> > > > > > > > > > >
> >> > > > > > > > > > > *Problem:*
> >> > > > > > > > > > > Some Kafka 0.8.x MBeans have names composed of things
> >> > like
> >> > > > > > > <consumer
> >> > > > > > > > > > > group>-<topic>-<metric name>.  Note how dashes are
> >> used
> >> > as
> >> > > > > > > > delimiters.
> >> > > > > > > > > > >  When <consumer group> and <topic> don't contain the
> >> > > > delimiter
> >> > > > > > > > > character
> >> > > > > > > > > > > all is good if you want to extract parts of this MBean
> >> > name
> >> > > > by
> >> > > > > > > simply
> >> > > > > > > > > > > splitting on the delimiter character.  The problem is
> >> > that
> >> > > > > dashes
> >> > > > > > > are
> >> > > > > > > > > > > allowed in topic and group names, so this splitting
> >> > doesn't
> >> > > > > work.
> >> > > > > > > > > > > Moreover, underscores are also used as delimiters, and
> >> > they
> >> > > > can
> >> > > > > > > also
> >> > > > > > > > be
> >> > > > > > > > > > > used in things like topic names.
> >> > > > > > > > > > >
> >> > > > > > > > > > > *Example*:
> >> > > > > > > > > > > This MBean's name is composed of <consumer
> >> > > > > > > > group>-<topic>-BytesPerSec:
> >> > > > > > > > > > >
> >> > > > > > > > > > > kafka.consumer:type="ConsumerTopicMetrics",
> >> > > > > > > > > name="*myGroup**-myTopic**-*
> >> > > > > > > > > > > BytesPerSec"
> >> > > > > > > > > > >
> >> > > > > > > > > > > Here we can actually split on "-" and extract all 3
> >> parts
> >> > > > from
> >> > > > > > the
> >> > > > > > > > > MBean
> >> > > > > > > > > > > name::
> >> > > > > > > > > > > * consumer group ('*myGroup*')
> >> > > > > > > > > > > * topic ('*myTopic*')
> >> > > > > > > > > > > * metric (‘BytesPerSec’)
> >> > > > > > > > > > >
> >> > > > > > > > > > > All good!
> >> > > > > > > > > > >
> >> > > > > > > > > > > But imagine if I named the group: *my-Group*
> >> > > > > > > > > > > And if I named the topic: *my-Topic*
> >> > > > > > > > > > >
> >> > > > > > > > > > > Then we'd have:
> >> > > > > > > > > > > kafka.consumer:type="ConsumerTopicMetrics",
> >> > > > > > > > > > name="*my-Group**-my-Topic**-*
> >> > > > > > > > > > > BytesPerSec"
> >> > > > > > > > > > >
> >> > > > > > > > > > > Now splitting on "-" would no longer work!  To extract
> >> > > > > "my-Group"
> >> > > > > > > and
> >> > > > > > > > > > > "my-Topic" and "BytesPerSec" parts I would have to
> >> know
> >> > the
> >> > > > > > > specific
> >> > > > > > > > > > group
> >> > > > > > > > > > > name and topic name to look for and could not use
> >> generic
> >> > > > > > approach
> >> > > > > > > of
> >> > > > > > > > > > just
> >> > > > > > > > > > > splitting the MBean name on the delimiter.
> >> > > > > > > > > > >
> >> > > > > > > > > > > *Solution*:
> >> > > > > > > > > > > The patch in
> >> > > > https://issues.apache.org/jira/browse/KAFKA-1481
> >> > > > > > > > replaces
> >> > > > > > > > > > > all _ and - characters where they are used as
> >> delimiters
> >> > in
> >> > > > > MBean
> >> > > > > > > > names
> >> > > > > > > > > > > with a "|" character.  Because the "I" character is
> >> not
> >> > > > allowed
> >> > > > > > in
> >> > > > > > > > > topic
> >> > > > > > > > > > > names, consumer groups, host names, splitting on this
> >> new
> >> > > and
> >> > > > > > > unified
> >> > > > > > > > > > > delimiter works.
> >> > > > > > > > > > >
> >> > > > > > > > > > > I hope this explains the problem, the solution, and
> >> that
> >> > > this
> >> > > > > can
> >> > > > > > > > make
> >> > > > > > > > > it
> >> > > > > > > > > > > in the next 0.8.x.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Otis
> >> > > > > > > > > > > --
> >> > > > > > > > > > > Monitoring * Alerting * Anomaly Detection *
> >> Centralized
> >> > Log
> >> > > > > > > > Management
> >> > > > > > > > > > > Solr & Elasticsearch Support * http://sematext.com/
> >> > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >

Reply via email to