Yes, agree here. While it can be a little confusing, I think it's better to
just disallow the character for all creation steps so you can't create more
"bad" topic names, but not try and enforce it for topics that already
exist. Anyone who is in that situation is already there with regards to
metrics, and so they are probably making sure they don't collide names that
only differ in the use of "_" and ".". However, we don't want a new user to
accidentally do it.

-Todd


On Fri, Jul 10, 2015 at 2:02 PM, Gwen Shapira <gshap...@cloudera.com> wrote:

> I don't think we should break existing topics. Just disallow new
> topics going forward.
>
> Agree that having both is horrible, but we should have a solution that
> fails when you run "kafka_topics.sh --create", not when you configure
> Ganglia.
>
> Gwen
>
> On Fri, Jul 10, 2015 at 1:53 PM, Jay Kreps <j...@confluent.io> wrote:
> > Unfortunately '.' is pretty common too. I agree that it is perverse, but
> > people seem to do it. Breaking all the topics with '.' in the name seems
> > like it could be worse than combining metrics for people who have a
> > 'foo_bar' AND 'foo.bar' (and after all, having both is DEEPLY perverse,
> > no?).
> >
> > Where is our Dean of Compatibility, Ewen, on this?
> >
> > -Jay
> >
> > On Fri, Jul 10, 2015 at 1:32 PM, Todd Palino <tpal...@gmail.com> wrote:
> >
> >> My selfish point of view is that we do #1, as we use "_" extensively in
> >> topic names here :) I also happen to think it's the right choice,
> >> specifically because "." has more special meanings, as you noted.
> >>
> >> -Todd
> >>
> >>
> >> On Fri, Jul 10, 2015 at 1:30 PM, Gwen Shapira <gshap...@cloudera.com>
> >> wrote:
> >>
> >> > Unintentional side effect from allowing IP addresses in consumer
> client
> >> > IDs :)
> >> >
> >> > So the question is, what do we do now?
> >> >
> >> > 1) disallow "."
> >> > 2) disallow "_"
> >> > 3) find a reversible way to encode "." and "_" that won't break
> existing
> >> > metrics
> >> > 4) all of the above?
> >> >
> >> > btw. it looks like "." and ".." are currently valid. Topic names are
> >> > used for directories, right? this sounds like fun :)
> >> >
> >> > I vote for option #1, although if someone has a good idea for #3 it
> >> > will be even better.
> >> >
> >> > Gwen
> >> >
> >> >
> >> >
> >> > On Fri, Jul 10, 2015 at 1:22 PM, Grant Henke <ghe...@cloudera.com>
> >> wrote:
> >> > > Found it was added here:
> >> https://issues.apache.org/jira/browse/KAFKA-697
> >> > >
> >> > > On Fri, Jul 10, 2015 at 3:18 PM, Todd Palino <tpal...@gmail.com>
> >> wrote:
> >> > >
> >> > >> This was definitely changed at some point after KAFKA-495. The
> >> question
> >> > is
> >> > >> when and why.
> >> > >>
> >> > >> Here's the relevant code from that patch:
> >> > >>
> >> > >> ===================================================================
> >> > >> --- core/src/main/scala/kafka/utils/Topic.scala (revision 1390178)
> >> > >> +++ core/src/main/scala/kafka/utils/Topic.scala (working copy)
> >> > >> @@ -21,24 +21,21 @@
> >> > >>  import util.matching.Regex
> >> > >>
> >> > >>  object Topic {
> >> > >> +  val legalChars = "[a-zA-Z0-9_-]"
> >> > >>
> >> > >>
> >> > >>
> >> > >> -Todd
> >> > >>
> >> > >>
> >> > >> On Fri, Jul 10, 2015 at 1:02 PM, Grant Henke <ghe...@cloudera.com>
> >> > wrote:
> >> > >>
> >> > >> > kafka.common.Topic shows that currently period is a valid
> character
> >> > and I
> >> > >> > have verified I can use kafka-topics.sh to create a new topic
> with a
> >> > >> > period.
> >> > >> >
> >> > >> >
> >> > >> > AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK
> currently
> >> > uses
> >> > >> > Topic.validate before writing to Zookeeper.
> >> > >> >
> >> > >> > Should period character support be removed? I was under the same
> >> > >> impression
> >> > >> > as Gwen, that a period was used by many as a way to "group"
> topics.
> >> > >> >
> >> > >> > The code is pasted below since its small:
> >> > >> >
> >> > >> > object Topic {
> >> > >> >   val legalChars = "[a-zA-Z0-9\\._\\-]"
> >> > >> >   private val maxNameLength = 255
> >> > >> >   private val rgx = new Regex(legalChars + "+")
> >> > >> >
> >> > >> >   val InternalTopics = Set(OffsetManager.OffsetsTopicName)
> >> > >> >
> >> > >> >   def validate(topic: String) {
> >> > >> >     if (topic.length <= 0)
> >> > >> >       throw new InvalidTopicException("topic name is illegal,
> can't
> >> be
> >> > >> > empty")
> >> > >> >     else if (topic.equals(".") || topic.equals(".."))
> >> > >> >       throw new InvalidTopicException("topic name cannot be
> \".\" or
> >> > >> > \"..\"")
> >> > >> >     else if (topic.length > maxNameLength)
> >> > >> >       throw new InvalidTopicException("topic name is illegal,
> can't
> >> be
> >> > >> > longer than " + maxNameLength + " characters")
> >> > >> >
> >> > >> >     rgx.findFirstIn(topic) match {
> >> > >> >       case Some(t) =>
> >> > >> >         if (!t.equals(topic))
> >> > >> >           throw new InvalidTopicException("topic name " + topic
> + "
> >> is
> >> > >> > illegal, contains a character other than ASCII alphanumerics,
> '.',
> >> '_'
> >> > >> and
> >> > >> > '-'")
> >> > >> >       case None => throw new InvalidTopicException("topic name "
> +
> >> > topic
> >> > >> +
> >> > >> > " is illegal,  contains a character other than ASCII
> alphanumerics,
> >> > '.',
> >> > >> > '_' and '-'")
> >> > >> >     }
> >> > >> >   }
> >> > >> > }
> >> > >> >
> >> > >> > On Fri, Jul 10, 2015 at 2:50 PM, Todd Palino <tpal...@gmail.com>
> >> > wrote:
> >> > >> >
> >> > >> > > I had to go look this one up again to make sure -
> >> > >> > > https://issues.apache.org/jira/browse/KAFKA-495
> >> > >> > >
> >> > >> > > The only valid character names for topics are alphanumeric,
> >> > underscore,
> >> > >> > and
> >> > >> > > dash. A period is not supposed to be a valid character to use.
> If
> >> > >> you're
> >> > >> > > seeing them, then one of two things have happened:
> >> > >> > >
> >> > >> > > 1) You have topic names that are grandfathered in from before
> that
> >> > >> patch
> >> > >> > > 2) The patch is not working properly and there is somewhere in
> the
> >> > >> broker
> >> > >> > > that the standard is not being enforced.
> >> > >> > >
> >> > >> > > -Todd
> >> > >> > >
> >> > >> > >
> >> > >> > > On Fri, Jul 10, 2015 at 12:13 PM, Brock Noland <
> br...@apache.org>
> >> > >> wrote:
> >> > >> > >
> >> > >> > > > On Fri, Jul 10, 2015 at 11:34 AM, Gwen Shapira <
> >> > >> gshap...@cloudera.com>
> >> > >> > > > wrote:
> >> > >> > > > > Hi Kafka Fans,
> >> > >> > > > >
> >> > >> > > > > If you have one topic named "kafka_lab_2" and the other
> named
> >> > >> > > > > "kafka.lab.2", the topic level metrics will be named
> >> kafka_lab_2
> >> > >> for
> >> > >> > > > > both, effectively making it impossible to monitor them
> >> properly.
> >> > >> > > > >
> >> > >> > > > > The reason this happens is that using "." in topic names is
> >> > pretty
> >> > >> > > > > common, especially as a way to group topics into data
> centers,
> >> > >> > > > > relevant apps, etc - basically a work-around to our current
> >> > lack of
> >> > >> > > > > name spaces. However, most metric monitoring systems using
> "."
> >> > to
> >> > >> > > > > annotate hierarchy, so to avoid issues around metric names,
> >> > Kafka
> >> > >> > > > > replaces the "." in the name with an underscore.
> >> > >> > > > >
> >> > >> > > > > This generates good metric names, but creates the problem
> with
> >> > name
> >> > >> > > > collisions.
> >> > >> > > > >
> >> > >> > > > > I'm wondering if it makes sense to simply limit the range
> of
> >> > >> > > > > characters permitted in a topic name and disallow "_"?
> >> Obviously
> >> > >> > > > > existing topics will need to remain as is, which is a bit
> >> > awkward.
> >> > >> > > >
> >> > >> > > > Interesting problem! Many if not most users I personally am
> >> aware
> >> > of
> >> > >> > > > use "_" as a separator in topic names. I am sure that many
> users
> >> > >> would
> >> > >> > > > be quite surprised by this limitation. With that said, I am
> sure
> >> > >> > > > they'd transition accordingly.
> >> > >> > > >
> >> > >> > > > >
> >> > >> > > > > If anyone has better backward-compatible solutions to this,
> >> I'm
> >> > all
> >> > >> > > ears
> >> > >> > > > :)
> >> > >> > > > >
> >> > >> > > > > Gwen
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >> >
> >> > >> >
> >> > >> > --
> >> > >> > Grant Henke
> >> > >> > Solutions Consultant | Cloudera
> >> > >> > ghe...@cloudera.com | twitter.com/gchenke |
> >> > linkedin.com/in/granthenke
> >> > >> >
> >> > >>
> >> > >
> >> > >
> >> > >
> >> > > --
> >> > > Grant Henke
> >> > > Solutions Consultant | Cloudera
> >> > > ghe...@cloudera.com | twitter.com/gchenke |
> linkedin.com/in/granthenke
> >> >
> >>
>

Reply via email to