KAFKA-2983: Remove Scala consumers and related code (#5230)

- Removed Scala consumers (`SimpleConsumer` and `ZooKeeperConsumerConnector`)
and their tests.
- Removed Scala request/response/message classes.
- Removed any mention of new consumer or new producer in the code
with the exception of MirrorMaker where the new.consumer option was
never deprecated so we have to keep it for now. The non-code
documentation has not been updated either, that will be done
separately.
- Removed a number of tools that only made sense in the context
of the Scala consumers (see upgrade notes).
- Updated some tools that worked with both Scala and Java consumers
so that they only support the latter (see upgrade notes).
- Removed `BaseConsumer` and related classes apart from `BaseRecord`
which is used in `MirrorMakerMessageHandler`. The latter is a pluggable
interface so effectively public API.
- Removed `ZkUtils` methods that were only used by the old consumers.
- Removed `ZkUtils.registerBroker` and `ZKCheckedEphemeral` since
the broker now uses the methods in `KafkaZkClient` and no-one else
should be using that method.
- Updated system tests so that they don't use the Scala consumers except
for multi-version tests.
- Updated LogDirFailureTest so that the consumer offsets topic would
continue to be available after all the failures. This was necessary for it
to work with the Java consumer.
- Some multi-version system tests had not been updated to include
recently released Kafka versions, fixed it.
- Updated findBugs and checkstyle configs not to refer to deleted
classes and packages.

Reviewers: Dong Lin <lindong28@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
This commit is contained in:
Ismael Juma 2018-06-19 07:32:54 -07:00 committed by GitHub
parent e8955f731e
commit cc4dce94af
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
207 changed files with 673 additions and 14927 deletions

View File

@ -1,17 +0,0 @@
#!/bin/bash
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
exec $(dirname $0)/kafka-run-class.sh kafka.tools.SimpleConsumerShell "$@"

View File

@ -1,17 +0,0 @@
@echo off
rem Licensed to the Apache Software Foundation (ASF) under one or more
rem contributor license agreements. See the NOTICE file distributed with
rem this work for additional information regarding copyright ownership.
rem The ASF licenses this file to You under the Apache License, Version 2.0
rem (the "License"); you may not use this file except in compliance with
rem the License. You may obtain a copy of the License at
rem
rem http://www.apache.org/licenses/LICENSE-2.0
rem
rem Unless required by applicable law or agreed to in writing, software
rem distributed under the License is distributed on an "AS IS" BASIS,
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
rem See the License for the specific language governing permissions and
rem limitations under the License.
"%~dp0kafka-run-class.bat" kafka.tools.SimpleConsumerShell %*

View File

@ -38,26 +38,9 @@
<allow pkg="kafka.serializer" />
<allow pkg="org.apache.kafka.common" />
<subpackage name="javaapi">
<subpackage name="consumer">
<allow pkg="kafka.consumer" />
</subpackage>
<subpackage name="message">
<allow pkg="kafka.message" />
</subpackage>
<subpackage name="producer">
<allow pkg="kafka.producer" />
</subpackage>
</subpackage>
<subpackage name="tools">
<allow pkg="org.apache.kafka.clients.admin" />
<allow pkg="kafka.admin" />
<allow pkg="kafka.javaapi" />
<allow pkg="kafka.producer" />
<allow pkg="kafka.consumer" />
<allow pkg="joptsimple" />
<allow pkg="org.apache.kafka.clients.consumer" />
<allow class="javax.xml.datatype.Duration" />
@ -71,9 +54,6 @@
<subpackage name="examples">
<allow pkg="org.apache.kafka.clients" />
<allow pkg="kafka.api" />
<allow pkg="kafka.javaapi" />
<allow pkg="kafka.message" />
</subpackage>
</import-control>

View File

@ -19,7 +19,6 @@ package org.apache.kafka.common.network;
import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.ScatteringByteChannel;
import org.apache.kafka.common.memory.MemoryPool;
import org.slf4j.Logger;
@ -90,33 +89,6 @@ public class NetworkReceive implements Receive {
}
public long readFrom(ScatteringByteChannel channel) throws IOException {
return readFromReadableChannel(channel);
}
@Override
public boolean requiredMemoryAmountKnown() {
return requestedBufferSize != -1;
}
@Override
public boolean memoryAllocated() {
return buffer != null;
}
@Override
public void close() throws IOException {
if (buffer != null && buffer != EMPTY_BUFFER) {
memoryPool.release(buffer);
buffer = null;
}
}
// Need a method to read from ReadableByteChannel because BlockingChannel requires read with timeout
// See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work
// This can go away after we get rid of BlockingChannel
@Deprecated
public long readFromReadableChannel(ReadableByteChannel channel) throws IOException {
int read = 0;
if (size.hasRemaining()) {
int bytesRead = channel.read(size);
@ -151,6 +123,25 @@ public class NetworkReceive implements Receive {
return read;
}
@Override
public boolean requiredMemoryAmountKnown() {
return requestedBufferSize != -1;
}
@Override
public boolean memoryAllocated() {
return buffer != null;
}
@Override
public void close() throws IOException {
if (buffer != null && buffer != EMPTY_BUFFER) {
memoryPool.release(buffer);
buffer = null;
}
}
public ByteBuffer payload() {
return this.buffer;
}

View File

@ -376,65 +376,6 @@ object AdminUtils extends Logging with AdminUtilities {
}
}
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
def isConsumerGroupActive(zkUtils: ZkUtils, group: String) = {
zkUtils.getConsumersInGroup(group).nonEmpty
}
/**
* Delete the whole directory of the given consumer group if the group is inactive.
*
* @param zkUtils Zookeeper utilities
* @param group Consumer group
* @return whether or not we deleted the consumer group information
*/
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
def deleteConsumerGroupInZK(zkUtils: ZkUtils, group: String) = {
if (!isConsumerGroupActive(zkUtils, group)) {
val dir = new ZKGroupDirs(group)
zkUtils.deletePathRecursive(dir.consumerGroupDir)
true
}
else false
}
/**
* Delete the given consumer group's information for the given topic in Zookeeper if the group is inactive.
* If the consumer group consumes no other topics, delete the whole consumer group directory.
*
* @param zkUtils Zookeeper utilities
* @param group Consumer group
* @param topic Topic of the consumer group information we wish to delete
* @return whether or not we deleted the consumer group information for the given topic
*/
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
def deleteConsumerGroupInfoForTopicInZK(zkUtils: ZkUtils, group: String, topic: String) = {
val topics = zkUtils.getTopicsByConsumerGroup(group)
if (topics == Seq(topic)) {
deleteConsumerGroupInZK(zkUtils, group)
}
else if (!isConsumerGroupActive(zkUtils, group)) {
val dir = new ZKGroupTopicDirs(group, topic)
zkUtils.deletePathRecursive(dir.consumerOwnerDir)
zkUtils.deletePathRecursive(dir.consumerOffsetDir)
true
}
else false
}
/**
* Delete every inactive consumer group's information about the given topic in Zookeeper.
*
* @param zkUtils Zookeeper utilities
* @param topic Topic of the consumer group information we wish to delete
*/
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
def deleteAllConsumerGroupInfoForTopicInZK(zkUtils: ZkUtils, topic: String): Set[String] = {
val groups = zkUtils.getAllConsumerGroupsForTopic(topic)
groups.foreach(group => deleteConsumerGroupInfoForTopicInZK(zkUtils, group, topic))
groups
}
def topicExists(zkUtils: ZkUtils, topic: String): Boolean =
zkUtils.pathExists(getTopicPath(topic))

View File

@ -22,28 +22,19 @@ import java.util.{Date, Properties}
import javax.xml.datatype.DatatypeFactory
import joptsimple.{OptionParser, OptionSpec}
import kafka.api.{OffsetFetchRequest, OffsetFetchResponse, OffsetRequest, PartitionOffsetRequestInfo}
import kafka.client.ClientUtils
import kafka.common.{OffsetMetadataAndError, TopicAndPartition}
import kafka.consumer.SimpleConsumer
import kafka.utils._
import kafka.utils.Implicits._
import org.I0Itec.zkclient.exception.ZkNoNodeException
import org.apache.kafka.clients.CommonClientConfigs
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata}
import org.apache.kafka.common.errors.BrokerNotAvailableException
import org.apache.kafka.common.{KafkaException, Node, TopicPartition}
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.security.JaasUtils
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.serialization.StringDeserializer
import org.apache.kafka.common.utils.Utils
import scala.collection.JavaConverters._
import scala.collection.{Seq, Set, mutable}
import scala.collection.{Seq, Set}
object ConsumerGroupCommand extends Logging {
@ -60,15 +51,7 @@ object ConsumerGroupCommand extends Logging {
opts.checkArgs()
val consumerGroupService = {
if (opts.useOldConsumer) {
Console.err.println("Note: This will only show information about consumers that use ZooKeeper (not those using the Java consumer API).")
new ZkConsumerGroupService(opts)
} else {
Console.err.println("Note: This will not show information about old Zookeeper-based consumers.")
new KafkaConsumerGroupService(opts)
}
}
val consumerGroupService = new ConsumerGroupService(opts)
try {
if (opts.options.has(opts.listOpt))
@ -125,31 +108,35 @@ object ConsumerGroupCommand extends Logging {
}
}
protected case class PartitionAssignmentState(group: String, coordinator: Option[Node], topic: Option[String],
private[admin] case class PartitionAssignmentState(group: String, coordinator: Option[Node], topic: Option[String],
partition: Option[Int], offset: Option[Long], lag: Option[Long],
consumerId: Option[String], host: Option[String],
clientId: Option[String], logEndOffset: Option[Long])
protected case class MemberAssignmentState(group: String, consumerId: String, host: String, clientId: String,
private[admin] case class MemberAssignmentState(group: String, consumerId: String, host: String, clientId: String,
numPartitions: Int, assignment: List[TopicPartition])
protected case class GroupState(group: String, coordinator: Node, assignmentStrategy: String, state: String, numMembers: Int)
private[admin] case class GroupState(group: String, coordinator: Node, assignmentStrategy: String, state: String, numMembers: Int)
sealed trait ConsumerGroupService {
class ConsumerGroupService(val opts: ConsumerGroupCommandOptions) {
def listGroups(): List[String]
private val adminClient = createAdminClient()
// `consumer` is only needed for `describe`, so we instantiate it lazily
private var consumer: KafkaConsumer[String, String] = _
def listGroups(): List[String] = {
adminClient.listAllConsumerGroupsFlattened().map(_.groupId)
}
private def shouldPrintMemberState(group: String, state: Option[String], numRows: Option[Int]): Boolean = {
// numRows contains the number of data rows, if any, compiled from the API call in the caller method.
// if it's undefined or 0, there is no relevant group information to display.
numRows match {
case None =>
// applies to both old and new consumer
printError(s"The consumer group '$group' does not exist.")
false
case Some(num) =>
opts.useOldConsumer || {
state match {
case Some(num) => state match {
case Some("Dead") =>
printError(s"Consumer group '$group' does not exist.")
case Some("Empty") =>
@ -164,7 +151,6 @@ object ConsumerGroupCommand extends Logging {
!state.contains("Dead") && num > 0
}
}
}
private def size(colOpt: Option[Seq[Object]]): Option[Int] = colOpt.map(_.size)
@ -178,30 +164,22 @@ object ConsumerGroupCommand extends Logging {
consumerAssignments.foreach { consumerAssignment =>
maxTopicLen = Math.max(maxTopicLen, consumerAssignment.topic.getOrElse(MISSING_COLUMN_VALUE).length)
maxConsumerIdLen = Math.max(maxConsumerIdLen, consumerAssignment.consumerId.getOrElse(MISSING_COLUMN_VALUE).length)
if (!opts.useOldConsumer)
maxHostLen = Math.max(maxHostLen, consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE).length)
}
}
print(s"\n%${-maxTopicLen}s %-10s %-15s %-15s %-15s %${-maxConsumerIdLen}s "
.format("TOPIC", "PARTITION", "CURRENT-OFFSET", "LOG-END-OFFSET", "LAG", "CONSUMER-ID"))
if (!opts.useOldConsumer)
print(s"%${-maxHostLen}s %s".format("HOST", "CLIENT-ID"))
println()
println(s"\n%${-maxTopicLen}s %-10s %-15s %-15s %-15s %${-maxConsumerIdLen}s %${-maxHostLen}s %s"
.format("TOPIC", "PARTITION", "CURRENT-OFFSET", "LOG-END-OFFSET", "LAG", "CONSUMER-ID", "HOST", "CLIENT-ID"))
assignments match {
case None => // do nothing
case Some(consumerAssignments) =>
consumerAssignments.foreach { consumerAssignment =>
print(s"%${-maxTopicLen}s %-10s %-15s %-15s %-15s %${-maxConsumerIdLen}s ".format(
println(s"%-${maxTopicLen}s %-10s %-15s %-15s %-15s %-${maxConsumerIdLen}s %-${maxHostLen}s %s".format(
consumerAssignment.topic.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.partition.getOrElse(MISSING_COLUMN_VALUE),
consumerAssignment.offset.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.logEndOffset.getOrElse(MISSING_COLUMN_VALUE),
consumerAssignment.lag.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.consumerId.getOrElse(MISSING_COLUMN_VALUE)))
if (!opts.useOldConsumer)
print(s"%${-maxHostLen}s %s".format(consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE),
consumerAssignment.clientId.getOrElse(MISSING_COLUMN_VALUE)))
println()
consumerAssignment.lag.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.consumerId.getOrElse(MISSING_COLUMN_VALUE),
consumerAssignment.host.getOrElse(MISSING_COLUMN_VALUE), consumerAssignment.clientId.getOrElse(MISSING_COLUMN_VALUE)))
}
}
}
@ -250,7 +228,6 @@ object ConsumerGroupCommand extends Logging {
}
private def printState(group: String, state: GroupState): Unit = {
// this method is reachable only for the new consumer option (where the given state is always defined)
if (shouldPrintMemberState(group, Some(state.state), Some(1))) {
val coordinator = s"${state.coordinator.host}:${state.coordinator.port} (${state.coordinator.idString})"
val coordinatorColLen = Math.max(25, coordinator.length)
@ -277,22 +254,7 @@ object ConsumerGroupCommand extends Logging {
printState(group, collectGroupState())
}
def close(): Unit
protected def opts: ConsumerGroupCommandOptions
protected def getLogEndOffset(topicPartition: TopicPartition): LogOffsetResult =
getLogEndOffsets(Seq(topicPartition)).getOrElse(topicPartition, LogOffsetResult.Ignore)
protected def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult]
def collectGroupOffsets(): (Option[String], Option[Seq[PartitionAssignmentState]])
def collectGroupMembers(verbose: Boolean): (Option[String], Option[Seq[MemberAssignmentState]]) = throw new UnsupportedOperationException
def collectGroupState(): GroupState = throw new UnsupportedOperationException
protected def collectConsumerAssignment(group: String,
private def collectConsumerAssignment(group: String,
coordinator: Option[Node],
topicPartitions: Seq[TopicPartition],
getPartitionOffset: TopicPartition => Option[Long],
@ -335,224 +297,7 @@ object ConsumerGroupCommand extends Logging {
}.toArray
}
def resetOffsets(): Map[TopicPartition, OffsetAndMetadata] = throw new UnsupportedOperationException
def exportOffsetsToReset(assignmentsToReset: Map[TopicPartition, OffsetAndMetadata]): String = throw new UnsupportedOperationException
def deleteGroups(): Map[String, Errors]
}
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class ZkConsumerGroupService(val opts: ConsumerGroupCommandOptions) extends ConsumerGroupService {
private val zkUtils = {
val zkUrl = opts.options.valueOf(opts.zkConnectOpt)
ZkUtils(zkUrl, 30000, 30000, JaasUtils.isZkSecurityEnabled)
}
def close() {
zkUtils.close()
}
def listGroups(): List[String] = {
zkUtils.getConsumerGroups().toList
}
def deleteGroups(): Map[String, Errors] = {
if (opts.options.has(opts.groupOpt) && opts.options.has(opts.topicOpt))
deleteGroupsInfoForTopic()
else if (opts.options.has(opts.groupOpt))
deleteGroupsInfo()
else if (opts.options.has(opts.topicOpt))
deleteAllGroupsInfoForTopic()
Map()
}
def collectGroupOffsets(): (Option[String], Option[Seq[PartitionAssignmentState]]) = {
val group = opts.options.valueOf(opts.groupOpt)
val props = if (opts.options.has(opts.commandConfigOpt)) Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) else new Properties()
val channelSocketTimeoutMs = props.getProperty("channelSocketTimeoutMs", "600").toInt
val channelRetryBackoffMs = props.getProperty("channelRetryBackoffMsOpt", "300").toInt
if (!zkUtils.getConsumerGroups().contains(group))
return (None, None)
val topics = zkUtils.getTopicsByConsumerGroup(group)
val topicPartitions = getAllTopicPartitions(topics)
var groupConsumerIds = zkUtils.getConsumersInGroup(group)
// mapping of topic partition -> consumer id
val consumerIdByTopicPartition = topicPartitions.map { topicPartition =>
val owner = zkUtils.readDataMaybeNull(new ZKGroupTopicDirs(group, topicPartition.topic).consumerOwnerDir + "/" + topicPartition.partition)._1
topicPartition -> owner.map(o => o.substring(0, o.lastIndexOf('-'))).getOrElse(MISSING_COLUMN_VALUE)
}.toMap
// mapping of consumer id -> list of topic partitions
val consumerTopicPartitions = consumerIdByTopicPartition groupBy{_._2} map {
case (key, value) => (key, value.unzip._1.toArray) }
// mapping of consumer id -> list of subscribed topics
val topicsByConsumerId = zkUtils.getTopicsPerMemberId(group)
var assignmentRows = topicPartitions.flatMap { topicPartition =>
val partitionOffsets = getPartitionOffsets(group, List(topicPartition), channelSocketTimeoutMs, channelRetryBackoffMs)
val consumerId = consumerIdByTopicPartition.get(topicPartition)
// since consumer id is repeated in client id, leave host and client id empty
consumerId.foreach(id => groupConsumerIds = groupConsumerIds.filterNot(_ == id))
collectConsumerAssignment(group, None, List(topicPartition), partitionOffsets.get, consumerId, None, None)
}
assignmentRows ++= groupConsumerIds.sortBy(- consumerTopicPartitions.get(_).size).flatMap { consumerId =>
topicsByConsumerId(consumerId).flatMap { _ =>
// since consumers with no topic partitions are processed here, we pass empty for topic partitions and offsets
// since consumer id is repeated in client id, leave host and client id empty
collectConsumerAssignment(group, None, Array[TopicPartition](), Map[TopicPartition, Option[Long]](), Some(consumerId), None, None)
}
}
(None, Some(assignmentRows))
}
private def getAllTopicPartitions(topics: Seq[String]): Seq[TopicPartition] = {
val topicPartitionMap = zkUtils.getPartitionsForTopics(topics)
topics.flatMap { topic =>
val partitions = topicPartitionMap.getOrElse(topic, Seq.empty)
partitions.map(new TopicPartition(topic, _))
}
}
protected def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = {
topicPartitions.map { topicPartition => (topicPartition,
zkUtils.getLeaderForPartition(topicPartition.topic, topicPartition.partition) match {
case Some(-1) => LogOffsetResult.Unknown
case Some(brokerId) =>
getZkConsumer(brokerId).map { consumer =>
val topicAndPartition = new TopicAndPartition(topicPartition)
val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1)))
val logEndOffset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head
consumer.close()
LogOffsetResult.LogOffset(logEndOffset)
}.getOrElse(LogOffsetResult.Ignore)
case None =>
printError(s"No broker for partition '$topicPartition'")
LogOffsetResult.Ignore
}
)}.toMap
}
private def getPartitionOffsets(group: String,
topicPartitions: Seq[TopicPartition],
channelSocketTimeoutMs: Int,
channelRetryBackoffMs: Int): Map[TopicPartition, Long] = {
val offsetMap = mutable.Map[TopicAndPartition, Long]()
val channel = ClientUtils.channelToOffsetManager(group, zkUtils, channelSocketTimeoutMs, channelRetryBackoffMs)
channel.send(OffsetFetchRequest(group, topicPartitions.map(new TopicAndPartition(_))))
val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload())
offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) =>
offsetAndMetadata match {
case OffsetMetadataAndError.NoOffset =>
val topicDirs = new ZKGroupTopicDirs(group, topicAndPartition.topic)
// this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool
// (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage)
try {
val offset = zkUtils.readData(topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1.toLong
offsetMap.put(topicAndPartition, offset)
} catch {
case z: ZkNoNodeException =>
printError(s"Could not fetch offset from zookeeper for group '$group' partition '$topicAndPartition' due to missing offset data in zookeeper.", Some(z))
}
case offsetAndMetaData if offsetAndMetaData.error == Errors.NONE =>
offsetMap.put(topicAndPartition, offsetAndMetadata.offset)
case _ =>
printError(s"Could not fetch offset from kafka for group '$group' partition '$topicAndPartition' due to ${offsetAndMetadata.error.message}.")
}
}
channel.disconnect()
offsetMap.map { case (topicAndPartition, offset) =>
(new TopicPartition(topicAndPartition.topic, topicAndPartition.partition), offset)
}.toMap
}
private def deleteGroupsInfo(): Map[String, Errors] = {
val groups = opts.options.valuesOf(opts.groupOpt)
groups.asScala.map { group =>
try {
if (AdminUtils.deleteConsumerGroupInZK(zkUtils, group)) {
println(s"Deleted all consumer group information for group '$group' in zookeeper.")
group -> Errors.NONE
}
else {
printError(s"Delete for group '$group' failed because its consumers are still active.")
group -> Errors.NON_EMPTY_GROUP
}
}
catch {
case e: ZkNoNodeException =>
printError(s"Delete for group '$group' failed because group does not exist.", Some(e))
group -> Errors.forException(e)
}
}.toMap
}
private def deleteGroupsInfoForTopic(): Map[String, Errors] = {
val groups = opts.options.valuesOf(opts.groupOpt)
val topic = opts.options.valueOf(opts.topicOpt)
Topic.validate(topic)
groups.asScala.map { group =>
try {
if (AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkUtils, group, topic)) {
println(s"Deleted consumer group information for group '$group' topic '$topic' in zookeeper.")
group -> Errors.NONE
}
else {
printError(s"Delete for group '$group' topic '$topic' failed because its consumers are still active.")
group -> Errors.NON_EMPTY_GROUP
}
}
catch {
case e: ZkNoNodeException =>
printError(s"Delete for group '$group' topic '$topic' failed because group does not exist.", Some(e))
group -> Errors.forException(e)
}
}.toMap
}
private def deleteAllGroupsInfoForTopic(): Map[String, Errors] = {
val topic = opts.options.valueOf(opts.topicOpt)
Topic.validate(topic)
val deletedGroups = AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkUtils, topic)
println(s"Deleted consumer group information for all inactive consumer groups for topic '$topic' in zookeeper.")
deletedGroups.map(_ -> Errors.NONE).toMap
}
private def getZkConsumer(brokerId: Int): Option[SimpleConsumer] = {
try {
zkUtils.getBrokerInfo(brokerId)
.map(_.brokerEndPoint(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)))
.map(endPoint => new SimpleConsumer(endPoint.host, endPoint.port, 10000, 100000, "ConsumerGroupCommand"))
.orElse(throw new BrokerNotAvailableException("Broker id %d does not exist".format(brokerId)))
} catch {
case t: Throwable =>
printError(s"Could not parse broker info due to ${t.getMessage}", Some(t))
None
}
}
}
class KafkaConsumerGroupService(val opts: ConsumerGroupCommandOptions) extends ConsumerGroupService {
private val adminClient = createAdminClient()
// `consumer` is only needed for `describe`, so we instantiate it lazily
private var consumer: KafkaConsumer[String, String] = _
override def listGroups(): List[String] = {
adminClient.listAllConsumerGroupsFlattened().map(_.groupId)
}
override def collectGroupOffsets(): (Option[String], Option[Seq[PartitionAssignmentState]]) = {
private[admin] def collectGroupOffsets(): (Option[String], Option[Seq[PartitionAssignmentState]]) = {
val group = opts.options.valueOf(opts.groupOpt)
val consumerGroupSummary = adminClient.describeConsumerGroup(group, opts.options.valueOf(opts.timeoutMsOpt))
val assignments = consumerGroupSummary.consumers.map { consumers =>
@ -587,7 +332,7 @@ object ConsumerGroupCommand extends Logging {
(Some(consumerGroupSummary.state), assignments)
}
override def collectGroupMembers(verbose: Boolean): (Option[String], Option[Seq[MemberAssignmentState]]) = {
private[admin] def collectGroupMembers(verbose: Boolean): (Option[String], Option[Seq[MemberAssignmentState]]) = {
val group = opts.options.valueOf(opts.groupOpt)
val consumerGroupSummary = adminClient.describeConsumerGroup(group, opts.options.valueOf(opts.timeoutMsOpt))
(Some(consumerGroupSummary.state),
@ -598,14 +343,14 @@ object ConsumerGroupCommand extends Logging {
)
}
override def collectGroupState(): GroupState = {
private[admin] def collectGroupState(): GroupState = {
val group = opts.options.valueOf(opts.groupOpt)
val consumerGroupSummary = adminClient.describeConsumerGroup(group, opts.options.valueOf(opts.timeoutMsOpt))
GroupState(group, consumerGroupSummary.coordinator, consumerGroupSummary.assignmentStrategy,
consumerGroupSummary.state, consumerGroupSummary.consumers.get.size)
}
protected def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = {
private def getLogEndOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = {
val offsets = getConsumer.endOffsets(topicPartitions.asJava)
topicPartitions.map { topicPartition =>
val logEndOffset = offsets.get(topicPartition)
@ -613,7 +358,7 @@ object ConsumerGroupCommand extends Logging {
}.toMap
}
protected def getLogStartOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = {
private def getLogStartOffsets(topicPartitions: Seq[TopicPartition]): Map[TopicPartition, LogOffsetResult] = {
val offsets = getConsumer.beginningOffsets(topicPartitions.asJava)
topicPartitions.map { topicPartition =>
val logStartOffset = offsets.get(topicPartition)
@ -621,7 +366,7 @@ object ConsumerGroupCommand extends Logging {
}.toMap
}
protected def getLogTimestampOffsets(topicPartitions: Seq[TopicPartition], timestamp: java.lang.Long): Map[TopicPartition, LogOffsetResult] = {
private def getLogTimestampOffsets(topicPartitions: Seq[TopicPartition], timestamp: java.lang.Long): Map[TopicPartition, LogOffsetResult] = {
val consumer = getConsumer
consumer.assign(topicPartitions.asJava)
@ -646,13 +391,13 @@ object ConsumerGroupCommand extends Logging {
AdminClient.create(props)
}
private def getConsumer = {
private def getConsumer: KafkaConsumer[String, String] = {
if (consumer == null)
consumer = createNewConsumer()
consumer = createConsumer
consumer
}
private def createNewConsumer(): KafkaConsumer[String, String] = {
private def createConsumer: KafkaConsumer[String, String] = {
val properties = new Properties()
val deserializer = (new StringDeserializer).getClass.getName
val brokerUrl = opts.options.valueOf(opts.bootstrapServerOpt)
@ -668,7 +413,7 @@ object ConsumerGroupCommand extends Logging {
new KafkaConsumer(properties)
}
override def resetOffsets(): Map[TopicPartition, OffsetAndMetadata] = {
def resetOffsets(): Map[TopicPartition, OffsetAndMetadata] = {
val groupId = opts.options.valueOf(opts.groupOpt)
val consumerGroupSummary = adminClient.describeConsumerGroup(groupId, opts.options.valueOf(opts.timeoutMsOpt))
consumerGroupSummary.state match {
@ -834,12 +579,12 @@ object ConsumerGroupCommand extends Logging {
}
}
override def exportOffsetsToReset(assignmentsToReset: Map[TopicPartition, OffsetAndMetadata]): String = {
def exportOffsetsToReset(assignmentsToReset: Map[TopicPartition, OffsetAndMetadata]): String = {
val rows = assignmentsToReset.map { case (k,v) => s"${k.topic},${k.partition},${v.offset}" }(collection.breakOut): List[String]
rows.foldRight("")(_ + "\n" + _)
}
override def deleteGroups(): Map[String, Errors] = {
def deleteGroups(): Map[String, Errors] = {
val groupsToDelete = opts.options.valuesOf(opts.groupOpt).asScala.toList
val result = adminClient.deleteConsumerGroups(groupsToDelete)
val successfullyDeleted = result.filter {
@ -871,9 +616,7 @@ object ConsumerGroupCommand extends Logging {
}
class ConsumerGroupCommandOptions(args: Array[String]) {
val ZkConnectDoc = "REQUIRED (for consumer groups based on the old consumer): The connection string for the zookeeper connection in the form host:port. " +
"Multiple URLS can be given to allow fail-over."
val BootstrapServerDoc = "REQUIRED (for consumer groups based on the new consumer): The server to connect to."
val BootstrapServerDoc = "REQUIRED: The server(s) to connect to."
val GroupDoc = "The consumer group we wish to act on."
val TopicDoc = "The topic whose consumer group information should be deleted or topic whose should be included in the reset offset process. " +
"In `reset-offsets` case, partitions can be specified using this format: `topic1:0,1,2`, where 0,1,2 are the partition to be included in the process. " +
@ -883,12 +626,7 @@ object ConsumerGroupCommand extends Logging {
val DescribeDoc = "Describe consumer group and list offset lag (number of messages not yet processed) related to given group."
val nl = System.getProperty("line.separator")
val DeleteDoc = "Pass in groups to delete topic partition offsets and ownership information " +
"over the entire consumer group. For instance --group g1 --group g2" + nl +
"Pass in groups with a single topic to just delete the given topic's partition offsets and ownership " +
"information for the given consumer groups. For instance --group g1 --group g2 --topic t1" + nl +
"Pass in just a topic to delete the given topic's partition offsets and ownership information " +
"for every consumer group. For instance --topic t1" + nl +
"WARNING: Group deletion only works for old ZK-based consumer groups, and one has to use it carefully to only delete groups that are not active."
"over the entire consumer group. For instance --group g1 --group g2"
val TimeoutMsDoc = "The timeout that can be set for some use cases. For example, it can be used when describing the group " +
"to specify the maximum amount of time in milliseconds to wait before the group stabilizes (when the group is just created, " +
"or is going through some changes)."
@ -921,10 +659,6 @@ object ConsumerGroupCommand extends Logging {
"Example: --bootstrap-server localhost:9092 --describe --group group1 --state"
val parser = new OptionParser(false)
val zkConnectOpt = parser.accepts("zookeeper", ZkConnectDoc)
.withRequiredArg
.describedAs("urls")
.ofType(classOf[String])
val bootstrapServerOpt = parser.accepts("bootstrap-server", BootstrapServerDoc)
.withRequiredArg
.describedAs("server to connect to")
@ -979,21 +713,17 @@ object ConsumerGroupCommand extends Logging {
.ofType(classOf[Long])
val membersOpt = parser.accepts("members", MembersDoc)
.availableIf(describeOpt)
.availableUnless(zkConnectOpt)
val verboseOpt = parser.accepts("verbose", VerboseDoc)
.availableIf(describeOpt)
.availableUnless(zkConnectOpt)
val offsetsOpt = parser.accepts("offsets", OffsetsDoc)
.availableIf(describeOpt)
.availableUnless(zkConnectOpt)
val stateOpt = parser.accepts("state", StateDoc)
.availableIf(describeOpt)
.availableUnless(zkConnectOpt)
parser.mutuallyExclusive(membersOpt, offsetsOpt, stateOpt)
val options = parser.parse(args : _*)
val useOldConsumer = options.has(zkConnectOpt)
val describeOptPresent = options.has(describeOpt)
val allConsumerGroupLevelOpts: Set[OptionSpec[_]] = Set(listOpt, describeOpt, deleteOpt, resetOffsetsOpt)
@ -1002,25 +732,20 @@ object ConsumerGroupCommand extends Logging {
def checkArgs() {
// check required args
if (options.has(timeoutMsOpt) && (!describeOptPresent || useOldConsumer))
debug(s"Option $timeoutMsOpt is applicable only when both $bootstrapServerOpt and $describeOpt are used.")
if (options.has(timeoutMsOpt) && !describeOptPresent)
debug(s"Option $timeoutMsOpt is applicable only when $describeOpt is used.")
if (useOldConsumer) {
if (options.has(bootstrapServerOpt))
CommandLineUtils.printUsageAndDie(parser, s"Option $bootstrapServerOpt is not valid with $zkConnectOpt.")
} else {
CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt)
if (options.has(deleteOpt) && options.has(topicOpt))
CommandLineUtils.printUsageAndDie(parser, s"When deleting a consumer group the option $topicOpt is only " +
s"valid with $zkConnectOpt. The new consumer does not support topic-specific offset deletion from a consumer group.")
}
CommandLineUtils.printUsageAndDie(parser, s"The consumer does not support topic-specific offset " +
"deletion from a consumer group.")
if (describeOptPresent)
CommandLineUtils.checkRequiredArgs(parser, options, groupOpt)
if (options.has(deleteOpt) && !options.has(groupOpt) && !options.has(topicOpt))
CommandLineUtils.printUsageAndDie(parser, s"Option $deleteOpt either takes $groupOpt, $topicOpt, or both")
if (options.has(deleteOpt) && !options.has(groupOpt))
CommandLineUtils.printUsageAndDie(parser, s"Option $deleteOpt takes $groupOpt")
if (options.has(resetOffsetsOpt)) {
if (options.has(dryRunOpt) && options.has(executeOpt))

View File

@ -22,7 +22,7 @@ import java.util.Properties
import joptsimple._
import kafka.common.AdminCommandFailedException
import kafka.utils.Implicits._
import kafka.consumer.Whitelist
import kafka.utils.Whitelist
import kafka.log.LogConfig
import kafka.server.ConfigType
import kafka.utils._

View File

@ -75,26 +75,4 @@ object ApiUtils {
}
}
/**
* Read an integer out of the bytebuffer from the current position and check that it falls within the given
* range. If not, throw KafkaException.
*/
def readIntInRange(buffer: ByteBuffer, name: String, range: (Int, Int)): Int = {
val value = buffer.getInt
if(value < range._1 || value > range._2)
throw new KafkaException(name + " has value " + value + " which is not in the range " + range + ".")
else value
}
/**
* Read a short out of the bytebuffer from the current position and check that it falls within the given
* range. If not, throw KafkaException.
*/
def readShortInRange(buffer: ByteBuffer, name: String, range: (Short, Short)): Short = {
val value = buffer.getShort
if(value < range._1 || value > range._2)
throw new KafkaException(name + " has value " + value + " which is not in the range " + range + ".")
else value
}
}

View File

@ -1,270 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import kafka.utils.nonthreadsafe
import kafka.api.ApiUtils._
import kafka.common.TopicAndPartition
import kafka.consumer.ConsumerConfig
import java.util.concurrent.atomic.AtomicInteger
import java.nio.ByteBuffer
import org.apache.kafka.common.protocol.ApiKeys
import scala.collection.mutable.ArrayBuffer
import scala.util.Random
case class PartitionFetchInfo(offset: Long, fetchSize: Int)
@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0")
object FetchRequest {
private val random = new Random
val CurrentVersion = 3.shortValue
val DefaultMaxWait = 0
val DefaultMinBytes = 0
val DefaultMaxBytes = Int.MaxValue
val DefaultCorrelationId = 0
def readFrom(buffer: ByteBuffer): FetchRequest = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
val replicaId = buffer.getInt
val maxWait = buffer.getInt
val minBytes = buffer.getInt
val maxBytes = if (versionId < 3) DefaultMaxBytes else buffer.getInt
val topicCount = buffer.getInt
val pairs = (1 to topicCount).flatMap(_ => {
val topic = readShortString(buffer)
val partitionCount = buffer.getInt
(1 to partitionCount).map(_ => {
val partitionId = buffer.getInt
val offset = buffer.getLong
val fetchSize = buffer.getInt
(TopicAndPartition(topic, partitionId), PartitionFetchInfo(offset, fetchSize))
})
})
FetchRequest(versionId, correlationId, clientId, replicaId, maxWait, minBytes, maxBytes, Vector(pairs:_*))
}
def shuffle(requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]): Seq[(TopicAndPartition, PartitionFetchInfo)] = {
val groupedByTopic = requestInfo.groupBy { case (tp, _) => tp.topic }.map { case (topic, values) =>
topic -> random.shuffle(values)
}
random.shuffle(groupedByTopic.toSeq).flatMap { case (_, partitions) =>
partitions.map { case (tp, fetchInfo) => tp -> fetchInfo }
}
}
def batchByTopic[T](s: Seq[(TopicAndPartition, T)]): Seq[(String, Seq[(Int, T)])] = {
val result = new ArrayBuffer[(String, ArrayBuffer[(Int, T)])]
s.foreach { case (TopicAndPartition(t, p), value) =>
if (result.isEmpty || result.last._1 != t)
result += (t -> new ArrayBuffer)
result.last._2 += (p -> value)
}
result
}
}
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion,
correlationId: Int = FetchRequest.DefaultCorrelationId,
clientId: String = ConsumerConfig.DefaultClientId,
replicaId: Int = Request.OrdinaryConsumerId,
maxWait: Int = FetchRequest.DefaultMaxWait,
minBytes: Int = FetchRequest.DefaultMinBytes,
maxBytes: Int = FetchRequest.DefaultMaxBytes,
requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)])
extends RequestOrResponse(Some(ApiKeys.FETCH.id)) {
/**
* Partitions the request info into a list of lists (one for each topic) while preserving request info ordering
*/
private type PartitionInfos = Seq[(Int, PartitionFetchInfo)]
private lazy val requestInfoGroupedByTopic: Seq[(String, PartitionInfos)] = FetchRequest.batchByTopic(requestInfo)
/** Public constructor for the clients */
@deprecated("The order of partitions in `requestInfo` is relevant, so this constructor is deprecated in favour of the " +
"one that takes a Seq", since = "0.10.1.0")
def this(correlationId: Int,
clientId: String,
maxWait: Int,
minBytes: Int,
maxBytes: Int,
requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) {
this(versionId = FetchRequest.CurrentVersion,
correlationId = correlationId,
clientId = clientId,
replicaId = Request.OrdinaryConsumerId,
maxWait = maxWait,
minBytes = minBytes,
maxBytes = maxBytes,
requestInfo = FetchRequest.shuffle(requestInfo.toSeq))
}
/** Public constructor for the clients */
def this(correlationId: Int,
clientId: String,
maxWait: Int,
minBytes: Int,
maxBytes: Int,
requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]) {
this(versionId = FetchRequest.CurrentVersion,
correlationId = correlationId,
clientId = clientId,
replicaId = Request.OrdinaryConsumerId,
maxWait = maxWait,
minBytes = minBytes,
maxBytes = maxBytes,
requestInfo = requestInfo)
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(replicaId)
buffer.putInt(maxWait)
buffer.putInt(minBytes)
if (versionId >= 3)
buffer.putInt(maxBytes)
buffer.putInt(requestInfoGroupedByTopic.size) // topic count
requestInfoGroupedByTopic.foreach {
case (topic, partitionFetchInfos) =>
writeShortString(buffer, topic)
buffer.putInt(partitionFetchInfos.size) // partition count
partitionFetchInfos.foreach {
case (partition, PartitionFetchInfo(offset, fetchSize)) =>
buffer.putInt(partition)
buffer.putLong(offset)
buffer.putInt(fetchSize)
}
}
}
def sizeInBytes: Int = {
2 + /* versionId */
4 + /* correlationId */
shortStringLength(clientId) +
4 + /* replicaId */
4 + /* maxWait */
4 + /* minBytes */
(if (versionId >= 3) 4 /* maxBytes */ else 0) +
4 + /* topic count */
requestInfoGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => {
val (topic, partitionFetchInfos) = currTopic
foldedTopics +
shortStringLength(topic) +
4 + /* partition count */
partitionFetchInfos.size * (
4 + /* partition id */
8 + /* offset */
4 /* fetch size */
)
})
}
def isFromFollower = Request.isValidBrokerId(replicaId)
def isFromOrdinaryConsumer = replicaId == Request.OrdinaryConsumerId
def isFromLowLevelConsumer = replicaId == Request.DebuggingConsumerId
def numPartitions = requestInfo.size
override def toString: String = {
describe(true)
}
override def describe(details: Boolean): String = {
val fetchRequest = new StringBuilder
fetchRequest.append("Name: " + this.getClass.getSimpleName)
fetchRequest.append("; Version: " + versionId)
fetchRequest.append("; CorrelationId: " + correlationId)
fetchRequest.append("; ClientId: " + clientId)
fetchRequest.append("; ReplicaId: " + replicaId)
fetchRequest.append("; MaxWait: " + maxWait + " ms")
fetchRequest.append("; MinBytes: " + minBytes + " bytes")
fetchRequest.append("; MaxBytes:" + maxBytes + " bytes")
if(details)
fetchRequest.append("; RequestInfo: " + requestInfo.mkString(","))
fetchRequest.toString()
}
}
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
@nonthreadsafe
class FetchRequestBuilder() {
private val correlationId = new AtomicInteger(0)
private var versionId = FetchRequest.CurrentVersion
private var clientId = ConsumerConfig.DefaultClientId
private var replicaId = Request.OrdinaryConsumerId
private var maxWait = FetchRequest.DefaultMaxWait
private var minBytes = FetchRequest.DefaultMinBytes
private var maxBytes = FetchRequest.DefaultMaxBytes
private val requestMap = new collection.mutable.ArrayBuffer[(TopicAndPartition, PartitionFetchInfo)]
def addFetch(topic: String, partition: Int, offset: Long, fetchSize: Int) = {
requestMap.append((TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)))
this
}
def clientId(clientId: String): FetchRequestBuilder = {
this.clientId = clientId
this
}
/**
* Only for internal use. Clients shouldn't set replicaId.
*/
private[kafka] def replicaId(replicaId: Int): FetchRequestBuilder = {
this.replicaId = replicaId
this
}
def maxWait(maxWait: Int): FetchRequestBuilder = {
this.maxWait = maxWait
this
}
def minBytes(minBytes: Int): FetchRequestBuilder = {
this.minBytes = minBytes
this
}
def maxBytes(maxBytes: Int): FetchRequestBuilder = {
this.maxBytes = maxBytes
this
}
def requestVersion(versionId: Short): FetchRequestBuilder = {
this.versionId = versionId
this
}
def build() = {
val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes,
maxBytes, new ArrayBuffer() ++ requestMap)
requestMap.clear()
fetchRequest
}
}

View File

@ -1,174 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import kafka.common.TopicAndPartition
import kafka.message.{ByteBufferMessageSet, MessageSet}
import kafka.api.ApiUtils._
import org.apache.kafka.common.protocol.Errors
import scala.collection._
object FetchResponsePartitionData {
def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = {
val error = Errors.forCode(buffer.getShort)
val hw = buffer.getLong
val messageSetSize = buffer.getInt
val messageSetBuffer = buffer.slice()
messageSetBuffer.limit(messageSetSize)
buffer.position(buffer.position() + messageSetSize)
new FetchResponsePartitionData(error, hw, new ByteBufferMessageSet(messageSetBuffer))
}
val headerSize =
2 + /* error code */
8 + /* high watermark */
4 /* messageSetSize */
}
case class FetchResponsePartitionData(error: Errors = Errors.NONE, hw: Long = -1L, messages: MessageSet) {
val sizeInBytes = FetchResponsePartitionData.headerSize + messages.sizeInBytes
}
object TopicData {
def readFrom(buffer: ByteBuffer): TopicData = {
val topic = readShortString(buffer)
val partitionCount = buffer.getInt
val topicPartitionDataPairs = (1 to partitionCount).map(_ => {
val partitionId = buffer.getInt
val partitionData = FetchResponsePartitionData.readFrom(buffer)
(partitionId, partitionData)
})
TopicData(topic, Seq(topicPartitionDataPairs:_*))
}
def headerSize(topic: String) =
shortStringLength(topic) +
4 /* partition count */
}
case class TopicData(topic: String, partitionData: Seq[(Int, FetchResponsePartitionData)]) {
val sizeInBytes =
TopicData.headerSize(topic) + partitionData.foldLeft(0)((folded, data) => {
folded + data._2.sizeInBytes + 4
} /*_ + _.sizeInBytes + 4*/)
val headerSize = TopicData.headerSize(topic)
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object FetchResponse {
// The request version is used to determine which fields we can expect in the response
def readFrom(buffer: ByteBuffer, requestVersion: Int): FetchResponse = {
val correlationId = buffer.getInt
val throttleTime = if (requestVersion > 0) buffer.getInt else 0
val topicCount = buffer.getInt
val pairs = (1 to topicCount).flatMap(_ => {
val topicData = TopicData.readFrom(buffer)
topicData.partitionData.map { case (partitionId, partitionData) =>
(TopicAndPartition(topicData.topic, partitionId), partitionData)
}
})
FetchResponse(correlationId, Vector(pairs:_*), requestVersion, throttleTime)
}
type FetchResponseEntry = (Int, FetchResponsePartitionData)
def batchByTopic(data: Seq[(TopicAndPartition, FetchResponsePartitionData)]): Seq[(String, Seq[FetchResponseEntry])] =
FetchRequest.batchByTopic(data)
// Returns the size of the response header
def headerSize(requestVersion: Int): Int = {
val throttleTimeSize = if (requestVersion > 0) 4 else 0
4 + /* correlationId */
4 + /* topic count */
throttleTimeSize
}
// Returns the size of entire fetch response in bytes (including the header size)
def responseSize(dataGroupedByTopic: Seq[(String, Seq[FetchResponseEntry])],
requestVersion: Int): Int = {
headerSize(requestVersion) +
dataGroupedByTopic.foldLeft(0) { case (folded, (topic, partitionDataMap)) =>
val topicData = TopicData(topic, partitionDataMap.map {
case (partitionId, partitionData) => (partitionId, partitionData)
})
folded + topicData.sizeInBytes
}
}
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class FetchResponse(correlationId: Int,
data: Seq[(TopicAndPartition, FetchResponsePartitionData)],
requestVersion: Int = 0,
throttleTimeMs: Int = 0)
extends RequestOrResponse() {
/**
* Partitions the data into a map of maps (one for each topic).
*/
private lazy val dataByTopicAndPartition = data.toMap
lazy val dataGroupedByTopic = FetchResponse.batchByTopic(data)
val headerSizeInBytes = FetchResponse.headerSize(requestVersion)
lazy val sizeInBytes = FetchResponse.responseSize(dataGroupedByTopic, requestVersion)
/*
* Writes the header of the FetchResponse to the input buffer
*/
def writeHeaderTo(buffer: ByteBuffer) = {
buffer.putInt(sizeInBytes)
buffer.putInt(correlationId)
// Include the throttleTime only if the client can read it
if (requestVersion > 0)
buffer.putInt(throttleTimeMs)
buffer.putInt(dataGroupedByTopic.size) // topic count
}
/*
* FetchResponse uses [sendfile](http://man7.org/linux/man-pages/man2/sendfile.2.html)
* api for data transfer through the FetchResponseSend, so `writeTo` aren't actually being used.
* It is implemented as an empty function to conform to `RequestOrResponse.writeTo`
* abstract method signature.
*/
def writeTo(buffer: ByteBuffer): Unit = throw new UnsupportedOperationException
override def describe(details: Boolean): String = toString
private def partitionDataFor(topic: String, partition: Int): FetchResponsePartitionData = {
val topicAndPartition = TopicAndPartition(topic, partition)
dataByTopicAndPartition.get(topicAndPartition) match {
case Some(partitionData) => partitionData
case _ =>
throw new IllegalArgumentException(
"No partition %s in fetch response %s".format(topicAndPartition, this.toString))
}
}
def messageSet(topic: String, partition: Int): ByteBufferMessageSet =
partitionDataFor(topic, partition).messages.asInstanceOf[ByteBufferMessageSet]
def highWatermark(topic: String, partition: Int) = partitionDataFor(topic, partition).hw
def hasError = dataByTopicAndPartition.values.exists(_.error != Errors.NONE)
def error(topic: String, partition: Int) = partitionDataFor(topic, partition).error
}

View File

@ -1,74 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import org.apache.kafka.common.protocol.ApiKeys
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object GroupCoordinatorRequest {
val CurrentVersion = 0.shortValue
val DefaultClientId = ""
def readFrom(buffer: ByteBuffer) = {
// envelope
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = ApiUtils.readShortString(buffer)
// request
val group = ApiUtils.readShortString(buffer)
GroupCoordinatorRequest(group, versionId, correlationId, clientId)
}
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class GroupCoordinatorRequest(group: String,
versionId: Short = GroupCoordinatorRequest.CurrentVersion,
correlationId: Int = 0,
clientId: String = GroupCoordinatorRequest.DefaultClientId)
extends RequestOrResponse(Some(ApiKeys.FIND_COORDINATOR.id)) {
def sizeInBytes =
2 + /* versionId */
4 + /* correlationId */
ApiUtils.shortStringLength(clientId) +
ApiUtils.shortStringLength(group)
def writeTo(buffer: ByteBuffer) {
// envelope
buffer.putShort(versionId)
buffer.putInt(correlationId)
ApiUtils.writeShortString(buffer, clientId)
// consumer metadata request
ApiUtils.writeShortString(buffer, group)
}
def describe(details: Boolean) = {
val consumerMetadataRequest = new StringBuilder
consumerMetadataRequest.append("Name: " + this.getClass.getSimpleName)
consumerMetadataRequest.append("; Version: " + versionId)
consumerMetadataRequest.append("; CorrelationId: " + correlationId)
consumerMetadataRequest.append("; ClientId: " + clientId)
consumerMetadataRequest.append("; Group: " + group)
consumerMetadataRequest.toString()
}
}

View File

@ -1,60 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import kafka.cluster.BrokerEndPoint
import org.apache.kafka.common.protocol.Errors
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object GroupCoordinatorResponse {
val CurrentVersion = 0
private val NoBrokerEndpointOpt = Some(BrokerEndPoint(id = -1, host = "", port = -1))
def readFrom(buffer: ByteBuffer) = {
val correlationId = buffer.getInt
val error = Errors.forCode(buffer.getShort)
val broker = BrokerEndPoint.readFrom(buffer)
val coordinatorOpt = if (error == Errors.NONE)
Some(broker)
else
None
GroupCoordinatorResponse(coordinatorOpt, error, correlationId)
}
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class GroupCoordinatorResponse (coordinatorOpt: Option[BrokerEndPoint], error: Errors, correlationId: Int)
extends RequestOrResponse() {
def sizeInBytes =
4 + /* correlationId */
2 + /* error code */
coordinatorOpt.orElse(GroupCoordinatorResponse.NoBrokerEndpointOpt).get.sizeInBytes
def writeTo(buffer: ByteBuffer) {
buffer.putInt(correlationId)
buffer.putShort(error.code)
coordinatorOpt.orElse(GroupCoordinatorResponse.NoBrokerEndpointOpt).foreach(_.writeTo(buffer))
}
def describe(details: Boolean) = toString
}

View File

@ -1,181 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.common.{OffsetAndMetadata, TopicAndPartition}
import kafka.utils.Logging
import org.apache.kafka.common.protocol.ApiKeys
import scala.collection._
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object OffsetCommitRequest extends Logging {
val CurrentVersion: Short = 2
val DefaultClientId = ""
def readFrom(buffer: ByteBuffer): OffsetCommitRequest = {
// Read values from the envelope
val versionId = buffer.getShort
assert(versionId == 0 || versionId == 1 || versionId == 2,
"Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0, 1 or 2.")
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
// Read the OffsetRequest
val groupId = readShortString(buffer)
// version 1 and 2 specific fields
val groupGenerationId: Int =
if (versionId >= 1)
buffer.getInt
else
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID
val memberId: String =
if (versionId >= 1)
readShortString(buffer)
else
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_MEMBER_ID
// version 2 specific fields
val retentionMs: Long =
if (versionId >= 2)
buffer.getLong
else
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME
val topicCount = buffer.getInt
val pairs = (1 to topicCount).flatMap(_ => {
val topic = readShortString(buffer)
val partitionCount = buffer.getInt
(1 to partitionCount).map(_ => {
val partitionId = buffer.getInt
val offset = buffer.getLong
val timestamp = {
// version 1 specific field
if (versionId == 1)
buffer.getLong
else
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP
}
val metadata = readShortString(buffer)
(TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp))
})
})
OffsetCommitRequest(groupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, memberId, retentionMs)
}
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class OffsetCommitRequest(groupId: String,
requestInfo: immutable.Map[TopicAndPartition, OffsetAndMetadata],
versionId: Short = OffsetCommitRequest.CurrentVersion,
correlationId: Int = 0,
clientId: String = OffsetCommitRequest.DefaultClientId,
groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID,
memberId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_MEMBER_ID,
retentionMs: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME)
extends RequestOrResponse(Some(ApiKeys.OFFSET_COMMIT.id)) {
assert(versionId == 0 || versionId == 1 || versionId == 2,
"Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0, 1 or 2.")
lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
def writeTo(buffer: ByteBuffer) {
// Write envelope
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
// Write OffsetCommitRequest
writeShortString(buffer, groupId) // consumer group
// version 1 and 2 specific data
if (versionId >= 1) {
buffer.putInt(groupGenerationId)
writeShortString(buffer, memberId)
}
// version 2 or above specific data
if (versionId >= 2) {
buffer.putLong(retentionMs)
}
buffer.putInt(requestInfoGroupedByTopic.size) // number of topics
requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError]
writeShortString(buffer, t1._1) // topic
buffer.putInt(t1._2.size) // number of partitions for this topic
t1._2.foreach( t2 => {
buffer.putInt(t2._1.partition)
buffer.putLong(t2._2.offset)
// version 1 specific data
if (versionId == 1)
buffer.putLong(t2._2.commitTimestamp)
writeShortString(buffer, t2._2.metadata)
})
})
}
override def sizeInBytes =
2 + /* versionId */
4 + /* correlationId */
shortStringLength(clientId) +
shortStringLength(groupId) +
(if (versionId >= 1) 4 /* group generation id */ + shortStringLength(memberId) else 0) +
(if (versionId >= 2) 8 /* retention time */ else 0) +
4 + /* topic count */
requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => {
val (topic, offsets) = topicAndOffsets
count +
shortStringLength(topic) + /* topic */
4 + /* number of partitions */
offsets.foldLeft(0)((innerCount, offsetAndMetadata) => {
innerCount +
4 /* partition */ +
8 /* offset */ +
(if (versionId == 1) 8 else 0) /* timestamp */ +
shortStringLength(offsetAndMetadata._2.metadata)
})
})
override def describe(details: Boolean): String = {
val offsetCommitRequest = new StringBuilder
offsetCommitRequest.append("Name: " + this.getClass.getSimpleName)
offsetCommitRequest.append("; Version: " + versionId)
offsetCommitRequest.append("; CorrelationId: " + correlationId)
offsetCommitRequest.append("; ClientId: " + clientId)
offsetCommitRequest.append("; GroupId: " + groupId)
offsetCommitRequest.append("; GroupGenerationId: " + groupGenerationId)
offsetCommitRequest.append("; MemberId: " + memberId)
offsetCommitRequest.append("; RetentionMs: " + retentionMs)
if(details)
offsetCommitRequest.append("; RequestInfo: " + requestInfo.mkString(","))
offsetCommitRequest.toString()
}
override def toString = {
describe(details = true)
}
}

View File

@ -1,82 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import kafka.utils.Logging
import kafka.common.TopicAndPartition
import org.apache.kafka.common.protocol.Errors
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object OffsetCommitResponse extends Logging {
val CurrentVersion: Short = 0
def readFrom(buffer: ByteBuffer): OffsetCommitResponse = {
val correlationId = buffer.getInt
val topicCount = buffer.getInt
val pairs = (1 to topicCount).flatMap(_ => {
val topic = ApiUtils.readShortString(buffer)
val partitionCount = buffer.getInt
(1 to partitionCount).map(_ => {
val partitionId = buffer.getInt
val error = Errors.forCode(buffer.getShort)
(TopicAndPartition(topic, partitionId), error)
})
})
OffsetCommitResponse(Map(pairs:_*), correlationId)
}
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Errors],
correlationId: Int = 0)
extends RequestOrResponse() {
lazy val commitStatusGroupedByTopic = commitStatus.groupBy(_._1.topic)
def hasError = commitStatus.values.exists(_ != Errors.NONE)
def writeTo(buffer: ByteBuffer) {
buffer.putInt(correlationId)
buffer.putInt(commitStatusGroupedByTopic.size)
commitStatusGroupedByTopic.foreach { case(topic, statusMap) =>
ApiUtils.writeShortString(buffer, topic)
buffer.putInt(statusMap.size) // partition count
statusMap.foreach { case(topicAndPartition, error) =>
buffer.putInt(topicAndPartition.partition)
buffer.putShort(error.code)
}
}
}
override def sizeInBytes =
4 + /* correlationId */
4 + /* topic count */
commitStatusGroupedByTopic.foldLeft(0)((count, partitionStatusMap) => {
val (topic, partitionStatus) = partitionStatusMap
count +
ApiUtils.shortStringLength(topic) +
4 + /* partition count */
partitionStatus.size * ( 4 /* partition */ + 2 /* error code */)
})
override def describe(details: Boolean):String = { toString }
}

View File

@ -1,108 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.common.TopicAndPartition
import kafka.utils.Logging
import org.apache.kafka.common.protocol.ApiKeys
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object OffsetFetchRequest extends Logging {
val CurrentVersion: Short = 2
val DefaultClientId = ""
def readFrom(buffer: ByteBuffer): OffsetFetchRequest = {
// Read values from the envelope
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
// Read the OffsetFetchRequest
val consumerGroupId = readShortString(buffer)
val topicCount = buffer.getInt
val pairs = (1 to topicCount).flatMap(_ => {
val topic = readShortString(buffer)
val partitionCount = buffer.getInt
(1 to partitionCount).map(_ => {
val partitionId = buffer.getInt
TopicAndPartition(topic, partitionId)
})
})
OffsetFetchRequest(consumerGroupId, pairs, versionId, correlationId, clientId)
}
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class OffsetFetchRequest(groupId: String,
requestInfo: Seq[TopicAndPartition],
versionId: Short = OffsetFetchRequest.CurrentVersion,
correlationId: Int = 0,
clientId: String = OffsetFetchRequest.DefaultClientId)
extends RequestOrResponse(Some(ApiKeys.OFFSET_FETCH.id)) {
lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic)
def writeTo(buffer: ByteBuffer) {
// Write envelope
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
// Write OffsetFetchRequest
writeShortString(buffer, groupId) // consumer group
buffer.putInt(requestInfoGroupedByTopic.size) // number of topics
requestInfoGroupedByTopic.foreach( t1 => { // (topic, Seq[TopicAndPartition])
writeShortString(buffer, t1._1) // topic
buffer.putInt(t1._2.size) // number of partitions for this topic
t1._2.foreach( t2 => {
buffer.putInt(t2.partition)
})
})
}
override def sizeInBytes =
2 + /* versionId */
4 + /* correlationId */
shortStringLength(clientId) +
shortStringLength(groupId) +
4 + /* topic count */
requestInfoGroupedByTopic.foldLeft(0)((count, t) => {
count + shortStringLength(t._1) + /* topic */
4 + /* number of partitions */
t._2.size * 4 /* partition */
})
override def describe(details: Boolean): String = {
val offsetFetchRequest = new StringBuilder
offsetFetchRequest.append("Name: " + this.getClass.getSimpleName)
offsetFetchRequest.append("; Version: " + versionId)
offsetFetchRequest.append("; CorrelationId: " + correlationId)
offsetFetchRequest.append("; ClientId: " + clientId)
offsetFetchRequest.append("; GroupId: " + groupId)
if (details)
offsetFetchRequest.append("; RequestInfo: " + requestInfo.mkString(","))
offsetFetchRequest.toString()
}
override def toString: String = {
describe(details = true)
}
}

View File

@ -1,107 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.common.{TopicAndPartition, OffsetMetadataAndError}
import kafka.utils.Logging
import org.apache.kafka.common.protocol.Errors
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object OffsetFetchResponse extends Logging {
def readFrom(buffer: ByteBuffer): OffsetFetchResponse = {
readFrom(buffer, OffsetFetchRequest.CurrentVersion)
}
def readFrom(buffer: ByteBuffer, requestVersion: Int): OffsetFetchResponse = {
val correlationId = buffer.getInt
val topicCount = buffer.getInt
val pairs = (1 to topicCount).flatMap(_ => {
val topic = readShortString(buffer)
val partitionCount = buffer.getInt
(1 to partitionCount).map(_ => {
val partitionId = buffer.getInt
val offset = buffer.getLong
val metadata = readShortString(buffer)
val error = Errors.forCode(buffer.getShort)
(TopicAndPartition(topic, partitionId), OffsetMetadataAndError(offset, metadata, error))
})
})
val error = requestVersion match {
case 0 | 1 => Errors.NONE
case _ => Errors.forCode(buffer.getShort)
}
OffsetFetchResponse(Map(pairs:_*), requestVersion, correlationId, error)
}
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class OffsetFetchResponse(requestInfo: Map[TopicAndPartition, OffsetMetadataAndError],
requestVersion: Int = OffsetFetchRequest.CurrentVersion,
correlationId: Int = 0,
error: Errors = Errors.NONE)
extends RequestOrResponse() {
lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
def writeTo(buffer: ByteBuffer) {
buffer.putInt(correlationId)
buffer.putInt(requestInfoGroupedByTopic.size) // number of topics
requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError]
writeShortString(buffer, t1._1) // topic
buffer.putInt(t1._2.size) // number of partitions for this topic
t1._2.foreach( t2 => { // TopicAndPartition -> OffsetMetadataAndError
buffer.putInt(t2._1.partition)
buffer.putLong(t2._2.offset)
writeShortString(buffer, t2._2.metadata)
buffer.putShort(t2._2.error.code)
})
})
// the top level error_code was introduced in v2
if (requestVersion > 1)
buffer.putShort(error.code)
}
override def sizeInBytes =
4 + /* correlationId */
4 + /* topic count */
requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => {
val (topic, offsets) = topicAndOffsets
count +
shortStringLength(topic) + /* topic */
4 + /* number of partitions */
offsets.foldLeft(0)((innerCount, offsetsAndMetadata) => {
innerCount +
4 /* partition */ +
8 /* offset */ +
shortStringLength(offsetsAndMetadata._2.metadata) +
2 /* error */
})
}) +
(if (requestVersion > 1) 2 else 0) /* error */
override def describe(details: Boolean):String = { toString }
}

View File

@ -1,126 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.common.TopicAndPartition
import org.apache.kafka.common.protocol.ApiKeys
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object OffsetRequest {
val CurrentVersion = 0.shortValue
val DefaultClientId = ""
val SmallestTimeString = "smallest"
val LargestTimeString = "largest"
val LatestTime = -1L
val EarliestTime = -2L
def readFrom(buffer: ByteBuffer): OffsetRequest = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = readShortString(buffer)
val replicaId = buffer.getInt
val topicCount = buffer.getInt
val pairs = (1 to topicCount).flatMap(_ => {
val topic = readShortString(buffer)
val partitionCount = buffer.getInt
(1 to partitionCount).map(_ => {
val partitionId = buffer.getInt
val time = buffer.getLong
val maxNumOffsets = buffer.getInt
(TopicAndPartition(topic, partitionId), PartitionOffsetRequestInfo(time, maxNumOffsets))
})
})
OffsetRequest(Map(pairs:_*), versionId= versionId, clientId = clientId, correlationId = correlationId, replicaId = replicaId)
}
}
case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int)
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo],
versionId: Short = OffsetRequest.CurrentVersion,
correlationId: Int = 0,
clientId: String = OffsetRequest.DefaultClientId,
replicaId: Int = Request.OrdinaryConsumerId)
extends RequestOrResponse(Some(ApiKeys.LIST_OFFSETS.id)) {
def this(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], correlationId: Int, replicaId: Int) = this(requestInfo, OffsetRequest.CurrentVersion, correlationId, OffsetRequest.DefaultClientId, replicaId)
lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(replicaId)
buffer.putInt(requestInfoGroupedByTopic.size) // topic count
requestInfoGroupedByTopic.foreach {
case((topic, partitionInfos)) =>
writeShortString(buffer, topic)
buffer.putInt(partitionInfos.size) // partition count
partitionInfos.foreach {
case (TopicAndPartition(_, partition), partitionInfo) =>
buffer.putInt(partition)
buffer.putLong(partitionInfo.time)
buffer.putInt(partitionInfo.maxNumOffsets)
}
}
}
def sizeInBytes =
2 + /* versionId */
4 + /* correlationId */
shortStringLength(clientId) +
4 + /* replicaId */
4 + /* topic count */
requestInfoGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => {
val (topic, partitionInfos) = currTopic
foldedTopics +
shortStringLength(topic) +
4 + /* partition count */
partitionInfos.size * (
4 + /* partition */
8 + /* time */
4 /* maxNumOffsets */
)
})
def isFromOrdinaryClient = replicaId == Request.OrdinaryConsumerId
def isFromDebuggingClient = replicaId == Request.DebuggingConsumerId
override def toString: String = {
describe(true)
}
override def describe(details: Boolean): String = {
val offsetRequest = new StringBuilder
offsetRequest.append("Name: " + this.getClass.getSimpleName)
offsetRequest.append("; Version: " + versionId)
offsetRequest.append("; CorrelationId: " + correlationId)
offsetRequest.append("; ClientId: " + clientId)
offsetRequest.append("; ReplicaId: " + replicaId)
if(details)
offsetRequest.append("; RequestInfo: " + requestInfo.mkString(","))
offsetRequest.toString()
}
}

View File

@ -1,102 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import kafka.common.TopicAndPartition
import kafka.api.ApiUtils._
import org.apache.kafka.common.protocol.Errors
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object OffsetResponse {
def readFrom(buffer: ByteBuffer): OffsetResponse = {
val correlationId = buffer.getInt
val numTopics = buffer.getInt
val pairs = (1 to numTopics).flatMap(_ => {
val topic = readShortString(buffer)
val numPartitions = buffer.getInt
(1 to numPartitions).map(_ => {
val partition = buffer.getInt
val error = Errors.forCode(buffer.getShort)
val numOffsets = buffer.getInt
val offsets = (1 to numOffsets).map(_ => buffer.getLong)
(TopicAndPartition(topic, partition), PartitionOffsetsResponse(error, offsets))
})
})
OffsetResponse(correlationId, Map(pairs:_*))
}
}
case class PartitionOffsetsResponse(error: Errors, offsets: Seq[Long]) {
override def toString: String = {
new String("error: " + error.exceptionName + " offsets: " + offsets.mkString)
}
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class OffsetResponse(correlationId: Int,
partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse])
extends RequestOrResponse() {
lazy val offsetsGroupedByTopic = partitionErrorAndOffsets.groupBy(_._1.topic)
def hasError = partitionErrorAndOffsets.values.exists(_.error != Errors.NONE)
val sizeInBytes = {
4 + /* correlation id */
4 + /* topic count */
offsetsGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => {
val (topic, errorAndOffsetsMap) = currTopic
foldedTopics +
shortStringLength(topic) +
4 + /* partition count */
errorAndOffsetsMap.foldLeft(0)((foldedPartitions, currPartition) => {
foldedPartitions +
4 + /* partition id */
2 + /* partition error */
4 + /* offset array length */
currPartition._2.offsets.size * 8 /* offset */
})
})
}
def writeTo(buffer: ByteBuffer) {
buffer.putInt(correlationId)
buffer.putInt(offsetsGroupedByTopic.size) // topic count
offsetsGroupedByTopic.foreach {
case((topic, errorAndOffsetsMap)) =>
writeShortString(buffer, topic)
buffer.putInt(errorAndOffsetsMap.size) // partition count
errorAndOffsetsMap.foreach {
case((TopicAndPartition(_, partition), errorAndOffsets)) =>
buffer.putInt(partition)
buffer.putShort(errorAndOffsets.error.code)
buffer.putInt(errorAndOffsets.offsets.size) // offset array length
errorAndOffsets.offsets.foreach(buffer.putLong(_))
}
}
}
override def describe(details: Boolean):String = { toString }
}

View File

@ -15,11 +15,13 @@
* limitations under the License.
*/
package kafka.consumer
package kafka.api
@deprecated("This trait has been deprecated and will be removed in a future release.", "0.11.0.0")
trait TopicEventHandler[T] {
def handleTopicEvent(allTopics: Seq[T])
object Request {
val OrdinaryConsumerId: Int = -1
val DebuggingConsumerId: Int = -2
val FutureLocalReplicaId: Int = -3
// Broker ids are non-negative int.
def isValidBrokerId(brokerId: Int): Boolean = brokerId >= 0
}

View File

@ -1,48 +0,0 @@
package kafka.api
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.nio._
import kafka.network.RequestChannel
import kafka.utils.Logging
object Request {
val OrdinaryConsumerId: Int = -1
val DebuggingConsumerId: Int = -2
val FutureLocalReplicaId: Int = -3
// Broker ids are non-negative int.
def isValidBrokerId(brokerId: Int): Boolean = brokerId >= 0
}
abstract class RequestOrResponse(val requestId: Option[Short] = None) extends Logging {
def sizeInBytes: Int
def writeTo(buffer: ByteBuffer): Unit
def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {}
/* The purpose of this API is to return a string description of the Request mainly for the purpose of request logging.
* This API has no meaning for a Response object.
* @param details If this is false, omit the parts of the request description that are proportional to the number of
* topics or partitions. This is mainly to control the amount of request logging. */
def describe(details: Boolean): String
}

View File

@ -1,153 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import kafka.cluster.BrokerEndPoint
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.utils.Logging
import org.apache.kafka.common.protocol.Errors
object TopicMetadata {
val NoLeaderNodeId = -1
def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndPoint]): TopicMetadata = {
val error = Errors.forCode(readShortInRange(buffer, "error code", (-1, Short.MaxValue)))
val topic = readShortString(buffer)
val numPartitions = readIntInRange(buffer, "number of partitions", (0, Int.MaxValue))
val partitionsMetadata: Array[PartitionMetadata] = new Array[PartitionMetadata](numPartitions)
for(i <- 0 until numPartitions) {
val partitionMetadata = PartitionMetadata.readFrom(buffer, brokers)
partitionsMetadata(i) = partitionMetadata
}
new TopicMetadata(topic, partitionsMetadata, error)
}
}
case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], error: Errors = Errors.NONE) extends Logging {
def sizeInBytes: Int = {
2 /* error code */ +
shortStringLength(topic) +
4 + partitionsMetadata.map(_.sizeInBytes).sum /* size and partition data array */
}
def writeTo(buffer: ByteBuffer) {
/* error code */
buffer.putShort(error.code)
/* topic */
writeShortString(buffer, topic)
/* number of partitions */
buffer.putInt(partitionsMetadata.size)
partitionsMetadata.foreach(m => m.writeTo(buffer))
}
override def toString: String = {
val topicMetadataInfo = new StringBuilder
topicMetadataInfo.append("{TopicMetadata for topic %s -> ".format(topic))
error match {
case Errors.NONE =>
partitionsMetadata.foreach { partitionMetadata =>
partitionMetadata.error match {
case Errors.NONE =>
topicMetadataInfo.append("\nMetadata for partition [%s,%d] is %s".format(topic,
partitionMetadata.partitionId, partitionMetadata.toString()))
case Errors.REPLICA_NOT_AVAILABLE =>
// this error message means some replica other than the leader is not available. The consumer
// doesn't care about non leader replicas, so ignore this
topicMetadataInfo.append("\nMetadata for partition [%s,%d] is %s".format(topic,
partitionMetadata.partitionId, partitionMetadata.toString()))
case error: Errors =>
topicMetadataInfo.append("\nMetadata for partition [%s,%d] is not available due to %s".format(topic,
partitionMetadata.partitionId, error.exceptionName))
}
}
case error: Errors =>
topicMetadataInfo.append("\nNo partition metadata for topic %s due to %s".format(topic,
error.exceptionName))
}
topicMetadataInfo.append("}")
topicMetadataInfo.toString()
}
}
object PartitionMetadata {
def readFrom(buffer: ByteBuffer, brokers: Map[Int, BrokerEndPoint]): PartitionMetadata = {
val error = Errors.forCode(readShortInRange(buffer, "error code", (-1, Short.MaxValue)))
val partitionId = readIntInRange(buffer, "partition id", (0, Int.MaxValue)) /* partition id */
val leaderId = buffer.getInt
val leader = brokers.get(leaderId)
/* list of all replicas */
val numReplicas = readIntInRange(buffer, "number of all replicas", (0, Int.MaxValue))
val replicaIds = (0 until numReplicas).map(_ => buffer.getInt)
val replicas = replicaIds.map(brokers)
/* list of in-sync replicas */
val numIsr = readIntInRange(buffer, "number of in-sync replicas", (0, Int.MaxValue))
val isrIds = (0 until numIsr).map(_ => buffer.getInt)
val isr = isrIds.map(brokers)
new PartitionMetadata(partitionId, leader, replicas, isr, error)
}
}
case class PartitionMetadata(partitionId: Int,
leader: Option[BrokerEndPoint],
replicas: Seq[BrokerEndPoint],
isr: Seq[BrokerEndPoint] = Seq.empty,
error: Errors = Errors.NONE) extends Logging {
def sizeInBytes: Int = {
2 /* error code */ +
4 /* partition id */ +
4 /* leader */ +
4 + 4 * replicas.size /* replica array */ +
4 + 4 * isr.size /* isr array */
}
def writeTo(buffer: ByteBuffer) {
buffer.putShort(error.code)
buffer.putInt(partitionId)
/* leader */
val leaderId = leader.fold(TopicMetadata.NoLeaderNodeId)(leader => leader.id)
buffer.putInt(leaderId)
/* number of replicas */
buffer.putInt(replicas.size)
replicas.foreach(r => buffer.putInt(r.id))
/* number of in-sync replicas */
buffer.putInt(isr.size)
isr.foreach(r => buffer.putInt(r.id))
}
override def toString: String = {
val partitionMetadataString = new StringBuilder
partitionMetadataString.append("\tpartition " + partitionId)
partitionMetadataString.append("\tleader: " + leader.getOrElse("none"))
partitionMetadataString.append("\treplicas: " + replicas.mkString(","))
partitionMetadataString.append("\tisr: " + isr.mkString(","))
partitionMetadataString.append("\tisUnderReplicated: " + (isr.size < replicas.size))
partitionMetadataString.toString()
}
}

View File

@ -1,72 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import java.nio.ByteBuffer
import kafka.api.ApiUtils._
import kafka.utils.Logging
import org.apache.kafka.common.protocol.ApiKeys
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object TopicMetadataRequest extends Logging {
val CurrentVersion = 0.shortValue
val DefaultClientId = ""
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class TopicMetadataRequest(versionId: Short,
correlationId: Int,
clientId: String,
topics: Seq[String])
extends RequestOrResponse(Some(ApiKeys.METADATA.id)){
def this(topics: Seq[String], correlationId: Int) =
this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics)
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
writeShortString(buffer, clientId)
buffer.putInt(topics.size)
topics.foreach(topic => writeShortString(buffer, topic))
}
def sizeInBytes: Int = {
2 + /* version id */
4 + /* correlation id */
shortStringLength(clientId) + /* client id */
4 + /* number of topics */
topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */
}
override def toString: String = {
describe(true)
}
override def describe(details: Boolean): String = {
val topicMetadataRequest = new StringBuilder
topicMetadataRequest.append("Name: " + this.getClass.getSimpleName)
topicMetadataRequest.append("; Version: " + versionId)
topicMetadataRequest.append("; CorrelationId: " + correlationId)
topicMetadataRequest.append("; ClientId: " + clientId)
if(details)
topicMetadataRequest.append("; Topics: " + topics.mkString(","))
topicMetadataRequest.toString()
}
}

View File

@ -1,57 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.api
import kafka.cluster.BrokerEndPoint
import java.nio.ByteBuffer
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
object TopicMetadataResponse {
def readFrom(buffer: ByteBuffer): TopicMetadataResponse = {
val correlationId = buffer.getInt
val brokerCount = buffer.getInt
val brokers = (0 until brokerCount).map(_ => BrokerEndPoint.readFrom(buffer))
val brokerMap = brokers.map(b => (b.id, b)).toMap
val topicCount = buffer.getInt
val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap))
new TopicMetadataResponse(brokers, topicsMetadata, correlationId)
}
}
@deprecated("This object has been deprecated and will be removed in a future release.", "1.0.0")
case class TopicMetadataResponse(brokers: Seq[BrokerEndPoint],
topicsMetadata: Seq[TopicMetadata],
correlationId: Int)
extends RequestOrResponse() {
val sizeInBytes: Int = {
4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum
}
def writeTo(buffer: ByteBuffer) {
buffer.putInt(correlationId)
/* brokers */
buffer.putInt(brokers.size)
brokers.foreach(_.writeTo(buffer))
/* topic metadata */
buffer.putInt(topicsMetadata.length)
topicsMetadata.foreach(_.writeTo(buffer))
}
override def describe(details: Boolean):String = { toString }
}

