But I think the flaw in trying to guess what kind of serializer they will
use is when we get it wrong. Basically let's say we guess "String". Say 30%
of the time we will be right and we will save the two configuration lines.
70% of the time we will be wrong and the user gets a super cryptic
ClassCastException: "xyz cannot be cast to [B" (because [B is how java
chooses to display the byte array class just to up the pain), then they
figure out how to subscribe to our mailing list and email us the cryptic
exception, then we explain about how we helpfully set these properties for
them to save them time. :-)

https://www.google.com/?gws_rd=ssl#q=kafka+classcastexception+%22%5BB%22

I think basically we did this experiment with the old clients and the
conclusion is that serialization is something you basically have to think
about to use Kafka and trying to guess just makes things worse.

-Jay

On Sat, Jan 24, 2015 at 2:51 PM, Joe Stein <joe.st...@stealth.ly> wrote:

> Maybe. I think the StringSerialzer could look more like a typical type of
> message.  Instead of encoding being a property it would be more typically
> just written in the bytes.
>
> On Sat, Jan 24, 2015 at 12:12 AM, Jay Kreps <jay.kr...@gmail.com> wrote:
>
> > I don't think so--see if you buy my explanation. We previously defaulted
> > to the byte array serializer and it was a source of unending frustration
> > and confusion. Since it wasn't a required config people just went along
> > plugging in whatever objects they had, and thinking that changing the
> > parametric types would somehow help. Then they would get a class case
> > exception and assume our stuff was somehow busted, not realizing we had
> > helpfully configured a type different from what they were passing in
> under
> > the covers. So I think it is actually good for people to think: how am I
> > serializing my data, and getting that exception will make them ask that
> > question right?
> >
> > -Jay
> >
> > On Fri, Jan 23, 2015 at 9:06 PM, Joe Stein <joe.st...@stealth.ly> wrote:
> >
> >> Should value.serializer in the new java producer be defaulted to
> >> Array[Byte] ?
> >>
> >> I was working on testing some upgrade paths and got this
> >>
> >>     ! return exception in callback when buffer cannot accept message
> >>
> >>       ConfigException: Missing required configuration "value.serializer"
> >> which has no default value. (ConfigDef.java:124)
> >>
> >>   org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:124)
> >>
> >>
> >>
> >>
> org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:48)
> >>
> >>
> >>
> >>
> org.apache.kafka.clients.producer.ProducerConfig.<init>(ProducerConfig.java:235)
> >>
> >>
> >>
> >>
> org.apache.kafka.clients.producer.KafkaProducer.<init>(KafkaProducer.java:129)
> >>
> >>
> >>
> >>
> ly.stealth.testing.BaseSpec$class.createNewKafkaProducer(BaseSpec.scala:42)
> >>
> >>
>  ly.stealth.testing.KafkaSpec.createNewKafkaProducer(KafkaSpec.scala:36)
> >>
> >>
> >>
> >>
> ly.stealth.testing.KafkaSpec$$anonfun$3$$anonfun$apply$37.apply(KafkaSpec.scala:175)
> >>
> >>
> >>
> >>
> ly.stealth.testing.KafkaSpec$$anonfun$3$$anonfun$apply$37.apply(KafkaSpec.scala:170)
> >>
> >>
> >>
> >> On Fri, Jan 23, 2015 at 5:55 PM, Jun Rao <j...@confluent.io> wrote:
> >>
> >> > This is a reminder that the deadline for the vote is this Monday, Jan
> >> 26,
> >> > 7pm PT.
> >> >
> >> > Thanks,
> >> >
> >> > Jun
> >> >
> >> > On Wed, Jan 21, 2015 at 8:28 AM, Jun Rao <j...@confluent.io> wrote:
> >> >
> >> >> This is the second candidate for release of Apache Kafka 0.8.2.0.
> There
> >> >> has been some changes since the 0.8.2 beta release, especially in the
> >> new
> >> >> java producer api and jmx mbean names. It would be great if people
> can
> >> test
> >> >> this out thoroughly.
> >> >>
> >> >> Release Notes for the 0.8.2.0 release
> >> >>
> >> >>
> >>
> https://people.apache.org/~junrao/kafka-0.8.2.0-candidate2/RELEASE_NOTES.html
> >> >>
> >> >> *** Please download, test and vote by Monday, Jan 26h, 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.0-candidate2/
> >> >>
> >> >> * 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.0-candidate2/scaladoc/
> >> >>
> >> >> * java-doc
> >> >> https://people.apache.org/~junrao/kafka-0.8.2.0-candidate2/javadoc/
> >> >>
> >> >> * The tag to be voted upon (off the 0.8.2 branch) is the 0.8.2.0 tag
> >> >>
> >> >>
> >>
> https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tag;h=058d58adef2ab2787e49d8efeefd61bb3d32f99c
> >> >> (commit 0b312a6b9f0833d38eec434bfff4c647c1814564)
> >> >>
> >> >> /*******************************************
> >> >>
> >> >> Thanks,
> >> >>
> >> >> Jun
> >> >>
> >> >>
> >> >  --
> >> > You received this message because you are subscribed to the Google
> >> Groups
> >> > "kafka-clients" group.
> >> > To unsubscribe from this group and stop receiving emails from it, send
> >> an
> >> > email to kafka-clients+unsubscr...@googlegroups.com.
> >> > To post to this group, send email to kafka-clie...@googlegroups.com.
> >> > Visit this group at http://groups.google.com/group/kafka-clients.
> >> > To view this discussion on the web visit
> >> >
> >>
> https://groups.google.com/d/msgid/kafka-clients/CAFc58G83a%3DsvoKkkB3476kpbcQ8p0Fob6vtJYj9CgxMACvvEEQ%40mail.gmail.com
> >> > <
> >>
> https://groups.google.com/d/msgid/kafka-clients/CAFc58G83a%3DsvoKkkB3476kpbcQ8p0Fob6vtJYj9CgxMACvvEEQ%40mail.gmail.com?utm_medium=email&utm_source=footer
> >> >
> >> > .
> >> >
> >> > For more options, visit https://groups.google.com/d/optout.
> >> >
> >>
> >
> >  --
> > You received this message because you are subscribed to the Google Groups
> > "kafka-clients" group.
> > To unsubscribe from this group and stop receiving emails from it, send an
> > email to kafka-clients+unsubscr...@googlegroups.com.
> > To post to this group, send email to kafka-clie...@googlegroups.com.
> > Visit this group at http://groups.google.com/group/kafka-clients.
> > To view this discussion on the web visit
> >
> https://groups.google.com/d/msgid/kafka-clients/CAOeJiJhB_ncKnk48Jw3TbpbULzq6oZRZSKDBOXdid%2BV_WzHuhQ%40mail.gmail.com
> > <
> https://groups.google.com/d/msgid/kafka-clients/CAOeJiJhB_ncKnk48Jw3TbpbULzq6oZRZSKDBOXdid%2BV_WzHuhQ%40mail.gmail.com?utm_medium=email&utm_source=footer
> >
> > .
> >
> > For more options, visit https://groups.google.com/d/optout.
> >
>

Reply via email to