[ https://issues.apache.org/jira/browse/KAFKA-1729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14290011#comment-14290011 ]
Joel Koshy commented on KAFKA-1729: ----------------------------------- Ideally yes because it makes Java access unwieldy but not a blocker because I think these are pre-existing even in 0.8.1 and there are work-arounds. Here is a brief summary: * https://github.com/apache/kafka/blob/0.8.1/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala The OffsetCommitResponse returns a map of [TopicAndPartition, (scala)Short]. JavaConversions apparently does not convert the individual values in the map to a Java short. E.g., if you change the signature of errors() in OffsetCommitResponse to return Map[TopicAndPartition, java.lang.Short] it won't compile. The correct fix I think is to do something similar to javaapi.FetchResponse - where we provide an explicit lookup method errorCode(topicAndPartition) and a hasError boolean method. Anyway, the workaround is to just fetch the object from the map and cast it to short. So (for example): (java.lang.Short) errors.get(topicAndPartition) * A lot of the requests don't provide default for fields such as clientId, correlationId and even version. Well there are defaults but the defaults are inaccessible from Java. We should ideally provide alternate constructors. * It would be useful to add readFrom wrapper methods in the javaapi versions. Otherwise if you want to read from a blocking channel you would need to do something like this: new kafka.javaapi.ConsumerMetadataResponse(kafka.api.ConsumerMetadataResponse.readFrom(channel.receive().buffer())); > add doc for Kafka-based offset management in 0.8.2 > -------------------------------------------------- > > Key: KAFKA-1729 > URL: https://issues.apache.org/jira/browse/KAFKA-1729 > Project: Kafka > Issue Type: Sub-task > Reporter: Jun Rao > Assignee: Joel Koshy > Fix For: 0.8.2 > > Attachments: KAFKA-1782-doc-v1.patch, KAFKA-1782-doc-v2.patch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)