View File

@ -1,203 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.client
import org.apache.kafka.common.protocol.Errors
import scala.collection._
import kafka.cluster._
import kafka.api._
import kafka.common.{BrokerEndPointNotAvailableException, KafkaException}
import kafka.utils.{CoreUtils, Logging}
import util.Random
import kafka.network.BlockingChannel
import kafka.utils.ZkUtils
import java.io.IOException
import kafka.consumer.SimpleConsumer
import org.apache.kafka.common.security.auth.SecurityProtocol
/**
* Helper functions common to clients (producer, consumer, or admin)
*/
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
object ClientUtils extends Logging {
/**
* Send a metadata request
* @param topics The topics for which the metadata needs to be fetched
* @param brokers The brokers in the cluster as configured on the client
* @param clientId The client's identifier
* @return topic metadata response
*/
def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndPoint], clientId: String, timeoutMs: Int,
correlationId: Int = 0): TopicMetadataResponse = {
var fetchMetaDataSucceeded: Boolean = false
var i: Int = 0
val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, clientId,
topics.toSeq)
var topicMetadataResponse: TopicMetadataResponse = null
var t: Throwable = null
// shuffle the list of brokers before sending metadata requests so that most requests don't get routed to the
// same broker
val shuffledBrokers = Random.shuffle(brokers)
while(i < shuffledBrokers.size && !fetchMetaDataSucceeded) {
val broker = shuffledBrokers(i)
val consumer = new SimpleConsumer(broker.host, broker.port, timeoutMs, BlockingChannel.UseDefaultBufferSize,
clientId)
info("Fetching metadata from broker %s with correlation id %d for %d topic(s) %s".format(shuffledBrokers(i),
correlationId, topics.size, topics))
try {
topicMetadataResponse = consumer.send(topicMetadataRequest)
fetchMetaDataSucceeded = true
}
catch {
case e: Throwable =>
warn("Fetching topic metadata with correlation id %d for topics [%s] from broker [%s] failed"
.format(correlationId, topics, shuffledBrokers(i).toString), e)
t = e
} finally {
i = i + 1
consumer.close()
}
}
if (!fetchMetaDataSucceeded) {
throw new KafkaException("fetching topic metadata for topics [%s] from broker [%s] failed".format(topics, shuffledBrokers), t)
} else {
debug("Successfully fetched metadata for %d topic(s) %s".format(topics.size, topics))
}
topicMetadataResponse
}
/**
* Parse a list of broker urls in the form host1:port1, host2:port2, ...
*/
def parseBrokerList(brokerListStr: String): Seq[BrokerEndPoint] = {
val brokersStr = CoreUtils.parseCsvList(brokerListStr)
brokersStr.zipWithIndex.map { case (address, brokerId) =>
BrokerEndPoint.createBrokerEndPoint(brokerId, address)
}
}
/**
* Creates a blocking channel to a random broker
*/
def channelToAnyBroker(zkUtils: ZkUtils, socketTimeoutMs: Int = 3000) : BlockingChannel = {
var channel: BlockingChannel = null
var connected = false
while (!connected) {
val allBrokers = getPlaintextBrokerEndPoints(zkUtils)
Random.shuffle(allBrokers).find { broker =>
trace("Connecting to broker %s:%d.".format(broker.host, broker.port))
try {
channel = new BlockingChannel(broker.host, broker.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, socketTimeoutMs)
channel.connect()
debug("Created channel to broker %s:%d.".format(channel.host, channel.port))
true
} catch {
case _: Exception =>
if (channel != null) channel.disconnect()
channel = null
info("Error while creating channel to %s:%d.".format(broker.host, broker.port))
false
}
}
connected = channel != null
}
channel
}
/**
* Returns the first end point from each broker with the PLAINTEXT security protocol.
*/
def getPlaintextBrokerEndPoints(zkUtils: ZkUtils): Seq[BrokerEndPoint] = {
zkUtils.getAllBrokersInCluster().map { broker =>
broker.endPoints.collectFirst {
case endPoint if endPoint.securityProtocol == SecurityProtocol.PLAINTEXT =>
new BrokerEndPoint(broker.id, endPoint.host, endPoint.port)
}.getOrElse(throw new BrokerEndPointNotAvailableException(s"End point with security protocol PLAINTEXT not found for broker ${broker.id}"))
}
}
/**
* Creates a blocking channel to the offset manager of the given group
*/
def channelToOffsetManager(group: String, zkUtils: ZkUtils, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = {
var queryChannel = channelToAnyBroker(zkUtils)
var offsetManagerChannelOpt: Option[BlockingChannel] = None
while (offsetManagerChannelOpt.isEmpty) {
var coordinatorOpt: Option[BrokerEndPoint] = None
while (coordinatorOpt.isEmpty) {
try {
if (!queryChannel.isConnected)
queryChannel = channelToAnyBroker(zkUtils)
debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group))
queryChannel.send(GroupCoordinatorRequest(group))
val response = queryChannel.receive()
val consumerMetadataResponse = GroupCoordinatorResponse.readFrom(response.payload())
debug("Consumer metadata response: " + consumerMetadataResponse.toString)
if (consumerMetadataResponse.error == Errors.NONE)
coordinatorOpt = consumerMetadataResponse.coordinatorOpt
else {
debug("Query to %s:%d to locate offset manager for %s failed - will retry in %d milliseconds."
.format(queryChannel.host, queryChannel.port, group, retryBackOffMs))
Thread.sleep(retryBackOffMs)
}
}
catch {
case _: IOException =>
info("Failed to fetch consumer metadata from %s:%d.".format(queryChannel.host, queryChannel.port))
queryChannel.disconnect()
}
}
val coordinator = coordinatorOpt.get
if (coordinator.host == queryChannel.host && coordinator.port == queryChannel.port) {
offsetManagerChannelOpt = Some(queryChannel)
} else {
val connectString = "%s:%d".format(coordinator.host, coordinator.port)
var offsetManagerChannel: BlockingChannel = null
try {
debug("Connecting to offset manager %s.".format(connectString))
offsetManagerChannel = new BlockingChannel(coordinator.host, coordinator.port,
BlockingChannel.UseDefaultBufferSize,
BlockingChannel.UseDefaultBufferSize,
socketTimeoutMs)
offsetManagerChannel.connect()
offsetManagerChannelOpt = Some(offsetManagerChannel)
queryChannel.disconnect()
}
catch {
case _: IOException => // offsets manager may have moved
info("Error while connecting to %s.".format(connectString))
if (offsetManagerChannel != null) offsetManagerChannel.disconnect()
Thread.sleep(retryBackOffMs)
offsetManagerChannelOpt = None // just in case someone decides to change shutdownChannel to not swallow exceptions
}
}
}
offsetManagerChannelOpt.get
}
}

