mirror of https://github.com/apache/kafka.git
KAFKA-9324: Drop support for Scala 2.11 (KIP-531) (#7859)
* Adjust build and documentation. * Use lambda syntax for SAM types in `core`, `streams-scala` and `connect-runtime` modules. * Remove `runnable` and `newThread` from `CoreUtils` as lambda syntax for SAM types make them unnecessary. * Remove stale comment in `FunctionsCompatConversions`, `KGroupedStream`, `KGroupedTable' and `KStream` about Scala 2.11, the conversions are needed for Scala 2.12 too. * Deprecate `org.apache.kafka.streams.scala.kstream.Suppressed` and use `org.apache.kafka.streams.kstream.Suppressed` instead. * Use `Admin.create` instead of `AdminClient.create`. Static methods in Java interfaces can be invoked since Scala 2.12. I noticed that MirrorMaker 2 uses `AdminClient.create`, but I did not change them as Connectors have restrictions on newer client APIs. * Improve efficiency in a few `Gauge` implementations by avoiding unnecessary intermediate collections. * Remove pointless `Option.apply` in `ZookeeperClient` `SessionState` metric. * Fix unused import/variable and other compiler warnings. * Reduce visibility of some vals/defs. Reviewers: Manikumar Reddy <manikumar@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Gwen Shapira <gwen@confluent.io>
This commit is contained in:
parent
42b0971e94
commit
6dc6f6a60d
|
|
@ -69,7 +69,7 @@ The release file can be found inside `./core/build/distributions/`.
|
|||
### Cleaning the build ###
|
||||
./gradlew clean
|
||||
|
||||
### Running a task with one of the Scala versions available (2.11.x, 2.12.x or 2.13.x) ###
|
||||
### Running a task with one of the Scala versions available (2.12.x or 2.13.x) ###
|
||||
*Note that if building the jars with a version other than 2.12.x, you need to set the `SCALA_VERSION` variable or change it in `bin/kafka-run-class.sh` to run the quick start.*
|
||||
|
||||
You can pass either the major version (eg 2.12) or the full version (eg 2.12.7):
|
||||
|
|
|
|||
45
build.gradle
45
build.gradle
|
|
@ -411,6 +411,7 @@ subprojects {
|
|||
"-language:postfixOps",
|
||||
"-language:implicitConversions",
|
||||
"-language:existentials",
|
||||
"-Xlint:constant",
|
||||
"-Xlint:delayedinit-select",
|
||||
"-Xlint:doc-detached",
|
||||
"-Xlint:missing-interpolator",
|
||||
|
|
@ -421,35 +422,29 @@ subprojects {
|
|||
"-Xlint:poly-implicit-overload",
|
||||
"-Xlint:private-shadow",
|
||||
"-Xlint:stars-align",
|
||||
"-Xlint:type-parameter-shadow"
|
||||
"-Xlint:type-parameter-shadow",
|
||||
"-Xlint:unused"
|
||||
]
|
||||
|
||||
if (versions.baseScala != '2.11') {
|
||||
scalaCompileOptions.additionalParameters += [
|
||||
"-Xlint:constant",
|
||||
"-Xlint:unused"
|
||||
]
|
||||
// Inline more aggressively when compiling the `core` jar since it's not meant to be used as a library.
|
||||
// More specifically, inline classes from the Scala library so that we can inline methods like `Option.exists`
|
||||
// and avoid lambda allocations. This is only safe if the Scala library version is the same at compile time
|
||||
// and runtime. We cannot guarantee this for libraries like kafka streams, so only inline classes from the
|
||||
// Kafka project in that case.
|
||||
List<String> inlineFrom
|
||||
if (project.name.equals('core'))
|
||||
inlineFrom = ["-opt-inline-from:scala.**", "-opt-inline-from:kafka.**", "-opt-inline-from:org.apache.kafka.**"]
|
||||
else
|
||||
inlineFrom = ["-opt-inline-from:org.apache.kafka.**"]
|
||||
|
||||
// Inline more aggressively when compiling the `core` jar since it's not meant to be used as a library.
|
||||
// More specifically, inline classes from the Scala library so that we can inline methods like `Option.exists`
|
||||
// and avoid lambda allocations. This is only safe if the Scala library version is the same at compile time
|
||||
// and runtime. We cannot guarantee this for libraries like kafka streams, so only inline classes from the
|
||||
// Kafka project in that case.
|
||||
List<String> inlineFrom
|
||||
if (project.name.equals('core'))
|
||||
inlineFrom = ["-opt-inline-from:scala.**", "-opt-inline-from:kafka.**", "-opt-inline-from:org.apache.kafka.**"]
|
||||
else
|
||||
inlineFrom = ["-opt-inline-from:org.apache.kafka.**"]
|
||||
|
||||
// Somewhat confusingly, `-opt:l:inline` enables all optimizations. `inlineFrom` configures what can be inlined.
|
||||
// See https://www.lightbend.com/blog/scala-inliner-optimizer for more information about the optimizer.
|
||||
scalaCompileOptions.additionalParameters += ["-opt:l:inline"]
|
||||
scalaCompileOptions.additionalParameters += inlineFrom
|
||||
}
|
||||
// Somewhat confusingly, `-opt:l:inline` enables all optimizations. `inlineFrom` configures what can be inlined.
|
||||
// See https://www.lightbend.com/blog/scala-inliner-optimizer for more information about the optimizer.
|
||||
scalaCompileOptions.additionalParameters += ["-opt:l:inline"]
|
||||
scalaCompileOptions.additionalParameters += inlineFrom
|
||||
|
||||
// these options are valid for Scala versions < 2.13 only
|
||||
// Scala 2.13 removes them, see https://github.com/scala/scala/pull/6502 and https://github.com/scala/scala/pull/5969
|
||||
if (versions.baseScala in ['2.11','2.12']) {
|
||||
// these options are valid for Scala versions < 2.13 only
|
||||
// Scala 2.13 removes them, see https://github.com/scala/scala/pull/6502 and https://github.com/scala/scala/pull/5969
|
||||
if (versions.baseScala == '2.12') {
|
||||
scalaCompileOptions.additionalParameters += [
|
||||
"-Xlint:by-name-right-associative",
|
||||
"-Xlint:unsound-match"
|
||||
|
|
|
|||
|
|
@ -46,11 +46,7 @@ public class KafkaThread extends Thread {
|
|||
|
||||
private void configureThread(final String name, boolean daemon) {
|
||||
setDaemon(daemon);
|
||||
setUncaughtExceptionHandler(new UncaughtExceptionHandler() {
|
||||
public void uncaughtException(Thread t, Throwable e) {
|
||||
log.error("Uncaught exception in thread '{}':", name, e);
|
||||
}
|
||||
});
|
||||
setUncaughtExceptionHandler((t, e) -> log.error("Uncaught exception in thread '{}':", name, e));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -305,12 +305,7 @@ public class ConnectMetrics {
|
|||
public <T> void addValueMetric(MetricNameTemplate nameTemplate, final LiteralSupplier<T> supplier) {
|
||||
MetricName metricName = metricName(nameTemplate);
|
||||
if (metrics().metric(metricName) == null) {
|
||||
metrics().addMetric(metricName, new Gauge<T>() {
|
||||
@Override
|
||||
public T value(MetricConfig config, long now) {
|
||||
return supplier.metricValue(now);
|
||||
}
|
||||
});
|
||||
metrics().addMetric(metricName, (Gauge<T>) (config, now) -> supplier.metricValue(now));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -324,12 +319,7 @@ public class ConnectMetrics {
|
|||
public <T> void addImmutableValueMetric(MetricNameTemplate nameTemplate, final T value) {
|
||||
MetricName metricName = metricName(nameTemplate);
|
||||
if (metrics().metric(metricName) == null) {
|
||||
metrics().addMetric(metricName, new Gauge<T>() {
|
||||
@Override
|
||||
public T value(MetricConfig config, long now) {
|
||||
return value;
|
||||
}
|
||||
});
|
||||
metrics().addMetric(metricName, (Gauge<T>) (config, now) -> value);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -454,4 +444,4 @@ public class ConnectMetrics {
|
|||
ConnectMetricsRegistry metrics = new ConnectMetricsRegistry();
|
||||
System.out.println(Metrics.toHtmlTable(JMX_PREFIX, metrics.getAllTemplates()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -25,7 +25,7 @@ import kafka.security.auth._
|
|||
import kafka.security.authorizer.AuthorizerUtils
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AdminClient => JAdminClient}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.acl._
|
||||
import org.apache.kafka.common.acl.AclOperation._
|
||||
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
|
||||
|
|
@ -102,7 +102,7 @@ object AclCommand extends Logging {
|
|||
else
|
||||
new Properties()
|
||||
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
|
||||
val adminClient = JAdminClient.create(props)
|
||||
val adminClient = Admin.create(props)
|
||||
|
||||
try {
|
||||
f(adminClient)
|
||||
|
|
@ -291,7 +291,7 @@ object AclCommand extends Logging {
|
|||
|
||||
class JAuthorizerService(val authorizerClass: Class[_ <: JAuthorizer], val opts: AclCommandOptions) extends AclCommandService with Logging {
|
||||
|
||||
private def withAuthorizer()(f: JAuthorizer => Unit) {
|
||||
private def withAuthorizer()(f: JAuthorizer => Unit): Unit = {
|
||||
val defaultProps = Map(KafkaConfig.ZkEnableSecureAclsProp -> JaasUtils.isZkSecurityEnabled)
|
||||
val authorizerProperties =
|
||||
if (opts.options.has(opts.authorizerPropertiesOpt)) {
|
||||
|
|
|
|||
|
|
@ -211,7 +211,7 @@ object AdminUtils extends Logging {
|
|||
*/
|
||||
private[admin] def getRackAlternatedBrokerList(brokerRackMap: Map[Int, String]): IndexedSeq[Int] = {
|
||||
val brokersIteratorByRack = getInverseMap(brokerRackMap).map { case (rack, brokers) =>
|
||||
(rack, brokers.toIterator)
|
||||
(rack, brokers.iterator)
|
||||
}
|
||||
val racks = brokersIteratorByRack.keys.toArray.sorted
|
||||
val result = new mutable.ArrayBuffer[Int]
|
||||
|
|
|
|||
|
|
@ -110,24 +110,22 @@ object BrokerApiVersionsCommand {
|
|||
@volatile var running: Boolean = true
|
||||
val pendingFutures = new ConcurrentLinkedQueue[RequestFuture[ClientResponse]]()
|
||||
|
||||
val networkThread = new KafkaThread("admin-client-network-thread", new Runnable {
|
||||
override def run(): Unit = {
|
||||
try {
|
||||
while (running)
|
||||
client.poll(time.timer(Long.MaxValue))
|
||||
} catch {
|
||||
case t : Throwable =>
|
||||
error("admin-client-network-thread exited", t)
|
||||
} finally {
|
||||
pendingFutures.asScala.foreach { future =>
|
||||
try {
|
||||
future.raise(Errors.UNKNOWN_SERVER_ERROR)
|
||||
} catch {
|
||||
case _: IllegalStateException => // It is OK if the future has been completed
|
||||
}
|
||||
val networkThread = new KafkaThread("admin-client-network-thread", () => {
|
||||
try {
|
||||
while (running)
|
||||
client.poll(time.timer(Long.MaxValue))
|
||||
} catch {
|
||||
case t: Throwable =>
|
||||
error("admin-client-network-thread exited", t)
|
||||
} finally {
|
||||
pendingFutures.asScala.foreach { future =>
|
||||
try {
|
||||
future.raise(Errors.UNKNOWN_SERVER_ERROR)
|
||||
} catch {
|
||||
case _: IllegalStateException => // It is OK if the future has been completed
|
||||
}
|
||||
pendingFutures.clear()
|
||||
}
|
||||
pendingFutures.clear()
|
||||
}
|
||||
}, true)
|
||||
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit, PasswordEncod
|
|||
import kafka.utils.Implicits._
|
||||
import kafka.zk.{AdminZkClient, KafkaZkClient}
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, AlterConfigsOptions, ConfigEntry, DescribeClusterOptions, AdminClient => JAdminClient, Config => JConfig, ListTopicsOptions}
|
||||
import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, AlterConfigsOptions, ConfigEntry, DescribeClusterOptions, Config => JConfig, ListTopicsOptions}
|
||||
import org.apache.kafka.common.config.ConfigResource
|
||||
import org.apache.kafka.common.config.types.Password
|
||||
import org.apache.kafka.common.errors.InvalidConfigurationException
|
||||
|
|
@ -276,7 +276,7 @@ object ConfigCommand extends Config {
|
|||
else
|
||||
new Properties()
|
||||
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
|
||||
val adminClient = JAdminClient.create(props)
|
||||
val adminClient = Admin.create(props)
|
||||
|
||||
if (opts.entityTypes.size != 1)
|
||||
throw new IllegalArgumentException(s"Exactly one entity type (out of ${BrokerSupportedConfigTypes.mkString(",")}) must be specified with --bootstrap-server")
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper
|
|||
import kafka.utils._
|
||||
import org.apache.kafka.clients.admin._
|
||||
import org.apache.kafka.clients.consumer.OffsetAndMetadata
|
||||
import org.apache.kafka.clients.{CommonClientConfigs, admin}
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.common.{KafkaException, Node, TopicPartition}
|
||||
|
||||
|
|
@ -640,7 +640,7 @@ object ConsumerGroupCommand extends Logging {
|
|||
val props = if (opts.options.has(opts.commandConfigOpt)) Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) else new Properties()
|
||||
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
|
||||
configOverrides.foreach { case (k, v) => props.put(k, v)}
|
||||
admin.AdminClient.create(props)
|
||||
Admin.create(props)
|
||||
}
|
||||
|
||||
private def withTimeoutMs [T <: AbstractOptions[T]] (options : T) = {
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ import java.util.Base64
|
|||
import joptsimple.ArgumentAcceptingOptionSpec
|
||||
import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Exit, Logging}
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.clients.admin.{Admin, CreateDelegationTokenOptions, DescribeDelegationTokenOptions, ExpireDelegationTokenOptions, RenewDelegationTokenOptions, AdminClient => JAdminClient}
|
||||
import org.apache.kafka.clients.admin.{Admin, CreateDelegationTokenOptions, DescribeDelegationTokenOptions, ExpireDelegationTokenOptions, RenewDelegationTokenOptions}
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||
import org.apache.kafka.common.security.token.delegation.DelegationToken
|
||||
import org.apache.kafka.common.utils.{SecurityUtils, Utils}
|
||||
|
|
@ -146,7 +146,7 @@ object DelegationTokenCommand extends Logging {
|
|||
private def createAdminClient(opts: DelegationTokenCommandOptions): Admin = {
|
||||
val props = Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
|
||||
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
|
||||
JAdminClient.create(props)
|
||||
Admin.create(props)
|
||||
}
|
||||
|
||||
class DelegationTokenCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
|
||||
|
|
|
|||
|
|
@ -23,8 +23,8 @@ import java.util.Properties
|
|||
import kafka.common.AdminCommandFailedException
|
||||
import kafka.utils.json.JsonValue
|
||||
import kafka.utils.{CommandDefaultOptions, CommandLineUtils, CoreUtils, Json}
|
||||
import org.apache.kafka.clients.admin.RecordsToDelete
|
||||
import org.apache.kafka.clients.{CommonClientConfigs, admin}
|
||||
import org.apache.kafka.clients.admin.{Admin, RecordsToDelete}
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
|
||||
|
|
@ -100,13 +100,13 @@ object DeleteRecordsCommand {
|
|||
adminClient.close()
|
||||
}
|
||||
|
||||
private def createAdminClient(opts: DeleteRecordsCommandOptions): admin.Admin = {
|
||||
private def createAdminClient(opts: DeleteRecordsCommandOptions): Admin = {
|
||||
val props = if (opts.options.has(opts.commandConfigOpt))
|
||||
Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
|
||||
else
|
||||
new Properties()
|
||||
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
|
||||
admin.AdminClient.create(props)
|
||||
Admin.create(props)
|
||||
}
|
||||
|
||||
class DeleteRecordsCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) {
|
||||
|
|
|
|||
|
|
@ -25,7 +25,7 @@ import kafka.utils.CommandLineUtils
|
|||
import kafka.utils.CoreUtils
|
||||
import kafka.utils.Json
|
||||
import kafka.utils.Logging
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AdminClient => JAdminClient}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.ElectionType
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.errors.ClusterAuthorizationException
|
||||
|
|
@ -81,7 +81,7 @@ object LeaderElectionCommand extends Logging {
|
|||
)
|
||||
props.setProperty(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, timeout.toMillis.toString)
|
||||
|
||||
JAdminClient.create(props)
|
||||
Admin.create(props)
|
||||
}
|
||||
|
||||
try {
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ import java.io.PrintStream
|
|||
import java.util.Properties
|
||||
|
||||
import kafka.utils.{CommandDefaultOptions, CommandLineUtils, Json}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, DescribeLogDirsResult, AdminClient => JAdminClient}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, DescribeLogDirsResult}
|
||||
import org.apache.kafka.common.requests.DescribeLogDirsResponse.LogDirInfo
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
|
||||
|
|
@ -89,7 +89,7 @@ object LogDirsCommand {
|
|||
new Properties()
|
||||
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
|
||||
props.putIfAbsent(AdminClientConfig.CLIENT_ID_CONFIG, "log-dirs-tool")
|
||||
JAdminClient.create(props)
|
||||
Admin.create(props)
|
||||
}
|
||||
|
||||
class LogDirsCommandOptions(args: Array[String]) extends CommandDefaultOptions(args){
|
||||
|
|
|
|||
|
|
@ -20,11 +20,12 @@ import collection.JavaConverters._
|
|||
import collection._
|
||||
import java.util.Properties
|
||||
import java.util.concurrent.ExecutionException
|
||||
|
||||
import joptsimple.OptionSpecBuilder
|
||||
import kafka.common.AdminCommandFailedException
|
||||
import kafka.utils._
|
||||
import kafka.zk.KafkaZkClient
|
||||
import org.apache.kafka.clients.admin.AdminClientConfig
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.ElectionType
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.errors.ClusterAuthorizationException
|
||||
|
|
@ -209,7 +210,7 @@ object PreferredReplicaLeaderElectionCommand extends Logging {
|
|||
class AdminClientCommand(adminClientProps: Properties)
|
||||
extends Command with Logging {
|
||||
|
||||
val adminClient = org.apache.kafka.clients.admin.AdminClient.create(adminClientProps)
|
||||
val adminClient = Admin.create(adminClientProps)
|
||||
|
||||
override def electPreferredLeaders(partitionsFromUser: Option[Set[TopicPartition]]): Unit = {
|
||||
val partitions = partitionsFromUser match {
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ import kafka.utils._
|
|||
import kafka.utils.json.JsonValue
|
||||
import kafka.zk.{AdminZkClient, KafkaZkClient}
|
||||
import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterReplicaLogDirsOptions, AdminClient => JAdminClient}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterReplicaLogDirsOptions}
|
||||
import org.apache.kafka.common.errors.ReplicaNotAvailableException
|
||||
import org.apache.kafka.common.security.JaasUtils
|
||||
import org.apache.kafka.common.utils.{Time, Utils}
|
||||
|
|
@ -78,7 +78,7 @@ object ReassignPartitionsCommand extends Logging {
|
|||
new Properties()
|
||||
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
|
||||
props.putIfAbsent(AdminClientConfig.CLIENT_ID_CONFIG, "reassign-partitions-tool")
|
||||
Some(JAdminClient.create(props))
|
||||
Some(Admin.create(props))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ import kafka.utils.Implicits._
|
|||
import kafka.utils._
|
||||
import kafka.zk.{AdminZkClient, KafkaZkClient}
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.clients.admin.{Admin, ConfigEntry, ListPartitionReassignmentsOptions, ListTopicsOptions, NewPartitions, NewTopic, PartitionReassignment, AdminClient => JAdminClient, Config => JConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, ConfigEntry, ListPartitionReassignmentsOptions, ListTopicsOptions, NewPartitions, NewTopic, PartitionReassignment, Config => JConfig}
|
||||
import org.apache.kafka.common.{Node, TopicPartition, TopicPartitionInfo}
|
||||
import org.apache.kafka.common.config.ConfigResource.Type
|
||||
import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
|
||||
|
|
@ -209,7 +209,7 @@ object TopicCommand extends Logging {
|
|||
case Some(serverList) => commandConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, serverList)
|
||||
case None =>
|
||||
}
|
||||
JAdminClient.create(commandConfig)
|
||||
Admin.create(commandConfig)
|
||||
}
|
||||
|
||||
def apply(commandConfig: Properties, bootstrapServer: Option[String]): AdminClientTopicService =
|
||||
|
|
|
|||
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package kafka.cluster
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock
|
||||
import java.util.{Optional, Properties}
|
||||
|
||||
|
|
@ -218,69 +217,18 @@ class Partition(val topicPartition: TopicPartition,
|
|||
|
||||
private val tags = Map("topic" -> topic, "partition" -> partitionId.toString)
|
||||
|
||||
newGauge("UnderReplicated",
|
||||
new Gauge[Int] {
|
||||
def value: Int = {
|
||||
if (isUnderReplicated) 1 else 0
|
||||
}
|
||||
},
|
||||
tags
|
||||
)
|
||||
|
||||
newGauge("InSyncReplicasCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = {
|
||||
if (isLeader) inSyncReplicaIds.size else 0
|
||||
}
|
||||
},
|
||||
tags
|
||||
)
|
||||
|
||||
newGauge("UnderMinIsr",
|
||||
new Gauge[Int] {
|
||||
def value: Int = {
|
||||
if (isUnderMinIsr) 1 else 0
|
||||
}
|
||||
},
|
||||
tags
|
||||
)
|
||||
|
||||
newGauge("AtMinIsr",
|
||||
new Gauge[Int] {
|
||||
def value: Int = {
|
||||
if (isAtMinIsr) 1 else 0
|
||||
}
|
||||
},
|
||||
tags
|
||||
)
|
||||
|
||||
newGauge("ReplicasCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = {
|
||||
if (isLeader) assignmentState.replicationFactor else 0
|
||||
}
|
||||
},
|
||||
tags
|
||||
)
|
||||
|
||||
newGauge("LastStableOffsetLag",
|
||||
new Gauge[Long] {
|
||||
def value: Long = {
|
||||
log.map(_.lastStableOffsetLag).getOrElse(0)
|
||||
}
|
||||
},
|
||||
tags
|
||||
)
|
||||
newGauge("UnderReplicated", () => if (isUnderReplicated) 1 else 0, tags)
|
||||
newGauge("InSyncReplicasCount", () => if (isLeader) inSyncReplicaIds.size else 0, tags)
|
||||
newGauge("UnderMinIsr", () => if (isUnderMinIsr) 1 else 0, tags)
|
||||
newGauge("AtMinIsr", () => if (isAtMinIsr) 1 else 0, tags)
|
||||
newGauge("ReplicasCount", () => if (isLeader) assignmentState.replicationFactor else 0, tags)
|
||||
newGauge("LastStableOffsetLag", () => log.map(_.lastStableOffsetLag).getOrElse(0), tags)
|
||||
|
||||
def isUnderReplicated: Boolean = isLeader && (assignmentState.replicationFactor - inSyncReplicaIds.size) > 0
|
||||
|
||||
def isUnderMinIsr: Boolean = {
|
||||
leaderLogIfLocal.exists { inSyncReplicaIds.size < _.config.minInSyncReplicas }
|
||||
}
|
||||
def isUnderMinIsr: Boolean = leaderLogIfLocal.exists { inSyncReplicaIds.size < _.config.minInSyncReplicas }
|
||||
|
||||
def isAtMinIsr: Boolean = {
|
||||
leaderLogIfLocal.exists { inSyncReplicaIds.size == _.config.minInSyncReplicas }
|
||||
}
|
||||
def isAtMinIsr: Boolean = leaderLogIfLocal.exists { inSyncReplicaIds.size == _.config.minInSyncReplicas }
|
||||
|
||||
def isReassigning: Boolean = assignmentState.isInstanceOf[OngoingReassignmentState]
|
||||
|
||||
|
|
|
|||
|
|
@ -58,12 +58,9 @@ class ControllerChannelManager(controllerContext: ControllerContext,
|
|||
private val brokerLock = new Object
|
||||
this.logIdent = "[Channel manager on controller " + config.brokerId + "]: "
|
||||
|
||||
newGauge(
|
||||
"TotalQueueSize",
|
||||
new Gauge[Int] {
|
||||
def value: Int = brokerLock synchronized {
|
||||
brokerStateInfo.values.iterator.map(_.messageQueue.size).sum
|
||||
}
|
||||
newGauge("TotalQueueSize",
|
||||
() => brokerLock synchronized {
|
||||
brokerStateInfo.values.iterator.map(_.messageQueue.size).sum
|
||||
}
|
||||
)
|
||||
|
||||
|
|
@ -175,13 +172,7 @@ class ControllerChannelManager(controllerContext: ControllerContext,
|
|||
brokerNode, config, time, requestRateAndQueueTimeMetrics, stateChangeLogger, threadName)
|
||||
requestThread.setDaemon(false)
|
||||
|
||||
val queueSizeGauge = newGauge(
|
||||
QueueSizeMetricName,
|
||||
new Gauge[Int] {
|
||||
def value: Int = messageQueue.size
|
||||
},
|
||||
brokerMetricTags(broker.id)
|
||||
)
|
||||
val queueSizeGauge = newGauge(QueueSizeMetricName, () => messageQueue.size, brokerMetricTags(broker.id))
|
||||
|
||||
brokerStateInfo.put(broker.id, ControllerBrokerStateInfo(networkClient, brokerNode, messageQueue,
|
||||
requestThread, queueSizeGauge, requestRateAndQueueTimeMetrics, reconfigurableChannelBuilder))
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import java.util.concurrent.atomic.AtomicBoolean
|
|||
import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue}
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
|
||||
import kafka.utils.CoreUtils.inLock
|
||||
import kafka.utils.ShutdownableThread
|
||||
|
|
@ -82,14 +81,7 @@ class ControllerEventManager(controllerId: Int,
|
|||
|
||||
private val eventQueueTimeHist = newHistogram(EventQueueTimeMetricName)
|
||||
|
||||
newGauge(
|
||||
EventQueueSizeMetricName,
|
||||
new Gauge[Int] {
|
||||
def value: Int = {
|
||||
queue.size()
|
||||
}
|
||||
}
|
||||
)
|
||||
newGauge(EventQueueSizeMetricName, () => queue.size)
|
||||
|
||||
def state: ControllerState = _state
|
||||
|
||||
|
|
|
|||
|
|
@ -18,7 +18,6 @@ package kafka.controller
|
|||
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.admin.AdminOperationException
|
||||
import kafka.api._
|
||||
import kafka.common._
|
||||
|
|
@ -120,75 +119,16 @@ class KafkaController(val config: KafkaConfig,
|
|||
/* single-thread scheduler to clean expired tokens */
|
||||
private val tokenCleanScheduler = new KafkaScheduler(threads = 1, threadNamePrefix = "delegation-token-cleaner")
|
||||
|
||||
newGauge(
|
||||
"ActiveControllerCount",
|
||||
new Gauge[Int] {
|
||||
def value = if (isActive) 1 else 0
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"OfflinePartitionsCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = offlinePartitionCount
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"PreferredReplicaImbalanceCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = preferredReplicaImbalanceCount
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"ControllerState",
|
||||
new Gauge[Byte] {
|
||||
def value: Byte = state.value
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"GlobalTopicCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = globalTopicCount
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"GlobalPartitionCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = globalPartitionCount
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"TopicsToDeleteCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = topicsToDeleteCount
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"ReplicasToDeleteCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = replicasToDeleteCount
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"TopicsIneligibleToDeleteCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = ineligibleTopicsToDeleteCount
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"ReplicasIneligibleToDeleteCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = ineligibleReplicasToDeleteCount
|
||||
}
|
||||
)
|
||||
newGauge("ActiveControllerCount", () => if (isActive) 1 else 0)
|
||||
newGauge("OfflinePartitionsCount", () => offlinePartitionCount)
|
||||
newGauge("PreferredReplicaImbalanceCount", () => preferredReplicaImbalanceCount)
|
||||
newGauge("ControllerState", () => state.value)
|
||||
newGauge("GlobalTopicCount", () => globalTopicCount)
|
||||
newGauge("GlobalPartitionCount", () => globalPartitionCount)
|
||||
newGauge("TopicsToDeleteCount", () => topicsToDeleteCount)
|
||||
newGauge("ReplicasToDeleteCount", () => replicasToDeleteCount)
|
||||
newGauge("TopicsIneligibleToDeleteCount", () => ineligibleTopicsToDeleteCount)
|
||||
newGauge("ReplicasIneligibleToDeleteCount", () => ineligibleReplicasToDeleteCount)
|
||||
|
||||
/**
|
||||
* Returns true if this broker is the current controller.
|
||||
|
|
|
|||
|
|
@ -124,50 +124,48 @@ class GroupMetadataManager(brokerId: Int,
|
|||
}
|
||||
|
||||
recreateGauge("NumOffsets",
|
||||
new Gauge[Int] {
|
||||
def value = groupMetadataCache.values.map(group => {
|
||||
group.inLock { group.numOffsets }
|
||||
}).sum
|
||||
})
|
||||
() => groupMetadataCache.values.map { group =>
|
||||
group.inLock { group.numOffsets }
|
||||
}.sum
|
||||
)
|
||||
|
||||
recreateGauge("NumGroups",
|
||||
new Gauge[Int] {
|
||||
def value = groupMetadataCache.size
|
||||
})
|
||||
() => groupMetadataCache.size
|
||||
)
|
||||
|
||||
recreateGauge("NumGroupsPreparingRebalance",
|
||||
new Gauge[Int] {
|
||||
def value(): Int = groupMetadataCache.values.count(group => {
|
||||
group synchronized { group.is(PreparingRebalance) }
|
||||
})
|
||||
() => groupMetadataCache.values.count { group =>
|
||||
group synchronized {
|
||||
group.is(PreparingRebalance)
|
||||
}
|
||||
})
|
||||
|
||||
recreateGauge("NumGroupsCompletingRebalance",
|
||||
new Gauge[Int] {
|
||||
def value(): Int = groupMetadataCache.values.count(group => {
|
||||
group synchronized { group.is(CompletingRebalance) }
|
||||
})
|
||||
() => groupMetadataCache.values.count { group =>
|
||||
group synchronized {
|
||||
group.is(CompletingRebalance)
|
||||
}
|
||||
})
|
||||
|
||||
recreateGauge("NumGroupsStable",
|
||||
new Gauge[Int] {
|
||||
def value(): Int = groupMetadataCache.values.count(group => {
|
||||
group synchronized { group.is(Stable) }
|
||||
})
|
||||
() => groupMetadataCache.values.count { group =>
|
||||
group synchronized {
|
||||
group.is(Stable)
|
||||
}
|
||||
})
|
||||
|
||||
recreateGauge("NumGroupsDead",
|
||||
new Gauge[Int] {
|
||||
def value(): Int = groupMetadataCache.values.count(group => {
|
||||
group synchronized { group.is(Dead) }
|
||||
})
|
||||
() => groupMetadataCache.values.count { group =>
|
||||
group synchronized {
|
||||
group.is(Dead)
|
||||
}
|
||||
})
|
||||
|
||||
recreateGauge("NumGroupsEmpty",
|
||||
new Gauge[Int] {
|
||||
def value(): Int = groupMetadataCache.values.count(group => {
|
||||
group synchronized { group.is(Empty) }
|
||||
})
|
||||
() => groupMetadataCache.values.count { group =>
|
||||
group synchronized {
|
||||
group.is(Empty)
|
||||
}
|
||||
})
|
||||
|
||||
def startup(enableMetadataExpiration: Boolean): Unit = {
|
||||
|
|
|
|||
|
|
@ -30,7 +30,6 @@ import org.apache.kafka.common.security.JaasContext
|
|||
import org.apache.kafka.common.utils.{LogContext, Time}
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.requests.WriteTxnMarkersRequest.TxnMarkerEntry
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import java.util
|
||||
import java.util.concurrent.{BlockingQueue, ConcurrentHashMap, LinkedBlockingQueue}
|
||||
|
||||
|
|
@ -143,19 +142,8 @@ class TransactionMarkerChannelManager(config: KafkaConfig,
|
|||
|
||||
override val requestTimeoutMs: Int = config.requestTimeoutMs
|
||||
|
||||
newGauge(
|
||||
"UnknownDestinationQueueSize",
|
||||
new Gauge[Int] {
|
||||
def value: Int = markersQueueForUnknownBroker.totalNumMarkers
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"LogAppendRetryQueueSize",
|
||||
new Gauge[Int] {
|
||||
def value: Int = txnLogAppendRetryQueue.size
|
||||
}
|
||||
)
|
||||
newGauge("UnknownDestinationQueueSize", () => markersQueueForUnknownBroker.totalNumMarkers)
|
||||
newGauge("LogAppendRetryQueueSize", () => txnLogAppendRetryQueue.size)
|
||||
|
||||
override def generateRequests() = drainQueuedTransactionMarkers()
|
||||
|
||||
|
|
|
|||
|
|
@ -76,7 +76,7 @@ object LazyIndex {
|
|||
|
||||
private sealed trait IndexWrapper {
|
||||
def file: File
|
||||
def file_=(f: File)
|
||||
def file_=(f: File): Unit
|
||||
}
|
||||
|
||||
private class IndexFile(@volatile var file: File) extends IndexWrapper
|
||||
|
|
|
|||
|
|
@ -27,7 +27,6 @@ import java.util.concurrent.atomic._
|
|||
import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap, TimeUnit}
|
||||
import java.util.regex.Pattern
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0}
|
||||
import kafka.common.{LogSegmentOffsetOverflowException, LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException}
|
||||
import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec}
|
||||
|
|
@ -465,29 +464,10 @@ class Log(@volatile var dir: File,
|
|||
Map("topic" -> topicPartition.topic, "partition" -> topicPartition.partition.toString) ++ maybeFutureTag
|
||||
}
|
||||
|
||||
newGauge(LogMetricNames.NumLogSegments,
|
||||
new Gauge[Int] {
|
||||
def value = numberOfSegments
|
||||
},
|
||||
tags)
|
||||
|
||||
newGauge(LogMetricNames.LogStartOffset,
|
||||
new Gauge[Long] {
|
||||
def value = logStartOffset
|
||||
},
|
||||
tags)
|
||||
|
||||
newGauge(LogMetricNames.LogEndOffset,
|
||||
new Gauge[Long] {
|
||||
def value = logEndOffset
|
||||
},
|
||||
tags)
|
||||
|
||||
newGauge(LogMetricNames.Size,
|
||||
new Gauge[Long] {
|
||||
def value = size
|
||||
},
|
||||
tags)
|
||||
newGauge(LogMetricNames.NumLogSegments, () => numberOfSegments, tags)
|
||||
newGauge(LogMetricNames.LogStartOffset, () => logStartOffset, tags)
|
||||
newGauge(LogMetricNames.LogEndOffset, () => logEndOffset, tags)
|
||||
newGauge(LogMetricNames.Size, () => size, tags)
|
||||
|
||||
val producerExpireCheck = scheduler.schedule(name = "PeriodicProducerExpirationCheck", fun = () => {
|
||||
lock synchronized {
|
||||
|
|
|
|||
|
|
@ -22,7 +22,6 @@ import java.nio._
|
|||
import java.util.Date
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.common._
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.server.{BrokerReconfigurable, KafkaConfig, LogDirFailureChannel}
|
||||
|
|
@ -114,34 +113,24 @@ class LogCleaner(initialConfig: CleanerConfig,
|
|||
|
||||
/* a metric to track the maximum utilization of any thread's buffer in the last cleaning */
|
||||
newGauge("max-buffer-utilization-percent",
|
||||
new Gauge[Int] {
|
||||
def value: Int = cleaners.map(_.lastStats).map(100 * _.bufferUtilization).max.toInt
|
||||
})
|
||||
() => cleaners.iterator.map(100 * _.lastStats.bufferUtilization).max.toInt)
|
||||
|
||||
/* a metric to track the recopy rate of each thread's last cleaning */
|
||||
newGauge("cleaner-recopy-percent",
|
||||
new Gauge[Int] {
|
||||
def value: Int = {
|
||||
val stats = cleaners.map(_.lastStats)
|
||||
val recopyRate = stats.map(_.bytesWritten).sum.toDouble / math.max(stats.map(_.bytesRead).sum, 1)
|
||||
(100 * recopyRate).toInt
|
||||
}
|
||||
})
|
||||
newGauge("cleaner-recopy-percent", () => {
|
||||
val stats = cleaners.map(_.lastStats)
|
||||
val recopyRate = stats.iterator.map(_.bytesWritten).sum.toDouble / math.max(stats.iterator.map(_.bytesRead).sum, 1)
|
||||
(100 * recopyRate).toInt
|
||||
})
|
||||
|
||||
/* a metric to track the maximum cleaning time for the last cleaning from each thread */
|
||||
newGauge("max-clean-time-secs",
|
||||
new Gauge[Int] {
|
||||
def value: Int = cleaners.map(_.lastStats).map(_.elapsedSecs).max.toInt
|
||||
})
|
||||
newGauge("max-clean-time-secs", () => cleaners.iterator.map(_.lastStats.elapsedSecs).max.toInt)
|
||||
|
||||
// a metric to track delay between the time when a log is required to be compacted
|
||||
// as determined by max compaction lag and the time of last cleaner run.
|
||||
newGauge("max-compaction-delay-secs",
|
||||
new Gauge[Int] {
|
||||
def value: Int = Math.max(0, (cleaners.map(_.lastPreCleanStats).map(_.maxCompactionDelayMs).max / 1000).toInt)
|
||||
})
|
||||
() => Math.max(0, (cleaners.iterator.map(_.lastPreCleanStats.maxCompactionDelayMs).max / 1000).toInt))
|
||||
|
||||
newGauge("DeadThreadCount",
|
||||
new Gauge[Int] {
|
||||
def value: Int = deadThreadCount
|
||||
})
|
||||
newGauge("DeadThreadCount", () => deadThreadCount)
|
||||
|
||||
private[log] def deadThreadCount: Int = cleaners.count(_.isThreadFailed)
|
||||
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import java.io.File
|
|||
import java.util.concurrent.TimeUnit
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.common.{KafkaException, LogCleaningAbortedException}
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.server.LogDirFailureChannel
|
||||
|
|
@ -89,48 +88,41 @@ private[log] class LogCleanerManager(val logDirs: Seq[File],
|
|||
|
||||
/* gauges for tracking the number of partitions marked as uncleanable for each log directory */
|
||||
for (dir <- logDirs) {
|
||||
newGauge(
|
||||
"uncleanable-partitions-count",
|
||||
new Gauge[Int] { def value = inLock(lock) { uncleanablePartitions.get(dir.getAbsolutePath).map(_.size).getOrElse(0) } },
|
||||
newGauge("uncleanable-partitions-count",
|
||||
() => inLock(lock) { uncleanablePartitions.get(dir.getAbsolutePath).map(_.size).getOrElse(0) },
|
||||
Map("logDirectory" -> dir.getAbsolutePath)
|
||||
)
|
||||
}
|
||||
|
||||
/* gauges for tracking the number of uncleanable bytes from uncleanable partitions for each log directory */
|
||||
for (dir <- logDirs) {
|
||||
newGauge(
|
||||
"uncleanable-bytes",
|
||||
new Gauge[Long] {
|
||||
def value = {
|
||||
inLock(lock) {
|
||||
uncleanablePartitions.get(dir.getAbsolutePath) match {
|
||||
case Some(partitions) => {
|
||||
val lastClean = allCleanerCheckpoints
|
||||
val now = Time.SYSTEM.milliseconds
|
||||
partitions.map { tp =>
|
||||
val log = logs.get(tp)
|
||||
val lastCleanOffset = lastClean.get(tp)
|
||||
val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now)
|
||||
val (_, uncleanableBytes) = calculateCleanableBytes(log, firstDirtyOffset, firstUncleanableDirtyOffset)
|
||||
uncleanableBytes
|
||||
}.sum
|
||||
}
|
||||
case _ => 0
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
Map("logDirectory" -> dir.getAbsolutePath)
|
||||
)
|
||||
}
|
||||
for (dir <- logDirs) {
|
||||
newGauge("uncleanable-bytes",
|
||||
() => inLock(lock) {
|
||||
uncleanablePartitions.get(dir.getAbsolutePath) match {
|
||||
case Some(partitions) =>
|
||||
val lastClean = allCleanerCheckpoints
|
||||
val now = Time.SYSTEM.milliseconds
|
||||
partitions.iterator.map { tp =>
|
||||
val log = logs.get(tp)
|
||||
val lastCleanOffset = lastClean.get(tp)
|
||||
val (firstDirtyOffset, firstUncleanableDirtyOffset) = cleanableOffsets(log, lastCleanOffset, now)
|
||||
val (_, uncleanableBytes) = calculateCleanableBytes(log, firstDirtyOffset, firstUncleanableDirtyOffset)
|
||||
uncleanableBytes
|
||||
}.sum
|
||||
case None => 0
|
||||
}
|
||||
},
|
||||
Map("logDirectory" -> dir.getAbsolutePath)
|
||||
)
|
||||
}
|
||||
|
||||
/* a gauge for tracking the cleanable ratio of the dirtiest log */
|
||||
@volatile private var dirtiestLogCleanableRatio = 0.0
|
||||
newGauge("max-dirty-percent", new Gauge[Int] { def value = (100 * dirtiestLogCleanableRatio).toInt })
|
||||
newGauge("max-dirty-percent", () => (100 * dirtiestLogCleanableRatio).toInt)
|
||||
|
||||
/* a gauge for tracking the time since the last log cleaner run, in milli seconds */
|
||||
@volatile private var timeOfLastRun : Long = Time.SYSTEM.milliseconds
|
||||
newGauge("time-since-last-run-ms", new Gauge[Long] { def value = Time.SYSTEM.milliseconds - timeOfLastRun })
|
||||
@volatile private var timeOfLastRun: Long = Time.SYSTEM.milliseconds
|
||||
newGauge("time-since-last-run-ms", () => Time.SYSTEM.milliseconds - timeOfLastRun)
|
||||
|
||||
/**
|
||||
* @return the position processed for all logs.
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import java.io._
|
|||
import java.nio.file.Files
|
||||
import java.util.concurrent._
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.server.checkpoints.OffsetCheckpointFile
|
||||
import kafka.server.{BrokerState, RecoveringFromUncleanShutdown, _}
|
||||
|
|
@ -118,28 +117,18 @@ class LogManager(logDirs: Seq[File],
|
|||
|
||||
loadLogs()
|
||||
|
||||
// public, so we can access this from kafka.admin.DeleteTopicTest
|
||||
val cleaner: LogCleaner =
|
||||
if(cleanerConfig.enableCleaner)
|
||||
private[kafka] val cleaner: LogCleaner =
|
||||
if (cleanerConfig.enableCleaner)
|
||||
new LogCleaner(cleanerConfig, liveLogDirs, currentLogs, logDirFailureChannel, time = time)
|
||||
else
|
||||
null
|
||||
|
||||
val offlineLogDirectoryCount = newGauge(
|
||||
"OfflineLogDirectoryCount",
|
||||
new Gauge[Int] {
|
||||
def value = offlineLogDirs.size
|
||||
}
|
||||
)
|
||||
newGauge("OfflineLogDirectoryCount", () => offlineLogDirs.size)
|
||||
|
||||
for (dir <- logDirs) {
|
||||
newGauge(
|
||||
"LogDirectoryOffline",
|
||||
new Gauge[Int] {
|
||||
def value = if (_liveLogDirs.contains(dir)) 0 else 1
|
||||
},
|
||||
Map("logDirectory" -> dir.getAbsolutePath)
|
||||
)
|
||||
newGauge("LogDirectoryOffline",
|
||||
() => if (_liveLogDirs.contains(dir)) 0 else 1,
|
||||
Map("logDirectory" -> dir.getAbsolutePath))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -348,7 +337,7 @@ class LogManager(logDirs: Seq[File],
|
|||
dirContent <- Option(dir.listFiles).toList
|
||||
logDir <- dirContent if logDir.isDirectory
|
||||
} yield {
|
||||
CoreUtils.runnable {
|
||||
val runnable: Runnable = () => {
|
||||
try {
|
||||
loadLog(logDir, recoveryPoints, logStartOffsets)
|
||||
} catch {
|
||||
|
|
@ -357,6 +346,7 @@ class LogManager(logDirs: Seq[File],
|
|||
error(s"Error while loading log dir ${dir.getAbsolutePath}", e)
|
||||
}
|
||||
}
|
||||
runnable
|
||||
}
|
||||
jobs(cleanShutdownFile) = jobsForDir.map(pool.submit)
|
||||
} catch {
|
||||
|
|
@ -459,12 +449,13 @@ class LogManager(logDirs: Seq[File],
|
|||
|
||||
val logsInDir = localLogsByDir.getOrElse(dir.toString, Map()).values
|
||||
|
||||
val jobsForDir = logsInDir map { log =>
|
||||
CoreUtils.runnable {
|
||||
val jobsForDir = logsInDir.map { log =>
|
||||
val runnable: Runnable = () => {
|
||||
// flush the log to ensure latest possible recovery point
|
||||
log.flush()
|
||||
log.close()
|
||||
}
|
||||
runnable
|
||||
}
|
||||
|
||||
jobs(dir) = jobsForDir.map(pool.submit).toSeq
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import java.nio.ByteBuffer
|
|||
import java.util.concurrent._
|
||||
|
||||
import com.typesafe.scalalogging.Logger
|
||||
import com.yammer.metrics.core.{Gauge, Meter}
|
||||
import com.yammer.metrics.core.Meter
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.utils.{Logging, NotNothing, Pool}
|
||||
import org.apache.kafka.common.memory.MemoryPool
|
||||
|
|
@ -281,12 +281,10 @@ class RequestChannel(val queueSize: Int, val metricNamePrefix : String) extends
|
|||
val requestQueueSizeMetricName = metricNamePrefix.concat(RequestQueueSizeMetric)
|
||||
val responseQueueSizeMetricName = metricNamePrefix.concat(ResponseQueueSizeMetric)
|
||||
|
||||
newGauge(requestQueueSizeMetricName, new Gauge[Int] {
|
||||
def value = requestQueue.size
|
||||
})
|
||||
newGauge(requestQueueSizeMetricName, () => requestQueue.size)
|
||||
|
||||
newGauge(responseQueueSizeMetricName, new Gauge[Int]{
|
||||
def value = processors.values.asScala.foldLeft(0) {(total, processor) =>
|
||||
newGauge(responseQueueSizeMetricName, () => {
|
||||
processors.values.asScala.foldLeft(0) {(total, processor) =>
|
||||
total + processor.responseQueueSize
|
||||
}
|
||||
})
|
||||
|
|
@ -295,12 +293,8 @@ class RequestChannel(val queueSize: Int, val metricNamePrefix : String) extends
|
|||
if (processors.putIfAbsent(processor.id, processor) != null)
|
||||
warn(s"Unexpected processor with processorId ${processor.id}")
|
||||
|
||||
newGauge(responseQueueSizeMetricName,
|
||||
new Gauge[Int] {
|
||||
def value = processor.responseQueueSize
|
||||
},
|
||||
Map(ProcessorMetricTag -> processor.id.toString)
|
||||
)
|
||||
newGauge(responseQueueSizeMetricName, () => processor.responseQueueSize,
|
||||
Map(ProcessorMetricTag -> processor.id.toString))
|
||||
}
|
||||
|
||||
def removeProcessor(processorId: Int): Unit = {
|
||||
|
|
|
|||
|
|
@ -27,7 +27,6 @@ import java.util.concurrent._
|
|||
import java.util.concurrent.atomic._
|
||||
import java.util.function.Supplier
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.cluster.{BrokerEndPoint, EndPoint}
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.network.RequestChannel.{CloseConnectionResponse, EndThrottlingResponse, NoOpResponse, SendResponse, StartThrottlingResponse}
|
||||
|
|
@ -127,68 +126,40 @@ class SocketServer(val config: KafkaConfig,
|
|||
}
|
||||
}
|
||||
|
||||
newGauge(s"${DataPlaneMetricPrefix}NetworkProcessorAvgIdlePercent",
|
||||
new Gauge[Double] {
|
||||
|
||||
def value = SocketServer.this.synchronized {
|
||||
val ioWaitRatioMetricNames = dataPlaneProcessors.values.asScala.map { p =>
|
||||
metrics.metricName("io-wait-ratio", MetricsGroup, p.metricTags)
|
||||
}
|
||||
ioWaitRatioMetricNames.map { metricName =>
|
||||
Option(metrics.metric(metricName)).fold(0.0)(m => Math.min(m.metricValue.asInstanceOf[Double], 1.0))
|
||||
}.sum / dataPlaneProcessors.size
|
||||
}
|
||||
newGauge(s"${DataPlaneMetricPrefix}NetworkProcessorAvgIdlePercent", () => SocketServer.this.synchronized {
|
||||
val ioWaitRatioMetricNames = dataPlaneProcessors.values.asScala.iterator.map { p =>
|
||||
metrics.metricName("io-wait-ratio", MetricsGroup, p.metricTags)
|
||||
}
|
||||
)
|
||||
newGauge(s"${ControlPlaneMetricPrefix}NetworkProcessorAvgIdlePercent",
|
||||
new Gauge[Double] {
|
||||
|
||||
def value = SocketServer.this.synchronized {
|
||||
val ioWaitRatioMetricName = controlPlaneProcessorOpt.map { p =>
|
||||
metrics.metricName("io-wait-ratio", "socket-server-metrics", p.metricTags)
|
||||
}
|
||||
ioWaitRatioMetricName.map { metricName =>
|
||||
Option(metrics.metric(metricName)).fold(0.0)(m => Math.min(m.metricValue.asInstanceOf[Double], 1.0))
|
||||
}.getOrElse(Double.NaN)
|
||||
}
|
||||
ioWaitRatioMetricNames.map { metricName =>
|
||||
Option(metrics.metric(metricName)).fold(0.0)(m => Math.min(m.metricValue.asInstanceOf[Double], 1.0))
|
||||
}.sum / dataPlaneProcessors.size
|
||||
})
|
||||
newGauge(s"${ControlPlaneMetricPrefix}NetworkProcessorAvgIdlePercent", () => SocketServer.this.synchronized {
|
||||
val ioWaitRatioMetricName = controlPlaneProcessorOpt.map { p =>
|
||||
metrics.metricName("io-wait-ratio", "socket-server-metrics", p.metricTags)
|
||||
}
|
||||
)
|
||||
newGauge("MemoryPoolAvailable",
|
||||
new Gauge[Long] {
|
||||
def value = memoryPool.availableMemory()
|
||||
ioWaitRatioMetricName.map { metricName =>
|
||||
Option(metrics.metric(metricName)).fold(0.0)(m => Math.min(m.metricValue.asInstanceOf[Double], 1.0))
|
||||
}.getOrElse(Double.NaN)
|
||||
})
|
||||
newGauge("MemoryPoolAvailable", () => memoryPool.availableMemory)
|
||||
newGauge("MemoryPoolUsed", () => memoryPool.size() - memoryPool.availableMemory)
|
||||
newGauge(s"${DataPlaneMetricPrefix}ExpiredConnectionsKilledCount", () => SocketServer.this.synchronized {
|
||||
val expiredConnectionsKilledCountMetricNames = dataPlaneProcessors.values.asScala.iterator.map { p =>
|
||||
metrics.metricName("expired-connections-killed-count", "socket-server-metrics", p.metricTags)
|
||||
}
|
||||
)
|
||||
newGauge("MemoryPoolUsed",
|
||||
new Gauge[Long] {
|
||||
def value = memoryPool.size() - memoryPool.availableMemory()
|
||||
expiredConnectionsKilledCountMetricNames.map { metricName =>
|
||||
Option(metrics.metric(metricName)).fold(0.0)(m => m.metricValue.asInstanceOf[Double])
|
||||
}.sum
|
||||
})
|
||||
newGauge(s"${ControlPlaneMetricPrefix}ExpiredConnectionsKilledCount", () => SocketServer.this.synchronized {
|
||||
val expiredConnectionsKilledCountMetricNames = controlPlaneProcessorOpt.map { p =>
|
||||
metrics.metricName("expired-connections-killed-count", "socket-server-metrics", p.metricTags)
|
||||
}
|
||||
)
|
||||
newGauge(s"${DataPlaneMetricPrefix}ExpiredConnectionsKilledCount",
|
||||
new Gauge[Double] {
|
||||
|
||||
def value = SocketServer.this.synchronized {
|
||||
val expiredConnectionsKilledCountMetricNames = dataPlaneProcessors.values.asScala.map { p =>
|
||||
metrics.metricName("expired-connections-killed-count", "socket-server-metrics", p.metricTags)
|
||||
}
|
||||
expiredConnectionsKilledCountMetricNames.map { metricName =>
|
||||
Option(metrics.metric(metricName)).fold(0.0)(m => m.metricValue.asInstanceOf[Double])
|
||||
}.sum
|
||||
}
|
||||
}
|
||||
)
|
||||
newGauge(s"${ControlPlaneMetricPrefix}ExpiredConnectionsKilledCount",
|
||||
new Gauge[Double] {
|
||||
|
||||
def value = SocketServer.this.synchronized {
|
||||
val expiredConnectionsKilledCountMetricNames = controlPlaneProcessorOpt.map { p =>
|
||||
metrics.metricName("expired-connections-killed-count", "socket-server-metrics", p.metricTags)
|
||||
}
|
||||
expiredConnectionsKilledCountMetricNames.map { metricName =>
|
||||
Option(metrics.metric(metricName)).fold(0.0)(m => m.metricValue.asInstanceOf[Double])
|
||||
}.getOrElse(0.0)
|
||||
}
|
||||
}
|
||||
)
|
||||
expiredConnectionsKilledCountMetricNames.map { metricName =>
|
||||
Option(metrics.metric(metricName)).fold(0.0)(m => m.metricValue.asInstanceOf[Double])
|
||||
}.getOrElse(0.0)
|
||||
})
|
||||
info(s"Started ${dataPlaneAcceptors.size} acceptor threads for data-plane")
|
||||
if (controlPlaneAcceptorOpt.isDefined)
|
||||
info("Started control-plane acceptor thread")
|
||||
|
|
@ -729,12 +700,9 @@ private[kafka] class Processor(val id: Int,
|
|||
NetworkProcessorMetricTag -> id.toString
|
||||
).asJava
|
||||
|
||||
newGauge(IdlePercentMetricName,
|
||||
new Gauge[Double] {
|
||||
def value = {
|
||||
Option(metrics.metric(metrics.metricName("io-wait-ratio", MetricsGroup, metricTags)))
|
||||
.fold(0.0)(m => Math.min(m.metricValue.asInstanceOf[Double], 1.0))
|
||||
}
|
||||
newGauge(IdlePercentMetricName, () => {
|
||||
Option(metrics.metric(metrics.metricName("io-wait-ratio", MetricsGroup, metricTags))).fold(0.0)(m =>
|
||||
Math.min(m.metricValue.asInstanceOf[Double], 1.0))
|
||||
},
|
||||
// for compatibility, only add a networkProcessor tag to the Yammer Metrics alias (the equivalent Selector metric
|
||||
// also includes the listener name)
|
||||
|
|
|
|||
|
|
@ -20,7 +20,6 @@ package kafka.server
|
|||
import kafka.utils.Logging
|
||||
import kafka.cluster.BrokerEndPoint
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
|
||||
|
|
@ -37,52 +36,27 @@ abstract class AbstractFetcherManager[T <: AbstractFetcherThread](val name: Stri
|
|||
val failedPartitions = new FailedPartitions
|
||||
this.logIdent = "[" + name + "] "
|
||||
|
||||
newGauge(
|
||||
"MaxLag",
|
||||
new Gauge[Long] {
|
||||
// current max lag across all fetchers/topics/partitions
|
||||
def value: Long = fetcherThreadMap.foldLeft(0L)((curMaxAll, fetcherThreadMapEntry) => {
|
||||
fetcherThreadMapEntry._2.fetcherLagStats.stats.foldLeft(0L)((curMaxThread, fetcherLagStatsEntry) => {
|
||||
curMaxThread.max(fetcherLagStatsEntry._2.lag)
|
||||
}).max(curMaxAll)
|
||||
})
|
||||
},
|
||||
Map("clientId" -> clientId)
|
||||
)
|
||||
private val tags = Map("clientId" -> clientId)
|
||||
|
||||
newGauge(
|
||||
"MinFetchRate", {
|
||||
new Gauge[Double] {
|
||||
// current min fetch rate across all fetchers/topics/partitions
|
||||
def value: Double = {
|
||||
val headRate: Double =
|
||||
fetcherThreadMap.headOption.map(_._2.fetcherStats.requestRate.oneMinuteRate).getOrElse(0)
|
||||
|
||||
fetcherThreadMap.foldLeft(headRate)((curMinAll, fetcherThreadMapEntry) => {
|
||||
fetcherThreadMapEntry._2.fetcherStats.requestRate.oneMinuteRate.min(curMinAll)
|
||||
})
|
||||
}
|
||||
newGauge("MaxLag", () => {
|
||||
// current max lag across all fetchers/topics/partitions
|
||||
fetcherThreadMap.values.foldLeft(0L) { (curMaxLagAll, fetcherThread) =>
|
||||
val maxLagThread = fetcherThread.fetcherLagStats.stats.values.foldLeft(0L)((curMaxLagThread, lagMetrics) =>
|
||||
math.max(curMaxLagThread, lagMetrics.lag))
|
||||
math.max(curMaxLagAll, maxLagThread)
|
||||
}
|
||||
},
|
||||
Map("clientId" -> clientId)
|
||||
)
|
||||
}, tags)
|
||||
|
||||
val failedPartitionsCount = newGauge(
|
||||
"FailedPartitionsCount", {
|
||||
new Gauge[Int] {
|
||||
def value: Int = failedPartitions.size
|
||||
}
|
||||
},
|
||||
Map("clientId" -> clientId)
|
||||
)
|
||||
newGauge("MinFetchRate", () => {
|
||||
// current min fetch rate across all fetchers/topics/partitions
|
||||
val headRate = fetcherThreadMap.values.headOption.map(_.fetcherStats.requestRate.oneMinuteRate).getOrElse(0.0)
|
||||
fetcherThreadMap.values.foldLeft(headRate)((curMinAll, fetcherThread) =>
|
||||
math.min(curMinAll, fetcherThread.fetcherStats.requestRate.oneMinuteRate))
|
||||
}, tags)
|
||||
|
||||
newGauge("DeadThreadCount", {
|
||||
new Gauge[Int] {
|
||||
def value: Int = {
|
||||
deadThreadCount
|
||||
}
|
||||
}
|
||||
}, Map("clientId" -> clientId))
|
||||
newGauge("FailedPartitionsCount", () => failedPartitions.size, tags)
|
||||
|
||||
newGauge("DeadThreadCount", () => deadThreadCount, tags)
|
||||
|
||||
private[server] def deadThreadCount: Int = lock synchronized { fetcherThreadMap.values.count(_.isThreadFailed) }
|
||||
|
||||
|
|
|
|||
|
|
@ -35,9 +35,7 @@ import scala.collection.{mutable, Map, Set}
|
|||
import scala.collection.JavaConverters._
|
||||
import java.util.concurrent.TimeUnit
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import java.util.function.BiConsumer
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.log.LogAppendInfo
|
||||
import kafka.server.AbstractFetcherThread.ReplicaFetch
|
||||
import kafka.server.AbstractFetcherThread.ResultWithPartitions
|
||||
|
|
@ -154,18 +152,16 @@ abstract class AbstractFetcherThread(name: String,
|
|||
val partitionsWithEpochs = mutable.Map.empty[TopicPartition, EpochData]
|
||||
val partitionsWithoutEpochs = mutable.Set.empty[TopicPartition]
|
||||
|
||||
partitionStates.partitionStateMap.forEach(new BiConsumer[TopicPartition, PartitionFetchState] {
|
||||
override def accept(tp: TopicPartition, state: PartitionFetchState): Unit = {
|
||||
if (state.isTruncating) {
|
||||
latestEpoch(tp) match {
|
||||
case Some(epoch) if isOffsetForLeaderEpochSupported =>
|
||||
partitionsWithEpochs += tp -> new EpochData(Optional.of(state.currentLeaderEpoch), epoch)
|
||||
case _ =>
|
||||
partitionsWithoutEpochs += tp
|
||||
}
|
||||
partitionStates.partitionStateMap.forEach { (tp, state) =>
|
||||
if (state.isTruncating) {
|
||||
latestEpoch(tp) match {
|
||||
case Some(epoch) if isOffsetForLeaderEpochSupported =>
|
||||
partitionsWithEpochs += tp -> new EpochData(Optional.of(state.currentLeaderEpoch), epoch)
|
||||
case _ =>
|
||||
partitionsWithoutEpochs += tp
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
(partitionsWithEpochs, partitionsWithoutEpochs)
|
||||
}
|
||||
|
|
@ -688,12 +684,7 @@ class FetcherLagMetrics(metricId: ClientIdTopicPartition) extends KafkaMetricsGr
|
|||
"topic" -> metricId.topicPartition.topic,
|
||||
"partition" -> metricId.topicPartition.partition.toString)
|
||||
|
||||
newGauge(FetcherMetrics.ConsumerLag,
|
||||
new Gauge[Long] {
|
||||
def value = lagVal.get
|
||||
},
|
||||
tags
|
||||
)
|
||||
newGauge(FetcherMetrics.ConsumerLag, () => lagVal.get, tags)
|
||||
|
||||
def lag_=(newLag: Long): Unit = {
|
||||
lagVal.set(newLag)
|
||||
|
|
|
|||
|
|
@ -92,7 +92,7 @@ class DelayedFuturePurgatory(purgatoryName: String, brokerId: Int) {
|
|||
delayedFuture
|
||||
}
|
||||
|
||||
def shutdown() {
|
||||
def shutdown(): Unit = {
|
||||
executor.shutdownNow()
|
||||
executor.awaitTermination(60, TimeUnit.SECONDS)
|
||||
purgatory.shutdown()
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import java.util.concurrent._
|
|||
import java.util.concurrent.atomic._
|
||||
import java.util.concurrent.locks.{Lock, ReentrantLock}
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.utils.CoreUtils.inLock
|
||||
import kafka.utils._
|
||||
|
|
@ -198,22 +197,8 @@ final class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: Stri
|
|||
private val expirationReaper = new ExpiredOperationReaper()
|
||||
|
||||
private val metricsTags = Map("delayedOperation" -> purgatoryName)
|
||||
|
||||
newGauge(
|
||||
"PurgatorySize",
|
||||
new Gauge[Int] {
|
||||
def value: Int = watched
|
||||
},
|
||||
metricsTags
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"NumDelayedOperations",
|
||||
new Gauge[Int] {
|
||||
def value: Int = numDelayed
|
||||
},
|
||||
metricsTags
|
||||
)
|
||||
newGauge("PurgatorySize", () => watched, metricsTags)
|
||||
newGauge("NumDelayedOperations", () => numDelayed, metricsTags)
|
||||
|
||||
if (reaperEnabled)
|
||||
expirationReaper.start()
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import java.util
|
|||
import java.util.Optional
|
||||
import java.util.concurrent.{ThreadLocalRandom, TimeUnit}
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.utils.Logging
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
|
|
@ -547,19 +546,11 @@ class FetchSessionCache(private val maxEntries: Int,
|
|||
|
||||
// Set up metrics.
|
||||
removeMetric(FetchSession.NUM_INCREMENTAL_FETCH_SESSISONS)
|
||||
newGauge(FetchSession.NUM_INCREMENTAL_FETCH_SESSISONS,
|
||||
new Gauge[Int] {
|
||||
def value = FetchSessionCache.this.size
|
||||
}
|
||||
)
|
||||
newGauge(FetchSession.NUM_INCREMENTAL_FETCH_SESSISONS, () => FetchSessionCache.this.size)
|
||||
removeMetric(FetchSession.NUM_INCREMENTAL_FETCH_PARTITIONS_CACHED)
|
||||
newGauge(FetchSession.NUM_INCREMENTAL_FETCH_PARTITIONS_CACHED,
|
||||
new Gauge[Long] {
|
||||
def value = FetchSessionCache.this.totalPartitions
|
||||
}
|
||||
)
|
||||
newGauge(FetchSession.NUM_INCREMENTAL_FETCH_PARTITIONS_CACHED, () => FetchSessionCache.this.totalPartitions)
|
||||
removeMetric(FetchSession.INCREMENTAL_FETCH_SESSIONS_EVICTIONS_PER_SEC)
|
||||
val evictionsMeter = newMeter(FetchSession.INCREMENTAL_FETCH_SESSIONS_EVICTIONS_PER_SEC,
|
||||
private[server] val evictionsMeter = newMeter(FetchSession.INCREMENTAL_FETCH_SESSIONS_EVICTIONS_PER_SEC,
|
||||
FetchSession.EVICTIONS, TimeUnit.SECONDS, Map.empty)
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -23,7 +23,6 @@ import java.util
|
|||
import java.util.concurrent._
|
||||
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.api.{KAFKA_0_9_0, KAFKA_2_2_IV0, KAFKA_2_4_IV1}
|
||||
import kafka.cluster.Broker
|
||||
import kafka.common.{GenerateBrokerIdException, InconsistentBrokerIdException, InconsistentBrokerMetadataException, InconsistentClusterIdException}
|
||||
|
|
@ -154,7 +153,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
|
|||
private var _clusterId: String = null
|
||||
private var _brokerTopicStats: BrokerTopicStats = null
|
||||
|
||||
|
||||
def clusterId: String = _clusterId
|
||||
|
||||
// Visible for testing
|
||||
|
|
@ -162,28 +160,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
|
|||
|
||||
private[kafka] def brokerTopicStats = _brokerTopicStats
|
||||
|
||||
newGauge(
|
||||
"BrokerState",
|
||||
new Gauge[Int] {
|
||||
def value = brokerState.currentState
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"ClusterId",
|
||||
new Gauge[String] {
|
||||
def value = clusterId
|
||||
}
|
||||
)
|
||||
|
||||
newGauge(
|
||||
"yammer-metrics-count",
|
||||
new Gauge[Int] {
|
||||
def value = {
|
||||
com.yammer.metrics.Metrics.defaultRegistry.allMetrics.size
|
||||
}
|
||||
}
|
||||
)
|
||||
newGauge("BrokerState", () => brokerState.currentState)
|
||||
newGauge("ClusterId", () => clusterId)
|
||||
newGauge("yammer-metrics-count", () => com.yammer.metrics.Metrics.defaultRegistry.allMetrics.size)
|
||||
|
||||
/**
|
||||
* Start up API for bringing up a single instance of the Kafka server.
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit
|
|||
import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
|
||||
import java.util.concurrent.locks.Lock
|
||||
|
||||
import com.yammer.metrics.core.{Gauge, Meter}
|
||||
import com.yammer.metrics.core.Meter
|
||||
import kafka.api._
|
||||
import kafka.cluster.{BrokerEndPoint, Partition}
|
||||
import kafka.common.RecordValidationException
|
||||
|
|
@ -231,50 +231,17 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
}
|
||||
}
|
||||
|
||||
val replicaSelectorOpt: Option[ReplicaSelector] = createReplicaSelector()
|
||||
// Visible for testing
|
||||
private[server] val replicaSelectorOpt: Option[ReplicaSelector] = createReplicaSelector()
|
||||
|
||||
val leaderCount = newGauge(
|
||||
"LeaderCount",
|
||||
new Gauge[Int] {
|
||||
def value = leaderPartitionsIterator.size
|
||||
}
|
||||
)
|
||||
val partitionCount = newGauge(
|
||||
"PartitionCount",
|
||||
new Gauge[Int] {
|
||||
def value = allPartitions.size
|
||||
}
|
||||
)
|
||||
val offlineReplicaCount = newGauge(
|
||||
"OfflineReplicaCount",
|
||||
new Gauge[Int] {
|
||||
def value = offlinePartitionCount
|
||||
}
|
||||
)
|
||||
val underReplicatedPartitions = newGauge(
|
||||
"UnderReplicatedPartitions",
|
||||
new Gauge[Int] {
|
||||
def value = underReplicatedPartitionCount
|
||||
}
|
||||
)
|
||||
val underMinIsrPartitionCount = newGauge(
|
||||
"UnderMinIsrPartitionCount",
|
||||
new Gauge[Int] {
|
||||
def value = leaderPartitionsIterator.count(_.isUnderMinIsr)
|
||||
}
|
||||
)
|
||||
val atMinIsrPartitionCount = newGauge(
|
||||
"AtMinIsrPartitionCount",
|
||||
new Gauge[Int] {
|
||||
def value = leaderPartitionsIterator.count(_.isAtMinIsr)
|
||||
}
|
||||
)
|
||||
val reassigningPartitions = newGauge(
|
||||
"ReassigningPartitions",
|
||||
new Gauge[Int] {
|
||||
def value = reassigningPartitionsCount
|
||||
}
|
||||
)
|
||||
newGauge("LeaderCount", () => leaderPartitionsIterator.size)
|
||||
// Visible for testing
|
||||
private[kafka] val partitionCount = newGauge("PartitionCount", () => allPartitions.size)
|
||||
newGauge("OfflineReplicaCount", () => offlinePartitionCount)
|
||||
newGauge("UnderReplicatedPartitions", () => underReplicatedPartitionCount)
|
||||
newGauge("UnderMinIsrPartitionCount", () => leaderPartitionsIterator.count(_.isUnderMinIsr))
|
||||
newGauge("AtMinIsrPartitionCount", () => leaderPartitionsIterator.count(_.isAtMinIsr))
|
||||
newGauge("ReassigningPartitions", () => reassigningPartitionsCount)
|
||||
|
||||
def reassigningPartitionsCount: Int = leaderPartitionsIterator.count(_.isReassigning)
|
||||
|
||||
|
|
@ -284,7 +251,7 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
|
||||
def underReplicatedPartitionCount: Int = leaderPartitionsIterator.count(_.isUnderReplicated)
|
||||
|
||||
def startHighWatermarkCheckPointThread() = {
|
||||
def startHighWatermarkCheckPointThread(): Unit = {
|
||||
if (highWatermarkCheckPointThreadStarted.compareAndSet(false, true))
|
||||
scheduler.schedule("highwatermark-checkpoint", checkpointHighWatermarks _, period = config.replicaHighWatermarkCheckpointIntervalMs, unit = TimeUnit.MILLISECONDS)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -19,11 +19,11 @@ package kafka.tools
|
|||
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.time.Duration
|
||||
import java.util.{Collections, Arrays, Properties}
|
||||
import java.util.{Arrays, Collections, Properties}
|
||||
|
||||
import kafka.utils.Exit
|
||||
import org.apache.kafka.clients.admin.NewTopic
|
||||
import org.apache.kafka.clients.{admin, CommonClientConfigs}
|
||||
import org.apache.kafka.clients.admin.{Admin, NewTopic}
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
|
||||
import org.apache.kafka.clients.producer._
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
|
|
@ -171,7 +171,7 @@ object EndToEndLatency {
|
|||
println("Topic \"%s\" does not exist. Will create topic with %d partition(s) and replication factor = %d"
|
||||
.format(topic, defaultNumPartitions, defaultReplicationFactor))
|
||||
|
||||
val adminClient = admin.AdminClient.create(props)
|
||||
val adminClient = Admin.create(props)
|
||||
val newTopic = new NewTopic(topic, defaultNumPartitions, defaultReplicationFactor)
|
||||
try adminClient.createTopics(Collections.singleton(newTopic)).all().get()
|
||||
finally Utils.closeQuietly(adminClient, "AdminClient")
|
||||
|
|
|
|||
|
|
@ -24,7 +24,6 @@ import java.util.concurrent.CountDownLatch
|
|||
import java.util.regex.Pattern
|
||||
import java.util.{Collections, Properties}
|
||||
|
||||
import com.yammer.metrics.core.Gauge
|
||||
import kafka.consumer.BaseConsumerRecord
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.utils._
|
||||
|
|
@ -76,10 +75,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
|
|||
// If a message send failed after retries are exhausted. The offset of the messages will also be removed from
|
||||
// the unacked offset list to avoid offset commit being stuck on that offset. In this case, the offset of that
|
||||
// message was not really acked, but was skipped. This metric records the number of skipped offsets.
|
||||
newGauge("MirrorMaker-numDroppedMessages",
|
||||
new Gauge[Int] {
|
||||
def value = numDroppedMessages.get()
|
||||
})
|
||||
newGauge("MirrorMaker-numDroppedMessages", () => numDroppedMessages.get())
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
|
||||
|
|
|
|||
|
|
@ -223,7 +223,7 @@ object ReplicaVerificationTool extends Logging {
|
|||
private def createAdminClient(brokerUrl: String): Admin = {
|
||||
val props = new Properties()
|
||||
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerUrl)
|
||||
admin.AdminClient.create(props)
|
||||
Admin.create(props)
|
||||
}
|
||||
|
||||
private def initialOffsets(topicPartitions: Seq[TopicPartition], consumerConfig: Properties,
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ import scala.collection.{Seq, mutable}
|
|||
import kafka.cluster.EndPoint
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.{KafkaThread, Utils}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.slf4j.event.Level
|
||||
|
||||
/**
|
||||
|
|
@ -55,27 +55,6 @@ object CoreUtils {
|
|||
def min[A, B >: A](iterable: Iterable[A], ifEmpty: A)(implicit cmp: Ordering[B]): A =
|
||||
if (iterable.isEmpty) ifEmpty else iterable.min(cmp)
|
||||
|
||||
/**
|
||||
* Wrap the given function in a java.lang.Runnable
|
||||
* @param fun A function
|
||||
* @return A Runnable that just executes the function
|
||||
*/
|
||||
def runnable(fun: => Unit): Runnable =
|
||||
new Runnable {
|
||||
def run() = fun
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a thread
|
||||
*
|
||||
* @param name The name of the thread
|
||||
* @param daemon Whether the thread should block JVM shutdown
|
||||
* @param fun The function to execute in the thread
|
||||
* @return The unstarted thread
|
||||
*/
|
||||
def newThread(name: String, daemon: Boolean)(fun: => Unit): Thread =
|
||||
new KafkaThread(name, runnable(fun), daemon)
|
||||
|
||||
/**
|
||||
* Do the given action and log any exceptions thrown without rethrowing them.
|
||||
*
|
||||
|
|
@ -142,16 +121,15 @@ object CoreUtils {
|
|||
val mbs = ManagementFactory.getPlatformMBeanServer()
|
||||
mbs synchronized {
|
||||
val objName = new ObjectName(name)
|
||||
if(mbs.isRegistered(objName))
|
||||
if (mbs.isRegistered(objName))
|
||||
mbs.unregisterMBean(objName)
|
||||
mbs.registerMBean(mbean, objName)
|
||||
true
|
||||
}
|
||||
} catch {
|
||||
case e: Exception => {
|
||||
case e: Exception =>
|
||||
logger.error(s"Failed to register Mbean $name", e)
|
||||
false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -163,7 +141,7 @@ object CoreUtils {
|
|||
val mbs = ManagementFactory.getPlatformMBeanServer()
|
||||
mbs synchronized {
|
||||
val objName = new ObjectName(name)
|
||||
if(mbs.isRegistered(objName))
|
||||
if (mbs.isRegistered(objName))
|
||||
mbs.unregisterMBean(objName)
|
||||
}
|
||||
}
|
||||
|
|
@ -175,7 +153,7 @@ object CoreUtils {
|
|||
def read(channel: ReadableByteChannel, buffer: ByteBuffer): Int = {
|
||||
channel.read(buffer) match {
|
||||
case -1 => throw new EOFException("Received -1 when reading from channel, socket has likely been closed.")
|
||||
case n: Int => n
|
||||
case n => n
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -201,11 +179,10 @@ object CoreUtils {
|
|||
* Whitespace surrounding the comma will be removed.
|
||||
*/
|
||||
def parseCsvList(csvList: String): Seq[String] = {
|
||||
if(csvList == null || csvList.isEmpty)
|
||||
if (csvList == null || csvList.isEmpty)
|
||||
Seq.empty[String]
|
||||
else {
|
||||
else
|
||||
csvList.split("\\s*,\\s*").filter(v => !v.equals(""))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -321,9 +298,8 @@ object CoreUtils {
|
|||
* may be invoked more than once if multiple threads attempt to insert a key at the same
|
||||
* time, but the same inserted value will be returned to all threads.
|
||||
*
|
||||
* In Scala 2.12, `ConcurrentMap.getOrElse` has the same behaviour as this method, but that
|
||||
* is not the case in Scala 2.11. We can remove this method once we drop support for Scala
|
||||
* 2.11.
|
||||
* In Scala 2.12, `ConcurrentMap.getOrElse` has the same behaviour as this method, but JConcurrentMapWrapper that
|
||||
* wraps Java maps does not.
|
||||
*/
|
||||
def atomicGetOrUpdate[K, V](map: concurrent.Map[K, V], key: K, createValue: => V): V = {
|
||||
map.get(key) match {
|
||||
|
|
|
|||
|
|
@ -81,10 +81,8 @@ class KafkaScheduler(val threads: Int,
|
|||
executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false)
|
||||
executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false)
|
||||
executor.setRemoveOnCancelPolicy(true)
|
||||
executor.setThreadFactory(new ThreadFactory() {
|
||||
def newThread(runnable: Runnable): Thread =
|
||||
new KafkaThread(threadNamePrefix + schedulerThreadId.getAndIncrement(), runnable, daemon)
|
||||
})
|
||||
executor.setThreadFactory(runnable =>
|
||||
new KafkaThread(threadNamePrefix + schedulerThreadId.getAndIncrement(), runnable, daemon))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -110,7 +108,7 @@ class KafkaScheduler(val threads: Int,
|
|||
.format(name, TimeUnit.MILLISECONDS.convert(delay, unit), TimeUnit.MILLISECONDS.convert(period, unit)))
|
||||
this synchronized {
|
||||
ensureRunning()
|
||||
val runnable = CoreUtils.runnable {
|
||||
val runnable: Runnable = () => {
|
||||
try {
|
||||
trace("Beginning execution of scheduled task '%s'.".format(name))
|
||||
fun()
|
||||
|
|
@ -120,7 +118,7 @@ class KafkaScheduler(val threads: Int,
|
|||
trace("Completed execution of scheduled task '%s'.".format(name))
|
||||
}
|
||||
}
|
||||
if(period >= 0)
|
||||
if (period >= 0)
|
||||
executor.scheduleAtFixedRate(runnable, delay, period, unit)
|
||||
else
|
||||
executor.schedule(runnable, delay, unit)
|
||||
|
|
|
|||
|
|
@ -59,9 +59,7 @@ class Pool[K,V](valueFactory: Option[K => V] = None) extends Iterable[(K, V)] {
|
|||
* @return The final value associated with the key.
|
||||
*/
|
||||
def getAndMaybePut(key: K, createValue: => V): V =
|
||||
pool.computeIfAbsent(key, new java.util.function.Function[K, V] {
|
||||
override def apply(k: K): V = createValue
|
||||
})
|
||||
pool.computeIfAbsent(key, _ => createValue)
|
||||
|
||||
def contains(id: K): Boolean = pool.containsKey(id)
|
||||
|
||||
|
|
|
|||
|
|
@ -81,27 +81,21 @@ object DecodeJson {
|
|||
if (node.isTextual) Right(node.textValue) else Left(s"Expected `String` value, received $node")
|
||||
}
|
||||
|
||||
implicit def decodeOption[E](implicit decodeJson: DecodeJson[E]): DecodeJson[Option[E]] = new DecodeJson[Option[E]] {
|
||||
def decodeEither(node: JsonNode): Either[String, Option[E]] = {
|
||||
if (node.isNull) Right(None)
|
||||
else decodeJson.decodeEither(node).right.map(Some(_))
|
||||
}
|
||||
implicit def decodeOption[E](implicit decodeJson: DecodeJson[E]): DecodeJson[Option[E]] = (node: JsonNode) => {
|
||||
if (node.isNull) Right(None)
|
||||
else decodeJson.decodeEither(node).right.map(Some(_))
|
||||
}
|
||||
|
||||
implicit def decodeSeq[E, S[+T] <: Seq[E]](implicit decodeJson: DecodeJson[E], factory: Factory[E, S[E]]): DecodeJson[S[E]] = new DecodeJson[S[E]] {
|
||||
def decodeEither(node: JsonNode): Either[String, S[E]] = {
|
||||
if (node.isArray)
|
||||
decodeIterator(node.elements.asScala)(decodeJson.decodeEither)
|
||||
else Left(s"Expected JSON array, received $node")
|
||||
}
|
||||
implicit def decodeSeq[E, S[+T] <: Seq[E]](implicit decodeJson: DecodeJson[E], factory: Factory[E, S[E]]): DecodeJson[S[E]] = (node: JsonNode) => {
|
||||
if (node.isArray)
|
||||
decodeIterator(node.elements.asScala)(decodeJson.decodeEither)
|
||||
else Left(s"Expected JSON array, received $node")
|
||||
}
|
||||
|
||||
implicit def decodeMap[V, M[K, +V] <: Map[K, V]](implicit decodeJson: DecodeJson[V], factory: Factory[(String, V), M[String, V]]): DecodeJson[M[String, V]] = new DecodeJson[M[String, V]] {
|
||||
def decodeEither(node: JsonNode): Either[String, M[String, V]] = {
|
||||
if (node.isObject)
|
||||
decodeIterator(node.fields.asScala)(e => decodeJson.decodeEither(e.getValue).right.map(v => (e.getKey, v)))
|
||||
else Left(s"Expected JSON object, received $node")
|
||||
}
|
||||
implicit def decodeMap[V, M[K, +V] <: Map[K, V]](implicit decodeJson: DecodeJson[V], factory: Factory[(String, V), M[String, V]]): DecodeJson[M[String, V]] = (node: JsonNode) => {
|
||||
if (node.isObject)
|
||||
decodeIterator(node.fields.asScala)(e => decodeJson.decodeEither(e.getValue).right.map(v => (e.getKey, v)))
|
||||
else Left(s"Expected JSON object, received $node")
|
||||
}
|
||||
|
||||
private def decodeIterator[S, T, C](it: Iterator[S])(f: S => Either[String, T])(implicit factory: Factory[T, C]): Either[String, C] = {
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@
|
|||
*/
|
||||
package kafka.utils.timer
|
||||
|
||||
import java.util.concurrent.{DelayQueue, Executors, ThreadFactory, TimeUnit}
|
||||
import java.util.concurrent.{DelayQueue, Executors, TimeUnit}
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock
|
||||
|
||||
|
|
@ -58,10 +58,8 @@ class SystemTimer(executorName: String,
|
|||
startMs: Long = Time.SYSTEM.hiResClockMs) extends Timer {
|
||||
|
||||
// timeout timer
|
||||
private[this] val taskExecutor = Executors.newFixedThreadPool(1, new ThreadFactory() {
|
||||
def newThread(runnable: Runnable): Thread =
|
||||
KafkaThread.nonDaemon("executor-"+executorName, runnable)
|
||||
})
|
||||
private[this] val taskExecutor = Executors.newFixedThreadPool(1,
|
||||
(runnable: Runnable) => KafkaThread.nonDaemon("executor-" + executorName, runnable))
|
||||
|
||||
private[this] val delayQueue = new DelayQueue[TimerTaskList]()
|
||||
private[this] val taskCounter = new AtomicInteger(0)
|
||||
|
|
|
|||
|
|
@ -600,19 +600,14 @@ sealed trait ZkAclChangeStore {
|
|||
def createChangeNode(resource: Resource): AclChangeNode = AclChangeNode(createPath, encode(resource))
|
||||
|
||||
def createListener(handler: AclChangeNotificationHandler, zkClient: KafkaZkClient): AclChangeSubscription = {
|
||||
val rawHandler: NotificationHandler = new NotificationHandler {
|
||||
def processNotification(bytes: Array[Byte]): Unit =
|
||||
handler.processNotification(decode(bytes))
|
||||
}
|
||||
val rawHandler: NotificationHandler = (bytes: Array[Byte]) => handler.processNotification(decode(bytes))
|
||||
|
||||
val aclChangeListener = new ZkNodeChangeNotificationListener(
|
||||
zkClient, aclChangePath, ZkAclChangeStore.SequenceNumberPrefix, rawHandler)
|
||||
|
||||
aclChangeListener.init()
|
||||
|
||||
new AclChangeSubscription {
|
||||
def close(): Unit = aclChangeListener.close()
|
||||
}
|
||||
() => aclChangeListener.close()
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -17,17 +17,17 @@
|
|||
|
||||
package kafka.zookeeper
|
||||
|
||||
import java.util
|
||||
import java.util.Locale
|
||||
import java.util.concurrent.locks.{ReentrantLock, ReentrantReadWriteLock}
|
||||
import java.util.concurrent._
|
||||
import java.util.{List => JList}
|
||||
|
||||
import com.yammer.metrics.core.{Gauge, MetricName}
|
||||
import com.yammer.metrics.core.MetricName
|
||||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.utils.CoreUtils.{inLock, inReadLock, inWriteLock}
|
||||
import kafka.utils.{KafkaScheduler, Logging}
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.zookeeper.AsyncCallback._
|
||||
import org.apache.zookeeper.AsyncCallback.{Children2Callback, DataCallback, StatCallback}
|
||||
import org.apache.zookeeper.KeeperException.Code
|
||||
import org.apache.zookeeper.Watcher.Event.{EventType, KeeperState}
|
||||
import org.apache.zookeeper.ZooKeeper.States
|
||||
|
|
@ -103,9 +103,7 @@ class ZooKeeperClient(connectString: String,
|
|||
// Fail-fast if there's an error during construction (so don't call initialize, which retries forever)
|
||||
@volatile private var zooKeeper = new ZooKeeper(connectString, sessionTimeoutMs, ZooKeeperClientWatcher)
|
||||
|
||||
newGauge("SessionState", new Gauge[String] {
|
||||
override def value: String = Option(connectionState.toString).getOrElse("DISCONNECTED")
|
||||
})
|
||||
newGauge("SessionState", () => connectionState.toString)
|
||||
|
||||
metricNames += "SessionState"
|
||||
|
||||
|
|
@ -186,57 +184,54 @@ class ZooKeeperClient(connectString: String,
|
|||
request match {
|
||||
case ExistsRequest(path, ctx) =>
|
||||
zooKeeper.exists(path, shouldWatch(request), new StatCallback {
|
||||
override def processResult(rc: Int, path: String, ctx: Any, stat: Stat): Unit =
|
||||
def processResult(rc: Int, path: String, ctx: Any, stat: Stat): Unit =
|
||||
callback(ExistsResponse(Code.get(rc), path, Option(ctx), stat, responseMetadata(sendTimeMs)))
|
||||
}, ctx.orNull)
|
||||
case GetDataRequest(path, ctx) =>
|
||||
zooKeeper.getData(path, shouldWatch(request), new DataCallback {
|
||||
override def processResult(rc: Int, path: String, ctx: Any, data: Array[Byte], stat: Stat): Unit =
|
||||
callback(GetDataResponse(Code.get(rc), path, Option(ctx), data, stat, responseMetadata(sendTimeMs)))
|
||||
def processResult(rc: Int, path: String, ctx: Any, data: Array[Byte], stat: Stat): Unit =
|
||||
callback(GetDataResponse(Code.get(rc), path, Option(ctx), data, stat, responseMetadata(sendTimeMs))),
|
||||
}, ctx.orNull)
|
||||
case GetChildrenRequest(path, ctx) =>
|
||||
zooKeeper.getChildren(path, shouldWatch(request), new Children2Callback {
|
||||
override def processResult(rc: Int, path: String, ctx: Any, children: java.util.List[String], stat: Stat): Unit =
|
||||
callback(GetChildrenResponse(Code.get(rc), path, Option(ctx),
|
||||
Option(children).map(_.asScala).getOrElse(Seq.empty), stat, responseMetadata(sendTimeMs)))
|
||||
def processResult(rc: Int, path: String, ctx: Any, children: JList[String], stat: Stat): Unit =
|
||||
callback(GetChildrenResponse(Code.get(rc), path, Option(ctx), Option(children).map(_.asScala).getOrElse(Seq.empty),
|
||||
stat, responseMetadata(sendTimeMs)))
|
||||
}, ctx.orNull)
|
||||
case CreateRequest(path, data, acl, createMode, ctx) =>
|
||||
zooKeeper.create(path, data, acl.asJava, createMode, new StringCallback {
|
||||
override def processResult(rc: Int, path: String, ctx: Any, name: String): Unit =
|
||||
callback(CreateResponse(Code.get(rc), path, Option(ctx), name, responseMetadata(sendTimeMs)))
|
||||
}, ctx.orNull)
|
||||
zooKeeper.create(path, data, acl.asJava, createMode,
|
||||
(rc, path, ctx, name) =>
|
||||
callback(CreateResponse(Code.get(rc), path, Option(ctx), name, responseMetadata(sendTimeMs))),
|
||||
ctx.orNull)
|
||||
case SetDataRequest(path, data, version, ctx) =>
|
||||
zooKeeper.setData(path, data, version, new StatCallback {
|
||||
override def processResult(rc: Int, path: String, ctx: Any, stat: Stat): Unit =
|
||||
callback(SetDataResponse(Code.get(rc), path, Option(ctx), stat, responseMetadata(sendTimeMs)))
|
||||
}, ctx.orNull)
|
||||
zooKeeper.setData(path, data, version,
|
||||
(rc, path, ctx, stat) =>
|
||||
callback(SetDataResponse(Code.get(rc), path, Option(ctx), stat, responseMetadata(sendTimeMs))),
|
||||
ctx.orNull)
|
||||
case DeleteRequest(path, version, ctx) =>
|
||||
zooKeeper.delete(path, version, new VoidCallback {
|
||||
override def processResult(rc: Int, path: String, ctx: Any): Unit =
|
||||
callback(DeleteResponse(Code.get(rc), path, Option(ctx), responseMetadata(sendTimeMs)))
|
||||
}, ctx.orNull)
|
||||
zooKeeper.delete(path, version,
|
||||
(rc, path, ctx) => callback(DeleteResponse(Code.get(rc), path, Option(ctx), responseMetadata(sendTimeMs))),
|
||||
ctx.orNull)
|
||||
case GetAclRequest(path, ctx) =>
|
||||
zooKeeper.getACL(path, null, new ACLCallback {
|
||||
override def processResult(rc: Int, path: String, ctx: Any, acl: java.util.List[ACL], stat: Stat): Unit = {
|
||||
zooKeeper.getACL(path, null,
|
||||
(rc, path, ctx, acl, stat) =>
|
||||
callback(GetAclResponse(Code.get(rc), path, Option(ctx), Option(acl).map(_.asScala).getOrElse(Seq.empty),
|
||||
stat, responseMetadata(sendTimeMs)))
|
||||
}}, ctx.orNull)
|
||||
stat, responseMetadata(sendTimeMs))),
|
||||
ctx.orNull)
|
||||
case SetAclRequest(path, acl, version, ctx) =>
|
||||
zooKeeper.setACL(path, acl.asJava, version, new StatCallback {
|
||||
override def processResult(rc: Int, path: String, ctx: Any, stat: Stat): Unit =
|
||||
callback(SetAclResponse(Code.get(rc), path, Option(ctx), stat, responseMetadata(sendTimeMs)))
|
||||
}, ctx.orNull)
|
||||
zooKeeper.setACL(path, acl.asJava, version,
|
||||
(rc, path, ctx, stat) =>
|
||||
callback(SetAclResponse(Code.get(rc), path, Option(ctx), stat, responseMetadata(sendTimeMs))),
|
||||
ctx.orNull)
|
||||
case MultiRequest(zkOps, ctx) =>
|
||||
zooKeeper.multi(zkOps.map(_.toZookeeperOp).asJava, new MultiCallback {
|
||||
override def processResult(rc: Int, path: String, ctx: Any, opResults: util.List[OpResult]): Unit = {
|
||||
callback(MultiResponse(Code.get(rc), path, Option(ctx),
|
||||
if (opResults == null)
|
||||
null
|
||||
else
|
||||
zkOps.zip(opResults.asScala) map { case (zkOp, result) => ZkOpResult(zkOp, result) },
|
||||
responseMetadata(sendTimeMs)))
|
||||
}
|
||||
}, ctx.orNull)
|
||||
def toZkOpResult(opResults: JList[OpResult]): Seq[ZkOpResult] =
|
||||
Option(opResults).map(results => zkOps.zip(results.asScala).map { case (zkOp, result) =>
|
||||
ZkOpResult(zkOp, result)
|
||||
}).orNull
|
||||
zooKeeper.multi(zkOps.map(_.toZookeeperOp).asJava,
|
||||
(rc, path, ctx, opResults) =>
|
||||
callback(MultiResponse(Code.get(rc), path, Option(ctx), toZkOpResult(opResults), responseMetadata(sendTimeMs))),
|
||||
ctx.orNull)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ import kafka.integration.KafkaServerTestHarness
|
|||
import kafka.log.LogConfig
|
||||
import kafka.server.{Defaults, KafkaConfig}
|
||||
import kafka.utils.{Logging, TestUtils}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig, AlterConfigsOptions, Config, ConfigEntry}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigsOptions, Config, ConfigEntry}
|
||||
import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
|
||||
import org.apache.kafka.common.errors.{InvalidRequestException, PolicyViolationException}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
|
|
@ -70,7 +70,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
|
|||
|
||||
@Test
|
||||
def testValidAlterConfigs(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
// Create topics
|
||||
val topic1 = "describe-alter-configs-topic-1"
|
||||
val topicResource1 = new ConfigResource(ConfigResource.Type.TOPIC, topic1)
|
||||
|
|
@ -88,13 +88,13 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
|
|||
|
||||
@Test
|
||||
def testInvalidAlterConfigs(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
PlaintextAdminIntegrationTest.checkInvalidAlterConfigs(zkClient, servers, client)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testInvalidAlterConfigsDueToPolicy(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
// Create topics
|
||||
val topic1 = "invalid-alter-configs-due-to-policy-topic-1"
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ import kafka.security.auth.{SimpleAclAuthorizer, Topic, ResourceType => AuthReso
|
|||
import kafka.security.authorizer.AuthorizerUtils.WildcardHost
|
||||
import kafka.server.{BaseRequestTest, KafkaConfig}
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig, AlterConfigOp}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp}
|
||||
import org.apache.kafka.clients.consumer._
|
||||
import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener
|
||||
import org.apache.kafka.clients.producer._
|
||||
|
|
@ -1734,7 +1734,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
|
|||
private def createAdminClient(): Admin = {
|
||||
val props = new Properties()
|
||||
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
|
||||
val adminClient = AdminClient.create(props)
|
||||
val adminClient = Admin.create(props)
|
||||
adminClients += adminClient
|
||||
adminClient
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ import kafka.security.auth.{Cluster, Topic}
|
|||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.Logging
|
||||
import kafka.utils.TestUtils._
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig, CreateTopicsOptions, CreateTopicsResult, DescribeClusterOptions, DescribeTopicsOptions, NewTopic, TopicDescription}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateTopicsOptions, CreateTopicsResult, DescribeClusterOptions, DescribeTopicsOptions, NewTopic, TopicDescription}
|
||||
import org.apache.kafka.common.acl.AclOperation
|
||||
import org.apache.kafka.common.errors.{TopicExistsException, UnknownTopicOrPartitionException}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
|
|
@ -67,7 +67,7 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg
|
|||
|
||||
@Test
|
||||
def testCreateDeleteTopics(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val topics = Seq("mytopic", "mytopic2", "mytopic3")
|
||||
val newTopics = Seq(
|
||||
new NewTopic("mytopic", Map((0: Integer) -> Seq[Integer](1, 2).asJava, (1: Integer) -> Seq[Integer](2, 0).asJava).asJava),
|
||||
|
|
@ -154,7 +154,7 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg
|
|||
|
||||
@Test
|
||||
def testAuthorizedOperations(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
// without includeAuthorizedOperations flag
|
||||
var result = client.describeCluster
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ import java.util.concurrent._
|
|||
import java.util.{Collection, Collections, Properties}
|
||||
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.{CoreUtils, Logging, ShutdownableThread, TestUtils}
|
||||
import kafka.utils.{Logging, ShutdownableThread, TestUtils}
|
||||
import org.apache.kafka.clients.consumer._
|
||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
|
|
@ -382,7 +382,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
|
|||
private def checkCloseDuringRebalance(groupId: String, topic: String, executor: ExecutorService, brokersAvailableDuringClose: Boolean): Unit = {
|
||||
|
||||
def subscribeAndPoll(consumer: KafkaConsumer[Array[Byte], Array[Byte]], revokeSemaphore: Option[Semaphore] = None): Future[Any] = {
|
||||
executor.submit(CoreUtils.runnable {
|
||||
executor.submit(() => {
|
||||
consumer.subscribe(Collections.singletonList(topic))
|
||||
revokeSemaphore.foreach(s => s.release())
|
||||
// requires to used deprecated `poll(long)` to trigger metadata update
|
||||
|
|
@ -452,7 +452,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
|
|||
|
||||
private def submitCloseAndValidate(consumer: KafkaConsumer[Array[Byte], Array[Byte]],
|
||||
closeTimeoutMs: Long, minCloseTimeMs: Option[Long], maxCloseTimeMs: Option[Long]): Future[Any] = {
|
||||
executor.submit(CoreUtils.runnable {
|
||||
executor.submit(() => {
|
||||
val closeGraceTimeMs = 2000
|
||||
val startMs = System.currentTimeMillis()
|
||||
info("Closing consumer with timeout " + closeTimeoutMs + " ms.")
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ import kafka.server._
|
|||
import kafka.utils.JaasTestUtils.ScramLoginModule
|
||||
import kafka.utils.{JaasTestUtils, Logging, TestUtils}
|
||||
import kafka.zk.ConfigEntityChangeNotificationZNode
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
|
||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
|
||||
import org.apache.kafka.common.{Cluster, Reconfigurable}
|
||||
|
|
@ -190,7 +190,7 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup {
|
|||
}
|
||||
config.put(SaslConfigs.SASL_JAAS_CONFIG,
|
||||
ScramLoginModule(JaasTestUtils.KafkaScramAdmin, JaasTestUtils.KafkaScramAdminPassword).toString)
|
||||
val adminClient = AdminClient.create(config)
|
||||
val adminClient = Admin.create(config)
|
||||
adminClients += adminClient
|
||||
adminClient
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ import java.util
|
|||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.{JaasTestUtils, TestUtils}
|
||||
import kafka.zk.ConfigEntityChangeNotificationZNode
|
||||
import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.security.scram.ScramCredential
|
||||
|
|
@ -93,7 +93,7 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest
|
|||
val clientLoginContext = jaasClientLoginModule(kafkaClientSaslMechanism)
|
||||
config.put(SaslConfigs.SASL_JAAS_CONFIG, clientLoginContext)
|
||||
|
||||
val adminClient = AdminClient.create(config)
|
||||
val adminClient = Admin.create(config)
|
||||
try {
|
||||
val token = adminClient.createDelegationToken().delegationToken().get()
|
||||
//wait for token to reach all the brokers
|
||||
|
|
|
|||
|
|
@ -117,7 +117,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS
|
|||
|
||||
@Test
|
||||
def testConsumerGroupAuthorizedOperations(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
val results = client.createAcls(List(group1Acl, group2Acl, group3Acl).asJava)
|
||||
assertEquals(Set(group1Acl, group2Acl, group3Acl), results.values.keySet.asScala)
|
||||
|
|
@ -141,7 +141,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS
|
|||
|
||||
@Test
|
||||
def testClusterAuthorizedOperations(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
// test without includeAuthorizedOperations flag
|
||||
var clusterDescribeResult = client.describeCluster()
|
||||
|
|
@ -169,7 +169,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS
|
|||
|
||||
@Test
|
||||
def testTopicAuthorizedOperations(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
createTopic(topic1)
|
||||
createTopic(topic2)
|
||||
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ import java.util.Properties
|
|||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig}
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.integration.KafkaServerTestHarness
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.network.{ListenerName, Mode}
|
||||
import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, Serializer}
|
||||
import org.junit.{After, Before}
|
||||
|
|
@ -142,7 +142,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
|
|||
val props = new Properties
|
||||
props ++= adminClientConfig
|
||||
props ++= configOverrides
|
||||
val adminClient = AdminClient.create(props)
|
||||
val adminClient = Admin.create(props)
|
||||
adminClients += adminClient
|
||||
adminClient
|
||||
}
|
||||
|
|
|
|||
|
|
@ -81,14 +81,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testClose(): Unit = {
|
||||
val client = AdminClient.create(createConfig())
|
||||
val client = Admin.create(createConfig())
|
||||
client.close()
|
||||
client.close() // double close has no effect
|
||||
}
|
||||
|
||||
@Test
|
||||
def testListNodes(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val brokerStrs = brokerList.split(",").toList.sorted
|
||||
var nodeStrs: List[String] = null
|
||||
do {
|
||||
|
|
@ -100,7 +100,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testCreateExistingTopicsThrowTopicExistsException(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val topic = "mytopic"
|
||||
val topics = Seq(topic)
|
||||
val newTopics = Seq(new NewTopic(topic, 1, 1.toShort))
|
||||
|
|
@ -117,7 +117,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testMetadataRefresh(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val topics = Seq("mytopic")
|
||||
val newTopics = Seq(new NewTopic("mytopic", 3, 3.toShort))
|
||||
client.createTopics(newTopics.asJava).all.get()
|
||||
|
|
@ -135,7 +135,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
*/
|
||||
@Test
|
||||
def testDescribeNonExistingTopic(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
val existingTopic = "existing-topic"
|
||||
client.createTopics(Seq(existingTopic).map(new NewTopic(_, 1, 1.toShort)).asJava).all.get()
|
||||
|
|
@ -150,7 +150,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testDescribeCluster(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val result = client.describeCluster
|
||||
val nodes = result.nodes.get()
|
||||
val clusterId = result.clusterId().get()
|
||||
|
|
@ -168,7 +168,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testDescribeLogDirs(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val topic = "topic"
|
||||
val leaderByPartition = createTopic(topic, numPartitions = 10, replicationFactor = 1)
|
||||
val partitionsByBroker = leaderByPartition.groupBy { case (_, leaderId) => leaderId }.mapValues(_.keys.toSeq)
|
||||
|
|
@ -192,7 +192,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testDescribeReplicaLogDirs(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val topic = "topic"
|
||||
val leaderByPartition = createTopic(topic, numPartitions = 10, replicationFactor = 1)
|
||||
val replicas = leaderByPartition.map { case (partition, brokerId) =>
|
||||
|
|
@ -209,7 +209,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testAlterReplicaLogDirs(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val topic = "topic"
|
||||
val tp = new TopicPartition(topic, 0)
|
||||
val randomNums = servers.map(server => server -> Random.nextInt(2)).toMap
|
||||
|
|
@ -297,7 +297,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testDescribeAndAlterConfigs(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
// Create topics
|
||||
val topic1 = "describe-alter-configs-topic-1"
|
||||
|
|
@ -368,7 +368,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testCreatePartitions(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
// Create topics
|
||||
val topic1 = "create-partitions-topic-1"
|
||||
|
|
@ -386,7 +386,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
val validateOnly = new CreatePartitionsOptions().validateOnly(true)
|
||||
val actuallyDoIt = new CreatePartitionsOptions().validateOnly(false)
|
||||
|
||||
def partitions(topic: String, expectedNumPartitionsOpt: Option[Int] = None): util.List[TopicPartitionInfo] = {
|
||||
def partitions(topic: String, expectedNumPartitionsOpt: Option[Int]): util.List[TopicPartitionInfo] = {
|
||||
getTopicMetadata(client, topic, expectedNumPartitionsOpt = expectedNumPartitionsOpt).partitions
|
||||
}
|
||||
|
||||
|
|
@ -633,7 +633,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
def testSeekAfterDeleteRecords(): Unit = {
|
||||
createTopic(topic, numPartitions = 2, replicationFactor = brokerCount)
|
||||
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val consumer = createConsumer()
|
||||
subscribeAndWaitForAssignment(topic, consumer)
|
||||
|
|
@ -662,7 +662,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
def testLogStartOffsetCheckpoint(): Unit = {
|
||||
createTopic(topic, numPartitions = 2, replicationFactor = brokerCount)
|
||||
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val consumer = createConsumer()
|
||||
subscribeAndWaitForAssignment(topic, consumer)
|
||||
|
|
@ -680,7 +680,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
client.close()
|
||||
brokerList = TestUtils.bootstrapServers(servers, listenerName)
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
TestUtils.waitUntilTrue(() => {
|
||||
// Need to retry if leader is not available for the partition
|
||||
|
|
@ -702,7 +702,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
def testLogStartOffsetAfterDeleteRecords(): Unit = {
|
||||
createTopic(topic, numPartitions = 2, replicationFactor = brokerCount)
|
||||
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val consumer = createConsumer()
|
||||
subscribeAndWaitForAssignment(topic, consumer)
|
||||
|
|
@ -740,7 +740,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
// we will produce to topic and delete records while one follower is down
|
||||
killBroker(followerIndex)
|
||||
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
val producer = createProducer()
|
||||
sendRecords(producer, 100, topicPartition)
|
||||
|
||||
|
|
@ -767,7 +767,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testAlterLogDirsAfterDeleteRecords(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
createTopic(topic, numPartitions = 1, replicationFactor = brokerCount)
|
||||
val expectedLEO = 100
|
||||
val producer = createProducer()
|
||||
|
|
@ -801,7 +801,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
def testOffsetsForTimesAfterDeleteRecords(): Unit = {
|
||||
createTopic(topic, numPartitions = 2, replicationFactor = brokerCount)
|
||||
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val consumer = createConsumer()
|
||||
subscribeAndWaitForAssignment(topic, consumer)
|
||||
|
|
@ -824,7 +824,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
val consumer = createConsumer()
|
||||
subscribeAndWaitForAssignment(topic, consumer)
|
||||
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val producer = createProducer()
|
||||
sendRecords(producer, 10, topicPartition)
|
||||
|
|
@ -847,7 +847,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
val consumer = createConsumer()
|
||||
subscribeAndWaitForAssignment(topic, consumer)
|
||||
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val producer = createProducer()
|
||||
sendRecords(producer, 10, topicPartition)
|
||||
|
|
@ -872,7 +872,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testDescribeConfigsForTopic(): Unit = {
|
||||
createTopic(topic, numPartitions = 2, replicationFactor = brokerCount)
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val existingTopic = new ConfigResource(ConfigResource.Type.TOPIC, topic)
|
||||
client.describeConfigs(Collections.singletonList(existingTopic)).values.get(existingTopic).get()
|
||||
|
|
@ -907,7 +907,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testInvalidAlterConfigs(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
checkInvalidAlterConfigs(zkClient, servers, client)
|
||||
}
|
||||
|
||||
|
|
@ -920,7 +920,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
def testAclOperations(): Unit = {
|
||||
val acl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL),
|
||||
new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW))
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
assertFutureExceptionTypeEquals(client.describeAcls(AclBindingFilter.ANY).values(), classOf[SecurityDisabledException])
|
||||
assertFutureExceptionTypeEquals(client.createAcls(Collections.singleton(acl)).all(),
|
||||
classOf[SecurityDisabledException])
|
||||
|
|
@ -934,7 +934,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
*/
|
||||
@Test
|
||||
def testDelayedClose(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val topics = Seq("mytopic", "mytopic2")
|
||||
val newTopics = topics.map(new NewTopic(_, 1, 1.toShort))
|
||||
val future = client.createTopics(newTopics.asJava, new CreateTopicsOptions().validateOnly(true)).all()
|
||||
|
|
@ -953,7 +953,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
def testForceClose(): Unit = {
|
||||
val config = createConfig()
|
||||
config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, s"localhost:${TestUtils.IncorrectBrokerPort}")
|
||||
client = AdminClient.create(config)
|
||||
client = Admin.create(config)
|
||||
// Because the bootstrap servers are set up incorrectly, this call will not complete, but must be
|
||||
// cancelled by the close operation.
|
||||
val future = client.createTopics(Seq("mytopic", "mytopic2").map(new NewTopic(_, 1, 1.toShort)).asJava,
|
||||
|
|
@ -971,7 +971,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
val config = createConfig()
|
||||
config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, s"localhost:${TestUtils.IncorrectBrokerPort}")
|
||||
config.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "0")
|
||||
client = AdminClient.create(config)
|
||||
client = Admin.create(config)
|
||||
val startTimeMs = Time.SYSTEM.milliseconds()
|
||||
val future = client.createTopics(Seq("mytopic", "mytopic2").map(new NewTopic(_, 1, 1.toShort)).asJava,
|
||||
new CreateTopicsOptions().timeoutMs(2)).all()
|
||||
|
|
@ -1004,7 +1004,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testConsumerGroups(): Unit = {
|
||||
val config = createConfig()
|
||||
client = AdminClient.create(config)
|
||||
client = Admin.create(config)
|
||||
try {
|
||||
// Verify that initially there are no consumer groups to list.
|
||||
val list1 = client.listConsumerGroups()
|
||||
|
|
@ -1166,7 +1166,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testDeleteConsumerGroupOffsets(): Unit = {
|
||||
val config = createConfig()
|
||||
client = AdminClient.create(config)
|
||||
client = Admin.create(config)
|
||||
try {
|
||||
val testTopicName = "test_topic"
|
||||
val testGroupId = "test_group_id"
|
||||
|
|
@ -1237,7 +1237,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testElectPreferredLeaders(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val prefer0 = Seq(0, 1, 2)
|
||||
val prefer1 = Seq(1, 2, 0)
|
||||
|
|
@ -1378,7 +1378,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testElectUncleanLeadersForOnePartition(): Unit = {
|
||||
// Case: unclean leader election with one topic partition
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val broker1 = 1
|
||||
val broker2 = 2
|
||||
|
|
@ -1403,7 +1403,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testElectUncleanLeadersForManyPartitions(): Unit = {
|
||||
// Case: unclean leader election with many topic partitions
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val broker1 = 1
|
||||
val broker2 = 2
|
||||
|
|
@ -1441,7 +1441,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testElectUncleanLeadersForAllPartitions(): Unit = {
|
||||
// Case: noop unclean leader election and valid unclean leader election for all partitions
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val broker1 = 1
|
||||
val broker2 = 2
|
||||
|
|
@ -1480,7 +1480,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testElectUncleanLeadersForUnknownPartitions(): Unit = {
|
||||
// Case: unclean leader election for unknown topic
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val broker1 = 1
|
||||
val broker2 = 2
|
||||
|
|
@ -1507,7 +1507,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testElectUncleanLeadersWhenNoLiveBrokers(): Unit = {
|
||||
// Case: unclean leader election with no live brokers
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val broker1 = 1
|
||||
val broker2 = 2
|
||||
|
|
@ -1537,7 +1537,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testElectUncleanLeadersNoop(): Unit = {
|
||||
// Case: noop unclean leader election with explicit topic partitions
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val broker1 = 1
|
||||
val broker2 = 2
|
||||
|
|
@ -1566,7 +1566,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
def testElectUncleanLeadersAndNoop(): Unit = {
|
||||
// Case: one noop unclean leader election and one valid unclean leader election
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
val broker1 = 1
|
||||
val broker2 = 2
|
||||
|
|
@ -1604,7 +1604,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testListReassignmentsDoesNotShowNonReassigningPartitions(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
// Create topics
|
||||
val topic = "list-reassignments-no-reassignments"
|
||||
|
|
@ -1620,7 +1620,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testListReassignmentsDoesNotShowDeletedPartitions(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
val topic = "list-reassignments-no-reassignments"
|
||||
val tp = new TopicPartition(topic, 0)
|
||||
|
|
@ -1634,7 +1634,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testValidIncrementalAlterConfigs(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
// Create topics
|
||||
val topic1 = "incremental-alter-configs-topic-1"
|
||||
|
|
@ -1729,7 +1729,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testInvalidIncrementalAlterConfigs(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
|
||||
// Create topics
|
||||
val topic1 = "incremental-alter-configs-topic-1"
|
||||
|
|
@ -1812,7 +1812,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testInvalidAlterPartitionReassignments(): Unit = {
|
||||
client = AdminClient.create(createConfig)
|
||||
client = Admin.create(createConfig)
|
||||
val topic = "alter-reassignments-topic-1"
|
||||
val tp1 = new TopicPartition(topic, 0)
|
||||
val tp2 = new TopicPartition(topic, 1)
|
||||
|
|
@ -1851,7 +1851,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testLongTopicNames(): Unit = {
|
||||
val client = AdminClient.create(createConfig)
|
||||
val client = Admin.create(createConfig)
|
||||
val longTopicName = String.join("", Collections.nCopies(249, "x"));
|
||||
val invalidTopicName = String.join("", Collections.nCopies(250, "x"));
|
||||
val newTopics2 = Seq(new NewTopic(invalidTopicName, 3, 3.toShort),
|
||||
|
|
@ -1869,7 +1869,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
|
||||
@Test
|
||||
def testDescribeConfigsForLog4jLogLevels(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
val loggerConfig = describeBrokerLoggers()
|
||||
val rootLogLevel = loggerConfig.get(Log4jController.ROOT_LOGGER).value()
|
||||
|
|
@ -1885,7 +1885,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
@Ignore // To be re-enabled once KAFKA-8779 is resolved
|
||||
def testIncrementalAlterConfigsForLog4jLogLevels(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
val initialLoggerConfig = describeBrokerLoggers()
|
||||
val initialRootLogLevel = initialLoggerConfig.get(Log4jController.ROOT_LOGGER).value()
|
||||
|
|
@ -1949,7 +1949,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
@Ignore // To be re-enabled once KAFKA-8779 is resolved
|
||||
def testIncrementalAlterConfigsForLog4jLogLevelsCanResetLoggerToCurrentRoot(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
// step 1 - configure root logger
|
||||
val initialRootLogLevel = LogLevelConfig.TRACE_LOG_LEVEL
|
||||
val alterRootLoggerEntry = Seq(
|
||||
|
|
@ -1991,7 +1991,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
@Ignore // To be re-enabled once KAFKA-8779 is resolved
|
||||
def testIncrementalAlterConfigsForLog4jLogLevelsCannotResetRootLogger(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val deleteRootLoggerEntry = Seq(
|
||||
new AlterConfigOp(new ConfigEntry(Log4jController.ROOT_LOGGER, ""), AlterConfigOp.OpType.DELETE)
|
||||
).asJavaCollection
|
||||
|
|
@ -2002,7 +2002,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
@Ignore // To be re-enabled once KAFKA-8779 is resolved
|
||||
def testIncrementalAlterConfigsForLog4jLogLevelsDoesNotWorkWithInvalidConfigs(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val validLoggerName = "kafka.server.KafkaRequestHandler"
|
||||
val expectedValidLoggerLogLevel = describeBrokerLoggers().get(validLoggerName)
|
||||
def assertLogLevelDidNotChange(): Unit = {
|
||||
|
|
@ -2047,7 +2047,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
|||
@Test
|
||||
@Ignore // To be re-enabled once KAFKA-8779 is resolved
|
||||
def testAlterConfigsForLog4jLogLevelsDoesNotWork(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
val alterLogLevelsEntries = Seq(
|
||||
new ConfigEntry("kafka.controller.KafkaController", LogLevelConfig.INFO_LOG_LEVEL)
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ import java.util.Collections
|
|||
import java.util.concurrent.{ExecutionException, TimeUnit}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
|
||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
|
||||
import org.apache.kafka.common.{KafkaException, TopicPartition}
|
||||
|
|
@ -132,7 +132,7 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with
|
|||
def testKafkaAdminClientWithAuthenticationFailure(): Unit = {
|
||||
val props = TestUtils.adminClientSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties)
|
||||
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
|
||||
val adminClient = AdminClient.create(props)
|
||||
val adminClient = Admin.create(props)
|
||||
|
||||
def describeTopic(): Unit = {
|
||||
try {
|
||||
|
|
|
|||
|
|
@ -118,7 +118,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
|
|||
|
||||
@Test
|
||||
def testAclOperations(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val acl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "mytopic3", PatternType.LITERAL),
|
||||
new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW))
|
||||
assertEquals(7, getAcls(AclBindingFilter.ANY).size)
|
||||
|
|
@ -139,7 +139,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
|
|||
|
||||
@Test
|
||||
def testAclOperations2(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val results = client.createAcls(List(acl2, acl2, transactionalIdAcl).asJava)
|
||||
assertEquals(Set(acl2, acl2, transactionalIdAcl), results.values.keySet.asScala)
|
||||
results.all.get()
|
||||
|
|
@ -165,7 +165,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
|
|||
|
||||
@Test
|
||||
def testAclDescribe(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl))
|
||||
|
||||
val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.ANY), AccessControlEntryFilter.ANY)
|
||||
|
|
@ -192,7 +192,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
|
|||
|
||||
@Test
|
||||
def testAclDelete(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl))
|
||||
|
||||
val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.MATCH), AccessControlEntryFilter.ANY)
|
||||
|
|
@ -242,7 +242,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
|
|||
//noinspection ScalaDeprecation - test explicitly covers clients using legacy / deprecated constructors
|
||||
@Test
|
||||
def testLegacyAclOpsNeverAffectOrReturnPrefixed(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
ensureAcls(Set(anyAcl, acl2, fooAcl, prefixAcl)) // <-- prefixed exists, but should never be returned.
|
||||
|
||||
val allTopicAcls = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, null, PatternType.MATCH), AccessControlEntryFilter.ANY)
|
||||
|
|
@ -279,7 +279,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
|
|||
|
||||
@Test
|
||||
def testAttemptToCreateInvalidAcls(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val clusterAcl = new AclBinding(new ResourcePattern(ResourceType.CLUSTER, "foobar", PatternType.LITERAL),
|
||||
new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.READ, AclPermissionType.ALLOW))
|
||||
val emptyResourceNameAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "", PatternType.LITERAL),
|
||||
|
|
@ -378,7 +378,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
|
|||
|
||||
@Test
|
||||
def testAclAuthorizationDenied(): Unit = {
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
|
||||
// Test that we cannot create or delete ACLs when Alter is denied.
|
||||
addClusterAcl(Deny, Alter)
|
||||
|
|
@ -415,7 +415,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
|
|||
val denyAcl = new AclBinding(new ResourcePattern(ResourceType.TOPIC, topic2, PatternType.LITERAL),
|
||||
new AccessControlEntry("User:*", "*", AclOperation.DESCRIBE_CONFIGS, AclPermissionType.DENY))
|
||||
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
client.createAcls(List(denyAcl).asJava, new CreateAclsOptions()).all().get()
|
||||
|
||||
val topics = Seq(topic1, topic2)
|
||||
|
|
|
|||
|
|
@ -16,7 +16,6 @@ import java.io.File
|
|||
import java.util
|
||||
import java.util.Collections
|
||||
import java.util.concurrent._
|
||||
import java.util.function.BiConsumer
|
||||
|
||||
import com.yammer.metrics.Metrics
|
||||
import com.yammer.metrics.core.Gauge
|
||||
|
|
@ -25,7 +24,7 @@ import kafka.security.authorizer.AuthorizerUtils.{WildcardHost, WildcardPrincipa
|
|||
import kafka.security.auth.{Operation, PermissionType}
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.{CoreUtils, TestUtils}
|
||||
import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, CreateAclsResult}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateAclsResult}
|
||||
import org.apache.kafka.common.acl._
|
||||
import org.apache.kafka.common.acl.AclOperation._
|
||||
import org.apache.kafka.common.acl.AclPermissionType._
|
||||
|
|
@ -65,14 +64,12 @@ object SslAdminIntegrationTest {
|
|||
semaphore.foreach(_.acquire())
|
||||
try {
|
||||
action.apply().asScala.zip(futures).foreach { case (baseFuture, resultFuture) =>
|
||||
baseFuture.whenComplete(new BiConsumer[T, Throwable]() {
|
||||
override def accept(result: T, exception: Throwable): Unit = {
|
||||
if (exception != null)
|
||||
resultFuture.completeExceptionally(exception)
|
||||
else
|
||||
resultFuture.complete(result)
|
||||
}
|
||||
})
|
||||
baseFuture.whenComplete { (result, exception) =>
|
||||
if (exception != null)
|
||||
resultFuture.completeExceptionally(exception)
|
||||
else
|
||||
resultFuture.complete(result)
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
semaphore.foreach(_.release())
|
||||
|
|
@ -96,7 +93,7 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest {
|
|||
|
||||
override protected def securityProtocol = SecurityProtocol.SSL
|
||||
override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
|
||||
private val adminClients = mutable.Buffer.empty[AdminClient]
|
||||
private val adminClients = mutable.Buffer.empty[Admin]
|
||||
|
||||
override def configureSecurityBeforeServersStart(): Unit = {
|
||||
val authorizer = CoreUtils.createObject[Authorizer](classOf[AclAuthorizer].getName)
|
||||
|
|
@ -247,7 +244,7 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest {
|
|||
val testSemaphore = new Semaphore(0)
|
||||
SslAdminIntegrationTest.semaphore = Some(testSemaphore)
|
||||
|
||||
client = AdminClient.create(createConfig())
|
||||
client = Admin.create(createConfig())
|
||||
val results = client.createAcls(List(acl2, acl3).asJava).values
|
||||
assertEquals(Set(acl2, acl3), results.keySet().asScala)
|
||||
assertFalse(results.values().asScala.exists(_.isDone))
|
||||
|
|
@ -269,10 +266,10 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest {
|
|||
validateRequestContext(SslAdminIntegrationTest.lastUpdateRequestContext.get, ApiKeys.DELETE_ACLS)
|
||||
}
|
||||
|
||||
private def createAdminClient: AdminClient = {
|
||||
private def createAdminClient: Admin = {
|
||||
val config = createConfig()
|
||||
config.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "40000")
|
||||
val client = AdminClient.create(config)
|
||||
val client = Admin.create(config)
|
||||
adminClients += client
|
||||
client
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,8 +24,8 @@ import java.util.Properties
|
|||
import java.util.concurrent._
|
||||
|
||||
import kafka.server.{BaseRequestTest, KafkaConfig}
|
||||
import kafka.utils.{CoreUtils, TestUtils}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig}
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
|
||||
|
|
@ -138,7 +138,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
|
|||
// Verify that connection blocked on the limit connects successfully when an existing connection is closed
|
||||
val plaintextConnections = (connectionCount until maxConnectionsPlaintext).map(_ => connect("PLAINTEXT"))
|
||||
executor = Executors.newSingleThreadExecutor
|
||||
val future = executor.submit(CoreUtils.runnable { createAndVerifyConnection() })
|
||||
val future = executor.submit((() => createAndVerifyConnection()): Runnable)
|
||||
Thread.sleep(100)
|
||||
assertFalse(future.isDone)
|
||||
plaintextConnections.head.close()
|
||||
|
|
@ -187,7 +187,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
|
|||
val config = new Properties()
|
||||
config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)
|
||||
config.put(AdminClientConfig.METADATA_MAX_AGE_CONFIG, "10")
|
||||
val adminClient = AdminClient.create(config)
|
||||
val adminClient = Admin.create(config)
|
||||
adminClient
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -1187,7 +1187,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet
|
|||
val bootstrapServers = TestUtils.bootstrapServers(servers, new ListenerName(listenerName))
|
||||
config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)
|
||||
config.put(AdminClientConfig.METADATA_MAX_AGE_CONFIG, "10")
|
||||
val adminClient = AdminClient.create(config)
|
||||
val adminClient = Admin.create(config)
|
||||
adminClients += adminClient
|
||||
adminClient
|
||||
}
|
||||
|
|
|
|||
|
|
@ -49,7 +49,7 @@ import org.apache.directory.server.kerberos.shared.keytab.{Keytab, KeytabEntry}
|
|||
import org.apache.directory.server.protocol.shared.transport.{TcpTransport, UdpTransport}
|
||||
import org.apache.directory.server.xdbm.Index
|
||||
import org.apache.directory.shared.kerberos.KerberosTime
|
||||
import org.apache.kafka.common.utils.{Java, Utils}
|
||||
import org.apache.kafka.common.utils.{Java, KafkaThread, Utils}
|
||||
|
||||
/**
|
||||
* Mini KDC based on Apache Directory Server that can be embedded in tests or used from command line as a standalone
|
||||
|
|
@ -390,7 +390,7 @@ object MiniKdc {
|
|||
|
|
||||
""".stripMargin
|
||||
println(infoMessage)
|
||||
Runtime.getRuntime.addShutdownHook(CoreUtils.newThread("minikdc-shutdown-hook", daemon = false) {
|
||||
Runtime.getRuntime.addShutdownHook(new KafkaThread("minikdc-shutdown-hook", false) {
|
||||
miniKdc.stop()
|
||||
})
|
||||
}
|
||||
|
|
|
|||
|
|
@ -26,8 +26,8 @@ import java.util.{Properties, Random}
|
|||
|
||||
import joptsimple.OptionParser
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.clients.admin.NewTopic
|
||||
import org.apache.kafka.clients.{CommonClientConfigs, admin}
|
||||
import org.apache.kafka.clients.admin.{Admin, NewTopic}
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
|
||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
|
||||
import org.apache.kafka.common.config.TopicConfig
|
||||
|
|
@ -138,7 +138,7 @@ object LogCompactionTester {
|
|||
def createTopics(brokerUrl: String, topics: Seq[String]): Unit = {
|
||||
val adminConfig = new Properties
|
||||
adminConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerUrl)
|
||||
val adminClient = admin.AdminClient.create(adminConfig)
|
||||
val adminClient = Admin.create(adminConfig)
|
||||
|
||||
try {
|
||||
val topicConfigs = Map(TopicConfig.CLEANUP_POLICY_CONFIG -> TopicConfig.CLEANUP_POLICY_COMPACT)
|
||||
|
|
@ -345,4 +345,4 @@ object TestRecord {
|
|||
val components = line.split("\t")
|
||||
new TestRecord(components(0), components(1).toInt, components(2).toLong, components(3) == "d")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -217,7 +217,7 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
|
|||
}
|
||||
|
||||
@Test(expected = classOf[IllegalArgumentException])
|
||||
def testInvalidJAuthorizerProperty() {
|
||||
def testInvalidJAuthorizerProperty(): Unit = {
|
||||
val args = Array("--authorizer-properties", "zookeeper.connect " + zkConnect)
|
||||
val aclCommandService = new AclCommand.JAuthorizerService(classOf[JAuthorizer], new AclCommandOptions(args))
|
||||
aclCommandService.listAcls()
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import kafka.admin.DelegationTokenCommand.DelegationTokenCommandOptions
|
|||
import kafka.api.{KafkaSasl, SaslSetup}
|
||||
import kafka.server.{BaseRequestTest, KafkaConfig}
|
||||
import kafka.utils.{JaasTestUtils, TestUtils}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.junit.Assert._
|
||||
import org.junit.{After, Before, Test}
|
||||
|
|
@ -68,7 +68,7 @@ class DelegationTokenCommandTest extends BaseRequestTest with SaslSetup {
|
|||
|
||||
@Test
|
||||
def testDelegationTokenRequests(): Unit = {
|
||||
adminClient = AdminClient.create(createAdminConfig)
|
||||
adminClient = Admin.create(createAdminConfig)
|
||||
val renewer1 = "User:renewer1"
|
||||
val renewer2 = "User:renewer2"
|
||||
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ import kafka.server.KafkaConfig
|
|||
import kafka.server.KafkaServer
|
||||
import kafka.utils.TestUtils
|
||||
import kafka.zk.ZooKeeperTestHarness
|
||||
import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.errors.TimeoutException
|
||||
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException
|
||||
|
|
@ -71,7 +71,7 @@ final class LeaderElectionCommandTest extends ZooKeeperTestHarness {
|
|||
|
||||
@Test
|
||||
def testAllTopicPartition(): Unit = {
|
||||
TestUtils.resource(AdminClient.create(createConfig(servers).asJava)) { client =>
|
||||
TestUtils.resource(Admin.create(createConfig(servers).asJava)) { client =>
|
||||
val topic = "unclean-topic"
|
||||
val partition = 0
|
||||
val assignment = Seq(broker2, broker3)
|
||||
|
|
@ -102,7 +102,7 @@ final class LeaderElectionCommandTest extends ZooKeeperTestHarness {
|
|||
|
||||
@Test
|
||||
def testTopicPartition(): Unit = {
|
||||
TestUtils.resource(AdminClient.create(createConfig(servers).asJava)) { client =>
|
||||
TestUtils.resource(Admin.create(createConfig(servers).asJava)) { client =>
|
||||
val topic = "unclean-topic"
|
||||
val partition = 0
|
||||
val assignment = Seq(broker2, broker3)
|
||||
|
|
@ -134,7 +134,7 @@ final class LeaderElectionCommandTest extends ZooKeeperTestHarness {
|
|||
|
||||
@Test
|
||||
def testPathToJsonFile(): Unit = {
|
||||
TestUtils.resource(AdminClient.create(createConfig(servers).asJava)) { client =>
|
||||
TestUtils.resource(Admin.create(createConfig(servers).asJava)) { client =>
|
||||
val topic = "unclean-topic"
|
||||
val partition = 0
|
||||
val assignment = Seq(broker2, broker3)
|
||||
|
|
@ -167,7 +167,7 @@ final class LeaderElectionCommandTest extends ZooKeeperTestHarness {
|
|||
|
||||
@Test
|
||||
def testPreferredReplicaElection(): Unit = {
|
||||
TestUtils.resource(AdminClient.create(createConfig(servers).asJava)) { client =>
|
||||
TestUtils.resource(Admin.create(createConfig(servers).asJava)) { client =>
|
||||
val topic = "unclean-topic"
|
||||
val partition = 0
|
||||
val assignment = Seq(broker2, broker3)
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import kafka.zk.{ReassignPartitionsZNode, ZkVersion, ZooKeeperTestHarness}
|
|||
import org.junit.Assert.{assertEquals, assertFalse, assertTrue}
|
||||
import org.junit.{After, Before, Test}
|
||||
import kafka.admin.ReplicationQuotaUtils._
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, NewPartitionReassignment, NewPartitions, PartitionReassignment, AdminClient => JAdminClient}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, NewPartitionReassignment, NewPartitions, PartitionReassignment}
|
||||
import org.apache.kafka.common.{TopicPartition, TopicPartitionReplica}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
|
@ -67,7 +67,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
|
|||
val props = new Properties()
|
||||
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers))
|
||||
props.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "10000")
|
||||
JAdminClient.create(props)
|
||||
Admin.create(props)
|
||||
}
|
||||
|
||||
def getRandomLogDirAssignment(brokerId: Int): String = {
|
||||
|
|
|
|||
|
|
@ -26,7 +26,6 @@ import kafka.utils.{Exit, Logging, TestUtils}
|
|||
import kafka.zk.{ConfigEntityChangeNotificationZNode, DeleteTopicsTopicZNode}
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
import org.apache.kafka.clients.admin._
|
||||
import org.apache.kafka.clients.producer.ProducerRecord
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.config.{ConfigException, ConfigResource, TopicConfig}
|
||||
import org.apache.kafka.common.internals.Topic
|
||||
|
|
@ -107,7 +106,7 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin
|
|||
// create adminClient
|
||||
val props = new Properties()
|
||||
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerList)
|
||||
adminClient = AdminClient.create(props)
|
||||
adminClient = Admin.create(props)
|
||||
topicService = AdminClientTopicService(adminClient)
|
||||
testTopicName = s"${testName.getMethodName}-${Random.alphanumeric.take(10).mkString}"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -42,7 +42,6 @@ import org.mockito.Mockito._
|
|||
import org.scalatest.Assertions.assertThrows
|
||||
import org.mockito.ArgumentMatchers
|
||||
import org.mockito.invocation.InvocationOnMock
|
||||
import org.mockito.stubbing.Answer
|
||||
import unit.kafka.cluster.AbstractPartitionTest
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
|
@ -908,13 +907,11 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
logManager)
|
||||
|
||||
when(delayedOperations.checkAndCompleteFetch())
|
||||
.thenAnswer(new Answer[Unit] {
|
||||
override def answer(invocation: InvocationOnMock): Unit = {
|
||||
// Acquire leaderIsrUpdate read lock of a different partition when completing delayed fetch
|
||||
val anotherPartition = (tp.partition + 1) % topicPartitions.size
|
||||
val partition = partitions(anotherPartition)
|
||||
partition.fetchOffsetSnapshot(Optional.of(leaderEpoch), fetchOnlyFromLeader = true)
|
||||
}
|
||||
.thenAnswer((invocation: InvocationOnMock) => {
|
||||
// Acquire leaderIsrUpdate read lock of a different partition when completing delayed fetch
|
||||
val anotherPartition = (tp.partition + 1) % topicPartitions.size
|
||||
val partition = partitions(anotherPartition)
|
||||
partition.fetchOffsetSnapshot(Optional.of(leaderEpoch), fetchOnlyFromLeader = true)
|
||||
})
|
||||
|
||||
partition.setLog(log, isFutureLog = false)
|
||||
|
|
@ -946,16 +943,16 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val executor = Executors.newFixedThreadPool(topicPartitions.size + 1)
|
||||
try {
|
||||
// Invoke some operation that acquires leaderIsrUpdate write lock on one thread
|
||||
executor.submit(CoreUtils.runnable {
|
||||
executor.submit((() => {
|
||||
while (!done.get) {
|
||||
partitions.foreach(_.maybeShrinkIsr())
|
||||
}
|
||||
})
|
||||
}): Runnable)
|
||||
// Append records to partitions, one partition-per-thread
|
||||
val futures = partitions.map { partition =>
|
||||
executor.submit(CoreUtils.runnable {
|
||||
executor.submit((() => {
|
||||
(1 to 10000).foreach { _ => partition.appendRecordsToLeader(createRecords(baseOffset = 0), isFromClient = true) }
|
||||
})
|
||||
}): Runnable)
|
||||
}
|
||||
futures.foreach(_.get(15, TimeUnit.SECONDS))
|
||||
done.set(true)
|
||||
|
|
@ -1589,11 +1586,9 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
@Test
|
||||
def testLogConfigDirtyAsTopicUpdated(): Unit = {
|
||||
val spyLogManager = spy(logManager)
|
||||
doAnswer(new Answer[Unit] {
|
||||
def answer(invocation: InvocationOnMock): Unit = {
|
||||
logManager.initializingLog(topicPartition)
|
||||
logManager.topicConfigUpdated(topicPartition.topic())
|
||||
}
|
||||
doAnswer((invocation: InvocationOnMock) => {
|
||||
logManager.initializingLog(topicPartition)
|
||||
logManager.topicConfigUpdated(topicPartition.topic())
|
||||
}).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition))
|
||||
|
||||
val partition = new Partition(topicPartition,
|
||||
|
|
@ -1626,11 +1621,9 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
@Test
|
||||
def testLogConfigDirtyAsBrokerUpdated(): Unit = {
|
||||
val spyLogManager = spy(logManager)
|
||||
doAnswer(new Answer[Unit] {
|
||||
def answer(invocation: InvocationOnMock): Unit = {
|
||||
logManager.initializingLog(topicPartition)
|
||||
logManager.brokerConfigUpdated()
|
||||
}
|
||||
doAnswer((invocation: InvocationOnMock) => {
|
||||
logManager.initializingLog(topicPartition)
|
||||
logManager.brokerConfigUpdated()
|
||||
}).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition))
|
||||
|
||||
val partition = new Partition(topicPartition,
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ package kafka.coordinator.transaction
|
|||
|
||||
import kafka.zk.KafkaZkClient
|
||||
import org.apache.kafka.common.KafkaException
|
||||
import org.easymock.{Capture, EasyMock, IAnswer}
|
||||
import org.easymock.{Capture, EasyMock}
|
||||
import org.junit.{After, Test}
|
||||
import org.junit.Assert._
|
||||
|
||||
|
|
@ -35,23 +35,21 @@ class ProducerIdManagerTest {
|
|||
def testGetProducerId(): Unit = {
|
||||
var zkVersion: Option[Int] = None
|
||||
var data: Array[Byte] = null
|
||||
EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(new IAnswer[(Option[Array[Byte]], Int)] {
|
||||
override def answer(): (Option[Array[Byte]], Int) = zkVersion.map(Some(data) -> _).getOrElse(None, 0)
|
||||
}).anyTimes()
|
||||
EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(() =>
|
||||
zkVersion.map(Some(data) -> _).getOrElse(None, 0)).anyTimes()
|
||||
|
||||
val capturedVersion: Capture[Int] = EasyMock.newCapture()
|
||||
val capturedData: Capture[Array[Byte]] = EasyMock.newCapture()
|
||||
EasyMock.expect(zkClient.conditionalUpdatePath(EasyMock.anyString(),
|
||||
EasyMock.capture(capturedData),
|
||||
EasyMock.capture(capturedVersion),
|
||||
EasyMock.anyObject[Option[(KafkaZkClient, String, Array[Byte]) => (Boolean, Int)]])).andAnswer(new IAnswer[(Boolean, Int)] {
|
||||
override def answer(): (Boolean, Int) = {
|
||||
val newZkVersion = capturedVersion.getValue + 1
|
||||
zkVersion = Some(newZkVersion)
|
||||
data = capturedData.getValue
|
||||
(true, newZkVersion)
|
||||
}
|
||||
}).anyTimes()
|
||||
EasyMock.anyObject[Option[(KafkaZkClient, String, Array[Byte]) => (Boolean, Int)]])
|
||||
).andAnswer(() => {
|
||||
val newZkVersion = capturedVersion.getValue + 1
|
||||
zkVersion = Some(newZkVersion)
|
||||
data = capturedData.getValue
|
||||
(true, newZkVersion)
|
||||
}).anyTimes()
|
||||
|
||||
EasyMock.replay(zkClient)
|
||||
|
||||
|
|
@ -76,12 +74,10 @@ class ProducerIdManagerTest {
|
|||
|
||||
@Test(expected = classOf[KafkaException])
|
||||
def testExceedProducerIdLimit(): Unit = {
|
||||
EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(new IAnswer[(Option[Array[Byte]], Int)] {
|
||||
override def answer(): (Option[Array[Byte]], Int) = {
|
||||
val json = ProducerIdManager.generateProducerIdBlockJson(
|
||||
ProducerIdBlock(0, Long.MaxValue - ProducerIdManager.PidBlockSize, Long.MaxValue))
|
||||
(Some(json), 0)
|
||||
}
|
||||
EasyMock.expect(zkClient.getDataAndVersion(EasyMock.anyString)).andAnswer(() => {
|
||||
val json = ProducerIdManager.generateProducerIdBlockJson(
|
||||
ProducerIdBlock(0, Long.MaxValue - ProducerIdManager.PidBlockSize, Long.MaxValue))
|
||||
(Some(json), 0)
|
||||
}).anyTimes()
|
||||
EasyMock.replay(zkClient)
|
||||
new ProducerIdManager(0, zkClient)
|
||||
|
|
|
|||
|
|
@ -79,11 +79,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren
|
|||
|
||||
val pidManager: ProducerIdManager = EasyMock.createNiceMock(classOf[ProducerIdManager])
|
||||
EasyMock.expect(pidManager.generateProducerId())
|
||||
.andAnswer(new IAnswer[Long]() {
|
||||
def answer(): Long = {
|
||||
if (bumpProducerId) producerId + 1 else producerId
|
||||
}
|
||||
})
|
||||
.andAnswer(() => if (bumpProducerId) producerId + 1 else producerId)
|
||||
.anyTimes()
|
||||
val txnMarkerPurgatory = new DelayedOperationPurgatory[DelayedTxnMarker]("txn-purgatory-name",
|
||||
new MockTimer,
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import org.apache.kafka.common.protocol.Errors
|
|||
import org.apache.kafka.common.record.RecordBatch
|
||||
import org.apache.kafka.common.requests.TransactionResult
|
||||
import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch}
|
||||
import org.easymock.{Capture, EasyMock, IAnswer}
|
||||
import org.easymock.{Capture, EasyMock}
|
||||
import org.junit.Assert._
|
||||
import org.junit.Test
|
||||
|
||||
|
|
@ -62,14 +62,10 @@ class TransactionCoordinatorTest {
|
|||
var error: Errors = Errors.NONE
|
||||
|
||||
private def mockPidManager(): Unit = {
|
||||
EasyMock.expect(pidManager.generateProducerId())
|
||||
.andAnswer(new IAnswer[Long] {
|
||||
override def answer(): Long = {
|
||||
nextPid += 1
|
||||
nextPid - 1
|
||||
}
|
||||
})
|
||||
.anyTimes()
|
||||
EasyMock.expect(pidManager.generateProducerId()).andAnswer(() => {
|
||||
nextPid += 1
|
||||
nextPid - 1
|
||||
}).anyTimes()
|
||||
}
|
||||
|
||||
private def initPidGenericMocks(transactionalId: String): Unit = {
|
||||
|
|
@ -110,26 +106,18 @@ class TransactionCoordinatorTest {
|
|||
.once()
|
||||
|
||||
EasyMock.expect(transactionManager.putTransactionStateIfNotExists(EasyMock.eq(transactionalId), EasyMock.capture(capturedTxn)))
|
||||
.andAnswer(new IAnswer[Either[Errors, CoordinatorEpochAndTxnMetadata]] {
|
||||
override def answer(): Either[Errors, CoordinatorEpochAndTxnMetadata] = {
|
||||
assertTrue(capturedTxn.hasCaptured)
|
||||
Right(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, capturedTxn.getValue))
|
||||
}
|
||||
})
|
||||
.once()
|
||||
.andAnswer(() => {
|
||||
assertTrue(capturedTxn.hasCaptured)
|
||||
Right(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, capturedTxn.getValue))
|
||||
}).once()
|
||||
|
||||
EasyMock.expect(transactionManager.appendTransactionToLog(
|
||||
EasyMock.eq(transactionalId),
|
||||
EasyMock.eq(coordinatorEpoch),
|
||||
EasyMock.anyObject().asInstanceOf[TxnTransitMetadata],
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}
|
||||
})
|
||||
.anyTimes()
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => capturedErrorsCallback.getValue.apply(Errors.NONE)).anyTimes()
|
||||
EasyMock.replay(pidManager, transactionManager)
|
||||
|
||||
coordinator.handleInitProducerId(transactionalId, txnTimeoutMs, None, initProducerIdMockCallback)
|
||||
|
|
@ -145,26 +133,18 @@ class TransactionCoordinatorTest {
|
|||
.once()
|
||||
|
||||
EasyMock.expect(transactionManager.putTransactionStateIfNotExists(EasyMock.eq(transactionalId), EasyMock.capture(capturedTxn)))
|
||||
.andAnswer(new IAnswer[Either[Errors, CoordinatorEpochAndTxnMetadata]] {
|
||||
override def answer(): Either[Errors, CoordinatorEpochAndTxnMetadata] = {
|
||||
assertTrue(capturedTxn.hasCaptured)
|
||||
Right(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, capturedTxn.getValue))
|
||||
}
|
||||
})
|
||||
.once()
|
||||
.andAnswer(() => {
|
||||
assertTrue(capturedTxn.hasCaptured)
|
||||
Right(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, capturedTxn.getValue))
|
||||
}).once()
|
||||
|
||||
EasyMock.expect(transactionManager.appendTransactionToLog(
|
||||
EasyMock.eq(transactionalId),
|
||||
EasyMock.eq(coordinatorEpoch),
|
||||
EasyMock.anyObject().asInstanceOf[TxnTransitMetadata],
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}
|
||||
})
|
||||
.anyTimes()
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => capturedErrorsCallback.getValue.apply(Errors.NONE)).anyTimes()
|
||||
EasyMock.replay(pidManager, transactionManager)
|
||||
|
||||
coordinator.handleInitProducerId(transactionalId, txnTimeoutMs, Some(new ProducerIdAndEpoch(producerId, producerEpoch)),
|
||||
|
|
@ -188,11 +168,7 @@ class TransactionCoordinatorTest {
|
|||
EasyMock.anyObject().asInstanceOf[TxnTransitMetadata],
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()
|
||||
)).andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}
|
||||
})
|
||||
)).andAnswer(() => capturedErrorsCallback.getValue.apply(Errors.NONE))
|
||||
|
||||
EasyMock.replay(pidManager, transactionManager)
|
||||
|
||||
|
|
@ -571,12 +547,8 @@ class TransactionCoordinatorTest {
|
|||
EasyMock.eq(coordinatorEpoch),
|
||||
EasyMock.eq(originalMetadata.prepareAbortOrCommit(PrepareAbort, time.milliseconds())),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}
|
||||
})
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => capturedErrorsCallback.getValue.apply(Errors.NONE))
|
||||
|
||||
EasyMock.replay(transactionManager)
|
||||
|
||||
|
|
@ -617,12 +589,8 @@ class TransactionCoordinatorTest {
|
|||
txnStartTimestamp = time.milliseconds(),
|
||||
txnLastUpdateTimestamp = time.milliseconds())),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}
|
||||
})
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => capturedErrorsCallback.getValue.apply(Errors.NONE))
|
||||
|
||||
EasyMock.replay(transactionManager)
|
||||
|
||||
|
|
@ -688,15 +656,11 @@ class TransactionCoordinatorTest {
|
|||
EasyMock.eq(coordinatorEpoch),
|
||||
EasyMock.anyObject().asInstanceOf[TxnTransitMetadata],
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
|
||||
txnMetadata.pendingState = None
|
||||
}
|
||||
})
|
||||
.times(2)
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
txnMetadata.pendingState = None
|
||||
}).times(2)
|
||||
|
||||
EasyMock.replay(pidManager, transactionManager)
|
||||
|
||||
|
|
@ -729,17 +693,13 @@ class TransactionCoordinatorTest {
|
|||
EasyMock.eq(coordinatorEpoch),
|
||||
EasyMock.capture(capturedTxnTransitMetadata),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
|
||||
txnMetadata.pendingState = None
|
||||
txnMetadata.producerEpoch = capturedTxnTransitMetadata.getValue.producerEpoch
|
||||
txnMetadata.lastProducerEpoch = capturedTxnTransitMetadata.getValue.lastProducerEpoch
|
||||
}
|
||||
})
|
||||
.times(2)
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
txnMetadata.pendingState = None
|
||||
txnMetadata.producerEpoch = capturedTxnTransitMetadata.getValue.producerEpoch
|
||||
txnMetadata.lastProducerEpoch = capturedTxnTransitMetadata.getValue.lastProducerEpoch
|
||||
}).times(2)
|
||||
|
||||
EasyMock.replay(pidManager, transactionManager)
|
||||
|
||||
|
|
@ -773,19 +733,15 @@ class TransactionCoordinatorTest {
|
|||
EasyMock.eq(coordinatorEpoch),
|
||||
EasyMock.capture(capturedTxnTransitMetadata),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
|
||||
txnMetadata.pendingState = None
|
||||
txnMetadata.producerId = capturedTxnTransitMetadata.getValue.producerId
|
||||
txnMetadata.lastProducerId = capturedTxnTransitMetadata.getValue.lastProducerId
|
||||
txnMetadata.producerEpoch = capturedTxnTransitMetadata.getValue.producerEpoch
|
||||
txnMetadata.lastProducerEpoch = capturedTxnTransitMetadata.getValue.lastProducerEpoch
|
||||
}
|
||||
})
|
||||
.once
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
txnMetadata.pendingState = None
|
||||
txnMetadata.producerId = capturedTxnTransitMetadata.getValue.producerId
|
||||
txnMetadata.lastProducerId = capturedTxnTransitMetadata.getValue.lastProducerId
|
||||
txnMetadata.producerEpoch = capturedTxnTransitMetadata.getValue.producerEpoch
|
||||
txnMetadata.lastProducerEpoch = capturedTxnTransitMetadata.getValue.lastProducerEpoch
|
||||
}).once
|
||||
|
||||
EasyMock.replay(pidManager, transactionManager)
|
||||
|
||||
|
|
@ -820,19 +776,15 @@ class TransactionCoordinatorTest {
|
|||
EasyMock.eq(coordinatorEpoch),
|
||||
EasyMock.capture(capturedTxnTransitMetadata),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
|
||||
txnMetadata.pendingState = None
|
||||
txnMetadata.producerId = capturedTxnTransitMetadata.getValue.producerId
|
||||
txnMetadata.lastProducerId = capturedTxnTransitMetadata.getValue.lastProducerId
|
||||
txnMetadata.producerEpoch = capturedTxnTransitMetadata.getValue.producerEpoch
|
||||
txnMetadata.lastProducerEpoch = capturedTxnTransitMetadata.getValue.lastProducerEpoch
|
||||
}
|
||||
})
|
||||
.once
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => {
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
txnMetadata.pendingState = None
|
||||
txnMetadata.producerId = capturedTxnTransitMetadata.getValue.producerId
|
||||
txnMetadata.lastProducerId = capturedTxnTransitMetadata.getValue.lastProducerId
|
||||
txnMetadata.producerEpoch = capturedTxnTransitMetadata.getValue.producerEpoch
|
||||
txnMetadata.lastProducerEpoch = capturedTxnTransitMetadata.getValue.lastProducerEpoch
|
||||
}).once
|
||||
|
||||
EasyMock.replay(pidManager, transactionManager)
|
||||
|
||||
|
|
@ -879,11 +831,8 @@ class TransactionCoordinatorTest {
|
|||
EasyMock.eq(coordinatorEpoch),
|
||||
EasyMock.eq(expectedTransition),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {}
|
||||
})
|
||||
.once()
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => {}).once()
|
||||
|
||||
EasyMock.replay(transactionManager, transactionMarkerChannelManager)
|
||||
|
||||
|
|
@ -966,11 +915,9 @@ class TransactionCoordinatorTest {
|
|||
EasyMock.capture(capturedNewMetadata),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()
|
||||
)).andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
metadata.completeTransitionTo(capturedNewMetadata.getValue)
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}
|
||||
)).andAnswer(() => {
|
||||
metadata.completeTransitionTo(capturedNewMetadata.getValue)
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
})
|
||||
|
||||
EasyMock.replay(pidManager, transactionManager)
|
||||
|
|
@ -1001,13 +948,11 @@ class TransactionCoordinatorTest {
|
|||
EasyMock.eq(coordinatorEpoch),
|
||||
EasyMock.eq(transition),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
if (runCallback)
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}
|
||||
}).once()
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => {
|
||||
if (runCallback)
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}).once()
|
||||
|
||||
new TransactionMetadata(transactionalId, producerId, producerId, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH,
|
||||
txnTimeoutMs, transactionState, partitions, time.milliseconds(), time.milliseconds())
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ import org.apache.kafka.clients.{ClientResponse, NetworkClient}
|
|||
import org.apache.kafka.common.requests.{RequestHeader, TransactionResult, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
|
||||
import org.apache.kafka.common.utils.MockTime
|
||||
import org.apache.kafka.common.{Node, TopicPartition}
|
||||
import org.easymock.{Capture, EasyMock, IAnswer}
|
||||
import org.easymock.{Capture, EasyMock}
|
||||
import org.junit.Assert._
|
||||
import org.junit.Test
|
||||
import com.yammer.metrics.Metrics
|
||||
|
|
@ -290,11 +290,9 @@ class TransactionMarkerChannelManagerTest {
|
|||
EasyMock.eq(txnTransitionMetadata2),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
txnMetadata2.completeTransitionTo(txnTransitionMetadata2)
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}
|
||||
.andAnswer(() => {
|
||||
txnMetadata2.completeTransitionTo(txnTransitionMetadata2)
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}).once()
|
||||
EasyMock.replay(txnStateManager, metadataCache)
|
||||
|
||||
|
|
@ -339,11 +337,9 @@ class TransactionMarkerChannelManagerTest {
|
|||
EasyMock.eq(txnTransitionMetadata2),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
txnMetadata2.pendingState = None
|
||||
capturedErrorsCallback.getValue.apply(Errors.NOT_COORDINATOR)
|
||||
}
|
||||
.andAnswer(() => {
|
||||
txnMetadata2.pendingState = None
|
||||
capturedErrorsCallback.getValue.apply(Errors.NOT_COORDINATOR)
|
||||
}).once()
|
||||
EasyMock.replay(txnStateManager, metadataCache)
|
||||
|
||||
|
|
@ -388,17 +384,11 @@ class TransactionMarkerChannelManagerTest {
|
|||
EasyMock.eq(txnTransitionMetadata2),
|
||||
EasyMock.capture(capturedErrorsCallback),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedErrorsCallback.getValue.apply(Errors.COORDINATOR_NOT_AVAILABLE)
|
||||
}
|
||||
})
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
.andAnswer(() => capturedErrorsCallback.getValue.apply(Errors.COORDINATOR_NOT_AVAILABLE))
|
||||
.andAnswer(() => {
|
||||
txnMetadata2.completeTransitionTo(txnTransitionMetadata2)
|
||||
capturedErrorsCallback.getValue.apply(Errors.NONE)
|
||||
}
|
||||
})
|
||||
})
|
||||
|
||||
EasyMock.replay(txnStateManager, metadataCache)
|
||||
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ import org.apache.kafka.common.TopicPartition
|
|||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||
import org.apache.kafka.common.record.RecordBatch
|
||||
import org.apache.kafka.common.requests.{RequestHeader, TransactionResult, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
|
||||
import org.easymock.{EasyMock, IAnswer}
|
||||
import org.easymock.EasyMock
|
||||
import org.junit.Assert._
|
||||
import org.junit.Test
|
||||
|
||||
|
|
@ -229,11 +229,7 @@ class TransactionMarkerRequestCompletionHandlerTest {
|
|||
|
||||
var completed = false
|
||||
EasyMock.expect(markerChannelManager.completeSendMarkersForTxnId(transactionalId))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
completed = true
|
||||
}
|
||||
})
|
||||
.andAnswer(() => completed = true)
|
||||
.once()
|
||||
EasyMock.replay(markerChannelManager)
|
||||
|
||||
|
|
@ -249,11 +245,7 @@ class TransactionMarkerRequestCompletionHandlerTest {
|
|||
|
||||
var removed = false
|
||||
EasyMock.expect(markerChannelManager.removeMarkersForTxnId(transactionalId))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
removed = true
|
||||
}
|
||||
})
|
||||
.andAnswer(() => removed = true)
|
||||
.once()
|
||||
EasyMock.replay(markerChannelManager)
|
||||
|
||||
|
|
|
|||
|
|
@ -534,15 +534,9 @@ class TransactionStateManagerTest {
|
|||
EasyMock.capture(capturedArgument),
|
||||
EasyMock.anyObject().asInstanceOf[Option[ReentrantLock]],
|
||||
EasyMock.anyObject()
|
||||
)).andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
capturedArgument.getValue.apply(
|
||||
Map(partition ->
|
||||
new PartitionResponse(error, 0L, RecordBatch.NO_TIMESTAMP, 0L)
|
||||
)
|
||||
)
|
||||
}
|
||||
})
|
||||
)).andAnswer(() => capturedArgument.getValue.apply(
|
||||
Map(partition -> new PartitionResponse(error, 0L, RecordBatch.NO_TIMESTAMP, 0L)))
|
||||
)
|
||||
case _ => // shouldn't append
|
||||
}
|
||||
|
||||
|
|
@ -646,13 +640,9 @@ class TransactionStateManagerTest {
|
|||
EasyMock.capture(capturedArgument),
|
||||
EasyMock.anyObject().asInstanceOf[Option[ReentrantLock]],
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = capturedArgument.getValue.apply(
|
||||
Map(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId) ->
|
||||
new PartitionResponse(error, 0L, RecordBatch.NO_TIMESTAMP, 0L)
|
||||
)
|
||||
)
|
||||
}
|
||||
).andAnswer(() => capturedArgument.getValue.apply(
|
||||
Map(new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId) ->
|
||||
new PartitionResponse(error, 0L, RecordBatch.NO_TIMESTAMP, 0L)))
|
||||
)
|
||||
EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject()))
|
||||
.andStubReturn(Some(RecordBatch.MAGIC_VALUE_V1))
|
||||
|
|
|
|||
|
|
@ -87,25 +87,23 @@ class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with
|
|||
|
||||
// Thread checking the metric continuously
|
||||
running = true
|
||||
val thread = new Thread(new Runnable {
|
||||
def run(): Unit = {
|
||||
while (running) {
|
||||
for ( s <- servers if running) {
|
||||
underReplicatedPartitionCount = s.replicaManager.underReplicatedPartitionCount
|
||||
if (underReplicatedPartitionCount > 0) {
|
||||
running = false
|
||||
}
|
||||
val thread = new Thread(() => {
|
||||
while (running) {
|
||||
for (s <- servers if running) {
|
||||
underReplicatedPartitionCount = s.replicaManager.underReplicatedPartitionCount
|
||||
if (underReplicatedPartitionCount > 0) {
|
||||
running = false
|
||||
}
|
||||
}
|
||||
|
||||
preferredReplicaImbalanceCount = preferredReplicaImbalanceCountGauge.value
|
||||
if (preferredReplicaImbalanceCount > 0) {
|
||||
running = false
|
||||
}
|
||||
preferredReplicaImbalanceCount = preferredReplicaImbalanceCountGauge.value
|
||||
if (preferredReplicaImbalanceCount > 0) {
|
||||
running = false
|
||||
}
|
||||
|
||||
offlinePartitionsCount = offlinePartitionsCountGauge.value
|
||||
if (offlinePartitionsCount > 0) {
|
||||
running = false
|
||||
}
|
||||
offlinePartitionsCount = offlinePartitionsCountGauge.value
|
||||
if (offlinePartitionsCount > 0) {
|
||||
running = false
|
||||
}
|
||||
}
|
||||
})
|
||||
|
|
|
|||
|
|
@ -36,7 +36,7 @@ import org.apache.kafka.common.errors.TimeoutException
|
|||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.serialization.StringDeserializer
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.junit.Assert._
|
||||
import org.scalatest.Assertions.intercept
|
||||
|
||||
|
|
@ -352,6 +352,6 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
|
|||
val bootstrapServers = TestUtils.bootstrapServers(servers, new ListenerName("PLAINTEXT"))
|
||||
config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)
|
||||
config.put(AdminClientConfig.METADATA_MAX_AGE_CONFIG, "10")
|
||||
AdminClient.create(config)
|
||||
Admin.create(config)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,8 +31,6 @@ import org.junit.Assert._
|
|||
import org.junit.{After, Before, Test}
|
||||
import org.mockito.ArgumentMatchers.any
|
||||
import org.mockito.Mockito.{doAnswer, spy}
|
||||
import org.mockito.invocation.InvocationOnMock
|
||||
import org.mockito.stubbing.Answer
|
||||
|
||||
import scala.collection.mutable
|
||||
import scala.util.{Failure, Try}
|
||||
|
|
@ -115,18 +113,16 @@ class LogManagerTest {
|
|||
logManager.shutdown()
|
||||
logManager = spy(createLogManager(dirs))
|
||||
val brokenDirs = mutable.Set[File]()
|
||||
doAnswer(new Answer[Try[File]] {
|
||||
override def answer(invocation: InvocationOnMock): Try[File] = {
|
||||
// The first half of directories tried will fail, the rest goes through.
|
||||
val logDir = invocation.getArgument[File](0)
|
||||
if (brokenDirs.contains(logDir) || brokenDirs.size < dirs.length / 2) {
|
||||
brokenDirs.add(logDir)
|
||||
Failure(new Throwable("broken dir"))
|
||||
} else {
|
||||
invocation.callRealMethod().asInstanceOf[Try[File]]
|
||||
}
|
||||
doAnswer { invocation =>
|
||||
// The first half of directories tried will fail, the rest goes through.
|
||||
val logDir = invocation.getArgument[File](0)
|
||||
if (brokenDirs.contains(logDir) || brokenDirs.size < dirs.length / 2) {
|
||||
brokenDirs.add(logDir)
|
||||
Failure(new Throwable("broken dir"))
|
||||
} else {
|
||||
invocation.callRealMethod().asInstanceOf[Try[File]]
|
||||
}
|
||||
}).when(logManager).createLogDirectory(any(), any())
|
||||
}.when(logManager).createLogDirectory(any(), any())
|
||||
logManager.startup()
|
||||
|
||||
// Request creating a new log.
|
||||
|
|
|
|||
|
|
@ -30,7 +30,7 @@ import javax.net.ssl._
|
|||
import kafka.security.CredentialProvider
|
||||
import kafka.server.{KafkaConfig, ThrottledChannel}
|
||||
import kafka.utils.Implicits._
|
||||
import kafka.utils.{CoreUtils, TestUtils}
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.memory.MemoryPool
|
||||
import org.apache.kafka.common.message.SaslAuthenticateRequestData
|
||||
|
|
@ -299,7 +299,7 @@ class SocketServerTest {
|
|||
val externalListener = new ListenerName("EXTERNAL")
|
||||
val externalEndpoint = updatedEndPoints.find(e => e.listenerName.get == externalListener.value).get
|
||||
val futures = Map(externalEndpoint -> externalReadyFuture)
|
||||
val startFuture = executor.submit(CoreUtils.runnable(testableServer.startDataPlaneProcessors(futures)))
|
||||
val startFuture = executor.submit((() => testableServer.startDataPlaneProcessors(futures)): Runnable)
|
||||
TestUtils.waitUntilTrue(() => listenerStarted(config.interBrokerListenerName), "Inter-broker listener not started")
|
||||
assertFalse("Socket server startup did not wait for future to complete", startFuture.isDone)
|
||||
|
||||
|
|
@ -869,7 +869,7 @@ class SocketServerTest {
|
|||
|
||||
/* Test that we update request metrics if the client closes the connection while the broker response is in flight. */
|
||||
@Test
|
||||
def testClientDisconnectionUpdatesRequestMetrics: Unit = {
|
||||
def testClientDisconnectionUpdatesRequestMetrics(): Unit = {
|
||||
// The way we detect a connection close from the client depends on the response size. If it's small, an
|
||||
// IOException ("Connection reset by peer") is thrown when the Selector reads from the socket. If
|
||||
// it's large, an IOException ("Broken pipe") is thrown when the Selector writes to the socket. We test
|
||||
|
|
|
|||
|
|
@ -25,7 +25,7 @@ import kafka.api.{ApiVersion, KAFKA_2_0_IV0, KAFKA_2_0_IV1}
|
|||
import kafka.security.auth.Resource
|
||||
import kafka.security.authorizer.AuthorizerUtils.{WildcardHost, WildcardPrincipal}
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.{CoreUtils, TestUtils}
|
||||
import kafka.utils.TestUtils
|
||||
import kafka.zk.{ZkAclStore, ZooKeeperTestHarness}
|
||||
import kafka.zookeeper.{GetChildrenRequest, GetDataRequest, ZooKeeperClient}
|
||||
import org.apache.kafka.common.acl._
|
||||
|
|
@ -389,7 +389,7 @@ class AclAuthorizerTest extends ZooKeeperTestHarness {
|
|||
}
|
||||
}
|
||||
try {
|
||||
val future = executor.submit(CoreUtils.runnable(aclAuthorizer3.configure(config.originals)))
|
||||
val future = executor.submit((() => aclAuthorizer3.configure(config.originals)): Runnable)
|
||||
configureSemaphore.acquire()
|
||||
val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
||||
val acls = Set(new AccessControlEntry(user1.toString, "host-1", READ, DENY))
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ package kafka.server
|
|||
import java.util
|
||||
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.errors.UnsupportedByAuthenticationException
|
||||
import org.junit.{After, Before, Test}
|
||||
import org.scalatest.Assertions.intercept
|
||||
|
|
@ -48,7 +48,7 @@ class DelegationTokenRequestsOnPlainTextTest extends BaseRequestTest {
|
|||
|
||||
@Test
|
||||
def testDelegationTokenRequests(): Unit = {
|
||||
adminClient = AdminClient.create(createAdminConfig)
|
||||
adminClient = Admin.create(createAdminConfig)
|
||||
|
||||
val createResult = adminClient.createDelegationToken()
|
||||
intercept[ExecutionException](createResult.delegationToken().get()).getCause.isInstanceOf[UnsupportedByAuthenticationException]
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ import java.util
|
|||
|
||||
import kafka.api.{KafkaSasl, SaslSetup}
|
||||
import kafka.utils.{JaasTestUtils, TestUtils}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig, CreateDelegationTokenOptions, DescribeDelegationTokenOptions}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateDelegationTokenOptions, DescribeDelegationTokenOptions}
|
||||
import org.apache.kafka.common.errors.InvalidPrincipalTypeException
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.SecurityUtils
|
||||
|
|
@ -67,7 +67,7 @@ class DelegationTokenRequestsTest extends BaseRequestTest with SaslSetup {
|
|||
|
||||
@Test
|
||||
def testDelegationTokenRequests(): Unit = {
|
||||
adminClient = AdminClient.create(createAdminConfig)
|
||||
adminClient = Admin.create(createAdminConfig)
|
||||
|
||||
// create token1 with renewer1
|
||||
val renewer1 = List(SecurityUtils.parseKafkaPrincipal("User:renewer1")).asJava
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ import java.util
|
|||
|
||||
import kafka.api.{KafkaSasl, SaslSetup}
|
||||
import kafka.utils.{JaasTestUtils, TestUtils}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClient, AdminClientConfig}
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||
import org.apache.kafka.common.errors.DelegationTokenDisabledException
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.junit.{After, Before, Test}
|
||||
|
|
@ -56,7 +56,7 @@ class DelegationTokenRequestsWithDisableTokenFeatureTest extends BaseRequestTest
|
|||
|
||||
@Test
|
||||
def testDelegationTokenRequests(): Unit = {
|
||||
adminClient = AdminClient.create(createAdminConfig)
|
||||
adminClient = Admin.create(createAdminConfig)
|
||||
|
||||
val createResult = adminClient.createDelegationToken()
|
||||
intercept[ExecutionException](createResult.delegationToken().get()).getCause.isInstanceOf[DelegationTokenDisabledException]
|
||||
|
|
|
|||
|
|
@ -301,11 +301,8 @@ class KafkaApisTest {
|
|||
EasyMock.anyObject(),
|
||||
EasyMock.capture(responseCallback),
|
||||
EasyMock.anyObject(),
|
||||
EasyMock.anyObject())).andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
responseCallback.getValue.apply(Map(tp2 -> new PartitionResponse(Errors.NONE)))
|
||||
}
|
||||
})
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => responseCallback.getValue.apply(Map(tp2 -> new PartitionResponse(Errors.NONE))))
|
||||
|
||||
EasyMock.expect(requestChannel.sendResponse(EasyMock.capture(capturedResponse)))
|
||||
EasyMock.replay(replicaManager, replicaQuotaManager, requestChannel)
|
||||
|
|
@ -340,11 +337,8 @@ class KafkaApisTest {
|
|||
EasyMock.anyObject(),
|
||||
EasyMock.capture(responseCallback),
|
||||
EasyMock.anyObject(),
|
||||
EasyMock.anyObject())).andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
responseCallback.getValue.apply(Map(tp2 -> new PartitionResponse(Errors.NONE)))
|
||||
}
|
||||
})
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => responseCallback.getValue.apply(Map(tp2 -> new PartitionResponse(Errors.NONE))))
|
||||
|
||||
EasyMock.expect(requestChannel.sendResponse(EasyMock.capture(capturedResponse)))
|
||||
EasyMock.replay(replicaManager, replicaQuotaManager, requestChannel)
|
||||
|
|
@ -831,7 +825,7 @@ class KafkaApisTest {
|
|||
assertReassignmentAndReplicationBytesOutPerSec(false)
|
||||
}
|
||||
|
||||
private def assertReassignmentAndReplicationBytesOutPerSec(isReassigning: Boolean) {
|
||||
private def assertReassignmentAndReplicationBytesOutPerSec(isReassigning: Boolean): Unit = {
|
||||
val leaderEpoch = 0
|
||||
val tp0 = new TopicPartition("tp", 0)
|
||||
|
||||
|
|
|
|||
|
|
@ -164,7 +164,7 @@ class LogDirFailureTest extends IntegrationTestHarness {
|
|||
}
|
||||
}
|
||||
|
||||
def testProduceAfterLogDirFailureOnLeader(failureType: LogDirFailureType) {
|
||||
def testProduceAfterLogDirFailureOnLeader(failureType: LogDirFailureType): Unit = {
|
||||
val consumer = createConsumer()
|
||||
subscribeAndWaitForAssignment(topic, consumer)
|
||||
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ import org.apache.kafka.common.protocol.Errors
|
|||
import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRequest}
|
||||
import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
|
||||
import org.easymock.EasyMock._
|
||||
import org.easymock.{Capture, CaptureType, EasyMock, IAnswer, IExpectationSetters}
|
||||
import org.easymock.{Capture, CaptureType, EasyMock, IExpectationSetters}
|
||||
import org.junit.Assert._
|
||||
import org.junit.Test
|
||||
|
||||
|
|
@ -396,12 +396,8 @@ class ReplicaAlterLogDirsThreadTest {
|
|||
EasyMock.anyObject(),
|
||||
EasyMock.capture(responseCallback),
|
||||
EasyMock.anyObject(),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
responseCallback.getValue.apply(Seq.empty[(TopicPartition, FetchPartitionData)])
|
||||
}
|
||||
}).anyTimes()
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => responseCallback.getValue.apply(Seq.empty[(TopicPartition, FetchPartitionData)])).anyTimes()
|
||||
|
||||
replay(replicaManager, logManager, quotaManager, partition, log, futureLog)
|
||||
|
||||
|
|
@ -631,11 +627,7 @@ class ReplicaAlterLogDirsThreadTest {
|
|||
EasyMock.anyObject(),
|
||||
EasyMock.capture(responseCallback),
|
||||
EasyMock.anyObject(),
|
||||
EasyMock.anyObject()))
|
||||
.andAnswer(new IAnswer[Unit] {
|
||||
override def answer(): Unit = {
|
||||
responseCallback.getValue.apply(Seq.empty[(TopicPartition, FetchPartitionData)])
|
||||
}
|
||||
}).anyTimes()
|
||||
EasyMock.anyObject())
|
||||
).andAnswer(() => responseCallback.getValue.apply(Seq.empty[(TopicPartition, FetchPartitionData)])).anyTimes()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -132,9 +132,5 @@ case class MockTask(name: String, fun: () => Unit, var nextExecution: Long, peri
|
|||
}
|
||||
}
|
||||
object MockTask {
|
||||
implicit def MockTaskOrdering : Ordering[MockTask] = new Ordering[MockTask] {
|
||||
def compare(x: MockTask, y: MockTask): Int = {
|
||||
x.compare(y)
|
||||
}
|
||||
}
|
||||
implicit def MockTaskOrdering: Ordering[MockTask] = (x, y) => x.compare(y)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1450,7 +1450,7 @@ object TestUtils extends Logging {
|
|||
offsetsToCommit.toMap
|
||||
}
|
||||
|
||||
def resetToCommittedPositions(consumer: KafkaConsumer[Array[Byte], Array[Byte]]) {
|
||||
def resetToCommittedPositions(consumer: KafkaConsumer[Array[Byte], Array[Byte]]): Unit = {
|
||||
val committed = consumer.committed(consumer.assignment).asScala.filter(_._2 != null).mapValues(_.offset)
|
||||
|
||||
consumer.assignment.asScala.foreach { topicPartition =>
|
||||
|
|
|
|||
|
|
@ -3761,7 +3761,7 @@ groupedTable
|
|||
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.ImplicitConversions</span></code>: Module that brings into scope the implicit conversions between the Scala and Java classes.</li>
|
||||
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.Serdes</span></code>: Module that contains all primitive SerDes that can be imported as implicits and a helper to create custom SerDes.</li>
|
||||
</ul>
|
||||
<p>The library is cross-built with Scala 2.11 and 2.12. To reference the library compiled against Scala {{scalaVersion}} include the following in your maven <code>pom.xml</code> add the following:</p>
|
||||
<p>The library is cross-built with Scala 2.12 and 2.13. To reference the library compiled against Scala {{scalaVersion}} include the following in your maven <code>pom.xml</code> add the following:</p>
|
||||
<pre class="brush: xml;">
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
|
|
|
|||
|
|
@ -81,7 +81,7 @@
|
|||
<tr class="row-even"><td><code class="docutils literal"><span class="pre">org.apache.kafka</span></code></td>
|
||||
<td><code class="docutils literal"><span class="pre">kafka-streams-scala</span></code></td>
|
||||
<td><code class="docutils literal"><span class="pre">{{fullDotVersion}}</span></code></td>
|
||||
<td>(Optional) Kafka Streams DSL for Scala library to write Scala Kafka Streams applications. When not using SBT you will need to suffix the artifact ID with the correct version of Scala your application is using (<code class="docutils literal"><span class="pre">_2.11</code></span>, <code class="docutils literal"><span class="pre">_2.12</code></span>)</td>
|
||||
<td>(Optional) Kafka Streams DSL for Scala library to write Scala Kafka Streams applications. When not using SBT you will need to suffix the artifact ID with the correct version of Scala your application is using (<code class="docutils literal"><span class="pre">_2.12</code></span>, <code class="docutils literal"><span class="pre">_2.13</code></span>)</td>
|
||||
</tr>
|
||||
</tbody>
|
||||
</table>
|
||||
|
|
|
|||
|
|
@ -25,6 +25,9 @@
|
|||
<code>zookeeper.session.timeout.ms</code> has been increased from 6s to 18s and
|
||||
<code>replica.lag.time.max.ms</code> from 10s to 30s.</li>
|
||||
<li>New DSL operator <code>cogroup()</code> has been added for aggregating multiple streams together at once</li>
|
||||
<li>Scala 2.11 is no longer supported. See
|
||||
<a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-531%3A+Drop+support+for+Scala+2.11+in+Kafka+2.5">KIP-531</a>
|
||||
for details.</li>
|
||||
</ul>
|
||||
|
||||
<h4><a id="upgrade_2_4_0" href="#upgrade_2_4_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x or 2.3.x to 2.4.0</a></h4>
|
||||
|
|
|
|||
|
|
@ -22,20 +22,17 @@ ext {
|
|||
libs = [:]
|
||||
|
||||
// Enabled by default when commands like `testAll` are invoked
|
||||
defaultScalaVersions = [ '2.11', '2.12', '2.13' ]
|
||||
defaultScalaVersions = [ '2.12', '2.13' ]
|
||||
// Available if -PscalaVersion is used. This is useful when we want to support a Scala version that has
|
||||
// a higher minimum Java requirement than Kafka. This was previously the case for Scala 2.12 and Java 7.
|
||||
availableScalaVersions = [ '2.11', '2.12', '2.13' ]
|
||||
availableScalaVersions = [ '2.12', '2.13' ]
|
||||
}
|
||||
|
||||
// Add Scala version
|
||||
def defaultScala211Version = '2.11.12'
|
||||
def defaultScala212Version = '2.12.10'
|
||||
def defaultScala213Version = '2.13.1'
|
||||
if (hasProperty('scalaVersion')) {
|
||||
if (scalaVersion == '2.11') {
|
||||
versions["scala"] = defaultScala211Version
|
||||
} else if (scalaVersion == '2.12') {
|
||||
if (scalaVersion == '2.12') {
|
||||
versions["scala"] = defaultScala212Version
|
||||
} else if (scalaVersion == '2.13') {
|
||||
versions["scala"] = defaultScala213Version
|
||||
|
|
@ -47,7 +44,7 @@ if (hasProperty('scalaVersion')) {
|
|||
}
|
||||
|
||||
/* Resolve base Scala version according to these patterns:
|
||||
1. generally available Scala versions (such as: 2.11.x, 2.12.y and 2.13.z) corresponding base versions will be: 2.11, 2.12 and 2.13 (respectively)
|
||||
1. generally available Scala versions (such as: 2.12.y and 2.13.z) corresponding base versions will be: 2.12 and 2.13 (respectively)
|
||||
2. pre-release Scala versions (i.e. milestone/rc, such as: 2.13.0-M5, 2.13.0-RC1, 2.14.0-M1, etc.) will have identical base versions;
|
||||
rationale: pre-release Scala versions are not binary compatible with each other and that's the reason why libraries include the full
|
||||
Scala release string in their name for pre-releases (see dependencies below with an artifact name suffix '_$versions.baseScala')
|
||||
|
|
|
|||
|
|
@ -28,86 +28,59 @@ import java.lang.{Iterable => JIterable}
|
|||
object FunctionConversions {
|
||||
|
||||
implicit private[scala] class ForeachActionFromFunction[K, V](val p: (K, V) => Unit) extends AnyVal {
|
||||
def asForeachAction: ForeachAction[K, V] = new ForeachAction[K, V] {
|
||||
override def apply(key: K, value: V): Unit = p(key, value)
|
||||
}
|
||||
def asForeachAction: ForeachAction[K, V] = (key, value) => p(key, value)
|
||||
}
|
||||
|
||||
implicit class PredicateFromFunction[K, V](val p: (K, V) => Boolean) extends AnyVal {
|
||||
def asPredicate: Predicate[K, V] = new Predicate[K, V] {
|
||||
override def test(key: K, value: V): Boolean = p(key, value)
|
||||
}
|
||||
def asPredicate: Predicate[K, V] = (key: K, value: V) => p(key, value)
|
||||
}
|
||||
|
||||
implicit class MapperFromFunction[T, U, VR](val f: (T, U) => VR) extends AnyVal {
|
||||
def asKeyValueMapper: KeyValueMapper[T, U, VR] = new KeyValueMapper[T, U, VR] {
|
||||
override def apply(key: T, value: U): VR = f(key, value)
|
||||
}
|
||||
def asValueJoiner: ValueJoiner[T, U, VR] = new ValueJoiner[T, U, VR] {
|
||||
override def apply(value1: T, value2: U): VR = f(value1, value2)
|
||||
}
|
||||
def asKeyValueMapper: KeyValueMapper[T, U, VR] = (key: T, value: U) => f(key, value)
|
||||
def asValueJoiner: ValueJoiner[T, U, VR] = (value1: T, value2: U) => f(value1, value2)
|
||||
}
|
||||
|
||||
implicit class KeyValueMapperFromFunction[K, V, KR, VR](val f: (K, V) => (KR, VR)) extends AnyVal {
|
||||
def asKeyValueMapper: KeyValueMapper[K, V, KeyValue[KR, VR]] = new KeyValueMapper[K, V, KeyValue[KR, VR]] {
|
||||
override def apply(key: K, value: V): KeyValue[KR, VR] = {
|
||||
val (kr, vr) = f(key, value)
|
||||
KeyValue.pair(kr, vr)
|
||||
}
|
||||
def asKeyValueMapper: KeyValueMapper[K, V, KeyValue[KR, VR]] = (key: K, value: V) => {
|
||||
val (kr, vr) = f(key, value)
|
||||
KeyValue.pair(kr, vr)
|
||||
}
|
||||
}
|
||||
|
||||
implicit class ValueMapperFromFunction[V, VR](val f: V => VR) extends AnyVal {
|
||||
def asValueMapper: ValueMapper[V, VR] = new ValueMapper[V, VR] {
|
||||
override def apply(value: V): VR = f(value)
|
||||
}
|
||||
def asValueMapper: ValueMapper[V, VR] = (value: V) => f(value)
|
||||
}
|
||||
|
||||
implicit class FlatValueMapperFromFunction[V, VR](val f: V => Iterable[VR]) extends AnyVal {
|
||||
def asValueMapper: ValueMapper[V, JIterable[VR]] = new ValueMapper[V, JIterable[VR]] {
|
||||
override def apply(value: V): JIterable[VR] = f(value).asJava
|
||||
}
|
||||
def asValueMapper: ValueMapper[V, JIterable[VR]] = (value: V) => f(value).asJava
|
||||
}
|
||||
|
||||
implicit class ValueMapperWithKeyFromFunction[K, V, VR](val f: (K, V) => VR) extends AnyVal {
|
||||
def asValueMapperWithKey: ValueMapperWithKey[K, V, VR] = new ValueMapperWithKey[K, V, VR] {
|
||||
override def apply(readOnlyKey: K, value: V): VR = f(readOnlyKey, value)
|
||||
}
|
||||
def asValueMapperWithKey: ValueMapperWithKey[K, V, VR] = (readOnlyKey: K, value: V) => f(readOnlyKey, value)
|
||||
}
|
||||
|
||||
implicit class FlatValueMapperWithKeyFromFunction[K, V, VR](val f: (K, V) => Iterable[VR]) extends AnyVal {
|
||||
def asValueMapperWithKey: ValueMapperWithKey[K, V, JIterable[VR]] = new ValueMapperWithKey[K, V, JIterable[VR]] {
|
||||
override def apply(readOnlyKey: K, value: V): JIterable[VR] = f(readOnlyKey, value).asJava
|
||||
}
|
||||
def asValueMapperWithKey: ValueMapperWithKey[K, V, JIterable[VR]] =
|
||||
(readOnlyKey: K, value: V) => f(readOnlyKey, value).asJava
|
||||
}
|
||||
|
||||
implicit class AggregatorFromFunction[K, V, VA](val f: (K, V, VA) => VA) extends AnyVal {
|
||||
def asAggregator: Aggregator[K, V, VA] = new Aggregator[K, V, VA] {
|
||||
override def apply(key: K, value: V, aggregate: VA): VA = f(key, value, aggregate)
|
||||
}
|
||||
def asAggregator: Aggregator[K, V, VA] = (key: K, value: V, aggregate: VA) => f(key, value, aggregate)
|
||||
}
|
||||
|
||||
implicit class MergerFromFunction[K, VR](val f: (K, VR, VR) => VR) extends AnyVal {
|
||||
def asMerger: Merger[K, VR] = new Merger[K, VR] {
|
||||
override def apply(aggKey: K, aggOne: VR, aggTwo: VR): VR = f(aggKey, aggOne, aggTwo)
|
||||
}
|
||||
def asMerger: Merger[K, VR] = (aggKey: K, aggOne: VR, aggTwo: VR) => f(aggKey, aggOne, aggTwo)
|
||||
}
|
||||
|
||||
implicit class ReducerFromFunction[V](val f: (V, V) => V) extends AnyVal {
|
||||
def asReducer: Reducer[V] = new Reducer[V] {
|
||||
override def apply(value1: V, value2: V): V = f(value1, value2)
|
||||
}
|
||||
def asReducer: Reducer[V] = (value1: V, value2: V) => f(value1, value2)
|
||||
}
|
||||
|
||||
implicit class InitializerFromFunction[VA](val f: () => VA) extends AnyVal {
|
||||
def asInitializer: Initializer[VA] = new Initializer[VA] {
|
||||
override def apply(): VA = f()
|
||||
}
|
||||
def asInitializer: Initializer[VA] = () => f()
|
||||
}
|
||||
|
||||
implicit class TransformerSupplierFromFunction[K, V, VO](val f: () => Transformer[K, V, VO]) extends AnyVal {
|
||||
def asTransformerSupplier: TransformerSupplier[K, V, VO] = new TransformerSupplier[K, V, VO] {
|
||||
override def get(): Transformer[K, V, VO] = f()
|
||||
}
|
||||
def asTransformerSupplier: TransformerSupplier[K, V, VO] = () => f()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -27,141 +27,102 @@ import org.apache.kafka.streams.processor.ProcessorContext
|
|||
* Implicit classes that offer conversions of Scala function literals to
|
||||
* SAM (Single Abstract Method) objects in Java. These make the Scala APIs much
|
||||
* more expressive, with less boilerplate and more succinct.
|
||||
* <p>
|
||||
* For Scala 2.11, most of these conversions need to be invoked explicitly, as Scala 2.11 does not
|
||||
* have full support for SAM types.
|
||||
*/
|
||||
private[scala] object FunctionsCompatConversions {
|
||||
|
||||
implicit class ForeachActionFromFunction[K, V](val p: (K, V) => Unit) extends AnyVal {
|
||||
def asForeachAction: ForeachAction[K, V] = new ForeachAction[K, V] {
|
||||
override def apply(key: K, value: V): Unit = p(key, value)
|
||||
}
|
||||
def asForeachAction: ForeachAction[K, V] = (key: K, value: V) => p(key, value)
|
||||
}
|
||||
|
||||
implicit class PredicateFromFunction[K, V](val p: (K, V) => Boolean) extends AnyVal {
|
||||
def asPredicate: Predicate[K, V] = new Predicate[K, V] {
|
||||
override def test(key: K, value: V): Boolean = p(key, value)
|
||||
}
|
||||
def asPredicate: Predicate[K, V] = (key: K, value: V) => p(key, value)
|
||||
}
|
||||
|
||||
implicit class MapperFromFunction[T, U, VR](val f: (T, U) => VR) extends AnyVal {
|
||||
def asKeyValueMapper: KeyValueMapper[T, U, VR] = new KeyValueMapper[T, U, VR] {
|
||||
override def apply(key: T, value: U): VR = f(key, value)
|
||||
}
|
||||
def asValueJoiner: ValueJoiner[T, U, VR] = new ValueJoiner[T, U, VR] {
|
||||
override def apply(value1: T, value2: U): VR = f(value1, value2)
|
||||
}
|
||||
def asKeyValueMapper: KeyValueMapper[T, U, VR] = (key: T, value: U) => f(key, value)
|
||||
def asValueJoiner: ValueJoiner[T, U, VR] = (value1: T, value2: U) => f(value1, value2)
|
||||
}
|
||||
|
||||
implicit class KeyValueMapperFromFunction[K, V, KR, VR](val f: (K, V) => (KR, VR)) extends AnyVal {
|
||||
def asKeyValueMapper: KeyValueMapper[K, V, KeyValue[KR, VR]] = new KeyValueMapper[K, V, KeyValue[KR, VR]] {
|
||||
override def apply(key: K, value: V): KeyValue[KR, VR] = {
|
||||
val (kr, vr) = f(key, value)
|
||||
KeyValue.pair(kr, vr)
|
||||
}
|
||||
def asKeyValueMapper: KeyValueMapper[K, V, KeyValue[KR, VR]] = (key: K, value: V) => {
|
||||
val (kr, vr) = f(key, value)
|
||||
KeyValue.pair(kr, vr)
|
||||
}
|
||||
}
|
||||
|
||||
implicit class FunctionFromFunction[V, VR](val f: V => VR) extends AnyVal {
|
||||
def asJavaFunction: java.util.function.Function[V, VR] = new java.util.function.Function[V, VR] {
|
||||
override def apply(value: V): VR = f(value)
|
||||
}
|
||||
def asJavaFunction: java.util.function.Function[V, VR] = (value: V) => f(value)
|
||||
}
|
||||
|
||||
implicit class ValueMapperFromFunction[V, VR](val f: V => VR) extends AnyVal {
|
||||
def asValueMapper: ValueMapper[V, VR] = new ValueMapper[V, VR] {
|
||||
override def apply(value: V): VR = f(value)
|
||||
}
|
||||
def asValueMapper: ValueMapper[V, VR] = (value: V) => f(value)
|
||||
}
|
||||
|
||||
implicit class FlatValueMapperFromFunction[V, VR](val f: V => Iterable[VR]) extends AnyVal {
|
||||
def asValueMapper: ValueMapper[V, JIterable[VR]] = new ValueMapper[V, JIterable[VR]] {
|
||||
override def apply(value: V): JIterable[VR] = f(value).asJava
|
||||
}
|
||||
def asValueMapper: ValueMapper[V, JIterable[VR]] = (value: V) => f(value).asJava
|
||||
}
|
||||
|
||||
implicit class ValueMapperWithKeyFromFunction[K, V, VR](val f: (K, V) => VR) extends AnyVal {
|
||||
def asValueMapperWithKey: ValueMapperWithKey[K, V, VR] = new ValueMapperWithKey[K, V, VR] {
|
||||
override def apply(readOnlyKey: K, value: V): VR = f(readOnlyKey, value)
|
||||
}
|
||||
def asValueMapperWithKey: ValueMapperWithKey[K, V, VR] = (readOnlyKey: K, value: V) => f(readOnlyKey, value)
|
||||
}
|
||||
|
||||
implicit class FlatValueMapperWithKeyFromFunction[K, V, VR](val f: (K, V) => Iterable[VR]) extends AnyVal {
|
||||
def asValueMapperWithKey: ValueMapperWithKey[K, V, JIterable[VR]] = new ValueMapperWithKey[K, V, JIterable[VR]] {
|
||||
override def apply(readOnlyKey: K, value: V): JIterable[VR] = f(readOnlyKey, value).asJava
|
||||
}
|
||||
def asValueMapperWithKey: ValueMapperWithKey[K, V, JIterable[VR]] =
|
||||
(readOnlyKey: K, value: V) => f(readOnlyKey, value).asJava
|
||||
}
|
||||
|
||||
implicit class AggregatorFromFunction[K, V, VA](val f: (K, V, VA) => VA) extends AnyVal {
|
||||
def asAggregator: Aggregator[K, V, VA] = new Aggregator[K, V, VA] {
|
||||
override def apply(key: K, value: V, aggregate: VA): VA = f(key, value, aggregate)
|
||||
}
|
||||
def asAggregator: Aggregator[K, V, VA] = (key: K, value: V, aggregate: VA) => f(key, value, aggregate)
|
||||
}
|
||||
|
||||
implicit class MergerFromFunction[K, VR](val f: (K, VR, VR) => VR) extends AnyVal {
|
||||
def asMerger: Merger[K, VR] = new Merger[K, VR] {
|
||||
override def apply(aggKey: K, aggOne: VR, aggTwo: VR): VR = f(aggKey, aggOne, aggTwo)
|
||||
}
|
||||
def asMerger: Merger[K, VR] = (aggKey: K, aggOne: VR, aggTwo: VR) => f(aggKey, aggOne, aggTwo)
|
||||
}
|
||||
|
||||
implicit class ReducerFromFunction[V](val f: (V, V) => V) extends AnyVal {
|
||||
def asReducer: Reducer[V] = new Reducer[V] {
|
||||
override def apply(value1: V, value2: V): V = f(value1, value2)
|
||||
}
|
||||
def asReducer: Reducer[V] = (value1: V, value2: V) => f(value1, value2)
|
||||
}
|
||||
|
||||
implicit class InitializerFromFunction[VA](val f: () => VA) extends AnyVal {
|
||||
def asInitializer: Initializer[VA] = new Initializer[VA] {
|
||||
override def apply(): VA = f()
|
||||
}
|
||||
def asInitializer: Initializer[VA] = () => f()
|
||||
}
|
||||
|
||||
implicit class TransformerSupplierFromFunction[K, V, VO](val f: () => Transformer[K, V, VO]) extends AnyVal {
|
||||
def asTransformerSupplier: TransformerSupplier[K, V, VO] = new TransformerSupplier[K, V, VO] {
|
||||
override def get(): Transformer[K, V, VO] = f()
|
||||
}
|
||||
def asTransformerSupplier: TransformerSupplier[K, V, VO] = () => f()
|
||||
}
|
||||
|
||||
implicit class TransformerSupplierAsJava[K, V, VO](val supplier: TransformerSupplier[K, V, Iterable[VO]])
|
||||
extends AnyVal {
|
||||
def asJava: TransformerSupplier[K, V, JIterable[VO]] = new TransformerSupplier[K, V, JIterable[VO]] {
|
||||
override def get(): Transformer[K, V, JIterable[VO]] = {
|
||||
val innerTransformer = supplier.get()
|
||||
new Transformer[K, V, JIterable[VO]] {
|
||||
override def transform(key: K, value: V): JIterable[VO] = innerTransformer.transform(key, value).asJava
|
||||
override def init(context: ProcessorContext): Unit = innerTransformer.init(context)
|
||||
override def close(): Unit = innerTransformer.close()
|
||||
}
|
||||
def asJava: TransformerSupplier[K, V, JIterable[VO]] = () => {
|
||||
val innerTransformer = supplier.get()
|
||||
new Transformer[K, V, JIterable[VO]] {
|
||||
override def transform(key: K, value: V): JIterable[VO] = innerTransformer.transform(key, value).asJava
|
||||
override def init(context: ProcessorContext): Unit = innerTransformer.init(context)
|
||||
override def close(): Unit = innerTransformer.close()
|
||||
}
|
||||
}
|
||||
}
|
||||
implicit class ValueTransformerSupplierAsJava[V, VO](val supplier: ValueTransformerSupplier[V, Iterable[VO]])
|
||||
extends AnyVal {
|
||||
def asJava: ValueTransformerSupplier[V, JIterable[VO]] = new ValueTransformerSupplier[V, JIterable[VO]] {
|
||||
override def get(): ValueTransformer[V, JIterable[VO]] = {
|
||||
val innerTransformer = supplier.get()
|
||||
new ValueTransformer[V, JIterable[VO]] {
|
||||
override def transform(value: V): JIterable[VO] = innerTransformer.transform(value).asJava
|
||||
override def init(context: ProcessorContext): Unit = innerTransformer.init(context)
|
||||
override def close(): Unit = innerTransformer.close()
|
||||
}
|
||||
def asJava: ValueTransformerSupplier[V, JIterable[VO]] = () => {
|
||||
val innerTransformer = supplier.get()
|
||||
new ValueTransformer[V, JIterable[VO]] {
|
||||
override def transform(value: V): JIterable[VO] = innerTransformer.transform(value).asJava
|
||||
override def init(context: ProcessorContext): Unit = innerTransformer.init(context)
|
||||
override def close(): Unit = innerTransformer.close()
|
||||
}
|
||||
}
|
||||
}
|
||||
implicit class ValueTransformerSupplierWithKeyAsJava[K, V, VO](
|
||||
val supplier: ValueTransformerWithKeySupplier[K, V, Iterable[VO]]
|
||||
) extends AnyVal {
|
||||
def asJava: ValueTransformerWithKeySupplier[K, V, JIterable[VO]] =
|
||||
new ValueTransformerWithKeySupplier[K, V, JIterable[VO]] {
|
||||
override def get(): ValueTransformerWithKey[K, V, JIterable[VO]] = {
|
||||
val innerTransformer = supplier.get()
|
||||
new ValueTransformerWithKey[K, V, JIterable[VO]] {
|
||||
override def transform(key: K, value: V): JIterable[VO] = innerTransformer.transform(key, value).asJava
|
||||
override def init(context: ProcessorContext): Unit = innerTransformer.init(context)
|
||||
override def close(): Unit = innerTransformer.close()
|
||||
}
|
||||
}
|
||||
def asJava: ValueTransformerWithKeySupplier[K, V, JIterable[VO]] = () => {
|
||||
val innerTransformer = supplier.get()
|
||||
new ValueTransformerWithKey[K, V, JIterable[VO]] {
|
||||
override def transform(key: K, value: V): JIterable[VO] = innerTransformer.transform(key, value).asJava
|
||||
override def init(context: ProcessorContext): Unit = innerTransformer.init(context)
|
||||
override def close(): Unit = innerTransformer.close()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -78,8 +78,6 @@ class KGroupedStream[K, V](val inner: KGroupedStreamJ[K, V]) {
|
|||
* @see `org.apache.kafka.streams.kstream.KGroupedStream#reduce`
|
||||
*/
|
||||
def reduce(reducer: (V, V) => V)(implicit materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
|
||||
// need this explicit asReducer for Scala 2.11 or else the SAM conversion doesn't take place
|
||||
// works perfectly with Scala 2.12 though
|
||||
new KTable(inner.reduce(reducer.asReducer, materialized))
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -66,8 +66,6 @@ class KGroupedTable[K, V](inner: KGroupedTableJ[K, V]) {
|
|||
*/
|
||||
def reduce(adder: (V, V) => V,
|
||||
subtractor: (V, V) => V)(implicit materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
|
||||
// need this explicit asReducer for Scala 2.11 or else the SAM conversion doesn't take place
|
||||
// works perfectly with Scala 2.12 though
|
||||
new KTable(inner.reduce(adder.asReducer, subtractor.asReducer, materialized))
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -424,10 +424,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @see `org.apache.kafka.streams.kstream.KStream#process`
|
||||
*/
|
||||
def process(processorSupplier: () => Processor[K, V], stateStoreNames: String*): Unit = {
|
||||
//noinspection ConvertExpressionToSAM // because of the 2.11 build
|
||||
val processorSupplierJ: ProcessorSupplier[K, V] = new ProcessorSupplier[K, V] {
|
||||
override def get(): Processor[K, V] = processorSupplier()
|
||||
}
|
||||
val processorSupplierJ: ProcessorSupplier[K, V] = () => processorSupplier()
|
||||
inner.process(processorSupplierJ, stateStoreNames: _*)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -34,10 +34,10 @@ import org.apache.kafka.streams.kstream.internals.suppress.{
|
|||
/**
|
||||
* Duplicates the static factory methods inside the Java interface [[org.apache.kafka.streams.kstream.Suppressed]].
|
||||
*
|
||||
* This is required for compatibility w/ Scala 2.11 + Java 1.8 because the Scala 2.11 compiler doesn't support the use
|
||||
* of static methods inside Java interfaces.
|
||||
* This was required for compatibility w/ Scala 2.11 + Java 1.8 because the Scala 2.11 compiler doesn't support the use
|
||||
* of static methods inside Java interfaces. We have since dropped Scala 2.11 support.
|
||||
*/
|
||||
// TODO: Deprecate this class if support for Scala 2.11 + Java 1.8 is dropped.
|
||||
@deprecated(message = "Use org.apache.kafka.streams.kstream.Suppressed", since = "2.5")
|
||||
object Suppressed {
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -31,9 +31,6 @@ import org.junit.experimental.categories.Category
|
|||
* <p>
|
||||
* The suite contains the test case using Scala APIs `testShouldCountClicksPerRegion` and the same test case using the
|
||||
* Java APIs `testShouldCountClicksPerRegionJava`. The idea is to demonstrate that both generate the same result.
|
||||
* <p>
|
||||
* Note: In the current project settings SAM type conversion is turned off as it's experimental in Scala 2.11.
|
||||
* Hence the native Java API based version is more verbose.
|
||||
*/
|
||||
@Category(Array(classOf[IntegrationTest]))
|
||||
class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends StreamToTableJoinScalaIntegrationTestBase {
|
||||
|
|
@ -81,7 +78,8 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends StreamToTableJ
|
|||
streams.close()
|
||||
}
|
||||
|
||||
@Test def testShouldCountClicksPerRegionWithNamedRepartitionTopic(): Unit = {
|
||||
@Test
|
||||
def testShouldCountClicksPerRegionWithNamedRepartitionTopic(): Unit = {
|
||||
|
||||
// DefaultSerdes brings into scope implicit serdes (mostly for primitives) that will set up all Grouped, Produced,
|
||||
// Consumed and Joined instances. So all APIs below that accept Grouped, Produced, Consumed or Joined will
|
||||
|
|
@ -124,7 +122,8 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends StreamToTableJ
|
|||
streams.close()
|
||||
}
|
||||
|
||||
@Test def testShouldCountClicksPerRegionJava(): Unit = {
|
||||
@Test
|
||||
def testShouldCountClicksPerRegionJava(): Unit = {
|
||||
|
||||
import java.lang.{Long => JLong}
|
||||
|
||||
|
|
@ -145,38 +144,28 @@ class StreamToTableJoinScalaIntegrationTestImplicitSerdes extends StreamToTableJ
|
|||
builder.table[String, String](userRegionsTopicJ, Consumed.`with`(Serdes.String, Serdes.String))
|
||||
|
||||
// Join the stream against the table.
|
||||
val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream
|
||||
.leftJoin(
|
||||
userRegionsTable,
|
||||
new ValueJoiner[JLong, String, (String, JLong)] {
|
||||
def apply(clicks: JLong, region: String): (String, JLong) =
|
||||
(if (region == null) "UNKNOWN" else region, clicks)
|
||||
},
|
||||
Joined.`with`[String, JLong, String](Serdes.String, Serdes.JavaLong, Serdes.String)
|
||||
)
|
||||
val valueJoinerJ: ValueJoiner[JLong, String, (String, JLong)] =
|
||||
(clicks: JLong, region: String) => (if (region == null) "UNKNOWN" else region, clicks)
|
||||
val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream.leftJoin(
|
||||
userRegionsTable,
|
||||
valueJoinerJ,
|
||||
Joined.`with`[String, JLong, String](Serdes.String, Serdes.JavaLong, Serdes.String)
|
||||
)
|
||||
|
||||
// Change the stream from <user> -> <region, clicks> to <region> -> <clicks>
|
||||
val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion
|
||||
.map {
|
||||
new KeyValueMapper[String, (String, JLong), KeyValue[String, JLong]] {
|
||||
def apply(k: String, regionWithClicks: (String, JLong)) =
|
||||
new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2)
|
||||
}
|
||||
}
|
||||
val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion.map { (_, regionWithClicks) =>
|
||||
new KeyValue(regionWithClicks._1, regionWithClicks._2)
|
||||
}
|
||||
|
||||
// Compute the total per region by summing the individual click counts per region.
|
||||
val clicksPerRegion: KTableJ[String, JLong] = clicksByRegion
|
||||
.groupByKey(Grouped.`with`[String, JLong](Serdes.String, Serdes.JavaLong))
|
||||
.reduce {
|
||||
new Reducer[JLong] {
|
||||
def apply(v1: JLong, v2: JLong): JLong = v1 + v2
|
||||
}
|
||||
}
|
||||
.groupByKey(Grouped.`with`(Serdes.String, Serdes.JavaLong))
|
||||
.reduce((v1, v2) => v1 + v2)
|
||||
|
||||
// Write the (continuously updating) results to the output topic.
|
||||
clicksPerRegion.toStream.to(outputTopicJ, Produced.`with`(Serdes.String, Serdes.JavaLong))
|
||||
|
||||
val streams: KafkaStreamsJ = new KafkaStreamsJ(builder.build(), streamsConfiguration)
|
||||
val streams = new KafkaStreamsJ(builder.build(), streamsConfiguration)
|
||||
|
||||
streams.start()
|
||||
produceNConsume(userClicksTopicJ, userRegionsTopicJ, outputTopicJ)
|
||||
|
|
|
|||
|
|
@ -27,14 +27,11 @@ import java.util.regex.Pattern
|
|||
import org.apache.kafka.common.serialization.{Serdes => SerdesJ}
|
||||
import org.apache.kafka.streams.kstream.{
|
||||
Aggregator,
|
||||
ForeachAction,
|
||||
Initializer,
|
||||
JoinWindows,
|
||||
KeyValueMapper,
|
||||
Predicate,
|
||||
Reducer,
|
||||
Transformer,
|
||||
TransformerSupplier,
|
||||
ValueJoiner,
|
||||
ValueMapper,
|
||||
StreamJoined => StreamJoinedJ,
|
||||
|
|
@ -64,7 +61,8 @@ class TopologyTest {
|
|||
|
||||
private val pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS)
|
||||
|
||||
@Test def shouldBuildIdenticalTopologyInJavaNScalaSimple(): Unit = {
|
||||
@Test
|
||||
def shouldBuildIdenticalTopologyInJavaNScalaSimple(): Unit = {
|
||||
|
||||
// build the Scala topology
|
||||
def getTopologyScala: TopologyDescription = {
|
||||
|
|
@ -74,23 +72,16 @@ class TopologyTest {
|
|||
val streamBuilder = new StreamsBuilder
|
||||
val textLines = streamBuilder.stream[String, String](inputTopic)
|
||||
|
||||
val _: KStream[String, String] =
|
||||
textLines.flatMapValues(v => pattern.split(v.toLowerCase))
|
||||
val _: KStream[String, String] = textLines.flatMapValues(v => pattern.split(v.toLowerCase))
|
||||
|
||||
streamBuilder.build().describe()
|
||||
}
|
||||
|
||||
// build the Java topology
|
||||
def getTopologyJava: TopologyDescription = {
|
||||
|
||||
val streamBuilder = new StreamsBuilderJ
|
||||
val textLines = streamBuilder.stream[String, String](inputTopic)
|
||||
|
||||
val _: KStreamJ[String, String] = textLines.flatMapValues(
|
||||
new ValueMapper[String, java.lang.Iterable[String]] {
|
||||
def apply(s: String): java.lang.Iterable[String] = pattern.split(s.toLowerCase).toIterable.asJava
|
||||
}
|
||||
)
|
||||
val _: KStreamJ[String, String] = textLines.flatMapValues(s => pattern.split(s.toLowerCase).toIterable.asJava)
|
||||
streamBuilder.build().describe()
|
||||
}
|
||||
|
||||
|
|
@ -98,7 +89,8 @@ class TopologyTest {
|
|||
assertEquals(getTopologyScala, getTopologyJava)
|
||||
}
|
||||
|
||||
@Test def shouldBuildIdenticalTopologyInJavaNScalaAggregate(): Unit = {
|
||||
@Test
|
||||
def shouldBuildIdenticalTopologyInJavaNScalaAggregate(): Unit = {
|
||||
|
||||
// build the Scala topology
|
||||
def getTopologyScala: TopologyDescription = {
|
||||
|
|
@ -123,17 +115,10 @@ class TopologyTest {
|
|||
val streamBuilder = new StreamsBuilderJ
|
||||
val textLines: KStreamJ[String, String] = streamBuilder.stream[String, String](inputTopic)
|
||||
|
||||
val splits: KStreamJ[String, String] = textLines.flatMapValues(
|
||||
new ValueMapper[String, java.lang.Iterable[String]] {
|
||||
def apply(s: String): java.lang.Iterable[String] = pattern.split(s.toLowerCase).toIterable.asJava
|
||||
}
|
||||
)
|
||||
val splits: KStreamJ[String, String] =
|
||||
textLines.flatMapValues(s => pattern.split(s.toLowerCase).toIterable.asJava)
|
||||
|
||||
val grouped: KGroupedStreamJ[String, String] = splits.groupBy {
|
||||
new KeyValueMapper[String, String, String] {
|
||||
def apply(k: String, v: String): String = v
|
||||
}
|
||||
}
|
||||
val grouped: KGroupedStreamJ[String, String] = splits.groupBy((_, v) => v)
|
||||
|
||||
grouped.count()
|
||||
|
||||
|
|
@ -144,7 +129,8 @@ class TopologyTest {
|
|||
assertEquals(getTopologyScala, getTopologyJava)
|
||||
}
|
||||
|
||||
@Test def shouldBuildIdenticalTopologyInJavaNScalaJoin(): Unit = {
|
||||
@Test
|
||||
def shouldBuildIdenticalTopologyInJavaNScalaJoin(): Unit = {
|
||||
|
||||
// build the Scala topology
|
||||
def getTopologyScala: TopologyDescription = {
|
||||
|
|
@ -180,33 +166,23 @@ class TopologyTest {
|
|||
builder.table[String, String](userRegionsTopic, Consumed.`with`[String, String])
|
||||
|
||||
// Join the stream against the table.
|
||||
val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream
|
||||
.leftJoin(
|
||||
userRegionsTable,
|
||||
new ValueJoiner[JLong, String, (String, JLong)] {
|
||||
def apply(clicks: JLong, region: String): (String, JLong) =
|
||||
(if (region == null) "UNKNOWN" else region, clicks)
|
||||
},
|
||||
Joined.`with`[String, JLong, String]
|
||||
)
|
||||
val valueJoinerJ: ValueJoiner[JLong, String, (String, JLong)] =
|
||||
(clicks: JLong, region: String) => (if (region == null) "UNKNOWN" else region, clicks)
|
||||
val userClicksJoinRegion: KStreamJ[String, (String, JLong)] = userClicksStream.leftJoin(
|
||||
userRegionsTable,
|
||||
valueJoinerJ,
|
||||
Joined.`with`[String, JLong, String]
|
||||
)
|
||||
|
||||
// Change the stream from <user> -> <region, clicks> to <region> -> <clicks>
|
||||
val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion
|
||||
.map {
|
||||
new KeyValueMapper[String, (String, JLong), KeyValue[String, JLong]] {
|
||||
def apply(k: String, regionWithClicks: (String, JLong)) =
|
||||
new KeyValue[String, JLong](regionWithClicks._1, regionWithClicks._2)
|
||||
}
|
||||
}
|
||||
val clicksByRegion: KStreamJ[String, JLong] = userClicksJoinRegion.map { (_, regionWithClicks) =>
|
||||
new KeyValue(regionWithClicks._1, regionWithClicks._2)
|
||||
}
|
||||
|
||||
// Compute the total per region by summing the individual click counts per region.
|
||||
clicksByRegion
|
||||
.groupByKey(Grouped.`with`[String, JLong])
|
||||
.reduce {
|
||||
new Reducer[JLong] {
|
||||
def apply(v1: JLong, v2: JLong): JLong = v1 + v2
|
||||
}
|
||||
}
|
||||
.reduce((v1, v2) => v1 + v2)
|
||||
|
||||
builder.build().describe()
|
||||
}
|
||||
|
|
@ -215,7 +191,8 @@ class TopologyTest {
|
|||
assertEquals(getTopologyScala, getTopologyJava)
|
||||
}
|
||||
|
||||
@Test def shouldBuildIdenticalTopologyInJavaNScalaTransform(): Unit = {
|
||||
@Test
|
||||
def shouldBuildIdenticalTopologyInJavaNScalaTransform(): Unit = {
|
||||
|
||||
// build the Scala topology
|
||||
def getTopologyScala: TopologyDescription = {
|
||||
|
|
@ -225,21 +202,18 @@ class TopologyTest {
|
|||
val streamBuilder = new StreamsBuilder
|
||||
val textLines = streamBuilder.stream[String, String](inputTopic)
|
||||
|
||||
val _: KTable[String, Long] =
|
||||
textLines
|
||||
.transform(new TransformerSupplier[String, String, KeyValue[String, String]] {
|
||||
override def get(): Transformer[String, String, KeyValue[String, String]] =
|
||||
new Transformer[String, String, KeyValue[String, String]] {
|
||||
override def init(context: ProcessorContext): Unit = ()
|
||||
|
||||
override def transform(key: String, value: String): KeyValue[String, String] =
|
||||
new KeyValue(key, value.toLowerCase)
|
||||
|
||||
override def close(): Unit = ()
|
||||
}
|
||||
})
|
||||
.groupBy((_, v) => v)
|
||||
.count()
|
||||
val _: KTable[String, Long] = textLines
|
||||
.transform(
|
||||
() =>
|
||||
new Transformer[String, String, KeyValue[String, String]] {
|
||||
override def init(context: ProcessorContext): Unit = ()
|
||||
override def transform(key: String, value: String): KeyValue[String, String] =
|
||||
new KeyValue(key, value.toLowerCase)
|
||||
override def close(): Unit = ()
|
||||
}
|
||||
)
|
||||
.groupBy((_, v) => v)
|
||||
.count()
|
||||
|
||||
streamBuilder.build().describe()
|
||||
}
|
||||
|
|
@ -250,24 +224,17 @@ class TopologyTest {
|
|||
val streamBuilder = new StreamsBuilderJ
|
||||
val textLines: KStreamJ[String, String] = streamBuilder.stream[String, String](inputTopic)
|
||||
|
||||
val lowered: KStreamJ[String, String] = textLines
|
||||
.transform(new TransformerSupplier[String, String, KeyValue[String, String]] {
|
||||
override def get(): Transformer[String, String, KeyValue[String, String]] =
|
||||
new Transformer[String, String, KeyValue[String, String]] {
|
||||
override def init(context: ProcessorContext): Unit = ()
|
||||
|
||||
override def transform(key: String, value: String): KeyValue[String, String] =
|
||||
new KeyValue(key, value.toLowerCase)
|
||||
|
||||
override def close(): Unit = ()
|
||||
}
|
||||
})
|
||||
|
||||
val grouped: KGroupedStreamJ[String, String] = lowered.groupBy {
|
||||
new KeyValueMapper[String, String, String] {
|
||||
def apply(k: String, v: String): String = v
|
||||
val lowered: KStreamJ[String, String] = textLines.transform(
|
||||
() =>
|
||||
new Transformer[String, String, KeyValue[String, String]] {
|
||||
override def init(context: ProcessorContext): Unit = ()
|
||||
override def transform(key: String, value: String): KeyValue[String, String] =
|
||||
new KeyValue(key, value.toLowerCase)
|
||||
override def close(): Unit = ()
|
||||
}
|
||||
}
|
||||
)
|
||||
|
||||
val grouped: KGroupedStreamJ[String, String] = lowered.groupBy((_, v) => v)
|
||||
|
||||
// word counts
|
||||
grouped.count()
|
||||
|
|
@ -279,7 +246,8 @@ class TopologyTest {
|
|||
assertEquals(getTopologyScala, getTopologyJava)
|
||||
}
|
||||
|
||||
@Test def shouldBuildIdenticalTopologyInJavaNScalaProperties(): Unit = {
|
||||
@Test
|
||||
def shouldBuildIdenticalTopologyInJavaNScalaProperties(): Unit = {
|
||||
|
||||
val props = new Properties()
|
||||
props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE)
|
||||
|
|
@ -345,32 +313,15 @@ class TopologyTest {
|
|||
def getTopologyJava: StreamsBuilderJ = {
|
||||
|
||||
val keyValueMapper: KeyValueMapper[String, String, KeyValue[String, String]] =
|
||||
new KeyValueMapper[String, String, KeyValue[String, String]] {
|
||||
override def apply(key: String, value: String): KeyValue[String, String] =
|
||||
KeyValue.pair(key.toUpperCase(Locale.getDefault), value)
|
||||
}
|
||||
val initializer: Initializer[Integer] = new Initializer[Integer] {
|
||||
override def apply(): Integer = 0
|
||||
}
|
||||
val aggregator: Aggregator[String, String, Integer] = new Aggregator[String, String, Integer] {
|
||||
override def apply(key: String, value: String, aggregate: Integer): Integer = aggregate + value.length
|
||||
}
|
||||
val reducer: Reducer[String] = new Reducer[String] {
|
||||
override def apply(v1: String, v2: String): String = v1 + ":" + v2
|
||||
}
|
||||
val valueMapper: ValueMapper[String, String] = new ValueMapper[String, String] {
|
||||
override def apply(v: String): String = v.toUpperCase(Locale.getDefault)
|
||||
}
|
||||
(key, value) => KeyValue.pair(key.toUpperCase(Locale.getDefault), value)
|
||||
val initializer: Initializer[Integer] = () => 0
|
||||
val aggregator: Aggregator[String, String, Integer] = (_, value, aggregate) => aggregate + value.length
|
||||
val reducer: Reducer[String] = (v1, v2) => v1 + ":" + v2
|
||||
val valueMapper: ValueMapper[String, String] = v => v.toUpperCase(Locale.getDefault)
|
||||
val processorValueCollector = new util.ArrayList[String]
|
||||
val processorSupplier: ProcessorSupplier[String, String] = new ProcessorSupplier[String, String] {
|
||||
override def get() = new SimpleProcessor(processorValueCollector)
|
||||
}
|
||||
val valueJoiner2: ValueJoiner[String, Integer, String] = new ValueJoiner[String, Integer, String] {
|
||||
override def apply(value1: String, value2: Integer): String = value1 + ":" + value2.toString
|
||||
}
|
||||
val valueJoiner3: ValueJoiner[String, String, String] = new ValueJoiner[String, String, String] {
|
||||
override def apply(value1: String, value2: String): String = value1 + ":" + value2.toString
|
||||
}
|
||||
val processorSupplier: ProcessorSupplier[String, String] = () => new SimpleProcessor(processorValueCollector)
|
||||
val valueJoiner2: ValueJoiner[String, Integer, String] = (value1, value2) => value1 + ":" + value2.toString
|
||||
val valueJoiner3: ValueJoiner[String, String, String] = (value1, value2) => value1 + ":" + value2
|
||||
|
||||
val builder = new StreamsBuilderJ
|
||||
|
||||
|
|
@ -379,9 +330,7 @@ class TopologyTest {
|
|||
val mappedStream: KStreamJ[String, String] =
|
||||
sourceStream.map(keyValueMapper)
|
||||
mappedStream
|
||||
.filter(new Predicate[String, String] {
|
||||
override def test(key: String, value: String): Boolean = key == "B"
|
||||
})
|
||||
.filter((key, _) => key == "B")
|
||||
.mapValues[String](valueMapper)
|
||||
.process(processorSupplier)
|
||||
|
||||
|
|
@ -392,21 +341,15 @@ class TopologyTest {
|
|||
|
||||
// adding operators for case where the repartition node is further downstream
|
||||
val stream3 = mappedStream
|
||||
.filter(new Predicate[String, String] {
|
||||
override def test(k: String, v: String) = true
|
||||
})
|
||||
.peek(new ForeachAction[String, String] {
|
||||
override def apply(k: String, v: String) = System.out.println(k + ":" + v)
|
||||
})
|
||||
.filter((_, _) => true)
|
||||
.peek((k, v) => System.out.println(k + ":" + v))
|
||||
.groupByKey
|
||||
.reduce(reducer, MaterializedJ.`with`(Serdes.String, Serdes.String))
|
||||
.toStream
|
||||
stream3.to(REDUCE_TOPIC, Produced.`with`(Serdes.String, Serdes.String))
|
||||
|
||||
mappedStream
|
||||
.filter(new Predicate[String, String] {
|
||||
override def test(key: String, value: String): Boolean = key == "A"
|
||||
})
|
||||
.filter((key, _) => key == "A")
|
||||
.join[Integer, String](stream2,
|
||||
valueJoiner2,
|
||||
JoinWindows.of(Duration.ofMillis(5000)),
|
||||
|
|
@ -414,9 +357,7 @@ class TopologyTest {
|
|||
.to(JOINED_TOPIC)
|
||||
|
||||
mappedStream
|
||||
.filter(new Predicate[String, String] {
|
||||
override def test(key: String, value: String): Boolean = key == "A"
|
||||
})
|
||||
.filter((key, _) => key == "A")
|
||||
.join(stream3,
|
||||
valueJoiner3,
|
||||
JoinWindows.of(Duration.ofMillis(5000)),
|
||||
|
|
|
|||
|
|
@ -41,9 +41,6 @@ import org.junit.experimental.categories.Category
|
|||
* <p>
|
||||
* The suite contains the test case using Scala APIs `testShouldCountWords` and the same test case using the
|
||||
* Java APIs `testShouldCountWordsJava`. The idea is to demonstrate that both generate the same result.
|
||||
* <p>
|
||||
* Note: In the current project settings SAM type conversion is turned off as it's experimental in Scala 2.11.
|
||||
* Hence the native Java API based version is more verbose.
|
||||
*/
|
||||
@Category(Array(classOf[IntegrationTest]))
|
||||
class WordCountTest extends WordCountTestData {
|
||||
|
|
@ -58,6 +55,7 @@ class WordCountTest extends WordCountTestData {
|
|||
|
||||
val tFolder: TemporaryFolder = new TemporaryFolder(TestUtils.tempDirectory())
|
||||
@Rule def testFolder: TemporaryFolder = tFolder
|
||||
|
||||
@Before
|
||||
def startKafkaCluster(): Unit = {
|
||||
cluster.createTopic(inputTopic)
|
||||
|
|
@ -66,7 +64,8 @@ class WordCountTest extends WordCountTestData {
|
|||
cluster.createTopic(outputTopicJ)
|
||||
}
|
||||
|
||||
@Test def testShouldCountWords(): Unit = {
|
||||
@Test
|
||||
def testShouldCountWords(): Unit = {
|
||||
import Serdes._
|
||||
|
||||
val streamsConfiguration = getStreamsConfiguration()
|
||||
|
|
@ -86,7 +85,7 @@ class WordCountTest extends WordCountTestData {
|
|||
// write to output topic
|
||||
wordCounts.toStream.to(outputTopic)
|
||||
|
||||
val streams: KafkaStreams = new KafkaStreams(streamBuilder.build(), streamsConfiguration)
|
||||
val streams = new KafkaStreams(streamBuilder.build(), streamsConfiguration)
|
||||
streams.start()
|
||||
|
||||
// produce and consume synchronously
|
||||
|
|
@ -98,7 +97,8 @@ class WordCountTest extends WordCountTestData {
|
|||
assertEquals(actualWordCounts.asScala.take(expectedWordCounts.size).sortBy(_.key), expectedWordCounts.sortBy(_.key))
|
||||
}
|
||||
|
||||
@Test def testShouldCountWordsMaterialized(): Unit = {
|
||||
@Test
|
||||
def testShouldCountWordsMaterialized(): Unit = {
|
||||
import Serdes._
|
||||
|
||||
val streamsConfiguration = getStreamsConfiguration()
|
||||
|
|
@ -118,7 +118,7 @@ class WordCountTest extends WordCountTestData {
|
|||
// write to output topic
|
||||
wordCounts.toStream.to(outputTopic)
|
||||
|
||||
val streams: KafkaStreams = new KafkaStreams(streamBuilder.build(), streamsConfiguration)
|
||||
val streams = new KafkaStreams(streamBuilder.build(), streamsConfiguration)
|
||||
streams.start()
|
||||
|
||||
// produce and consume synchronously
|
||||
|
|
@ -130,7 +130,8 @@ class WordCountTest extends WordCountTestData {
|
|||
assertEquals(actualWordCounts.asScala.take(expectedWordCounts.size).sortBy(_.key), expectedWordCounts.sortBy(_.key))
|
||||
}
|
||||
|
||||
@Test def testShouldCountWordsJava(): Unit = {
|
||||
@Test
|
||||
def testShouldCountWordsJava(): Unit = {
|
||||
|
||||
import org.apache.kafka.streams.{KafkaStreams => KafkaStreamsJ, StreamsBuilder => StreamsBuilderJ}
|
||||
import org.apache.kafka.streams.kstream.{
|
||||
|
|
@ -150,16 +151,12 @@ class WordCountTest extends WordCountTestData {
|
|||
|
||||
val pattern = Pattern.compile("\\W+", Pattern.UNICODE_CHARACTER_CLASS)
|
||||
|
||||
val splits: KStreamJ[String, String] = textLines.flatMapValues {
|
||||
new ValueMapper[String, java.lang.Iterable[String]] {
|
||||
def apply(s: String): java.lang.Iterable[String] = pattern.split(s.toLowerCase).toIterable.asJava
|
||||
}
|
||||
val splits: KStreamJ[String, String] = textLines.flatMapValues { line =>
|
||||
pattern.split(line.toLowerCase).toIterable.asJava
|
||||
}
|
||||
|
||||
val grouped: KGroupedStreamJ[String, String] = splits.groupBy {
|
||||
new KeyValueMapper[String, String, String] {
|
||||
def apply(k: String, v: String): String = v
|
||||
}
|
||||
val grouped: KGroupedStreamJ[String, String] = splits.groupBy { (_, v) =>
|
||||
v
|
||||
}
|
||||
|
||||
val wordCounts: KTableJ[String, java.lang.Long] = grouped.count()
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue