mirror of https://github.com/apache/kafka.git
KAFKA-3264; Deprecate the old Scala consumer (KIP-109)
Author: Vahid Hashemian <vahidhashemian@us.ibm.com> Reviewers: Ismael Juma <ismael@juma.me.uk> This patch had conflicts when merged, resolved by Committer: Ismael Juma <ismael@juma.me.uk> Closes #2328 from vahidhashemian/KAFKA-3264
This commit is contained in:
parent
5d46348619
commit
f85c18032b
|
@ -352,6 +352,7 @@ 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) = {
|
def isConsumerGroupActive(zkUtils: ZkUtils, group: String) = {
|
||||||
zkUtils.getConsumersInGroup(group).nonEmpty
|
zkUtils.getConsumersInGroup(group).nonEmpty
|
||||||
}
|
}
|
||||||
|
@ -363,6 +364,7 @@ object AdminUtils extends Logging with AdminUtilities {
|
||||||
* @param group Consumer group
|
* @param group Consumer group
|
||||||
* @return whether or not we deleted the consumer group information
|
* @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) = {
|
def deleteConsumerGroupInZK(zkUtils: ZkUtils, group: String) = {
|
||||||
if (!isConsumerGroupActive(zkUtils, group)) {
|
if (!isConsumerGroupActive(zkUtils, group)) {
|
||||||
val dir = new ZKGroupDirs(group)
|
val dir = new ZKGroupDirs(group)
|
||||||
|
@ -381,6 +383,7 @@ object AdminUtils extends Logging with AdminUtilities {
|
||||||
* @param topic Topic of the consumer group information we wish to delete
|
* @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
|
* @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) = {
|
def deleteConsumerGroupInfoForTopicInZK(zkUtils: ZkUtils, group: String, topic: String) = {
|
||||||
val topics = zkUtils.getTopicsByConsumerGroup(group)
|
val topics = zkUtils.getTopicsByConsumerGroup(group)
|
||||||
if (topics == Seq(topic)) {
|
if (topics == Seq(topic)) {
|
||||||
|
@ -401,6 +404,7 @@ object AdminUtils extends Logging with AdminUtilities {
|
||||||
* @param zkUtils Zookeeper utilities
|
* @param zkUtils Zookeeper utilities
|
||||||
* @param topic Topic of the consumer group information we wish to delete
|
* @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) {
|
def deleteAllConsumerGroupInfoForTopicInZK(zkUtils: ZkUtils, topic: String) {
|
||||||
val groups = zkUtils.getAllConsumerGroupsForTopic(topic)
|
val groups = zkUtils.getAllConsumerGroupsForTopic(topic)
|
||||||
groups.foreach(group => deleteConsumerGroupInfoForTopicInZK(zkUtils, group, topic))
|
groups.foreach(group => deleteConsumerGroupInfoForTopicInZK(zkUtils, group, topic))
|
||||||
|
|
|
@ -231,6 +231,7 @@ object ConsumerGroupCommand extends Logging {
|
||||||
def exportOffsetsToReset(assignmentsToReset: Map[TopicPartition, OffsetAndMetadata]): String = throw new UnsupportedOperationException
|
def exportOffsetsToReset(assignmentsToReset: Map[TopicPartition, OffsetAndMetadata]): String = throw new UnsupportedOperationException
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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 {
|
class ZkConsumerGroupService(val opts: ConsumerGroupCommandOptions) extends ConsumerGroupService {
|
||||||
|
|
||||||
private val zkUtils = {
|
private val zkUtils = {
|
||||||
|
|
|
@ -37,6 +37,7 @@ import scala.util.Random
|
||||||
|
|
||||||
case class PartitionFetchInfo(offset: Long, fetchSize: Int)
|
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 {
|
object FetchRequest {
|
||||||
|
|
||||||
private val random = new Random
|
private val random = new Random
|
||||||
|
@ -90,6 +91,7 @@ object FetchRequest {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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,
|
case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion,
|
||||||
correlationId: Int = FetchRequest.DefaultCorrelationId,
|
correlationId: Int = FetchRequest.DefaultCorrelationId,
|
||||||
clientId: String = ConsumerConfig.DefaultClientId,
|
clientId: String = ConsumerConfig.DefaultClientId,
|
||||||
|
@ -227,6 +229,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
@nonthreadsafe
|
@nonthreadsafe
|
||||||
class FetchRequestBuilder() {
|
class FetchRequestBuilder() {
|
||||||
private val correlationId = new AtomicInteger(0)
|
private val correlationId = new AtomicInteger(0)
|
||||||
|
|
|
@ -34,6 +34,7 @@ import java.io.IOException
|
||||||
/**
|
/**
|
||||||
* Helper functions common to clients (producer, consumer, or admin)
|
* 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 {
|
object ClientUtils extends Logging {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -34,6 +34,8 @@ import org.apache.kafka.common.header.internals.RecordHeaders
|
||||||
* this class should be removed (along with BaseProducer)
|
* this class should be removed (along with BaseProducer)
|
||||||
* once we deprecate old consumer
|
* 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 {
|
trait BaseConsumer {
|
||||||
def receive(): BaseConsumerRecord
|
def receive(): BaseConsumerRecord
|
||||||
def stop()
|
def stop()
|
||||||
|
@ -41,6 +43,8 @@ trait BaseConsumer {
|
||||||
def commit()
|
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,
|
case class BaseConsumerRecord(topic: String,
|
||||||
partition: Int,
|
partition: Int,
|
||||||
offset: Long,
|
offset: Long,
|
||||||
|
@ -50,6 +54,8 @@ case class BaseConsumerRecord(topic: String,
|
||||||
value: Array[Byte],
|
value: Array[Byte],
|
||||||
headers: Headers = new RecordHeaders())
|
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], consumerProps: Properties, val timeoutMs: Long = Long.MaxValue) extends BaseConsumer {
|
class NewShinyConsumer(topic: Option[String], partitionId: Option[Int], offset: Option[Long], whitelist: Option[String], consumerProps: Properties, val timeoutMs: Long = Long.MaxValue) extends BaseConsumer {
|
||||||
import org.apache.kafka.clients.consumer.KafkaConsumer
|
import org.apache.kafka.clients.consumer.KafkaConsumer
|
||||||
|
|
||||||
|
@ -117,6 +123,8 @@ class NewShinyConsumer(topic: Option[String], partitionId: Option[Int], offset:
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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 {
|
class OldConsumer(topicFilter: TopicFilter, consumerProps: Properties) extends BaseConsumer {
|
||||||
import kafka.serializer.DefaultDecoder
|
import kafka.serializer.DefaultDecoder
|
||||||
|
|
||||||
|
|
|
@ -23,6 +23,8 @@ import kafka.utils._
|
||||||
import kafka.common.{InvalidConfigException, Config}
|
import kafka.common.{InvalidConfigException, Config}
|
||||||
import java.util.Locale
|
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 {
|
object ConsumerConfig extends Config {
|
||||||
val RefreshMetadataBackoffMs = 200
|
val RefreshMetadataBackoffMs = 200
|
||||||
val SocketTimeout = 30 * 1000
|
val SocketTimeout = 30 * 1000
|
||||||
|
@ -99,6 +101,8 @@ object ConsumerConfig extends Config {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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) {
|
class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig(props) {
|
||||||
import ConsumerConfig._
|
import ConsumerConfig._
|
||||||
|
|
||||||
|
|
|
@ -14,6 +14,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package kafka.consumer
|
package kafka.consumer
|
||||||
|
|
||||||
import kafka.common.{OffsetAndMetadata, TopicAndPartition}
|
import kafka.common.{OffsetAndMetadata, TopicAndPartition}
|
||||||
|
@ -26,6 +27,7 @@ import kafka.serializer._
|
||||||
/**
|
/**
|
||||||
* Main interface for consumer
|
* Main interface for consumer
|
||||||
*/
|
*/
|
||||||
|
@deprecated("This trait has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
trait ConsumerConnector {
|
trait ConsumerConnector {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -97,6 +99,8 @@ trait ConsumerConnector {
|
||||||
def shutdown()
|
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 {
|
object Consumer extends Logging {
|
||||||
/**
|
/**
|
||||||
* Create a ConsumerConnector
|
* Create a ConsumerConnector
|
||||||
|
|
|
@ -38,6 +38,7 @@ import java.util.concurrent.atomic.AtomicInteger
|
||||||
* Once ConsumerFetcherManager is created, startConnections() and stopAllConnections() can be called repeatedly
|
* Once ConsumerFetcherManager is created, startConnections() and stopAllConnections() can be called repeatedly
|
||||||
* until shutdown() is called.
|
* 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,
|
class ConsumerFetcherManager(private val consumerIdString: String,
|
||||||
private val config: ConsumerConfig,
|
private val config: ConsumerConfig,
|
||||||
private val zkUtils : ZkUtils)
|
private val zkUtils : ZkUtils)
|
||||||
|
|
|
@ -30,6 +30,8 @@ import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.record.MemoryRecords
|
import org.apache.kafka.common.record.MemoryRecords
|
||||||
import org.apache.kafka.common.requests.EpochEndOffset
|
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(name: String,
|
class ConsumerFetcherThread(name: String,
|
||||||
val config: ConsumerConfig,
|
val config: ConsumerConfig,
|
||||||
sourceBroker: BrokerEndPoint,
|
sourceBroker: BrokerEndPoint,
|
||||||
|
@ -120,6 +122,8 @@ class ConsumerFetcherThread(name: String,
|
||||||
override def maybeTruncate(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): Map[TopicPartition, Long] = { Map() }
|
override def maybeTruncate(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): Map[TopicPartition, Long] = { Map() }
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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 {
|
object ConsumerFetcherThread {
|
||||||
|
|
||||||
class FetchRequest(val underlying: kafka.api.FetchRequest) extends AbstractFetcherThread.FetchRequest {
|
class FetchRequest(val underlying: kafka.api.FetchRequest) extends AbstractFetcherThread.FetchRequest {
|
||||||
|
|
|
@ -30,6 +30,7 @@ import kafka.common.{KafkaException, MessageSizeTooLargeException}
|
||||||
* The iterator takes a shutdownCommand object which can be added to the queue to trigger a shutdown
|
* 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],
|
class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk],
|
||||||
consumerTimeoutMs: Int,
|
consumerTimeoutMs: Int,
|
||||||
private val keyDecoder: Decoder[K],
|
private val keyDecoder: Decoder[K],
|
||||||
|
@ -116,5 +117,7 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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()
|
class ConsumerTimeoutException() extends RuntimeException()
|
||||||
|
|
||||||
|
|
|
@ -23,6 +23,7 @@ import kafka.metrics.KafkaMetricsGroup
|
||||||
import kafka.common.{ClientIdTopic, ClientIdAllTopics, ClientIdAndTopic}
|
import kafka.common.{ClientIdTopic, ClientIdAllTopics, ClientIdAndTopic}
|
||||||
|
|
||||||
@threadsafe
|
@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 {
|
class ConsumerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup {
|
||||||
val tags = metricId match {
|
val tags = metricId match {
|
||||||
case ClientIdAndTopic(clientId, topic) => Map("clientId" -> clientId, "topic" -> topic)
|
case ClientIdAndTopic(clientId, topic) => Map("clientId" -> clientId, "topic" -> topic)
|
||||||
|
@ -37,6 +38,7 @@ class ConsumerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup {
|
||||||
* Tracks metrics for each topic the given consumer client has consumed data from.
|
* Tracks metrics for each topic the given consumer client has consumed data from.
|
||||||
* @param clientId The clientId of the given consumer client.
|
* @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 {
|
class ConsumerTopicStats(clientId: String) extends Logging {
|
||||||
private val valueFactory = (k: ClientIdAndTopic) => new ConsumerTopicMetrics(k)
|
private val valueFactory = (k: ClientIdAndTopic) => new ConsumerTopicMetrics(k)
|
||||||
private val stats = new Pool[ClientIdAndTopic, ConsumerTopicMetrics](Some(valueFactory))
|
private val stats = new Pool[ClientIdAndTopic, ConsumerTopicMetrics](Some(valueFactory))
|
||||||
|
@ -52,6 +54,7 @@ class ConsumerTopicStats(clientId: String) extends Logging {
|
||||||
/**
|
/**
|
||||||
* Stores the topic stats information of each consumer client in a (clientId -> ConsumerTopicStats) map.
|
* 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 {
|
object ConsumerTopicStatsRegistry {
|
||||||
private val valueFactory = (k: String) => new ConsumerTopicStats(k)
|
private val valueFactory = (k: String) => new ConsumerTopicStats(k)
|
||||||
private val globalStats = new Pool[String, ConsumerTopicStats](Some(valueFactory))
|
private val globalStats = new Pool[String, ConsumerTopicStats](Some(valueFactory))
|
||||||
|
|
|
@ -23,6 +23,7 @@ import kafka.common.{ClientIdAllBrokers, ClientIdBroker, ClientIdAndBroker}
|
||||||
import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
|
import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
|
||||||
import kafka.utils.Pool
|
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 {
|
class FetchRequestAndResponseMetrics(metricId: ClientIdBroker) extends KafkaMetricsGroup {
|
||||||
val tags = metricId match {
|
val tags = metricId match {
|
||||||
case ClientIdAndBroker(clientId, brokerHost, brokerPort) =>
|
case ClientIdAndBroker(clientId, brokerHost, brokerPort) =>
|
||||||
|
@ -41,6 +42,7 @@ class FetchRequestAndResponseMetrics(metricId: ClientIdBroker) extends KafkaMetr
|
||||||
* Tracks metrics of the requests made by a given consumer client to all brokers, and the responses obtained from the brokers.
|
* 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
|
* @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) {
|
class FetchRequestAndResponseStats(clientId: String) {
|
||||||
private val valueFactory = (k: ClientIdBroker) => new FetchRequestAndResponseMetrics(k)
|
private val valueFactory = (k: ClientIdBroker) => new FetchRequestAndResponseMetrics(k)
|
||||||
private val stats = new Pool[ClientIdBroker, FetchRequestAndResponseMetrics](Some(valueFactory))
|
private val stats = new Pool[ClientIdBroker, FetchRequestAndResponseMetrics](Some(valueFactory))
|
||||||
|
@ -56,6 +58,7 @@ class FetchRequestAndResponseStats(clientId: String) {
|
||||||
/**
|
/**
|
||||||
* Stores the fetch request and response stats information of each consumer client in a (clientId -> FetchRequestAndResponseStats) map.
|
* 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 {
|
object FetchRequestAndResponseStatsRegistry {
|
||||||
private val valueFactory = (k: String) => new FetchRequestAndResponseStats(k)
|
private val valueFactory = (k: String) => new FetchRequestAndResponseStats(k)
|
||||||
private val globalStats = new Pool[String, FetchRequestAndResponseStats](Some(valueFactory))
|
private val globalStats = new Pool[String, FetchRequestAndResponseStats](Some(valueFactory))
|
||||||
|
|
|
@ -19,6 +19,7 @@ package kafka.consumer
|
||||||
|
|
||||||
import kafka.message.ByteBufferMessageSet
|
import kafka.message.ByteBufferMessageSet
|
||||||
|
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
case class FetchedDataChunk(messages: ByteBufferMessageSet,
|
case class FetchedDataChunk(messages: ByteBufferMessageSet,
|
||||||
topicInfo: PartitionTopicInfo,
|
topicInfo: PartitionTopicInfo,
|
||||||
fetchOffset: Long)
|
fetchOffset: Long)
|
||||||
|
|
|
@ -22,6 +22,8 @@ import java.util.concurrent.BlockingQueue
|
||||||
import kafka.serializer.Decoder
|
import kafka.serializer.Decoder
|
||||||
import kafka.message.MessageAndMetadata
|
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],
|
class KafkaStream[K,V](private val queue: BlockingQueue[FetchedDataChunk],
|
||||||
consumerTimeoutMs: Int,
|
consumerTimeoutMs: Int,
|
||||||
private val keyDecoder: Decoder[K],
|
private val keyDecoder: Decoder[K],
|
||||||
|
|
|
@ -23,6 +23,8 @@ import kafka.utils.{Pool, CoreUtils, ZkUtils, Logging}
|
||||||
|
|
||||||
import scala.collection.mutable
|
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 {
|
trait PartitionAssignor {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -34,6 +36,8 @@ trait PartitionAssignor {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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 {
|
object PartitionAssignor {
|
||||||
def createInstance(assignmentStrategy: String) = assignmentStrategy match {
|
def createInstance(assignmentStrategy: String) = assignmentStrategy match {
|
||||||
case "roundrobin" => new RoundRobinAssignor()
|
case "roundrobin" => new RoundRobinAssignor()
|
||||||
|
@ -41,6 +45,7 @@ object PartitionAssignor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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) {
|
class AssignmentContext(group: String, val consumerId: String, excludeInternalTopics: Boolean, zkUtils: ZkUtils) {
|
||||||
val myTopicThreadIds: collection.Map[String, collection.Set[ConsumerThreadId]] = {
|
val myTopicThreadIds: collection.Map[String, collection.Set[ConsumerThreadId]] = {
|
||||||
val myTopicCount = TopicCount.constructTopicCount(group, consumerId, zkUtils, excludeInternalTopics)
|
val myTopicCount = TopicCount.constructTopicCount(group, consumerId, zkUtils, excludeInternalTopics)
|
||||||
|
@ -63,6 +68,8 @@ class AssignmentContext(group: String, val consumerId: String, excludeInternalTo
|
||||||
* instances are identical, then the partitions will be uniformly distributed. (i.e., the partition ownership counts
|
* 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.)
|
* 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 {
|
class RoundRobinAssignor() extends PartitionAssignor with Logging {
|
||||||
|
|
||||||
def assign(ctx: AssignmentContext) = {
|
def assign(ctx: AssignmentContext) = {
|
||||||
|
@ -117,6 +124,8 @@ class RoundRobinAssignor() extends PartitionAssignor with Logging {
|
||||||
* will get at least one partition and the first consumer thread will get one extra partition. So the assignment will be:
|
* 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
|
* 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 {
|
class RangeAssignor() extends PartitionAssignor with Logging {
|
||||||
|
|
||||||
def assign(ctx: AssignmentContext) = {
|
def assign(ctx: AssignmentContext) = {
|
||||||
|
|
|
@ -22,6 +22,7 @@ import java.util.concurrent.atomic._
|
||||||
import kafka.message._
|
import kafka.message._
|
||||||
import kafka.utils.Logging
|
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,
|
class PartitionTopicInfo(val topic: String,
|
||||||
val partitionId: Int,
|
val partitionId: Int,
|
||||||
private val chunkQueue: BlockingQueue[FetchedDataChunk],
|
private val chunkQueue: BlockingQueue[FetchedDataChunk],
|
||||||
|
@ -71,6 +72,7 @@ class PartitionTopicInfo(val topic: String,
|
||||||
": consumed offset = " + consumedOffset.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 {
|
object PartitionTopicInfo {
|
||||||
val InvalidOffset = -1L
|
val InvalidOffset = -1L
|
||||||
|
|
||||||
|
|
|
@ -32,6 +32,8 @@ import org.apache.kafka.common.utils.Utils._
|
||||||
/**
|
/**
|
||||||
* A consumer of kafka messages
|
* 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
|
@threadsafe
|
||||||
class SimpleConsumer(val host: String,
|
class SimpleConsumer(val host: String,
|
||||||
val port: Int,
|
val port: Int,
|
||||||
|
|
|
@ -21,6 +21,7 @@ import scala.collection._
|
||||||
import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, CoreUtils}
|
import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, CoreUtils}
|
||||||
import kafka.common.KafkaException
|
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 {
|
private[kafka] trait TopicCount {
|
||||||
|
|
||||||
def getConsumerThreadIdsPerTopic: Map[String, Set[ConsumerThreadId]]
|
def getConsumerThreadIdsPerTopic: Map[String, Set[ConsumerThreadId]]
|
||||||
|
@ -29,12 +30,14 @@ private[kafka] trait TopicCount {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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] {
|
case class ConsumerThreadId(consumer: String, threadId: Int) extends Ordered[ConsumerThreadId] {
|
||||||
override def toString = "%s-%d".format(consumer, threadId)
|
override def toString = "%s-%d".format(consumer, threadId)
|
||||||
|
|
||||||
def compare(that: ConsumerThreadId) = toString.compare(that.toString)
|
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 {
|
private[kafka] object TopicCount extends Logging {
|
||||||
val whiteListPattern = "white_list"
|
val whiteListPattern = "white_list"
|
||||||
val blackListPattern = "black_list"
|
val blackListPattern = "black_list"
|
||||||
|
@ -105,6 +108,7 @@ private[kafka] object TopicCount extends Logging {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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,
|
private[kafka] class StaticTopicCount(val consumerIdString: String,
|
||||||
val topicCountMap: Map[String, Int])
|
val topicCountMap: Map[String, Int])
|
||||||
extends TopicCount {
|
extends TopicCount {
|
||||||
|
@ -116,6 +120,7 @@ private[kafka] class StaticTopicCount(val consumerIdString: String,
|
||||||
def pattern = TopicCount.staticPattern
|
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,
|
private[kafka] class WildcardTopicCount(zkUtils: ZkUtils,
|
||||||
consumerIdString: String,
|
consumerIdString: String,
|
||||||
topicFilter: TopicFilter,
|
topicFilter: TopicFilter,
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
|
|
||||||
package kafka.consumer
|
package kafka.consumer
|
||||||
|
|
||||||
|
@deprecated("This trait has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
trait TopicEventHandler[T] {
|
trait TopicEventHandler[T] {
|
||||||
|
|
||||||
def handleTopicEvent(allTopics: Seq[T])
|
def handleTopicEvent(allTopics: Seq[T])
|
||||||
|
|
|
@ -22,6 +22,7 @@ import java.util.regex.{Pattern, PatternSyntaxException}
|
||||||
|
|
||||||
import org.apache.kafka.common.internals.Topic
|
import org.apache.kafka.common.internals.Topic
|
||||||
|
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
sealed abstract class TopicFilter(rawRegex: String) extends Logging {
|
sealed abstract class TopicFilter(rawRegex: String) extends Logging {
|
||||||
|
|
||||||
val regex = rawRegex
|
val regex = rawRegex
|
||||||
|
@ -44,6 +45,7 @@ sealed abstract class TopicFilter(rawRegex: String) extends Logging {
|
||||||
def isTopicAllowed(topic: String, excludeInternalTopics: Boolean): Boolean
|
def isTopicAllowed(topic: String, excludeInternalTopics: Boolean): Boolean
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) {
|
case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) {
|
||||||
override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = {
|
override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = {
|
||||||
val allowed = topic.matches(regex) && !(Topic.isInternal(topic) && excludeInternalTopics)
|
val allowed = topic.matches(regex) && !(Topic.isInternal(topic) && excludeInternalTopics)
|
||||||
|
@ -53,10 +55,9 @@ case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) {
|
||||||
|
|
||||||
allowed
|
allowed
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) {
|
case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) {
|
||||||
override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = {
|
override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = {
|
||||||
val allowed = (!topic.matches(regex)) && !(Topic.isInternal(topic) && excludeInternalTopics)
|
val allowed = (!topic.matches(regex)) && !(Topic.isInternal(topic) && excludeInternalTopics)
|
||||||
|
|
|
@ -80,10 +80,12 @@ import scala.collection.JavaConverters._
|
||||||
* Each consumer tracks the offset of the latest message consumed for each partition.
|
* Each consumer tracks the offset of the latest message consumed for each partition.
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
|
@deprecated("This object has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
private[kafka] object ZookeeperConsumerConnector {
|
private[kafka] object ZookeeperConsumerConnector {
|
||||||
val shutdownCommand: FetchedDataChunk = new FetchedDataChunk(null, null, -1L)
|
val shutdownCommand: FetchedDataChunk = new FetchedDataChunk(null, null, -1L)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@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,
|
private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
||||||
val enableFetcher: Boolean) // for testing only
|
val enableFetcher: Boolean) // for testing only
|
||||||
extends ConsumerConnector with Logging with KafkaMetricsGroup {
|
extends ConsumerConnector with Logging with KafkaMetricsGroup {
|
||||||
|
|
|
@ -22,6 +22,7 @@ import kafka.utils.{ZkUtils, Logging}
|
||||||
import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener}
|
import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener}
|
||||||
import org.apache.zookeeper.Watcher.Event.KeeperState
|
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,
|
class ZookeeperTopicEventWatcher(val zkUtils: ZkUtils,
|
||||||
val eventHandler: TopicEventHandler[String]) extends Logging {
|
val eventHandler: TopicEventHandler[String]) extends Logging {
|
||||||
|
|
||||||
|
|
|
@ -25,6 +25,10 @@ import kafka.serializer.Decoder;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @deprecated since 0.11.0.0, this interface will be removed in a future release.
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
public interface ConsumerConnector {
|
public interface ConsumerConnector {
|
||||||
/**
|
/**
|
||||||
* Create a list of MessageStreams of type T for each topic.
|
* Create a list of MessageStreams of type T for each topic.
|
||||||
|
|
|
@ -25,6 +25,10 @@ import java.util.Set;
|
||||||
* This listener is used for execution of tasks defined by user when a consumer rebalance
|
* This listener is used for execution of tasks defined by user when a consumer rebalance
|
||||||
* occurs in {@link kafka.consumer.ZookeeperConsumerConnector}
|
* 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 {
|
public interface ConsumerRebalanceListener {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -24,6 +24,8 @@ import kafka.javaapi.OffsetRequest
|
||||||
/**
|
/**
|
||||||
* A consumer of kafka messages
|
* 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
|
@threadsafe
|
||||||
class SimpleConsumer(val host: String,
|
class SimpleConsumer(val host: String,
|
||||||
val port: Int,
|
val port: Int,
|
||||||
|
|
|
@ -59,6 +59,7 @@ import scala.collection.JavaConverters._
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
@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,
|
private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
|
||||||
val enableFetcher: Boolean) // for testing only
|
val enableFetcher: Boolean) // for testing only
|
||||||
extends ConsumerConnector {
|
extends ConsumerConnector {
|
||||||
|
|
|
@ -181,6 +181,7 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging {
|
||||||
else None
|
else None
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def removeAllConsumerMetrics(clientId: String) {
|
def removeAllConsumerMetrics(clientId: String) {
|
||||||
FetchRequestAndResponseStatsRegistry.removeConsumerFetchRequestAndResponseStats(clientId)
|
FetchRequestAndResponseStatsRegistry.removeConsumerFetchRequestAndResponseStats(clientId)
|
||||||
ConsumerTopicStatsRegistry.removeConsumerTopicStat(clientId)
|
ConsumerTopicStatsRegistry.removeConsumerTopicStat(clientId)
|
||||||
|
|
|
@ -25,6 +25,7 @@ import kafka.utils.{Logging, nonthreadsafe}
|
||||||
import org.apache.kafka.common.network.NetworkReceive
|
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{
|
object BlockingChannel{
|
||||||
val UseDefaultBufferSize = -1
|
val UseDefaultBufferSize = -1
|
||||||
}
|
}
|
||||||
|
@ -34,6 +35,7 @@ object BlockingChannel{
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
@nonthreadsafe
|
@nonthreadsafe
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
class BlockingChannel( val host: String,
|
class BlockingChannel( val host: String,
|
||||||
val port: Int,
|
val port: Int,
|
||||||
val readBufferSize: Int,
|
val readBufferSize: Int,
|
||||||
|
|
|
@ -241,7 +241,7 @@ abstract class AbstractFetcherThread(name: String,
|
||||||
!partitionStates.contains(tp)
|
!partitionStates.contains(tp)
|
||||||
}.map { case (tp, offset) =>
|
}.map { case (tp, offset) =>
|
||||||
val fetchState =
|
val fetchState =
|
||||||
if (PartitionTopicInfo.isOffsetInvalid(offset))
|
if (offset < 0)
|
||||||
new PartitionFetchState(handleOffsetOutOfRange(tp), includeLogTruncation)
|
new PartitionFetchState(handleOffsetOutOfRange(tp), includeLogTruncation)
|
||||||
else
|
else
|
||||||
new PartitionFetchState(offset, includeLogTruncation)
|
new PartitionFetchState(offset, includeLogTruncation)
|
||||||
|
|
|
@ -34,6 +34,7 @@ import kafka.api.PartitionOffsetRequestInfo
|
||||||
import org.I0Itec.zkclient.exception.ZkNoNodeException
|
import org.I0Itec.zkclient.exception.ZkNoNodeException
|
||||||
import org.apache.kafka.common.network.ListenerName
|
import org.apache.kafka.common.network.ListenerName
|
||||||
|
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
object ConsumerOffsetChecker extends Logging {
|
object ConsumerOffsetChecker extends Logging {
|
||||||
|
|
||||||
private val consumerMap: mutable.Map[Int, Option[SimpleConsumer]] = mutable.Map()
|
private val consumerMap: mutable.Map[Int, Option[SimpleConsumer]] = mutable.Map()
|
||||||
|
|
|
@ -43,10 +43,12 @@ import scala.collection.JavaConverters._
|
||||||
* log4j.logger.kafka.tools.ExportZkOffsets$=DEBUG
|
* log4j.logger.kafka.tools.ExportZkOffsets$=DEBUG
|
||||||
* (for eclipse debugging, copy log4j.properties to the binary directory in "core" such as core/bin)
|
* (for eclipse debugging, copy log4j.properties to the binary directory in "core" such as core/bin)
|
||||||
*/
|
*/
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
object ExportZkOffsets extends Logging {
|
object ExportZkOffsets extends Logging {
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val parser = new OptionParser(false)
|
val parser = new OptionParser(false)
|
||||||
|
warn("WARNING: ExportZkOffsets is deprecated and will be dropped in a future release following 0.11.0.0.")
|
||||||
|
|
||||||
val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.")
|
val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.")
|
||||||
.withRequiredArg()
|
.withRequiredArg()
|
||||||
|
|
|
@ -31,11 +31,14 @@ import org.apache.kafka.common.utils.Time
|
||||||
/**
|
/**
|
||||||
* Performance test for the simple consumer
|
* Performance test for the simple consumer
|
||||||
*/
|
*/
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
object SimpleConsumerPerformance {
|
object SimpleConsumerPerformance {
|
||||||
|
|
||||||
private val logger = Logger.getLogger(getClass())
|
private val logger = Logger.getLogger(getClass())
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
|
logger.warn("WARNING: SimpleConsumerPerformance is deprecated and will be dropped in a future release following 0.11.0.0.")
|
||||||
|
|
||||||
val config = new ConsumerPerfConfig(args)
|
val config = new ConsumerPerfConfig(args)
|
||||||
logger.info("Starting SimpleConsumer...")
|
logger.info("Starting SimpleConsumer...")
|
||||||
|
|
||||||
|
|
|
@ -32,11 +32,13 @@ import org.apache.kafka.common.utils.Utils
|
||||||
/**
|
/**
|
||||||
* Command line program to dump out messages to standard out using the simple consumer
|
* Command line program to dump out messages to standard out using the simple consumer
|
||||||
*/
|
*/
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
object SimpleConsumerShell extends Logging {
|
object SimpleConsumerShell extends Logging {
|
||||||
|
|
||||||
def UseLeaderReplica = -1
|
def UseLeaderReplica = -1
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
|
warn("WARNING: SimpleConsumerShell is deprecated and will be dropped in a future release following 0.11.0.0.")
|
||||||
|
|
||||||
val parser = new OptionParser(false)
|
val parser = new OptionParser(false)
|
||||||
val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
|
val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
|
||||||
|
|
|
@ -17,11 +17,10 @@
|
||||||
|
|
||||||
package kafka.tools
|
package kafka.tools
|
||||||
|
|
||||||
import org.I0Itec.zkclient.ZkClient
|
|
||||||
import kafka.consumer.{ConsumerConfig, SimpleConsumer}
|
import kafka.consumer.{ConsumerConfig, SimpleConsumer}
|
||||||
import kafka.api.{OffsetRequest, PartitionOffsetRequestInfo}
|
import kafka.api.{OffsetRequest, PartitionOffsetRequestInfo}
|
||||||
import kafka.common.{KafkaException, TopicAndPartition}
|
import kafka.common.{KafkaException, TopicAndPartition}
|
||||||
import kafka.utils.{CoreUtils, Exit, ZKGroupTopicDirs, ZkUtils}
|
import kafka.utils.{Exit, Logging, ZKGroupTopicDirs, ZkUtils}
|
||||||
import org.apache.kafka.common.network.ListenerName
|
import org.apache.kafka.common.network.ListenerName
|
||||||
import org.apache.kafka.common.protocol.SecurityProtocol
|
import org.apache.kafka.common.protocol.SecurityProtocol
|
||||||
import org.apache.kafka.common.security.JaasUtils
|
import org.apache.kafka.common.security.JaasUtils
|
||||||
|
@ -30,11 +29,14 @@ import org.apache.kafka.common.utils.Utils
|
||||||
/**
|
/**
|
||||||
* A utility that updates the offset of every broker partition to the offset of earliest or latest log segment file, in ZK.
|
* A utility that updates the offset of every broker partition to the offset of earliest or latest log segment file, in ZK.
|
||||||
*/
|
*/
|
||||||
object UpdateOffsetsInZK {
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
|
object UpdateOffsetsInZK extends Logging {
|
||||||
val Earliest = "earliest"
|
val Earliest = "earliest"
|
||||||
val Latest = "latest"
|
val Latest = "latest"
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
|
warn("WARNING: UpdateOffsetsInZK is deprecated and will be dropped in releases following 0.11.0.0.")
|
||||||
|
|
||||||
if(args.length < 3)
|
if(args.length < 3)
|
||||||
usage
|
usage
|
||||||
val config = new ConsumerConfig(Utils.loadProps(args(1)))
|
val config = new ConsumerConfig(Utils.loadProps(args(1)))
|
||||||
|
|
|
@ -21,9 +21,11 @@ import joptsimple.OptionParser
|
||||||
import org.apache.kafka.common.security._
|
import org.apache.kafka.common.security._
|
||||||
import kafka.utils.{CommandLineUtils, Exit, Logging, ZKGroupTopicDirs, ZkUtils}
|
import kafka.utils.{CommandLineUtils, Exit, Logging, ZKGroupTopicDirs, ZkUtils}
|
||||||
|
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
object VerifyConsumerRebalance extends Logging {
|
object VerifyConsumerRebalance extends Logging {
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val parser = new OptionParser(false)
|
val parser = new OptionParser(false)
|
||||||
|
warn("WARNING: VerifyConsumerRebalance is deprecated and will be dropped in a future release following 0.11.0.0.")
|
||||||
|
|
||||||
val zkConnectOpt = parser.accepts("zookeeper.connect", "ZooKeeper connect string.").
|
val zkConnectOpt = parser.accepts("zookeeper.connect", "ZooKeeper connect string.").
|
||||||
withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String])
|
withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String])
|
||||||
|
|
|
@ -421,6 +421,7 @@ class ZkUtils(val zkClient: ZkClient,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def getConsumerPartitionOwnerPath(group: String, topic: String, partition: Int): String = {
|
def getConsumerPartitionOwnerPath(group: String, topic: String, partition: Int): String = {
|
||||||
val topicDirs = new ZKGroupTopicDirs(group, topic)
|
val topicDirs = new ZKGroupTopicDirs(group, topic)
|
||||||
topicDirs.consumerOwnerDir + "/" + partition
|
topicDirs.consumerOwnerDir + "/" + partition
|
||||||
|
@ -823,11 +824,13 @@ class ZkUtils(val zkClient: ZkClient,
|
||||||
zkClient.delete(brokerPartTopicPath)
|
zkClient.delete(brokerPartTopicPath)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def getConsumersInGroup(group: String): Seq[String] = {
|
def getConsumersInGroup(group: String): Seq[String] = {
|
||||||
val dirs = new ZKGroupDirs(group)
|
val dirs = new ZKGroupDirs(group)
|
||||||
getChildren(dirs.consumerRegistryDir)
|
getChildren(dirs.consumerRegistryDir)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def getConsumersPerTopic(group: String, excludeInternalTopics: Boolean): mutable.Map[String, List[ConsumerThreadId]] = {
|
def getConsumersPerTopic(group: String, excludeInternalTopics: Boolean): mutable.Map[String, List[ConsumerThreadId]] = {
|
||||||
val dirs = new ZKGroupDirs(group)
|
val dirs = new ZKGroupDirs(group)
|
||||||
val consumers = getChildrenParentMayNotExist(dirs.consumerRegistryDir)
|
val consumers = getChildrenParentMayNotExist(dirs.consumerRegistryDir)
|
||||||
|
@ -847,6 +850,7 @@ class ZkUtils(val zkClient: ZkClient,
|
||||||
consumersPerTopicMap
|
consumersPerTopicMap
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def getTopicsPerMemberId(group: String, excludeInternalTopics: Boolean = true): Map[String, List[String]] = {
|
def getTopicsPerMemberId(group: String, excludeInternalTopics: Boolean = true): Map[String, List[String]] = {
|
||||||
val dirs = new ZKGroupDirs(group)
|
val dirs = new ZKGroupDirs(group)
|
||||||
val memberIds = getChildrenParentMayNotExist(dirs.consumerRegistryDir)
|
val memberIds = getChildrenParentMayNotExist(dirs.consumerRegistryDir)
|
||||||
|
@ -916,14 +920,17 @@ class ZkUtils(val zkClient: ZkClient,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def getConsumerGroups() = {
|
def getConsumerGroups() = {
|
||||||
getChildren(ConsumersPath)
|
getChildren(ConsumersPath)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def getTopicsByConsumerGroup(consumerGroup:String) = {
|
def getTopicsByConsumerGroup(consumerGroup:String) = {
|
||||||
getChildrenParentMayNotExist(new ZKGroupDirs(consumerGroup).consumerGroupOwnersDir)
|
getChildrenParentMayNotExist(new ZKGroupDirs(consumerGroup).consumerGroupOwnersDir)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def getAllConsumerGroupsForTopic(topic: String): Set[String] = {
|
def getAllConsumerGroupsForTopic(topic: String): Set[String] = {
|
||||||
val groups = getChildrenParentMayNotExist(ConsumersPath)
|
val groups = getChildrenParentMayNotExist(ConsumersPath)
|
||||||
if (groups == null) Set.empty
|
if (groups == null) Set.empty
|
||||||
|
@ -957,6 +964,7 @@ private object ZKStringSerializer extends ZkSerializer {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
class ZKGroupDirs(val group: String) {
|
class ZKGroupDirs(val group: String) {
|
||||||
def consumerDir = ConsumersPath
|
def consumerDir = ConsumersPath
|
||||||
def consumerGroupDir = consumerDir + "/" + group
|
def consumerGroupDir = consumerDir + "/" + group
|
||||||
|
@ -965,6 +973,7 @@ class ZKGroupDirs(val group: String) {
|
||||||
def consumerGroupOwnersDir = consumerGroupDir + "/owners"
|
def consumerGroupOwnersDir = consumerGroupDir + "/owners"
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This class has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
class ZKGroupTopicDirs(group: String, topic: String) extends ZKGroupDirs(group) {
|
class ZKGroupTopicDirs(group: String, topic: String) extends ZKGroupDirs(group) {
|
||||||
def consumerOffsetDir = consumerGroupOffsetsDir + "/" + topic
|
def consumerOffsetDir = consumerGroupOffsetsDir + "/" + topic
|
||||||
def consumerOwnerDir = consumerGroupOwnersDir + "/" + topic
|
def consumerOwnerDir = consumerGroupOwnersDir + "/" + topic
|
||||||
|
|
|
@ -279,6 +279,7 @@ object TestLogCleaning {
|
||||||
producedFile
|
producedFile
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def makeConsumer(zkUrl: String, topics: Array[String]): ZookeeperConsumerConnector = {
|
def makeConsumer(zkUrl: String, topics: Array[String]): ZookeeperConsumerConnector = {
|
||||||
val consumerProps = new Properties
|
val consumerProps = new Properties
|
||||||
consumerProps.setProperty("group.id", "log-cleaner-test-" + new Random().nextInt(Int.MaxValue))
|
consumerProps.setProperty("group.id", "log-cleaner-test-" + new Random().nextInt(Int.MaxValue))
|
||||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
|
||||||
import kafka.integration.KafkaServerTestHarness
|
import kafka.integration.KafkaServerTestHarness
|
||||||
|
|
||||||
|
|
||||||
|
@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
class DeleteConsumerGroupTest extends KafkaServerTestHarness {
|
class DeleteConsumerGroupTest extends KafkaServerTestHarness {
|
||||||
def generateConfigs() = TestUtils.createBrokerConfigs(3, zkConnect, false, true).map(KafkaConfig.fromProps)
|
def generateConfigs() = TestUtils.createBrokerConfigs(3, zkConnect, false, true).map(KafkaConfig.fromProps)
|
||||||
|
|
||||||
|
|
|
@ -61,6 +61,7 @@ class DescribeConsumerGroupTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def testDescribeNonExistingGroup() {
|
def testDescribeNonExistingGroup() {
|
||||||
// mocks
|
// mocks
|
||||||
props.setProperty("zookeeper.connect", zkConnect)
|
props.setProperty("zookeeper.connect", zkConnect)
|
||||||
|
@ -82,6 +83,7 @@ class DescribeConsumerGroupTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def testDescribeExistingGroup() {
|
def testDescribeExistingGroup() {
|
||||||
// mocks
|
// mocks
|
||||||
props.setProperty("zookeeper.connect", zkConnect)
|
props.setProperty("zookeeper.connect", zkConnect)
|
||||||
|
@ -108,6 +110,7 @@ class DescribeConsumerGroupTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def testDescribeExistingGroupWithNoMembers() {
|
def testDescribeExistingGroupWithNoMembers() {
|
||||||
// mocks
|
// mocks
|
||||||
props.setProperty("zookeeper.connect", zkConnect)
|
props.setProperty("zookeeper.connect", zkConnect)
|
||||||
|
@ -141,6 +144,7 @@ class DescribeConsumerGroupTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def testDescribeConsumersWithNoAssignedPartitions() {
|
def testDescribeConsumersWithNoAssignedPartitions() {
|
||||||
// mocks
|
// mocks
|
||||||
props.setProperty("zookeeper.connect", zkConnect)
|
props.setProperty("zookeeper.connect", zkConnect)
|
||||||
|
|
|
@ -33,6 +33,7 @@ import org.junit.{Before, Test}
|
||||||
import kafka.serializer._
|
import kafka.serializer._
|
||||||
import kafka.integration.KafkaServerTestHarness
|
import kafka.integration.KafkaServerTestHarness
|
||||||
|
|
||||||
|
@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
class ConsumerIteratorTest extends KafkaServerTestHarness {
|
class ConsumerIteratorTest extends KafkaServerTestHarness {
|
||||||
|
|
||||||
val numNodes = 1
|
val numNodes = 1
|
||||||
|
|
|
@ -28,6 +28,7 @@ import kafka.consumer.PartitionAssignorTest.Scenario
|
||||||
import kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo
|
import kafka.consumer.PartitionAssignorTest.WildcardSubscriptionInfo
|
||||||
import org.junit.Test
|
import org.junit.Test
|
||||||
|
|
||||||
|
@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
class PartitionAssignorTest extends Logging {
|
class PartitionAssignorTest extends Logging {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.junit.Assert._
|
||||||
import org.scalatest.junit.JUnitSuite
|
import org.scalatest.junit.JUnitSuite
|
||||||
import org.junit.Test
|
import org.junit.Test
|
||||||
|
|
||||||
|
@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
class TopicFilterTest extends JUnitSuite {
|
class TopicFilterTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -29,6 +29,7 @@ import kafka.server._
|
||||||
import kafka.consumer._
|
import kafka.consumer._
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
|
|
||||||
|
@deprecated("This test has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
class FetcherTest extends KafkaServerTestHarness {
|
class FetcherTest extends KafkaServerTestHarness {
|
||||||
val numNodes = 1
|
val numNodes = 1
|
||||||
def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps)
|
def generateConfigs() = TestUtils.createBrokerConfigs(numNodes, zkConnect).map(KafkaConfig.fromProps)
|
||||||
|
|
|
@ -653,6 +653,8 @@ object TestUtils extends Logging {
|
||||||
props
|
props
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = {
|
def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = {
|
||||||
val zkUtils = ZkUtils(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
|
val zkUtils = ZkUtils(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
|
||||||
zkUtils.updatePersistentPath(path, offset.toString)
|
zkUtils.updatePersistentPath(path, offset.toString)
|
||||||
|
@ -1082,6 +1084,7 @@ object TestUtils extends Logging {
|
||||||
* If not specified, then all available messages will be consumed, and no exception is thrown.
|
* If not specified, then all available messages will be consumed, and no exception is thrown.
|
||||||
* @return the list of messages consumed.
|
* @return the list of messages consumed.
|
||||||
*/
|
*/
|
||||||
|
@deprecated("This method has been deprecated and will be removed in a future release.", "0.11.0.0")
|
||||||
def getMessages(topicMessageStreams: Map[String, List[KafkaStream[String, String]]],
|
def getMessages(topicMessageStreams: Map[String, List[KafkaStream[String, String]]],
|
||||||
nMessagesPerThread: Int = -1): List[String] = {
|
nMessagesPerThread: Int = -1): List[String] = {
|
||||||
|
|
||||||
|
|
|
@ -30,6 +30,10 @@ import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @deprecated since 0.11.0.0. This class will be removed in a future release.
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
public class SimpleConsumerDemo {
|
public class SimpleConsumerDemo {
|
||||||
|
|
||||||
private static void printMessages(ByteBufferMessageSet messageSet) throws UnsupportedEncodingException {
|
private static void printMessages(ByteBufferMessageSet messageSet) throws UnsupportedEncodingException {
|
||||||
|
|
Loading…
Reference in New Issue