View File

@ -20,8 +20,7 @@ package kafka.cluster
import java.util.concurrent.locks.ReentrantReadWriteLock
import com.yammer.metrics.core.Gauge
import kafka.api.LeaderAndIsr
import kafka.api.Request
import kafka.api.{LeaderAndIsr, Request}
import kafka.common.UnexpectedAppendOffsetException
import kafka.controller.KafkaController
import kafka.log.{LogAppendInfo, LogConfig}

View File

@ -1,36 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Exception thrown when a principal is not authorized to perform an operation.
* @param message
*/
abstract class AuthorizationException(message: String) extends RuntimeException(message) {
}
class TopicAuthorizationException(message: String) extends AuthorizationException(message) {
def this() = this(null)
}
class GroupAuthorizationException(message: String) extends AuthorizationException(message) {
def this() = this(null)
}
class ClusterAuthorizationException(message: String) extends AuthorizationException(message) {
def this() = this(null)
}

View File

@ -1,22 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
class BrokerNotAvailableException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,22 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
class ConsumerCoordinatorNotAvailableException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,26 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Thrown when a request is made for broker but no brokers with that topic
* exist.
*/
class ConsumerRebalanceFailedException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,23 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
class ControllerMovedException(message: String, cause: Throwable) extends RuntimeException(message, cause) {
def this(message: String) = this(message, null)
def this() = this(null, null)
}

