----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29072/#review66523 -----------------------------------------------------------
I'm a little late to this party but I think there are a couple of issues we should address before the release: 1. Currently the producer defaults to "org.apache.kafka.clients.consumer.ByteArrayDeserializer". It should not take a default. Defaulting to byte array serializer was a huge huge source of confusion with the old producer because people would try to put in objects of other types (after changing the parameteric types, thinking that mattered) and then get a very confusing error message. By having no default and requiring the user to set this themselves they will then not be confused. 2. The serializer and deserializer interface are in with the producer and consumer respectively, but the SerializationException and DeserializationException are under common. The serializers will need to be shared more broadly than just the consumer and producer--we may end up serializing other things. These should all go under org.apache.kafka.common.serialization. I also recommend we have a single SerializationException (no point in distinguishing serialization vs deserialization). 3. We should include a StringSerializer/StringDeserializer 4. We should implement the ByteArray(De)Serializer and String(De)Serializer as <Object, Object> so that we can do the type check in the serializer and provide good error messages if the wrong type is provided (e.g. "You are using the ByteArraySerializer which expects only byte[] but have provided an object that is of type X, probably you have configured the wrong serializer and need to check your configuration.") - Jay Kreps On Dec. 17, 2014, 5:47 p.m., Jun Rao wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/29072/ > ----------------------------------------------------------- > > (Updated Dec. 17, 2014, 5:47 p.m.) > > > Review request for kafka. > > > Bugs: kafka-1797 > https://issues.apache.org/jira/browse/kafka-1797 > > > Repository: kafka > > > Description > ------- > > fix imports > > > address Neha's comments > > > Diffs > ----- > > > clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java > 227f5646ee708af1b861c15237eda2140cfd4900 > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java > 46efc0c8483acacf42b2984ac3f3b9e0a4566187 > > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java > f026ae41ce8203928e411f049002851952af5d65 > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java > 436d8a479166eda29f2672b50fc99f288bbe3fa9 > > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java > 2ecfc8aaea90a7353bd0dabc4c0ebcc6fd9535ec > clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java > fe93afa24fc20b03830f1d190a276041d15bd3b9 > clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java > c3aad3b4d6b677f759583f309061193f2f109250 > > clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java > 32f444ebbd27892275af7a0947b86a6b8317a374 > clients/src/main/java/org/apache/kafka/clients/producer/Producer.java > 36e8398416036cab84faad1f07159e5adefd8086 > clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java > 9095caf0db1e41a4acb4216fb197626fbd85b806 > clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java > c3181b368b6cf15e7134b04e8ff5655a9321ee0b > clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java > PRE-CREATION > > clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java > 40e8234f8771098b097bf757a86d5ac98604df5e > > clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java > PRE-CREATION > > clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java > PRE-CREATION > core/src/main/scala/kafka/producer/BaseProducer.scala > b0207930dd0543f2c51f0b35002e13bf104340ff > core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala > 4b5b823b85477394cd50eb2a66877a3b8b35b57f > core/src/main/scala/kafka/tools/MirrorMaker.scala > f399105087588946987bbc84e3759935d9498b6a > core/src/main/scala/kafka/tools/ReplayLogProducer.scala > 3393a3dd574ac45a27bf7eda646b737146c55038 > core/src/main/scala/kafka/tools/TestEndToEndLatency.scala > 67196f30af1cfcd40ded20ca970082b78504f6af > core/src/main/scala/kafka/tools/TestLogCleaning.scala > 1d4ea93f2ba8d4d4d47a307cd47f54a15d3d30dd > core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala > 6379f2b60af797b084981c94fd84b3d7740aa8a5 > core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala > 209a409cb47eb24f83cee79f4e064dbc5f5e9d62 > core/src/test/scala/integration/kafka/api/ProducerSendTest.scala > d407af9144ef6930d737a6dcf23591c1f6342f87 > core/src/test/scala/unit/kafka/utils/TestUtils.scala > 0da774d0ed015bdc0461b854e3540ee6e48d1838 > > Diff: https://reviews.apache.org/r/29072/diff/ > > > Testing > ------- > > > Thanks, > > Jun Rao > >