View File

@ -1,117 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
import java.nio.ByteBuffer
import kafka.message.InvalidMessageException
import org.apache.kafka.common.errors.InvalidTopicException
import scala.Predef._
/**
* A bi-directional mapping between error codes and exceptions
*/
object ErrorMapping {
val EmptyByteBuffer = ByteBuffer.allocate(0)
val UnknownCode: Short = -1
val NoError: Short = 0
val OffsetOutOfRangeCode: Short = 1
val InvalidMessageCode: Short = 2
val UnknownTopicOrPartitionCode: Short = 3
val InvalidFetchSizeCode: Short = 4
val LeaderNotAvailableCode: Short = 5
val NotLeaderForPartitionCode: Short = 6
val RequestTimedOutCode: Short = 7
val BrokerNotAvailableCode: Short = 8
val ReplicaNotAvailableCode: Short = 9
val MessageSizeTooLargeCode: Short = 10
val StaleControllerEpochCode: Short = 11
val OffsetMetadataTooLargeCode: Short = 12
val StaleLeaderEpochCode: Short = 13
val OffsetsLoadInProgressCode: Short = 14
val ConsumerCoordinatorNotAvailableCode: Short = 15
val NotCoordinatorForConsumerCode: Short = 16
val InvalidTopicCode: Short = 17
val MessageSetSizeTooLargeCode: Short = 18
val NotEnoughReplicasCode: Short = 19
val NotEnoughReplicasAfterAppendCode: Short = 20
// 21: InvalidRequiredAcks
// 22: IllegalConsumerGeneration
// 23: INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY
// 24: UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY
// 25: UNKNOWN_CONSUMER_ID
// 26: INVALID_SESSION_TIMEOUT
// 27: REBALANCE_IN_PROGRESS
// 28: INVALID_COMMIT_OFFSET_SIZE
val TopicAuthorizationCode: Short = 29
val GroupAuthorizationCode: Short = 30
val ClusterAuthorizationCode: Short = 31
// 32: INVALID_TIMESTAMP
// 33: UNSUPPORTED_SASL_MECHANISM
// 34: ILLEGAL_SASL_STATE
// 35: UNSUPPORTED_VERSION
// 36: TOPIC_ALREADY_EXISTS
// 37: INVALID_PARTITIONS
// 38: INVALID_REPLICATION_FACTOR
// 39: INVALID_REPLICA_ASSIGNMENT
// 40: INVALID_CONFIG
// 41: NOT_CONTROLLER
// 42: INVALID_REQUEST
private val exceptionToCode =
Map[Class[Throwable], Short](
classOf[OffsetOutOfRangeException].asInstanceOf[Class[Throwable]] -> OffsetOutOfRangeCode,
classOf[InvalidMessageException].asInstanceOf[Class[Throwable]] -> InvalidMessageCode,
classOf[UnknownTopicOrPartitionException].asInstanceOf[Class[Throwable]] -> UnknownTopicOrPartitionCode,
classOf[InvalidMessageSizeException].asInstanceOf[Class[Throwable]] -> InvalidFetchSizeCode,
classOf[LeaderNotAvailableException].asInstanceOf[Class[Throwable]] -> LeaderNotAvailableCode,
classOf[NotLeaderForPartitionException].asInstanceOf[Class[Throwable]] -> NotLeaderForPartitionCode,
classOf[RequestTimedOutException].asInstanceOf[Class[Throwable]] -> RequestTimedOutCode,
classOf[BrokerNotAvailableException].asInstanceOf[Class[Throwable]] -> BrokerNotAvailableCode,
classOf[ReplicaNotAvailableException].asInstanceOf[Class[Throwable]] -> ReplicaNotAvailableCode,
classOf[MessageSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSizeTooLargeCode,
classOf[ControllerMovedException].asInstanceOf[Class[Throwable]] -> StaleControllerEpochCode,
classOf[OffsetMetadataTooLargeException].asInstanceOf[Class[Throwable]] -> OffsetMetadataTooLargeCode,
classOf[OffsetsLoadInProgressException].asInstanceOf[Class[Throwable]] -> OffsetsLoadInProgressCode,
classOf[ConsumerCoordinatorNotAvailableException].asInstanceOf[Class[Throwable]] -> ConsumerCoordinatorNotAvailableCode,
classOf[NotCoordinatorForConsumerException].asInstanceOf[Class[Throwable]] -> NotCoordinatorForConsumerCode,
classOf[InvalidTopicException].asInstanceOf[Class[Throwable]] -> InvalidTopicCode,
classOf[MessageSetSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSetSizeTooLargeCode,
classOf[NotEnoughReplicasException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasCode,
classOf[NotEnoughReplicasAfterAppendException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasAfterAppendCode,
classOf[TopicAuthorizationException].asInstanceOf[Class[Throwable]] -> TopicAuthorizationCode,
classOf[GroupAuthorizationException].asInstanceOf[Class[Throwable]] -> GroupAuthorizationCode,
classOf[ClusterAuthorizationException].asInstanceOf[Class[Throwable]] -> ClusterAuthorizationCode
).withDefaultValue(UnknownCode)
/* invert the mapping */
private val codeToException =
(Map[Short, Class[Throwable]]() ++ exceptionToCode.iterator.map(p => (p._2, p._1))).withDefaultValue(classOf[UnknownException])
def codeFor(exception: Class[Throwable]): Short = exceptionToCode(exception)
def maybeThrowException(code: Short) =
if(code != 0)
throw codeToException(code).newInstance()
def exceptionFor(code: Short) : Throwable = codeToException(code).newInstance()
def exceptionNameFor(code: Short) : String = codeToException(code).getName()
}

View File

@ -1,23 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Indicates a producer pool initialization problem
*/
class FailedToSendMessageException(message: String, t: Throwable) extends RuntimeException(message, t) {
}

View File

@ -1,26 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Indicates the client has requested a range no longer available on the server
*/
class InvalidMessageSizeException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,22 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
class MessageSetSizeTooLargeException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,22 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
class MessageSizeTooLargeException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,23 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Indicates a createMessageStreams can't be called more than once
*/
class MessageStreamsExistException(message: String, t: Throwable) extends RuntimeException(message, t) {
}

View File

@ -1,26 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Thrown when a request is made for broker but no brokers with that topic
* exist.
*/
class NoBrokersForPartitionException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,22 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
class NotCoordinatorForConsumerException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,27 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Number of insync replicas for the partition is lower than min.insync.replicas
* This exception is raised when the low ISR size is discovered *after* the message
* was already appended to the log. Producer retries will cause duplicates.
*/
class NotEnoughReplicasAfterAppendException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,25 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Message was rejected because number of insync replicas for the partition is lower than min.insync.replicas
*/
class NotEnoughReplicasException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,25 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Thrown when a request is made for partition on a broker that is NOT a leader for that partition
*/
class NotLeaderForPartitionException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,27 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Indicates the client has specified offset metadata that exceeds the configured
* maximum size in bytes
*/
class OffsetMetadataTooLargeException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,26 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Indicates the client has requested a range no longer available on the server
*/
class OffsetOutOfRangeException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,26 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Indicates that offsets are currently being loaded from disk into the cache so offset fetch requests cannot be satisfied.
*/
class OffsetsLoadInProgressException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,23 +0,0 @@
package kafka.common
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/* Indicates the queue for sending messages is full of unsent messages */
class QueueFullException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,26 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Thrown when a request is made for partition, but no leader exists for that partition
*/
class ReplicaNotAvailableException(cause: Throwable, message: String = "") extends RuntimeException(cause) {
def this() = this(null, "")
def this(message: String) = this(null, message)
}

View File

@ -1,29 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Thrown when a produce request times out - i.e., if one or more partitions it
* sends messages to receives fewer than the requiredAcks that is specified in
* the produce request.
*/
class RequestTimedOutException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,23 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* An exception that indicates KafkaStream has ended.
*/
class StreamEndException() extends RuntimeException {
}

View File

@ -1,24 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Indicates a producer pool initialization problem
*/
class UnavailableProducerException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,23 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* If we don't know what else it is, call it this
*/
class UnknownException extends RuntimeException

View File

@ -1,26 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Indicates the client has requested a range no longer available on the server
*/
class UnknownMagicByteException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,26 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.common
/**
* Indicates one of the following situation:
* 1. Producer does not have the partition metadata for this id upon sending messages
* 2. Broker does not have the specified partition by id upon receiving messages
*/
class UnknownTopicOrPartitionException(message: String) extends RuntimeException(message) {
def this() = this(null)
}

View File

@ -1,172 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import java.util.{Collections, Properties}
import java.util.regex.Pattern
import kafka.api.OffsetRequest
import kafka.common.StreamEndException
import kafka.message.Message
import org.apache.kafka.clients.consumer.Consumer
import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.header.Headers
import org.apache.kafka.common.header.internals.RecordHeaders
/**
* A base consumer used to abstract both old and new consumer
* this class should be removed (along with BaseProducer)
* once we deprecate old consumer
*/
@deprecated("This trait has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0")
trait BaseConsumer {
def receive(): BaseConsumerRecord
def stop()
def cleanup()
def commit()
}
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.ConsumerRecord instead.", "0.11.0.0")
case class BaseConsumerRecord(topic: String,
partition: Int,
offset: Long,
timestamp: Long = Message.NoTimestamp,
timestampType: TimestampType = TimestampType.NO_TIMESTAMP_TYPE,
key: Array[Byte],
value: Array[Byte],
headers: Headers = new RecordHeaders())
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0")
class NewShinyConsumer(topic: Option[String], partitionId: Option[Int], offset: Option[Long], whitelist: Option[String],
consumer: Consumer[Array[Byte], Array[Byte]], val timeoutMs: Long = Long.MaxValue) extends BaseConsumer {
consumerInit()
var recordIter = consumer.poll(0).iterator
def consumerInit() {
(topic, partitionId, offset, whitelist) match {
case (Some(topic), Some(partitionId), Some(offset), None) =>
seek(topic, partitionId, offset)
case (Some(topic), Some(partitionId), None, None) =>
// default to latest if no offset is provided
seek(topic, partitionId, OffsetRequest.LatestTime)
case (Some(topic), None, None, None) =>
consumer.subscribe(Collections.singletonList(topic))
case (None, None, None, Some(whitelist)) =>
consumer.subscribe(Pattern.compile(whitelist))
case _ =>
throw new IllegalArgumentException("An invalid combination of arguments is provided. " +
"Exactly one of 'topic' or 'whitelist' must be provided. " +
"If 'topic' is provided, an optional 'partition' may also be provided. " +
"If 'partition' is provided, an optional 'offset' may also be provided, otherwise, consumption starts from the end of the partition.")
}
}
def seek(topic: String, partitionId: Int, offset: Long) {
val topicPartition = new TopicPartition(topic, partitionId)
consumer.assign(Collections.singletonList(topicPartition))
offset match {
case OffsetRequest.EarliestTime => consumer.seekToBeginning(Collections.singletonList(topicPartition))
case OffsetRequest.LatestTime => consumer.seekToEnd(Collections.singletonList(topicPartition))
case _ => consumer.seek(topicPartition, offset)
}
}
def resetUnconsumedOffsets() {
val smallestUnconsumedOffsets = collection.mutable.Map[TopicPartition, Long]()
while (recordIter.hasNext) {
val record = recordIter.next()
val tp = new TopicPartition(record.topic, record.partition)
// avoid auto-committing offsets which haven't been consumed
smallestUnconsumedOffsets.getOrElseUpdate(tp, record.offset)
}
smallestUnconsumedOffsets.foreach { case (tp, offset) => consumer.seek(tp, offset) }
}
override def receive(): BaseConsumerRecord = {
if (!recordIter.hasNext) {
recordIter = consumer.poll(timeoutMs).iterator
if (!recordIter.hasNext)
throw new ConsumerTimeoutException
}
val record = recordIter.next
BaseConsumerRecord(record.topic,
record.partition,
record.offset,
record.timestamp,
record.timestampType,
record.key,
record.value,
record.headers)
}
override def stop() {
this.consumer.wakeup()
}
override def cleanup() {
resetUnconsumedOffsets()
this.consumer.close()
}
override def commit() {
this.consumer.commitSync()
}
}
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0")
class OldConsumer(topicFilter: TopicFilter, consumerProps: Properties) extends BaseConsumer {
import kafka.serializer.DefaultDecoder
val consumerConnector = Consumer.create(new ConsumerConfig(consumerProps))
val stream: KafkaStream[Array[Byte], Array[Byte]] =
consumerConnector.createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder(), new DefaultDecoder()).head
val iter = stream.iterator
override def receive(): BaseConsumerRecord = {
if (!iter.hasNext())
throw new StreamEndException
val messageAndMetadata = iter.next
BaseConsumerRecord(messageAndMetadata.topic,
messageAndMetadata.partition,
messageAndMetadata.offset,
messageAndMetadata.timestamp,
messageAndMetadata.timestampType,
messageAndMetadata.key,
messageAndMetadata.message,
new RecordHeaders())
}
override def stop() {
this.consumerConnector.shutdown()
}
override def cleanup() {
this.consumerConnector.shutdown()
}
override def commit() {
this.consumerConnector.commitOffsets
}
}

View File

@ -17,9 +17,17 @@
package kafka.consumer
import kafka.message.ByteBufferMessageSet
import org.apache.kafka.common.header.Headers
import org.apache.kafka.common.header.internals.RecordHeaders
import org.apache.kafka.common.record.{RecordBatch, TimestampType}
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
case class FetchedDataChunk(messages: ByteBufferMessageSet,
topicInfo: PartitionTopicInfo,
fetchOffset: Long)
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.ConsumerRecord instead.", "0.11.0.0")
case class BaseConsumerRecord(topic: String,
partition: Int,
offset: Long,
timestamp: Long = RecordBatch.NO_TIMESTAMP,
timestampType: TimestampType = TimestampType.NO_TIMESTAMP_TYPE,
key: Array[Byte],
value: Array[Byte],
headers: Headers = new RecordHeaders())

View File

@ -1,205 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import java.util.Properties
import kafka.api.OffsetRequest
import kafka.utils._
import kafka.common.{InvalidConfigException, Config}
import java.util.Locale
@deprecated("This object has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.ConsumerConfig instead.", "0.11.0.0")
object ConsumerConfig extends Config {
val RefreshMetadataBackoffMs = 200
val SocketTimeout = 30 * 1000
val SocketBufferSize = 64*1024
val FetchSize = 1024 * 1024
val MaxFetchSize = 10*FetchSize
val NumConsumerFetchers = 1
val DefaultFetcherBackoffMs = 1000
val AutoCommit = true
val AutoCommitInterval = 60 * 1000
val MaxQueuedChunks = 2
val MaxRebalanceRetries = 4
val AutoOffsetReset = OffsetRequest.LargestTimeString
val ConsumerTimeoutMs = -1
val MinFetchBytes = 1
val MaxFetchBytes = 50 * 1024 * 1024
val MaxFetchWaitMs = 100
val MirrorTopicsWhitelist = ""
val MirrorTopicsBlacklist = ""
val MirrorConsumerNumThreads = 1
val OffsetsChannelBackoffMs = 1000
val OffsetsChannelSocketTimeoutMs = 10000
val OffsetsCommitMaxRetries = 5
val OffsetsStorage = "zookeeper"
val MirrorTopicsWhitelistProp = "mirror.topics.whitelist"
val MirrorTopicsBlacklistProp = "mirror.topics.blacklist"
val ExcludeInternalTopics = true
val DefaultPartitionAssignmentStrategy = "range" /* select between "range", and "roundrobin" */
val MirrorConsumerNumThreadsProp = "mirror.consumer.numthreads"
val DefaultClientId = ""
def validate(config: ConsumerConfig) {
validateClientId(config.clientId)
validateGroupId(config.groupId)
validateAutoOffsetReset(config.autoOffsetReset)
validateOffsetsStorage(config.offsetsStorage)
validatePartitionAssignmentStrategy(config.partitionAssignmentStrategy)
}
def validateClientId(clientId: String) {
validateChars("client.id", clientId)
}
def validateGroupId(groupId: String) {
validateChars("group.id", groupId)
}
def validateAutoOffsetReset(autoOffsetReset: String) {
autoOffsetReset match {
case OffsetRequest.SmallestTimeString =>
case OffsetRequest.LargestTimeString =>
case _ => throw new InvalidConfigException("Wrong value " + autoOffsetReset + " of auto.offset.reset in ConsumerConfig; " +
"Valid values are " + OffsetRequest.SmallestTimeString + " and " + OffsetRequest.LargestTimeString)
}
}
def validateOffsetsStorage(storage: String) {
storage match {
case "zookeeper" =>
case "kafka" =>
case _ => throw new InvalidConfigException("Wrong value " + storage + " of offsets.storage in consumer config; " +
"Valid values are 'zookeeper' and 'kafka'")
}
}
def validatePartitionAssignmentStrategy(strategy: String) {
strategy match {
case "range" =>
case "roundrobin" =>
case _ => throw new InvalidConfigException("Wrong value " + strategy + " of partition.assignment.strategy in consumer config; " +
"Valid values are 'range' and 'roundrobin'")
}
}
}
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.ConsumerConfig instead.", "0.11.0.0")
class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) {
import ConsumerConfig._
def this(originalProps: Properties) {
this(new VerifiableProperties(originalProps))
props.verify()
}
/** a string that uniquely identifies a set of consumers within the same consumer group */
val groupId = props.getString("group.id")
/** consumer id: generated automatically if not set.
* Set this explicitly for only testing purpose. */
val consumerId: Option[String] = Option(props.getString("consumer.id", null))
/** the socket timeout for network requests. Its value should be at least fetch.wait.max.ms. */
val socketTimeoutMs = props.getInt("socket.timeout.ms", SocketTimeout)
/** the socket receive buffer for network requests */
val socketReceiveBufferBytes = props.getInt("socket.receive.buffer.bytes", SocketBufferSize)
/** the number of bytes of messages to attempt to fetch from each partition */
val fetchMessageMaxBytes = props.getInt("fetch.message.max.bytes", FetchSize)
/** the number threads used to fetch data */
val numConsumerFetchers = props.getInt("num.consumer.fetchers", NumConsumerFetchers)
/** if true, periodically commit to zookeeper the offset of messages already fetched by the consumer */
val autoCommitEnable = props.getBoolean("auto.commit.enable", AutoCommit)
/** the frequency in ms that the consumer offsets are committed to zookeeper */
val autoCommitIntervalMs = props.getInt("auto.commit.interval.ms", AutoCommitInterval)
/** max number of message chunks buffered for consumption, each chunk can be up to fetch.message.max.bytes*/
val queuedMaxMessages = props.getInt("queued.max.message.chunks", MaxQueuedChunks)
/** max number of retries during rebalance */
val rebalanceMaxRetries = props.getInt("rebalance.max.retries", MaxRebalanceRetries)
/** the minimum amount of data the server should return for a fetch request. If insufficient data is available the request will block */
val fetchMinBytes = props.getInt("fetch.min.bytes", MinFetchBytes)
/** the maximum amount of data the server should return for a fetch request */
val fetchMaxBytes = props.getInt("fetch.max.bytes", MaxFetchBytes)
/** the maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy fetch.min.bytes */
val fetchWaitMaxMs = props.getInt("fetch.wait.max.ms", MaxFetchWaitMs)
require(fetchWaitMaxMs <= socketTimeoutMs, "socket.timeout.ms should always be at least fetch.wait.max.ms" +
" to prevent unnecessary socket timeouts")
/** backoff time between retries during rebalance */
val rebalanceBackoffMs = props.getInt("rebalance.backoff.ms", zkSyncTimeMs)
/** backoff time to refresh the leader of a partition after it loses the current leader */
val refreshLeaderBackoffMs = props.getInt("refresh.leader.backoff.ms", RefreshMetadataBackoffMs)
/** backoff time to reconnect the offsets channel or to retry offset fetches/commits */
val offsetsChannelBackoffMs = props.getInt("offsets.channel.backoff.ms", OffsetsChannelBackoffMs)
/** socket timeout to use when reading responses for Offset Fetch/Commit requests. This timeout will also be used for
* the ConsumerMetdata requests that are used to query for the offset coordinator. */
val offsetsChannelSocketTimeoutMs = props.getInt("offsets.channel.socket.timeout.ms", OffsetsChannelSocketTimeoutMs)
/** Retry the offset commit up to this many times on failure. This retry count only applies to offset commits during
* shut-down. It does not apply to commits from the auto-commit thread. It also does not apply to attempts to query
* for the offset coordinator before committing offsets. i.e., if a consumer metadata request fails for any reason,
* it is retried and that retry does not count toward this limit. */
val offsetsCommitMaxRetries = props.getInt("offsets.commit.max.retries", OffsetsCommitMaxRetries)
/** Specify whether offsets should be committed to "zookeeper" (default) or "kafka" */
val offsetsStorage = props.getString("offsets.storage", OffsetsStorage).toLowerCase(Locale.ROOT)
/** If you are using "kafka" as offsets.storage, you can dual commit offsets to ZooKeeper (in addition to Kafka). This
* is required during migration from zookeeper-based offset storage to kafka-based offset storage. With respect to any
* given consumer group, it is safe to turn this off after all instances within that group have been migrated to
* the new jar that commits offsets to the broker (instead of directly to ZooKeeper). */
val dualCommitEnabled = props.getBoolean("dual.commit.enabled", offsetsStorage == "kafka")
/* what to do if an offset is out of range.
smallest : automatically reset the offset to the smallest offset
largest : automatically reset the offset to the largest offset
anything else: throw exception to the consumer */
val autoOffsetReset = props.getString("auto.offset.reset", AutoOffsetReset)
/** throw a timeout exception to the consumer if no message is available for consumption after the specified interval */
val consumerTimeoutMs = props.getInt("consumer.timeout.ms", ConsumerTimeoutMs)
/**
* Client id is specified by the kafka consumer client, used to distinguish different clients
*/
val clientId = props.getString("client.id", groupId)
/** Whether messages from internal topics (such as offsets) should be exposed to the consumer. */
val excludeInternalTopics = props.getBoolean("exclude.internal.topics", ExcludeInternalTopics)
/** Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin */
val partitionAssignmentStrategy = props.getString("partition.assignment.strategy", DefaultPartitionAssignmentStrategy)
validate(this)
}

View File

@ -1,126 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import kafka.common.{OffsetAndMetadata, TopicAndPartition}
import kafka.javaapi.consumer.ConsumerRebalanceListener
import scala.collection._
import kafka.utils.Logging
import kafka.serializer._
/**
* Main interface for consumer
*/
@deprecated("This trait has been deprecated and will be removed in a future release.", "0.11.0.0")
trait ConsumerConnector {
/**
* Create a list of MessageStreams for each topic.
*
* @param topicCountMap a map of (topic, #streams) pair
* @return a map of (topic, list of KafkaStream) pairs.
* The number of items in the list is #streams. Each stream supports
* an iterator over message/metadata pairs.
*/
def createMessageStreams(topicCountMap: Map[String,Int]): Map[String, List[KafkaStream[Array[Byte],Array[Byte]]]]
/**
* Create a list of MessageStreams for each topic.
*
* @param topicCountMap a map of (topic, #streams) pair
* @param keyDecoder Decoder to decode the key portion of the message
* @param valueDecoder Decoder to decode the value portion of the message
* @return a map of (topic, list of KafkaStream) pairs.
* The number of items in the list is #streams. Each stream supports
* an iterator over message/metadata pairs.
*/
def createMessageStreams[K,V](topicCountMap: Map[String,Int],
keyDecoder: Decoder[K],
valueDecoder: Decoder[V])
: Map[String,List[KafkaStream[K,V]]]
/**
* Create a list of message streams for all topics that match a given filter.
*
* @param topicFilter Either a Whitelist or Blacklist TopicFilter object.
* @param numStreams Number of streams to return
* @param keyDecoder Decoder to decode the key portion of the message
* @param valueDecoder Decoder to decode the value portion of the message
* @return a list of KafkaStream each of which provides an
* iterator over message/metadata pairs over allowed topics.
*/
def createMessageStreamsByFilter[K,V](topicFilter: TopicFilter,
numStreams: Int = 1,
keyDecoder: Decoder[K] = new DefaultDecoder(),
valueDecoder: Decoder[V] = new DefaultDecoder())
: Seq[KafkaStream[K,V]]
/**
* Commit the offsets of all broker partitions connected by this connector.
*/
def commitOffsets(retryOnFailure: Boolean)
/**
* KAFKA-1743: This method added for backward compatibility.
*/
def commitOffsets()
/**
* Commit offsets from an external offsets map.
* @param offsetsToCommit the offsets to be committed.
*/
def commitOffsets(offsetsToCommit: immutable.Map[TopicAndPartition, OffsetAndMetadata], retryOnFailure: Boolean)
/**
* Wire in a consumer rebalance listener to be executed when consumer rebalance occurs.
* @param listener The consumer rebalance listener to wire in
*/
def setConsumerRebalanceListener(listener: ConsumerRebalanceListener)
/**
* Shut down the connector
*/
def shutdown()
}
@deprecated("This object has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.Consumer instead.", "0.11.0.0")
object Consumer extends Logging {
/**
* Create a ConsumerConnector
*
* @param config at the minimum, need to specify the groupid of the consumer and the zookeeper
* connection string zookeeper.connect.
*/
def create(config: ConsumerConfig): ConsumerConnector = {
val consumerConnect = new ZookeeperConsumerConnector(config)
consumerConnect
}
/**
* Create a ConsumerConnector
*
* @param config at the minimum, need to specify the groupid of the consumer and the zookeeper
* connection string zookeeper.connect.
*/
def createJavaConsumerConnector(config: ConsumerConfig): kafka.javaapi.consumer.ConsumerConnector = {
val consumerConnect = new kafka.javaapi.consumer.ZookeeperConsumerConnector(config)
consumerConnect
}
}

View File

@ -1,162 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import kafka.server.{AbstractFetcherManager, AbstractFetcherThread, BrokerAndInitialOffset}
import kafka.cluster.{BrokerEndPoint, Cluster}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.utils.Time
import scala.collection.immutable
import collection.mutable.HashMap
import scala.collection.mutable
import java.util.concurrent.locks.ReentrantLock
import kafka.utils.CoreUtils.inLock
import kafka.utils.ZkUtils
import kafka.utils.ShutdownableThread
import kafka.client.ClientUtils
import java.util.concurrent.atomic.AtomicInteger
/**
* Usage:
* Once ConsumerFetcherManager is created, startConnections() and stopAllConnections() can be called repeatedly
* until shutdown() is called.
*/
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class ConsumerFetcherManager(private val consumerIdString: String,
private val config: ConsumerConfig,
private val zkUtils : ZkUtils)
extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(Time.SYSTEM.milliseconds),
config.clientId, config.numConsumerFetchers) {
private var partitionMap: immutable.Map[TopicPartition, PartitionTopicInfo] = null
private val noLeaderPartitionSet = new mutable.HashSet[TopicPartition]
private val lock = new ReentrantLock
private val cond = lock.newCondition()
private var leaderFinderThread: ShutdownableThread = null
private val correlationId = new AtomicInteger(0)
private class LeaderFinderThread(name: String) extends ShutdownableThread(name) {
// thread responsible for adding the fetcher to the right broker when leader is available
override def doWork() {
val leaderForPartitionsMap = new HashMap[TopicPartition, BrokerEndPoint]
lock.lock()
try {
while (noLeaderPartitionSet.isEmpty) {
trace("No partition for leader election.")
cond.await()
}
trace("Partitions without leader %s".format(noLeaderPartitionSet))
val brokers = ClientUtils.getPlaintextBrokerEndPoints(zkUtils)
val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet,
brokers,
config.clientId,
config.socketTimeoutMs,
correlationId.getAndIncrement).topicsMetadata
if(isDebugEnabled) topicsMetadata.foreach(topicMetadata => debug(topicMetadata.toString()))
topicsMetadata.foreach { tmd =>
val topic = tmd.topic
tmd.partitionsMetadata.foreach { pmd =>
val topicAndPartition = new TopicPartition(topic, pmd.partitionId)
if(pmd.leader.isDefined && noLeaderPartitionSet.contains(topicAndPartition)) {
val leaderBroker = pmd.leader.get
leaderForPartitionsMap.put(topicAndPartition, leaderBroker)
noLeaderPartitionSet -= topicAndPartition
}
}
}
} catch {
case t: Throwable => {
if (!isRunning)
throw t /* If this thread is stopped, propagate this exception to kill the thread. */
else
warn("Failed to find leader for %s".format(noLeaderPartitionSet), t)
}
} finally {
lock.unlock()
}
try {
addFetcherForPartitions(leaderForPartitionsMap.map { case (topicPartition, broker) =>
topicPartition -> BrokerAndInitialOffset(broker, partitionMap(topicPartition).getFetchOffset())}
)
} catch {
case t: Throwable =>
if (!isRunning)
throw t /* If this thread is stopped, propagate this exception to kill the thread. */
else {
warn("Failed to add leader for partitions %s; will retry".format(leaderForPartitionsMap.keySet.mkString(",")), t)
lock.lock()
noLeaderPartitionSet ++= leaderForPartitionsMap.keySet
lock.unlock()
}
}
shutdownIdleFetcherThreads()
Thread.sleep(config.refreshLeaderBackoffMs)
}
}
override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = {
new ConsumerFetcherThread(consumerIdString, fetcherId, config, sourceBroker, partitionMap, this)
}
def startConnections(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster) {
leaderFinderThread = new LeaderFinderThread(consumerIdString + "-leader-finder-thread")
leaderFinderThread.start()
inLock(lock) {
partitionMap = topicInfos.map(tpi => (new TopicPartition(tpi.topic, tpi.partitionId), tpi)).toMap
noLeaderPartitionSet ++= topicInfos.map(tpi => new TopicPartition(tpi.topic, tpi.partitionId))
cond.signalAll()
}
}
def stopConnections() {
/*
* Stop the leader finder thread first before stopping fetchers. Otherwise, if there are more partitions without
* leader, then the leader finder thread will process these partitions (before shutting down) and add fetchers for
* these partitions.
*/
info("Stopping leader finder thread")
if (leaderFinderThread != null) {
leaderFinderThread.shutdown()
leaderFinderThread = null
}
info("Stopping all fetchers")
closeAllFetchers()
// no need to hold the lock for the following since leaderFindThread and all fetchers have been stopped
partitionMap = null
noLeaderPartitionSet.clear()
info("All connections stopped")
}
def addPartitionsWithError(partitionList: Iterable[TopicPartition]) {
debug("adding partitions with error %s".format(partitionList))
inLock(lock) {
if (partitionMap != null) {
noLeaderPartitionSet ++= partitionList
cond.signalAll()
}
}
}
}

View File

@ -1,158 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import kafka.api.{FetchRequestBuilder, FetchResponsePartitionData, OffsetRequest, Request}
import kafka.cluster.BrokerEndPoint
import kafka.message.ByteBufferMessageSet
import kafka.server.{AbstractFetcherThread, PartitionFetchState, OffsetTruncationState}
import AbstractFetcherThread.ResultWithPartitions
import kafka.common.{ErrorMapping, TopicAndPartition}
import scala.collection.Map
import ConsumerFetcherThread._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.EpochEndOffset
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.internals.Fetcher instead.", "0.11.0.0")
class ConsumerFetcherThread(consumerIdString: String,
fetcherId: Int,
val config: ConsumerConfig,
sourceBroker: BrokerEndPoint,
partitionMap: Map[TopicPartition, PartitionTopicInfo],
val consumerFetcherManager: ConsumerFetcherManager)
extends AbstractFetcherThread(name = s"ConsumerFetcherThread-$consumerIdString-$fetcherId-${sourceBroker.id}",
clientId = config.clientId,
sourceBroker = sourceBroker,
fetchBackOffMs = config.refreshLeaderBackoffMs,
isInterruptible = true,
includeLogTruncation = false) {
type REQ = FetchRequest
type PD = PartitionData
this.logIdent = s"[ConsumerFetcher consumerId=$consumerIdString, leaderId=${sourceBroker.id}, " +
s"fetcherId=$fetcherId] "
private val clientId = config.clientId
private val fetchSize = config.fetchMessageMaxBytes
private val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, config.socketTimeoutMs,
config.socketReceiveBufferBytes, config.clientId)
private val fetchRequestBuilder = new FetchRequestBuilder().
clientId(clientId).
replicaId(Request.OrdinaryConsumerId).
maxWait(config.fetchWaitMaxMs).
minBytes(config.fetchMinBytes).
requestVersion(3) // for now, the old consumer is pinned to the old message format through the fetch request
override def initiateShutdown(): Boolean = {
val justShutdown = super.initiateShutdown()
if (justShutdown && isInterruptible)
simpleConsumer.disconnectToHandleJavaIOBug()
justShutdown
}
override def shutdown(): Unit = {
super.shutdown()
simpleConsumer.close()
}
// process fetched data
def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: PartitionData) {
val pti = partitionMap(topicPartition)
if (pti.getFetchOffset != fetchOffset)
throw new RuntimeException("Offset doesn't match for partition [%s,%d] pti offset: %d fetch offset: %d"
.format(topicPartition.topic, topicPartition.partition, pti.getFetchOffset, fetchOffset))
pti.enqueue(partitionData.underlying.messages.asInstanceOf[ByteBufferMessageSet])
}
// handle a partition whose offset is out of range and return a new fetch offset
def handleOffsetOutOfRange(topicPartition: TopicPartition): Long = {
val startTimestamp = config.autoOffsetReset match {
case OffsetRequest.SmallestTimeString => OffsetRequest.EarliestTime
case _ => OffsetRequest.LatestTime
}
val topicAndPartition = TopicAndPartition(topicPartition.topic, topicPartition.partition)
val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId)
val pti = partitionMap(topicPartition)
pti.resetFetchOffset(newOffset)
pti.resetConsumeOffset(newOffset)
newOffset
}
// any logic for partitions whose leader has changed
def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]) {
if (partitions.nonEmpty) {
removePartitions(partitions.toSet)
consumerFetcherManager.addPartitionsWithError(partitions)
}
}
protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): ResultWithPartitions[FetchRequest] = {
partitionMap.foreach { case ((topicPartition, partitionFetchState)) =>
if (partitionFetchState.isReadyForFetch)
fetchRequestBuilder.addFetch(topicPartition.topic, topicPartition.partition, partitionFetchState.fetchOffset, fetchSize)
}
ResultWithPartitions(new FetchRequest(fetchRequestBuilder.build()), Set())
}
protected def fetch(fetchRequest: FetchRequest): Seq[(TopicPartition, PartitionData)] =
simpleConsumer.fetch(fetchRequest.underlying).data.map { case (TopicAndPartition(t, p), value) =>
new TopicPartition(t, p) -> new PartitionData(value)
}
override def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): ResultWithPartitions[Map[TopicPartition, Int]] = {
ResultWithPartitions(Map(), Set())
}
override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = { Map() }
override def maybeTruncate(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): ResultWithPartitions[Map[TopicPartition, OffsetTruncationState]] = {
ResultWithPartitions(Map(), Set())
}
}
@deprecated("This object has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.internals.Fetcher instead.", "0.11.0.0")
object ConsumerFetcherThread {
class FetchRequest(val underlying: kafka.api.FetchRequest) extends AbstractFetcherThread.FetchRequest {
private lazy val tpToOffset: Map[TopicPartition, Long] = underlying.requestInfo.map { case (tp, fetchInfo) =>
new TopicPartition(tp.topic, tp.partition) -> fetchInfo.offset
}.toMap
def isEmpty: Boolean = underlying.requestInfo.isEmpty
def offset(topicPartition: TopicPartition): Long = tpToOffset(topicPartition)
override def toString = underlying.toString
}
class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData {
def error = underlying.error
def toRecords: MemoryRecords = underlying.messages.asInstanceOf[ByteBufferMessageSet].asRecords
def highWatermark: Long = underlying.hw
def exception: Option[Throwable] =
if (error == Errors.NONE) None else Some(ErrorMapping.exceptionFor(error.code))
override def toString = underlying.toString
}
}

View File

@ -1,123 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import kafka.utils.{IteratorTemplate, Logging}
import java.util.concurrent.{TimeUnit, BlockingQueue}
import kafka.serializer.Decoder
import java.util.concurrent.atomic.AtomicReference
import kafka.message.{MessageAndOffset, MessageAndMetadata}
import kafka.common.{KafkaException, MessageSizeTooLargeException}
/**
* An iterator that blocks until a value can be read from the supplied queue.
* The iterator takes a shutdownCommand object which can be added to the queue to trigger a shutdown
*
*/
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk],
consumerTimeoutMs: Int,
private val keyDecoder: Decoder[K],
private val valueDecoder: Decoder[V],
val clientId: String)
extends IteratorTemplate[MessageAndMetadata[K, V]] with Logging {
private val current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null)
private var currentTopicInfo: PartitionTopicInfo = null
private var consumedOffset: Long = -1L
private val consumerTopicStats = ConsumerTopicStatsRegistry.getConsumerTopicStat(clientId)
override def next(): MessageAndMetadata[K, V] = {
val item = super.next()
if(consumedOffset < 0)
throw new KafkaException("Offset returned by the message set is invalid %d".format(consumedOffset))
currentTopicInfo.resetConsumeOffset(consumedOffset)
val topic = currentTopicInfo.topic
trace("Setting %s consumed offset to %d".format(topic, consumedOffset))
consumerTopicStats.getConsumerTopicStats(topic).messageRate.mark()
consumerTopicStats.getConsumerAllTopicStats().messageRate.mark()
item
}
protected def makeNext(): MessageAndMetadata[K, V] = {
var currentDataChunk: FetchedDataChunk = null
// if we don't have an iterator, get one
var localCurrent = current.get()
if(localCurrent == null || !localCurrent.hasNext) {
if (consumerTimeoutMs < 0)
currentDataChunk = channel.take
else {
currentDataChunk = channel.poll(consumerTimeoutMs, TimeUnit.MILLISECONDS)
if (currentDataChunk == null) {
// reset state to make the iterator re-iterable
resetState()
throw new ConsumerTimeoutException
}
}
if(currentDataChunk eq ZookeeperConsumerConnector.shutdownCommand) {
debug("Received the shutdown command")
return allDone
} else {
currentTopicInfo = currentDataChunk.topicInfo
val cdcFetchOffset = currentDataChunk.fetchOffset
val ctiConsumeOffset = currentTopicInfo.getConsumeOffset
if (ctiConsumeOffset < cdcFetchOffset) {
error("consumed offset: %d doesn't match fetch offset: %d for %s;\n Consumer may lose data"
.format(ctiConsumeOffset, cdcFetchOffset, currentTopicInfo))
currentTopicInfo.resetConsumeOffset(cdcFetchOffset)
}
localCurrent = currentDataChunk.messages.iterator
current.set(localCurrent)
}
// if we just updated the current chunk and it is empty that means the fetch size is too small!
if(currentDataChunk.messages.validBytes == 0)
throw new MessageSizeTooLargeException("Found a message larger than the maximum fetch size of this consumer on topic " +
"%s partition %d at fetch offset %d. Increase the fetch size, or decrease the maximum message size the broker will allow."
.format(currentDataChunk.topicInfo.topic, currentDataChunk.topicInfo.partitionId, currentDataChunk.fetchOffset))
}
var item = localCurrent.next()
// reject the messages that have already been consumed
while (item.offset < currentTopicInfo.getConsumeOffset && localCurrent.hasNext) {
item = localCurrent.next()
}
consumedOffset = item.nextOffset
item.message.ensureValid() // validate checksum of message to ensure it is valid
new MessageAndMetadata(currentTopicInfo.topic,
currentTopicInfo.partitionId,
item.message,
item.offset,
keyDecoder,
valueDecoder,
item.message.timestamp,
item.message.timestampType)
}
def clearCurrentChunk() {
debug("Clearing the current data chunk for this consumer iterator")
current.set(null)
}
}
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.common.errors.TimeoutException instead.", "0.11.0.0")
class ConsumerTimeoutException() extends RuntimeException()

View File

@ -1,69 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import kafka.utils.{Pool, threadsafe, Logging}
import java.util.concurrent.TimeUnit
import kafka.metrics.KafkaMetricsGroup
import kafka.common.{ClientIdTopic, ClientIdAllTopics, ClientIdAndTopic}
@threadsafe
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class ConsumerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup {
val tags = metricId match {
case ClientIdAndTopic(clientId, topic) => Map("clientId" -> clientId, "topic" -> topic)
case ClientIdAllTopics(clientId) => Map("clientId" -> clientId)
}
val messageRate = newMeter("MessagesPerSec", "messages", TimeUnit.SECONDS, tags)
val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, tags)
}
/**
* Tracks metrics for each topic the given consumer client has consumed data from.
* @param clientId The clientId of the given consumer client.
*/
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class ConsumerTopicStats(clientId: String) extends Logging {
private val valueFactory = (k: ClientIdAndTopic) => new ConsumerTopicMetrics(k)
private val stats = new Pool[ClientIdAndTopic, ConsumerTopicMetrics](Some(valueFactory))
private val allTopicStats = new ConsumerTopicMetrics(new ClientIdAllTopics(clientId)) // to differentiate from a topic named AllTopics
def getConsumerAllTopicStats(): ConsumerTopicMetrics = allTopicStats
def getConsumerTopicStats(topic: String): ConsumerTopicMetrics = {
stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic))
}
}
/**
* Stores the topic stats information of each consumer client in a (clientId -> ConsumerTopicStats) map.
*/
@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0")
object ConsumerTopicStatsRegistry {
private val valueFactory = (k: String) => new ConsumerTopicStats(k)
private val globalStats = new Pool[String, ConsumerTopicStats](Some(valueFactory))
def getConsumerTopicStat(clientId: String) = {
globalStats.getAndMaybePut(clientId)
}
def removeConsumerTopicStat(clientId: String) {
globalStats.remove(clientId)
}
}

View File

@ -1,82 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import java.util.concurrent.TimeUnit
import kafka.common.{ClientIdAllBrokers, ClientIdBroker, ClientIdAndBroker}
import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
import kafka.utils.Pool
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class FetchRequestAndResponseMetrics(metricId: ClientIdBroker) extends KafkaMetricsGroup {
val tags = metricId match {
case ClientIdAndBroker(clientId, brokerHost, brokerPort) =>
Map("clientId" -> clientId, "brokerHost" -> brokerHost,
"brokerPort" -> brokerPort.toString)
case ClientIdAllBrokers(clientId) =>
Map("clientId" -> clientId)
}
val requestTimer = new KafkaTimer(newTimer("FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, tags))
val requestSizeHist = newHistogram("FetchResponseSize", biased = true, tags)
val throttleTimeStats = newTimer("FetchRequestThrottleRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, tags)
}
/**
* Tracks metrics of the requests made by a given consumer client to all brokers, and the responses obtained from the brokers.
* @param clientId ClientId of the given consumer
*/
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class FetchRequestAndResponseStats(clientId: String) {
private val valueFactory = (k: ClientIdBroker) => new FetchRequestAndResponseMetrics(k)
private val stats = new Pool[ClientIdBroker, FetchRequestAndResponseMetrics](Some(valueFactory))
private val allBrokersStats = new FetchRequestAndResponseMetrics(new ClientIdAllBrokers(clientId))
def getFetchRequestAndResponseAllBrokersStats(): FetchRequestAndResponseMetrics = allBrokersStats
def getFetchRequestAndResponseStats(brokerHost: String, brokerPort: Int): FetchRequestAndResponseMetrics = {
stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerHost, brokerPort))
}
}
/**
* Stores the fetch request and response stats information of each consumer client in a (clientId -> FetchRequestAndResponseStats) map.
*/
@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0")
object FetchRequestAndResponseStatsRegistry {
private val valueFactory = (k: String) => new FetchRequestAndResponseStats(k)
private val globalStats = new Pool[String, FetchRequestAndResponseStats](Some(valueFactory))
def getFetchRequestAndResponseStats(clientId: String) = {
globalStats.getAndMaybePut(clientId)
}
def removeConsumerFetchRequestAndResponseStats(clientId: String) {
val pattern = (".*" + clientId + ".*").r
val keys = globalStats.keys
for (key <- keys) {
pattern.findFirstIn(key) match {
case Some(_) => globalStats.remove(key)
case _ =>
}
}
}
}

View File

@ -1,53 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import java.util.concurrent.BlockingQueue
import kafka.serializer.Decoder
import kafka.message.MessageAndMetadata
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.streams.KafkaStreams instead.", "0.11.0.0")
class KafkaStream[K,V](private val queue: BlockingQueue[FetchedDataChunk],
consumerTimeoutMs: Int,
private val keyDecoder: Decoder[K],
private val valueDecoder: Decoder[V],
val clientId: String)
extends Iterable[MessageAndMetadata[K,V]] with java.lang.Iterable[MessageAndMetadata[K,V]] {
private val iter: ConsumerIterator[K,V] =
new ConsumerIterator[K,V](queue, consumerTimeoutMs, keyDecoder, valueDecoder, clientId)
/**
* Create an iterator over messages in the stream.
*/
def iterator: ConsumerIterator[K,V] = iter
/**
* This method clears the queue being iterated during the consumer rebalancing. This is mainly
* to reduce the number of duplicates received by the consumer
*/
def clear() {
iter.clearCurrentChunk()
}
override def toString: String = {
"%s kafka stream".format(clientId)
}
}

View File

@ -1,172 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import kafka.common.TopicAndPartition
import kafka.utils.{Pool, CoreUtils, ZkUtils, Logging}
import scala.collection.mutable
@deprecated("This trait has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.internals.PartitionAssignor instead.", "0.11.0.0")
trait PartitionAssignor {
/**
* Assigns partitions to consumer instances in a group.
* @return An assignment map of partition to this consumer group. This includes assignments for threads that belong
* to the same consumer group.
*/
def assign(ctx: AssignmentContext): Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]]
}
@deprecated("This object has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.internals.PartitionAssignor instead.", "0.11.0.0")
object PartitionAssignor {
def createInstance(assignmentStrategy: String) = assignmentStrategy match {
case "roundrobin" => new RoundRobinAssignor()
case _ => new RangeAssignor()
}
}
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class AssignmentContext(group: String, val consumerId: String, excludeInternalTopics: Boolean, zkUtils: ZkUtils) {
val myTopicThreadIds: collection.Map[String, collection.Set[ConsumerThreadId]] = {
val myTopicCount = TopicCount.constructTopicCount(group, consumerId, zkUtils, excludeInternalTopics)
myTopicCount.getConsumerThreadIdsPerTopic
}
val consumersForTopic: collection.Map[String, List[ConsumerThreadId]] =
zkUtils.getConsumersPerTopic(group, excludeInternalTopics)
// Some assignment strategies require knowledge of all topics consumed by any member of the group
val partitionsForTopic: collection.Map[String, Seq[Int]] =
zkUtils.getPartitionsForTopics(consumersForTopic.keySet.toSeq)
val consumers: Seq[String] = zkUtils.getConsumersInGroup(group).sorted
}
/**
* The round-robin partition assignor lays out all the available partitions and all the available consumer threads. It
* then proceeds to do a round-robin assignment from partition to consumer thread. If the subscriptions of all consumer
* instances are identical, then the partitions will be uniformly distributed. (i.e., the partition ownership counts
* will be within a delta of exactly one across all consumer threads.)
*/
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.RoundRobinAssignor instead.", "0.11.0.0")
class RoundRobinAssignor() extends PartitionAssignor with Logging {
def assign(ctx: AssignmentContext) = {
val valueFactory = (_: String) => new mutable.HashMap[TopicAndPartition, ConsumerThreadId]
val partitionAssignment =
new Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]](Some(valueFactory))
if (ctx.consumersForTopic.nonEmpty) {
// Collect consumer thread ids across all topics, remove duplicates, and sort to ensure determinism
val allThreadIds = ctx.consumersForTopic.flatMap { case (_, threadIds) =>
threadIds
}.toSet.toSeq.sorted
val threadAssignor = CoreUtils.circularIterator(allThreadIds)
info("Starting round-robin assignment with consumers " + ctx.consumers)
val allTopicPartitions = ctx.partitionsForTopic.flatMap { case (topic, partitions) =>
info("Consumer %s rebalancing the following partitions for topic %s: %s"
.format(ctx.consumerId, topic, partitions))
partitions.map(partition => {
TopicAndPartition(topic, partition)
})
}.toSeq.sortWith((topicPartition1, topicPartition2) => {
/*
* Randomize the order by taking the hashcode to reduce the likelihood of all partitions of a given topic ending
* up on one consumer (if it has a high enough stream count).
*/
topicPartition1.toString.hashCode < topicPartition2.toString.hashCode
})
allTopicPartitions.foreach(topicPartition => {
val threadId = threadAssignor.dropWhile(threadId => !ctx.consumersForTopic(topicPartition.topic).contains(threadId)).next
// record the partition ownership decision
val assignmentForConsumer = partitionAssignment.getAndMaybePut(threadId.consumer)
assignmentForConsumer += (topicPartition -> threadId)
})
}
// assign Map.empty for the consumers which are not associated with topic partitions
ctx.consumers.foreach(consumerId => partitionAssignment.getAndMaybePut(consumerId))
partitionAssignment
}
}
/**
* Range partitioning works on a per-topic basis. For each topic, we lay out the available partitions in numeric order
* and the consumer threads in lexicographic order. We then divide the number of partitions by the total number of
* consumer streams (threads) to determine the number of partitions to assign to each consumer. If it does not evenly
* divide, then the first few consumers will have one extra partition. For example, suppose there are two consumers C1
* and C2 with two streams each, and there are five available partitions (p0, p1, p2, p3, p4). So each consumer thread
* will get at least one partition and the first consumer thread will get one extra partition. So the assignment will be:
* p0 -> C1-0, p1 -> C1-0, p2 -> C1-1, p3 -> C2-0, p4 -> C2-1
*/
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.RangeAssignor instead.", "0.11.0.0")
class RangeAssignor() extends PartitionAssignor with Logging {
def assign(ctx: AssignmentContext) = {
val valueFactory = (_: String) => new mutable.HashMap[TopicAndPartition, ConsumerThreadId]
val partitionAssignment =
new Pool[String, mutable.Map[TopicAndPartition, ConsumerThreadId]](Some(valueFactory))
for (topic <- ctx.myTopicThreadIds.keySet) {
val curConsumers = ctx.consumersForTopic(topic)
val curPartitions: Seq[Int] = ctx.partitionsForTopic(topic)
val nPartsPerConsumer = curPartitions.size / curConsumers.size
val nConsumersWithExtraPart = curPartitions.size % curConsumers.size
info("Consumer " + ctx.consumerId + " rebalancing the following partitions: " + curPartitions +
" for topic " + topic + " with consumers: " + curConsumers)
for (consumerThreadId <- curConsumers) {
val myConsumerPosition = curConsumers.indexOf(consumerThreadId)
assert(myConsumerPosition >= 0)
val startPart = nPartsPerConsumer * myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart)
val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1)
/**
* Range-partition the sorted partitions to consumers for better locality.
* The first few consumers pick up an extra partition, if any.
*/
if (nParts <= 0)
warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic)
else {
for (i <- startPart until startPart + nParts) {
val partition = curPartitions(i)
info(consumerThreadId + " attempting to claim partition " + partition)
// record the partition ownership decision
val assignmentForConsumer = partitionAssignment.getAndMaybePut(consumerThreadId.consumer)
assignmentForConsumer += (TopicAndPartition(topic, partition) -> consumerThreadId)
}
}
}
}
// assign Map.empty for the consumers which are not associated with topic partitions
ctx.consumers.foreach(consumerId => partitionAssignment.getAndMaybePut(consumerId))
partitionAssignment
}
}

View File

@ -1,80 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import java.util.concurrent._
import java.util.concurrent.atomic._
import kafka.message._
import kafka.utils.Logging
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class PartitionTopicInfo(val topic: String,
val partitionId: Int,
private val chunkQueue: BlockingQueue[FetchedDataChunk],
private val consumedOffset: AtomicLong,
private val fetchedOffset: AtomicLong,
private val fetchSize: AtomicInteger,
private val clientId: String) extends Logging {
debug("initial consumer offset of " + this + " is " + consumedOffset.get)
debug("initial fetch offset of " + this + " is " + fetchedOffset.get)
private val consumerTopicStats = ConsumerTopicStatsRegistry.getConsumerTopicStat(clientId)
def getConsumeOffset() = consumedOffset.get
def getFetchOffset() = fetchedOffset.get
def resetConsumeOffset(newConsumeOffset: Long) = {
consumedOffset.set(newConsumeOffset)
debug("reset consume offset of " + this + " to " + newConsumeOffset)
}
def resetFetchOffset(newFetchOffset: Long) = {
fetchedOffset.set(newFetchOffset)
debug("reset fetch offset of ( %s ) to %d".format(this, newFetchOffset))
}
/**
* Enqueue a message set for processing.
*/
def enqueue(messages: ByteBufferMessageSet) {
val size = messages.validBytes
if(size > 0) {
val next = messages.shallowIterator.toSeq.last.nextOffset
trace("Updating fetch offset = " + fetchedOffset.get + " to " + next)
chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get))
fetchedOffset.set(next)
debug("updated fetch offset of (%s) to %d".format(this, next))
consumerTopicStats.getConsumerTopicStats(topic).byteRate.mark(size)
consumerTopicStats.getConsumerAllTopicStats().byteRate.mark(size)
} else if(messages.sizeInBytes > 0) {
chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get))
}
}
override def toString: String = topic + ":" + partitionId.toString + ": fetched offset = " + fetchedOffset.get +
": consumed offset = " + consumedOffset.get
}
@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0")
object PartitionTopicInfo {
val InvalidOffset = -1L
def isOffsetInvalid(offset: Long) = offset < 0L
}

View File

@ -1,199 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import java.nio.channels.{AsynchronousCloseException, ClosedByInterruptException}
import java.util.concurrent.TimeUnit
import kafka.api._
import kafka.network._
import kafka.utils._
import kafka.common.{ErrorMapping, TopicAndPartition}
import org.apache.kafka.common.network.{NetworkReceive}
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.utils.Utils._
/**
* A consumer of kafka messages
*/
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0")
@threadsafe
class SimpleConsumer(val host: String,
val port: Int,
val soTimeout: Int,
val bufferSize: Int,
val clientId: String) extends Logging {
ConsumerConfig.validateClientId(clientId)
private val lock = new Object()
private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout)
private val fetchRequestAndResponseStats = FetchRequestAndResponseStatsRegistry.getFetchRequestAndResponseStats(clientId)
private var isClosed = false
private def connect(): BlockingChannel = {
close
blockingChannel.connect()
blockingChannel
}
private def disconnect() = {
debug("Disconnecting from " + formatAddress(host, port))
blockingChannel.disconnect()
}
private def reconnect() {
disconnect()
connect()
}
/**
* Unblock thread by closing channel and triggering AsynchronousCloseException if a read operation is in progress.
*
* This handles a bug found in Java 1.7 and below, where interrupting a thread can not correctly unblock
* the thread from waiting on ReadableByteChannel.read().
*/
def disconnectToHandleJavaIOBug() = {
disconnect()
}
def close() {
lock synchronized {
disconnect()
isClosed = true
}
}
private def sendRequest(request: RequestOrResponse): NetworkReceive = {
lock synchronized {
var response: NetworkReceive = null
try {
getOrMakeConnection()
blockingChannel.send(request)
response = blockingChannel.receive()
} catch {
case e : ClosedByInterruptException =>
throw e
// Should not observe this exception when running Kafka with Java 1.8
case e: AsynchronousCloseException =>
throw e
case e : Throwable =>
info("Reconnect due to error:", e)
// retry once
try {
reconnect()
blockingChannel.send(request)
response = blockingChannel.receive()
} catch {
case e: Throwable =>
disconnect()
throw e
}
}
response
}
}
def send(request: TopicMetadataRequest): TopicMetadataResponse = {
val response = sendRequest(request)
TopicMetadataResponse.readFrom(response.payload())
}
def send(request: GroupCoordinatorRequest): GroupCoordinatorResponse = {
val response = sendRequest(request)
GroupCoordinatorResponse.readFrom(response.payload())
}
/**
* Fetch a set of messages from a topic.
*
* @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched.
* @return a set of fetched messages
*/
def fetch(request: FetchRequest): FetchResponse = {
var response: NetworkReceive = null
val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestTimer
val aggregateTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestTimer
aggregateTimer.time {
specificTimer.time {
response = sendRequest(request)
}
}
val fetchResponse = FetchResponse.readFrom(response.payload(), request.versionId)
val fetchedSize = fetchResponse.sizeInBytes
fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestSizeHist.update(fetchedSize)
fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize)
fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).throttleTimeStats.update(fetchResponse.throttleTimeMs, TimeUnit.MILLISECONDS)
fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.throttleTimeStats.update(fetchResponse.throttleTimeMs, TimeUnit.MILLISECONDS)
fetchResponse
}
/**
* Get a list of valid offsets (up to maxSize) before the given time.
* @param request a [[kafka.api.OffsetRequest]] object.
* @return a [[kafka.api.OffsetResponse]] object.
*/
def getOffsetsBefore(request: OffsetRequest) = OffsetResponse.readFrom(sendRequest(request).payload())
/**
* Commit offsets for a topic
* Version 0 of the request will commit offsets to Zookeeper and version 1 and above will commit offsets to Kafka.
* @param request a [[kafka.api.OffsetCommitRequest]] object.
* @return a [[kafka.api.OffsetCommitResponse]] object.
*/
def commitOffsets(request: OffsetCommitRequest) = {
// TODO: With KAFKA-1012, we have to first issue a ConsumerMetadataRequest and connect to the coordinator before
// we can commit offsets.
OffsetCommitResponse.readFrom(sendRequest(request).payload())
}
/**
* Fetch offsets for a topic
* Version 0 of the request will fetch offsets from Zookeeper and version 1 and above will fetch offsets from Kafka.
* @param request a [[kafka.api.OffsetFetchRequest]] object.
* @return a [[kafka.api.OffsetFetchResponse]] object.
*/
def fetchOffsets(request: OffsetFetchRequest) = OffsetFetchResponse.readFrom(sendRequest(request).payload(), request.versionId)
private def getOrMakeConnection() {
if(!isClosed && !blockingChannel.isConnected) {
connect()
}
}
/**
* Get the earliest or latest offset of a given topic, partition.
* @param topicAndPartition Topic and partition of which the offset is needed.
* @param earliestOrLatest A value to indicate earliest or latest offset.
* @param consumerId Id of the consumer which could be a consumer client, SimpleConsumerShell or a follower broker.
* @return Requested offset.
*/
def earliestOrLatestOffset(topicAndPartition: TopicAndPartition, earliestOrLatest: Long, consumerId: Int): Long = {
val request = OffsetRequest(requestInfo = Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)),
clientId = clientId,
replicaId = consumerId)
val partitionErrorAndOffset = getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition)
val offset = partitionErrorAndOffset.error match {
case Errors.NONE => partitionErrorAndOffset.offsets.head
case _ => throw ErrorMapping.exceptionFor(partitionErrorAndOffset.error.code)
}
offset
}
}

View File

@ -1,145 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import scala.collection._
import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, CoreUtils}
import kafka.common.KafkaException
@deprecated("This trait has been deprecated and will be removed in a future release.", "0.11.0.0")
private[kafka] trait TopicCount {
def getConsumerThreadIdsPerTopic: Map[String, Set[ConsumerThreadId]]
def getTopicCountMap: Map[String, Int]
def pattern: String
}
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
case class ConsumerThreadId(consumer: String, threadId: Int) extends Ordered[ConsumerThreadId] {
override def toString = "%s-%d".format(consumer, threadId)
def compare(that: ConsumerThreadId) = toString.compare(that.toString)
}
@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0")
private[kafka] object TopicCount extends Logging {
val whiteListPattern = "white_list"
val blackListPattern = "black_list"
val staticPattern = "static"
def makeThreadId(consumerIdString: String, threadId: Int) = consumerIdString + "-" + threadId
def makeConsumerThreadIdsPerTopic(consumerIdString: String,
topicCountMap: Map[String, Int]) = {
val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[ConsumerThreadId]]()
for ((topic, nConsumers) <- topicCountMap) {
val consumerSet = new mutable.HashSet[ConsumerThreadId]
assert(nConsumers >= 1)
for (i <- 0 until nConsumers)
consumerSet += ConsumerThreadId(consumerIdString, i)
consumerThreadIdsPerTopicMap.put(topic, consumerSet)
}
consumerThreadIdsPerTopicMap
}
def constructTopicCount(group: String, consumerId: String, zkUtils: ZkUtils, excludeInternalTopics: Boolean) : TopicCount = {
val dirs = new ZKGroupDirs(group)
val topicCountString = zkUtils.readData(dirs.consumerRegistryDir + "/" + consumerId)._1
var subscriptionPattern: String = null
var topMap: Map[String, Int] = null
try {
Json.parseFull(topicCountString) match {
case Some(js) =>
val consumerRegistrationMap = js.asJsonObject
consumerRegistrationMap.get("pattern") match {
case Some(pattern) => subscriptionPattern = pattern.to[String]
case None => throw new KafkaException("error constructing TopicCount : " + topicCountString)
}
consumerRegistrationMap.get("subscription") match {
case Some(sub) => topMap = sub.to[Map[String, Int]]
case None => throw new KafkaException("error constructing TopicCount : " + topicCountString)
}
case None => throw new KafkaException("error constructing TopicCount : " + topicCountString)
}
} catch {
case e: Throwable =>
error("error parsing consumer json string " + topicCountString, e)
throw e
}
val hasWhiteList = whiteListPattern.equals(subscriptionPattern)
val hasBlackList = blackListPattern.equals(subscriptionPattern)
if (topMap.isEmpty || !(hasWhiteList || hasBlackList)) {
new StaticTopicCount(consumerId, topMap)
} else {
val regex = topMap.head._1
val numStreams = topMap.head._2
val filter =
if (hasWhiteList)
new Whitelist(regex)
else
new Blacklist(regex)
new WildcardTopicCount(zkUtils, consumerId, filter, numStreams, excludeInternalTopics)
}
}
def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) =
new StaticTopicCount(consumerIdString, topicCount)
def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkUtils: ZkUtils, excludeInternalTopics: Boolean) =
new WildcardTopicCount(zkUtils, consumerIdString, filter, numStreams, excludeInternalTopics)
}
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
private[kafka] class StaticTopicCount(val consumerIdString: String,
val topicCountMap: Map[String, Int])
extends TopicCount {
def getConsumerThreadIdsPerTopic = TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, topicCountMap)
def getTopicCountMap = topicCountMap
def pattern = TopicCount.staticPattern
}
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
private[kafka] class WildcardTopicCount(zkUtils: ZkUtils,
consumerIdString: String,
topicFilter: TopicFilter,
numStreams: Int,
excludeInternalTopics: Boolean) extends TopicCount {
def getConsumerThreadIdsPerTopic = {
val wildcardTopics = zkUtils.getChildrenParentMayNotExist(ZkUtils.BrokerTopicsPath)
.filter(topic => topicFilter.isTopicAllowed(topic, excludeInternalTopics))
TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*))
}
def getTopicCountMap = Map(CoreUtils.JSONEscapeString(topicFilter.regex) -> numStreams)
def pattern: String = {
topicFilter match {
case _: Whitelist => TopicCount.whiteListPattern
case _: Blacklist => TopicCount.blackListPattern
}
}
}

View File

@ -1,103 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.consumer
import scala.collection.JavaConverters._
import kafka.utils.{ZkUtils, Logging}
import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener}
import org.apache.zookeeper.Watcher.Event.KeeperState
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class ZookeeperTopicEventWatcher(val zkUtils: ZkUtils,
val eventHandler: TopicEventHandler[String]) extends Logging {
val lock = new Object()
startWatchingTopicEvents()
private def startWatchingTopicEvents() {
val topicEventListener = new ZkTopicEventListener()
zkUtils.makeSurePersistentPathExists(ZkUtils.BrokerTopicsPath)
zkUtils.subscribeStateChanges(new ZkSessionExpireListener(topicEventListener))
val topics = zkUtils.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicEventListener).getOrElse {
throw new AssertionError(s"Expected ${ZkUtils.BrokerTopicsPath} to exist, but it does not. ")
}
// call to bootstrap topic list
topicEventListener.handleChildChange(ZkUtils.BrokerTopicsPath, topics.asJava)
}
private def stopWatchingTopicEvents() { zkUtils.unsubscribeAll() }
def shutdown() {
lock.synchronized {
info("Shutting down topic event watcher.")
if (zkUtils != null) {
stopWatchingTopicEvents()
}
else {
warn("Cannot shutdown since the embedded zookeeper client has already closed.")
}
}
}
class ZkTopicEventListener extends IZkChildListener {
@throws[Exception]
def handleChildChange(parent: String, children: java.util.List[String]) {
lock.synchronized {
try {
if (zkUtils != null) {
val latestTopics = zkUtils.getChildren(ZkUtils.BrokerTopicsPath)
debug("all topics: %s".format(latestTopics))
eventHandler.handleTopicEvent(latestTopics)
}
}
catch {
case e: Throwable =>
error("error in handling child changes", e)
}
}
}
}
class ZkSessionExpireListener(val topicEventListener: ZkTopicEventListener)
extends IZkStateListener {
@throws[Exception]
def handleStateChanged(state: KeeperState) { }
@throws[Exception]
def handleNewSession() {
lock.synchronized {
if (zkUtils != null) {
info("ZK expired: resubscribing topic event listener to topic registry")
zkUtils.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicEventListener)
}
}
}
override def handleSessionEstablishmentError(error: Throwable): Unit = {
//no-op ZookeeperConsumerConnector should log error.
}
}
}

View File

@ -1,19 +0,0 @@
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
This is the consumer API for kafka.

View File

@ -1,70 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import java.util
import kafka.common.TopicAndPartition
import kafka.api.{PartitionFetchInfo, Request}
import scala.collection.JavaConverters._
object FetchRequest {
private def seqToLinkedHashMap[K, V](s: Seq[(K, V)]): util.LinkedHashMap[K, V] = {
val map = new util.LinkedHashMap[K, V]
s.foreach { case (k, v) => map.put(k, v) }
map
}
}
class FetchRequest(correlationId: Int,
clientId: String,
maxWait: Int,
minBytes: Int,
requestInfo: util.LinkedHashMap[TopicAndPartition, PartitionFetchInfo]) {
@deprecated("The order of partitions in `requestInfo` is relevant, so this constructor is deprecated in favour of the " +
"one that takes a LinkedHashMap", since = "0.10.1.0")
def this(correlationId: Int, clientId: String, maxWait: Int, minBytes: Int,
requestInfo: java.util.Map[TopicAndPartition, PartitionFetchInfo]) {
this(correlationId, clientId, maxWait, minBytes,
FetchRequest.seqToLinkedHashMap(kafka.api.FetchRequest.shuffle(requestInfo.asScala.toSeq)))
}
val underlying = kafka.api.FetchRequest(
correlationId = correlationId,
clientId = clientId,
replicaId = Request.OrdinaryConsumerId,
maxWait = maxWait,
minBytes = minBytes,
requestInfo = requestInfo.asScala.toBuffer
)
override def toString = underlying.toString
override def equals(obj: Any): Boolean = {
obj match {
case null => false
case other: FetchRequest => this.underlying.equals(other.underlying)
case _ => false
}
}
override def hashCode = underlying.hashCode
}

View File

@ -1,44 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
class FetchResponse(private val underlying: kafka.api.FetchResponse) {
def messageSet(topic: String, partition: Int): kafka.javaapi.message.ByteBufferMessageSet = {
import Implicits._
underlying.messageSet(topic, partition)
}
def highWatermark(topic: String, partition: Int) = underlying.highWatermark(topic, partition)
def hasError = underlying.hasError
def error(topic: String, partition: Int) = underlying.error(topic, partition)
def errorCode(topic: String, partition: Int) = error(topic, partition).code
override def equals(obj: Any): Boolean = {
obj match {
case null => false
case other: FetchResponse => this.underlying.equals(other.underlying)
case _ => false
}
}
override def hashCode = underlying.hashCode
}

View File

@ -1,50 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import java.nio.ByteBuffer
import kafka.cluster.BrokerEndPoint
class GroupCoordinatorResponse(private val underlying: kafka.api.GroupCoordinatorResponse) {
def error = underlying.error
def errorCode = error.code
def coordinator: BrokerEndPoint = {
import kafka.javaapi.Implicits._
underlying.coordinatorOpt
}
override def equals(obj: Any): Boolean = {
obj match {
case null => false
case other: GroupCoordinatorResponse => this.underlying.equals(other.underlying)
case _ => false
}
}
override def hashCode = underlying.hashCode
override def toString = underlying.toString
}
object GroupCoordinatorResponse {
def readFrom(buffer: ByteBuffer) = new GroupCoordinatorResponse(kafka.api.GroupCoordinatorResponse.readFrom(buffer))
}

View File

@ -1,50 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import kafka.utils.Logging
private[javaapi] object Implicits extends Logging {
implicit def scalaMessageSetToJavaMessageSet(messageSet: kafka.message.ByteBufferMessageSet):
kafka.javaapi.message.ByteBufferMessageSet = {
new kafka.javaapi.message.ByteBufferMessageSet(messageSet.buffer)
}
implicit def toJavaFetchResponse(response: kafka.api.FetchResponse): kafka.javaapi.FetchResponse =
new kafka.javaapi.FetchResponse(response)
implicit def toJavaTopicMetadataResponse(response: kafka.api.TopicMetadataResponse): kafka.javaapi.TopicMetadataResponse =
new kafka.javaapi.TopicMetadataResponse(response)
implicit def toJavaOffsetResponse(response: kafka.api.OffsetResponse): kafka.javaapi.OffsetResponse =
new kafka.javaapi.OffsetResponse(response)
implicit def toJavaOffsetFetchResponse(response: kafka.api.OffsetFetchResponse): kafka.javaapi.OffsetFetchResponse =
new kafka.javaapi.OffsetFetchResponse(response)
implicit def toJavaOffsetCommitResponse(response: kafka.api.OffsetCommitResponse): kafka.javaapi.OffsetCommitResponse =
new kafka.javaapi.OffsetCommitResponse(response)
implicit def optionToJavaRef[T](opt: Option[T]): T = {
opt match {
case Some(obj) => obj
case None => null.asInstanceOf[T]
}
}
}

View File

@ -1,59 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import kafka.common.{OffsetAndMetadata, TopicAndPartition}
import scala.collection.JavaConverters._
class OffsetCommitRequest(groupId: String,
requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata],
correlationId: Int,
clientId: String,
versionId: Short) {
val underlying = {
val scalaMap: collection.immutable.Map[TopicAndPartition, OffsetAndMetadata] = requestInfo.asScala.toMap
kafka.api.OffsetCommitRequest(
groupId = groupId,
requestInfo = scalaMap,
versionId = versionId,
correlationId = correlationId,
clientId = clientId
)
}
def this(groupId: String,
requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata],
correlationId: Int,
clientId: String) {
// by default bind to version 0 so that it commits to Zookeeper
this(groupId, requestInfo, correlationId, clientId, 0)
}
override def toString = underlying.toString
override def equals(obj: Any): Boolean = {
obj match {
case null => false
case other: OffsetCommitRequest => this.underlying.equals(other.underlying)
case _ => false
}
}
override def hashCode = underlying.hashCode
}

View File

@ -1,39 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import java.nio.ByteBuffer
import kafka.common.TopicAndPartition
import org.apache.kafka.common.protocol.Errors
import scala.collection.JavaConverters._
class OffsetCommitResponse(private val underlying: kafka.api.OffsetCommitResponse) {
def errors: java.util.Map[TopicAndPartition, Errors] = underlying.commitStatus.asJava
def hasError = underlying.hasError
def error(topicAndPartition: TopicAndPartition) = underlying.commitStatus(topicAndPartition)
def errorCode(topicAndPartition: TopicAndPartition) = error(topicAndPartition).code
}
object OffsetCommitResponse {
def readFrom(buffer: ByteBuffer) = new OffsetCommitResponse(kafka.api.OffsetCommitResponse.readFrom(buffer))
}

View File

@ -1,60 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import kafka.common.TopicAndPartition
import collection.JavaConverters._
class OffsetFetchRequest(groupId: String,
requestInfo: java.util.List[TopicAndPartition],
versionId: Short,
correlationId: Int,
clientId: String) {
def this(groupId: String,
requestInfo: java.util.List[TopicAndPartition],
correlationId: Int,
clientId: String) {
// by default bind to version 0 so that it fetches from ZooKeeper
this(groupId, requestInfo, 0, correlationId, clientId)
}
val underlying = {
kafka.api.OffsetFetchRequest(
groupId = groupId,
requestInfo = requestInfo.asScala,
versionId = versionId,
correlationId = correlationId,
clientId = clientId
)
}
override def toString = underlying.toString
override def equals(obj: Any): Boolean = {
obj match {
case null => false
case other: OffsetFetchRequest => this.underlying.equals(other.underlying)
case _ => false
}
}
override def hashCode = underlying.hashCode
}

View File

@ -1,33 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import java.nio.ByteBuffer
import kafka.common.{TopicAndPartition, OffsetMetadataAndError}
import collection.JavaConverters._
class OffsetFetchResponse(private val underlying: kafka.api.OffsetFetchResponse) {
def offsets: java.util.Map[TopicAndPartition, OffsetMetadataAndError] = underlying.requestInfo.asJava
}
object OffsetFetchResponse {
def readFrom(buffer: ByteBuffer) = new OffsetFetchResponse(kafka.api.OffsetFetchResponse.readFrom(buffer))
}

View File

@ -1,50 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import kafka.common.TopicAndPartition
import kafka.api.{Request, PartitionOffsetRequestInfo}
import scala.collection.JavaConverters._
class OffsetRequest(requestInfo: java.util.Map[TopicAndPartition, PartitionOffsetRequestInfo],
versionId: Short,
clientId: String) {
val underlying = {
val scalaMap = requestInfo.asScala.toMap
kafka.api.OffsetRequest(
requestInfo = scalaMap,
versionId = versionId,
clientId = clientId,
replicaId = Request.OrdinaryConsumerId
)
}
override def toString = underlying.toString
override def equals(obj: Any): Boolean = {
obj match {
case null => false
case other: OffsetRequest => this.underlying.equals(other.underlying)
case _ => false
}
}
override def hashCode = underlying.hashCode
}

View File

@ -1,45 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import kafka.common.TopicAndPartition
class OffsetResponse(private val underlying: kafka.api.OffsetResponse) {
def hasError = underlying.hasError
def error(topic: String, partition: Int) =
underlying.partitionErrorAndOffsets(TopicAndPartition(topic, partition)).error
def errorCode(topic: String, partition: Int) = error(topic, partition).code
def offsets(topic: String, partition: Int) =
underlying.partitionErrorAndOffsets(TopicAndPartition(topic, partition)).offsets.toArray
override def equals(obj: Any): Boolean = {
obj match {
case null => false
case other: OffsetResponse => this.underlying.equals(other.underlying)
case _ => false
}
}
override def hashCode = underlying.hashCode
override def toString = underlying.toString
}

View File

@ -1,67 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import kafka.cluster.BrokerEndPoint
import scala.collection.JavaConverters._
private[javaapi] object MetadataListImplicits {
implicit def toJavaTopicMetadataList(topicMetadataSeq: Seq[kafka.api.TopicMetadata]):
java.util.List[kafka.javaapi.TopicMetadata] = topicMetadataSeq.map(new kafka.javaapi.TopicMetadata(_)).asJava
implicit def toPartitionMetadataList(partitionMetadataSeq: Seq[kafka.api.PartitionMetadata]):
java.util.List[kafka.javaapi.PartitionMetadata] = partitionMetadataSeq.map(new kafka.javaapi.PartitionMetadata(_)).asJava
}
class TopicMetadata(private val underlying: kafka.api.TopicMetadata) {
def topic: String = underlying.topic
def partitionsMetadata: java.util.List[PartitionMetadata] = {
import kafka.javaapi.MetadataListImplicits._
underlying.partitionsMetadata
}
def error = underlying.error
def errorCode = error.code
def sizeInBytes: Int = underlying.sizeInBytes
override def toString = underlying.toString
}
class PartitionMetadata(private val underlying: kafka.api.PartitionMetadata) {
def partitionId: Int = underlying.partitionId
def leader: BrokerEndPoint = {
import kafka.javaapi.Implicits._
underlying.leader
}
def replicas: java.util.List[BrokerEndPoint] = underlying.replicas.asJava
def isr: java.util.List[BrokerEndPoint] = underlying.isr.asJava
def error = underlying.error
def errorCode = error.code
def sizeInBytes: Int = underlying.sizeInBytes
override def toString = underlying.toString
}

View File

@ -1,66 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
import java.nio.ByteBuffer
import kafka.api._
import org.apache.kafka.common.protocol.ApiKeys
import scala.collection.JavaConverters._
class TopicMetadataRequest(val versionId: Short,
val correlationId: Int,
val clientId: String,
val topics: java.util.List[String])
extends RequestOrResponse(Some(ApiKeys.METADATA.id)) {
val underlying: kafka.api.TopicMetadataRequest = new kafka.api.TopicMetadataRequest(versionId, correlationId, clientId, topics.asScala)
def this(topics: java.util.List[String]) =
this(kafka.api.TopicMetadataRequest.CurrentVersion, 0, kafka.api.TopicMetadataRequest.DefaultClientId, topics)
def this(topics: java.util.List[String], correlationId: Int) =
this(kafka.api.TopicMetadataRequest.CurrentVersion, correlationId, kafka.api.TopicMetadataRequest.DefaultClientId, topics)
def writeTo(buffer: ByteBuffer) = underlying.writeTo(buffer)
def sizeInBytes: Int = underlying.sizeInBytes
override def toString: String = {
describe(true)
}
override def describe(details: Boolean): String = {
val topicMetadataRequest = new StringBuilder
topicMetadataRequest.append("Name: " + this.getClass.getSimpleName)
topicMetadataRequest.append("; Version: " + versionId)
topicMetadataRequest.append("; CorrelationId: " + correlationId)
topicMetadataRequest.append("; ClientId: " + clientId)
if(details) {
topicMetadataRequest.append("; Topics: ")
val topicIterator = topics.iterator()
while (topicIterator.hasNext) {
val topic = topicIterator.next()
topicMetadataRequest.append("%s".format(topic))
if(topicIterator.hasNext)
topicMetadataRequest.append(",")
}
}
topicMetadataRequest.toString()
}
}

View File

@ -1,38 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi
class TopicMetadataResponse(private val underlying: kafka.api.TopicMetadataResponse) {
def sizeInBytes: Int = underlying.sizeInBytes
def topicsMetadata: java.util.List[kafka.javaapi.TopicMetadata] = {
import kafka.javaapi.MetadataListImplicits._
underlying.topicsMetadata
}
override def equals(obj: Any): Boolean = {
obj match {
case null => false
case other: TopicMetadataResponse => this.underlying.equals(other.underlying)
case _ => false
}
}
override def hashCode = underlying.hashCode
override def toString = underlying.toString
}

View File

@ -1,91 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi.consumer;
import kafka.common.OffsetAndMetadata;
import kafka.common.TopicAndPartition;
import kafka.consumer.KafkaStream;
import kafka.consumer.TopicFilter;
import kafka.serializer.Decoder;
import java.util.List;
import java.util.Map;
/**
* @deprecated since 0.11.0.0, this interface will be removed in a future release.
*/
@Deprecated
public interface ConsumerConnector {
/**
* Create a list of MessageStreams of type T for each topic.
*
* @param topicCountMap a map of (topic, #streams) pair
* @param keyDecoder a decoder that decodes the message key
* @param valueDecoder a decoder that decodes the message itself
* @return a map of (topic, list of KafkaStream) pairs.
* The number of items in the list is #streams. Each stream supports
* an iterator over message/metadata pairs.
*/
public <K, V> Map<String, List<KafkaStream<K, V>>>
createMessageStreams(Map<String, Integer> topicCountMap, Decoder<K> keyDecoder, Decoder<V> valueDecoder);
public Map<String, List<KafkaStream<byte[], byte[]>>> createMessageStreams(Map<String, Integer> topicCountMap);
/**
* Create a list of MessageAndTopicStreams containing messages of type T.
*
* @param topicFilter a TopicFilter that specifies which topics to
* subscribe to (encapsulates a whitelist or a blacklist).
* @param numStreams the number of message streams to return.
* @param keyDecoder a decoder that decodes the message key
* @param valueDecoder a decoder that decodes the message itself
* @return a list of KafkaStream. Each stream supports an
* iterator over its MessageAndMetadata elements.
*/
public <K, V> List<KafkaStream<K, V>>
createMessageStreamsByFilter(TopicFilter topicFilter, int numStreams, Decoder<K> keyDecoder, Decoder<V> valueDecoder);
public List<KafkaStream<byte[], byte[]>> createMessageStreamsByFilter(TopicFilter topicFilter, int numStreams);
public List<KafkaStream<byte[], byte[]>> createMessageStreamsByFilter(TopicFilter topicFilter);
/**
* Commit the offsets of all broker partitions connected by this connector.
*/
public void commitOffsets();
public void commitOffsets(boolean retryOnFailure);
/**
* Commit offsets using the provided offsets map
*
* @param offsetsToCommit a map containing the offset to commit for each partition.
* @param retryOnFailure enable retries on the offset commit if it fails.
*/
public void commitOffsets(Map<TopicAndPartition, OffsetAndMetadata> offsetsToCommit, boolean retryOnFailure);
/**
* Wire in a consumer rebalance listener to be executed when consumer rebalance occurs.
* @param listener The consumer rebalance listener to wire in
*/
public void setConsumerRebalanceListener(ConsumerRebalanceListener listener);
/**
* Shut down the connector
*/
public void shutdown();
}

View File

@ -1,54 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi.consumer;
import kafka.consumer.ConsumerThreadId;
import java.util.Map;
import java.util.Set;
/**
* This listener is used for execution of tasks defined by user when a consumer rebalance
* occurs in {@link kafka.consumer.ZookeeperConsumerConnector}
*/
/**
* @deprecated since 0.11.0.0, this interface will be removed in a future release.
*/
@Deprecated
public interface ConsumerRebalanceListener {
/**
* This method is called after all the fetcher threads are stopped but before the
* ownership of partitions are released. Depending on whether auto offset commit is
* enabled or not, offsets may or may not have been committed.
* This listener is initially added to prevent duplicate messages on consumer rebalance
* in mirror maker, where offset auto commit is disabled to prevent data loss. It could
* also be used in more general cases.
* @param partitionOwnership The partition this consumer currently owns.
*/
public void beforeReleasingPartitions(Map<String, Set<Integer>> partitionOwnership);
/**
* This method is called after the new partition assignment is finished but before fetcher
* threads start. A map of new global partition assignment is passed in as parameter.
* @param consumerId The consumer Id string of the consumer invoking this callback.
* @param globalPartitionAssignment A Map[topic, Map[Partition, ConsumerThreadId]]. It is the global partition
* assignment of this consumer group.
*/
public void beforeStartingFetchers(String consumerId, Map<String, Map<Integer, ConsumerThreadId>> globalPartitionAssignment);
}

View File

@ -1,107 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi.consumer
import kafka.utils.threadsafe
import kafka.javaapi.FetchResponse
import kafka.javaapi.OffsetRequest
/**
* A consumer of kafka messages
*/
@deprecated("This class has been deprecated and will be removed in a future release. " +
"Please use org.apache.kafka.clients.consumer.KafkaConsumer instead.", "0.11.0.0")
@threadsafe
class SimpleConsumer(val host: String,
val port: Int,
val soTimeout: Int,
val bufferSize: Int,
val clientId: String) {
private val underlying = new kafka.consumer.SimpleConsumer(host, port, soTimeout, bufferSize, clientId)
/**
* Fetch a set of messages from a topic. This version of the fetch method
* takes the Scala version of a fetch request (i.e.,
* [[kafka.api.FetchRequest]] and is intended for use with the
* [[kafka.api.FetchRequestBuilder]].
*
* @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched.
* @return a set of fetched messages
*/
def fetch(request: kafka.api.FetchRequest): FetchResponse = {
import kafka.javaapi.Implicits._
underlying.fetch(request)
}
/**
* Fetch a set of messages from a topic.
*
* @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched.
* @return a set of fetched messages
*/
def fetch(request: kafka.javaapi.FetchRequest): FetchResponse = {
fetch(request.underlying)
}
/**
* Fetch metadata for a sequence of topics.
*
* @param request specifies the versionId, clientId, sequence of topics.
* @return metadata for each topic in the request.
*/
def send(request: kafka.javaapi.TopicMetadataRequest): kafka.javaapi.TopicMetadataResponse = {
import kafka.javaapi.Implicits._
underlying.send(request.underlying)
}
/**
* Get a list of valid offsets (up to maxSize) before the given time.
*
* @param request a [[kafka.javaapi.OffsetRequest]] object.
* @return a [[kafka.javaapi.OffsetResponse]] object.
*/
def getOffsetsBefore(request: OffsetRequest): kafka.javaapi.OffsetResponse = {
import kafka.javaapi.Implicits._
underlying.getOffsetsBefore(request.underlying)
}
/**
* Commit offsets for a topic to Zookeeper
* @param request a [[kafka.javaapi.OffsetCommitRequest]] object.
* @return a [[kafka.javaapi.OffsetCommitResponse]] object.
*/
def commitOffsets(request: kafka.javaapi.OffsetCommitRequest): kafka.javaapi.OffsetCommitResponse = {
import kafka.javaapi.Implicits._
underlying.commitOffsets(request.underlying)
}
/**
* Fetch offsets for a topic from Zookeeper
* @param request a [[kafka.javaapi.OffsetFetchRequest]] object.
* @return a [[kafka.javaapi.OffsetFetchResponse]] object.
*/
def fetchOffsets(request: kafka.javaapi.OffsetFetchRequest): kafka.javaapi.OffsetFetchResponse = {
import kafka.javaapi.Implicits._
underlying.fetchOffsets(request.underlying)
}
def close() {
underlying.close
}
}

View File

@ -1,126 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi.consumer
import kafka.serializer._
import kafka.consumer._
import kafka.common.{OffsetAndMetadata, TopicAndPartition, MessageStreamsExistException}
import java.util.concurrent.atomic.AtomicBoolean
import scala.collection.JavaConverters._
/**
* This class handles the consumers interaction with zookeeper
*
* Directories:
* 1. Consumer id registry:
* /consumers/[group_id]/ids[consumer_id] -> topic1,...topicN
* A consumer has a unique consumer id within a consumer group. A consumer registers its id as an ephemeral znode
* and puts all topics that it subscribes to as the value of the znode. The znode is deleted when the client is gone.
* A consumer subscribes to event changes of the consumer id registry within its group.
*
* The consumer id is picked up from configuration, instead of the sequential id assigned by ZK. Generated sequential
* ids are hard to recover during temporary connection loss to ZK, since it's difficult for the client to figure out
* whether the creation of a sequential znode has succeeded or not. More details can be found at
* (http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling)
*
* 2. Broker node registry:
* /brokers/[0...N] --> { "host" : "host:port",
* "topics" : {"topic1": ["partition1" ... "partitionN"], ...,
* "topicN": ["partition1" ... "partitionN"] } }
* This is a list of all present broker brokers. A unique logical node id is configured on each broker node. A broker
* node registers itself on start-up and creates a znode with the logical node id under /brokers. The value of the znode
* is a JSON String that contains (1) the host name and the port the broker is listening to, (2) a list of topics that
* the broker serves, (3) a list of logical partitions assigned to each topic on the broker.
* A consumer subscribes to event changes of the broker node registry.
*
* 3. Partition owner registry:
* /consumers/[group_id]/owner/[topic]/[broker_id-partition_id] --> consumer_node_id
* This stores the mapping before broker partitions and consumers. Each partition is owned by a unique consumer
* within a consumer group. The mapping is reestablished after each rebalancing.
*
* 4. Consumer offset tracking:
* /consumers/[group_id]/offsets/[topic]/[broker_id-partition_id] --> offset_counter_value
* Each consumer tracks the offset of the latest message consumed for each partition.
*
*/
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val enableFetcher: Boolean) // for testing only
extends ConsumerConnector {
private val underlying = new kafka.consumer.ZookeeperConsumerConnector(config, enableFetcher)
private val messageStreamCreated = new AtomicBoolean(false)
def this(config: ConsumerConfig) = this(config, true)
// for java client
def createMessageStreams[K,V](
topicCountMap: java.util.Map[String,java.lang.Integer],
keyDecoder: Decoder[K],
valueDecoder: Decoder[V])
: java.util.Map[String,java.util.List[KafkaStream[K,V]]] = {
if (messageStreamCreated.getAndSet(true))
throw new MessageStreamsExistException(this.getClass.getSimpleName +
" can create message streams at most once",null)
val scalaTopicCountMap: Map[String, Int] = {
Map.empty[String, Int] ++ topicCountMap.asInstanceOf[java.util.Map[String, Int]].asScala
}
val scalaReturn = underlying.consume(scalaTopicCountMap, keyDecoder, valueDecoder)
val ret = new java.util.HashMap[String,java.util.List[KafkaStream[K,V]]]
for ((topic, streams) <- scalaReturn) {
val javaStreamList = new java.util.ArrayList[KafkaStream[K,V]]
for (stream <- streams)
javaStreamList.add(stream)
ret.put(topic, javaStreamList)
}
ret
}
def createMessageStreams(topicCountMap: java.util.Map[String,java.lang.Integer]): java.util.Map[String,java.util.List[KafkaStream[Array[Byte],Array[Byte]]]] =
createMessageStreams(topicCountMap, new DefaultDecoder(), new DefaultDecoder())
def createMessageStreamsByFilter[K,V](topicFilter: TopicFilter, numStreams: Int, keyDecoder: Decoder[K], valueDecoder: Decoder[V]) =
underlying.createMessageStreamsByFilter(topicFilter, numStreams, keyDecoder, valueDecoder).asJava
def createMessageStreamsByFilter(topicFilter: TopicFilter, numStreams: Int) =
createMessageStreamsByFilter(topicFilter, numStreams, new DefaultDecoder(), new DefaultDecoder())
def createMessageStreamsByFilter(topicFilter: TopicFilter) =
createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder(), new DefaultDecoder())
def commitOffsets() {
underlying.commitOffsets(true)
}
def commitOffsets(retryOnFailure: Boolean) {
underlying.commitOffsets(retryOnFailure)
}
def commitOffsets(offsetsToCommit: java.util.Map[TopicAndPartition, OffsetAndMetadata], retryOnFailure: Boolean) {
underlying.commitOffsets(offsetsToCommit.asScala.toMap, retryOnFailure)
}
def setConsumerRebalanceListener(consumerRebalanceListener: ConsumerRebalanceListener) {
underlying.setConsumerRebalanceListener(consumerRebalanceListener)
}
def shutdown() {
underlying.shutdown
}
}

View File

@ -1,67 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi.message
import java.nio.ByteBuffer
import kafka.common.LongRef
import kafka.message._
import scala.collection.JavaConverters._
class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet {
private val underlying: kafka.message.ByteBufferMessageSet = new kafka.message.ByteBufferMessageSet(buffer)
def this(compressionCodec: CompressionCodec, messages: java.util.List[Message]) {
this(new kafka.message.ByteBufferMessageSet(compressionCodec, new LongRef(0), messages.asScala: _*).buffer)
}
def this(messages: java.util.List[Message]) {
this(NoCompressionCodec, messages)
}
def validBytes: Int = underlying.validBytes
def getBuffer = buffer
override def iterator: java.util.Iterator[MessageAndOffset] = new java.util.Iterator[MessageAndOffset] {
val underlyingIterator = underlying.iterator
override def hasNext(): Boolean = {
underlyingIterator.hasNext
}
override def next(): MessageAndOffset = {
underlyingIterator.next
}
override def remove = throw new UnsupportedOperationException("remove API on MessageSet is not supported")
}
override def toString: String = underlying.toString
def sizeInBytes: Int = underlying.sizeInBytes
override def equals(other: Any): Boolean = {
other match {
case that: ByteBufferMessageSet => buffer.equals(that.buffer)
case _ => false
}
}
override def hashCode: Int = buffer.hashCode
}

View File

@ -1,55 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.javaapi.message
import kafka.message.{MessageAndOffset, InvalidMessageException}
/**
* A set of messages. A message set has a fixed serialized form, though the container
* for the bytes could be either in-memory or on disk. A The format of each message is
* as follows:
* 4 byte size containing an integer N
* N message bytes as described in the message class
*/
abstract class MessageSet extends java.lang.Iterable[MessageAndOffset] {
/**
* Provides an iterator over the messages in this set
*/
def iterator: java.util.Iterator[MessageAndOffset]
/**
* Gives the total size of this message set in bytes
*/
def sizeInBytes: Int
/**
* Validate the checksum of all the messages in the set. Throws an InvalidMessageException if the checksum doesn't
* match the payload for any message.
*/
def validate(): Unit = {
val thisIterator = this.iterator
while(thisIterator.hasNext) {
val messageAndOffset = thisIterator.next
if(!messageAndOffset.message.isValid)
throw new InvalidMessageException
}
}
}

View File

@ -1,197 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.message
import java.nio.ByteBuffer
import kafka.common.LongRef
import kafka.utils.Logging
import org.apache.kafka.common.record._
import scala.collection.JavaConverters._
object ByteBufferMessageSet {
private def create(offsetAssigner: OffsetAssigner,
compressionCodec: CompressionCodec,
timestampType: TimestampType,
messages: Message*): ByteBuffer = {
if (messages.isEmpty)
MessageSet.Empty.buffer
else {
val buffer = ByteBuffer.allocate(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
val builder = MemoryRecords.builder(buffer, messages.head.magic, CompressionType.forId(compressionCodec.codec),
timestampType, offsetAssigner.baseOffset)
for (message <- messages)
builder.appendWithOffset(offsetAssigner.nextAbsoluteOffset(), message.asRecord)
builder.build().buffer
}
}
}
private object OffsetAssigner {
def apply(offsetCounter: LongRef, size: Int): OffsetAssigner =
new OffsetAssigner(offsetCounter.value to offsetCounter.addAndGet(size))
}
private class OffsetAssigner(offsets: Seq[Long]) {
private var index = 0
def nextAbsoluteOffset(): Long = {
val result = offsets(index)
index += 1
result
}
def baseOffset = offsets.head
def toInnerOffset(offset: Long): Long = offset - offsets.head
}
/**
* A sequence of messages stored in a byte buffer
*
* There are two ways to create a ByteBufferMessageSet
*
* Option 1: From a ByteBuffer which already contains the serialized message set. Consumers will use this method.
*
* Option 2: Give it a list of messages along with instructions relating to serialization format. Producers will use this method.
*
*
* Message format v1 has the following changes:
* - For non-compressed messages, timestamp and timestamp type attributes have been added. The offsets of
* the messages remain absolute offsets.
* - For compressed messages, timestamp and timestamp type attributes have been added and inner offsets (IO) are used
* for inner messages of compressed messages (see offset calculation details below). The timestamp type
* attribute is only set in wrapper messages. Inner messages always have CreateTime as the timestamp type in attributes.
*
* We set the timestamp in the following way:
* For non-compressed messages: the timestamp and timestamp type message attributes are set and used.
* For compressed messages:
* 1. Wrapper messages' timestamp type attribute is set to the proper value
* 2. Wrapper messages' timestamp is set to:
* - the max timestamp of inner messages if CreateTime is used
* - the current server time if wrapper message's timestamp = LogAppendTime.
* In this case the wrapper message timestamp is used and all the timestamps of inner messages are ignored.
* 3. Inner messages' timestamp will be:
* - used when wrapper message's timestamp type is CreateTime
* - ignored when wrapper message's timestamp type is LogAppendTime
* 4. Inner messages' timestamp type will always be ignored with one exception: producers must set the inner message
* timestamp type to CreateTime, otherwise the messages will be rejected by broker.
*
* Absolute offsets are calculated in the following way:
* Ideally the conversion from relative offset(RO) to absolute offset(AO) should be:
*
* AO = AO_Of_Last_Inner_Message + RO
*
* However, note that the message sets sent by producers are compressed in a streaming way.
* And the relative offset of an inner message compared with the last inner message is not known until
* the last inner message is written.
* Unfortunately we are not able to change the previously written messages after the last message is written to
* the message set when stream compression is used.
*
* To solve this issue, we use the following solution:
*
* 1. When the producer creates a message set, it simply writes all the messages into a compressed message set with
* offset 0, 1, ... (inner offset).
* 2. The broker will set the offset of the wrapper message to the absolute offset of the last message in the
* message set.
* 3. When a consumer sees the message set, it first decompresses the entire message set to find out the inner
* offset (IO) of the last inner message. Then it computes RO and AO of previous messages:
*
* RO = IO_of_a_message - IO_of_the_last_message
* AO = AO_Of_Last_Inner_Message + RO
*
* 4. This solution works for compacted message sets as well.
*
*/
class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging {
private[kafka] def this(compressionCodec: CompressionCodec,
offsetCounter: LongRef,
timestampType: TimestampType,
messages: Message*) {
this(ByteBufferMessageSet.create(OffsetAssigner(offsetCounter, messages.size), compressionCodec,
timestampType, messages:_*))
}
def this(compressionCodec: CompressionCodec, offsetCounter: LongRef, messages: Message*) {
this(compressionCodec, offsetCounter, TimestampType.CREATE_TIME, messages:_*)
}
def this(compressionCodec: CompressionCodec, offsetSeq: Seq[Long], messages: Message*) {
this(ByteBufferMessageSet.create(new OffsetAssigner(offsetSeq), compressionCodec,
TimestampType.CREATE_TIME, messages:_*))
}
def this(compressionCodec: CompressionCodec, messages: Message*) {
this(compressionCodec, new LongRef(0L), messages: _*)
}
def this(messages: Message*) {
this(NoCompressionCodec, messages: _*)
}
def getBuffer = buffer
override def asRecords: MemoryRecords = MemoryRecords.readableRecords(buffer.duplicate())
/** default iterator that iterates over decompressed messages */
override def iterator: Iterator[MessageAndOffset] = internalIterator()
/** iterator over compressed messages without decompressing */
def shallowIterator: Iterator[MessageAndOffset] = internalIterator(isShallow = true)
/** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. **/
private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = {
if (isShallow)
asRecords.batches.asScala.iterator.map(MessageAndOffset.fromRecordBatch)
else
asRecords.records.asScala.iterator.map(MessageAndOffset.fromRecord)
}
/**
* The total number of bytes in this message set, including any partial trailing messages
*/
def sizeInBytes: Int = buffer.limit()
/**
* The total number of bytes in this message set not including any partial, trailing messages
*/
def validBytes: Int = asRecords.validBytes
/**
* Two message sets are equal if their respective byte buffers are equal
*/
override def equals(other: Any): Boolean = {
other match {
case that: ByteBufferMessageSet =>
buffer.equals(that.buffer)
case _ => false
}
}
override def hashCode: Int = buffer.hashCode
}

View File

@ -1,33 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.message
import org.apache.kafka.common.errors.CorruptRecordException
/**
* Indicates that a message failed its checksum and is corrupt
*
* InvalidMessageException extends CorruptRecordException for temporary compatibility with the old Scala clients.
* We want to update the server side code to use and catch the new CorruptRecordException.
* Because ByteBufferMessageSet.scala and Message.scala are used in both server and client code having
* InvalidMessageException extend CorruptRecordException allows us to change server code without affecting the client.
*/
class InvalidMessageException(message: String, throwable: Throwable) extends CorruptRecordException(message, throwable) {
def this(message: String) = this(null, null)
def this() = this(null)
}

View File

@ -1,379 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.message
import java.nio._
import org.apache.kafka.common.record.{CompressionType, LegacyRecord, TimestampType}
import scala.math._
import org.apache.kafka.common.utils.{ByteUtils, Crc32}
/**
* Constants related to messages
*/
object Message {
/**
* The current offset and size for all the fixed-length fields
*/
val CrcOffset = 0
val CrcLength = 4
val MagicOffset = CrcOffset + CrcLength
val MagicLength = 1
val AttributesOffset = MagicOffset + MagicLength
val AttributesLength = 1
// Only message format version 1 has the timestamp field.
val TimestampOffset = AttributesOffset + AttributesLength
val TimestampLength = 8
val KeySizeOffset_V0 = AttributesOffset + AttributesLength
val KeySizeOffset_V1 = TimestampOffset + TimestampLength
val KeySizeLength = 4
val KeyOffset_V0 = KeySizeOffset_V0 + KeySizeLength
val KeyOffset_V1 = KeySizeOffset_V1 + KeySizeLength
val ValueSizeLength = 4
private val MessageHeaderSizeMap = Map (
(0: Byte) -> (CrcLength + MagicLength + AttributesLength + KeySizeLength + ValueSizeLength),
(1: Byte) -> (CrcLength + MagicLength + AttributesLength + TimestampLength + KeySizeLength + ValueSizeLength))
/**
* The amount of overhead bytes in a message
* This value is only used to check if the message size is valid or not. So the minimum possible message bytes is
* used here, which comes from a message in message format V0 with empty key and value.
*/
val MinMessageOverhead = KeyOffset_V0 + ValueSizeLength
/**
* The "magic" value
* When magic value is 0, the message uses absolute offset and does not have a timestamp field.
* When magic value is 1, the message uses relative offset and has a timestamp field.
*/
val MagicValue_V0: Byte = 0
val MagicValue_V1: Byte = 1
val CurrentMagicValue: Byte = 1
/**
* Specifies the mask for the compression code. 3 bits to hold the compression codec.
* 0 is reserved to indicate no compression
*/
val CompressionCodeMask: Int = 0x07
/**
* Specifies the mask for timestamp type. 1 bit at the 4th least significant bit.
* 0 for CreateTime, 1 for LogAppendTime
*/
val TimestampTypeMask: Byte = 0x08
val TimestampTypeAttributeBitOffset: Int = 3
/**
* Compression code for uncompressed messages
*/
val NoCompression: Int = 0
/**
* To indicate timestamp is not defined so "magic" value 0 will be used.
*/
val NoTimestamp: Long = -1
/**
* Give the header size difference between different message versions.
*/
def headerSizeDiff(fromMagicValue: Byte, toMagicValue: Byte) : Int =
MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue)
def fromRecord(record: LegacyRecord): Message = {
val wrapperTimestamp: Option[Long] = if (record.wrapperRecordTimestamp == null) None else Some(record.wrapperRecordTimestamp)
val wrapperTimestampType = Option(record.wrapperRecordTimestampType)
new Message(record.buffer, wrapperTimestamp, wrapperTimestampType)
}
}
/**
* A message. The format of an N byte message is the following:
*
* 1. 4 byte CRC32 of the message
* 2. 1 byte "magic" identifier to allow format changes, value is 0 or 1
* 3. 1 byte "attributes" identifier to allow annotations on the message independent of the version
* bit 0 ~ 2 : Compression codec.
* 0 : no compression
* 1 : gzip
* 2 : snappy
* 3 : lz4
* bit 3 : Timestamp type
* 0 : create time
* 1 : log append time
* bit 4 ~ 7 : reserved
* 4. (Optional) 8 byte timestamp only if "magic" identifier is greater than 0
* 5. 4 byte key length, containing length K
* 6. K byte key
* 7. 4 byte payload length, containing length V
* 8. V byte payload
*
* Default constructor wraps an existing ByteBuffer with the Message object with no change to the contents.
* @param buffer the byte buffer of this message.
* @param wrapperMessageTimestamp the wrapper message timestamp, which is only defined when the message is an inner
* message of a compressed message.
* @param wrapperMessageTimestampType the wrapper message timestamp type, which is only defined when the message is an
* inner message of a compressed message.
*/
class Message(val buffer: ByteBuffer,
private val wrapperMessageTimestamp: Option[Long] = None,
private val wrapperMessageTimestampType: Option[TimestampType] = None) {
import kafka.message.Message._
private[message] def asRecord: LegacyRecord = wrapperMessageTimestamp match {
case None => new LegacyRecord(buffer)
case Some(timestamp) => new LegacyRecord(buffer, timestamp, wrapperMessageTimestampType.orNull)
}
/**
* A constructor to create a Message
* @param bytes The payload of the message
* @param key The key of the message (null, if none)
* @param timestamp The timestamp of the message.
* @param timestampType The timestamp type of the message.
* @param codec The compression codec used on the contents of the message (if any)
* @param payloadOffset The offset into the payload array used to extract payload
* @param payloadSize The size of the payload to use
* @param magicValue the magic value to use
*/
def this(bytes: Array[Byte],
key: Array[Byte],
timestamp: Long,
timestampType: TimestampType,
codec: CompressionCodec,
payloadOffset: Int,
payloadSize: Int,
magicValue: Byte) = {
this(ByteBuffer.allocate(Message.CrcLength +
Message.MagicLength +
Message.AttributesLength +
(if (magicValue == Message.MagicValue_V0) 0
else Message.TimestampLength) +
Message.KeySizeLength +
(if(key == null) 0 else key.length) +
Message.ValueSizeLength +
(if(bytes == null) 0
else if(payloadSize >= 0) payloadSize
else bytes.length - payloadOffset)))
validateTimestampAndMagicValue(timestamp, magicValue)
// skip crc, we will fill that in at the end
buffer.position(MagicOffset)
buffer.put(magicValue)
val attributes: Byte = LegacyRecord.computeAttributes(magicValue, CompressionType.forId(codec.codec), timestampType)
buffer.put(attributes)
// Only put timestamp when "magic" value is greater than 0
if (magic > MagicValue_V0)
buffer.putLong(timestamp)
if(key == null) {
buffer.putInt(-1)
} else {
buffer.putInt(key.length)
buffer.put(key, 0, key.length)
}
val size = if(bytes == null) -1
else if(payloadSize >= 0) payloadSize
else bytes.length - payloadOffset
buffer.putInt(size)
if(bytes != null)
buffer.put(bytes, payloadOffset, size)
buffer.rewind()
// now compute the checksum and fill it in
ByteUtils.writeUnsignedInt(buffer, CrcOffset, computeChecksum)
}
def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, timestampType: TimestampType, codec: CompressionCodec, magicValue: Byte) =
this(bytes = bytes, key = key, timestamp = timestamp, timestampType = timestampType, codec = codec, payloadOffset = 0, payloadSize = -1, magicValue = magicValue)
def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) =
this(bytes = bytes, key = key, timestamp = timestamp, timestampType = TimestampType.CREATE_TIME, codec = codec, payloadOffset = 0, payloadSize = -1, magicValue = magicValue)
def this(bytes: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) =
this(bytes = bytes, key = null, timestamp = timestamp, codec = codec, magicValue = magicValue)
def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, magicValue: Byte) =
this(bytes = bytes, key = key, timestamp = timestamp, codec = NoCompressionCodec, magicValue = magicValue)
def this(bytes: Array[Byte], timestamp: Long, magicValue: Byte) =
this(bytes = bytes, key = null, timestamp = timestamp, codec = NoCompressionCodec, magicValue = magicValue)
def this(bytes: Array[Byte]) =
this(bytes = bytes, key = null, timestamp = Message.NoTimestamp, codec = NoCompressionCodec, magicValue = Message.CurrentMagicValue)
/**
* Compute the checksum of the message from the message contents
*/
def computeChecksum: Long =
Crc32.crc32(buffer, MagicOffset, buffer.limit() - MagicOffset)
/**
* Retrieve the previously computed CRC for this message
*/
def checksum: Long = ByteUtils.readUnsignedInt(buffer, CrcOffset)
/**
* Returns true if the crc stored with the message matches the crc computed off the message contents
*/
def isValid: Boolean = checksum == computeChecksum
/**
* Throw an InvalidMessageException if isValid is false for this message
*/
def ensureValid() {
if(!isValid)
throw new InvalidMessageException(s"Message is corrupt (stored crc = ${checksum}, computed crc = ${computeChecksum})")
}
/**
* The complete serialized size of this message in bytes (including crc, header attributes, etc)
*/
def size: Int = buffer.limit()
/**
* The position where the key size is stored.
*/
private def keySizeOffset = {
if (magic == MagicValue_V0) KeySizeOffset_V0
else KeySizeOffset_V1
}
/**
* The length of the key in bytes
*/
def keySize: Int = buffer.getInt(keySizeOffset)
/**
* Does the message have a key?
*/
def hasKey: Boolean = keySize >= 0
/**
* The position where the payload size is stored
*/
private def payloadSizeOffset = {
if (magic == MagicValue_V0) KeyOffset_V0 + max(0, keySize)
else KeyOffset_V1 + max(0, keySize)
}
/**
* The length of the message value in bytes
*/
def payloadSize: Int = buffer.getInt(payloadSizeOffset)
/**
* Is the payload of this message null
*/
def isNull: Boolean = payloadSize < 0
/**
* The magic version of this message
*/
def magic: Byte = buffer.get(MagicOffset)
/**
* The attributes stored with this message
*/
def attributes: Byte = buffer.get(AttributesOffset)
/**
* The timestamp of the message, only available when the "magic" value is greater than 0
* When magic > 0, The timestamp of a message is determined in the following way:
* 1. wrapperMessageTimestampType = None and wrapperMessageTimestamp is None - Uncompressed message, timestamp and timestamp type are in the message.
* 2. wrapperMessageTimestampType = LogAppendTime and wrapperMessageTimestamp is defined - Compressed message using LogAppendTime
* 3. wrapperMessageTimestampType = CreateTime and wrapperMessageTimestamp is defined - Compressed message using CreateTime
*/
def timestamp: Long = {
if (magic == MagicValue_V0)
Message.NoTimestamp
// Case 2
else if (wrapperMessageTimestampType.exists(_ == TimestampType.LOG_APPEND_TIME) && wrapperMessageTimestamp.isDefined)
wrapperMessageTimestamp.get
else // case 1, 3
buffer.getLong(Message.TimestampOffset)
}
/**
* The timestamp type of the message
*/
def timestampType = LegacyRecord.timestampType(magic, wrapperMessageTimestampType.orNull, attributes)
/**
* The compression codec used with this message
*/
def compressionCodec: CompressionCodec =
CompressionCodec.getCompressionCodec(buffer.get(AttributesOffset) & CompressionCodeMask)
/**
* A ByteBuffer containing the content of the message
*/
def payload: ByteBuffer = sliceDelimited(payloadSizeOffset)
/**
* A ByteBuffer containing the message key
*/
def key: ByteBuffer = sliceDelimited(keySizeOffset)
/**
* Read a size-delimited byte buffer starting at the given offset
*/
private def sliceDelimited(start: Int): ByteBuffer = {
val size = buffer.getInt(start)
if(size < 0) {
null
} else {
var b = buffer.duplicate()
b.position(start + 4)
b = b.slice()
b.limit(size)
b.rewind
b
}
}
/**
* Validate the timestamp and "magic" value
*/
private def validateTimestampAndMagicValue(timestamp: Long, magic: Byte) {
if (magic != MagicValue_V0 && magic != MagicValue_V1)
throw new IllegalArgumentException(s"Invalid magic value $magic")
if (timestamp < 0 && timestamp != NoTimestamp)
throw new IllegalArgumentException(s"Invalid message timestamp $timestamp")
if (magic == MagicValue_V0 && timestamp != NoTimestamp)
throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be $NoTimestamp when magic = $MagicValue_V0")
}
override def toString: String = {
if (magic == MagicValue_V0)
s"Message(magic = $magic, attributes = $attributes, crc = $checksum, key = $key, payload = $payload)"
else
s"Message(magic = $magic, attributes = $attributes, $timestampType = $timestamp, crc = $checksum, key = $key, payload = $payload)"
}
override def equals(any: Any): Boolean = {
any match {
case that: Message => this.buffer.equals(that.buffer)
case _ => false
}
}
override def hashCode(): Int = buffer.hashCode
}

View File

@ -1,39 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.message
import kafka.serializer.Decoder
import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.common.utils.Utils
case class MessageAndMetadata[K, V](topic: String,
partition: Int,
private val rawMessage: Message,
offset: Long,
keyDecoder: Decoder[K], valueDecoder: Decoder[V],
timestamp: Long = Message.NoTimestamp,
timestampType: TimestampType = TimestampType.CREATE_TIME) {
/**
* Return the decoded message key and payload
*/
def key(): K = if(rawMessage.key == null) null.asInstanceOf[K] else keyDecoder.fromBytes(Utils.readBytes(rawMessage.key))
def message(): V = if(rawMessage.isNull) null.asInstanceOf[V] else valueDecoder.fromBytes(Utils.readBytes(rawMessage.payload))
}

View File

@ -1,54 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.message
import org.apache.kafka.common.record.{AbstractLegacyRecordBatch, Record, RecordBatch}
object MessageAndOffset {
def fromRecordBatch(batch: RecordBatch): MessageAndOffset = {
batch match {
case legacyBatch: AbstractLegacyRecordBatch =>
MessageAndOffset(Message.fromRecord(legacyBatch.outerRecord), legacyBatch.lastOffset)
case _ =>
throw new IllegalArgumentException(s"Illegal batch type ${batch.getClass}. The older message format classes " +
s"only support conversion from ${classOf[AbstractLegacyRecordBatch]}, which is used for magic v0 and v1")
}
}
def fromRecord(record: Record): MessageAndOffset = {
record match {
case legacyBatch: AbstractLegacyRecordBatch =>
MessageAndOffset(Message.fromRecord(legacyBatch.outerRecord), legacyBatch.lastOffset)
case _ =>
throw new IllegalArgumentException(s"Illegal record type ${record.getClass}. The older message format classes " +
s"only support conversion from ${classOf[AbstractLegacyRecordBatch]}, which is used for magic v0 and v1")
}
}
}
case class MessageAndOffset(message: Message, offset: Long) {
/**
* Compute the offset of the next message in the log
*/
def nextOffset: Long = offset + 1
}

View File

@ -1,24 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.message
/**
* Indicates the presence of a message that exceeds the maximum acceptable
* length (whatever that happens to be)
*/
class MessageLengthException(message: String) extends RuntimeException(message)

View File

@ -1,111 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.message
import java.nio._
import org.apache.kafka.common.record.Records
/**
* Message set helper functions
*/
object MessageSet {
val MessageSizeLength = 4
val OffsetLength = 8
val LogOverhead = MessageSizeLength + OffsetLength
val Empty = new ByteBufferMessageSet(ByteBuffer.allocate(0))
/**
* The size of a message set containing the given messages
*/
def messageSetSize(messages: Iterable[Message]): Int =
messages.foldLeft(0)(_ + entrySize(_))
/**
* The size of a size-delimited entry in a message set
*/
def entrySize(message: Message): Int = LogOverhead + message.size
/**
* Validate that all "magic" values in `messages` are the same and return their magic value and max timestamp
*/
def magicAndLargestTimestamp(messages: Seq[Message]): MagicAndTimestamp = {
val firstMagicValue = messages.head.magic
var largestTimestamp = Message.NoTimestamp
for (message <- messages) {
if (message.magic != firstMagicValue)
throw new IllegalStateException("Messages in the same message set must have same magic value")
if (firstMagicValue > Message.MagicValue_V0)
largestTimestamp = math.max(largestTimestamp, message.timestamp)
}
MagicAndTimestamp(firstMagicValue, largestTimestamp)
}
}
case class MagicAndTimestamp(magic: Byte, timestamp: Long)
/**
* A set of messages with offsets. A message set has a fixed serialized form, though the container
* for the bytes could be either in-memory or on disk. The format of each message is
* as follows:
* 8 byte message offset number
* 4 byte size containing an integer N
* N message bytes as described in the Message class
*/
abstract class MessageSet extends Iterable[MessageAndOffset] {
/**
* Provides an iterator over the message/offset pairs in this set
*/
def iterator: Iterator[MessageAndOffset]
/**
* Gives the total size of this message set in bytes
*/
def sizeInBytes: Int
/**
* Get the client representation of the message set
*/
def asRecords: Records
/**
* Print this message set's contents. If the message set has more than 100 messages, just
* print the first 100.
*/
override def toString: String = {
val builder = new StringBuilder()
builder.append(getClass.getSimpleName + "(")
val iter = this.asRecords.batches.iterator
var i = 0
while(iter.hasNext && i < 100) {
val message = iter.next
builder.append(message)
if(iter.hasNext)
builder.append(", ")
i += 1
}
if(iter.hasNext)
builder.append("...")
builder.append(")")
builder.toString
}
}

View File

@ -1,19 +0,0 @@
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
Messages and everything related to them.

View File

@ -21,14 +21,9 @@ import java.util.concurrent.TimeUnit
import com.yammer.metrics.Metrics
import com.yammer.metrics.core.{Gauge, MetricName}
import kafka.consumer.{ConsumerTopicStatsRegistry, FetchRequestAndResponseStatsRegistry}
import kafka.utils.Logging
import org.apache.kafka.common.utils.Sanitizer
import scala.collection.immutable
import scala.collection.JavaConverters._
trait KafkaMetricsGroup extends Logging {
/**
@ -63,8 +58,8 @@ trait KafkaMetricsGroup extends Logging {
nameBuilder.append(name)
}
val scope: String = KafkaMetricsGroup.toScope(tags).getOrElse(null)
val tagsName = KafkaMetricsGroup.toMBeanName(tags)
val scope: String = toScope(tags).getOrElse(null)
val tagsName = toMBeanName(tags)
tagsName.foreach(nameBuilder.append(",").append(_))
new MetricName(group, typeName, name, scope, nameBuilder.toString)
@ -85,72 +80,6 @@ trait KafkaMetricsGroup extends Logging {
def removeMetric(name: String, tags: scala.collection.Map[String, String] = Map.empty) =
Metrics.defaultRegistry().removeMetric(metricName(name, tags))
}
object KafkaMetricsGroup extends KafkaMetricsGroup with Logging {
/**
* To make sure all the metrics be de-registered after consumer/producer close, the metric names should be
* put into the metric name set.
*/
private val consumerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName](
// kafka.consumer.ZookeeperConsumerConnector
new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "FetchQueueSize"),
new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "KafkaCommitsPerSec"),
new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "ZooKeeperCommitsPerSec"),
new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "RebalanceRateAndTime"),
new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "OwnedPartitionsCount"),
// kafka.consumer.ConsumerFetcherManager
new MetricName("kafka.consumer", "ConsumerFetcherManager", "MaxLag"),
new MetricName("kafka.consumer", "ConsumerFetcherManager", "MinFetchRate"),
// kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread
new MetricName("kafka.server", "FetcherLagMetrics", "ConsumerLag"),
// kafka.consumer.ConsumerTopicStats <-- kafka.consumer.{ConsumerIterator, PartitionTopicInfo}
new MetricName("kafka.consumer", "ConsumerTopicMetrics", "MessagesPerSec"),
// kafka.consumer.ConsumerTopicStats
new MetricName("kafka.consumer", "ConsumerTopicMetrics", "BytesPerSec"),
// kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread
new MetricName("kafka.server", "FetcherStats", "BytesPerSec"),
new MetricName("kafka.server", "FetcherStats", "RequestsPerSec"),
// kafka.consumer.FetchRequestAndResponseStats <-- kafka.consumer.SimpleConsumer
new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchResponseSize"),
new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchRequestRateAndTimeMs"),
new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchRequestThrottleRateAndTimeMs"),
/**
* ProducerRequestStats <-- SyncProducer
* metric for SyncProducer in fetchTopicMetaData() needs to be removed when consumer is closed.
*/
new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"),
new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize")
)
private val producerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName](
// kafka.producer.ProducerStats <-- DefaultEventHandler <-- Producer
new MetricName("kafka.producer", "ProducerStats", "SerializationErrorsPerSec"),
new MetricName("kafka.producer", "ProducerStats", "ResendsPerSec"),
new MetricName("kafka.producer", "ProducerStats", "FailedSendsPerSec"),
// kafka.producer.ProducerSendThread
new MetricName("kafka.producer.async", "ProducerSendThread", "ProducerQueueSize"),
// kafka.producer.ProducerTopicStats <-- kafka.producer.{Producer, async.DefaultEventHandler}
new MetricName("kafka.producer", "ProducerTopicMetrics", "MessagesPerSec"),
new MetricName("kafka.producer", "ProducerTopicMetrics", "DroppedMessagesPerSec"),
new MetricName("kafka.producer", "ProducerTopicMetrics", "BytesPerSec"),
// kafka.producer.ProducerRequestStats <-- SyncProducer
new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"),
new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize"),
new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestThrottleRateAndTimeMs")
)
private def toMBeanName(tags: collection.Map[String, String]): Option[String] = {
val filteredTags = tags.filter { case (_, tagValue) => tagValue != "" }
if (filteredTags.nonEmpty) {
@ -174,33 +103,6 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging {
else None
}
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
def removeAllConsumerMetrics(clientId: String) {
FetchRequestAndResponseStatsRegistry.removeConsumerFetchRequestAndResponseStats(clientId)
ConsumerTopicStatsRegistry.removeConsumerTopicStat(clientId)
removeAllMetricsInList(KafkaMetricsGroup.consumerMetricNameList, clientId)
}
private def removeAllMetricsInList(metricNameList: immutable.List[MetricName], clientId: String) {
metricNameList.foreach(metric => {
val pattern = (".*clientId=" + clientId + ".*").r
val registeredMetrics = Metrics.defaultRegistry().allMetrics().keySet().asScala
for (registeredMetric <- registeredMetrics) {
if (registeredMetric.getGroup == metric.getGroup &&
registeredMetric.getName == metric.getName &&
registeredMetric.getType == metric.getType) {
pattern.findFirstIn(registeredMetric.getMBeanName) match {
case Some(_) => {
val beforeRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size
Metrics.defaultRegistry().removeMetric(registeredMetric)
val afterRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size
trace("Removing metric %s. Metrics registry size reduced from %d to %d".format(
registeredMetric, beforeRemovalSize, afterRemovalSize))
}
case _ =>
}
}
}
})
}
}
object KafkaMetricsGroup extends KafkaMetricsGroup

View File

@ -1,135 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.network
import java.net.InetSocketAddress
import java.nio.channels._
import kafka.api.RequestOrResponse
import kafka.utils.{CoreUtils, Logging, nonthreadsafe}
import org.apache.kafka.common.network.NetworkReceive
@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0")
object BlockingChannel{
val UseDefaultBufferSize = -1
}
/**
* A simple blocking channel with timeouts correctly enabled.
*
*/
@nonthreadsafe
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
class BlockingChannel( val host: String,
val port: Int,
val readBufferSize: Int,
val writeBufferSize: Int,
val readTimeoutMs: Int ) extends Logging {
private var connected = false
private var channel: SocketChannel = null
private var readChannel: ReadableByteChannel = null
private var writeChannel: GatheringByteChannel = null
private val lock = new Object()
private val connectTimeoutMs = readTimeoutMs
private var connectionId: String = ""
def connect() = lock synchronized {
if(!connected) {
try {
channel = SocketChannel.open()
if(readBufferSize > 0)
channel.socket.setReceiveBufferSize(readBufferSize)
if(writeBufferSize > 0)
channel.socket.setSendBufferSize(writeBufferSize)
channel.configureBlocking(true)
channel.socket.setSoTimeout(readTimeoutMs)
channel.socket.setKeepAlive(true)
channel.socket.setTcpNoDelay(true)
channel.socket.connect(new InetSocketAddress(host, port), connectTimeoutMs)
writeChannel = channel
// Need to create a new ReadableByteChannel from input stream because SocketChannel doesn't implement read with timeout
// See: http://stackoverflow.com/questions/2866557/timeout-for-socketchannel-doesnt-work
readChannel = Channels.newChannel(channel.socket().getInputStream)
connected = true
val localHost = channel.socket.getLocalAddress.getHostAddress
val localPort = channel.socket.getLocalPort
val remoteHost = channel.socket.getInetAddress.getHostAddress
val remotePort = channel.socket.getPort
connectionId = localHost + ":" + localPort + "-" + remoteHost + ":" + remotePort
// settings may not match what we requested above
val msg = "Created socket with SO_TIMEOUT = %d (requested %d), SO_RCVBUF = %d (requested %d), SO_SNDBUF = %d (requested %d), connectTimeoutMs = %d."
debug(msg.format(channel.socket.getSoTimeout,
readTimeoutMs,
channel.socket.getReceiveBufferSize,
readBufferSize,
channel.socket.getSendBufferSize,
writeBufferSize,
connectTimeoutMs))
} catch {
case _: Throwable => disconnect()
}
}
}
def disconnect() = lock synchronized {
if(channel != null) {
CoreUtils.swallow(channel.close(), this)
CoreUtils.swallow(channel.socket.close(), this)
channel = null
writeChannel = null
}
// closing the main socket channel *should* close the read channel
// but let's do it to be sure.
if(readChannel != null) {
CoreUtils.swallow(readChannel.close(), this)
readChannel = null
}
connected = false
}
def isConnected = connected
def send(request: RequestOrResponse): Long = {
if(!connected)
throw new ClosedChannelException()
val send = new RequestOrResponseSend(connectionId, request)
send.writeCompletely(writeChannel)
}
def receive(): NetworkReceive = {
if(!connected)
throw new ClosedChannelException()
val response = readCompletely(readChannel)
response.payload().rewind()
response
}
private def readCompletely(channel: ReadableByteChannel): NetworkReceive = {
val response = new NetworkReceive
while (!response.complete())
response.readFromReadableChannel(channel)
response
}
}

View File

@ -1,53 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.network
import java.nio.ByteBuffer
import java.nio.channels.GatheringByteChannel
import kafka.api.RequestOrResponse
import kafka.utils.Logging
import org.apache.kafka.common.network.NetworkSend
object RequestOrResponseSend {
def serialize(request: RequestOrResponse): ByteBuffer = {
val buffer = ByteBuffer.allocate(request.sizeInBytes + request.requestId.fold(0)(_ => 2))
request.requestId.foreach(buffer.putShort)
request.writeTo(buffer)
buffer.rewind()
buffer
}
}
class RequestOrResponseSend(val dest: String, val buffer: ByteBuffer) extends NetworkSend(dest, buffer) with Logging {
def this(dest: String, request: RequestOrResponse) {
this(dest, RequestOrResponseSend.serialize(request))
}
def writeCompletely(channel: GatheringByteChannel): Long = {
var totalWritten = 0L
while(!completed()) {
val written = writeTo(channel)
trace(written + " bytes written.")
totalWritten += written
}
totalWritten
}
}

View File

@ -20,7 +20,7 @@ package kafka.server
import java.util
import AbstractFetcherThread.ResultWithPartitions
import kafka.api.{FetchRequest => _, _}
import kafka.api._
import kafka.cluster.BrokerEndPoint
import kafka.log.LogConfig
import kafka.server.ReplicaFetcherThread._

Some files were not shown because too many files have changed in this diff Show More