From 2e0c0ee31f0244393b55d158f2a8d68aceeca941 Mon Sep 17 00:00:00 2001 From: zheolong Date: Tue, 21 Jul 2015 14:25:43 +0800 Subject: [PATCH 01/17] Add logkafka - Collect logs and send lines to Apache Kafka, refer to https://github.com/Qihoo360/logkafka/ --- app/controllers/Logkafka.scala | 210 ++++++++++++++++++ app/kafka/manager/ActorModel.scala | 70 ++++++ app/kafka/manager/BrokerViewCacheActor.scala | 29 +++ app/kafka/manager/ClusterManagerActor.scala | 44 ++++ app/kafka/manager/KafkaCommandActor.scala | 31 +++ app/kafka/manager/KafkaManager.scala | 91 +++++++- app/kafka/manager/KafkaStateActor.scala | 128 +++++++++++ app/kafka/manager/utils/AdminUtils.scala | 83 +++++++ app/kafka/manager/utils/Logkafka.scala | 112 ++++++++++ .../manager/utils/LogkafkaNewConfigs.scala | 44 ++++ app/kafka/manager/utils/ZkUtils.scala | 8 + .../manager/utils/logkafka81/LogConfig.scala | 159 +++++++++++++ .../manager/utils/logkafka82/LogConfig.scala | 159 +++++++++++++ app/models/form/LogkafkaOperation.scala | 19 ++ app/models/navigation/BreadCrumbs.scala | 33 +++ app/models/navigation/Menus.scala | 6 +- app/models/navigation/QuickRoutes.scala | 19 +- app/views/logkafka/createLogkafka.scala.html | 52 +++++ app/views/logkafka/logkafkaList.scala.html | 33 +++ .../logkafka/logkafkaListContent.scala.html | 62 ++++++ app/views/logkafka/logkafkaView.scala.html | 28 +++ .../logkafka/logkafkaViewContent.scala.html | 71 ++++++ app/views/logkafka/updateConfig.scala.html | 53 +++++ conf/routes | 7 + 24 files changed, 1548 insertions(+), 3 deletions(-) create mode 100644 app/controllers/Logkafka.scala create mode 100644 app/kafka/manager/utils/Logkafka.scala create mode 100644 app/kafka/manager/utils/LogkafkaNewConfigs.scala create mode 100644 app/kafka/manager/utils/logkafka81/LogConfig.scala create mode 100644 app/kafka/manager/utils/logkafka82/LogConfig.scala create mode 100644 app/models/form/LogkafkaOperation.scala create mode 100644 app/views/logkafka/createLogkafka.scala.html create mode 100644 app/views/logkafka/logkafkaList.scala.html create mode 100644 app/views/logkafka/logkafkaListContent.scala.html create mode 100644 app/views/logkafka/logkafkaView.scala.html create mode 100644 app/views/logkafka/logkafkaViewContent.scala.html create mode 100644 app/views/logkafka/updateConfig.scala.html diff --git a/app/controllers/Logkafka.scala b/app/controllers/Logkafka.scala new file mode 100644 index 000000000..1cfc90257 --- /dev/null +++ b/app/controllers/Logkafka.scala @@ -0,0 +1,210 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ + +package controllers + +import java.util.Properties + +import kafka.manager.ActorModel.LogkafkaIdentity +import kafka.manager.utils.LogkafkaNewConfigs +import kafka.manager.{Kafka_0_8_2_1, ApiError, Kafka_0_8_2_0, Kafka_0_8_1_1} +import models.FollowLink +import models.form._ +import models.navigation.Menus +import play.api.data.Form +import play.api.data.Forms._ +import play.api.data.validation.{Valid, Invalid, Constraint} +import play.api.data.validation.Constraints._ +import play.api.mvc._ + +import scala.concurrent.Future +import scala.util.{Success, Failure, Try} +import scalaz.{\/-, -\/} + +/** + * @author hiral + */ +object Logkafka extends Controller{ + import play.api.libs.concurrent.Execution.Implicits.defaultContext + + private[this] val kafkaManager = KafkaManagerContext.getKafkaManager + + val validateHostname : Constraint[String] = Constraint("validate name") { name => + Try { + kafka.manager.utils.Logkafka.validateHostname(name) + } match { + case Failure(t) => Invalid(t.getMessage) + case Success(_) => Valid + } + } + + val validatePath: Constraint[String] = Constraint("validate path") { name => + Try { + kafka.manager.utils.Logkafka.validatePath(name) + } match { + case Failure(t) => Invalid(t.getMessage) + case Success(_) => Valid + } + } + + val kafka_0_8_1_1_Default = CreateLogkafka("","", + LogkafkaNewConfigs.configMaps(Kafka_0_8_1_1).map{case(k,v) => LKConfig(k,Some(v))}.toList) + val kafka_0_8_2_0_Default = CreateLogkafka("","", + LogkafkaNewConfigs.configMaps(Kafka_0_8_2_0).map{case(k,v) => LKConfig(k,Some(v))}.toList) + val kafka_0_8_2_1_Default = CreateLogkafka("","", + LogkafkaNewConfigs.configMaps(Kafka_0_8_2_1).map{case(k,v) => LKConfig(k,Some(v))}.toList) + + val defaultCreateForm = Form( + mapping( + "hostname" -> nonEmptyText.verifying(maxLength(250), validateHostname), + "log_path" -> nonEmptyText.verifying(maxLength(250), validatePath), + "configs" -> list( + mapping( + "name" -> nonEmptyText, + "value" -> optional(text) + )(LKConfig.apply)(LKConfig.unapply) + ) + )(CreateLogkafka.apply)(CreateLogkafka.unapply) + ) + + val defaultDeleteForm = Form( + mapping( + "hostname" -> nonEmptyText.verifying(maxLength(250), validateHostname), + "log_path" -> nonEmptyText.verifying(maxLength(250), validatePath) + )(DeleteLogkafka.apply)(DeleteLogkafka.unapply) + ) + + val defaultUpdateConfigForm = Form( + mapping( + "hostname" -> nonEmptyText.verifying(maxLength(250), validateHostname), + "log_path" -> nonEmptyText.verifying(maxLength(250), validatePath), + "configs" -> list( + mapping( + "name" -> nonEmptyText, + "value" -> optional(text) + )(LKConfig.apply)(LKConfig.unapply) + ) + )(UpdateLogkafkaConfig.apply)(UpdateLogkafkaConfig.unapply) + ) + + private def createLogkafkaForm(clusterName: String) = { + kafkaManager.getClusterConfig(clusterName).map { errorOrConfig => + errorOrConfig.map { clusterConfig => + clusterConfig.version match { + case Kafka_0_8_1_1 => defaultCreateForm.fill(kafka_0_8_1_1_Default) + case Kafka_0_8_2_0 => defaultCreateForm.fill(kafka_0_8_2_0_Default) + case Kafka_0_8_2_1 => defaultCreateForm.fill(kafka_0_8_2_1_Default) + } + } + } + } + + def logkafkas(c: String) = Action.async { + kafkaManager.getLogkafkaListExtended(c).map { errorOrLogkafkaList => + Ok(views.html.logkafka.logkafkaList(c,errorOrLogkafkaList)) + } + } + + def logkafka(c: String, h: String, l:String) = Action.async { + kafkaManager.getLogkafkaIdentity(c,h).map { errorOrLogkafkaIdentity => + Ok(views.html.logkafka.logkafkaView(c,h,l,errorOrLogkafkaIdentity)) + } + } + + def createLogkafka(clusterName: String) = Action.async { implicit request => + createLogkafkaForm(clusterName).map { errorOrForm => + Ok(views.html.logkafka.createLogkafka(clusterName, errorOrForm)) + } + } + + def handleCreateLogkafka(clusterName: String) = Action.async { implicit request => + defaultCreateForm.bindFromRequest.fold( + formWithErrors => Future.successful(BadRequest(views.html.logkafka.createLogkafka(clusterName,\/-(formWithErrors)))), + cl => { + val props = new Properties() + cl.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name,c.value.get)) + kafkaManager.createLogkafka(clusterName,cl.hostname,cl.log_path,props).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName,"Logkafka","Create",Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndCluster("Logkafkas",clusterName,"Create Logkafka"), + errorOrSuccess, + "Create Logkafka", + FollowLink("Go to hostname view.",routes.Logkafka.logkafka(clusterName, cl.hostname, cl.log_path).toString()), + FollowLink("Try again.",routes.Logkafka.createLogkafka(clusterName).toString()) + )) + } + } + ) + } + + def handleDeleteLogkafka(clusterName: String, hostname: String, log_path: String) = Action.async { implicit request => + defaultDeleteForm.bindFromRequest.fold( + formWithErrors => Future.successful( + BadRequest(views.html.logkafka.logkafkaView( + clusterName, + hostname, + log_path, + -\/(ApiError(formWithErrors.error("logkafka").map(_.toString).getOrElse("Unknown error deleting logkafka!")))))), + deleteLogkafka => { + kafkaManager.deleteLogkafka(clusterName,deleteLogkafka.hostname,deleteLogkafka.log_path).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName,"Logkafka","Logkafka View",Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndLogkafka("Logkafka View",clusterName,hostname,log_path,"Delete Logkafka"), + errorOrSuccess, + "Delete Logkafka", + FollowLink("Go to logkafka list.",routes.Logkafka.logkafkas(clusterName).toString()), + FollowLink("Try again.",routes.Logkafka.logkafka(clusterName, hostname, log_path).toString()) + )) + } + } + ) + } + + private def updateConfigForm(clusterName: String, log_path: String, li: LogkafkaIdentity) = { + kafkaManager.getClusterConfig(clusterName).map { errorOrConfig => + errorOrConfig.map { clusterConfig => + val defaultConfigMap = clusterConfig.version match { + case Kafka_0_8_1_1 => LogkafkaNewConfigs.configNames(Kafka_0_8_1_1).map(n => (n,LKConfig(n,None))).toMap + case Kafka_0_8_2_0 => LogkafkaNewConfigs.configNames(Kafka_0_8_2_0).map(n => (n,LKConfig(n,None))).toMap + case Kafka_0_8_2_1 => LogkafkaNewConfigs.configNames(Kafka_0_8_2_1).map(n => (n,LKConfig(n,None))).toMap + } + val config: Map[String, String] = li.identityMap.get(log_path).get._1.get + val combinedMap = defaultConfigMap ++ config.map(tpl => tpl._1 -> LKConfig(tpl._1,Option(tpl._2))) + defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,combinedMap.toList.map(_._2))) + } + } + } + + def updateConfig(clusterName: String, hostname: String, log_path: String) = Action.async { implicit request => + val errorOrFormFuture = kafkaManager.getLogkafkaIdentity(clusterName, hostname).flatMap { errorOrLogkafkaIdentity => + errorOrLogkafkaIdentity.fold( e => Future.successful(-\/(e)) ,{ logkafkaIdentity => + updateConfigForm(clusterName, log_path, logkafkaIdentity) + }) + } + errorOrFormFuture.map { errorOrForm => + Ok(views.html.logkafka.updateConfig(clusterName, hostname, log_path, errorOrForm)) + } + } + + def handleUpdateConfig(clusterName: String, hostname: String, log_path: String) = Action.async { implicit request => + defaultUpdateConfigForm.bindFromRequest.fold( + formWithErrors => Future.successful(BadRequest(views.html.logkafka.updateConfig(clusterName, hostname, log_path, \/-(formWithErrors)))), + updateLogkafkaConfig => { + val props = new Properties() + updateLogkafkaConfig.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name,c.value.get)) + kafkaManager.updateLogkafkaConfig(clusterName,updateLogkafkaConfig.hostname,updateLogkafkaConfig.log_path, props).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName,"Logkafka","Logkafka View",Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndLogkafka("Logkafka View",clusterName, hostname, log_path, "Update Config"), + errorOrSuccess, + "Update Config", + FollowLink("Go to logkafka view.",routes.Logkafka.logkafka(clusterName, updateLogkafkaConfig.hostname, updateLogkafkaConfig.log_path).toString()), + FollowLink("Try again.",routes.Logkafka.updateConfig(clusterName, hostname, log_path).toString()) + )) + } + } + ) + } +} diff --git a/app/kafka/manager/ActorModel.scala b/app/kafka/manager/ActorModel.scala index 745961793..45e62f48d 100644 --- a/app/kafka/manager/ActorModel.scala +++ b/app/kafka/manager/ActorModel.scala @@ -33,6 +33,7 @@ object ActorModel { case object BVForceUpdate extends CommandRequest case object BVGetTopicIdentities extends BVRequest + case object BVGetLogkafkaIdentities extends BVRequest case class BVGetView(id: Int) extends BVRequest case object BVGetViews extends BVRequest case class BVGetTopicMetrics(topic: String) extends BVRequest @@ -67,6 +68,16 @@ object ActorModel { case class CMGeneratePartitionAssignments(topics: Set[String], brokers: Seq[Int]) extends CommandRequest case class CMManualPartitionAssignments(assignments: List[(String, List[(Int, List[Int])])]) extends CommandRequest + case class CMGetLogkafkaIdentity(hostname: String) extends QueryRequest + case class CMLogkafkaIdentity(logkafkaIdentity: Try[LogkafkaIdentity]) extends QueryResponse + case class CMCreateLogkafka(hostname: String, + log_path: String, + config: Properties = new Properties + ) extends CommandRequest + case class CMUpdateLogkafkaConfig(hostname: String, + log_path: String, + config: Properties) extends CommandRequest + case class CMDeleteLogkafka(hostname: String, log_path: String) extends CommandRequest case class CMCommandResult(result: Try[Unit]) extends CommandResponse case class CMCommandResults(result: IndexedSeq[Try[Unit]]) extends CommandResponse @@ -87,6 +98,18 @@ object ActorModel { case class KCReassignPartition(currentTopicIdentity: Map[String, TopicIdentity], generatedTopicIdentity: Map[String, TopicIdentity]) extends CommandRequest + case class KCCreateLogkafka(hostname: String, + log_path: String, + config: Properties, + logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest + case class KCDeleteLogkafka(hostname: String, + log_path: String, + logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest + case class KCUpdateLogkafkaConfig(hostname: String, + log_path: String, + config: Properties, + logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest + case class KCCommandResult(result: Try[Unit]) extends CommandResponse case object KMGetActiveClusters extends QueryRequest @@ -126,6 +149,14 @@ object ActorModel { case object KSGetBrokers extends KSRequest case class KSGetBrokerState(id: String) extends KSRequest + case object KSGetLogkafkaHostnames extends KSRequest + case class KSGetLogkafkaConfig(hostname: String) extends KSRequest + case class KSGetLogkafkaClient(hostname: String) extends KSRequest + case class KSGetLogkafkaConfigs(hostnames: Set[String]) extends KSRequest + case class KSGetLogkafkaClients(hostnames: Set[String]) extends KSRequest + case class KSGetAllLogkafkaConfigs(lastUpdateMillis: Option[Long]= None) extends KSRequest + case class KSGetAllLogkafkaClients(lastUpdateMillis: Option[Long]= None) extends KSRequest + case class TopicList(list: IndexedSeq[String], deleteSet: Set[String]) extends QueryResponse case class TopicConfig(topic: String, config: Option[(Int,String)]) extends QueryResponse @@ -141,6 +172,12 @@ object ActorModel { case class PreferredReplicaElection(startTime: DateTime, topicAndPartition: Set[TopicAndPartition], endTime: Option[DateTime]) extends QueryResponse case class ReassignPartitions(startTime: DateTime, partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]], endTime: Option[DateTime]) extends QueryResponse + case class LogkafkaHostnameList(list: IndexedSeq[String], deleteSet: Set[String]) extends QueryResponse + case class LogkafkaConfig(hostname: String, config: Option[String]) extends QueryResponse + case class LogkafkaClient(hostname: String, client: Option[String]) extends QueryResponse + case class LogkafkaConfigs(configs: IndexedSeq[LogkafkaConfig], lastUpdateMillis: Long) extends QueryResponse + case class LogkafkaClients(clients: IndexedSeq[LogkafkaClient], lastUpdateMillis: Long) extends QueryResponse + case object DCUpdateState extends CommandRequest case class BrokerIdentity(id: Int, host: String, port: Int, jmxPort: Int) @@ -311,6 +348,39 @@ object ActorModel { } } + case class LogkafkaIdentity(hostname: String, + active: Boolean, + deleteSupported: Boolean = true, + identityMap: Map[String, (Option[Map[String, String]], Option[Map[String, String]])]) { + } + + object LogkafkaIdentity { + + lazy val logger = LoggerFactory.getLogger(this.getClass) + + implicit def from(hostname: String, lcg: Option[LogkafkaConfig], lct: Option[LogkafkaClient]) : LogkafkaIdentity = { + val configJsonStr = lcg match { + case Some(l) => l.config.getOrElse[String]("{}") + case None => "{}" + } + + val configMap: Map[String, Map[String, String]] = utils.Logkafka.parseJsonStr(hostname, configJsonStr) + + val clientJsonStr = lct match { + case Some(l) => l.client.getOrElse[String]("{}") + case None => "{}" + } + + val clientMap: Map[String, Map[String, String]] = utils.Logkafka.parseJsonStr(hostname, clientJsonStr) + + val hostnameSet = configMap.keySet ++ clientMap.keySet + val identitySet = if (!hostnameSet.isEmpty) { + hostnameSet map { l => l -> ((if(!configMap.isEmpty) configMap.get(l) else None, if(!clientMap.isEmpty) clientMap.get(l) else None)) } + } else { Set() } + LogkafkaIdentity(hostname, lct.isDefined, true, identitySet.toMap) + } + } + case class BrokerMetrics(bytesInPerSec: MeterMetric, bytesOutPerSec: MeterMetric, bytesRejectedPerSec: MeterMetric, diff --git a/app/kafka/manager/BrokerViewCacheActor.scala b/app/kafka/manager/BrokerViewCacheActor.scala index 24f5bbeea..40dcc35f1 100644 --- a/app/kafka/manager/BrokerViewCacheActor.scala +++ b/app/kafka/manager/BrokerViewCacheActor.scala @@ -40,6 +40,12 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni private[this] var combinedBrokerMetric : Option[BrokerMetrics] = None + private[this] var logkafkaIdentities : Map[String, LogkafkaIdentity] = Map.empty + + private[this] var logkafkaConfigsOption : Option[LogkafkaConfigs] = None + + private[this] var logkafkaClientsOption : Option[LogkafkaClients] = None + private[this] val EMPTY_BVVIEW = BVView(Map.empty, config.clusterConfig, Option(BrokerMetrics.DEFAULT)) override def preStart() = { @@ -109,6 +115,8 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni val lastUpdateMillisOption: Option[Long] = topicDescriptionsOption.map(_.lastUpdateMillis) context.actorSelection(config.kafkaStateActorPath).tell(KSGetAllTopicDescriptions(lastUpdateMillisOption), self) context.actorSelection(config.kafkaStateActorPath).tell(KSGetBrokers, self) + context.actorSelection(config.kafkaStateActorPath).tell(KSGetAllLogkafkaConfigs(lastUpdateMillisOption), self) + context.actorSelection(config.kafkaStateActorPath).tell(KSGetAllLogkafkaClients(lastUpdateMillisOption), self) case BVGetViews => sender ! allBrokerViews() @@ -128,6 +136,9 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni case BVGetTopicIdentities => sender ! topicIdentities + case BVGetLogkafkaIdentities => + sender ! logkafkaIdentities + case BVUpdateTopicMetricsForBroker(id, metrics) => metrics.foreach { case (topic, bm) => @@ -157,6 +168,14 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni brokerListOption = Some(bl) updateView() + case lcg: LogkafkaConfigs => + logkafkaConfigsOption = Some(lcg) + updateView() + + case lct: LogkafkaClients => + logkafkaClientsOption = Some(lct) + updateView() + case any: Any => log.warning("bvca : processActorResponse : Received unknown message: {}", any) } } @@ -239,5 +258,15 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni brokerId,BVView(topicPartitionsMap, config.clusterConfig, brokerMetrics.get(brokerId))) } } + + for { + logkafkaConfigs <- logkafkaConfigsOption + logkafkaClients <- logkafkaClientsOption + } { + val lcgMap = Map(logkafkaConfigs.configs map { a => a.hostname -> a }: _*) + val lctMap = Map(logkafkaClients.clients map { a => a.hostname -> a }: _*) + logkafkaIdentities = lcgMap.map (kv => + kv._1 -> LogkafkaIdentity.from(kv._1, Some(kv._2), lctMap.get(kv._1))) + } } } diff --git a/app/kafka/manager/ClusterManagerActor.scala b/app/kafka/manager/ClusterManagerActor.scala index 6ae9241cf..ee77fea1f 100644 --- a/app/kafka/manager/ClusterManagerActor.scala +++ b/app/kafka/manager/ClusterManagerActor.scala @@ -180,6 +180,16 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) } yield tdO.map( td => CMTopicIdentity(Try(TopicIdentity.from(bl,td,tm,cmConfig.clusterConfig)))) result pipeTo sender + case CMGetLogkafkaIdentity(hostname) => + implicit val ec = context.dispatcher + val eventualLogkafkaConfig= withKafkaStateActor(KSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) + val eventualLogkafkaClient= withKafkaStateActor(KSGetLogkafkaClient(hostname))(identity[Option[LogkafkaClient]]) + val result: Future[Option[CMLogkafkaIdentity]] = for { + lcg <- eventualLogkafkaConfig + lct <- eventualLogkafkaClient + } yield Some(CMLogkafkaIdentity(Try(LogkafkaIdentity.from(hostname,lcg,lct)))) + result pipeTo sender + case any: Any => log.warning("cma : processQueryResponse : Received unknown message: {}", any) } } @@ -367,6 +377,40 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) } } pipeTo sender() + case CMDeleteLogkafka(hostname, log_path) => + implicit val ec = longRunningExecutionContext + val eventualLogkafkaConfig = withKafkaStateActor(KSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) + eventualLogkafkaConfig.map { logkafkaConfigOption => + logkafkaConfigOption.fold { + Future.successful(CMCommandResult(Failure(new IllegalArgumentException(s"Hostname doesn't exists : $hostname")))) + } { td => + withKafkaCommandActor(KCDeleteLogkafka(hostname, log_path, logkafkaConfigOption)) { + kcResponse: KCCommandResult => + CMCommandResult(kcResponse.result) + } + } + } pipeTo sender() + + case CMCreateLogkafka(hostname, log_path, config) => + implicit val ec = longRunningExecutionContext + val eventualLogkafkaConfig = withKafkaStateActor(KSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) + eventualLogkafkaConfig.map { logkafkaConfigOption => + withKafkaCommandActor(KCCreateLogkafka(hostname, log_path, config, logkafkaConfigOption)) { + kcResponse: KCCommandResult => + CMCommandResult(kcResponse.result) + } + } pipeTo sender() + + case CMUpdateLogkafkaConfig(hostname, log_path, config) => + implicit val ec = longRunningExecutionContext + val eventualLogkafkaConfig = withKafkaStateActor(KSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) + eventualLogkafkaConfig.map { logkafkaConfigOption => + withKafkaCommandActor(KCUpdateLogkafkaConfig(hostname, log_path, config, logkafkaConfigOption)) { + kcResponse: KCCommandResult => + CMCommandResult(kcResponse.result) + } + } pipeTo sender() + case any: Any => log.warning("cma : processCommandRequest : Received unknown message: {}", any) } } diff --git a/app/kafka/manager/KafkaCommandActor.scala b/app/kafka/manager/KafkaCommandActor.scala index e94f3a90c..41af93eb6 100644 --- a/app/kafka/manager/KafkaCommandActor.scala +++ b/app/kafka/manager/KafkaCommandActor.scala @@ -127,6 +127,37 @@ class KafkaCommandActor(kafkaCommandActorConfig: KafkaCommandActorConfig) extend ) } } + case KCDeleteLogkafka(hostname, log_path, logkafkaConfig) => + kafkaCommandActorConfig.version match { + case Kafka_0_8_1_1 => + val result : KCCommandResult = KCCommandResult(Failure(new UnsupportedOperationException( + s"Delete topic not supported for kafka version ${kafkaCommandActorConfig.version}"))) + sender ! result + case Kafka_0_8_2_0 | Kafka_0_8_2_1 => + longRunning { + Future { + KCCommandResult(Try { + adminUtils.deleteLogkafka(kafkaCommandActorConfig.curator, hostname, log_path, logkafkaConfig) + }) + } + } + } + case KCCreateLogkafka(hostname, log_path, config, logkafkaConfig) => + longRunning { + Future { + KCCommandResult(Try { + adminUtils.createLogkafka(kafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) + }) + } + } + case KCUpdateLogkafkaConfig(hostname, log_path, config, logkafkaConfig) => + longRunning { + Future { + KCCommandResult(Try { + adminUtils.changeLogkafkaConfig(kafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) + }) + } + } case any: Any => log.warning("kca : processCommandRequest : Received unknown message: {}", any) } } diff --git a/app/kafka/manager/KafkaManager.scala b/app/kafka/manager/KafkaManager.scala index cfe6dcc02..af829d191 100644 --- a/app/kafka/manager/KafkaManager.scala +++ b/app/kafka/manager/KafkaManager.scala @@ -25,6 +25,7 @@ import scala.util.{Success, Failure, Try} */ case class TopicListExtended(list: IndexedSeq[(String, Option[TopicIdentity])], deleteSet: Set[String], underReassignments: IndexedSeq[String]) case class BrokerListExtended(list: IndexedSeq[BrokerIdentity], metrics: Map[Int,BrokerMetrics], combinedMetric: Option[BrokerMetrics], clusterConfig: ClusterConfig) +case class LogkafkaListExtended(list: IndexedSeq[(String, Option[LogkafkaIdentity])], deleteSet: Set[String]) case class ApiError(msg: String) object ApiError { private[this] val log : Logger = LoggerFactory.getLogger(classOf[ApiError]) @@ -356,6 +357,52 @@ class KafkaManager(akkaConfig: Config) } } + def createLogkafka( + clusterName: String, + hostname: String, + log_path: String, + config: Properties = new Properties + ): Future[ApiError \/ Unit] = + { + implicit val ec = apiExecutionContext + withKafkaManagerActor(KMClusterCommandRequest(clusterName, CMCreateLogkafka(hostname, log_path, config))) { + result: Future[CMCommandResult] => + result.map(cmr => toDisjunction(cmr.result)) + } + } + + def updateLogkafkaConfig( + clusterName: String, + hostname: String, + log_path: String, + config: Properties + ): Future[ApiError \/ Unit] = + { + implicit val ec = apiExecutionContext + withKafkaManagerActor( + KMClusterCommandRequest( + clusterName, + CMUpdateLogkafkaConfig(hostname, log_path, config) + ) + ) { + result: Future[CMCommandResult] => + result.map(cmr => toDisjunction(cmr.result)) + } + } + + def deleteLogkafka( + clusterName: String, + hostname: String, + log_path: String + ): Future[ApiError \/ Unit] = + { + implicit val ec = apiExecutionContext + withKafkaManagerActor(KMClusterCommandRequest(clusterName, CMDeleteLogkafka(hostname, log_path))) { + result: Future[CMCommandResult] => + result.map(cmr => toDisjunction(cmr.result)) + } + } + //--------------------Queries-------------------------- def getClusterConfig(clusterName: String): Future[ApiError \/ ClusterConfig] = { tryWithKafkaManagerActor(KMGetClusterConfig(clusterName)) { result: KMClusterConfigResult => @@ -522,4 +569,46 @@ class KafkaManager(akkaConfig: Config) ) )(identity[Option[ReassignPartitions]]) } -} \ No newline at end of file + + def getLogkafkaListExtended(clusterName: String): Future[ApiError \/ LogkafkaListExtended] = { + val futureLogkafkaIdentities = tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, BVGetLogkafkaIdentities))(identity[Map[String, LogkafkaIdentity]]) + val futureLogkafkaList = tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, KSGetLogkafkaHostnames))(identity[LogkafkaHostnameList]) + implicit val ec = apiExecutionContext + for { + errOrLi <- futureLogkafkaIdentities + errOrLl <- futureLogkafkaList + } yield { + for { + li <- errOrLi + ll <- errOrLl + } yield { + LogkafkaListExtended(ll.list.map(l => (l, li.get(l))), ll.deleteSet) + } + } + } + + def getLogkafkaIdentity(clusterName: String, hostname: String): Future[ApiError \/ LogkafkaIdentity] = { + val futureCMLogkafkaIdentity = tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, CMGetLogkafkaIdentity(hostname)))( + identity[Option[CMLogkafkaIdentity]] + ) + implicit val ec = apiExecutionContext + futureCMLogkafkaIdentity.map[ApiError \/ LogkafkaIdentity] { errOrLI => + errOrLI.fold[ApiError \/ LogkafkaIdentity]( + { err: ApiError => + -\/[ApiError](err) + }, { liOption: Option[CMLogkafkaIdentity] => + liOption.fold[ApiError \/ LogkafkaIdentity] { + -\/(ApiError(s"Logkafka not found $hostname for cluster $clusterName")) + } { cmLogkafkaIdentity => + cmLogkafkaIdentity.logkafkaIdentity match { + case scala.util.Failure(l) => + -\/[ApiError](l) + case scala.util.Success(li) => + \/-(li) + } + } + } + ) + } + } +} diff --git a/app/kafka/manager/KafkaStateActor.scala b/app/kafka/manager/KafkaStateActor.scala index 6abbf9d3a..bf884ce64 100644 --- a/app/kafka/manager/KafkaStateActor.scala +++ b/app/kafka/manager/KafkaStateActor.scala @@ -39,6 +39,16 @@ class KafkaStateActor(curator: CuratorFramework, @volatile private[this] var topicsTreeCacheLastUpdateMillis : Long = System.currentTimeMillis() + private[this] val logkafkaConfigTreeCache = new TreeCache(curator,ZkUtils.LogkafkaConfigPath) + + private[this] val logkafkaClientTreeCache = new TreeCache(curator,ZkUtils.LogkafkaClientPath) + + @volatile + private[this] var logkafkaConfigTreeCacheLastUpdateMillis : Long = System.currentTimeMillis() + + @volatile + private[this] var logkafkaClientTreeCacheLastUpdateMillis : Long = System.currentTimeMillis() + private[this] val topicsTreeCacheListener = new TreeCacheListener { override def childEvent(client: CuratorFramework, event: TreeCacheEvent): Unit = { event.getType match { @@ -51,6 +61,30 @@ class KafkaStateActor(curator: CuratorFramework, } } + private[this] val logkafkaConfigTreeCacheListener = new TreeCacheListener { + override def childEvent(client: CuratorFramework, event: TreeCacheEvent): Unit = { + event.getType match { + case TreeCacheEvent.Type.INITIALIZED | TreeCacheEvent.Type.NODE_ADDED | + TreeCacheEvent.Type.NODE_REMOVED | TreeCacheEvent.Type.NODE_UPDATED => + logkafkaConfigTreeCacheLastUpdateMillis = System.currentTimeMillis() + case _ => + //do nothing + } + } + } + + private[this] val logkafkaClientTreeCacheListener = new TreeCacheListener { + override def childEvent(client: CuratorFramework, event: TreeCacheEvent): Unit = { + event.getType match { + case TreeCacheEvent.Type.INITIALIZED | TreeCacheEvent.Type.NODE_ADDED | + TreeCacheEvent.Type.NODE_REMOVED | TreeCacheEvent.Type.NODE_UPDATED => + logkafkaClientTreeCacheLastUpdateMillis = System.currentTimeMillis() + case _ => + //do nothing + } + } + } + @volatile private[this] var preferredLeaderElection : Option[PreferredReplicaElection] = None @@ -123,9 +157,17 @@ class KafkaStateActor(curator: CuratorFramework, adminPathCache.start(StartMode.BUILD_INITIAL_CACHE) log.info("Starting delete topics path cache...") deleteTopicsPathCache.start(StartMode.BUILD_INITIAL_CACHE) + log.info("Starting logkafka config tree cache...") + logkafkaConfigTreeCache.start() + log.info("Starting logkafka client tree cache...") + logkafkaClientTreeCache.start() log.info("Adding topics tree cache listener...") topicsTreeCache.getListenable.addListener(topicsTreeCacheListener) + log.info("Adding logkafka config tree cache listener...") + logkafkaConfigTreeCache.getListenable.addListener(logkafkaConfigTreeCacheListener) + log.info("Adding logkafka client tree cache listener...") + logkafkaClientTreeCache.getListenable.addListener(logkafkaClientTreeCacheListener) log.info("Adding admin path cache listener...") adminPathCache.getListenable.addListener(adminPathCacheListener) } @@ -146,6 +188,10 @@ class KafkaStateActor(curator: CuratorFramework, Try(adminPathCache.getListenable.removeListener(adminPathCacheListener)) log.info("Removing topics tree cache listener...") Try(topicsTreeCache.getListenable.removeListener(topicsTreeCacheListener)) + log.info("Removing logkafka config tree cache listener...") + Try(logkafkaConfigTreeCache.getListenable.removeListener(logkafkaConfigTreeCacheListener)) + log.info("Removing logkafka client tree cache listener...") + Try(logkafkaClientTreeCache.getListenable.removeListener(logkafkaClientTreeCacheListener)) log.info("Shutting down delete topics path cache...") Try(deleteTopicsPathCache.close()) @@ -157,6 +203,10 @@ class KafkaStateActor(curator: CuratorFramework, Try(topicsConfigPathCache.close()) log.info("Shutting down topics tree cache...") Try(topicsTreeCache.close()) + log.info("Shutting down logkafka config tree cache...") + Try(logkafkaConfigTreeCache.close()) + log.info("Shutting down logkafka client tree cache...") + Try(logkafkaClientTreeCache.close()) super.postStop() } @@ -178,6 +228,18 @@ class KafkaStateActor(curator: CuratorFramework, } yield TopicDescription(topic, description, Option(states),config, deleteSupported) } + def getLogkafkaConfig(hostname: String) : Option[LogkafkaConfig] = { + for { + config <- getLogkafkaConfigString(hostname) + } yield LogkafkaConfig(hostname, Some(config)) + } + + def getLogkafkaClient(hostname: String) : Option[LogkafkaClient] = { + for { + client <- getLogkafkaClientString(hostname) + } yield LogkafkaClient(hostname, Some(client)) + } + override def processActorResponse(response: ActorResponse): Unit = { response match { case any: Any => log.warning("ksa : processActorResponse : Received unknown message: {}", any.toString) @@ -190,6 +252,16 @@ class KafkaStateActor(curator: CuratorFramework, result.map(cd => (cd.getStat.getVersion,asString(cd.getData))) } + private[this] def getLogkafkaConfigString(hostname: String) : Option[String] = { + val hostnamePath = "%s/%s".format(ZkUtils.LogkafkaConfigPath,hostname) + Option(logkafkaConfigTreeCache.getCurrentData(hostnamePath)).map( childData => asString(childData.getData)) + } + + private[this] def getLogkafkaClientString(hostname: String) : Option[String] = { + val hostnamePath = "%s/%s".format(ZkUtils.LogkafkaClientPath,hostname) + Option(logkafkaClientTreeCache.getCurrentData(hostnamePath)).map( childData => asString(childData.getData)) + } + override def processQueryRequest(request: QueryRequest): Unit = { request match { case KSGetTopics => @@ -258,6 +330,54 @@ class KafkaStateActor(curator: CuratorFramework, case KSGetReassignPartition => sender ! reassignPartitions + case KSGetLogkafkaHostnames => + val deleteSet: Set[String] = Set.empty + withLogkafkaConfigTreeCache { cache => + cache.getCurrentChildren(ZkUtils.LogkafkaConfigPath) + }.fold { + sender ! LogkafkaHostnameList(IndexedSeq.empty, deleteSet) + } { data: java.util.Map[String, ChildData] => + sender ! LogkafkaHostnameList(data.asScala.map(kv => kv._1).toIndexedSeq, deleteSet) + } + + case KSGetLogkafkaConfig(hostname) => + sender ! getLogkafkaConfig(hostname) + + case KSGetLogkafkaClient(hostname) => + sender ! getLogkafkaClient(hostname) + + case KSGetLogkafkaConfigs(hostnames) => + sender ! LogkafkaConfigs(hostnames.toIndexedSeq.map(getLogkafkaConfig).flatten, logkafkaConfigTreeCacheLastUpdateMillis) + + case KSGetLogkafkaClients(hostnames) => + sender ! LogkafkaClients(hostnames.toIndexedSeq.map(getLogkafkaClient).flatten, logkafkaClientTreeCacheLastUpdateMillis) + + case KSGetAllLogkafkaConfigs(lastUpdateMillisOption) => + val lastUpdateMillis = lastUpdateMillisOption.getOrElse(0L) + if (logkafkaConfigTreeCacheLastUpdateMillis > lastUpdateMillis) { + //we have option here since there may be no logkafka configs at all! + withLogkafkaConfigTreeCache { cache: TreeCache => + cache.getCurrentChildren(ZkUtils.LogkafkaConfigPath) + }.fold { + sender ! LogkafkaConfigs(IndexedSeq.empty, logkafkaConfigTreeCacheLastUpdateMillis) + } { data: java.util.Map[String, ChildData] => + sender ! LogkafkaConfigs(data.asScala.keys.toIndexedSeq.map(getLogkafkaConfig).flatten, logkafkaConfigTreeCacheLastUpdateMillis) + } + } // else no updates to send + + case KSGetAllLogkafkaClients(lastUpdateMillisOption) => + val lastUpdateMillis = lastUpdateMillisOption.getOrElse(0L) + if (logkafkaClientTreeCacheLastUpdateMillis > lastUpdateMillis) { + //we have option here since there may be no logkafka clients at all! + withLogkafkaClientTreeCache { cache: TreeCache => + cache.getCurrentChildren(ZkUtils.LogkafkaClientPath) + }.fold { + sender ! LogkafkaClients(IndexedSeq.empty, logkafkaClientTreeCacheLastUpdateMillis) + } { data: java.util.Map[String, ChildData] => + sender ! LogkafkaClients(data.asScala.keys.toIndexedSeq.map(getLogkafkaClient).flatten, logkafkaClientTreeCacheLastUpdateMillis) + } + } // else no updates to send + case any: Any => log.warning("ksa : processQueryRequest : Received unknown message: {}", any.toString) } } @@ -329,5 +449,13 @@ class KafkaStateActor(curator: CuratorFramework, Option(fn(topicsTreeCache)) } + private[this] def withLogkafkaConfigTreeCache[T](fn: TreeCache => T) : Option[T] = { + Option(fn(logkafkaConfigTreeCache)) + } + + private[this] def withLogkafkaClientTreeCache[T](fn: TreeCache => T) : Option[T] = { + Option(fn(logkafkaClientTreeCache)) + } + } diff --git a/app/kafka/manager/utils/AdminUtils.scala b/app/kafka/manager/utils/AdminUtils.scala index e4cb047d5..a334c2b2a 100644 --- a/app/kafka/manager/utils/AdminUtils.scala +++ b/app/kafka/manager/utils/AdminUtils.scala @@ -114,6 +114,82 @@ class AdminUtils(version: KafkaVersion) { createOrUpdateTopicPartitionAssignmentPathInZK(curator, topic, replicaAssignment, topicConfig) } + def deleteLogkafka(curator: CuratorFramework, + hostname: String, + log_path: String, + logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig]): Unit = { + logkafkaConfigOption.map { lcg => + lcg.config.map { c => + val configMap =kafka.manager.utils.Logkafka.parseJsonStr(hostname, c) + if (!configMap.isEmpty || !(configMap - log_path).isEmpty ) { + writeLogkafkaConfig(curator, hostname, configMap - log_path, -1) + } + } getOrElse { LogkafkaErrors.HostnameNotExists(hostname) } + } getOrElse { LogkafkaErrors.HostnameNotExists(hostname) } + } + + def createLogkafka(curator: CuratorFramework, + hostname: String, + log_path: String, + config: Properties = new Properties, + logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig] + ): Unit = { + createOrUpdateLogkafkaConfigPathInZK(curator, hostname, log_path, config, logkafkaConfigOption) + } + + def createOrUpdateLogkafkaConfigPathInZK(curator: CuratorFramework, + hostname: String, + log_path: String, + config: Properties = new Properties, + logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig], + update: Boolean = false, + readVersion: Int = -1 + ) { + // validate arguments + Logkafka.validateHostname(hostname) + LogkafkaNewConfigs.validate(version,config) + + val configMap: mutable.Map[String, String] = { + import scala.collection.JavaConverters._ + config.asScala + } + val newConfigMap = Map(log_path -> Map(configMap.toSeq:_*)) + + val logkafkaConfigMap = logkafkaConfigOption.map { lcg => + lcg.config.map { c => + kafka.manager.utils.Logkafka.parseJsonStr(hostname, c) + } getOrElse { Map.empty } + } getOrElse { Map.empty } + + if(!update ) { + // write out the config on create, not update, if there is any + writeLogkafkaConfig(curator, hostname, logkafkaConfigMap ++ newConfigMap, readVersion) + } else { + val merged = logkafkaConfigMap.toSeq ++ newConfigMap.toSeq + val grouped = merged.groupBy(_._1) + val cleaned = grouped.mapValues(_.map(_._2).fold(Map.empty)(_ ++ _)) + writeLogkafkaConfig(curator, hostname, cleaned, readVersion) + } + } + + /** + * Update the config for an existing (hostname,log_path) + * @param curator: The zk client handle used to write the new config to zookeeper + * @param hostname: The hostname for which configs are being changed + * @param log_path: The log_path for which configs are being changed + * @param config: The final set of configs that will be applied to the topic. If any new configs need to be added or + * existing configs need to be deleted, it should be done prior to invoking this API + * + */ + def changeLogkafkaConfig(curator: CuratorFramework, + hostname: String, + log_path: String, + config: Properties = new Properties, + logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig] + ): Unit = { + createOrUpdateLogkafkaConfigPathInZK(curator, hostname, log_path, config, logkafkaConfigOption, true) + } + def createOrUpdateTopicPartitionAssignmentPathInZK(curator: CuratorFramework, topic: String, partitionReplicaAssignment: Map[Int, Seq[Int]], @@ -154,6 +230,13 @@ class AdminUtils(version: KafkaVersion) { ZkUtils.updatePersistentPath(curator, ZkUtils.getTopicConfigPath(topic), toJson(map), readVersion) } + /** + * Write out the logkafka config to zk, if there is any + */ + private def writeLogkafkaConfig(curator: CuratorFramework, hostname: String, configMap: Map[String, Map[String, String]], readVersion: Int = -1) { + ZkUtils.updatePersistentPath(curator, ZkUtils.getLogkafkaConfigPath(hostname), toJson(configMap), readVersion) + } + private def writeTopicPartitionAssignment(curator: CuratorFramework, topic: String, replicaAssignment: Map[Int, Seq[Int]], diff --git a/app/kafka/manager/utils/Logkafka.scala b/app/kafka/manager/utils/Logkafka.scala new file mode 100644 index 000000000..63e43c4e0 --- /dev/null +++ b/app/kafka/manager/utils/Logkafka.scala @@ -0,0 +1,112 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.manager.utils + + +import scala.util.matching.Regex +import org.slf4j.LoggerFactory + +/** + * Borrowed from kafka 0.8.1.1 + * https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=blob;f=core/src/main/scala/kafka/common/Logkafka.scala + */ +object Logkafka { + import kafka.manager.utils.LogkafkaErrors._ + + val legalChars = "[a-zA-Z0-9\\._\\-]" + val validHostnameRegex = "^(([a-zA-Z0-9]|[a-zA-Z0-9][a-zA-Z0-9\\-]*[a-zA-Z0-9])\\.)*([A-Za-z0-9]|[A-Za-z0-9][A-Za-z0-9\\-]*[A-Za-z0-9])$"; + val maxNameLength = 255 + val illegalPathChars = "[\\?*:|\"<>]" + val maxPathLength = 255 + private val rgx = new Regex(legalChars + "+") + private val illRgxPath = new Regex(illegalPathChars) + lazy val logger = LoggerFactory.getLogger(this.getClass) + + def validateHostname(hostname: String) { + checkCondition(hostname.length > 0, HostnameEmpty) + checkCondition(hostname.length <= maxNameLength, InvalidHostnameLength) + rgx.findFirstIn(hostname) match { + case Some(t) => + checkCondition(t.equals(hostname), IllegalCharacterInName(hostname)) + case None => + checkCondition(false, IllegalCharacterInName(hostname)) + } + checkCondition(hostname.matches(validHostnameRegex), InvalidHostname) + } + + def validatePath(log_path: String) { + checkCondition(log_path.length > 0, LogPathEmpty) + checkCondition(log_path.startsWith("/"), LogPathNotAbsolute) + checkCondition(log_path.length <= maxPathLength, InvalidLogPathLength) + illRgxPath.findFirstIn(log_path) match { + case Some(t) => + checkCondition(false, IllegalCharacterInName(log_path)) + case None => + checkCondition(true, IllegalCharacterInName(log_path)) + } + } + + def parseJsonStr(hostname: String, jsonStr: String): Map[String, Map[String, String]] = { + import org.json4s.jackson.JsonMethods._ + import org.json4s.scalaz.JsonScalaz._ + import scala.language.reflectiveCalls + import org.json4s.JsonAST._ + try { + implicit val formats = org.json4s.DefaultFormats + val json = parse(jsonStr) + val mapMutable: collection.mutable.Map[String, Map[String,String]] = collection.mutable.Map.empty + for (JObject(list) <- json) { + for ((log_path, JObject(c)) <- list) { + mapMutable += log_path -> ((for {(k, JString(v)) <- c} yield (k -> v)).toMap) + } + } + collection.immutable.Map(mapMutable.toList: _*) + } catch { + case e: Exception => + logger.error(s"[hostname=${hostname}] Failed to parse logkafka hostname config : ${jsonStr}",e) + Map.empty + } + } +} + +object LogkafkaErrors { + class HostnameEmpty private[LogkafkaErrors] extends UtilError("hostname is illegal, can't be empty") + class LogPathEmpty private[LogkafkaErrors] extends UtilError("log path is illegal, can't be empty") + class LogPathNotAbsolute private[LogkafkaErrors] extends UtilError("log path is illegal, must be absolute") + class InvalidHostname private[LogkafkaErrors] extends UtilError(s"hostname is illegal, does not match regex ${Logkafka.validHostnameRegex}") + class InvalidHostnameLength private[LogkafkaErrors] extends UtilError( + "hostname is illegal, can't be longer than " + Logkafka.maxNameLength + " characters") + class InvalidLogPathLength private[LogkafkaErrors] extends UtilError( + "log path is illegal, can't be longer than " + Logkafka.maxPathLength + " characters") + class IllegalCharacterInName private[LogkafkaErrors] (hostname: String) extends UtilError( + "hostname " + hostname + " is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'") + class IllegalCharacterInPath private[LogkafkaErrors] (log_path: String) extends UtilError( + "log path " + log_path + " is illegal, contains a character in " + Logkafka.illegalPathChars) + class HostnameNotExists private[LogkafkaErrors] (hostname: String) extends UtilError(s"Hostname not exists : $hostname") + + val HostnameEmpty = new HostnameEmpty + val LogPathEmpty = new LogPathEmpty + val LogPathNotAbsolute = new LogPathNotAbsolute + val InvalidHostname = new InvalidHostname + val InvalidHostnameLength = new InvalidHostnameLength + val InvalidLogPathLength = new InvalidLogPathLength + def IllegalCharacterInName(hostname: String) = new IllegalCharacterInName(hostname) + def IllegalCharacterInPath(log_path: String) = new IllegalCharacterInPath(log_path) + def HostnameNotExists(hostname: String) = new HostnameNotExists(hostname) +} + diff --git a/app/kafka/manager/utils/LogkafkaNewConfigs.scala b/app/kafka/manager/utils/LogkafkaNewConfigs.scala new file mode 100644 index 000000000..2db0986a3 --- /dev/null +++ b/app/kafka/manager/utils/LogkafkaNewConfigs.scala @@ -0,0 +1,44 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ + +package kafka.manager.utils + +import java.util.Properties + +import kafka.manager.{Kafka_0_8_2_1, Kafka_0_8_2_0, Kafka_0_8_1_1, KafkaVersion} + +trait LogkafkaNewConfigs { + def configNames : Set[String] + def configMaps: Map[String, String] + def validate(props: Properties) +} + +object LogkafkaNewConfigs { + + val logkafkaConfigsByVersion : Map[KafkaVersion, LogkafkaNewConfigs] = Map( + Kafka_0_8_1_1 -> logkafka81.LogConfig, + Kafka_0_8_2_0 -> logkafka82.LogConfig, + Kafka_0_8_2_1 -> logkafka82.LogConfig + ) + + def configNames(version: KafkaVersion) : Set[String] = { + logkafkaConfigsByVersion.get(version) match { + case Some(tc) => tc.configNames + case None => throw new IllegalArgumentException(s"Undefined logkafka configs for version : $version, cannot get config names") + } + } + def configMaps(version: KafkaVersion) : Map[String, String] = { + logkafkaConfigsByVersion.get(version) match { + case Some(tc) => tc.configMaps + case None => throw new IllegalArgumentException(s"Undefined logkafka configs for version : $version, cannot get config maps") + } + } + def validate(version: KafkaVersion, props: Properties) : Unit = { + logkafkaConfigsByVersion.get(version) match { + case Some(tc) => tc.validate(props) + case None => throw new IllegalArgumentException(s"Undefined logkafka configs for version : $version, cannot validate config") + } + } +} diff --git a/app/kafka/manager/utils/ZkUtils.scala b/app/kafka/manager/utils/ZkUtils.scala index 2c7173ebd..447819e0c 100644 --- a/app/kafka/manager/utils/ZkUtils.scala +++ b/app/kafka/manager/utils/ZkUtils.scala @@ -34,6 +34,8 @@ object ZkUtils { val BrokerTopicsPath = "/brokers/topics" val TopicConfigPath = "/config/topics" val TopicConfigChangesPath = "/config/changes" + val LogkafkaConfigPath = "/logkafka/config" + val LogkafkaClientPath = "/logkafka/client" val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" val ReassignPartitionsPath = "/admin/reassign_partitions" @@ -55,6 +57,12 @@ object ZkUtils { def getDeleteTopicPath(topic: String): String = DeleteTopicsPath + "/" + topic + def getLogkafkaConfigPath(hostname: String): String = + LogkafkaConfigPath + "/" + hostname + + def getLogkafkaClientPath(hostname: String): String = + LogkafkaClientPath + "/" + hostname + implicit def serializeString(str: String): Array[Byte] = { str.getBytes(StandardCharsets.UTF_8) } diff --git a/app/kafka/manager/utils/logkafka81/LogConfig.scala b/app/kafka/manager/utils/logkafka81/LogConfig.scala new file mode 100644 index 000000000..d7018f24b --- /dev/null +++ b/app/kafka/manager/utils/logkafka81/LogConfig.scala @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.manager.utils.logkafka81 + +import java.util.Properties +import kafka.manager.utils.LogkafkaNewConfigs + +object Defaults { + val Valid = true + val FollowLast = true + val BatchSize = 200 + val Topic = "" + val Key = "" + val Partition = -1 + val CompressionCodec= "none" + val RequiredAcks = 1 + val MessageTimeoutMs = 0 +} + +/** + * Configuration settings for a log + * @param valid Enable now or not + * @param followLast If set to "false", when restarting logkafka process, the log_path formatted with current time will be collect; + If set to "true", when restarting logkafka process, the last collecting file will be collected continually + * @param batchSize The batch size of messages to be sent + * @param topic The topic of messages to be sent + * @param key The key of messages to be sent + * @param partition The partition of messages to be sent. + -1 : random + n(>=0): partition n + * @param compressionCodec Optional compression method of messages: none, gzip, snappy + * @param requiredAcks Number of required acks + * @param messageTimeoutMs Local message timeout. This value is only enforced locally + and limits the time a produced message waits for successful delivery. + A time of 0 is infinite. + * + */ +case class LogConfig(val valid: Boolean = Defaults.Valid, + val followLast: Boolean = Defaults.FollowLast, + val batchSize: Long = Defaults.BatchSize, + val topic: String = Defaults.Topic, + val key: String = Defaults.Key, + val partition: Int = Defaults.Partition, + val compressionCodec: String = Defaults.CompressionCodec, + val requiredAcks: Int = Defaults.RequiredAcks, + val messageTimeoutMs: Long = Defaults.MessageTimeoutMs) { + + def toProps: Properties = { + val props = new Properties() + import LogConfig._ + props.put(ValidProp, valid.toString) + props.put(FollowLastProp, followLast.toString) + props.put(BatchSizeProp, batchSize.toString) + props.put(TopicProp, topic.toString) + props.put(KeyProp, key.toString) + props.put(PartitionProp, partition.toString) + props.put(CompressionCodecProp, compressionCodec.toString) + props.put(RequiredAcksProp, requiredAcks.toString) + props.put(MessageTimeoutMsProp, messageTimeoutMs.toString) + props + } + + /** + * Get the absolute value of the given number. If the number is Int.MinValue return 0. + * This is different from java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!). + */ + def abs(n: Int) = if(n == Integer.MIN_VALUE) 0 else math.abs(n) +} + +object LogConfig extends LogkafkaNewConfigs { + val ValidProp = "valid" + val FollowLastProp = "follow_last" + val BatchSizeProp = "batchsize" + val TopicProp = "topic" + val KeyProp = "key" + val PartitionProp = "partition" + val CompressionCodecProp = "compression_codec" + val RequiredAcksProp = "required_acks" + val MessageTimeoutMsProp = "message_timeout_ms" + + val ConfigMaps = Map(ValidProp -> Defaults.Valid.toString, + FollowLastProp -> Defaults.FollowLast.toString, + BatchSizeProp -> Defaults.BatchSize.toString, + TopicProp -> Defaults.Topic.toString, + KeyProp -> Defaults.Key.toString, + PartitionProp -> Defaults.Partition.toString, + CompressionCodecProp -> Defaults.CompressionCodec.toString, + RequiredAcksProp -> Defaults.RequiredAcks.toString, + MessageTimeoutMsProp -> Defaults.MessageTimeoutMs.toString) + def configMaps = ConfigMaps + val ConfigNames = ConfigMaps.keySet + def configNames = ConfigNames + + /** + * Parse the given properties instance into a LogConfig object + */ + def fromProps(props: Properties): LogConfig = { + new LogConfig(valid = props.getProperty(ValidProp, Defaults.Valid.toString).toBoolean, + followLast = props.getProperty(FollowLastProp, Defaults.FollowLast.toString).toBoolean, + batchSize = props.getProperty(BatchSizeProp, Defaults.BatchSize.toString).toLong, + topic = props.getProperty(TopicProp, Defaults.Topic.toString).toString, + key = props.getProperty(KeyProp, Defaults.Key.toString).toString, + partition = props.getProperty(PartitionProp, Defaults.Partition.toString).toInt, + compressionCodec = props.getProperty(CompressionCodecProp, Defaults.CompressionCodec.toString).toString, + requiredAcks= props.getProperty(RequiredAcksProp, Defaults.RequiredAcks.toString).toInt, + messageTimeoutMs = props.getProperty(MessageTimeoutMsProp, Defaults.MessageTimeoutMs.toString).toLong) + } + + /** + * Create a log config instance using the given properties and defaults + */ + def fromProps(defaults: Properties, overrides: Properties): LogConfig = { + val props = new Properties(defaults) + props.putAll(overrides) + fromProps(props) + } + + /** + * Check that property names are valid + */ + def validateNames(props: Properties) { + import scala.collection.JavaConverters._ + for(name <- props.keys().asScala) + require(LogConfig.ConfigNames.asJava.contains(name), "Unknown configuration \"%s\".".format(name)) + } + + /** + * Check that the given properties contain only valid log config names, and that all values can be parsed. + */ + def validate(props: Properties) { + validateNames(props) + validateTopic(props) + LogConfig.fromProps(LogConfig().toProps, props) // check that we can parse the values + } + + /** + * Check that Topic is reasonable + */ + private def validateTopic(props: Properties) { + val topic = props.getProperty(TopicProp) + require(topic != null , "Topic is null") + } + +} diff --git a/app/kafka/manager/utils/logkafka82/LogConfig.scala b/app/kafka/manager/utils/logkafka82/LogConfig.scala new file mode 100644 index 000000000..4c0baae77 --- /dev/null +++ b/app/kafka/manager/utils/logkafka82/LogConfig.scala @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.manager.utils.logkafka82 + +import java.util.Properties +import kafka.manager.utils.LogkafkaNewConfigs + +object Defaults { + val Valid = true + val FollowLast = true + val BatchSize = 200 + val Topic = "" + val Key = "" + val Partition = -1 + val CompressionCodec= "none" + val RequiredAcks = 1 + val MessageTimeoutMs = 0 +} + +/** + * Configuration settings for a log + * @param valid Enable now or not + * @param followLast If set to "false", when restarting logkafka process, the log_path formatted with current time will be collect; + If set to "true", when restarting logkafka process, the last collecting file will be collected continually + * @param batchSize The batch size of messages to be sent + * @param topic The topic of messages to be sent + * @param key The key of messages to be sent + * @param partition The partition of messages to be sent. + -1 : random + n(>=0): partition n + * @param compressionCodec Optional compression method of messages: none, gzip, snappy + * @param requiredAcks Number of required acks + * @param messageTimeoutMs Local message timeout. This value is only enforced locally + and limits the time a produced message waits for successful delivery. + A time of 0 is infinite. + * + */ +case class LogConfig(val valid: Boolean = Defaults.Valid, + val followLast: Boolean = Defaults.FollowLast, + val batchSize: Long = Defaults.BatchSize, + val topic: String = Defaults.Topic, + val key: String = Defaults.Key, + val partition: Int = Defaults.Partition, + val compressionCodec: String = Defaults.CompressionCodec, + val requiredAcks: Int = Defaults.RequiredAcks, + val messageTimeoutMs: Long = Defaults.MessageTimeoutMs) { + + def toProps: Properties = { + val props = new Properties() + import LogConfig._ + props.put(ValidProp, valid.toString) + props.put(FollowLastProp, followLast.toString) + props.put(BatchSizeProp, batchSize.toString) + props.put(TopicProp, topic.toString) + props.put(KeyProp, key.toString) + props.put(PartitionProp, partition.toString) + props.put(CompressionCodecProp, compressionCodec.toString) + props.put(RequiredAcksProp, requiredAcks.toString) + props.put(MessageTimeoutMsProp, messageTimeoutMs.toString) + props + } + + /** + * Get the absolute value of the given number. If the number is Int.MinValue return 0. + * This is different from java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!). + */ + def abs(n: Int) = if(n == Integer.MIN_VALUE) 0 else math.abs(n) +} + +object LogConfig extends LogkafkaNewConfigs { + val ValidProp = "valid" + val FollowLastProp = "follow_last" + val BatchSizeProp = "batchsize" + val TopicProp = "topic" + val KeyProp = "key" + val PartitionProp = "partition" + val CompressionCodecProp = "compression_codec" + val RequiredAcksProp = "required_acks" + val MessageTimeoutMsProp = "message_timeout_ms" + + val ConfigMaps = Map(ValidProp -> Defaults.Valid.toString, + FollowLastProp -> Defaults.FollowLast.toString, + BatchSizeProp -> Defaults.BatchSize.toString, + TopicProp -> Defaults.Topic.toString, + KeyProp -> Defaults.Key.toString, + PartitionProp -> Defaults.Partition.toString, + CompressionCodecProp -> Defaults.CompressionCodec.toString, + RequiredAcksProp -> Defaults.RequiredAcks.toString, + MessageTimeoutMsProp -> Defaults.MessageTimeoutMs.toString) + def configMaps = ConfigMaps + val ConfigNames = ConfigMaps.keySet + def configNames = ConfigNames + + /** + * Parse the given properties instance into a LogConfig object + */ + def fromProps(props: Properties): LogConfig = { + new LogConfig(valid = props.getProperty(ValidProp, Defaults.Valid.toString).toBoolean, + followLast = props.getProperty(FollowLastProp, Defaults.FollowLast.toString).toBoolean, + batchSize = props.getProperty(BatchSizeProp, Defaults.BatchSize.toString).toLong, + topic = props.getProperty(TopicProp, Defaults.Topic.toString).toString, + key = props.getProperty(KeyProp, Defaults.Key.toString).toString, + partition = props.getProperty(PartitionProp, Defaults.Partition.toString).toInt, + compressionCodec = props.getProperty(CompressionCodecProp, Defaults.CompressionCodec.toString).toString, + requiredAcks= props.getProperty(RequiredAcksProp, Defaults.RequiredAcks.toString).toInt, + messageTimeoutMs = props.getProperty(MessageTimeoutMsProp, Defaults.MessageTimeoutMs.toString).toLong) + } + + /** + * Create a log config instance using the given properties and defaults + */ + def fromProps(defaults: Properties, overrides: Properties): LogConfig = { + val props = new Properties(defaults) + props.putAll(overrides) + fromProps(props) + } + + /** + * Check that property names are valid + */ + def validateNames(props: Properties) { + import scala.collection.JavaConverters._ + for(name <- props.keys().asScala) + require(LogConfig.ConfigNames.asJava.contains(name), "Unknown configuration \"%s\".".format(name)) + } + + /** + * Check that the given properties contain only valid log config names, and that all values can be parsed. + */ + def validate(props: Properties) { + validateNames(props) + validateTopic(props) + LogConfig.fromProps(LogConfig().toProps, props) // check that we can parse the values + } + + /** + * Check that Topic is reasonable + */ + private def validateTopic(props: Properties) { + val topic = props.getProperty(TopicProp) + require(topic != null , "Topic is null") + } + +} diff --git a/app/models/form/LogkafkaOperation.scala b/app/models/form/LogkafkaOperation.scala new file mode 100644 index 000000000..5359bdf4b --- /dev/null +++ b/app/models/form/LogkafkaOperation.scala @@ -0,0 +1,19 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ + +package models.form + +/** + * @author hiral + */ + +sealed trait LogkafkaOperation + +case class LKConfig(name: String, value: Option[String]) + +case class CreateLogkafka(hostname: String, log_path: String, configs: List[LKConfig]) extends LogkafkaOperation +case class DeleteLogkafka(hostname: String, log_path: String) extends LogkafkaOperation +case class UpdateLogkafkaConfig(hostname: String, log_path: String, configs: List[LKConfig]) extends LogkafkaOperation +case class UnknownLKO(op: String) extends LogkafkaOperation diff --git a/app/models/navigation/BreadCrumbs.scala b/app/models/navigation/BreadCrumbs.scala index fa3becb7c..558e55fd2 100644 --- a/app/models/navigation/BreadCrumbs.scala +++ b/app/models/navigation/BreadCrumbs.scala @@ -52,6 +52,15 @@ object BreadCrumbs { "Clusters".baseRouteBreadCrumb, BCDynamicNamedLink(identity,"Summary".clusterRoute), "Topics".clusterRouteBreadCrumb), + "Logkafkas" -> IndexedSeq("Clusters".baseRouteBreadCrumb,BCDynamicNamedLink(identity,"Summary".clusterRoute)), + "Create Logkafka" -> IndexedSeq( + "Clusters".baseRouteBreadCrumb, + BCDynamicNamedLink(identity,"Summary".clusterRoute), + "Logkafkas".clusterRouteBreadCrumb), + "Logkafka View" -> IndexedSeq( + "Clusters".baseRouteBreadCrumb, + BCDynamicNamedLink(identity,"Summary".clusterRoute), + "Logkafkas".clusterRouteBreadCrumb), "Preferred Replica Election" -> IndexedSeq( "Clusters".baseRouteBreadCrumb, BCDynamicNamedLink(identity,"Summary".clusterRoute)), @@ -74,6 +83,15 @@ object BreadCrumbs { ) ) + val logkafkaBreadCrumbs: Map[String, IndexedSeq[BreadCrumb]] = Map( + "Logkafka View" -> IndexedSeq( + "Clusters".baseRouteBreadCrumb, + BCDynamicNamedLink(identity,"Summary".clusterRoute), + "Logkafkas".clusterRouteBreadCrumb, + BCDynamicMultiNamedLink(identity,"Logkafka View".logkafkaRoute) + ) + ) + def withView(s: String) : IndexedSeq[BreadCrumbRendered] = { val rendered : IndexedSeq[BreadCrumbRendered] = baseBreadCrumbs.getOrElse(s,IndexedSeq.empty[BreadCrumb]) map { case BCStaticLink(n,c) => BCLink(n,c.toString()) @@ -113,4 +131,19 @@ object BreadCrumbs { def withNamedViewAndClusterAndTopic(s: String, clusterName: String, topic: String, name: String) : IndexedSeq[BreadCrumbRendered] = { renderWithClusterAndTopic(s, clusterName,topic) :+ BCActive(name) } + + private[this] def renderWithClusterAndLogkafka(s: String, clusterName: String, hostname: String, log_path: String) : IndexedSeq[BreadCrumbRendered] = { + val hl = hostname + "?" + log_path + logkafkaBreadCrumbs.getOrElse(s,IndexedSeq.empty[BreadCrumb]) map { + case BCStaticLink(n,c) => BCLink(n,c.toString()) + case BCDynamicNamedLink(cn, cl) => BCLink(cn(clusterName),cl(clusterName).toString()) + case BCDynamicMultiNamedLink(cn, cl) => BCLink(cn(hl),cl(clusterName,hl).toString()) + case BCDynamicLink(cn, cl) => BCLink(cn,cl(clusterName).toString()) + case BCDynamicText(cn) => BCText(cn(clusterName)) + } + } + + def withNamedViewAndClusterAndLogkafka(s: String, clusterName: String, hostname: String, log_path: String, name: String) : IndexedSeq[BreadCrumbRendered] = { + renderWithClusterAndLogkafka(s, clusterName, hostname, log_path) :+ BCActive(name) + } } diff --git a/app/models/navigation/Menus.scala b/app/models/navigation/Menus.scala index 09657e692..1a7e7d2b7 100644 --- a/app/models/navigation/Menus.scala +++ b/app/models/navigation/Menus.scala @@ -22,7 +22,11 @@ object Menus { "Create".clusterRouteMenuItem(cluster)), None), "Preferred Replica Election".clusterMenu(cluster), - "Reassign Partitions".clusterMenu(cluster) + "Reassign Partitions".clusterMenu(cluster), + Menu("Logkafka",IndexedSeq( + "List Logkafka".clusterRouteMenuItem(cluster), + "Create Logkafka".clusterRouteMenuItem(cluster)), + None) ) def indexMenu : IndexedSeq[Menu] = IndexedSeq( diff --git a/app/models/navigation/QuickRoutes.scala b/app/models/navigation/QuickRoutes.scala index 08803ecfc..da118aef0 100644 --- a/app/models/navigation/QuickRoutes.scala +++ b/app/models/navigation/QuickRoutes.scala @@ -26,13 +26,20 @@ object QuickRoutes { "List" -> controllers.routes.Topic.topics, "Create" -> controllers.routes.Topic.createTopic, "Preferred Replica Election" -> controllers.routes.PreferredReplicaElection.preferredReplicaElection, - "Reassign Partitions" -> controllers.routes.ReassignPartitions.reassignPartitions + "Reassign Partitions" -> controllers.routes.ReassignPartitions.reassignPartitions, + "Logkafkas" -> controllers.routes.Logkafka.logkafkas, + "List Logkafka" -> controllers.routes.Logkafka.logkafkas, + "Create Logkafka" -> controllers.routes.Logkafka.createLogkafka ) val topicRoutes : Map[String, (String, String) => Call] = Map( "Topic View" -> controllers.routes.Topic.topic, "Add Partitions" -> controllers.routes.Topic.addPartitions, "Update Config" -> controllers.routes.Topic.addPartitions ) + val logkafkaRoutes : Map[String, (String, String, String) => Call] = Map( + "Logkafka View" -> controllers.routes.Logkafka.logkafka, + "Update Config" -> controllers.routes.Logkafka.updateConfig + ) implicit class BaseRoute(s: String) { def baseRouteMenuItem : (String, Call) = { @@ -72,4 +79,14 @@ object QuickRoutes { topicRoutes(s)(c,t) } } + + implicit class LogkafkaRoute(s: String) { + def logkafkaRouteMenuItem(c: String, h: String, l:String): (String, Call) = { + s -> logkafkaRoutes(s)(c,h,l) + } + def logkafkaRoute(c: String, hl: String): Call = { + val hlArray = hl.split("\\?") + logkafkaRoutes(s)(c,hlArray(0),hlArray(1)) + } + } } diff --git a/app/views/logkafka/createLogkafka.scala.html b/app/views/logkafka/createLogkafka.scala.html new file mode 100644 index 000000000..a05f08473 --- /dev/null +++ b/app/views/logkafka/createLogkafka.scala.html @@ -0,0 +1,52 @@ +@* +* Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 +* See accompanying LICENSE file. +*@ +@import scalaz.{\/} +@(cluster: String, errorOrForm: kafka.manager.ApiError \/ Form[models.form.CreateLogkafka])(implicit request: RequestHeader) + +@import helper._ +@import b3.vertical.fieldConstructor +@import controllers.routes + +@theMenu = { + @views.html.navigation.clusterMenu(cluster,"Logkafka","Create",models.navigation.Menus.clusterMenus(cluster)) +} + +@renderForm(createLogkafkaForm: Form[models.form.CreateLogkafka]) = { +
+ @b3.form(routes.Logkafka.handleCreateLogkafka(cluster)) { + + + + + + + +
+ @b3.text(createLogkafkaForm("hostname"), '_label -> "Hostname", 'placeholder -> "test.logkafka.net", 'autofocus -> true ) + @b3.text(createLogkafkaForm("log_path"), '_label -> "Log Path", 'placeholder -> "/usr/local/apache2/logs/access_log.%Y%m%d") + @b3.submit('class -> "submit-button btn btn-primary"){ Create } + Cancel + + @helper.repeat(createLogkafkaForm("configs"), min = 1) { configsForm => + @b3.hidden(configsForm("name").name, configsForm("name").value.getOrElse("")) + @b3.text(configsForm("value"), '_label -> configsForm("name").value.getOrElse("")) + } +
+ } +
+} + +@main( + "Create Logkafka", + menu = theMenu, + breadcrumbs=views.html.navigation.breadCrumbs(models.navigation.BreadCrumbs.withViewAndCluster("Create Logkafka",cluster))) { +
+
+

Create Logkafka

+ @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) +
+
+} + diff --git a/app/views/logkafka/logkafkaList.scala.html b/app/views/logkafka/logkafkaList.scala.html new file mode 100644 index 000000000..4d9f1f046 --- /dev/null +++ b/app/views/logkafka/logkafkaList.scala.html @@ -0,0 +1,33 @@ +@* +* Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 +* See accompanying LICENSE file. +*@ +@import scalaz.{\/} +@(cluster:String, errorOrLogkafkas: kafka.manager.ApiError \/ kafka.manager.LogkafkaListExtended) + +@theMenu = { + @views.html.navigation.clusterMenu(cluster,"Logkafka","List",models.navigation.Menus.clusterMenus(cluster)) +} + +@logkafkaScripts = { + +} + +@main( + "Logkafka List", + menu = theMenu, + breadcrumbs=views.html.navigation.breadCrumbs(models.navigation.BreadCrumbs.withViewAndCluster("Logkafkas",cluster)), + scripts=logkafkaScripts) { +
+
+

Logkafkas

+ @errorOrLogkafkas.fold( + views.html.errors.onApiError(_), + tl => views.html.logkafka.logkafkaListContent(cluster,tl.list.map(t => (t, tl.deleteSet(t._1))))) +
+
+} diff --git a/app/views/logkafka/logkafkaListContent.scala.html b/app/views/logkafka/logkafkaListContent.scala.html new file mode 100644 index 000000000..3ae690ad0 --- /dev/null +++ b/app/views/logkafka/logkafkaListContent.scala.html @@ -0,0 +1,62 @@ +@* +* Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 +* See accompanying LICENSE file. +*@ +@(cluster: String, logkafkas: IndexedSeq[((String, Option[kafka.manager.ActorModel.LogkafkaIdentity]),Boolean)]) + +@getFilesizeStatus(filesize: Int) = { +@filesize match { + case i if i < 0 => {warning} + case i => {} +} +} + +@getLogkafkaStatus(active: Boolean) = { + @if(active) {} else {danger} +} + + + + + + + @for( ((hostname, logkafkaIdentity), deleted) <- logkafkas) { + @logkafkaIdentity.map{ li => + @for( (log_path, im) <- li.identityMap) { + + + + @im._1.map { c => + + }.getOrElse{} + + @im._2.map { c => + @c.get("realpath").map { d => + + }.getOrElse{} + }.getOrElse{} + + @im._2.map { c => + @c.get("filepos").map { d => + + }.getOrElse{} + }.getOrElse{} + + @im._2.map { c => + @c.get("filesize").map { d => + + }.getOrElse{} + }.getOrElse{} + + @im._1.map { c => + @c.get("topic").map { d => + + }.getOrElse{} + }.getOrElse{} + + + } + }.getOrElse{} + } + +
Hostname# Log Path# Real Path# File Pos# File Size# Topic
@hostname@log_path @d no corresponding file to collect logkafka in @hostname is inactive @d @d @d
diff --git a/app/views/logkafka/logkafkaView.scala.html b/app/views/logkafka/logkafkaView.scala.html new file mode 100644 index 000000000..bfd37bdc7 --- /dev/null +++ b/app/views/logkafka/logkafkaView.scala.html @@ -0,0 +1,28 @@ +@* +* Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 +* See accompanying LICENSE file. +*@ +@import scalaz.{\/} +@(cluster:String, + hostname: String, + log_path: String, + errorOrLogkafkaIdentity: kafka.manager.ApiError \/ kafka.manager.ActorModel.LogkafkaIdentity +) + +@theMenu = { + @views.html.navigation.clusterMenu(cluster,"Logkafka","",models.navigation.Menus.clusterMenus(cluster)) +} + +@main( + "Logkafka View", + menu = theMenu, + breadcrumbs=views.html.navigation.breadCrumbs(models.navigation.BreadCrumbs.withNamedViewAndCluster("Logkafka View",cluster,hostname))) { +
+
+
+

@hostname
@log_path

+
+ @errorOrLogkafkaIdentity.fold(views.html.errors.onApiError(_),views.html.logkafka.logkafkaViewContent(cluster,hostname,log_path,_)) +
+
+} diff --git a/app/views/logkafka/logkafkaViewContent.scala.html b/app/views/logkafka/logkafkaViewContent.scala.html new file mode 100644 index 000000000..ab2652bbb --- /dev/null +++ b/app/views/logkafka/logkafkaViewContent.scala.html @@ -0,0 +1,71 @@ +@* +* Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 +* See accompanying LICENSE file. +*@ +@import b3.vertical.fieldConstructor +@(cluster:String, hostname: String, log_path: String, logkafkaIdentity: kafka.manager.ActorModel.LogkafkaIdentity) + +
+
+
+

Logkafka Summary

+ + + + + + + + + + +
Hostname@hostname
Log Path@log_path
+ @defining(logkafkaIdentity.identityMap.get(log_path).get) { identityTuple => + @if(identityTuple._1.isDefined) { + + + + + + @identityTuple._1.map { config => + @for((k, v) <- config) { + + + + + } + } + +
ConfigValue
@k@v
+ } + } +
+
+
+
+

Operations

+ + + + @if(logkafkaIdentity.deleteSupported) { + + } + + + + + +
+ @b3.form(routes.Logkafka.handleDeleteLogkafka(cluster, hostname, log_path)) { +
+ @b3.hidden("hostname", hostname) + @b3.hidden("log_path", log_path) + @b3.submit('class -> "submit-button btn btn-primary"){ Delete Logkafka } +
+ } +
+ Update Config +
+
+
+
diff --git a/app/views/logkafka/updateConfig.scala.html b/app/views/logkafka/updateConfig.scala.html new file mode 100644 index 000000000..4a75a6f9a --- /dev/null +++ b/app/views/logkafka/updateConfig.scala.html @@ -0,0 +1,53 @@ +@* +* Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 +* See accompanying LICENSE file. +*@ +@import scalaz.{\/} +@(cluster: String, hostname: String, log_path: String, errorOrForm: kafka.manager.ApiError \/ Form[models.form.UpdateLogkafkaConfig])(implicit request: RequestHeader) + +@import helper._ +@import b3.vertical.fieldConstructor +@import controllers.routes + +@theMenu = { + @views.html.navigation.clusterMenu(cluster,"Logkafka","Update Config",models.navigation.Menus.clusterMenus(cluster)) +} + +@renderForm(updateLogkafkaConfigForm: Form[models.form.UpdateLogkafkaConfig]) = { + @b3.form(routes.Logkafka.handleUpdateConfig(cluster, hostname, log_path)) { + + + + + + + + + +
Update Config
+ @b3.text(updateLogkafkaConfigForm("hostname"), '_label -> "Hostname", 'placeholder -> "", 'autofocus -> true ) + @b3.text(updateLogkafkaConfigForm("log_path"), '_label -> "Log Path", 'placeholder -> "", 'autofocus -> true ) + @helper.repeat(updateLogkafkaConfigForm("configs"), min = 1) { configsForm => + @b3.hidden(configsForm("name").name, configsForm("name").value.getOrElse("")) + @b3.text(configsForm("value"), '_label -> configsForm("name").value.getOrElse("")) + } +
+ @b3.submit('class -> "submit-button btn btn-primary"){ Update Config } + Cancel + } +} + +@main( + "Update Config", + menu = theMenu, + breadcrumbs=views.html.navigation.breadCrumbs(models.navigation.BreadCrumbs.withNamedViewAndClusterAndLogkafka("Logkafka View",cluster,hostname,log_path,"Update Config"))) { +
+
+

Update Config

+
+ @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) +
+
+
+} + diff --git a/conf/routes b/conf/routes index 73c2dec6b..44353cb86 100644 --- a/conf/routes +++ b/conf/routes @@ -10,6 +10,8 @@ GET / controllers.Applicatio GET /clusters/:c controllers.Application.cluster(c:String) GET /clusters/:c/topics controllers.Topic.topics(c:String) GET /clusters/:c/topics/:t controllers.Topic.topic(c:String, t:String) +GET /clusters/:c/logkafkas controllers.Logkafka.logkafkas(c:String) +GET /clusters/:c/logkafkas/:h/:l controllers.Logkafka.logkafka(c:String, h:String, l:String) GET /clusters/:c/brokers controllers.Application.brokers(c: String) GET /clusters/:c/brokers/:b controllers.Application.broker(c: String, b:Int) GET /clusters/:c/leader controllers.PreferredReplicaElection.preferredReplicaElection(c:String) @@ -35,6 +37,11 @@ GET /clusters/:c/topics/:t/addPartitions controllers.Topic.addP POST /clusters/:c/topics/:t/addPartitions controllers.Topic.handleAddPartitions(c:String, t: String) GET /clusters/:c/topics/:t/updateConfig controllers.Topic.updateConfig(c:String, t: String) POST /clusters/:c/topics/:t/updateConfig controllers.Topic.handleUpdateConfig(c:String, t: String) +GET /clusters/:c/createLogkafka controllers.Logkafka.createLogkafka(c:String) +POST /clusters/:c/logkafkas/create controllers.Logkafka.handleCreateLogkafka(c:String) +POST /clusters/:c/logkafkas/delete controllers.Logkafka.handleDeleteLogkafka(c:String, h:String, l:String) +GET /clusters/:c/logkafkas/:h/:l/updateConfig controllers.Logkafka.updateConfig(c:String, h:String, l:String) +POST /clusters/:c/logkafkas/:h/:l/updateConfig controllers.Logkafka.handleUpdateConfig(c:String, h:String, l:String) GET /api/status/:c/availableBrokers controllers.api.KafkaHealthCheck.availableBrokers(c:String) GET /api/status/:c/:t/underReplicatedPartitions controllers.api.KafkaHealthCheck.underReplicatedPartitions(c:String, t: String) GET /api/status/:c/:t/unavailablePartitions controllers.api.KafkaHealthCheck.unavailablePartitions(c:String, t:String) From dc93f760157c1bd2f9b2e767c143950dd78f4d88 Mon Sep 17 00:00:00 2001 From: zheolong Date: Thu, 23 Jul 2015 13:36:53 +0800 Subject: [PATCH 02/17] Logkafka Add "Enable Logkafka" check box to "Add Cluster" and "Update Cluster" form When enabled, LogkafkaStateActor and LogkafkaViewCacheActor will be started --- app/controllers/Cluster.scala | 13 +- app/controllers/Logkafka.scala | 16 +- app/kafka/manager/ActorModel.scala | 21 +- app/kafka/manager/BrokerViewCacheActor.scala | 29 --- app/kafka/manager/ClusterManagerActor.scala | 37 ++- app/kafka/manager/KafkaManager.scala | 12 +- app/kafka/manager/KafkaManagerActor.scala | 17 +- app/kafka/manager/KafkaStateActor.scala | 128 ----------- app/kafka/manager/LogkafkaStateActor.scala | 210 ++++++++++++++++++ .../manager/LogkafkaViewCacheActor.scala | 104 +++++++++ app/models/form/ClusterOperation.scala | 8 +- app/views/cluster/addCluster.scala.html | 1 + app/views/cluster/updateCluster.scala.html | 1 + .../logkafka/logkafkaViewContent.scala.html | 36 +-- 14 files changed, 421 insertions(+), 212 deletions(-) create mode 100644 app/kafka/manager/LogkafkaStateActor.scala create mode 100644 app/kafka/manager/LogkafkaViewCacheActor.scala diff --git a/app/controllers/Cluster.scala b/app/controllers/Cluster.scala index dcceadbd1..62ce60d74 100644 --- a/app/controllers/Cluster.scala +++ b/app/controllers/Cluster.scala @@ -67,7 +67,8 @@ object Cluster extends Controller { "kafkaVersion" -> nonEmptyText.verifying(validateKafkaVersion), "zkHosts" -> nonEmptyText.verifying(validateZkHosts), "zkMaxRetry" -> ignored(100 : Int), - "jmxEnabled" -> boolean + "jmxEnabled" -> boolean, + "logkafkaEnabled" -> boolean )(ClusterConfig.apply)(ClusterConfig.customUnapply) ) @@ -78,7 +79,8 @@ object Cluster extends Controller { "kafkaVersion" -> nonEmptyText.verifying(validateKafkaVersion), "zkHosts" -> nonEmptyText.verifying(validateZkHosts), "zkMaxRetry" -> ignored(100 : Int), - "jmxEnabled" -> boolean + "jmxEnabled" -> boolean, + "logkafkaEnabled" -> boolean )(ClusterOperation.apply)(ClusterOperation.customUnapply) ) @@ -89,7 +91,7 @@ object Cluster extends Controller { def updateCluster(c: String) = Action.async { implicit request => kafkaManager.getClusterConfig(c).map { errorOrClusterConfig => Ok(views.html.cluster.updateCluster(c,errorOrClusterConfig.map { cc => - updateForm.fill(ClusterOperation.apply(Update.toString,cc.name,cc.version.toString,cc.curatorConfig.zkConnect,cc.curatorConfig.zkMaxRetry,cc.jmxEnabled)) + updateForm.fill(ClusterOperation.apply(Update.toString,cc.name,cc.version.toString,cc.curatorConfig.zkConnect,cc.curatorConfig.zkMaxRetry,cc.jmxEnabled,cc.logkafkaEnabled)) })) } } @@ -98,7 +100,7 @@ object Cluster extends Controller { clusterConfigForm.bindFromRequest.fold( formWithErrors => Future.successful(BadRequest(views.html.cluster.addCluster(formWithErrors))), clusterConfig => { - kafkaManager.addCluster(clusterConfig.name, clusterConfig.version.toString, clusterConfig.curatorConfig.zkConnect, clusterConfig.jmxEnabled).map { errorOrSuccess => + kafkaManager.addCluster(clusterConfig.name, clusterConfig.version.toString, clusterConfig.curatorConfig.zkConnect, clusterConfig.jmxEnabled, clusterConfig.logkafkaEnabled).map { errorOrSuccess => Ok(views.html.common.resultOfCommand( views.html.navigation.defaultMenu(), models.navigation.BreadCrumbs.withView("Add Cluster"), @@ -154,7 +156,8 @@ object Cluster extends Controller { clusterOperation.clusterConfig.name, clusterOperation.clusterConfig.version.toString, clusterOperation.clusterConfig.curatorConfig.zkConnect, - clusterOperation.clusterConfig.jmxEnabled + clusterOperation.clusterConfig.jmxEnabled, + clusterOperation.clusterConfig.logkafkaEnabled ).map { errorOrSuccess => Ok(views.html.common.resultOfCommand( views.html.navigation.defaultMenu(), diff --git a/app/controllers/Logkafka.scala b/app/controllers/Logkafka.scala index 1cfc90257..e98e04f67 100644 --- a/app/controllers/Logkafka.scala +++ b/app/controllers/Logkafka.scala @@ -170,9 +170,19 @@ object Logkafka extends Controller{ case Kafka_0_8_2_0 => LogkafkaNewConfigs.configNames(Kafka_0_8_2_0).map(n => (n,LKConfig(n,None))).toMap case Kafka_0_8_2_1 => LogkafkaNewConfigs.configNames(Kafka_0_8_2_1).map(n => (n,LKConfig(n,None))).toMap } - val config: Map[String, String] = li.identityMap.get(log_path).get._1.get - val combinedMap = defaultConfigMap ++ config.map(tpl => tpl._1 -> LKConfig(tpl._1,Option(tpl._2))) - defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,combinedMap.toList.map(_._2))) + val identityOption = li.identityMap.get(log_path) + if (identityOption.isDefined) { + val configOption = identityOption.get._1 + if (configOption.isDefined) { + val config: Map[String, String] = configOption.get + val combinedMap = defaultConfigMap ++ config.map(tpl => tpl._1 -> LKConfig(tpl._1,Option(tpl._2))) + defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,combinedMap.toList.map(_._2))) + } else { + defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,List(LKConfig("",None)))) + } + } else { + defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,List(LKConfig("",None)))) + } } } } diff --git a/app/kafka/manager/ActorModel.scala b/app/kafka/manager/ActorModel.scala index 45e62f48d..c2078dfbb 100644 --- a/app/kafka/manager/ActorModel.scala +++ b/app/kafka/manager/ActorModel.scala @@ -33,7 +33,6 @@ object ActorModel { case object BVForceUpdate extends CommandRequest case object BVGetTopicIdentities extends BVRequest - case object BVGetLogkafkaIdentities extends BVRequest case class BVGetView(id: Int) extends BVRequest case object BVGetViews extends BVRequest case class BVGetTopicMetrics(topic: String) extends BVRequest @@ -47,6 +46,11 @@ object ActorModel { case class BVUpdateTopicMetricsForBroker(id: Int, metrics: IndexedSeq[(String,BrokerMetrics)]) extends CommandRequest case class BVUpdateBrokerMetrics(id: Int, metric: BrokerMetrics) extends CommandRequest + sealed trait LKVRequest extends QueryRequest + + case object LKVForceUpdate extends CommandRequest + case object LKVGetLogkafkaIdentities extends LKVRequest + case object CMGetView extends QueryRequest case class CMGetTopicIdentity(topic: String) extends QueryRequest case class CMView(topicsCount: Int, brokersCount: Int, clusterConfig: ClusterConfig) extends QueryResponse @@ -149,13 +153,14 @@ object ActorModel { case object KSGetBrokers extends KSRequest case class KSGetBrokerState(id: String) extends KSRequest - case object KSGetLogkafkaHostnames extends KSRequest - case class KSGetLogkafkaConfig(hostname: String) extends KSRequest - case class KSGetLogkafkaClient(hostname: String) extends KSRequest - case class KSGetLogkafkaConfigs(hostnames: Set[String]) extends KSRequest - case class KSGetLogkafkaClients(hostnames: Set[String]) extends KSRequest - case class KSGetAllLogkafkaConfigs(lastUpdateMillis: Option[Long]= None) extends KSRequest - case class KSGetAllLogkafkaClients(lastUpdateMillis: Option[Long]= None) extends KSRequest + sealed trait LKSRequest extends QueryRequest + case object LKSGetLogkafkaHostnames extends LKSRequest + case class LKSGetLogkafkaConfig(hostname: String) extends LKSRequest + case class LKSGetLogkafkaClient(hostname: String) extends LKSRequest + case class LKSGetLogkafkaConfigs(hostnames: Set[String]) extends LKSRequest + case class LKSGetLogkafkaClients(hostnames: Set[String]) extends LKSRequest + case class LKSGetAllLogkafkaConfigs(lastUpdateMillis: Option[Long]= None) extends LKSRequest + case class LKSGetAllLogkafkaClients(lastUpdateMillis: Option[Long]= None) extends LKSRequest case class TopicList(list: IndexedSeq[String], deleteSet: Set[String]) extends QueryResponse case class TopicConfig(topic: String, config: Option[(Int,String)]) extends QueryResponse diff --git a/app/kafka/manager/BrokerViewCacheActor.scala b/app/kafka/manager/BrokerViewCacheActor.scala index 40dcc35f1..24f5bbeea 100644 --- a/app/kafka/manager/BrokerViewCacheActor.scala +++ b/app/kafka/manager/BrokerViewCacheActor.scala @@ -40,12 +40,6 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni private[this] var combinedBrokerMetric : Option[BrokerMetrics] = None - private[this] var logkafkaIdentities : Map[String, LogkafkaIdentity] = Map.empty - - private[this] var logkafkaConfigsOption : Option[LogkafkaConfigs] = None - - private[this] var logkafkaClientsOption : Option[LogkafkaClients] = None - private[this] val EMPTY_BVVIEW = BVView(Map.empty, config.clusterConfig, Option(BrokerMetrics.DEFAULT)) override def preStart() = { @@ -115,8 +109,6 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni val lastUpdateMillisOption: Option[Long] = topicDescriptionsOption.map(_.lastUpdateMillis) context.actorSelection(config.kafkaStateActorPath).tell(KSGetAllTopicDescriptions(lastUpdateMillisOption), self) context.actorSelection(config.kafkaStateActorPath).tell(KSGetBrokers, self) - context.actorSelection(config.kafkaStateActorPath).tell(KSGetAllLogkafkaConfigs(lastUpdateMillisOption), self) - context.actorSelection(config.kafkaStateActorPath).tell(KSGetAllLogkafkaClients(lastUpdateMillisOption), self) case BVGetViews => sender ! allBrokerViews() @@ -136,9 +128,6 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni case BVGetTopicIdentities => sender ! topicIdentities - case BVGetLogkafkaIdentities => - sender ! logkafkaIdentities - case BVUpdateTopicMetricsForBroker(id, metrics) => metrics.foreach { case (topic, bm) => @@ -168,14 +157,6 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni brokerListOption = Some(bl) updateView() - case lcg: LogkafkaConfigs => - logkafkaConfigsOption = Some(lcg) - updateView() - - case lct: LogkafkaClients => - logkafkaClientsOption = Some(lct) - updateView() - case any: Any => log.warning("bvca : processActorResponse : Received unknown message: {}", any) } } @@ -258,15 +239,5 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni brokerId,BVView(topicPartitionsMap, config.clusterConfig, brokerMetrics.get(brokerId))) } } - - for { - logkafkaConfigs <- logkafkaConfigsOption - logkafkaClients <- logkafkaClientsOption - } { - val lcgMap = Map(logkafkaConfigs.configs map { a => a.hostname -> a }: _*) - val lctMap = Map(logkafkaClients.clients map { a => a.hostname -> a }: _*) - logkafkaIdentities = lcgMap.map (kv => - kv._1 -> LogkafkaIdentity.from(kv._1, Some(kv._2), lctMap.get(kv._1))) - } } } diff --git a/app/kafka/manager/ClusterManagerActor.scala b/app/kafka/manager/ClusterManagerActor.scala index ee77fea1f..00ee8ad45 100644 --- a/app/kafka/manager/ClusterManagerActor.scala +++ b/app/kafka/manager/ClusterManagerActor.scala @@ -90,6 +90,9 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) private[this] val ksProps = Props(classOf[KafkaStateActor],sharedClusterCurator, adminUtils.isDeleteSupported, cmConfig.clusterConfig) private[this] val kafkaStateActor : ActorPath = context.actorOf(ksProps.withDispatcher(cmConfig.pinnedDispatcherName),"kafka-state").path + private[this] val lksProps = Props(classOf[LogkafkaStateActor],sharedClusterCurator, adminUtils.isDeleteSupported, cmConfig.clusterConfig) + private[this] val logkafkaStateActor : ActorPath = context.actorOf(lksProps.withDispatcher(cmConfig.pinnedDispatcherName),"logkafka-state").path + private[this] val bvConfig = BrokerViewCacheActorConfig( kafkaStateActor, cmConfig.clusterConfig, @@ -98,6 +101,14 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) private[this] val bvcProps = Props(classOf[BrokerViewCacheActor],bvConfig) private[this] val brokerViewCacheActor : ActorPath = context.actorOf(bvcProps,"broker-view").path + private[this] val lkvConfig = LogkafkaViewCacheActorConfig( + logkafkaStateActor, + cmConfig.clusterConfig, + LongRunningPoolConfig(Runtime.getRuntime.availableProcessors(), 1000), + cmConfig.updatePeriod) + private[this] val lkvcProps = Props(classOf[LogkafkaViewCacheActor],lkvConfig) + private[this] val logkafkaViewCacheActor : ActorPath = context.actorOf(lkvcProps,"logkafka-view").path + private[this] val kcProps = { val kcaConfig = KafkaCommandActorConfig( sharedClusterCurator, @@ -149,9 +160,15 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) case ksRequest: KSRequest => context.actorSelection(kafkaStateActor).forward(ksRequest) + case lksRequest: LKSRequest => + context.actorSelection(logkafkaStateActor).forward(lksRequest) + case bvRequest: BVRequest => context.actorSelection(brokerViewCacheActor).forward(bvRequest) + case lkvRequest: LKVRequest => + context.actorSelection(logkafkaViewCacheActor).forward(lkvRequest) + case CMGetView => implicit val ec = context.dispatcher val eventualBrokerList = withKafkaStateActor(KSGetBrokers)(identity[BrokerList]) @@ -182,8 +199,8 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) case CMGetLogkafkaIdentity(hostname) => implicit val ec = context.dispatcher - val eventualLogkafkaConfig= withKafkaStateActor(KSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) - val eventualLogkafkaClient= withKafkaStateActor(KSGetLogkafkaClient(hostname))(identity[Option[LogkafkaClient]]) + val eventualLogkafkaConfig= withLogkafkaStateActor(LKSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) + val eventualLogkafkaClient= withLogkafkaStateActor(LKSGetLogkafkaClient(hostname))(identity[Option[LogkafkaClient]]) val result: Future[Option[CMLogkafkaIdentity]] = for { lcg <- eventualLogkafkaConfig lct <- eventualLogkafkaClient @@ -379,7 +396,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) case CMDeleteLogkafka(hostname, log_path) => implicit val ec = longRunningExecutionContext - val eventualLogkafkaConfig = withKafkaStateActor(KSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) + val eventualLogkafkaConfig = withLogkafkaStateActor(LKSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) eventualLogkafkaConfig.map { logkafkaConfigOption => logkafkaConfigOption.fold { Future.successful(CMCommandResult(Failure(new IllegalArgumentException(s"Hostname doesn't exists : $hostname")))) @@ -393,7 +410,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) case CMCreateLogkafka(hostname, log_path, config) => implicit val ec = longRunningExecutionContext - val eventualLogkafkaConfig = withKafkaStateActor(KSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) + val eventualLogkafkaConfig = withLogkafkaStateActor(LKSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) eventualLogkafkaConfig.map { logkafkaConfigOption => withKafkaCommandActor(KCCreateLogkafka(hostname, log_path, config, logkafkaConfigOption)) { kcResponse: KCCommandResult => @@ -403,7 +420,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) case CMUpdateLogkafkaConfig(hostname, log_path, config) => implicit val ec = longRunningExecutionContext - val eventualLogkafkaConfig = withKafkaStateActor(KSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) + val eventualLogkafkaConfig = withLogkafkaStateActor(LKSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) eventualLogkafkaConfig.map { logkafkaConfigOption => withKafkaCommandActor(KCUpdateLogkafkaConfig(hostname, log_path, config, logkafkaConfigOption)) { kcResponse: KCCommandResult => @@ -420,11 +437,21 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) context.actorSelection(kafkaStateActor).ask(msg).mapTo[Output].map(fn) } + private[this] def withLogkafkaStateActor[Input,Output,FOutput] + (msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output], ec: ExecutionContext) : Future[FOutput] = { + context.actorSelection(logkafkaStateActor).ask(msg).mapTo[Output].map(fn) + } + private[this] def withBrokerViewCacheActor[Input,Output,FOutput] (msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output], ec: ExecutionContext) : Future[FOutput] = { context.actorSelection(brokerViewCacheActor).ask(msg).mapTo[Output].map(fn) } + private[this] def withLogkafkaViewCacheActor[Input,Output,FOutput] + (msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output], ec: ExecutionContext) : Future[FOutput] = { + context.actorSelection(logkafkaViewCacheActor).ask(msg).mapTo[Output].map(fn) + } + private[this] def withKafkaCommandActor[Input,Output,FOutput] (msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output], ec: ExecutionContext) : Future[FOutput] = { context.actorSelection(kafkaCommandActor).ask(msg).mapTo[Output].map(fn) diff --git a/app/kafka/manager/KafkaManager.scala b/app/kafka/manager/KafkaManager.scala index af829d191..71f3dd511 100644 --- a/app/kafka/manager/KafkaManager.scala +++ b/app/kafka/manager/KafkaManager.scala @@ -171,19 +171,19 @@ class KafkaManager(akkaConfig: Config) } //--------------------Commands-------------------------- - def addCluster(clusterName: String, version: String, zkHosts: String, jmxEnabled: Boolean): Future[ApiError \/ + def addCluster(clusterName: String, version: String, zkHosts: String, jmxEnabled: Boolean, logkafkaEnabled: Boolean = false): Future[ApiError \/ Unit] = { - val cc = ClusterConfig(clusterName, version, zkHosts, jmxEnabled = jmxEnabled) + val cc = ClusterConfig(clusterName, version, zkHosts, jmxEnabled = jmxEnabled, logkafkaEnabled = logkafkaEnabled) tryWithKafkaManagerActor(KMAddCluster(cc)) { result: KMCommandResult => result.result.get } } - def updateCluster(clusterName: String, version: String, zkHosts: String, jmxEnabled: Boolean): Future[ApiError \/ + def updateCluster(clusterName: String, version: String, zkHosts: String, jmxEnabled: Boolean, logkafkaEnabled: Boolean = false): Future[ApiError \/ Unit] = { - val cc = ClusterConfig(clusterName, version, zkHosts, jmxEnabled = jmxEnabled) + val cc = ClusterConfig(clusterName, version, zkHosts, jmxEnabled = jmxEnabled, logkafkaEnabled = logkafkaEnabled) tryWithKafkaManagerActor(KMUpdateCluster(cc)) { result: KMCommandResult => result.result.get } @@ -571,8 +571,8 @@ class KafkaManager(akkaConfig: Config) } def getLogkafkaListExtended(clusterName: String): Future[ApiError \/ LogkafkaListExtended] = { - val futureLogkafkaIdentities = tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, BVGetLogkafkaIdentities))(identity[Map[String, LogkafkaIdentity]]) - val futureLogkafkaList = tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, KSGetLogkafkaHostnames))(identity[LogkafkaHostnameList]) + val futureLogkafkaIdentities = tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, LKVGetLogkafkaIdentities))(identity[Map[String, LogkafkaIdentity]]) + val futureLogkafkaList = tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, LKSGetLogkafkaHostnames))(identity[LogkafkaHostnameList]) implicit val ec = apiExecutionContext for { errOrLi <- futureLogkafkaIdentities diff --git a/app/kafka/manager/KafkaManagerActor.scala b/app/kafka/manager/KafkaManagerActor.scala index cc8f1a6ff..2b1e4d6a1 100644 --- a/app/kafka/manager/KafkaManagerActor.scala +++ b/app/kafka/manager/KafkaManagerActor.scala @@ -81,18 +81,18 @@ object ClusterConfig { require(zkHosts.length > 0, "cluster zk hosts is illegal, can't be empty!") } - def apply(name: String, version : String, zkHosts: String, zkMaxRetry: Int = 100, jmxEnabled: Boolean) : ClusterConfig = { + def apply(name: String, version : String, zkHosts: String, zkMaxRetry: Int = 100, jmxEnabled: Boolean, logkafkaEnabled: Boolean = false) : ClusterConfig = { val kafkaVersion = KafkaVersion(version) //validate cluster name validateName(name) //validate zk hosts validateZkHosts(zkHosts) val cleanZkHosts = zkHosts.replaceAll(" ","") - new ClusterConfig(name, CuratorConfig(cleanZkHosts, zkMaxRetry), true, kafkaVersion, jmxEnabled) + new ClusterConfig(name, CuratorConfig(cleanZkHosts, zkMaxRetry), true, kafkaVersion, jmxEnabled, logkafkaEnabled) } - def customUnapply(cc: ClusterConfig) : Option[(String, String, String, Int, Boolean)] = { - Some((cc.name, cc.version.toString, cc.curatorConfig.zkConnect, cc.curatorConfig.zkMaxRetry, cc.jmxEnabled)) + def customUnapply(cc: ClusterConfig) : Option[(String, String, String, Int, Boolean, Boolean)] = { + Some((cc.name, cc.version.toString, cc.curatorConfig.zkConnect, cc.curatorConfig.zkMaxRetry, cc.jmxEnabled, cc.logkafkaEnabled)) } import scalaz.{Failure,Success} @@ -123,6 +123,7 @@ object ClusterConfig { :: ("enabled" -> toJSON(config.enabled)) :: ("kafkaVersion" -> toJSON(config.version.toString)) :: ("jmxEnabled" -> toJSON(config.jmxEnabled)) + :: ("logkafkaEnabled" -> toJSON(config.logkafkaEnabled)) :: Nil) compact(render(json)).getBytes(StandardCharsets.UTF_8) } @@ -137,7 +138,8 @@ object ClusterConfig { val versionString = field[String]("kafkaVersion")(json) val version = versionString.map(KafkaVersion.apply).getOrElse(Kafka_0_8_1_1) val jmxEnabled = field[Boolean]("jmxEnabled")(json) - ClusterConfig.apply(name,curatorConfig,enabled,version,jmxEnabled.getOrElse(false)) + val logkafkaEnabled = field[Boolean]("logkafkaEnabled")(json) + ClusterConfig.apply(name,curatorConfig,enabled,version,jmxEnabled.getOrElse(false),logkafkaEnabled.getOrElse(false)) } result match { @@ -152,7 +154,7 @@ object ClusterConfig { } -case class ClusterConfig (name: String, curatorConfig : CuratorConfig, enabled: Boolean, version: KafkaVersion, jmxEnabled: Boolean) +case class ClusterConfig (name: String, curatorConfig : CuratorConfig, enabled: Boolean, version: KafkaVersion, jmxEnabled: Boolean, logkafkaEnabled: Boolean) object KafkaManagerActor { val ZkRoot : String = "/kafka-manager" @@ -544,7 +546,8 @@ class KafkaManagerActor(kafkaManagerConfig: KafkaManagerActorConfig) if(newConfig.curatorConfig.zkConnect == currentConfig.curatorConfig.zkConnect && newConfig.enabled == currentConfig.enabled && newConfig.version == currentConfig.version - && newConfig.jmxEnabled == currentConfig.jmxEnabled) { + && newConfig.jmxEnabled == currentConfig.jmxEnabled + && newConfig.logkafkaEnabled == currentConfig.logkafkaEnabled) { //nothing changed false } else { diff --git a/app/kafka/manager/KafkaStateActor.scala b/app/kafka/manager/KafkaStateActor.scala index bf884ce64..6abbf9d3a 100644 --- a/app/kafka/manager/KafkaStateActor.scala +++ b/app/kafka/manager/KafkaStateActor.scala @@ -39,16 +39,6 @@ class KafkaStateActor(curator: CuratorFramework, @volatile private[this] var topicsTreeCacheLastUpdateMillis : Long = System.currentTimeMillis() - private[this] val logkafkaConfigTreeCache = new TreeCache(curator,ZkUtils.LogkafkaConfigPath) - - private[this] val logkafkaClientTreeCache = new TreeCache(curator,ZkUtils.LogkafkaClientPath) - - @volatile - private[this] var logkafkaConfigTreeCacheLastUpdateMillis : Long = System.currentTimeMillis() - - @volatile - private[this] var logkafkaClientTreeCacheLastUpdateMillis : Long = System.currentTimeMillis() - private[this] val topicsTreeCacheListener = new TreeCacheListener { override def childEvent(client: CuratorFramework, event: TreeCacheEvent): Unit = { event.getType match { @@ -61,30 +51,6 @@ class KafkaStateActor(curator: CuratorFramework, } } - private[this] val logkafkaConfigTreeCacheListener = new TreeCacheListener { - override def childEvent(client: CuratorFramework, event: TreeCacheEvent): Unit = { - event.getType match { - case TreeCacheEvent.Type.INITIALIZED | TreeCacheEvent.Type.NODE_ADDED | - TreeCacheEvent.Type.NODE_REMOVED | TreeCacheEvent.Type.NODE_UPDATED => - logkafkaConfigTreeCacheLastUpdateMillis = System.currentTimeMillis() - case _ => - //do nothing - } - } - } - - private[this] val logkafkaClientTreeCacheListener = new TreeCacheListener { - override def childEvent(client: CuratorFramework, event: TreeCacheEvent): Unit = { - event.getType match { - case TreeCacheEvent.Type.INITIALIZED | TreeCacheEvent.Type.NODE_ADDED | - TreeCacheEvent.Type.NODE_REMOVED | TreeCacheEvent.Type.NODE_UPDATED => - logkafkaClientTreeCacheLastUpdateMillis = System.currentTimeMillis() - case _ => - //do nothing - } - } - } - @volatile private[this] var preferredLeaderElection : Option[PreferredReplicaElection] = None @@ -157,17 +123,9 @@ class KafkaStateActor(curator: CuratorFramework, adminPathCache.start(StartMode.BUILD_INITIAL_CACHE) log.info("Starting delete topics path cache...") deleteTopicsPathCache.start(StartMode.BUILD_INITIAL_CACHE) - log.info("Starting logkafka config tree cache...") - logkafkaConfigTreeCache.start() - log.info("Starting logkafka client tree cache...") - logkafkaClientTreeCache.start() log.info("Adding topics tree cache listener...") topicsTreeCache.getListenable.addListener(topicsTreeCacheListener) - log.info("Adding logkafka config tree cache listener...") - logkafkaConfigTreeCache.getListenable.addListener(logkafkaConfigTreeCacheListener) - log.info("Adding logkafka client tree cache listener...") - logkafkaClientTreeCache.getListenable.addListener(logkafkaClientTreeCacheListener) log.info("Adding admin path cache listener...") adminPathCache.getListenable.addListener(adminPathCacheListener) } @@ -188,10 +146,6 @@ class KafkaStateActor(curator: CuratorFramework, Try(adminPathCache.getListenable.removeListener(adminPathCacheListener)) log.info("Removing topics tree cache listener...") Try(topicsTreeCache.getListenable.removeListener(topicsTreeCacheListener)) - log.info("Removing logkafka config tree cache listener...") - Try(logkafkaConfigTreeCache.getListenable.removeListener(logkafkaConfigTreeCacheListener)) - log.info("Removing logkafka client tree cache listener...") - Try(logkafkaClientTreeCache.getListenable.removeListener(logkafkaClientTreeCacheListener)) log.info("Shutting down delete topics path cache...") Try(deleteTopicsPathCache.close()) @@ -203,10 +157,6 @@ class KafkaStateActor(curator: CuratorFramework, Try(topicsConfigPathCache.close()) log.info("Shutting down topics tree cache...") Try(topicsTreeCache.close()) - log.info("Shutting down logkafka config tree cache...") - Try(logkafkaConfigTreeCache.close()) - log.info("Shutting down logkafka client tree cache...") - Try(logkafkaClientTreeCache.close()) super.postStop() } @@ -228,18 +178,6 @@ class KafkaStateActor(curator: CuratorFramework, } yield TopicDescription(topic, description, Option(states),config, deleteSupported) } - def getLogkafkaConfig(hostname: String) : Option[LogkafkaConfig] = { - for { - config <- getLogkafkaConfigString(hostname) - } yield LogkafkaConfig(hostname, Some(config)) - } - - def getLogkafkaClient(hostname: String) : Option[LogkafkaClient] = { - for { - client <- getLogkafkaClientString(hostname) - } yield LogkafkaClient(hostname, Some(client)) - } - override def processActorResponse(response: ActorResponse): Unit = { response match { case any: Any => log.warning("ksa : processActorResponse : Received unknown message: {}", any.toString) @@ -252,16 +190,6 @@ class KafkaStateActor(curator: CuratorFramework, result.map(cd => (cd.getStat.getVersion,asString(cd.getData))) } - private[this] def getLogkafkaConfigString(hostname: String) : Option[String] = { - val hostnamePath = "%s/%s".format(ZkUtils.LogkafkaConfigPath,hostname) - Option(logkafkaConfigTreeCache.getCurrentData(hostnamePath)).map( childData => asString(childData.getData)) - } - - private[this] def getLogkafkaClientString(hostname: String) : Option[String] = { - val hostnamePath = "%s/%s".format(ZkUtils.LogkafkaClientPath,hostname) - Option(logkafkaClientTreeCache.getCurrentData(hostnamePath)).map( childData => asString(childData.getData)) - } - override def processQueryRequest(request: QueryRequest): Unit = { request match { case KSGetTopics => @@ -330,54 +258,6 @@ class KafkaStateActor(curator: CuratorFramework, case KSGetReassignPartition => sender ! reassignPartitions - case KSGetLogkafkaHostnames => - val deleteSet: Set[String] = Set.empty - withLogkafkaConfigTreeCache { cache => - cache.getCurrentChildren(ZkUtils.LogkafkaConfigPath) - }.fold { - sender ! LogkafkaHostnameList(IndexedSeq.empty, deleteSet) - } { data: java.util.Map[String, ChildData] => - sender ! LogkafkaHostnameList(data.asScala.map(kv => kv._1).toIndexedSeq, deleteSet) - } - - case KSGetLogkafkaConfig(hostname) => - sender ! getLogkafkaConfig(hostname) - - case KSGetLogkafkaClient(hostname) => - sender ! getLogkafkaClient(hostname) - - case KSGetLogkafkaConfigs(hostnames) => - sender ! LogkafkaConfigs(hostnames.toIndexedSeq.map(getLogkafkaConfig).flatten, logkafkaConfigTreeCacheLastUpdateMillis) - - case KSGetLogkafkaClients(hostnames) => - sender ! LogkafkaClients(hostnames.toIndexedSeq.map(getLogkafkaClient).flatten, logkafkaClientTreeCacheLastUpdateMillis) - - case KSGetAllLogkafkaConfigs(lastUpdateMillisOption) => - val lastUpdateMillis = lastUpdateMillisOption.getOrElse(0L) - if (logkafkaConfigTreeCacheLastUpdateMillis > lastUpdateMillis) { - //we have option here since there may be no logkafka configs at all! - withLogkafkaConfigTreeCache { cache: TreeCache => - cache.getCurrentChildren(ZkUtils.LogkafkaConfigPath) - }.fold { - sender ! LogkafkaConfigs(IndexedSeq.empty, logkafkaConfigTreeCacheLastUpdateMillis) - } { data: java.util.Map[String, ChildData] => - sender ! LogkafkaConfigs(data.asScala.keys.toIndexedSeq.map(getLogkafkaConfig).flatten, logkafkaConfigTreeCacheLastUpdateMillis) - } - } // else no updates to send - - case KSGetAllLogkafkaClients(lastUpdateMillisOption) => - val lastUpdateMillis = lastUpdateMillisOption.getOrElse(0L) - if (logkafkaClientTreeCacheLastUpdateMillis > lastUpdateMillis) { - //we have option here since there may be no logkafka clients at all! - withLogkafkaClientTreeCache { cache: TreeCache => - cache.getCurrentChildren(ZkUtils.LogkafkaClientPath) - }.fold { - sender ! LogkafkaClients(IndexedSeq.empty, logkafkaClientTreeCacheLastUpdateMillis) - } { data: java.util.Map[String, ChildData] => - sender ! LogkafkaClients(data.asScala.keys.toIndexedSeq.map(getLogkafkaClient).flatten, logkafkaClientTreeCacheLastUpdateMillis) - } - } // else no updates to send - case any: Any => log.warning("ksa : processQueryRequest : Received unknown message: {}", any.toString) } } @@ -449,13 +329,5 @@ class KafkaStateActor(curator: CuratorFramework, Option(fn(topicsTreeCache)) } - private[this] def withLogkafkaConfigTreeCache[T](fn: TreeCache => T) : Option[T] = { - Option(fn(logkafkaConfigTreeCache)) - } - - private[this] def withLogkafkaClientTreeCache[T](fn: TreeCache => T) : Option[T] = { - Option(fn(logkafkaClientTreeCache)) - } - } diff --git a/app/kafka/manager/LogkafkaStateActor.scala b/app/kafka/manager/LogkafkaStateActor.scala new file mode 100644 index 000000000..5745962e7 --- /dev/null +++ b/app/kafka/manager/LogkafkaStateActor.scala @@ -0,0 +1,210 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ + +package kafka.manager + +import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode +import org.apache.curator.framework.recipes.cache._ +import org.apache.curator.framework.CuratorFramework +import org.joda.time.{DateTimeZone, DateTime} +import kafka.manager.utils.{ZkUtils} + +import scala.collection.mutable +import scala.util.{Success, Failure, Try} + +/** + * @author hiral + */ +import ActorModel._ +import kafka.manager.utils._ +import scala.collection.JavaConverters._ +class LogkafkaStateActor(curator: CuratorFramework, + deleteSupported: Boolean, + clusterConfig: ClusterConfig) extends BaseQueryCommandActor { + + private[this] val logkafkaConfigTreeCache = new TreeCache(curator,ZkUtils.LogkafkaConfigPath) + + private[this] val logkafkaClientTreeCache = new TreeCache(curator,ZkUtils.LogkafkaClientPath) + + @volatile + private[this] var logkafkaConfigTreeCacheLastUpdateMillis : Long = System.currentTimeMillis() + + @volatile + private[this] var logkafkaClientTreeCacheLastUpdateMillis : Long = System.currentTimeMillis() + + private[this] val logkafkaConfigTreeCacheListener = new TreeCacheListener { + override def childEvent(client: CuratorFramework, event: TreeCacheEvent): Unit = { + event.getType match { + case TreeCacheEvent.Type.INITIALIZED | TreeCacheEvent.Type.NODE_ADDED | + TreeCacheEvent.Type.NODE_REMOVED | TreeCacheEvent.Type.NODE_UPDATED => + logkafkaConfigTreeCacheLastUpdateMillis = System.currentTimeMillis() + case _ => + //do nothing + } + } + } + + private[this] val logkafkaClientTreeCacheListener = new TreeCacheListener { + override def childEvent(client: CuratorFramework, event: TreeCacheEvent): Unit = { + event.getType match { + case TreeCacheEvent.Type.INITIALIZED | TreeCacheEvent.Type.NODE_ADDED | + TreeCacheEvent.Type.NODE_REMOVED | TreeCacheEvent.Type.NODE_UPDATED => + logkafkaClientTreeCacheLastUpdateMillis = System.currentTimeMillis() + case _ => + //do nothing + } + } + } + + @scala.throws[Exception](classOf[Exception]) + override def preStart() = { + if (clusterConfig.logkafkaEnabled) { + log.info("Started actor %s".format(self.path)) + log.info("Starting logkafka config tree cache...") + logkafkaConfigTreeCache.start() + log.info("Starting logkafka client tree cache...") + logkafkaClientTreeCache.start() + + log.info("Adding logkafka config tree cache listener...") + logkafkaConfigTreeCache.getListenable.addListener(logkafkaConfigTreeCacheListener) + log.info("Adding logkafka client tree cache listener...") + logkafkaClientTreeCache.getListenable.addListener(logkafkaClientTreeCacheListener) + } + } + + @scala.throws[Exception](classOf[Exception]) + override def preRestart(reason: Throwable, message: Option[Any]) { + log.error(reason, "Restarting due to [{}] when processing [{}]", + reason.getMessage, message.getOrElse("")) + super.preRestart(reason, message) + } + + + @scala.throws[Exception](classOf[Exception]) + override def postStop(): Unit = { + log.info("Stopped actor %s".format(self.path)) + + log.info("Removing logkafka config tree cache listener...") + Try(logkafkaConfigTreeCache.getListenable.removeListener(logkafkaConfigTreeCacheListener)) + log.info("Removing logkafka client tree cache listener...") + Try(logkafkaClientTreeCache.getListenable.removeListener(logkafkaClientTreeCacheListener)) + + log.info("Shutting down logkafka config tree cache...") + Try(logkafkaConfigTreeCache.close()) + log.info("Shutting down logkafka client tree cache...") + Try(logkafkaClientTreeCache.close()) + + super.postStop() + } + + def getLogkafkaConfig(hostname: String) : Option[LogkafkaConfig] = { + for { + config <- getLogkafkaConfigString(hostname) + } yield LogkafkaConfig(hostname, Some(config)) + } + + def getLogkafkaClient(hostname: String) : Option[LogkafkaClient] = { + for { + client <- getLogkafkaClientString(hostname) + } yield LogkafkaClient(hostname, Some(client)) + } + + override def processActorResponse(response: ActorResponse): Unit = { + response match { + case any: Any => log.warning("ksa : processActorResponse : Received unknown message: {}", any.toString) + } + } + + private[this] def getLogkafkaConfigString(hostname: String) : Option[String] = { + val hostnamePath = "%s/%s".format(ZkUtils.LogkafkaConfigPath,hostname) + Option(logkafkaConfigTreeCache.getCurrentData(hostnamePath)).map( childData => asString(childData.getData)) + } + + private[this] def getLogkafkaClientString(hostname: String) : Option[String] = { + val hostnamePath = "%s/%s".format(ZkUtils.LogkafkaClientPath,hostname) + Option(logkafkaClientTreeCache.getCurrentData(hostnamePath)).map( childData => asString(childData.getData)) + } + + override def processQueryRequest(request: QueryRequest): Unit = { + request match { + case LKSGetLogkafkaHostnames => + val deleteSet: Set[String] = Set.empty + withLogkafkaConfigTreeCache { cache => + cache.getCurrentChildren(ZkUtils.LogkafkaConfigPath) + }.fold { + sender ! LogkafkaHostnameList(IndexedSeq.empty, deleteSet) + } { data: java.util.Map[String, ChildData] => + sender ! LogkafkaHostnameList(data.asScala.map(kv => kv._1).toIndexedSeq, deleteSet) + } + + case LKSGetLogkafkaConfig(hostname) => + sender ! getLogkafkaConfig(hostname) + + case LKSGetLogkafkaClient(hostname) => + sender ! getLogkafkaClient(hostname) + + case LKSGetLogkafkaConfigs(hostnames) => + sender ! LogkafkaConfigs(hostnames.toIndexedSeq.map(getLogkafkaConfig).flatten, logkafkaConfigTreeCacheLastUpdateMillis) + + case LKSGetLogkafkaClients(hostnames) => + sender ! LogkafkaClients(hostnames.toIndexedSeq.map(getLogkafkaClient).flatten, logkafkaClientTreeCacheLastUpdateMillis) + + case LKSGetAllLogkafkaConfigs(lastUpdateMillisOption) => + val lastUpdateMillis = lastUpdateMillisOption.getOrElse(0L) + if (logkafkaConfigTreeCacheLastUpdateMillis > lastUpdateMillis) { + //we have option here since there may be no logkafka configs at all! + withLogkafkaConfigTreeCache { cache: TreeCache => + cache.getCurrentChildren(ZkUtils.LogkafkaConfigPath) + }.fold { + sender ! LogkafkaConfigs(IndexedSeq.empty, logkafkaConfigTreeCacheLastUpdateMillis) + } { data: java.util.Map[String, ChildData] => + sender ! LogkafkaConfigs(data.asScala.keys.toIndexedSeq.map(getLogkafkaConfig).flatten, logkafkaConfigTreeCacheLastUpdateMillis) + } + } // else no updates to send + + case LKSGetAllLogkafkaClients(lastUpdateMillisOption) => + val lastUpdateMillis = lastUpdateMillisOption.getOrElse(0L) + if (logkafkaClientTreeCacheLastUpdateMillis > lastUpdateMillis) { + //we have option here since there may be no logkafka clients at all! + withLogkafkaClientTreeCache { cache: TreeCache => + cache.getCurrentChildren(ZkUtils.LogkafkaClientPath) + }.fold { + sender ! LogkafkaClients(IndexedSeq.empty, logkafkaClientTreeCacheLastUpdateMillis) + } { data: java.util.Map[String, ChildData] => + sender ! LogkafkaClients(data.asScala.keys.toIndexedSeq.map(getLogkafkaClient).flatten, logkafkaClientTreeCacheLastUpdateMillis) + } + } // else no updates to send + + case any: Any => log.warning("ksa : processQueryRequest : Received unknown message: {}", any.toString) + } + } + + override def processCommandRequest(request: CommandRequest): Unit = { + request match { + case any: Any => log.warning("ksa : processCommandRequest : Received unknown message: {}", any.toString) + } + } + + private[this] def getDateTime(millis: Long) : DateTime = new DateTime(millis,DateTimeZone.UTC) + + private[this] def safeExecute(fn: => Any) : Unit = { + Try(fn) match { + case Failure(t) => + log.error("Failed!",t) + case Success(_) => + //do nothing + } + } + + private[this] def withLogkafkaConfigTreeCache[T](fn: TreeCache => T) : Option[T] = { + Option(fn(logkafkaConfigTreeCache)) + } + + private[this] def withLogkafkaClientTreeCache[T](fn: TreeCache => T) : Option[T] = { + Option(fn(logkafkaClientTreeCache)) + } + +} + diff --git a/app/kafka/manager/LogkafkaViewCacheActor.scala b/app/kafka/manager/LogkafkaViewCacheActor.scala new file mode 100644 index 000000000..146ba8006 --- /dev/null +++ b/app/kafka/manager/LogkafkaViewCacheActor.scala @@ -0,0 +1,104 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ + +package kafka.manager + +import akka.actor.{ActorRef, Cancellable, ActorPath} +import scala.collection.mutable +import scala.concurrent.Future +import scala.concurrent.duration._ +import scala.util.Try + +/** + * @author hiral + */ +import ActorModel._ +case class LogkafkaViewCacheActorConfig(logkafkaStateActorPath: ActorPath, + clusterConfig: ClusterConfig, + longRunningPoolConfig: LongRunningPoolConfig, + updatePeriod: FiniteDuration = 10 seconds) +class LogkafkaViewCacheActor(config: LogkafkaViewCacheActorConfig) extends LongRunningPoolActor { + + private[this] val ZERO = BigDecimal(0) + + private[this] var cancellable : Option[Cancellable] = None + + private[this] var logkafkaIdentities : Map[String, LogkafkaIdentity] = Map.empty + + private[this] var logkafkaConfigsOption : Option[LogkafkaConfigs] = None + + private[this] var logkafkaClientsOption : Option[LogkafkaClients] = None + + override def preStart() = { + if (config.clusterConfig.logkafkaEnabled) { + log.info("Started actor %s".format(self.path)) + log.info("Scheduling updater for %s".format(config.updatePeriod)) + cancellable = Some( + context.system.scheduler.schedule(0 seconds, + config.updatePeriod, + self, + LKVForceUpdate)(context.system.dispatcher,self) + ) + } + } + + @scala.throws[Exception](classOf[Exception]) + override def postStop(): Unit = { + log.info("Stopped actor %s".format(self.path)) + log.info("Cancelling updater...") + Try(cancellable.map(_.cancel())) + super.postStop() + } + + override protected def longRunningPoolConfig: LongRunningPoolConfig = config.longRunningPoolConfig + + override protected def longRunningQueueFull(): Unit = { + log.error("Long running pool queue full, skipping!") + } + + override def processActorRequest(request: ActorRequest): Unit = { + request match { + case LKVForceUpdate => + log.info("Updating logkafka view...") + //ask for logkafka configs + val lastLogkafkaConfigsUpdateMillisOption: Option[Long] = logkafkaConfigsOption.map(_.lastUpdateMillis) + context.actorSelection(config.logkafkaStateActorPath).tell(LKSGetAllLogkafkaConfigs(lastLogkafkaConfigsUpdateMillisOption), self) + //ask for logkafka clients + val lastLogkafkaClientsUpdateMillisOption: Option[Long] = logkafkaClientsOption.map(_.lastUpdateMillis) + context.actorSelection(config.logkafkaStateActorPath).tell(LKSGetAllLogkafkaClients(lastLogkafkaClientsUpdateMillisOption), self) + + case LKVGetLogkafkaIdentities => + sender ! logkafkaIdentities + + case any: Any => log.warning("bvca : processActorRequest : Received unknown message: {}", any) + } + } + + override def processActorResponse(response: ActorResponse): Unit = { + response match { + case lcg: LogkafkaConfigs => + logkafkaConfigsOption = Some(lcg) + updateView() + + case lct: LogkafkaClients => + logkafkaClientsOption = Some(lct) + updateView() + + case any: Any => log.warning("bvca : processActorResponse : Received unknown message: {}", any) + } + } + + private[this] def updateView(): Unit = { + for { + logkafkaConfigs <- logkafkaConfigsOption + logkafkaClients <- logkafkaClientsOption + } { + val lcgMap = Map(logkafkaConfigs.configs map { a => a.hostname -> a }: _*) + val lctMap = Map(logkafkaClients.clients map { a => a.hostname -> a }: _*) + logkafkaIdentities = lcgMap.map (kv => + kv._1 -> LogkafkaIdentity.from(kv._1, Some(kv._2), lctMap.get(kv._1))) + } + } +} diff --git a/app/models/form/ClusterOperation.scala b/app/models/form/ClusterOperation.scala index 331f68476..2ac6e35d2 100644 --- a/app/models/form/ClusterOperation.scala +++ b/app/models/form/ClusterOperation.scala @@ -30,12 +30,12 @@ object Operation { } object ClusterOperation { - def apply(operation: String, name: String, version: String, zkHosts: String, zkMaxRetry: Int, jmxEnabled: Boolean): ClusterOperation = { - ClusterOperation(operation,ClusterConfig(name, version, zkHosts, zkMaxRetry, jmxEnabled)) + def apply(operation: String, name: String, version: String, zkHosts: String, zkMaxRetry: Int, jmxEnabled: Boolean, logkafkaEnabled: Boolean): ClusterOperation = { + ClusterOperation(operation,ClusterConfig(name, version, zkHosts, zkMaxRetry, jmxEnabled, logkafkaEnabled)) } - def customUnapply(co: ClusterOperation) : Option[(String, String, String, String, Int, Boolean)] = { - Option((co.op.toString,co.clusterConfig.name, co.clusterConfig.version.toString,co.clusterConfig.curatorConfig.zkConnect,co.clusterConfig.curatorConfig.zkMaxRetry,co.clusterConfig.jmxEnabled)) + def customUnapply(co: ClusterOperation) : Option[(String, String, String, String, Int, Boolean, Boolean)] = { + Option((co.op.toString,co.clusterConfig.name, co.clusterConfig.version.toString,co.clusterConfig.curatorConfig.zkConnect,co.clusterConfig.curatorConfig.zkMaxRetry,co.clusterConfig.jmxEnabled,co.clusterConfig.logkafkaEnabled)) } } diff --git a/app/views/cluster/addCluster.scala.html b/app/views/cluster/addCluster.scala.html index f271e2b01..5ea0222bd 100644 --- a/app/views/cluster/addCluster.scala.html +++ b/app/views/cluster/addCluster.scala.html @@ -18,6 +18,7 @@ @b3.text(form("zkHosts"), '_label -> "Cluster Zookeeper Hosts", 'placeholder -> "zk1:2181,zk2:2181,zk3:2181") @b3.select( form("kafkaVersion"), options = kafka.manager.KafkaVersion.formSelectList, '_label -> "Kafka Version" ) @b3.checkbox(form("jmxEnabled"), '_text -> "Enable JMX Polling") + @b3.checkbox(form("logkafkaEnabled"), '_text -> "Enable Logkafka") @b3.submit('class -> "submit-button btn btn-primary"){ Save } Cancel diff --git a/app/views/cluster/updateCluster.scala.html b/app/views/cluster/updateCluster.scala.html index 6229b9afc..f67ff483a 100644 --- a/app/views/cluster/updateCluster.scala.html +++ b/app/views/cluster/updateCluster.scala.html @@ -20,6 +20,7 @@ @b3.text(form("zkHosts"), '_label -> "Cluster Zookeeper Hosts", 'placeholder -> "zk1:2181,zk2:2181,zk3:2181", 'autoFocus -> true) @b3.select( form("kafkaVersion"), options = kafka.manager.KafkaVersion.formSelectList, '_label -> "Kafka Version" ) @b3.checkbox(form("jmxEnabled"), '_text -> "Enable JMX Polling") + @b3.checkbox(form("logkafkaEnabled"), '_text -> "Enable Logkafka") @b3.submit('class -> "submit-button btn btn-primary btn"){ Save } Cancel diff --git a/app/views/logkafka/logkafkaViewContent.scala.html b/app/views/logkafka/logkafkaViewContent.scala.html index ab2652bbb..241ba8a3d 100644 --- a/app/views/logkafka/logkafkaViewContent.scala.html +++ b/app/views/logkafka/logkafkaViewContent.scala.html @@ -20,23 +20,25 @@ - @defining(logkafkaIdentity.identityMap.get(log_path).get) { identityTuple => - @if(identityTuple._1.isDefined) { - - - - - - @identityTuple._1.map { config => - @for((k, v) <- config) { - - - - - } - } - -
ConfigValue
@k@v
+ @if(!logkafkaIdentity.identityMap.isEmpty && !logkafkaIdentity.identityMap.get(log_path).isEmpty) { + @defining(logkafkaIdentity.identityMap.get(log_path).get) { identityTuple => + @if(identityTuple._1.isDefined) { + + + + + + @identityTuple._1.map { config => + @for((k, v) <- config) { + + + + + } + } + +
ConfigValue
@k@v
+ } } } From 1ccd191e9e493446415d988d5fb64f6472b0df0a Mon Sep 17 00:00:00 2001 From: zheolong Date: Mon, 27 Jul 2015 17:37:34 +0800 Subject: [PATCH 03/17] Logkafka add LogkafkaCommandActor and two tests --- app/kafka/manager/ActorModel.scala | 10 +- app/kafka/manager/ClusterManagerActor.scala | 33 ++++-- app/kafka/manager/KafkaCommandActor.scala | 31 ------ app/kafka/manager/LogkafkaCommandActor.scala | 102 ++++++++++++++++++ app/kafka/manager/LogkafkaStateActor.scala | 6 +- .../manager/TestLogkafkaStateActor.scala | 96 +++++++++++++++++ .../manager/TestLogkafkaViewCacheActor.scala | 70 ++++++++++++ 7 files changed, 301 insertions(+), 47 deletions(-) create mode 100644 app/kafka/manager/LogkafkaCommandActor.scala create mode 100644 test/kafka/manager/TestLogkafkaStateActor.scala create mode 100644 test/kafka/manager/TestLogkafkaViewCacheActor.scala diff --git a/app/kafka/manager/ActorModel.scala b/app/kafka/manager/ActorModel.scala index c2078dfbb..854a10eac 100644 --- a/app/kafka/manager/ActorModel.scala +++ b/app/kafka/manager/ActorModel.scala @@ -102,19 +102,21 @@ object ActorModel { case class KCReassignPartition(currentTopicIdentity: Map[String, TopicIdentity], generatedTopicIdentity: Map[String, TopicIdentity]) extends CommandRequest - case class KCCreateLogkafka(hostname: String, + case class KCCommandResult(result: Try[Unit]) extends CommandResponse + + case class LKCCreateLogkafka(hostname: String, log_path: String, config: Properties, logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest - case class KCDeleteLogkafka(hostname: String, + case class LKCDeleteLogkafka(hostname: String, log_path: String, logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest - case class KCUpdateLogkafkaConfig(hostname: String, + case class LKCUpdateLogkafkaConfig(hostname: String, log_path: String, config: Properties, logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest - case class KCCommandResult(result: Try[Unit]) extends CommandResponse + case class LKCCommandResult(result: Try[Unit]) extends CommandResponse case object KMGetActiveClusters extends QueryRequest case object KMGetAllClusters extends QueryRequest diff --git a/app/kafka/manager/ClusterManagerActor.scala b/app/kafka/manager/ClusterManagerActor.scala index 00ee8ad45..97fa55c2d 100644 --- a/app/kafka/manager/ClusterManagerActor.scala +++ b/app/kafka/manager/ClusterManagerActor.scala @@ -119,6 +119,16 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) } private[this] val kafkaCommandActor : ActorPath = context.actorOf(kcProps,"kafka-command").path + private[this] val lkcProps = { + val lkcaConfig = LogkafkaCommandActorConfig( + sharedClusterCurator, + LongRunningPoolConfig(cmConfig.threadPoolSize, cmConfig.maxQueueSize), + cmConfig.askTimeoutMillis, + cmConfig.clusterConfig.version) + Props(classOf[LogkafkaCommandActor],lkcaConfig) + } + private[this] val logkafkaCommandActor : ActorPath = context.actorOf(lkcProps,"logkafka-command").path + private[this] implicit val timeout: Timeout = FiniteDuration(cmConfig.askTimeoutMillis,MILLISECONDS) private[this] val clusterManagerTopicsPathCache = new PathChildrenCache(curator,baseTopicsZkPath,true) @@ -401,9 +411,9 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) logkafkaConfigOption.fold { Future.successful(CMCommandResult(Failure(new IllegalArgumentException(s"Hostname doesn't exists : $hostname")))) } { td => - withKafkaCommandActor(KCDeleteLogkafka(hostname, log_path, logkafkaConfigOption)) { - kcResponse: KCCommandResult => - CMCommandResult(kcResponse.result) + withLogkafkaCommandActor(LKCDeleteLogkafka(hostname, log_path, logkafkaConfigOption)) { + lkcResponse: LKCCommandResult => + CMCommandResult(lkcResponse.result) } } } pipeTo sender() @@ -412,9 +422,9 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) implicit val ec = longRunningExecutionContext val eventualLogkafkaConfig = withLogkafkaStateActor(LKSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) eventualLogkafkaConfig.map { logkafkaConfigOption => - withKafkaCommandActor(KCCreateLogkafka(hostname, log_path, config, logkafkaConfigOption)) { - kcResponse: KCCommandResult => - CMCommandResult(kcResponse.result) + withLogkafkaCommandActor(LKCCreateLogkafka(hostname, log_path, config, logkafkaConfigOption)) { + lkcResponse: LKCCommandResult => + CMCommandResult(lkcResponse.result) } } pipeTo sender() @@ -422,9 +432,9 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) implicit val ec = longRunningExecutionContext val eventualLogkafkaConfig = withLogkafkaStateActor(LKSGetLogkafkaConfig(hostname))(identity[Option[LogkafkaConfig]]) eventualLogkafkaConfig.map { logkafkaConfigOption => - withKafkaCommandActor(KCUpdateLogkafkaConfig(hostname, log_path, config, logkafkaConfigOption)) { - kcResponse: KCCommandResult => - CMCommandResult(kcResponse.result) + withLogkafkaCommandActor(LKCUpdateLogkafkaConfig(hostname, log_path, config, logkafkaConfigOption)) { + lkcResponse: LKCCommandResult => + CMCommandResult(lkcResponse.result) } } pipeTo sender() @@ -457,6 +467,11 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) context.actorSelection(kafkaCommandActor).ask(msg).mapTo[Output].map(fn) } + private[this] def withLogkafkaCommandActor[Input,Output,FOutput] + (msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output], ec: ExecutionContext) : Future[FOutput] = { + context.actorSelection(logkafkaCommandActor).ask(msg).mapTo[Output].map(fn) + } + private[this] def modify[T](fn: => T): T = { try { mutex.acquire(cmConfig.mutexTimeoutMillis,TimeUnit.MILLISECONDS) diff --git a/app/kafka/manager/KafkaCommandActor.scala b/app/kafka/manager/KafkaCommandActor.scala index 41af93eb6..e94f3a90c 100644 --- a/app/kafka/manager/KafkaCommandActor.scala +++ b/app/kafka/manager/KafkaCommandActor.scala @@ -127,37 +127,6 @@ class KafkaCommandActor(kafkaCommandActorConfig: KafkaCommandActorConfig) extend ) } } - case KCDeleteLogkafka(hostname, log_path, logkafkaConfig) => - kafkaCommandActorConfig.version match { - case Kafka_0_8_1_1 => - val result : KCCommandResult = KCCommandResult(Failure(new UnsupportedOperationException( - s"Delete topic not supported for kafka version ${kafkaCommandActorConfig.version}"))) - sender ! result - case Kafka_0_8_2_0 | Kafka_0_8_2_1 => - longRunning { - Future { - KCCommandResult(Try { - adminUtils.deleteLogkafka(kafkaCommandActorConfig.curator, hostname, log_path, logkafkaConfig) - }) - } - } - } - case KCCreateLogkafka(hostname, log_path, config, logkafkaConfig) => - longRunning { - Future { - KCCommandResult(Try { - adminUtils.createLogkafka(kafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) - }) - } - } - case KCUpdateLogkafkaConfig(hostname, log_path, config, logkafkaConfig) => - longRunning { - Future { - KCCommandResult(Try { - adminUtils.changeLogkafkaConfig(kafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) - }) - } - } case any: Any => log.warning("kca : processCommandRequest : Received unknown message: {}", any) } } diff --git a/app/kafka/manager/LogkafkaCommandActor.scala b/app/kafka/manager/LogkafkaCommandActor.scala new file mode 100644 index 000000000..377bd2da5 --- /dev/null +++ b/app/kafka/manager/LogkafkaCommandActor.scala @@ -0,0 +1,102 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ + +package kafka.manager + +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, ThreadPoolExecutor} + +import akka.pattern._ +import akka.util.Timeout +import org.apache.curator.framework.CuratorFramework +import kafka.manager.utils.{AdminUtils, ZkUtils} + +import scala.concurrent.{Future, ExecutionContext} +import scala.concurrent.duration._ +import scala.util.{Failure, Try} + +/** + * @author hiral + */ + +import ActorModel._ + +case class LogkafkaCommandActorConfig(curator: CuratorFramework, + longRunningPoolConfig: LongRunningPoolConfig, + askTimeoutMillis: Long = 400, + version: KafkaVersion) +class LogkafkaCommandActor(logkafkaCommandActorConfig: LogkafkaCommandActorConfig) extends BaseCommandActor with LongRunningPoolActor { + + //private[this] val askTimeout: Timeout = logkafkaCommandActorConfig.askTimeoutMillis.milliseconds + + private[this] val adminUtils = new AdminUtils(logkafkaCommandActorConfig.version) + + @scala.throws[Exception](classOf[Exception]) + override def preStart() = { + log.info("Started actor %s".format(self.path)) + } + + @scala.throws[Exception](classOf[Exception]) + override def preRestart(reason: Throwable, message: Option[Any]) { + log.error(reason, "Restarting due to [{}] when processing [{}]", + reason.getMessage, message.getOrElse("")) + super.preRestart(reason, message) + } + + @scala.throws[Exception](classOf[Exception]) + override def postStop(): Unit = { + super.postStop() + } + + override protected def longRunningPoolConfig: LongRunningPoolConfig = logkafkaCommandActorConfig.longRunningPoolConfig + + override protected def longRunningQueueFull(): Unit = { + sender ! LKCCommandResult(Try(throw new UnsupportedOperationException("Long running executor blocking queue is full!"))) + } + + override def processActorResponse(response: ActorResponse): Unit = { + response match { + case any: Any => log.warning("lkca : processActorResponse : Received unknown message: {}", any) + } + } + + override def processCommandRequest(request: CommandRequest): Unit = { + implicit val ec = longRunningExecutionContext + request match { + case LKCDeleteLogkafka(hostname, log_path, logkafkaConfig) => + logkafkaCommandActorConfig.version match { + case Kafka_0_8_1_1 => + val result : LKCCommandResult = LKCCommandResult(Failure(new UnsupportedOperationException( + s"Delete logkafka not supported for kafka version ${logkafkaCommandActorConfig.version}"))) + sender ! result + case Kafka_0_8_2_0 | Kafka_0_8_2_1 => + longRunning { + Future { + LKCCommandResult(Try { + adminUtils.deleteLogkafka(logkafkaCommandActorConfig.curator, hostname, log_path, logkafkaConfig) + }) + } + } + } + case LKCCreateLogkafka(hostname, log_path, config, logkafkaConfig) => + longRunning { + Future { + LKCCommandResult(Try { + adminUtils.createLogkafka(logkafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) + }) + } + } + case LKCUpdateLogkafkaConfig(hostname, log_path, config, logkafkaConfig) => + longRunning { + Future { + LKCCommandResult(Try { + adminUtils.changeLogkafkaConfig(logkafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) + }) + } + } + case any: Any => log.warning("lkca : processCommandRequest : Received unknown message: {}", any) + } + } +} + diff --git a/app/kafka/manager/LogkafkaStateActor.scala b/app/kafka/manager/LogkafkaStateActor.scala index 5745962e7..b8fb03d19 100644 --- a/app/kafka/manager/LogkafkaStateActor.scala +++ b/app/kafka/manager/LogkafkaStateActor.scala @@ -113,7 +113,7 @@ class LogkafkaStateActor(curator: CuratorFramework, override def processActorResponse(response: ActorResponse): Unit = { response match { - case any: Any => log.warning("ksa : processActorResponse : Received unknown message: {}", any.toString) + case any: Any => log.warning("lksa : processActorResponse : Received unknown message: {}", any.toString) } } @@ -177,13 +177,13 @@ class LogkafkaStateActor(curator: CuratorFramework, } } // else no updates to send - case any: Any => log.warning("ksa : processQueryRequest : Received unknown message: {}", any.toString) + case any: Any => log.warning("lksa : processQueryRequest : Received unknown message: {}", any.toString) } } override def processCommandRequest(request: CommandRequest): Unit = { request match { - case any: Any => log.warning("ksa : processCommandRequest : Received unknown message: {}", any.toString) + case any: Any => log.warning("lksa : processCommandRequest : Received unknown message: {}", any.toString) } } diff --git a/test/kafka/manager/TestLogkafkaStateActor.scala b/test/kafka/manager/TestLogkafkaStateActor.scala new file mode 100644 index 000000000..12d5893e9 --- /dev/null +++ b/test/kafka/manager/TestLogkafkaStateActor.scala @@ -0,0 +1,96 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ +package kafka.manager + +import java.util.Properties + +import akka.actor.{ActorRef, ActorSystem, Kill, Props} +import akka.pattern._ +import akka.util.Timeout +import akka.util.Timeout._ +import com.typesafe.config.{Config, ConfigFactory} +import kafka.manager.utils.KafkaServerInTest +import ActorModel._ +import kafka.test.SeededBroker + +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.reflect.ClassTag +import scala.util.Try + +/** + * @author hiral + */ +class TestLogkafkaStateActor extends KafkaServerInTest { + + private[this] val akkaConfig: Properties = new Properties() + akkaConfig.setProperty("pinned-dispatcher.type","PinnedDispatcher") + akkaConfig.setProperty("pinned-dispatcher.executor","thread-pool-executor") + private[this] val config : Config = ConfigFactory.parseProperties(akkaConfig) + private[this] val system = ActorSystem("test-logkafka-state-actor",config) + private[this] val broker = new SeededBroker("ks-test",4) + override val kafkaServerZkPath = broker.getZookeeperConnectionString + private[this] var logkafkaStateActor : Option[ActorRef] = None + private[this] implicit val timeout: Timeout = 10.seconds + private[this] val defaultClusterConfig = ClusterConfig("test","0.8.2.0","localhost:2818",100,false,true) + + override protected def beforeAll(): Unit = { + super.beforeAll() + val props = Props(classOf[LogkafkaStateActor],sharedCurator, true, defaultClusterConfig) + + logkafkaStateActor = Some(system.actorOf(props.withDispatcher("pinned-dispatcher"),"lksa")) + } + + override protected def afterAll(): Unit = { + logkafkaStateActor.foreach( _ ! Kill ) + system.shutdown() + Try(broker.shutdown()) + super.afterAll() + } + + private[this] def withLogkafkaStateActor[Input,Output,FOutput](msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output]) : FOutput = { + require(logkafkaStateActor.isDefined, "logkafkaStateActor undefined!") + val future = ask(logkafkaStateActor.get, msg).mapTo[Output] + val result = Await.result(future,10.seconds) + fn(result) + } + + test("get logkafka hostname list") { + withLogkafkaStateActor(LKSGetLogkafkaHostnames) { result: TopicList => + result.list foreach println + } + } + + test("get logkafka config") { + withLogkafkaStateActor(LKSGetLogkafkaHostnames) { result: LogkafkaHostnameList => + val configs = result.list map { hostname => + withLogkafkaStateActor(LKSGetLogkafkaConfig(hostname)) { logkafkaConfig: LogkafkaConfig => logkafkaConfig } + } + configs foreach println + } + } + + test("get logkafka client") { + withLogkafkaStateActor(LKSGetLogkafkaHostnames) { result: LogkafkaHostnameList => + val clients = result.list map { hostname => + withLogkafkaStateActor(LKSGetLogkafkaClient(hostname)) { logkafkaClient: LogkafkaClient => logkafkaClient } + } + clients foreach println + } + } + + test("get logkafka configs") { + withLogkafkaStateActor(LKSGetAllLogkafkaConfigs()) { lc: LogkafkaConfigs => + lc.configs foreach println + } + } + + test("get logkafka clients") { + withLogkafkaStateActor(LKSGetAllLogkafkaClients()) { lc: LogkafkaClients => + lc.clients foreach println + } + } + +} diff --git a/test/kafka/manager/TestLogkafkaViewCacheActor.scala b/test/kafka/manager/TestLogkafkaViewCacheActor.scala new file mode 100644 index 000000000..1712696fd --- /dev/null +++ b/test/kafka/manager/TestLogkafkaViewCacheActor.scala @@ -0,0 +1,70 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ +package kafka.manager + +import java.util.Properties + +import akka.actor.{ActorRef, ActorSystem, Kill, Props} +import akka.pattern._ +import akka.util.Timeout +import com.typesafe.config.{Config, ConfigFactory} +import kafka.manager.utils.KafkaServerInTest +import ActorModel._ +import kafka.test.SeededBroker + +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.reflect.ClassTag +import scala.util.Try + +/** + * @author hiral + */ +class TestLogkafkaViewCacheActor extends KafkaServerInTest { + private[this] val akkaConfig: Properties = new Properties() + akkaConfig.setProperty("pinned-dispatcher.type","PinnedDispatcher") + akkaConfig.setProperty("pinned-dispatcher.executor","thread-pool-executor") + private[this] val config : Config = ConfigFactory.parseProperties(akkaConfig) + private[this] val system = ActorSystem("test-logkafka-view-cache-actor",config) + private[this] val broker = new SeededBroker("lkvc-test",4) + override val kafkaServerZkPath = broker.getZookeeperConnectionString + private[this] var logkafkaStateActor : Option[ActorRef] = None + private[this] implicit val timeout: Timeout = 10.seconds + + private[this] var logkafkaViewCacheActor : Option[ActorRef] = None + private[this] val defaultClusterConfig = ClusterConfig("test","0.8.2.0","localhost:2818",100,false,true) + + override protected def beforeAll(): Unit = { + super.beforeAll() + val clusterConfig = ClusterConfig("dev","0.8.2.0",kafkaServerZkPath, jmxEnabled = false, logkafkaEnabled = true) + val props = Props(classOf[KafkaStateActor],sharedCurator, true, defaultClusterConfig) + + logkafkaStateActor = Some(system.actorOf(props.withDispatcher("pinned-dispatcher"),"lksa")) + + val lkvConfig = LogkafkaViewCacheActorConfig(logkafkaStateActor.get.path, clusterConfig, LongRunningPoolConfig(2,100), FiniteDuration(10, SECONDS)) + val lkvcProps = Props(classOf[LogkafkaViewCacheActor],lkvConfig) + + logkafkaViewCacheActor = Some(system.actorOf(lkvcProps,"logkafka-view")) + + logkafkaViewCacheActor.get ! BVForceUpdate + Thread.sleep(10000) + } + + override protected def afterAll(): Unit = { + logkafkaViewCacheActor.foreach( _ ! Kill ) + logkafkaStateActor.foreach( _ ! Kill ) + system.shutdown() + Try(broker.shutdown()) + super.afterAll() + } + + private[this] def withLogkafkaViewCacheActor[Input,Output,FOutput] + (msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output]) : FOutput = { + require(logkafkaViewCacheActor.isDefined, "logkafkaViewCacheActor undefined!") + val future = ask(logkafkaViewCacheActor.get, msg).mapTo[Output] + val result = Await.result(future,10.seconds) + fn(result) + } +} From e69b35e6c1037e3437365296bb2bdf0492753aa7 Mon Sep 17 00:00:00 2001 From: zheolong Date: Mon, 27 Jul 2015 20:19:16 +0800 Subject: [PATCH 04/17] Logkafka little fix --- test/kafka/manager/TestLogkafkaStateActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/kafka/manager/TestLogkafkaStateActor.scala b/test/kafka/manager/TestLogkafkaStateActor.scala index 12d5893e9..18e90d896 100644 --- a/test/kafka/manager/TestLogkafkaStateActor.scala +++ b/test/kafka/manager/TestLogkafkaStateActor.scala @@ -58,7 +58,7 @@ class TestLogkafkaStateActor extends KafkaServerInTest { } test("get logkafka hostname list") { - withLogkafkaStateActor(LKSGetLogkafkaHostnames) { result: TopicList => + withLogkafkaStateActor(LKSGetLogkafkaHostnames) { result: LogkafkaHostnameList => result.list foreach println } } From 29812f5b50d20149174e7758ada4b765f1607b23 Mon Sep 17 00:00:00 2001 From: zheolong Date: Tue, 28 Jul 2015 13:53:22 +0800 Subject: [PATCH 05/17] Logkafka Add LogkafkaAdminUtils and LogkafkaZkUtils --- app/kafka/manager/LogkafkaCommandActor.scala | 10 +- app/kafka/manager/LogkafkaStateActor.scala | 16 +-- app/kafka/manager/utils/AdminUtils.scala | 83 ------------ .../manager/utils/LogkafkaAdminUtils.scala | 126 ++++++++++++++++++ app/kafka/manager/utils/LogkafkaZkUtils.scala | 36 +++++ app/kafka/manager/utils/ZkUtils.scala | 8 -- 6 files changed, 175 insertions(+), 104 deletions(-) create mode 100644 app/kafka/manager/utils/LogkafkaAdminUtils.scala create mode 100644 app/kafka/manager/utils/LogkafkaZkUtils.scala diff --git a/app/kafka/manager/LogkafkaCommandActor.scala b/app/kafka/manager/LogkafkaCommandActor.scala index 377bd2da5..e76576f14 100644 --- a/app/kafka/manager/LogkafkaCommandActor.scala +++ b/app/kafka/manager/LogkafkaCommandActor.scala @@ -10,7 +10,7 @@ import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, ThreadPoolExecutor} import akka.pattern._ import akka.util.Timeout import org.apache.curator.framework.CuratorFramework -import kafka.manager.utils.{AdminUtils, ZkUtils} +import kafka.manager.utils.{LogkafkaAdminUtils, ZkUtils} import scala.concurrent.{Future, ExecutionContext} import scala.concurrent.duration._ @@ -30,7 +30,7 @@ class LogkafkaCommandActor(logkafkaCommandActorConfig: LogkafkaCommandActorConfi //private[this] val askTimeout: Timeout = logkafkaCommandActorConfig.askTimeoutMillis.milliseconds - private[this] val adminUtils = new AdminUtils(logkafkaCommandActorConfig.version) + private[this] val logkafkaAdminUtils = new LogkafkaAdminUtils(logkafkaCommandActorConfig.version) @scala.throws[Exception](classOf[Exception]) override def preStart() = { @@ -74,7 +74,7 @@ class LogkafkaCommandActor(logkafkaCommandActorConfig: LogkafkaCommandActorConfi longRunning { Future { LKCCommandResult(Try { - adminUtils.deleteLogkafka(logkafkaCommandActorConfig.curator, hostname, log_path, logkafkaConfig) + logkafkaAdminUtils.deleteLogkafka(logkafkaCommandActorConfig.curator, hostname, log_path, logkafkaConfig) }) } } @@ -83,7 +83,7 @@ class LogkafkaCommandActor(logkafkaCommandActorConfig: LogkafkaCommandActorConfi longRunning { Future { LKCCommandResult(Try { - adminUtils.createLogkafka(logkafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) + logkafkaAdminUtils.createLogkafka(logkafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) }) } } @@ -91,7 +91,7 @@ class LogkafkaCommandActor(logkafkaCommandActorConfig: LogkafkaCommandActorConfi longRunning { Future { LKCCommandResult(Try { - adminUtils.changeLogkafkaConfig(logkafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) + logkafkaAdminUtils.changeLogkafkaConfig(logkafkaCommandActorConfig.curator, hostname, log_path, config, logkafkaConfig) }) } } diff --git a/app/kafka/manager/LogkafkaStateActor.scala b/app/kafka/manager/LogkafkaStateActor.scala index b8fb03d19..133a5ecb1 100644 --- a/app/kafka/manager/LogkafkaStateActor.scala +++ b/app/kafka/manager/LogkafkaStateActor.scala @@ -9,7 +9,7 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode import org.apache.curator.framework.recipes.cache._ import org.apache.curator.framework.CuratorFramework import org.joda.time.{DateTimeZone, DateTime} -import kafka.manager.utils.{ZkUtils} +import kafka.manager.utils.{LogkafkaZkUtils} import scala.collection.mutable import scala.util.{Success, Failure, Try} @@ -24,9 +24,9 @@ class LogkafkaStateActor(curator: CuratorFramework, deleteSupported: Boolean, clusterConfig: ClusterConfig) extends BaseQueryCommandActor { - private[this] val logkafkaConfigTreeCache = new TreeCache(curator,ZkUtils.LogkafkaConfigPath) + private[this] val logkafkaConfigTreeCache = new TreeCache(curator,LogkafkaZkUtils.LogkafkaConfigPath) - private[this] val logkafkaClientTreeCache = new TreeCache(curator,ZkUtils.LogkafkaClientPath) + private[this] val logkafkaClientTreeCache = new TreeCache(curator,LogkafkaZkUtils.LogkafkaClientPath) @volatile private[this] var logkafkaConfigTreeCacheLastUpdateMillis : Long = System.currentTimeMillis() @@ -118,12 +118,12 @@ class LogkafkaStateActor(curator: CuratorFramework, } private[this] def getLogkafkaConfigString(hostname: String) : Option[String] = { - val hostnamePath = "%s/%s".format(ZkUtils.LogkafkaConfigPath,hostname) + val hostnamePath = "%s/%s".format(LogkafkaZkUtils.LogkafkaConfigPath,hostname) Option(logkafkaConfigTreeCache.getCurrentData(hostnamePath)).map( childData => asString(childData.getData)) } private[this] def getLogkafkaClientString(hostname: String) : Option[String] = { - val hostnamePath = "%s/%s".format(ZkUtils.LogkafkaClientPath,hostname) + val hostnamePath = "%s/%s".format(LogkafkaZkUtils.LogkafkaClientPath,hostname) Option(logkafkaClientTreeCache.getCurrentData(hostnamePath)).map( childData => asString(childData.getData)) } @@ -132,7 +132,7 @@ class LogkafkaStateActor(curator: CuratorFramework, case LKSGetLogkafkaHostnames => val deleteSet: Set[String] = Set.empty withLogkafkaConfigTreeCache { cache => - cache.getCurrentChildren(ZkUtils.LogkafkaConfigPath) + cache.getCurrentChildren(LogkafkaZkUtils.LogkafkaConfigPath) }.fold { sender ! LogkafkaHostnameList(IndexedSeq.empty, deleteSet) } { data: java.util.Map[String, ChildData] => @@ -156,7 +156,7 @@ class LogkafkaStateActor(curator: CuratorFramework, if (logkafkaConfigTreeCacheLastUpdateMillis > lastUpdateMillis) { //we have option here since there may be no logkafka configs at all! withLogkafkaConfigTreeCache { cache: TreeCache => - cache.getCurrentChildren(ZkUtils.LogkafkaConfigPath) + cache.getCurrentChildren(LogkafkaZkUtils.LogkafkaConfigPath) }.fold { sender ! LogkafkaConfigs(IndexedSeq.empty, logkafkaConfigTreeCacheLastUpdateMillis) } { data: java.util.Map[String, ChildData] => @@ -169,7 +169,7 @@ class LogkafkaStateActor(curator: CuratorFramework, if (logkafkaClientTreeCacheLastUpdateMillis > lastUpdateMillis) { //we have option here since there may be no logkafka clients at all! withLogkafkaClientTreeCache { cache: TreeCache => - cache.getCurrentChildren(ZkUtils.LogkafkaClientPath) + cache.getCurrentChildren(LogkafkaZkUtils.LogkafkaClientPath) }.fold { sender ! LogkafkaClients(IndexedSeq.empty, logkafkaClientTreeCacheLastUpdateMillis) } { data: java.util.Map[String, ChildData] => diff --git a/app/kafka/manager/utils/AdminUtils.scala b/app/kafka/manager/utils/AdminUtils.scala index a334c2b2a..e4cb047d5 100644 --- a/app/kafka/manager/utils/AdminUtils.scala +++ b/app/kafka/manager/utils/AdminUtils.scala @@ -114,82 +114,6 @@ class AdminUtils(version: KafkaVersion) { createOrUpdateTopicPartitionAssignmentPathInZK(curator, topic, replicaAssignment, topicConfig) } - def deleteLogkafka(curator: CuratorFramework, - hostname: String, - log_path: String, - logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig]): Unit = { - logkafkaConfigOption.map { lcg => - lcg.config.map { c => - val configMap =kafka.manager.utils.Logkafka.parseJsonStr(hostname, c) - if (!configMap.isEmpty || !(configMap - log_path).isEmpty ) { - writeLogkafkaConfig(curator, hostname, configMap - log_path, -1) - } - } getOrElse { LogkafkaErrors.HostnameNotExists(hostname) } - } getOrElse { LogkafkaErrors.HostnameNotExists(hostname) } - } - - def createLogkafka(curator: CuratorFramework, - hostname: String, - log_path: String, - config: Properties = new Properties, - logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig] - ): Unit = { - createOrUpdateLogkafkaConfigPathInZK(curator, hostname, log_path, config, logkafkaConfigOption) - } - - def createOrUpdateLogkafkaConfigPathInZK(curator: CuratorFramework, - hostname: String, - log_path: String, - config: Properties = new Properties, - logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig], - update: Boolean = false, - readVersion: Int = -1 - ) { - // validate arguments - Logkafka.validateHostname(hostname) - LogkafkaNewConfigs.validate(version,config) - - val configMap: mutable.Map[String, String] = { - import scala.collection.JavaConverters._ - config.asScala - } - val newConfigMap = Map(log_path -> Map(configMap.toSeq:_*)) - - val logkafkaConfigMap = logkafkaConfigOption.map { lcg => - lcg.config.map { c => - kafka.manager.utils.Logkafka.parseJsonStr(hostname, c) - } getOrElse { Map.empty } - } getOrElse { Map.empty } - - if(!update ) { - // write out the config on create, not update, if there is any - writeLogkafkaConfig(curator, hostname, logkafkaConfigMap ++ newConfigMap, readVersion) - } else { - val merged = logkafkaConfigMap.toSeq ++ newConfigMap.toSeq - val grouped = merged.groupBy(_._1) - val cleaned = grouped.mapValues(_.map(_._2).fold(Map.empty)(_ ++ _)) - writeLogkafkaConfig(curator, hostname, cleaned, readVersion) - } - } - - /** - * Update the config for an existing (hostname,log_path) - * @param curator: The zk client handle used to write the new config to zookeeper - * @param hostname: The hostname for which configs are being changed - * @param log_path: The log_path for which configs are being changed - * @param config: The final set of configs that will be applied to the topic. If any new configs need to be added or - * existing configs need to be deleted, it should be done prior to invoking this API - * - */ - def changeLogkafkaConfig(curator: CuratorFramework, - hostname: String, - log_path: String, - config: Properties = new Properties, - logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig] - ): Unit = { - createOrUpdateLogkafkaConfigPathInZK(curator, hostname, log_path, config, logkafkaConfigOption, true) - } - def createOrUpdateTopicPartitionAssignmentPathInZK(curator: CuratorFramework, topic: String, partitionReplicaAssignment: Map[Int, Seq[Int]], @@ -230,13 +154,6 @@ class AdminUtils(version: KafkaVersion) { ZkUtils.updatePersistentPath(curator, ZkUtils.getTopicConfigPath(topic), toJson(map), readVersion) } - /** - * Write out the logkafka config to zk, if there is any - */ - private def writeLogkafkaConfig(curator: CuratorFramework, hostname: String, configMap: Map[String, Map[String, String]], readVersion: Int = -1) { - ZkUtils.updatePersistentPath(curator, ZkUtils.getLogkafkaConfigPath(hostname), toJson(configMap), readVersion) - } - private def writeTopicPartitionAssignment(curator: CuratorFramework, topic: String, replicaAssignment: Map[Int, Seq[Int]], diff --git a/app/kafka/manager/utils/LogkafkaAdminUtils.scala b/app/kafka/manager/utils/LogkafkaAdminUtils.scala new file mode 100644 index 000000000..8dfaa25fb --- /dev/null +++ b/app/kafka/manager/utils/LogkafkaAdminUtils.scala @@ -0,0 +1,126 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.manager.utils + +import java.util.Properties + +import kafka.manager.{Kafka_0_8_2_0, KafkaVersion} +import org.apache.curator.framework.CuratorFramework +import org.apache.zookeeper.CreateMode +import org.apache.zookeeper.KeeperException.NodeExistsException +import org.slf4j.LoggerFactory + +import scala.collection.mutable +import scala.util.Random + +class LogkafkaAdminUtils(version: KafkaVersion) { + + private[this] lazy val logger = LoggerFactory.getLogger(this.getClass) + + val rand = new Random + + def isDeleteSupported : Boolean = { + version match { + case Kafka_0_8_2_0 => true + case _ => false + } + } + + def deleteLogkafka(curator: CuratorFramework, + hostname: String, + log_path: String, + logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig]): Unit = { + logkafkaConfigOption.map { lcg => + lcg.config.map { c => + val configMap =kafka.manager.utils.Logkafka.parseJsonStr(hostname, c) + if (!configMap.isEmpty || !(configMap - log_path).isEmpty ) { + writeLogkafkaConfig(curator, hostname, configMap - log_path, -1) + } + } getOrElse { LogkafkaErrors.HostnameNotExists(hostname) } + } getOrElse { LogkafkaErrors.HostnameNotExists(hostname) } + } + + def createLogkafka(curator: CuratorFramework, + hostname: String, + log_path: String, + config: Properties = new Properties, + logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig] + ): Unit = { + createOrUpdateLogkafkaConfigPathInZK(curator, hostname, log_path, config, logkafkaConfigOption) + } + + def createOrUpdateLogkafkaConfigPathInZK(curator: CuratorFramework, + hostname: String, + log_path: String, + config: Properties = new Properties, + logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig], + update: Boolean = false, + readVersion: Int = -1 + ) { + // validate arguments + Logkafka.validateHostname(hostname) + LogkafkaNewConfigs.validate(version,config) + + val configMap: mutable.Map[String, String] = { + import scala.collection.JavaConverters._ + config.asScala + } + val newConfigMap = Map(log_path -> Map(configMap.toSeq:_*)) + + val logkafkaConfigMap = logkafkaConfigOption.map { lcg => + lcg.config.map { c => + kafka.manager.utils.Logkafka.parseJsonStr(hostname, c) + } getOrElse { Map.empty } + } getOrElse { Map.empty } + + if(!update ) { + // write out the config on create, not update, if there is any + writeLogkafkaConfig(curator, hostname, logkafkaConfigMap ++ newConfigMap, readVersion) + } else { + val merged = logkafkaConfigMap.toSeq ++ newConfigMap.toSeq + val grouped = merged.groupBy(_._1) + val cleaned = grouped.mapValues(_.map(_._2).fold(Map.empty)(_ ++ _)) + writeLogkafkaConfig(curator, hostname, cleaned, readVersion) + } + } + + /** + * Update the config for an existing (hostname,log_path) + * @param curator: The zk client handle used to write the new config to zookeeper + * @param hostname: The hostname for which configs are being changed + * @param log_path: The log_path for which configs are being changed + * @param config: The final set of configs that will be applied to the topic. If any new configs need to be added or + * existing configs need to be deleted, it should be done prior to invoking this API + * + */ + def changeLogkafkaConfig(curator: CuratorFramework, + hostname: String, + log_path: String, + config: Properties = new Properties, + logkafkaConfigOption: Option[kafka.manager.ActorModel.LogkafkaConfig] + ): Unit = { + createOrUpdateLogkafkaConfigPathInZK(curator, hostname, log_path, config, logkafkaConfigOption, true) + } + + /** + * Write out the logkafka config to zk, if there is any + */ + private def writeLogkafkaConfig(curator: CuratorFramework, hostname: String, configMap: Map[String, Map[String, String]], readVersion: Int = -1) { + ZkUtils.updatePersistentPath(curator, LogkafkaZkUtils.getLogkafkaConfigPath(hostname), toJson(configMap), readVersion) + } +} diff --git a/app/kafka/manager/utils/LogkafkaZkUtils.scala b/app/kafka/manager/utils/LogkafkaZkUtils.scala new file mode 100644 index 000000000..787e1a2d1 --- /dev/null +++ b/app/kafka/manager/utils/LogkafkaZkUtils.scala @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.manager.utils + +import java.nio.charset.StandardCharsets + +import org.apache.curator.framework.CuratorFramework +import org.apache.zookeeper.CreateMode +import org.apache.zookeeper.KeeperException.{NodeExistsException, NoNodeException} +import org.apache.zookeeper.data.Stat + +object LogkafkaZkUtils { + val LogkafkaConfigPath = "/logkafka/config" + val LogkafkaClientPath = "/logkafka/client" + + def getLogkafkaConfigPath(hostname: String): String = + LogkafkaConfigPath + "/" + hostname + + def getLogkafkaClientPath(hostname: String): String = + LogkafkaClientPath + "/" + hostname +} diff --git a/app/kafka/manager/utils/ZkUtils.scala b/app/kafka/manager/utils/ZkUtils.scala index 447819e0c..2c7173ebd 100644 --- a/app/kafka/manager/utils/ZkUtils.scala +++ b/app/kafka/manager/utils/ZkUtils.scala @@ -34,8 +34,6 @@ object ZkUtils { val BrokerTopicsPath = "/brokers/topics" val TopicConfigPath = "/config/topics" val TopicConfigChangesPath = "/config/changes" - val LogkafkaConfigPath = "/logkafka/config" - val LogkafkaClientPath = "/logkafka/client" val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" val ReassignPartitionsPath = "/admin/reassign_partitions" @@ -57,12 +55,6 @@ object ZkUtils { def getDeleteTopicPath(topic: String): String = DeleteTopicsPath + "/" + topic - def getLogkafkaConfigPath(hostname: String): String = - LogkafkaConfigPath + "/" + hostname - - def getLogkafkaClientPath(hostname: String): String = - LogkafkaClientPath + "/" + hostname - implicit def serializeString(str: String): Array[Byte] = { str.getBytes(StandardCharsets.UTF_8) } From 951f035b9a0dfc4d09b00f781d8beaa003e70e49 Mon Sep 17 00:00:00 2001 From: zheolong Date: Wed, 29 Jul 2015 18:10:15 +0800 Subject: [PATCH 06/17] Logkafka change type of logkafka related var to Option in ClusterManagerActor --- app/kafka/manager/ClusterManagerActor.scala | 77 +++++++++++++++------ 1 file changed, 55 insertions(+), 22 deletions(-) diff --git a/app/kafka/manager/ClusterManagerActor.scala b/app/kafka/manager/ClusterManagerActor.scala index 97fa55c2d..2abd08082 100644 --- a/app/kafka/manager/ClusterManagerActor.scala +++ b/app/kafka/manager/ClusterManagerActor.scala @@ -90,8 +90,16 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) private[this] val ksProps = Props(classOf[KafkaStateActor],sharedClusterCurator, adminUtils.isDeleteSupported, cmConfig.clusterConfig) private[this] val kafkaStateActor : ActorPath = context.actorOf(ksProps.withDispatcher(cmConfig.pinnedDispatcherName),"kafka-state").path - private[this] val lksProps = Props(classOf[LogkafkaStateActor],sharedClusterCurator, adminUtils.isDeleteSupported, cmConfig.clusterConfig) - private[this] val logkafkaStateActor : ActorPath = context.actorOf(lksProps.withDispatcher(cmConfig.pinnedDispatcherName),"logkafka-state").path + private[this] val lksProps: Option[Props] = + cmConfig.clusterConfig.logkafkaEnabled match { + case true => Some(Props(classOf[LogkafkaStateActor],sharedClusterCurator, adminUtils.isDeleteSupported, cmConfig.clusterConfig)) + case false => None + } + private[this] val logkafkaStateActor : Option[ActorPath] = + cmConfig.clusterConfig.logkafkaEnabled match { + case true => Some(context.actorOf(lksProps.get.withDispatcher(cmConfig.pinnedDispatcherName),"logkafka-state").path) + case false => None + } private[this] val bvConfig = BrokerViewCacheActorConfig( kafkaStateActor, @@ -101,13 +109,25 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) private[this] val bvcProps = Props(classOf[BrokerViewCacheActor],bvConfig) private[this] val brokerViewCacheActor : ActorPath = context.actorOf(bvcProps,"broker-view").path - private[this] val lkvConfig = LogkafkaViewCacheActorConfig( - logkafkaStateActor, - cmConfig.clusterConfig, - LongRunningPoolConfig(Runtime.getRuntime.availableProcessors(), 1000), - cmConfig.updatePeriod) - private[this] val lkvcProps = Props(classOf[LogkafkaViewCacheActor],lkvConfig) - private[this] val logkafkaViewCacheActor : ActorPath = context.actorOf(lkvcProps,"logkafka-view").path + private[this] val lkvConfig: Option[LogkafkaViewCacheActorConfig] = + cmConfig.clusterConfig.logkafkaEnabled match { + case true => Some(LogkafkaViewCacheActorConfig( + logkafkaStateActor.get, + cmConfig.clusterConfig, + LongRunningPoolConfig(Runtime.getRuntime.availableProcessors(), 1000), + cmConfig.updatePeriod)) + case false => None + } + private[this] val lkvcProps: Option[Props] = + cmConfig.clusterConfig.logkafkaEnabled match { + case true => Some(Props(classOf[LogkafkaViewCacheActor],lkvConfig.get)) + case false => None + } + private[this] val logkafkaViewCacheActor: Option[ActorPath] = + cmConfig.clusterConfig.logkafkaEnabled match { + case true => Some(context.actorOf(lkvcProps.get,"logkafka-view").path) + case false => None + } private[this] val kcProps = { val kcaConfig = KafkaCommandActorConfig( @@ -119,15 +139,22 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) } private[this] val kafkaCommandActor : ActorPath = context.actorOf(kcProps,"kafka-command").path - private[this] val lkcProps = { - val lkcaConfig = LogkafkaCommandActorConfig( - sharedClusterCurator, - LongRunningPoolConfig(cmConfig.threadPoolSize, cmConfig.maxQueueSize), - cmConfig.askTimeoutMillis, - cmConfig.clusterConfig.version) - Props(classOf[LogkafkaCommandActor],lkcaConfig) + private[this] val lkcProps: Option[Props] = { + cmConfig.clusterConfig.logkafkaEnabled match { + case true => { val lkcaConfig = LogkafkaCommandActorConfig( + sharedClusterCurator, + LongRunningPoolConfig(cmConfig.threadPoolSize, cmConfig.maxQueueSize), + cmConfig.askTimeoutMillis, + cmConfig.clusterConfig.version) + Some(Props(classOf[LogkafkaCommandActor],lkcaConfig)) } + case false => None + } } - private[this] val logkafkaCommandActor : ActorPath = context.actorOf(lkcProps,"logkafka-command").path + private[this] val logkafkaCommandActor : Option[ActorPath] = + cmConfig.clusterConfig.logkafkaEnabled match { + case true => Some(context.actorOf(lkcProps.get,"logkafka-command").path) + case false => None + } private[this] implicit val timeout: Timeout = FiniteDuration(cmConfig.askTimeoutMillis,MILLISECONDS) @@ -171,13 +198,19 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) context.actorSelection(kafkaStateActor).forward(ksRequest) case lksRequest: LKSRequest => - context.actorSelection(logkafkaStateActor).forward(lksRequest) + logkafkaStateActor.isDefined match { + case true => context.actorSelection(logkafkaStateActor.get).forward(lksRequest) + case false => log.warning("cma: processQueryResponse : Received LKSRequest", lksRequest) + } case bvRequest: BVRequest => context.actorSelection(brokerViewCacheActor).forward(bvRequest) case lkvRequest: LKVRequest => - context.actorSelection(logkafkaViewCacheActor).forward(lkvRequest) + logkafkaStateActor.isDefined match { + case true => context.actorSelection(logkafkaViewCacheActor.get).forward(lkvRequest) + case false => log.warning("cma: processQueryResponse : Received LKVRequest", lkvRequest) + } case CMGetView => implicit val ec = context.dispatcher @@ -449,7 +482,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) private[this] def withLogkafkaStateActor[Input,Output,FOutput] (msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output], ec: ExecutionContext) : Future[FOutput] = { - context.actorSelection(logkafkaStateActor).ask(msg).mapTo[Output].map(fn) + context.actorSelection(logkafkaStateActor.get).ask(msg).mapTo[Output].map(fn) } private[this] def withBrokerViewCacheActor[Input,Output,FOutput] @@ -459,7 +492,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) private[this] def withLogkafkaViewCacheActor[Input,Output,FOutput] (msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output], ec: ExecutionContext) : Future[FOutput] = { - context.actorSelection(logkafkaViewCacheActor).ask(msg).mapTo[Output].map(fn) + context.actorSelection(logkafkaViewCacheActor.get).ask(msg).mapTo[Output].map(fn) } private[this] def withKafkaCommandActor[Input,Output,FOutput] @@ -469,7 +502,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) private[this] def withLogkafkaCommandActor[Input,Output,FOutput] (msg: Input)(fn: Output => FOutput)(implicit tag: ClassTag[Output], ec: ExecutionContext) : Future[FOutput] = { - context.actorSelection(logkafkaCommandActor).ask(msg).mapTo[Output].map(fn) + context.actorSelection(logkafkaCommandActor.get).ask(msg).mapTo[Output].map(fn) } private[this] def modify[T](fn: => T): T = { From 3ad04a2ce6b17e13bdf20f5f0f4556555abd5f91 Mon Sep 17 00:00:00 2001 From: zheolong Date: Thu, 30 Jul 2015 17:25:09 +0800 Subject: [PATCH 07/17] Logkafka Add logkafka related tests in TestKafkaManager --- app/kafka/manager/KafkaManager.scala | 4 ++ test/kafka/manager/TestKafkaManager.scala | 71 +++++++++++++++++++++++ 2 files changed, 75 insertions(+) diff --git a/app/kafka/manager/KafkaManager.scala b/app/kafka/manager/KafkaManager.scala index 71f3dd511..409722f9f 100644 --- a/app/kafka/manager/KafkaManager.scala +++ b/app/kafka/manager/KafkaManager.scala @@ -570,6 +570,10 @@ class KafkaManager(akkaConfig: Config) )(identity[Option[ReassignPartitions]]) } + def getLogkafkaHostnameList(clusterName: String): Future[ApiError \/ LogkafkaHostnameList] = { + tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, LKSGetLogkafkaHostnames))(identity[LogkafkaHostnameList]) + } + def getLogkafkaListExtended(clusterName: String): Future[ApiError \/ LogkafkaListExtended] = { val futureLogkafkaIdentities = tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, LKVGetLogkafkaIdentities))(identity[Map[String, LogkafkaIdentity]]) val futureLogkafkaList = tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, LKSGetLogkafkaHostnames))(identity[LogkafkaHostnameList]) diff --git a/test/kafka/manager/TestKafkaManager.scala b/test/kafka/manager/TestKafkaManager.scala index 8cd04dbc2..bd99a685d 100644 --- a/test/kafka/manager/TestKafkaManager.scala +++ b/test/kafka/manager/TestKafkaManager.scala @@ -34,6 +34,9 @@ class TestKafkaManager extends CuratorAwareTest { private[this] val duration = FiniteDuration(10,SECONDS) private[this] val createTopicName = "km-unit-test" + private[this] val createLogkafkaHostname = "km-unit-test-logkafka-hostname" + private[this] val createLogkafkaLogPath = "/km-unit-test-logkafka-logpath" + private[this] val createLogkafkaTopic = "km-unit-test-logkafka-topic" override protected def beforeAll() : Unit = { super.beforeAll() @@ -275,6 +278,74 @@ class TestKafkaManager extends CuratorAwareTest { Thread.sleep(2000) } + test("update cluster logkafka enabled") { + val future = kafkaManager.updateCluster("dev","0.8.2.0",testServer.getConnectString, jmxEnabled = false, logkafkaEnabled = true) + val result = Await.result(future,duration) + assert(result.isRight === true) + + val future2 = kafkaManager.getClusterList + val result2 = Await.result(future2,duration) + assert(result2.isRight === true) + assert((result2.toOption.get.pending.nonEmpty === true) || + (result2.toOption.get.active.find(c => c.name == "dev").get.logkafkaEnabled === true)) + Thread.sleep(3000) + } + + test("create logkafka") { + val config = new Properties() + config.put(kafka.manager.utils.logkafka82.LogConfig.TopicProp,createLogkafkaTopic) + val future = kafkaManager.createLogkafka("dev",createLogkafkaHostname,createLogkafkaLogPath,config) + val result = Await.result(future,duration) + assert(result.isRight === true) + Thread.sleep(2000) + } + + test("get logkafka identity") { + val future = kafkaManager.getLogkafkaHostnameList("dev") + val result = Await.result(future,duration) + assert(result.isRight === true) + assert(result.toOption.get.list.nonEmpty === true) + result.toOption.get.list.foreach { hostname => + val future2 = kafkaManager.getLogkafkaIdentity("dev",hostname) + val result2 = Await.result(future2, duration) + assert(result2.isRight === true) + } + } + + test("update logkafka config") { + val liFuture= kafkaManager.getLogkafkaIdentity("dev",createLogkafkaHostname) + val liOrError = Await.result(liFuture, duration) + assert(liOrError.isRight, "Failed to get logkafka identity!") + val li = liOrError.toOption.get + val config = new Properties() + config.put(kafka.manager.utils.logkafka82.LogConfig.TopicProp,createLogkafkaTopic) + config.put(kafka.manager.utils.logkafka82.LogConfig.PartitionProp,"1") + val future = kafkaManager.updateLogkafkaConfig("dev",createLogkafkaHostname,createLogkafkaLogPath,config) + val result = Await.result(future,duration) + assert(result.isRight === true) + + //check new logkafka config + { + val liFuture= kafkaManager.getLogkafkaIdentity("dev",createLogkafkaHostname) + val liOrError = Await.result(liFuture, duration) + assert(liOrError.isRight, "Failed to get logkafka identity!") + val li = liOrError.toOption.get + assert(li.identityMap.get(createLogkafkaLogPath).get._1.get.apply(kafka.manager.utils.logkafka82.LogConfig.PartitionProp) === "1") + } + } + + test("delete logkafka") { + val future = kafkaManager.deleteLogkafka("dev",createLogkafkaHostname,createLogkafkaLogPath) + val result = Await.result(future,duration) + assert(result.isRight === true, result) + val liFuture= kafkaManager.getLogkafkaIdentity("dev",createLogkafkaHostname) + val liOrError = Await.result(liFuture, duration) + assert(liOrError.isRight, "Failed to get logkafka identity!") + val li = liOrError.toOption.get + assert(li.identityMap.get(createLogkafkaLogPath) === None) + Thread.sleep(2000) + } + test("delete cluster") { //first have to disable in order to delete { From 9e21e13ca3ac440eb64f82b3b456e60b9cf5e38e Mon Sep 17 00:00:00 2001 From: zheolong Date: Sat, 1 Aug 2015 10:56:18 +0800 Subject: [PATCH 08/17] Logkafka Add logkafka related tests in TestClusterManagerActor --- .../manager/TestClusterManagerActor.scala | 55 ++++++++++++++++++- 1 file changed, 54 insertions(+), 1 deletion(-) diff --git a/test/kafka/manager/TestClusterManagerActor.scala b/test/kafka/manager/TestClusterManagerActor.scala index 0a7b5a75c..4c021fa72 100644 --- a/test/kafka/manager/TestClusterManagerActor.scala +++ b/test/kafka/manager/TestClusterManagerActor.scala @@ -36,10 +36,13 @@ class TestClusterManagerActor extends CuratorAwareTest { private[this] var clusterManagerActor : Option[ActorRef] = None private[this] implicit val timeout: Timeout = 10.seconds private[this] val createTopicName = "cm-unit-test" + private[this] val createLogkafkaHostname = "km-unit-test-logkafka-hostname" + private[this] val createLogkafkaLogPath = "/km-unit-test-logkafka-logpath" + private[this] val createLogkafkaTopic = "km-unit-test-logkafka-topic" override protected def beforeAll(): Unit = { super.beforeAll() - val clusterConfig = ClusterConfig("dev","0.8.2.0",kafkaServerZkPath, jmxEnabled = false) + val clusterConfig = ClusterConfig("dev","0.8.2.0",kafkaServerZkPath, jmxEnabled = false, logkafkaEnabled = true) val curatorConfig = CuratorConfig(testServer.getConnectString) val config = ClusterManagerActorConfig("pinned-dispatcher","/kafka-manager/clusters/dev",curatorConfig,clusterConfig,FiniteDuration(1,SECONDS)) val props = Props(classOf[ClusterManagerActor],config) @@ -240,4 +243,54 @@ class TestClusterManagerActor extends CuratorAwareTest { assert(!result.list.contains(createTopicName),"Failed to delete topic") } } + + test("create logkafka") { + val config = new Properties() + config.put(kafka.manager.utils.logkafka82.LogConfig.TopicProp,createLogkafkaTopic) + withClusterManagerActor(CMCreateLogkafka(createLogkafkaHostname,createLogkafkaLogPath,config)) { cmResultFuture: Future[CMCommandResult] => + val cmResult = Await.result(cmResultFuture,10 seconds) + if(cmResult.result.isFailure) { + cmResult.result.get + } + Thread.sleep(500) + } + + withClusterManagerActor(LKSGetLogkafkaHostnames) { result: LogkafkaHostnameList => + assert(result.list.contains(createLogkafkaHostname),"Failed to create logkafka") + } + + withClusterManagerActor(CMGetLogkafkaIdentity(createLogkafkaHostname)) { result: Option[CMLogkafkaIdentity] => + assert(result.get.logkafkaIdentity.get.identityMap.contains(createLogkafkaLogPath),"Failed to create logkafka") + } + } + + test("get logkafka hostname list") { + withClusterManagerActor(LKSGetLogkafkaHostnames) { result: LogkafkaHostnameList => + assert(result.list.nonEmpty,"Failed to get logkafka hostname list") + result.list foreach println + } + } + + test("get logkafka config") { + withClusterManagerActor(LKSGetLogkafkaHostnames) { result: LogkafkaHostnameList => + val configs = result.list map { hostname => + withClusterManagerActor(LKSGetLogkafkaConfig(hostname)) { logkafkaConfigOption: Option[LogkafkaConfig] => logkafkaConfigOption.get } + } + configs foreach println + } + } + + test("delete logkafka") { + withClusterManagerActor(CMDeleteLogkafka(createLogkafkaHostname,createLogkafkaLogPath)) { cmResultFuture: Future[CMCommandResult] => + val cmResult = Await.result(cmResultFuture,10 seconds) + if(cmResult.result.isFailure) { + cmResult.result.get + } + Thread.sleep(500) + } + + withClusterManagerActor(CMGetLogkafkaIdentity(createLogkafkaHostname)) { result: Option[CMLogkafkaIdentity] => + assert(!result.get.logkafkaIdentity.get.identityMap.contains(createLogkafkaLogPath),"Failed to delete logkafka") + } + } } From 5469b711ad771a39b2ef356864889db6e9e6895b Mon Sep 17 00:00:00 2001 From: zheolong Date: Sat, 1 Aug 2015 14:11:10 +0800 Subject: [PATCH 09/17] Logkafka Add logkafka related tests in TestKafkaManagerActor --- test/kafka/manager/TestKafkaManagerActor.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/test/kafka/manager/TestKafkaManagerActor.scala b/test/kafka/manager/TestKafkaManagerActor.scala index 24a0a50c7..b1a392f29 100644 --- a/test/kafka/manager/TestKafkaManagerActor.scala +++ b/test/kafka/manager/TestKafkaManagerActor.scala @@ -148,4 +148,15 @@ class TestKafkaManagerActor extends CuratorAwareTest { result: Seq[BVView] => result.nonEmpty } } + + test("update cluster logkafka enabled") { + val cc2 = ClusterConfig("dev","0.8.2.0",kafkaServerZkPath, jmxEnabled = false, logkafkaEnabled = true) + withKafkaManagerActor(KMUpdateCluster(cc2)) { result: KMCommandResult => + result.result.get + Thread.sleep(3000) + } + withKafkaManagerActor(KMClusterQueryRequest("dev",LKSGetLogkafkaHostnames)) { result: LogkafkaHostnameList => + result.list.nonEmpty + } + } } From 18d370dd60896100927dc17ced766bd358b56645 Mon Sep 17 00:00:00 2001 From: patelh Date: Sun, 23 Aug 2015 23:45:08 -0700 Subject: [PATCH 10/17] First pass at adding feature gates --- app/controllers/Application.scala | 24 +- app/controllers/Cluster.scala | 198 ++++---- app/controllers/Logkafka.scala | 202 ++++---- .../PreferredReplicaElection.scala | 70 +-- app/controllers/ReassignPartitions.scala | 434 +++++++++++------- app/controllers/Topic.scala | 334 +++++++++----- app/controllers/package.scala | 68 +++ app/features/ApplicationFeature.scala | 65 +++ app/features/package.scala | 17 + app/kafka/manager/ActorModel.scala | 186 ++++---- app/kafka/manager/BrokerViewCacheActor.scala | 17 +- app/kafka/manager/ClusterManagerActor.scala | 95 ++-- app/kafka/manager/KafkaCommandActor.scala | 41 +- app/kafka/manager/KafkaManager.scala | 34 +- app/kafka/manager/KafkaManagerActor.scala | 2 + app/kafka/manager/KafkaStateActor.scala | 22 +- app/kafka/manager/LogkafkaCommandActor.scala | 28 +- app/kafka/manager/LogkafkaStateActor.scala | 9 +- .../manager/LogkafkaViewCacheActor.scala | 5 +- app/kafka/manager/features/KMFeature.scala | 71 +++ app/kafka/manager/utils/AdminUtils.scala | 7 - .../zero81/ReassignPartitionCommand.scala | 3 +- app/models/navigation/Menus.scala | 99 +++- app/models/navigation/QuickRoutes.scala | 4 +- app/views/broker/brokerList.scala.html | 10 +- app/views/broker/brokerListContent.scala.html | 2 +- app/views/broker/brokerView.scala.html | 7 +- app/views/broker/brokerViewContent.scala.html | 4 +- app/views/cluster/addCluster.scala.html | 2 +- app/views/cluster/clusterList.scala.html | 52 ++- app/views/cluster/clusterView.scala.html | 7 +- .../cluster/clusterViewContent.scala.html | 6 +- app/views/common/resultOfCommand.scala.html | 2 +- .../common/shortBrokerMetrics.scala.html | 8 +- app/views/index.scala.html | 3 +- app/views/logkafka/createLogkafka.scala.html | 9 +- app/views/logkafka/logkafkaList.scala.html | 9 +- app/views/logkafka/logkafkaView.scala.html | 10 +- .../logkafka/logkafkaViewContent.scala.html | 2 - app/views/logkafka/updateConfig.scala.html | 10 +- app/views/preferredReplicaElection.scala.html | 24 +- app/views/reassignPartitions.scala.html | 7 +- app/views/topic/addPartitions.scala.html | 9 +- .../addPartitionsToMultipleTopics.scala.html | 9 +- app/views/topic/confirmAssignment.scala.html | 10 +- .../confirmMultipleAssignments.scala.html | 10 +- app/views/topic/createTopic.scala.html | 9 +- ...cala.html => manualAssignments.scala.html} | 5 +- .../topic/runMultipleAssignments.scala.html | 10 +- app/views/topic/topicList.scala.html | 14 +- app/views/topic/topicView.scala.html | 6 +- app/views/topic/topicViewContent.scala.html | 18 +- app/views/topic/updateConfig.scala.html | 9 +- conf/application.conf | 1 + conf/routes | 16 +- .../manager/TestBrokerViewCacheActor.scala | 7 +- .../manager/TestClusterManagerActor.scala | 2 +- test/kafka/manager/TestKafkaManager.scala | 8 + test/kafka/manager/TestKafkaStateActor.scala | 6 +- .../manager/TestLogkafkaStateActor.scala | 4 +- .../manager/TestLogkafkaViewCacheActor.scala | 7 +- .../kafka/manager/utils/TestCreateTopic.scala | 18 +- .../utils/TestReassignPartitions.scala | 10 +- 63 files changed, 1510 insertions(+), 887 deletions(-) create mode 100644 app/controllers/package.scala create mode 100644 app/features/ApplicationFeature.scala create mode 100644 app/features/package.scala create mode 100644 app/kafka/manager/features/KMFeature.scala rename app/views/topic/{manualMultipleAssignments.scala.html => manualAssignments.scala.html} (96%) diff --git a/app/controllers/Application.scala b/app/controllers/Application.scala index 0fc914248..bbf1c1a74 100644 --- a/app/controllers/Application.scala +++ b/app/controllers/Application.scala @@ -5,6 +5,8 @@ package controllers +import features.ApplicationFeatures +import kafka.manager.features.ClusterFeatures import play.api.mvc._ /** @@ -16,29 +18,11 @@ object Application extends Controller { private[this] val kafkaManager = KafkaManagerContext.getKafkaManager + private[this] implicit val af: ApplicationFeatures = ApplicationFeatures.features + def index = Action.async { kafkaManager.getClusterList.map { errorOrClusterList => Ok(views.html.index(errorOrClusterList)) } } - - def cluster(c: String) = Action.async { - kafkaManager.getClusterView(c).map { errorOrClusterView => - Ok(views.html.cluster.clusterView(c,errorOrClusterView)) - } - } - - def brokers(c: String) = Action.async { - kafkaManager.getBrokerList(c).map { errorOrBrokerList => - Ok(views.html.broker.brokerList(c,errorOrBrokerList)) - } - } - - def broker(c: String, b: Int) = Action.async { - kafkaManager.getBrokerView(c,b).map { errorOrBrokerView => - Ok(views.html.broker.brokerView(c,b,errorOrBrokerView)) - } - } - - } diff --git a/app/controllers/Cluster.scala b/app/controllers/Cluster.scala index 62ce60d74..a92593200 100644 --- a/app/controllers/Cluster.scala +++ b/app/controllers/Cluster.scala @@ -5,6 +5,7 @@ package controllers +import features.{KMClusterManagerFeature, ApplicationFeatures} import kafka.manager.{KafkaVersion, ApiError, ClusterConfig} import models.FollowLink import models.form._ @@ -25,6 +26,7 @@ object Cluster extends Controller { import play.api.libs.concurrent.Execution.Implicits.defaultContext private[this] val kafkaManager = KafkaManagerContext.getKafkaManager + private[this] implicit val af: ApplicationFeatures = ApplicationFeatures.features val validateName : Constraint[String] = Constraint("validate name") { name => Try { @@ -84,100 +86,134 @@ object Cluster extends Controller { )(ClusterOperation.apply)(ClusterOperation.customUnapply) ) - def addCluster = Action.async { implicit request => - Future.successful(Ok(views.html.cluster.addCluster(clusterConfigForm))) + def cluster(c: String) = Action.async { + kafkaManager.getClusterView(c).map { errorOrClusterView => + Ok(views.html.cluster.clusterView(c,errorOrClusterView)) + } + } + + def brokers(c: String) = Action.async { + kafkaManager.getBrokerList(c).map { errorOrBrokerList => + Ok(views.html.broker.brokerList(c,errorOrBrokerList)) + } } - def updateCluster(c: String) = Action.async { implicit request => - kafkaManager.getClusterConfig(c).map { errorOrClusterConfig => - Ok(views.html.cluster.updateCluster(c,errorOrClusterConfig.map { cc => - updateForm.fill(ClusterOperation.apply(Update.toString,cc.name,cc.version.toString,cc.curatorConfig.zkConnect,cc.curatorConfig.zkMaxRetry,cc.jmxEnabled,cc.logkafkaEnabled)) - })) + def broker(c: String, b: Int) = Action.async { + kafkaManager.getBrokerView(c,b).map { errorOrBrokerView => + Ok(views.html.broker.brokerView(c,b,errorOrBrokerView)) + } + } + + def addCluster = Action.async { implicit request => + featureGate(KMClusterManagerFeature) { + Future.successful(Ok(views.html.cluster.addCluster(clusterConfigForm))) } } - def handleAddCluster = Action.async { implicit request => - clusterConfigForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.cluster.addCluster(formWithErrors))), - clusterConfig => { - kafkaManager.addCluster(clusterConfig.name, clusterConfig.version.toString, clusterConfig.curatorConfig.zkConnect, clusterConfig.jmxEnabled, clusterConfig.logkafkaEnabled).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.defaultMenu(), - models.navigation.BreadCrumbs.withView("Add Cluster"), - errorOrSuccess, - "Add Cluster", - FollowLink("Go to cluster view.",routes.Application.cluster(clusterConfig.name).toString()), - FollowLink("Try again.",routes.Cluster.addCluster().toString()) - )) - } + def updateCluster(c: String) = Action.async { implicit request => + featureGate(KMClusterManagerFeature) { + kafkaManager.getClusterConfig(c).map { errorOrClusterConfig => + Ok(views.html.cluster.updateCluster(c,errorOrClusterConfig.map { cc => + updateForm.fill(ClusterOperation.apply( + Update.toString, + cc.name, + cc.version.toString, + cc.curatorConfig.zkConnect, + cc.curatorConfig.zkMaxRetry, + cc.jmxEnabled, + cc.logkafkaEnabled)) + })) } - ) + } + } - def handleUpdateCluster(c: String) = Action.async { implicit request => - updateForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.cluster.updateCluster(c,\/-(formWithErrors)))), - clusterOperation => clusterOperation.op match { - case Enable => - kafkaManager.enableCluster(c).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.defaultMenu(), - models.navigation.BreadCrumbs.withViewAndCluster("Enable Cluster",c), - errorOrSuccess, - "Enable Cluster", - FollowLink("Go to cluster list.",routes.Application.index().toString()), - FollowLink("Back to cluster list.",routes.Application.index().toString()) - )) - } - case Disable => - kafkaManager.disableCluster(c).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.defaultMenu(), - models.navigation.BreadCrumbs.withViewAndCluster("Disable Cluster",c), - errorOrSuccess, - "Disable Cluster", - FollowLink("Back to cluster list.",routes.Application.index().toString()), - FollowLink("Back to cluster list.",routes.Application.index().toString()) - )) - } - case Delete => - kafkaManager.deleteCluster(c).map { errorOrSuccess => + def handleAddCluster = Action.async { implicit request => + featureGate(KMClusterManagerFeature) { + clusterConfigForm.bindFromRequest.fold( + formWithErrors => Future.successful(BadRequest(views.html.cluster.addCluster(formWithErrors))), + clusterConfig => { + kafkaManager.addCluster(clusterConfig.name, clusterConfig.version.toString, clusterConfig.curatorConfig.zkConnect, clusterConfig.jmxEnabled, clusterConfig.logkafkaEnabled).map { errorOrSuccess => Ok(views.html.common.resultOfCommand( views.html.navigation.defaultMenu(), - models.navigation.BreadCrumbs.withViewAndCluster("Delete Cluster",c), + models.navigation.BreadCrumbs.withView("Add Cluster"), errorOrSuccess, - "Delete Cluster", - FollowLink("Back to cluster list.",routes.Application.index().toString()), - FollowLink("Back to cluster list.",routes.Application.index().toString()) + "Add Cluster", + FollowLink("Go to cluster view.",routes.Cluster.cluster(clusterConfig.name).toString()), + FollowLink("Try again.",routes.Cluster.addCluster().toString()) )) } - case Update => - kafkaManager.updateCluster( - clusterOperation.clusterConfig.name, - clusterOperation.clusterConfig.version.toString, - clusterOperation.clusterConfig.curatorConfig.zkConnect, - clusterOperation.clusterConfig.jmxEnabled, - clusterOperation.clusterConfig.logkafkaEnabled - ).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( + } + ) + } + } + + def handleUpdateCluster(c: String) = Action.async { implicit request => + featureGate(KMClusterManagerFeature) { + updateForm.bindFromRequest.fold( + formWithErrors => Future.successful(BadRequest(views.html.cluster.updateCluster(c, \/-(formWithErrors)))), + clusterOperation => clusterOperation.op match { + case Enable => + kafkaManager.enableCluster(c).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.defaultMenu(), + models.navigation.BreadCrumbs.withViewAndCluster("Enable Cluster", c), + errorOrSuccess, + "Enable Cluster", + FollowLink("Go to cluster list.", routes.Application.index().toString()), + FollowLink("Back to cluster list.", routes.Application.index().toString()) + )) + } + case Disable => + kafkaManager.disableCluster(c).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.defaultMenu(), + models.navigation.BreadCrumbs.withViewAndCluster("Disable Cluster", c), + errorOrSuccess, + "Disable Cluster", + FollowLink("Back to cluster list.", routes.Application.index().toString()), + FollowLink("Back to cluster list.", routes.Application.index().toString()) + )) + } + case Delete => + kafkaManager.deleteCluster(c).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.defaultMenu(), + models.navigation.BreadCrumbs.withViewAndCluster("Delete Cluster", c), + errorOrSuccess, + "Delete Cluster", + FollowLink("Back to cluster list.", routes.Application.index().toString()), + FollowLink("Back to cluster list.", routes.Application.index().toString()) + )) + } + case Update => + kafkaManager.updateCluster( + clusterOperation.clusterConfig.name, + clusterOperation.clusterConfig.version.toString, + clusterOperation.clusterConfig.curatorConfig.zkConnect, + clusterOperation.clusterConfig.jmxEnabled, + clusterOperation.clusterConfig.logkafkaEnabled + ).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.defaultMenu(), + models.navigation.BreadCrumbs.withViewAndCluster("Update Cluster", c), + errorOrSuccess, + "Update Cluster", + FollowLink("Go to cluster view.", routes.Cluster.cluster(clusterOperation.clusterConfig.name).toString()), + FollowLink("Try again.", routes.Cluster.updateCluster(c).toString()) + )) + } + case Unknown(opString) => + Future.successful(Ok(views.html.common.resultOfCommand( views.html.navigation.defaultMenu(), - models.navigation.BreadCrumbs.withViewAndCluster("Update Cluster",c), - errorOrSuccess, - "Update Cluster", - FollowLink("Go to cluster view.",routes.Application.cluster(clusterOperation.clusterConfig.name).toString()), - FollowLink("Try again.",routes.Cluster.updateCluster(c).toString()) - )) - } - case Unknown(opString) => - Future.successful(Ok(views.html.common.resultOfCommand( - views.html.navigation.defaultMenu(), - models.navigation.BreadCrumbs.withViewAndCluster("Unknown Cluster Operation",c), - -\/(ApiError(s"Unknown operation $opString")), - "Unknown Cluster Operation", - FollowLink("Back to cluster list.",routes.Application.index().toString()), - FollowLink("Back to cluster list.",routes.Application.index().toString()) - ))) - } - ) + models.navigation.BreadCrumbs.withViewAndCluster("Unknown Cluster Operation", c), + -\/(ApiError(s"Unknown operation $opString")), + "Unknown Cluster Operation", + FollowLink("Back to cluster list.", routes.Application.index().toString()), + FollowLink("Back to cluster list.", routes.Application.index().toString()) + ))) + } + ) + } } } diff --git a/app/controllers/Logkafka.scala b/app/controllers/Logkafka.scala index e98e04f67..34c47f075 100644 --- a/app/controllers/Logkafka.scala +++ b/app/controllers/Logkafka.scala @@ -7,9 +7,11 @@ package controllers import java.util.Properties +import _root_.features.ApplicationFeatures import kafka.manager.ActorModel.LogkafkaIdentity +import kafka.manager.features.KMLogKafkaFeature import kafka.manager.utils.LogkafkaNewConfigs -import kafka.manager.{Kafka_0_8_2_1, ApiError, Kafka_0_8_2_0, Kafka_0_8_1_1} +import kafka.manager._ import models.FollowLink import models.form._ import models.navigation.Menus @@ -29,7 +31,8 @@ import scalaz.{\/-, -\/} object Logkafka extends Controller{ import play.api.libs.concurrent.Execution.Implicits.defaultContext - private[this] val kafkaManager = KafkaManagerContext.getKafkaManager + implicit private[this] val kafkaManager = KafkaManagerContext.getKafkaManager + private[this] implicit val af: ApplicationFeatures = ApplicationFeatures.features val validateHostname : Constraint[String] = Constraint("validate name") { name => Try { @@ -90,131 +93,144 @@ object Logkafka extends Controller{ ) private def createLogkafkaForm(clusterName: String) = { - kafkaManager.getClusterConfig(clusterName).map { errorOrConfig => - errorOrConfig.map { clusterConfig => - clusterConfig.version match { - case Kafka_0_8_1_1 => defaultCreateForm.fill(kafka_0_8_1_1_Default) - case Kafka_0_8_2_0 => defaultCreateForm.fill(kafka_0_8_2_0_Default) - case Kafka_0_8_2_1 => defaultCreateForm.fill(kafka_0_8_2_1_Default) + kafkaManager.getClusterContext(clusterName).map { errorOrConfig => + errorOrConfig.map { clusterContext => + clusterContext.config.version match { + case Kafka_0_8_1_1 => (defaultCreateForm.fill(kafka_0_8_1_1_Default), clusterContext) + case Kafka_0_8_2_0 => (defaultCreateForm.fill(kafka_0_8_2_0_Default), clusterContext) + case Kafka_0_8_2_1 => (defaultCreateForm.fill(kafka_0_8_2_1_Default), clusterContext) } } } } def logkafkas(c: String) = Action.async { - kafkaManager.getLogkafkaListExtended(c).map { errorOrLogkafkaList => - Ok(views.html.logkafka.logkafkaList(c,errorOrLogkafkaList)) + clusterFeatureGate(c, KMLogKafkaFeature) { clusterContext => + kafkaManager.getLogkafkaListExtended(c).map { errorOrLogkafkaList => + Ok(views.html.logkafka.logkafkaList(c, errorOrLogkafkaList.map( lkle => (lkle, clusterContext)))) + } } } def logkafka(c: String, h: String, l:String) = Action.async { - kafkaManager.getLogkafkaIdentity(c,h).map { errorOrLogkafkaIdentity => - Ok(views.html.logkafka.logkafkaView(c,h,l,errorOrLogkafkaIdentity)) + clusterFeatureGate(c, KMLogKafkaFeature) { clusterContext => + kafkaManager.getLogkafkaIdentity(c, h).map { errorOrLogkafkaIdentity => + Ok(views.html.logkafka.logkafkaView(c, h, l, errorOrLogkafkaIdentity.map( lki => (lki, clusterContext)))) + } } } def createLogkafka(clusterName: String) = Action.async { implicit request => - createLogkafkaForm(clusterName).map { errorOrForm => - Ok(views.html.logkafka.createLogkafka(clusterName, errorOrForm)) + clusterFeatureGate(clusterName, KMLogKafkaFeature) { clusterContext => + createLogkafkaForm(clusterName).map { errorOrForm => + Ok(views.html.logkafka.createLogkafka(clusterName, errorOrForm)) + } } } def handleCreateLogkafka(clusterName: String) = Action.async { implicit request => - defaultCreateForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.logkafka.createLogkafka(clusterName,\/-(formWithErrors)))), - cl => { - val props = new Properties() - cl.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name,c.value.get)) - kafkaManager.createLogkafka(clusterName,cl.hostname,cl.log_path,props).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(clusterName,"Logkafka","Create",Menus.clusterMenus(clusterName)), - models.navigation.BreadCrumbs.withNamedViewAndCluster("Logkafkas",clusterName,"Create Logkafka"), - errorOrSuccess, - "Create Logkafka", - FollowLink("Go to hostname view.",routes.Logkafka.logkafka(clusterName, cl.hostname, cl.log_path).toString()), - FollowLink("Try again.",routes.Logkafka.createLogkafka(clusterName).toString()) - )) + clusterFeatureGate(clusterName, KMLogKafkaFeature) { clusterContext => + implicit val clusterFeatures = clusterContext.clusterFeatures + defaultCreateForm.bindFromRequest.fold( + formWithErrors => { + Future.successful(BadRequest(views.html.logkafka.createLogkafka(clusterName, \/-((formWithErrors, clusterContext))))) + }, + cl => { + val props = new Properties() + cl.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name, c.value.get)) + kafkaManager.createLogkafka(clusterName, cl.hostname, cl.log_path, props).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Logkafka", "Create", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndCluster("Logkafkas", clusterName, "Create Logkafka"), + errorOrSuccess, + "Create Logkafka", + FollowLink("Go to hostname view.", routes.Logkafka.logkafka(clusterName, cl.hostname, cl.log_path).toString()), + FollowLink("Try again.", routes.Logkafka.createLogkafka(clusterName).toString()) + )) + } } - } - ) + ) + } } def handleDeleteLogkafka(clusterName: String, hostname: String, log_path: String) = Action.async { implicit request => - defaultDeleteForm.bindFromRequest.fold( - formWithErrors => Future.successful( - BadRequest(views.html.logkafka.logkafkaView( - clusterName, - hostname, - log_path, - -\/(ApiError(formWithErrors.error("logkafka").map(_.toString).getOrElse("Unknown error deleting logkafka!")))))), - deleteLogkafka => { - kafkaManager.deleteLogkafka(clusterName,deleteLogkafka.hostname,deleteLogkafka.log_path).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(clusterName,"Logkafka","Logkafka View",Menus.clusterMenus(clusterName)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndLogkafka("Logkafka View",clusterName,hostname,log_path,"Delete Logkafka"), - errorOrSuccess, - "Delete Logkafka", - FollowLink("Go to logkafka list.",routes.Logkafka.logkafkas(clusterName).toString()), - FollowLink("Try again.",routes.Logkafka.logkafka(clusterName, hostname, log_path).toString()) - )) + clusterFeatureGate(clusterName, KMLogKafkaFeature) { clusterContext => + implicit val clusterFeatures = clusterContext.clusterFeatures + defaultDeleteForm.bindFromRequest.fold( + formWithErrors => Future.successful( + BadRequest(views.html.logkafka.logkafkaView( + clusterName, + hostname, + log_path, + -\/(ApiError(formWithErrors.error("logkafka").map(_.toString).getOrElse("Unknown error deleting logkafka!")))))), + deleteLogkafka => { + kafkaManager.deleteLogkafka(clusterName, deleteLogkafka.hostname, deleteLogkafka.log_path).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Logkafka", "Logkafka View", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndLogkafka("Logkafka View", clusterName, hostname, log_path, "Delete Logkafka"), + errorOrSuccess, + "Delete Logkafka", + FollowLink("Go to logkafka list.", routes.Logkafka.logkafkas(clusterName).toString()), + FollowLink("Try again.", routes.Logkafka.logkafka(clusterName, hostname, log_path).toString()) + )) + } } - } - ) + ) + } } - private def updateConfigForm(clusterName: String, log_path: String, li: LogkafkaIdentity) = { - kafkaManager.getClusterConfig(clusterName).map { errorOrConfig => - errorOrConfig.map { clusterConfig => - val defaultConfigMap = clusterConfig.version match { - case Kafka_0_8_1_1 => LogkafkaNewConfigs.configNames(Kafka_0_8_1_1).map(n => (n,LKConfig(n,None))).toMap - case Kafka_0_8_2_0 => LogkafkaNewConfigs.configNames(Kafka_0_8_2_0).map(n => (n,LKConfig(n,None))).toMap - case Kafka_0_8_2_1 => LogkafkaNewConfigs.configNames(Kafka_0_8_2_1).map(n => (n,LKConfig(n,None))).toMap - } - val identityOption = li.identityMap.get(log_path) - if (identityOption.isDefined) { - val configOption = identityOption.get._1 - if (configOption.isDefined) { - val config: Map[String, String] = configOption.get - val combinedMap = defaultConfigMap ++ config.map(tpl => tpl._1 -> LKConfig(tpl._1,Option(tpl._2))) - defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,combinedMap.toList.map(_._2))) - } else { - defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,List(LKConfig("",None)))) - } - } else { - defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,List(LKConfig("",None)))) - } + private def updateConfigForm(clusterContext: ClusterContext, log_path: String, li: LogkafkaIdentity) = { + val defaultConfigMap = clusterContext.config.version match { + case Kafka_0_8_1_1 => LogkafkaNewConfigs.configNames(Kafka_0_8_1_1).map(n => (n,LKConfig(n,None))).toMap + case Kafka_0_8_2_0 => LogkafkaNewConfigs.configNames(Kafka_0_8_2_0).map(n => (n,LKConfig(n,None))).toMap + case Kafka_0_8_2_1 => LogkafkaNewConfigs.configNames(Kafka_0_8_2_1).map(n => (n,LKConfig(n,None))).toMap + } + val identityOption = li.identityMap.get(log_path) + if (identityOption.isDefined) { + val configOption = identityOption.get._1 + if (configOption.isDefined) { + val config: Map[String, String] = configOption.get + val combinedMap = defaultConfigMap ++ config.map(tpl => tpl._1 -> LKConfig(tpl._1,Option(tpl._2))) + defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,combinedMap.toList.map(_._2))) + } else { + defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,List(LKConfig("",None)))) } + } else { + defaultUpdateConfigForm.fill(UpdateLogkafkaConfig(li.hostname,log_path,List(LKConfig("",None)))) } } def updateConfig(clusterName: String, hostname: String, log_path: String) = Action.async { implicit request => - val errorOrFormFuture = kafkaManager.getLogkafkaIdentity(clusterName, hostname).flatMap { errorOrLogkafkaIdentity => - errorOrLogkafkaIdentity.fold( e => Future.successful(-\/(e)) ,{ logkafkaIdentity => - updateConfigForm(clusterName, log_path, logkafkaIdentity) - }) - } - errorOrFormFuture.map { errorOrForm => - Ok(views.html.logkafka.updateConfig(clusterName, hostname, log_path, errorOrForm)) + clusterFeatureGate(clusterName, KMLogKafkaFeature) { clusterContext => + val errorOrFormFuture = kafkaManager.getLogkafkaIdentity(clusterName, hostname).map( + _.map(lki => (updateConfigForm(clusterContext, log_path, lki), clusterContext)) + ) + errorOrFormFuture.map { errorOrForm => + Ok(views.html.logkafka.updateConfig(clusterName, hostname, log_path, errorOrForm)) + } } } def handleUpdateConfig(clusterName: String, hostname: String, log_path: String) = Action.async { implicit request => - defaultUpdateConfigForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.logkafka.updateConfig(clusterName, hostname, log_path, \/-(formWithErrors)))), - updateLogkafkaConfig => { - val props = new Properties() - updateLogkafkaConfig.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name,c.value.get)) - kafkaManager.updateLogkafkaConfig(clusterName,updateLogkafkaConfig.hostname,updateLogkafkaConfig.log_path, props).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(clusterName,"Logkafka","Logkafka View",Menus.clusterMenus(clusterName)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndLogkafka("Logkafka View",clusterName, hostname, log_path, "Update Config"), - errorOrSuccess, - "Update Config", - FollowLink("Go to logkafka view.",routes.Logkafka.logkafka(clusterName, updateLogkafkaConfig.hostname, updateLogkafkaConfig.log_path).toString()), - FollowLink("Try again.",routes.Logkafka.updateConfig(clusterName, hostname, log_path).toString()) - )) + clusterFeatureGate(clusterName, KMLogKafkaFeature) { clusterContext => + implicit val clusterFeatures = clusterContext.clusterFeatures + defaultUpdateConfigForm.bindFromRequest.fold( + formWithErrors => Future.successful(BadRequest(views.html.logkafka.updateConfig(clusterName, hostname, log_path, \/-((formWithErrors, clusterContext))))), + updateLogkafkaConfig => { + val props = new Properties() + updateLogkafkaConfig.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name, c.value.get)) + kafkaManager.updateLogkafkaConfig(clusterName, updateLogkafkaConfig.hostname, updateLogkafkaConfig.log_path, props).map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Logkafka", "Logkafka View", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndLogkafka("Logkafka View", clusterName, hostname, log_path, "Update Config"), + errorOrSuccess, + "Update Config", + FollowLink("Go to logkafka view.", routes.Logkafka.logkafka(clusterName, updateLogkafkaConfig.hostname, updateLogkafkaConfig.log_path).toString()), + FollowLink("Try again.", routes.Logkafka.updateConfig(clusterName, hostname, log_path).toString()) + )) + } } - } - ) + ) + } } } diff --git a/app/controllers/PreferredReplicaElection.scala b/app/controllers/PreferredReplicaElection.scala index ac8b8223b..937a0430e 100644 --- a/app/controllers/PreferredReplicaElection.scala +++ b/app/controllers/PreferredReplicaElection.scala @@ -5,7 +5,9 @@ package controllers +import features.{ApplicationFeatures, KMPreferredReplicaElectionFeature} import kafka.manager.ApiError +import kafka.manager.features.ClusterFeatures import models.navigation.Menus import models.{navigation, FollowLink} import models.form.{UnknownPREO, RunElection, PreferredReplicaElectionOperation} @@ -24,6 +26,8 @@ object PreferredReplicaElection extends Controller{ import play.api.libs.concurrent.Execution.Implicits.defaultContext private[this] val kafkaManager = KafkaManagerContext.getKafkaManager + private[this] implicit val af: ApplicationFeatures = ApplicationFeatures.features + private[this] implicit val cf: ClusterFeatures = ClusterFeatures.default val validateOperation : Constraint[String] = Constraint("validate operation value") { @@ -45,37 +49,39 @@ object PreferredReplicaElection extends Controller{ def handleRunElection(c: String) = Action.async { implicit request => - preferredReplicaElectionForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.preferredReplicaElection(c,-\/(ApiError("Unknown operation!")),formWithErrors))), - op => op match { - case RunElection => - val errorOrSuccessFuture = kafkaManager.getTopicList(c).flatMap { errorOrTopicList => - errorOrTopicList.fold( { e => - Future.successful(-\/(e)) - }, { topicList => - kafkaManager.runPreferredLeaderElection(c, topicList.list.toSet) - }) - } - errorOrSuccessFuture.map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(c,"Preferred Replica Election","",navigation.Menus.clusterMenus(c)), - models.navigation.BreadCrumbs.withViewAndCluster("Run Election",c), - errorOrSuccess, - "Run Election", - FollowLink("Go to preferred replica election.",routes.PreferredReplicaElection.preferredReplicaElection(c).toString()), - FollowLink("Try again.",routes.PreferredReplicaElection.preferredReplicaElection(c).toString()) - )) - } - case UnknownPREO(opString) => - Future.successful(Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(c,"Preferred Replica Election","",Menus.clusterMenus(c)), - models.navigation.BreadCrumbs.withNamedViewAndCluster("Preferred Replica Election",c,"Unknown Operation"), - -\/(ApiError(s"Unknown operation $opString")), - "Unknown Preferred Replica Election Operation", - FollowLink("Back to preferred replica election.",routes.PreferredReplicaElection.preferredReplicaElection(c).toString()), - FollowLink("Back to preferred replica election.",routes.PreferredReplicaElection.preferredReplicaElection(c).toString()) - ))) - } - ) + featureGate(KMPreferredReplicaElectionFeature) { + preferredReplicaElectionForm.bindFromRequest.fold( + formWithErrors => Future.successful(BadRequest(views.html.preferredReplicaElection(c, -\/(ApiError("Unknown operation!")), formWithErrors))), + op => op match { + case RunElection => + val errorOrSuccessFuture = kafkaManager.getTopicList(c).flatMap { errorOrTopicList => + errorOrTopicList.fold({ e => + Future.successful(-\/(e)) + }, { topicList => + kafkaManager.runPreferredLeaderElection(c, topicList.list.toSet) + }) + } + errorOrSuccessFuture.map { errorOrSuccess => + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(c, "Preferred Replica Election", "", navigation.Menus.clusterMenus(c)), + models.navigation.BreadCrumbs.withViewAndCluster("Run Election", c), + errorOrSuccess, + "Run Election", + FollowLink("Go to preferred replica election.", routes.PreferredReplicaElection.preferredReplicaElection(c).toString()), + FollowLink("Try again.", routes.PreferredReplicaElection.preferredReplicaElection(c).toString()) + )) + } + case UnknownPREO(opString) => + Future.successful(Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(c, "Preferred Replica Election", "", Menus.clusterMenus(c)), + models.navigation.BreadCrumbs.withNamedViewAndCluster("Preferred Replica Election", c, "Unknown Operation"), + -\/(ApiError(s"Unknown operation $opString")), + "Unknown Preferred Replica Election Operation", + FollowLink("Back to preferred replica election.", routes.PreferredReplicaElection.preferredReplicaElection(c).toString()), + FollowLink("Back to preferred replica election.", routes.PreferredReplicaElection.preferredReplicaElection(c).toString()) + ))) + } + ) + } } } diff --git a/app/controllers/ReassignPartitions.scala b/app/controllers/ReassignPartitions.scala index 65042c45c..e61288334 100644 --- a/app/controllers/ReassignPartitions.scala +++ b/app/controllers/ReassignPartitions.scala @@ -5,7 +5,9 @@ package controllers +import features.{KMReassignPartitionsFeature, ApplicationFeatures} import kafka.manager.ActorModel._ +import kafka.manager.features.ClusterFeatures import kafka.manager.{BrokerListExtended, ApiError, TopicListExtended} import models.navigation.Menus import models.{navigation, FollowLink} @@ -24,7 +26,8 @@ import scalaz.{\/, \/-, -\/} object ReassignPartitions extends Controller{ import play.api.libs.concurrent.Execution.Implicits.defaultContext - private[this] val kafkaManager = KafkaManagerContext.getKafkaManager + private[this] implicit val kafkaManager = KafkaManagerContext.getKafkaManager + private[this] implicit val af: ApplicationFeatures = ApplicationFeatures.features val validateOperation : Constraint[String] = Constraint("validate operation value") { case "confirm" => Valid @@ -102,212 +105,325 @@ object ReassignPartitions extends Controller{ } def runMultipleAssignments(c: String) = Action.async { - kafkaManager.getTopicList(c).map { errorOrSuccess => - Ok(views.html.topic.runMultipleAssignments( - c, errorOrSuccess.map(l => reassignMultipleTopicsForm.fill(RunMultipleAssignments(l.list.map(TopicSelect.from)))) - )) + featureGate(KMReassignPartitionsFeature) { + kafkaManager.getTopicList(c).flatMap { errorOrSuccess => + withClusterContext(c)( + err => Future.successful( + Ok(views.html.errors.onApiError(err, Option(FollowLink("Try Again", routes.ReassignPartitions.runMultipleAssignments(c).toString())))) + ), + cc => Future.successful( + Ok(views.html.topic.runMultipleAssignments( + c, errorOrSuccess.map(l => + (reassignMultipleTopicsForm.fill(RunMultipleAssignments(l.list.map(TopicSelect.from))), cc)) + )) + ) + ) + } } } def confirmAssignment(c: String, t: String) = Action.async { - kafkaManager.getBrokerList(c).map { errorOrSuccess => - Ok(views.html.topic.confirmAssignment( - c, t, errorOrSuccess.map(l => generateAssignmentsForm.fill(GenerateAssignment(l.list.map(BrokerSelect.from)))) - )) + featureGate(KMReassignPartitionsFeature) { + kafkaManager.getBrokerList(c).flatMap { errorOrSuccess => + withClusterContext(c)( + err => Future.successful( + Ok(views.html.errors.onApiError(err, Option(FollowLink("Try Again", routes.ReassignPartitions.confirmAssignment(c, t).toString())))) + ), + cc => Future.successful( + Ok(views.html.topic.confirmAssignment( + c, t, errorOrSuccess.map(l => + (generateAssignmentsForm.fill(GenerateAssignment(l.list.map(BrokerSelect.from))), cc)) + )) + ) + ) + } } } def confirmMultipleAssignments(c: String) = Action.async { - kafkaManager.getTopicList(c).flatMap { errOrTL => - errOrTL.fold( - { err: ApiError => - Future.successful( Ok(views.html.topic.confirmMultipleAssignments( c, -\/(err) ))) - }, { tL: TopicList => - kafkaManager.getBrokerList(c).map { errorOrSuccess => - Ok(views.html.topic.confirmMultipleAssignments( - c, errorOrSuccess.map(l => generateMultipleAssignmentsForm.fill(GenerateMultipleAssignments(tL.list.map(TopicSelect.from), l.list.map(BrokerSelect.from)))) - )) - } + featureGate(KMReassignPartitionsFeature) { + kafkaManager.getTopicList(c).flatMap { errOrTL => + withClusterContext(c)( + err => Future.successful( + Ok(views.html.errors.onApiError(err, Option(FollowLink("Try Again", routes.ReassignPartitions.confirmMultipleAssignments(c).toString())))) + ), + cc => + errOrTL.fold( + { err: ApiError => + Future.successful(Ok(views.html.topic.confirmMultipleAssignments(c, -\/(err)))) + }, { tL: TopicList => + kafkaManager.getBrokerList(c).map { errorOrSuccess => + Ok(views.html.topic.confirmMultipleAssignments( + c, errorOrSuccess.map(l => + (generateMultipleAssignmentsForm.fill(GenerateMultipleAssignments(tL.list.map(TopicSelect.from), l.list.map(BrokerSelect.from))), + cc) + ) + )) + } + } + ) + ) } - ) } } + + private[this] def flattenTopicIdentity(td: TopicIdentity) = { + (td.topic, td.partitionsIdentity.toList.map { case (partition, identity) => + (partition, identity.replicas.toList) + }) + } - def manualMultipleAssignments(c: String): Action[AnyContent] = Action.async { - val topicList = kafkaManager.getTopicListExtended(c) - val brokersViews = kafkaManager.getBrokersView(c) + def manualAssignments(c: String, t: String): Action[AnyContent] = Action.async { + featureGate(KMReassignPartitionsFeature) { + + withClusterFeatures(c)( err => { + Future.successful(Ok(views.html.errors.onApiError(err, + Option(FollowLink("Try Again", routes.ReassignPartitions.manualAssignments(c, t).toString()))))) + }, implicit clusterFeatures => { + val futureTopicIdentity = kafkaManager.getTopicIdentity(c, t) + val futureBrokersViews = kafkaManager.getBrokersView(c) + val futureBrokerList = kafkaManager.getBrokerList(c) - def flattenedTopicListExtended(topicListExtended: TopicListExtended) = { - topicListExtended.list.map { - case (topic, Some(topicIdentity)) => - (topic, topicIdentity.partitionsIdentity.toList.map { case (partition, identity) => - (partition, identity.replicas.toList) - }) - case (topic, None) => (topic, List[(Int, List[Int])]()) - } toList - } + /* + def flattenedTopicListExtended(topicListExtended: TopicListExtended) = { + topicListExtended.list + .filter(_._2.isDefined) + .sortBy(_._1) + .slice(offset, offset+maxResults) + .map(tpl => flattenTopicIdentity(tpl._2.get)).toList + }*/ - topicList.flatMap { errOrTL => - errOrTL.fold( - { err: ApiError => - Future.successful( Ok(views.html.topic.confirmMultipleAssignments( c, -\/(err) ))) - }, - { topics: TopicListExtended => + val futureResult: Future[Result] = for { + tiOrError <- futureTopicIdentity + bvOrError <- futureBrokersViews + blOrError <- futureBrokerList + } yield { + val errorOrResult: ApiError \/ Result = for { + ti <- tiOrError + bv <- bvOrError + bl <- blOrError + } yield { + Ok(views.html.topic.manualAssignments( + c, t, manualReassignmentForm.fill(List(flattenTopicIdentity(ti))), bl, bv, manualReassignmentForm.errors + )) + } + errorOrResult.fold(err => { + Ok(views.html.errors.onApiError(err, + Option(FollowLink("Try Again", routes.ReassignPartitions.manualAssignments(c, t).toString())))) + }, identity[Result]) + } + + futureResult.recover { + case err => + Ok(views.html.errors.onApiError(ApiError(s"Unknown error : ${err.getMessage}"), + Option(FollowLink("Try Again", routes.ReassignPartitions.manualAssignments(c, t).toString())))) + } + + /* + topicList.flatMap { errOrTL => + errOrTL.fold( + { err: ApiError => + Future.successful(Ok(views.html.topic.confirmMultipleAssignments(c, -\/(err)))) + }, { topics: TopicListExtended => kafkaManager.getBrokerList(c).flatMap { errOrCV => errOrCV.fold( - {err: ApiError => - Future.successful( Ok(views.html.topic.confirmMultipleAssignments( c, -\/(err) ))) - }, - { brokers: BrokerListExtended => { - brokersViews.flatMap { errorOrBVs => - errorOrBVs.fold ( - {err: ApiError => Future.successful( Ok(views.html.topic.confirmMultipleAssignments( c, -\/(err) )))}, - {bVs: Seq[BVView] => Future { - Ok(views.html.topic.manualMultipleAssignments( - c, manualReassignmentForm.fill(flattenedTopicListExtended(topics)), brokers , bVs, manualReassignmentForm.errors - )) - }} - ) + { err: ApiError => + Future.successful(Ok(views.html.topic.confirmMultipleAssignments(c, -\/(err)))) + }, { brokers: BrokerListExtended => { + brokersViews.flatMap { errorOrBVs => + errorOrBVs.fold( + { err: ApiError => Future.successful(Ok(views.html.topic.confirmMultipleAssignments(c, -\/(err))))}, { bVs: Seq[BVView] => Future { + Ok(views.html.topic.manualMultipleAssignments( + c, manualReassignmentForm.fill(flattenedTopicListExtended(topics)), brokers, bVs, manualReassignmentForm.errors + )) + } } + ) } } + } ) } } + ) + }*/ + } ) } } - def handleManualAssignment(c: String) = Action.async { implicit request => - def validateAssignment(assignment: List[(String, List[(Int, List[Int])])]) = { - (for { - (topic, assign) <- assignment - (partition, replicas) <- assign - } yield { - replicas.size == replicas.toSet.size - }) forall { b => b } - } + def handleManualAssignment(c: String, t: String) = Action.async { implicit request => + featureGate(KMReassignPartitionsFeature) { + def validateAssignment(assignment: List[(String, List[(Int, List[Int])])]) = { + (for { + (topic, assign) <- assignment + (partition, replicas) <- assign + } yield { + replicas.size == replicas.toSet.size + }) forall { b => b} + } - def responseScreen(title: String, errorOrResult: \/[IndexedSeq[ApiError], Unit]) = { - Ok(views.html.common.resultsOfCommand( - views.html.navigation.clusterMenu(c, title, "", Menus.clusterMenus(c)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Manual Reassignment View", c, "", title), - errorOrResult, - title, - FollowLink("Go to topic list.", routes.Topic.topics(c).toString()), - FollowLink("Try again.", routes.Topic.topics(c).toString()) - )) - } + def responseScreen(title: String, errorOrResult: \/[IndexedSeq[ApiError], Unit]): Future[Result] = { + withClusterFeatures(c)( err => { + Future.successful(Ok(views.html.errors.onApiError(err, + Option(FollowLink("Try Again", routes.ReassignPartitions.manualAssignments(c, t).toString()))))) + }, implicit clusterFeatures => { + Future.successful(Ok(views.html.common.resultsOfCommand( + views.html.navigation.clusterMenu(c, title, "", Menus.clusterMenus(c)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Manual Reassignment View", c, "", title), + errorOrResult, + title, + FollowLink("Go to topic view.", routes.Topic.topic(c, t).toString()), + FollowLink("Try again.", routes.Topic.topics(c).toString()) + ))) + }) + } - manualReassignmentForm.bindFromRequest.fold ( - errors => kafkaManager.getClusterList.map { errorOrClusterList => - responseScreen( - "Manual Reassign Partitions Failure", - -\/(IndexedSeq(ApiError("There is something really wrong with your submitted data!"))) - ) - }, - assignment => { - if (validateAssignment(assignment)) { - kafkaManager.manualPartitionAssignments(c, assignment).map { errorOrClusterList => - responseScreen("Manual Partitions Reassignment Successful", errorOrClusterList) - } - } else { - Future { + manualReassignmentForm.bindFromRequest.fold( + errors => kafkaManager.getClusterList.flatMap { errorOrClusterList => + responseScreen( + "Manual Reassign Partitions Failure", + -\/(IndexedSeq(ApiError("There is something really wrong with your submitted data!"))) + ) + }, + assignment => { + if (validateAssignment(assignment)) { + kafkaManager.manualPartitionAssignments(c, assignment).flatMap { errorOrClusterList => + responseScreen("Manual Partitions Reassignment Successful", errorOrClusterList) + } + } else { responseScreen( "Manual Partitions Reassignment Failure", -\/(IndexedSeq(ApiError("You cannot (or at least should not) assign two replicas of the same partition to the same broker!!"))) ) } } - } - ) + ) + } } def handleGenerateAssignment(c: String, t: String) = Action.async { implicit request => - generateAssignmentsForm.bindFromRequest.fold( - errors => Future.successful( Ok(views.html.topic.confirmAssignment( c, t, \/-(errors) ))), - assignment => { - kafkaManager.generatePartitionAssignments(c, Set(t), assignment.brokers.filter(_.selected).map(_.id)).map { errorOrSuccess => - Ok(views.html.common.resultsOfCommand( - views.html.navigation.clusterMenu(c, "Reassign Partitions", "", Menus.clusterMenus(c)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", c, t, "Generate Partition Assignments"), - errorOrSuccess, - s"Generate Partition Assignments - $t", - FollowLink("Go to topic view.", routes.Topic.topic(c, t).toString()), - FollowLink("Try again.", routes.Topic.topic(c, t).toString()) - )) + featureGate(KMReassignPartitionsFeature) { + withClusterContext(c)( + err => Future.successful( + Ok(views.html.errors.onApiError(err, Option(FollowLink("Try Again", routes.Topic.topic(c, t).toString())))) + ), + cc => + generateAssignmentsForm.bindFromRequest.fold( + errors => Future.successful(Ok(views.html.topic.confirmAssignment(c, t, \/-((errors, cc))))), + assignment => { + kafkaManager.generatePartitionAssignments(c, Set(t), assignment.brokers.filter(_.selected).map(_.id)).map { errorOrSuccess => + implicit val clusterFeatures = cc.clusterFeatures + Ok(views.html.common.resultsOfCommand( + views.html.navigation.clusterMenu(c, "Reassign Partitions", "", Menus.clusterMenus(c)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", c, t, "Generate Partition Assignments"), + errorOrSuccess, + s"Generate Partition Assignments - $t", + FollowLink("Go to topic view.", routes.Topic.topic(c, t).toString()), + FollowLink("Try again.", routes.Topic.topic(c, t).toString()) + )) - } - } - ) + } + } + ) + ) + } } def handleGenerateMultipleAssignments(c: String) = Action.async { implicit request => - generateMultipleAssignmentsForm.bindFromRequest.fold( - errors => Future.successful( Ok(views.html.topic.confirmMultipleAssignments( c, \/-(errors) ))), - assignment => { - kafkaManager.generatePartitionAssignments(c, assignment.topics.filter(_.selected).map(_.name).toSet, assignment.brokers.filter(_.selected).map(_.id)).map { errorOrSuccess => - Ok(views.html.common.resultsOfCommand( - views.html.navigation.clusterMenu(c, "Reassign Partitions", "", Menus.clusterMenus(c)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", c, "", "Generate Partition Assignments"), - errorOrSuccess, - s"Generate Partition Assignments", - FollowLink("Go to topic list.", routes.Topic.topics(c).toString()), - FollowLink("Try again.", routes.Topic.topics(c).toString()) - )) + featureGate(KMReassignPartitionsFeature) { + withClusterContext(c)( + err => Future.successful( + Ok(views.html.errors.onApiError(err, Option(FollowLink("Try Again", routes.Topic.topics(c).toString())))) + ), + cc => + generateMultipleAssignmentsForm.bindFromRequest.fold( + errors => Future.successful(Ok(views.html.topic.confirmMultipleAssignments(c, \/-((errors, cc))))), + assignment => { + kafkaManager.generatePartitionAssignments(c, assignment.topics.filter(_.selected).map(_.name).toSet, assignment.brokers.filter(_.selected).map(_.id)).map { errorOrSuccess => + implicit val clusterFeatures = cc.clusterFeatures + Ok(views.html.common.resultsOfCommand( + views.html.navigation.clusterMenu(c, "Reassign Partitions", "", Menus.clusterMenus(c)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", c, "", "Generate Partition Assignments"), + errorOrSuccess, + s"Generate Partition Assignments", + FollowLink("Go to topic list.", routes.Topic.topics(c).toString()), + FollowLink("Try again.", routes.Topic.topics(c).toString()) + )) - } - } - ) + } + } + ) + ) + } } def handleRunMultipleAssignments(c: String) = Action.async { implicit request => - reassignMultipleTopicsForm.bindFromRequest.fold( - errors => Future.successful( Ok(views.html.topic.runMultipleAssignments( c, \/-(errors) ))), - assignment => { - kafkaManager - .runReassignPartitions(c, assignment.topics.filter(_.selected).map(_.name).toSet) - .map { errorOrSuccess => - Ok( - views.html.common.resultsOfCommand( - views.html.navigation.clusterMenu(c, "Reassign Partitions", "", navigation.Menus.clusterMenus(c)), - models.navigation.BreadCrumbs.withNamedViewAndCluster("Topics", c, "Reassign Partitions"), - errorOrSuccess, - s"Run Reassign Partitions", - FollowLink("Go to reassign partitions.", routes.ReassignPartitions.reassignPartitions(c).toString()), - FollowLink("Try again.", routes.Topic.topics(c).toString()) - ) + featureGate(KMReassignPartitionsFeature) { + withClusterContext(c)( + err => Future.successful( + Ok(views.html.errors.onApiError(err, Option(FollowLink("Try Again", routes.Topic.topics(c).toString())))) + ), + cc => + reassignMultipleTopicsForm.bindFromRequest.fold( + errors => Future.successful(Ok(views.html.topic.runMultipleAssignments(c, \/-((errors, cc))))), + assignment => { + kafkaManager + .runReassignPartitions(c, assignment.topics.filter(_.selected).map(_.name).toSet) + .map { errorOrSuccess => + implicit val clusterFeatures = cc.clusterFeatures + Ok( + views.html.common.resultsOfCommand( + views.html.navigation.clusterMenu(c, "Reassign Partitions", "", navigation.Menus.clusterMenus(c)), + models.navigation.BreadCrumbs.withNamedViewAndCluster("Topics", c, "Reassign Partitions"), + errorOrSuccess, + s"Run Reassign Partitions", + FollowLink("Go to reassign partitions.", routes.ReassignPartitions.reassignPartitions(c).toString()), + FollowLink("Try again.", routes.Topic.topics(c).toString()) + ) + ) + } + } ) - } - } - ) + ) + } } def handleOperation(c: String, t: String) = Action.async { implicit request => - reassignPartitionsForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.topic.topicView(c,t,-\/(ApiError("Unknown operation!"))))), - op => op match { - case RunAssignment => - kafkaManager.runReassignPartitions(c,Set(t)).map { errorOrSuccess => - Ok(views.html.common.resultsOfCommand( - views.html.navigation.clusterMenu(c,"Reassign Partitions","",navigation.Menus.clusterMenus(c)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View",c,t,"Run Reassign Partitions"), - errorOrSuccess, - s"Run Reassign Partitions - $t", - FollowLink("Go to reassign partitions.",routes.ReassignPartitions.reassignPartitions(c).toString()), - FollowLink("Try again.",routes.Topic.topic(c,t).toString()) - )) - } - case UnknownRPO(opString) => - Future.successful(Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(c,"Reassign Partitions","",navigation.Menus.clusterMenus(c)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View",c,t,"Unknown Reassign Partitions Operation"), - -\/(ApiError(s"Unknown operation $opString")), - "Unknown Reassign Partitions Operation", - FollowLink("Back to reassign partitions.",routes.ReassignPartitions.reassignPartitions(c).toString()), - FollowLink("Back to reassign partitions.",routes.ReassignPartitions.reassignPartitions(c).toString()) - ))) - } - ) + featureGate(KMReassignPartitionsFeature) { + withClusterContext(c)( + err => Future.successful( + Ok(views.html.errors.onApiError(err, Option(FollowLink("Try Again", routes.Topic.topic(c, t).toString())))) + ), + cc => + reassignPartitionsForm.bindFromRequest.fold( + formWithErrors => Future.successful(BadRequest(views.html.topic.topicView(c, t, -\/(ApiError("Unknown operation!"))))), + op => op match { + case RunAssignment => + implicit val clusterFeatures = cc.clusterFeatures + kafkaManager.runReassignPartitions(c, Set(t)).map { errorOrSuccess => + Ok(views.html.common.resultsOfCommand( + views.html.navigation.clusterMenu(c, "Reassign Partitions", "", navigation.Menus.clusterMenus(c)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", c, t, "Run Reassign Partitions"), + errorOrSuccess, + s"Run Reassign Partitions - $t", + FollowLink("Go to reassign partitions.", routes.ReassignPartitions.reassignPartitions(c).toString()), + FollowLink("Try again.", routes.Topic.topic(c, t).toString()) + )) + } + case UnknownRPO(opString) => + implicit val clusterFeatures = cc.clusterFeatures + Future.successful(Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(c, "Reassign Partitions", "", navigation.Menus.clusterMenus(c)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", c, t, "Unknown Reassign Partitions Operation"), + -\/(ApiError(s"Unknown operation $opString")), + "Unknown Reassign Partitions Operation", + FollowLink("Back to reassign partitions.", routes.ReassignPartitions.reassignPartitions(c).toString()), + FollowLink("Back to reassign partitions.", routes.ReassignPartitions.reassignPartitions(c).toString()) + ))) + } + ) + ) + } } } diff --git a/app/controllers/Topic.scala b/app/controllers/Topic.scala index 47ddc5d8c..0f0c07a34 100644 --- a/app/controllers/Topic.scala +++ b/app/controllers/Topic.scala @@ -7,7 +7,9 @@ package controllers import java.util.Properties +import features.{KMTopicManagerFeature, ApplicationFeatures} import kafka.manager.ActorModel.TopicIdentity +import kafka.manager.features.ClusterFeatures import kafka.manager.utils.TopicConfigs import kafka.manager.{Kafka_0_8_2_1, ApiError, Kafka_0_8_2_0, Kafka_0_8_1_1, TopicListExtended} import models.FollowLink @@ -30,6 +32,7 @@ object Topic extends Controller{ import play.api.libs.concurrent.Execution.Implicits.defaultContext private[this] val kafkaManager = KafkaManagerContext.getKafkaManager + private[this] implicit val af: ApplicationFeatures = ApplicationFeatures.features val validateName : Constraint[String] = Constraint("validate name") { name => Try { @@ -118,12 +121,12 @@ object Topic extends Controller{ ) private def createTopicForm(clusterName: String) = { - kafkaManager.getClusterConfig(clusterName).map { errorOrConfig => - errorOrConfig.map { clusterConfig => - clusterConfig.version match { - case Kafka_0_8_1_1 => defaultCreateForm.fill(kafka_0_8_1_1_Default) - case Kafka_0_8_2_0 => defaultCreateForm.fill(kafka_0_8_2_0_Default) - case Kafka_0_8_2_1 => defaultCreateForm.fill(kafka_0_8_2_1_Default) + kafkaManager.getClusterContext(clusterName).map { errorOrConfig => + errorOrConfig.map { clusterContext => + clusterContext.config.version match { + case Kafka_0_8_1_1 => (defaultCreateForm.fill(kafka_0_8_1_1_Default), clusterContext) + case Kafka_0_8_2_0 => (defaultCreateForm.fill(kafka_0_8_2_0_Default), clusterContext) + case Kafka_0_8_2_1 => (defaultCreateForm.fill(kafka_0_8_2_1_Default), clusterContext) } } } @@ -142,125 +145,191 @@ object Topic extends Controller{ } def createTopic(clusterName: String) = Action.async { implicit request => - createTopicForm(clusterName).map { errorOrForm => - Ok(views.html.topic.createTopic(clusterName, errorOrForm)) + featureGate(KMTopicManagerFeature) { + createTopicForm(clusterName).map { errorOrForm => + Ok(views.html.topic.createTopic(clusterName, errorOrForm)) + } } } def handleCreateTopic(clusterName: String) = Action.async { implicit request => - defaultCreateForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.topic.createTopic(clusterName,\/-(formWithErrors)))), - ct => { - val props = new Properties() - ct.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name,c.value.get)) - kafkaManager.createTopic(clusterName,ct.topic,ct.partitions,ct.replication,props).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(clusterName,"Topic","Create",Menus.clusterMenus(clusterName)), - models.navigation.BreadCrumbs.withNamedViewAndCluster("Topics",clusterName,"Create Topic"), - errorOrSuccess, - "Create Topic", - FollowLink("Go to topic view.",routes.Topic.topic(clusterName, ct.topic).toString()), - FollowLink("Try again.",routes.Topic.createTopic(clusterName).toString()) - )) + featureGate(KMTopicManagerFeature) { + defaultCreateForm.bindFromRequest.fold( + formWithErrors => { + kafkaManager.getClusterContext(clusterName).map { clusterContext => + BadRequest(views.html.topic.createTopic(clusterName, clusterContext.map(c => (formWithErrors, c)))) + }.recover { + case t => + implicit val clusterFeatures = ClusterFeatures.default + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Topic", "Create", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndCluster("Topics", clusterName, "Create Topic"), + -\/(ApiError(s"Unknown error : ${t.getMessage}")), + "Create Topic", + FollowLink("Try again.", routes.Topic.createTopic(clusterName).toString()), + FollowLink("Try again.", routes.Topic.createTopic(clusterName).toString()) + )) + } + }, + ct => { + val props = new Properties() + ct.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name, c.value.get)) + kafkaManager.createTopic(clusterName, ct.topic, ct.partitions, ct.replication, props).map { errorOrSuccess => + implicit val clusterFeatures = errorOrSuccess.toOption.map(_.clusterFeatures).getOrElse(ClusterFeatures.default) + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Topic", "Create", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndCluster("Topics", clusterName, "Create Topic"), + errorOrSuccess, + "Create Topic", + FollowLink("Go to topic view.", routes.Topic.topic(clusterName, ct.topic).toString()), + FollowLink("Try again.", routes.Topic.createTopic(clusterName).toString()) + )) + } } - } - ) + ) + } } def handleDeleteTopic(clusterName: String, topic: String) = Action.async { implicit request => - defaultDeleteForm.bindFromRequest.fold( - formWithErrors => Future.successful( - BadRequest(views.html.topic.topicView( - clusterName, - topic, - -\/(ApiError(formWithErrors.error("topic").map(_.toString).getOrElse("Unknown error deleting topic!")))))), - deleteTopic => { - kafkaManager.deleteTopic(clusterName,deleteTopic.topic).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(clusterName,"Topic","Topic View",Menus.clusterMenus(clusterName)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View",clusterName,topic,"Delete Topic"), - errorOrSuccess, - "Delete Topic", - FollowLink("Go to topic list.",routes.Topic.topics(clusterName).toString()), - FollowLink("Try again.",routes.Topic.topic(clusterName, topic).toString()) - )) + featureGate(KMTopicManagerFeature) { + defaultDeleteForm.bindFromRequest.fold( + formWithErrors => Future.successful( + BadRequest(views.html.topic.topicView( + clusterName, + topic, + -\/(ApiError(formWithErrors.error("topic").map(_.toString).getOrElse("Unknown error deleting topic!"))))) + ), + deleteTopic => { + kafkaManager.deleteTopic(clusterName, deleteTopic.topic).map { errorOrSuccess => + implicit val clusterFeatures = errorOrSuccess.toOption.map(_.clusterFeatures).getOrElse(ClusterFeatures.default) + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Topic", "Topic View", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", clusterName, topic, "Delete Topic"), + errorOrSuccess, + "Delete Topic", + FollowLink("Go to topic list.", routes.Topic.topics(clusterName).toString()), + FollowLink("Try again.", routes.Topic.topic(clusterName, topic).toString()) + )) + } } - } - ) + ) + } } def addPartitions(clusterName: String, topic: String) = Action.async { implicit request => - val errorOrFormFuture = kafkaManager.getTopicIdentity(clusterName, topic).flatMap { errorOrTopicIdentity => - errorOrTopicIdentity.fold( e => Future.successful(-\/(e)),{ topicIdentity => - kafkaManager.getBrokerList(clusterName).map { errorOrBrokerList => - errorOrBrokerList.map { bl => - defaultAddPartitionsForm.fill(AddTopicPartitions(topic,bl.list.map(bi => BrokerSelect.from(bi)),topicIdentity.partitions,topicIdentity.readVersion)) + featureGate(KMTopicManagerFeature) { + val errorOrFormFuture = kafkaManager.getTopicIdentity(clusterName, topic).flatMap { errorOrTopicIdentity => + errorOrTopicIdentity.fold(e => Future.successful(-\/(e)), { topicIdentity => + kafkaManager.getBrokerList(clusterName).map { errorOrBrokerList => + errorOrBrokerList.map { bl => + (defaultAddPartitionsForm.fill(AddTopicPartitions(topic, bl.list.map(bi => BrokerSelect.from(bi)), topicIdentity.partitions, topicIdentity.readVersion)), + bl.clusterContext) + } } - } - }) - } - errorOrFormFuture.map { errorOrForm => - Ok(views.html.topic.addPartitions(clusterName, topic, errorOrForm)) + }) + } + errorOrFormFuture.map { errorOrForm => + Ok(views.html.topic.addPartitions(clusterName, topic, errorOrForm)) + } } } def addPartitionsToMultipleTopics(clusterName: String) = Action.async { implicit request => - val errorOrFormFuture = kafkaManager.getTopicListExtended(clusterName).flatMap { errorOrTle => - errorOrTle.fold( e => Future.successful(-\/(e)),{ topicListExtended => - kafkaManager.getBrokerList(clusterName).map { errorOrBrokerList => - errorOrBrokerList.map { bl => - val tl = kafkaManager.topicListSortedByNumPartitions(topicListExtended) - val topics = tl.map(t => t._1).map(t => TopicSelect.from(t)) - // default value is the largest number of partitions among existing topics with topic identity - val partitions = tl.head._2.map(_.partitions).getOrElse(0) - val readVersions = tl.map(t => t._2).flatMap(t => t).map(ti => ReadVersion(ti.topic, ti.readVersion)) - defaultAddMultipleTopicsPartitionsForm.fill(AddMultipleTopicsPartitions(topics,bl.list.map(bi => BrokerSelect.from(bi)),partitions,readVersions)) + featureGate(KMTopicManagerFeature) { + val errorOrFormFuture = kafkaManager.getTopicListExtended(clusterName).flatMap { errorOrTle => + errorOrTle.fold(e => Future.successful(-\/(e)), { topicListExtended => + kafkaManager.getBrokerList(clusterName).map { errorOrBrokerList => + errorOrBrokerList.map { bl => + val tl = kafkaManager.topicListSortedByNumPartitions(topicListExtended) + val topics = tl.map(t => t._1).map(t => TopicSelect.from(t)) + // default value is the largest number of partitions among existing topics with topic identity + val partitions = tl.head._2.map(_.partitions).getOrElse(0) + val readVersions = tl.map(t => t._2).flatMap(t => t).map(ti => ReadVersion(ti.topic, ti.readVersion)) + (defaultAddMultipleTopicsPartitionsForm.fill(AddMultipleTopicsPartitions(topics, bl.list.map(bi => BrokerSelect.from(bi)), partitions, readVersions)), + topicListExtended.clusterContext) + } } - } - }) - } - errorOrFormFuture.map { errorOrForm => - Ok(views.html.topic.addPartitionsToMultipleTopics(clusterName, errorOrForm)) + }) + } + errorOrFormFuture.map { errorOrForm => + Ok(views.html.topic.addPartitionsToMultipleTopics(clusterName, errorOrForm)) + } } } def handleAddPartitions(clusterName: String, topic: String) = Action.async { implicit request => - defaultAddPartitionsForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.topic.addPartitions(clusterName, topic,\/-(formWithErrors)))), - addTopicPartitions => { - kafkaManager.addTopicPartitions(clusterName,addTopicPartitions.topic,addTopicPartitions.brokers.filter(_.selected).map(_.id),addTopicPartitions.partitions,addTopicPartitions.readVersion).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(clusterName,"Topic","Topic View",Menus.clusterMenus(clusterName)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View",clusterName, topic,"Add Partitions"), - errorOrSuccess, - "Add Partitions", - FollowLink("Go to topic view.",routes.Topic.topic(clusterName, addTopicPartitions.topic).toString()), - FollowLink("Try again.",routes.Topic.addPartitions(clusterName, topic).toString()) - )) + featureGate(KMTopicManagerFeature) { + defaultAddPartitionsForm.bindFromRequest.fold( + formWithErrors => { + kafkaManager.getClusterContext(clusterName).map { clusterContext => + BadRequest(views.html.topic.addPartitions(clusterName, topic, clusterContext.map(c => (formWithErrors, c)))) + }.recover { + case t => + implicit val clusterFeatures = ClusterFeatures.default + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Topic", "Topic View", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", clusterName, topic, "Add Partitions"), + -\/(ApiError(s"Unknown error : ${t.getMessage}")), + "Add Partitions", + FollowLink("Try again.", routes.Topic.addPartitions(clusterName, topic).toString()), + FollowLink("Try again.", routes.Topic.addPartitions(clusterName, topic).toString()) + )) + } + }, + addTopicPartitions => { + kafkaManager.addTopicPartitions(clusterName, addTopicPartitions.topic, addTopicPartitions.brokers.filter(_.selected).map(_.id), addTopicPartitions.partitions, addTopicPartitions.readVersion).map { errorOrSuccess => + implicit val clusterFeatures = errorOrSuccess.toOption.map(_.clusterFeatures).getOrElse(ClusterFeatures.default) + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Topic", "Topic View", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", clusterName, topic, "Add Partitions"), + errorOrSuccess, + "Add Partitions", + FollowLink("Go to topic view.", routes.Topic.topic(clusterName, addTopicPartitions.topic).toString()), + FollowLink("Try again.", routes.Topic.addPartitions(clusterName, topic).toString()) + )) + } } - } - ) + ) + } } def handleAddPartitionsToMultipleTopics(clusterName: String) = Action.async { implicit request => - defaultAddMultipleTopicsPartitionsForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.topic.addPartitionsToMultipleTopics(clusterName, \/-(formWithErrors)))), - addMultipleTopicsPartitions => { - val topics = addMultipleTopicsPartitions.topics.filter(_.selected).map(_.name) - val brokers = addMultipleTopicsPartitions.brokers.filter(_.selected).map(_.id) - val readVersions = addMultipleTopicsPartitions.readVersions.map{ rv => (rv.topic, rv.version) }.toMap - kafkaManager.addMultipleTopicsPartitions(clusterName, topics, brokers, addMultipleTopicsPartitions.partitions, readVersions).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(clusterName,"Topics","Add Partitions to Multiple Topics",Menus.clusterMenus(clusterName)), - models.navigation.BreadCrumbs.withNamedViewAndCluster("Topics",clusterName,"Add Partitions to Multiple Topics"), - errorOrSuccess, - "Add Partitions to All Topics", - FollowLink("Go to topic list.",routes.Topic.topics(clusterName).toString()), - FollowLink("Try again.",routes.Topic.addPartitionsToMultipleTopics(clusterName).toString()) - )) + featureGate(KMTopicManagerFeature) { + defaultAddMultipleTopicsPartitionsForm.bindFromRequest.fold( + formWithErrors => { + kafkaManager.getClusterContext(clusterName).map { clusterContext => + BadRequest(views.html.topic.addPartitionsToMultipleTopics(clusterName, clusterContext.map(c => (formWithErrors, c)))) + }.recover { + case t => + implicit val clusterFeatures = ClusterFeatures.default + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Topics", "Add Partitions to Multiple Topics", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndCluster("Topics", clusterName, "Add Partitions to Multiple Topics"), + -\/(ApiError(s"Unknown error : ${t.getMessage}")), + "Add Partitions to All Topics", + FollowLink("Try again.", routes.Topic.addPartitionsToMultipleTopics(clusterName).toString()), + FollowLink("Try again.", routes.Topic.addPartitionsToMultipleTopics(clusterName).toString()) + )) + } + }, + addMultipleTopicsPartitions => { + val topics = addMultipleTopicsPartitions.topics.filter(_.selected).map(_.name) + val brokers = addMultipleTopicsPartitions.brokers.filter(_.selected).map(_.id) + val readVersions = addMultipleTopicsPartitions.readVersions.map { rv => (rv.topic, rv.version)}.toMap + kafkaManager.addMultipleTopicsPartitions(clusterName, topics, brokers, addMultipleTopicsPartitions.partitions, readVersions).map { errorOrSuccess => + implicit val clusterFeatures = errorOrSuccess.toOption.map(_.clusterFeatures).getOrElse(ClusterFeatures.default) + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Topics", "Add Partitions to Multiple Topics", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndCluster("Topics", clusterName, "Add Partitions to Multiple Topics"), + errorOrSuccess, + "Add Partitions to All Topics", + FollowLink("Go to topic list.", routes.Topic.topics(clusterName).toString()), + FollowLink("Try again.", routes.Topic.addPartitionsToMultipleTopics(clusterName).toString()) + )) + } } - } - ) + ) + } } private def updateConfigForm(clusterName: String, ti: TopicIdentity) = { @@ -272,39 +341,60 @@ object Topic extends Controller{ case Kafka_0_8_2_1 => TopicConfigs.configNames(Kafka_0_8_2_1).map(n => (n,TConfig(n,None))).toMap } val combinedMap = defaultConfigMap ++ ti.config.toMap.map(tpl => tpl._1 -> TConfig(tpl._1,Option(tpl._2))) - defaultUpdateConfigForm.fill(UpdateTopicConfig(ti.topic,combinedMap.toList.map(_._2),ti.configReadVersion)) + (defaultUpdateConfigForm.fill(UpdateTopicConfig(ti.topic,combinedMap.toList.map(_._2),ti.configReadVersion)), + ti.clusterContext) } } } def updateConfig(clusterName: String, topic: String) = Action.async { implicit request => - val errorOrFormFuture = kafkaManager.getTopicIdentity(clusterName, topic).flatMap { errorOrTopicIdentity => - errorOrTopicIdentity.fold( e => Future.successful(-\/(e)) ,{ topicIdentity => - updateConfigForm(clusterName, topicIdentity) - }) - } - errorOrFormFuture.map { errorOrForm => - Ok(views.html.topic.updateConfig(clusterName, topic, errorOrForm)) + featureGate(KMTopicManagerFeature) { + val errorOrFormFuture = kafkaManager.getTopicIdentity(clusterName, topic).flatMap { errorOrTopicIdentity => + errorOrTopicIdentity.fold(e => Future.successful(-\/(e)), { topicIdentity => + updateConfigForm(clusterName, topicIdentity) + }) + } + errorOrFormFuture.map { errorOrForm => + Ok(views.html.topic.updateConfig(clusterName, topic, errorOrForm)) + } } } def handleUpdateConfig(clusterName: String, topic: String) = Action.async { implicit request => - defaultUpdateConfigForm.bindFromRequest.fold( - formWithErrors => Future.successful(BadRequest(views.html.topic.updateConfig(clusterName, topic,\/-(formWithErrors)))), - updateTopicConfig => { - val props = new Properties() - updateTopicConfig.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name,c.value.get)) - kafkaManager.updateTopicConfig(clusterName,updateTopicConfig.topic,props,updateTopicConfig.readVersion).map { errorOrSuccess => - Ok(views.html.common.resultOfCommand( - views.html.navigation.clusterMenu(clusterName,"Topic","Topic View",Menus.clusterMenus(clusterName)), - models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View",clusterName, topic,"Update Config"), - errorOrSuccess, - "Update Config", - FollowLink("Go to topic view.",routes.Topic.topic(clusterName, updateTopicConfig.topic).toString()), - FollowLink("Try again.",routes.Topic.updateConfig(clusterName, topic).toString()) - )) + featureGate(KMTopicManagerFeature) { + defaultUpdateConfigForm.bindFromRequest.fold( + formWithErrors => { + kafkaManager.getClusterContext(clusterName).map { clusterContext => + BadRequest(views.html.topic.updateConfig(clusterName, topic, clusterContext.map(c => (formWithErrors, c)))) + }.recover { + case t => + implicit val clusterFeatures = ClusterFeatures.default + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Topic", "Topic View", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", clusterName, topic, "Update Config"), + -\/(ApiError(s"Unknown error : ${t.getMessage}")), + "Update Config", + FollowLink("Try again.", routes.Topic.updateConfig(clusterName, topic).toString()), + FollowLink("Try again.", routes.Topic.updateConfig(clusterName, topic).toString()) + )) + } + }, + updateTopicConfig => { + val props = new Properties() + updateTopicConfig.configs.filter(_.value.isDefined).foreach(c => props.setProperty(c.name, c.value.get)) + kafkaManager.updateTopicConfig(clusterName, updateTopicConfig.topic, props, updateTopicConfig.readVersion).map { errorOrSuccess => + implicit val clusterFeatures = errorOrSuccess.toOption.map(_.clusterFeatures).getOrElse(ClusterFeatures.default) + Ok(views.html.common.resultOfCommand( + views.html.navigation.clusterMenu(clusterName, "Topic", "Topic View", Menus.clusterMenus(clusterName)), + models.navigation.BreadCrumbs.withNamedViewAndClusterAndTopic("Topic View", clusterName, topic, "Update Config"), + errorOrSuccess, + "Update Config", + FollowLink("Go to topic view.", routes.Topic.topic(clusterName, updateTopicConfig.topic).toString()), + FollowLink("Try again.", routes.Topic.updateConfig(clusterName, topic).toString()) + )) + } } - } - ) + ) + } } } diff --git a/app/controllers/package.scala b/app/controllers/package.scala new file mode 100644 index 000000000..63c441c23 --- /dev/null +++ b/app/controllers/package.scala @@ -0,0 +1,68 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ +import features.{ApplicationFeatures, ApplicationFeature} +import kafka.manager.{ClusterContext, KafkaManager, ApiError} +import kafka.manager.features.{ClusterFeatures, ClusterFeature} +import play.api.mvc._ +import play.api.mvc.Results._ + +import scala.concurrent.Future +import scalaz.{\/-, -\/, \/} + +/** + * Created by hiral on 8/23/15. + */ +package object controllers { + + import play.api.libs.concurrent.Execution.Implicits.defaultContext + + def featureGate(af: ApplicationFeature)(fn: => Future[Result])(implicit features: ApplicationFeatures) : Future[Result] = { + if(features.features(af)) { + fn + } else { + Future.successful(Ok(views.html.errors.onApiError(ApiError(s"Feature disabled $af")))) + } + } + + def clusterFeatureGate(clusterName: String, cf: ClusterFeature)(fn: ClusterContext => Future[Result]) + (implicit km: KafkaManager) : Future[Result] = { + km.getClusterContext(clusterName).flatMap { clusterContextOrError => + clusterContextOrError.fold( + error => { + Future.successful(Ok(views.html.errors.onApiError(error, None))) + }, + clusterContext => { + if(clusterContext.clusterFeatures.features(cf)) { + fn(clusterContext) + } else { + Future.successful(Ok(views.html.errors.onApiError(ApiError(s"Unsupported feature : $cf"), None))) + } + }) + }.recover { + case t => + Ok(views.html.errors.onApiError(ApiError(t.getMessage), None)) + } + } + + def withClusterFeatures(clusterName: String)(err: ApiError => Future[Result], fn: ClusterFeatures => Future[Result]) + (implicit km: KafkaManager) : Future[Result] = { + km.getClusterContext(clusterName).flatMap { clusterContextOrError => + clusterContextOrError.map(_.clusterFeatures) match { + case -\/(error) => err(error) + case \/-(f) => fn(f) + } + } + } + + def withClusterContext(clusterName: String)(err: ApiError => Future[Result], fn: ClusterContext => Future[Result]) + (implicit km: KafkaManager) : Future[Result] = { + km.getClusterContext(clusterName).flatMap { clusterContextOrError => + clusterContextOrError match { + case -\/(error) => err(error) + case \/-(f) => fn(f) + } + } + } +} diff --git a/app/features/ApplicationFeature.scala b/app/features/ApplicationFeature.scala new file mode 100644 index 000000000..019f4b818 --- /dev/null +++ b/app/features/ApplicationFeature.scala @@ -0,0 +1,65 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ + +package features + +import com.typesafe.config.Config +import kafka.manager.features.KMFeature +import org.slf4j.LoggerFactory + +import scala.util.{Success, Failure, Try} + +sealed trait ApplicationFeature extends KMFeature + +case object KMClusterManagerFeature extends ApplicationFeature +case object KMTopicManagerFeature extends ApplicationFeature +case object KMPreferredReplicaElectionFeature extends ApplicationFeature +case object KMReassignPartitionsFeature extends ApplicationFeature +case object KMBootstrapClusterConfigFeature extends ApplicationFeature + +object ApplicationFeature { + private lazy val log = LoggerFactory.getLogger(classOf[ApplicationFeature]) + import scala.reflect.runtime.universe + + val runtimeMirror = universe.runtimeMirror(getClass.getClassLoader) + + def from(s: String) : Option[ApplicationFeature] = { + Try { + val clazz = s"features.$s" + val module = runtimeMirror.staticModule(clazz) + val obj = runtimeMirror.reflectModule(module) + obj.instance match { + case f: ApplicationFeature => + f + case _ => + throw new IllegalArgumentException(s"Unknown application feature $s") + } + } match { + case Failure(t) => + log.error(s"Unknown application feature $s") + None + case Success(f) => Option(f) + } + } + +} + +case class ApplicationFeatures(features: Set[ApplicationFeature]) + +object ApplicationFeatures { + import play.api.Play.current + + lazy val features = { + getApplicationFeatures(play.api.Play.configuration.underlying) + } + + def getApplicationFeatures(config: Config) : ApplicationFeatures = { + import scala.collection.JavaConverters._ + val configFeatures: List[String] = config.getStringList("application.features").asScala.toList + + val f = configFeatures.map(ApplicationFeature.from).flatten + ApplicationFeatures(f.toSet) + } +} diff --git a/app/features/package.scala b/app/features/package.scala new file mode 100644 index 000000000..60b931f92 --- /dev/null +++ b/app/features/package.scala @@ -0,0 +1,17 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ +import play.twirl.api.Html + +package object features { + val empty: Html = Html("") + + def app(f: ApplicationFeature)(content: Html)(implicit af: ApplicationFeatures): Html = { + if(af.features(f)) + content + else + empty + } + +} diff --git a/app/kafka/manager/ActorModel.scala b/app/kafka/manager/ActorModel.scala index 1b3e8a49a..dac47a661 100644 --- a/app/kafka/manager/ActorModel.scala +++ b/app/kafka/manager/ActorModel.scala @@ -19,14 +19,14 @@ import scalaz.{NonEmptyList, Validation} * @author hiral */ object ActorModel { - sealed trait ActorRequest - sealed trait ActorResponse + trait ActorRequest + trait ActorResponse - sealed trait CommandRequest extends ActorRequest - sealed trait CommandResponse extends ActorResponse + trait CommandRequest extends ActorRequest + trait CommandResponse extends ActorResponse - sealed trait QueryRequest extends ActorRequest - sealed trait QueryResponse extends ActorResponse + trait QueryRequest extends ActorRequest + trait QueryResponse extends ActorResponse case class ActorErrorResponse(msg: String, throwableOption: Option[Throwable] = None) extends ActorResponse @@ -38,7 +38,7 @@ object ActorModel { case object BVGetViews extends BVRequest case class BVGetTopicMetrics(topic: String) extends BVRequest case object BVGetBrokerMetrics extends BVRequest - case class BVView(topicPartitions: Map[TopicIdentity, IndexedSeq[Int]], clusterConfig: ClusterConfig, + case class BVView(topicPartitions: Map[TopicIdentity, IndexedSeq[Int]], clusterContext: ClusterContext, metrics: Option[BrokerMetrics] = None, messagesPerSecCountHistory: Option[Queue[BrokerMessagesPerSecCount]] = None, stats: Option[BrokerClusterStats] = None) extends QueryResponse { @@ -48,14 +48,10 @@ object ActorModel { case class BVUpdateTopicMetricsForBroker(id: Int, metrics: IndexedSeq[(String,BrokerMetrics)]) extends CommandRequest case class BVUpdateBrokerMetrics(id: Int, metric: BrokerMetrics) extends CommandRequest - sealed trait LKVRequest extends QueryRequest - - case object LKVForceUpdate extends CommandRequest - case object LKVGetLogkafkaIdentities extends LKVRequest - case object CMGetView extends QueryRequest case class CMGetTopicIdentity(topic: String) extends QueryRequest - case class CMView(topicsCount: Int, brokersCount: Int, clusterConfig: ClusterConfig) extends QueryResponse + case object CMGetClusterContext extends QueryRequest + case class CMView(topicsCount: Int, brokersCount: Int, clusterContext: ClusterContext) extends QueryResponse case class CMTopicIdentity(topicIdentity: Try[TopicIdentity]) extends QueryResponse case object CMShutdown extends CommandRequest case class CMCreateTopic(topic: String, @@ -78,6 +74,8 @@ object ActorModel { case class CMGeneratePartitionAssignments(topics: Set[String], brokers: Seq[Int]) extends CommandRequest case class CMManualPartitionAssignments(assignments: List[(String, List[(Int, List[Int])])]) extends CommandRequest + //these are used by Logkafka + //########## case class CMGetLogkafkaIdentity(hostname: String) extends QueryRequest case class CMLogkafkaIdentity(logkafkaIdentity: Try[LogkafkaIdentity]) extends QueryResponse case class CMCreateLogkafka(hostname: String, @@ -88,8 +86,9 @@ object ActorModel { log_path: String, config: Properties) extends CommandRequest case class CMDeleteLogkafka(hostname: String, log_path: String) extends CommandRequest + //########## - case class CMCommandResult(result: Try[Unit]) extends CommandResponse + case class CMCommandResult(result: Try[ClusterContext]) extends CommandResponse case class CMCommandResults(result: IndexedSeq[Try[Unit]]) extends CommandResponse case class KCCreateTopic(topic: String, @@ -114,20 +113,6 @@ object ActorModel { case class KCCommandResult(result: Try[Unit]) extends CommandResponse - case class LKCCreateLogkafka(hostname: String, - log_path: String, - config: Properties, - logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest - case class LKCDeleteLogkafka(hostname: String, - log_path: String, - logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest - case class LKCUpdateLogkafkaConfig(hostname: String, - log_path: String, - config: Properties, - logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest - - case class LKCCommandResult(result: Try[Unit]) extends CommandResponse - case object KMGetActiveClusters extends QueryRequest case object KMGetAllClusters extends QueryRequest case class KMGetClusterConfig(clusterName: String) extends QueryRequest @@ -165,35 +150,25 @@ object ActorModel { case object KSGetBrokers extends KSRequest case class KSGetBrokerState(id: String) extends KSRequest - sealed trait LKSRequest extends QueryRequest - case object LKSGetLogkafkaHostnames extends LKSRequest - case class LKSGetLogkafkaConfig(hostname: String) extends LKSRequest - case class LKSGetLogkafkaClient(hostname: String) extends LKSRequest - case class LKSGetLogkafkaConfigs(hostnames: Set[String]) extends LKSRequest - case class LKSGetLogkafkaClients(hostnames: Set[String]) extends LKSRequest - case class LKSGetAllLogkafkaConfigs(lastUpdateMillis: Option[Long]= None) extends LKSRequest - case class LKSGetAllLogkafkaClients(lastUpdateMillis: Option[Long]= None) extends LKSRequest - - case class TopicList(list: IndexedSeq[String], deleteSet: Set[String]) extends QueryResponse + case class TopicList(list: IndexedSeq[String], deleteSet: Set[String], clusterContext: ClusterContext) extends QueryResponse case class TopicConfig(topic: String, config: Option[(Int,String)]) extends QueryResponse case class TopicDescription(topic: String, description: (Int,String), partitionState: Option[Map[String, String]], - config:Option[(Int,String)], - deleteSupported: Boolean) extends QueryResponse + config:Option[(Int,String)]) extends QueryResponse case class TopicDescriptions(descriptions: IndexedSeq[TopicDescription], lastUpdateMillis: Long) extends QueryResponse - case class BrokerList(list: IndexedSeq[BrokerIdentity], clusterConfig: ClusterConfig) extends QueryResponse + case class BrokerList(list: IndexedSeq[BrokerIdentity], clusterContext: ClusterContext) extends QueryResponse - case class PreferredReplicaElection(startTime: DateTime, topicAndPartition: Set[TopicAndPartition], endTime: Option[DateTime]) extends QueryResponse - case class ReassignPartitions(startTime: DateTime, partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]], endTime: Option[DateTime]) extends QueryResponse - - case class LogkafkaHostnameList(list: IndexedSeq[String], deleteSet: Set[String]) extends QueryResponse - case class LogkafkaConfig(hostname: String, config: Option[String]) extends QueryResponse - case class LogkafkaClient(hostname: String, client: Option[String]) extends QueryResponse - case class LogkafkaConfigs(configs: IndexedSeq[LogkafkaConfig], lastUpdateMillis: Long) extends QueryResponse - case class LogkafkaClients(clients: IndexedSeq[LogkafkaClient], lastUpdateMillis: Long) extends QueryResponse + case class PreferredReplicaElection(startTime: DateTime, + topicAndPartition: Set[TopicAndPartition], + endTime: Option[DateTime], + clusterContext: ClusterContext) extends QueryResponse + case class ReassignPartitions(startTime: DateTime, + partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]], + endTime: Option[DateTime], + clusterContext: ClusterContext) extends QueryResponse case object DCUpdateState extends CommandRequest @@ -256,8 +231,7 @@ object ActorModel { numBrokers: Int, configReadVersion: Int, config: List[(String,String)], - deleteSupported: Boolean, - clusterConfig: ClusterConfig, + clusterContext: ClusterContext, metrics: Option[BrokerMetrics] = None) { val replicationFactor : Int = partitionsIdentity.head._2.replicas.size @@ -305,7 +279,7 @@ object ActorModel { import org.json4s.scalaz.JsonScalaz._ import scala.language.reflectiveCalls - implicit def from(brokers: Int,td: TopicDescription, tm: Option[BrokerMetrics], clusterConfig: ClusterConfig) : TopicIdentity = { + implicit def from(brokers: Int,td: TopicDescription, tm: Option[BrokerMetrics], clusterContext: ClusterContext) : TopicIdentity = { val descJson = parse(td.description._2) //val partMap = (descJson \ "partitions").as[Map[String,Seq[Int]]] val partMap = field[Map[String,List[Int]]]("partitions")(descJson).fold({ e => @@ -333,11 +307,11 @@ object ActorModel { (-1,Map.empty[String, String]) } } - TopicIdentity(td.topic,td.description._1,partMap.size,tpi,brokers,config._1,config._2.toList,td.deleteSupported, clusterConfig, tm) + TopicIdentity(td.topic,td.description._1,partMap.size,tpi,brokers,config._1,config._2.toList, clusterContext, tm) } - implicit def from(bl: BrokerList,td: TopicDescription, tm: Option[BrokerMetrics], clusterConfig: ClusterConfig) : TopicIdentity = { - from(bl.list.size, td, tm, clusterConfig) + implicit def from(bl: BrokerList,td: TopicDescription, tm: Option[BrokerMetrics], clusterContext: ClusterContext) : TopicIdentity = { + from(bl.list.size, td, tm, clusterContext) } implicit def reassignReplicas(currentTopicIdentity: TopicIdentity, @@ -358,45 +332,12 @@ object ActorModel { currentTopicIdentity.numBrokers, currentTopicIdentity.configReadVersion, currentTopicIdentity.config, - currentTopicIdentity.deleteSupported, - currentTopicIdentity.clusterConfig, + currentTopicIdentity.clusterContext, currentTopicIdentity.metrics) } } } - case class LogkafkaIdentity(hostname: String, - active: Boolean, - deleteSupported: Boolean = true, - identityMap: Map[String, (Option[Map[String, String]], Option[Map[String, String]])]) { - } - - object LogkafkaIdentity { - - lazy val logger = LoggerFactory.getLogger(this.getClass) - - implicit def from(hostname: String, lcg: Option[LogkafkaConfig], lct: Option[LogkafkaClient]) : LogkafkaIdentity = { - val configJsonStr = lcg match { - case Some(l) => l.config.getOrElse[String]("{}") - case None => "{}" - } - - val configMap: Map[String, Map[String, String]] = utils.Logkafka.parseJsonStr(hostname, configJsonStr) - - val clientJsonStr = lct match { - case Some(l) => l.client.getOrElse[String]("{}") - case None => "{}" - } - - val clientMap: Map[String, Map[String, String]] = utils.Logkafka.parseJsonStr(hostname, clientJsonStr) - - val hostnameSet = configMap.keySet ++ clientMap.keySet - val identitySet = if (!hostnameSet.isEmpty) { - hostnameSet map { l => l -> ((if(!configMap.isEmpty) configMap.get(l) else None, if(!clientMap.isEmpty) clientMap.get(l) else None)) } - } else { Set() } - LogkafkaIdentity(hostname, lct.isDefined, true, identitySet.toMap) - } - } case class BrokerMessagesPerSecCount(date: DateTime, count: Long) @@ -433,4 +374,71 @@ object ActorModel { } case class BrokerClusterStats(perMessages: BigDecimal, perIncoming: BigDecimal, perOutgoing: BigDecimal) + + sealed trait LKVRequest extends QueryRequest + + case object LKVForceUpdate extends CommandRequest + case object LKVGetLogkafkaIdentities extends LKVRequest + + case class LKCCreateLogkafka(hostname: String, + log_path: String, + config: Properties, + logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest + case class LKCDeleteLogkafka(hostname: String, + log_path: String, + logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest + case class LKCUpdateLogkafkaConfig(hostname: String, + log_path: String, + config: Properties, + logkafkaConfig: Option[LogkafkaConfig]) extends CommandRequest + + case class LKCCommandResult(result: Try[Unit]) extends CommandResponse + + sealed trait LKSRequest extends QueryRequest + case object LKSGetLogkafkaHostnames extends LKSRequest + case class LKSGetLogkafkaConfig(hostname: String) extends LKSRequest + case class LKSGetLogkafkaClient(hostname: String) extends LKSRequest + case class LKSGetLogkafkaConfigs(hostnames: Set[String]) extends LKSRequest + case class LKSGetLogkafkaClients(hostnames: Set[String]) extends LKSRequest + case class LKSGetAllLogkafkaConfigs(lastUpdateMillis: Option[Long]= None) extends LKSRequest + case class LKSGetAllLogkafkaClients(lastUpdateMillis: Option[Long]= None) extends LKSRequest + + case class LogkafkaHostnameList(list: IndexedSeq[String], deleteSet: Set[String]) extends QueryResponse + case class LogkafkaConfig(hostname: String, config: Option[String]) extends QueryResponse + case class LogkafkaClient(hostname: String, client: Option[String]) extends QueryResponse + case class LogkafkaConfigs(configs: IndexedSeq[LogkafkaConfig], lastUpdateMillis: Long) extends QueryResponse + case class LogkafkaClients(clients: IndexedSeq[LogkafkaClient], lastUpdateMillis: Long) extends QueryResponse + + + case class LogkafkaIdentity(hostname: String, + active: Boolean, + identityMap: Map[String, (Option[Map[String, String]], Option[Map[String, String]])]) { + } + + object LogkafkaIdentity { + + lazy val logger = LoggerFactory.getLogger(this.getClass) + + implicit def from(hostname: String, lcg: Option[LogkafkaConfig], lct: Option[LogkafkaClient]) : LogkafkaIdentity = { + val configJsonStr = lcg match { + case Some(l) => l.config.getOrElse[String]("{}") + case None => "{}" + } + + val configMap: Map[String, Map[String, String]] = utils.Logkafka.parseJsonStr(hostname, configJsonStr) + + val clientJsonStr = lct match { + case Some(l) => l.client.getOrElse[String]("{}") + case None => "{}" + } + + val clientMap: Map[String, Map[String, String]] = utils.Logkafka.parseJsonStr(hostname, clientJsonStr) + + val hostnameSet = configMap.keySet ++ clientMap.keySet + val identitySet = if (!hostnameSet.isEmpty) { + hostnameSet map { l => l -> ((if(!configMap.isEmpty) configMap.get(l) else None, if(!clientMap.isEmpty) clientMap.get(l) else None)) } + } else { Set() } + LogkafkaIdentity(hostname, lct.isDefined, identitySet.toMap) + } + } } diff --git a/app/kafka/manager/BrokerViewCacheActor.scala b/app/kafka/manager/BrokerViewCacheActor.scala index 0731138ce..0a5f86cfe 100644 --- a/app/kafka/manager/BrokerViewCacheActor.scala +++ b/app/kafka/manager/BrokerViewCacheActor.scala @@ -6,6 +6,7 @@ package kafka.manager import akka.actor.{ActorRef, Cancellable, ActorPath} +import kafka.manager.features.KMJMXMetricsFeature import kafka.manager.utils.FiniteQueue import org.joda.time.DateTime @@ -20,7 +21,7 @@ import scala.util.Try */ import ActorModel._ case class BrokerViewCacheActorConfig(kafkaStateActorPath: ActorPath, - clusterConfig: ClusterConfig, + clusterContext: ClusterContext, longRunningPoolConfig: LongRunningPoolConfig, updatePeriod: FiniteDuration = 10 seconds) class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunningPoolActor { @@ -44,7 +45,7 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni private[this] var combinedBrokerMetric : Option[BrokerMetrics] = None - private[this] val EMPTY_BVVIEW = BVView(Map.empty, config.clusterConfig, Option(BrokerMetrics.DEFAULT)) + private[this] val EMPTY_BVVIEW = BVView(Map.empty, config.clusterContext, Option(BrokerMetrics.DEFAULT)) private[this] var brokerMessagesPerSecCountHistory : Map[Int, Queue[BrokerMessagesPerSecCount]] = Map.empty @@ -184,13 +185,13 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni topicDescriptions <- topicDescriptionsOption } { val topicIdentity : IndexedSeq[TopicIdentity] = topicDescriptions.descriptions.map( - TopicIdentity.from(brokerList.list.size,_,None, config.clusterConfig)) + TopicIdentity.from(brokerList.list.size,_,None, config.clusterContext)) topicIdentities = topicIdentity.map(ti => (ti.topic, ti)).toMap val topicPartitionByBroker = topicIdentity.flatMap( ti => ti.partitionsByBroker.map(btp => (ti,btp.id,btp.partitions))).groupBy(_._2) //check for 2*broker list size since we schedule 2 jmx calls for each broker - if (config.clusterConfig.jmxEnabled && hasCapacityFor(2*brokerListOption.size)) { + if (config.clusterContext.clusterFeatures.features(KMJMXMetricsFeature) && hasCapacityFor(2*brokerListOption.size)) { implicit val ec = longRunningExecutionContext val brokerLookup = brokerList.list.map(bi => bi.id -> bi).toMap topicPartitionByBroker.foreach { @@ -205,7 +206,7 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni topicPartitions.map { case (topic, id, partitions) => (topic.topic, - KafkaMetrics.getBrokerMetrics(config.clusterConfig.version, mbsc, Option(topic.topic))) + KafkaMetrics.getBrokerMetrics(config.clusterContext.config.version, mbsc, Option(topic.topic))) } } val result = tryResult match { @@ -229,7 +230,7 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni Future { val tryResult = KafkaJMX.doWithConnection(broker.host, broker.jmxPort) { mbsc => - KafkaMetrics.getBrokerMetrics(config.clusterConfig.version, mbsc) + KafkaMetrics.getBrokerMetrics(config.clusterContext.config.version, mbsc) } val result = tryResult match { @@ -242,7 +243,7 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni } } } - } else if(config.clusterConfig.jmxEnabled) { + } else if(config.clusterContext.clusterFeatures.features(KMJMXMetricsFeature)) { log.warning("Not scheduling update of JMX for all brokers, not enough capacity!") } @@ -253,7 +254,7 @@ class BrokerViewCacheActor(config: BrokerViewCacheActorConfig) extends LongRunni (topic, partitions) }.toMap brokerTopicPartitions.put( - brokerId, BVView(topicPartitionsMap, config.clusterConfig, brokerMetrics.get(brokerId))) + brokerId, BVView(topicPartitionsMap, config.clusterContext, brokerMetrics.get(brokerId))) } } } diff --git a/app/kafka/manager/ClusterManagerActor.scala b/app/kafka/manager/ClusterManagerActor.scala index 1b2f9c31b..be7cf9072 100644 --- a/app/kafka/manager/ClusterManagerActor.scala +++ b/app/kafka/manager/ClusterManagerActor.scala @@ -11,6 +11,7 @@ import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, ThreadPoolExecutor} import akka.pattern._ import akka.actor.{ActorPath, Props} import akka.util.Timeout +import kafka.manager.features.{KMJMXMetricsFeature, KMLogKafkaFeature, ClusterFeatures} import org.apache.curator.framework.CuratorFramework import org.apache.curator.framework.recipes.cache.PathChildrenCache import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode @@ -59,6 +60,8 @@ case class ClusterManagerActorConfig(pinnedDispatcherName: String, class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) extends BaseQueryCommandActor with CuratorAwareActor with BaseZkPath { + private[this] val clusterContext = ClusterContext(ClusterFeatures.from(cmConfig.clusterConfig), cmConfig.clusterConfig) + //this is from base zk path trait override def baseZkPath : String = cmConfig.baseZkPath @@ -87,71 +90,72 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) private[this] val adminUtils = new AdminUtils(cmConfig.clusterConfig.version) - private[this] val ksProps = Props(classOf[KafkaStateActor],sharedClusterCurator, adminUtils.isDeleteSupported, cmConfig.clusterConfig) + private[this] val ksProps = Props(classOf[KafkaStateActor],sharedClusterCurator, clusterContext) private[this] val kafkaStateActor : ActorPath = context.actorOf(ksProps.withDispatcher(cmConfig.pinnedDispatcherName),"kafka-state").path - private[this] val lksProps: Option[Props] = - cmConfig.clusterConfig.logkafkaEnabled match { - case true => Some(Props(classOf[LogkafkaStateActor],sharedClusterCurator, adminUtils.isDeleteSupported, cmConfig.clusterConfig)) - case false => None - } - private[this] val logkafkaStateActor : Option[ActorPath] = - cmConfig.clusterConfig.logkafkaEnabled match { - case true => Some(context.actorOf(lksProps.get.withDispatcher(cmConfig.pinnedDispatcherName),"logkafka-state").path) - case false => None - } - private[this] val bvConfig = BrokerViewCacheActorConfig( kafkaStateActor, - cmConfig.clusterConfig, + clusterContext, LongRunningPoolConfig(Runtime.getRuntime.availableProcessors(), 1000), cmConfig.updatePeriod) private[this] val bvcProps = Props(classOf[BrokerViewCacheActor],bvConfig) private[this] val brokerViewCacheActor : ActorPath = context.actorOf(bvcProps,"broker-view").path - private[this] val lkvConfig: Option[LogkafkaViewCacheActorConfig] = - cmConfig.clusterConfig.logkafkaEnabled match { + private[this] val kcProps = { + val kcaConfig = KafkaCommandActorConfig( + sharedClusterCurator, + LongRunningPoolConfig(cmConfig.threadPoolSize, cmConfig.maxQueueSize), + cmConfig.askTimeoutMillis, + clusterContext, + adminUtils) + Props(classOf[KafkaCommandActor],kcaConfig) + } + private[this] val kafkaCommandActor : ActorPath = context.actorOf(kcProps,"kafka-command").path + + private[this] val lksProps: Option[Props] = + clusterContext.clusterFeatures.features(KMLogKafkaFeature) match { + case true => Some(Props(classOf[LogkafkaStateActor],sharedClusterCurator, clusterContext)) + case false => None + } + private[this] val logkafkaStateActor : Option[ActorPath] = + clusterContext.clusterFeatures.features(KMLogKafkaFeature) match { + case true => Some(context.actorOf(lksProps.get.withDispatcher(cmConfig.pinnedDispatcherName),"logkafka-state").path) + case false => None + } + + private[this] val lkvConfig: Option[LogkafkaViewCacheActorConfig] = + clusterContext.clusterFeatures.features(KMLogKafkaFeature) match { case true => Some(LogkafkaViewCacheActorConfig( - logkafkaStateActor.get, - cmConfig.clusterConfig, + logkafkaStateActor.get, + clusterContext, LongRunningPoolConfig(Runtime.getRuntime.availableProcessors(), 1000), cmConfig.updatePeriod)) case false => None } - private[this] val lkvcProps: Option[Props] = - cmConfig.clusterConfig.logkafkaEnabled match { + private[this] val lkvcProps: Option[Props] = + clusterContext.clusterFeatures.features(KMLogKafkaFeature) match { case true => Some(Props(classOf[LogkafkaViewCacheActor],lkvConfig.get)) case false => None } - private[this] val logkafkaViewCacheActor: Option[ActorPath] = - cmConfig.clusterConfig.logkafkaEnabled match { + private[this] val logkafkaViewCacheActor: Option[ActorPath] = + clusterContext.clusterFeatures.features(KMLogKafkaFeature) match { case true => Some(context.actorOf(lkvcProps.get,"logkafka-view").path) case false => None } - private[this] val kcProps = { - val kcaConfig = KafkaCommandActorConfig( - sharedClusterCurator, - LongRunningPoolConfig(cmConfig.threadPoolSize, cmConfig.maxQueueSize), - cmConfig.askTimeoutMillis, - cmConfig.clusterConfig.version) - Props(classOf[KafkaCommandActor],kcaConfig) - } - private[this] val kafkaCommandActor : ActorPath = context.actorOf(kcProps,"kafka-command").path - private[this] val lkcProps: Option[Props] = { - cmConfig.clusterConfig.logkafkaEnabled match { + clusterContext.clusterFeatures.features(KMLogKafkaFeature) match { case true => { val lkcaConfig = LogkafkaCommandActorConfig( sharedClusterCurator, LongRunningPoolConfig(cmConfig.threadPoolSize, cmConfig.maxQueueSize), cmConfig.askTimeoutMillis, - cmConfig.clusterConfig.version) + clusterContext) Some(Props(classOf[LogkafkaCommandActor],lkcaConfig)) } case false => None } } - private[this] val logkafkaCommandActor : Option[ActorPath] = - cmConfig.clusterConfig.logkafkaEnabled match { + private[this] val logkafkaCommandActor : Option[ActorPath] = + clusterContext.clusterFeatures.features(KMLogKafkaFeature) match { case true => Some(context.actorOf(lkcProps.get,"logkafka-command").path) case false => None } @@ -212,6 +216,9 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) case false => log.warning("cma: processQueryResponse : Received LKVRequest", lkvRequest) } + case CMGetClusterContext => + sender ! clusterContext + case CMGetView => implicit val ec = context.dispatcher val eventualBrokerList = withKafkaStateActor(KSGetBrokers)(identity[BrokerList]) @@ -219,14 +226,14 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) val result = for { bl <- eventualBrokerList tl <- eventualTopicList - } yield CMView(tl.list.size, bl.list.size, cmConfig.clusterConfig) + } yield CMView(tl.list.size, bl.list.size, clusterContext) result pipeTo sender case CMGetTopicIdentity(topic) => implicit val ec = context.dispatcher val eventualBrokerList = withKafkaStateActor(KSGetBrokers)(identity[BrokerList]) val eventualTopicMetrics : Future[Option[BrokerMetrics]] = { - if(cmConfig.clusterConfig.jmxEnabled) { + if(clusterContext.clusterFeatures.features(KMJMXMetricsFeature)) { withBrokerViewCacheActor(BVGetTopicMetrics(topic))(identity[Option[BrokerMetrics]]) } else { Future.successful(None) @@ -237,7 +244,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) bl <- eventualBrokerList tm <- eventualTopicMetrics tdO <- eventualTopicDescription - } yield tdO.map( td => CMTopicIdentity(Try(TopicIdentity.from(bl,td,tm,cmConfig.clusterConfig)))) + } yield tdO.map( td => CMTopicIdentity(Try(TopicIdentity.from(bl,td,tm,clusterContext)))) result pipeTo sender case CMGetLogkafkaIdentity(hostname) => @@ -269,6 +276,10 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) } } } + + implicit private def toTryClusterContext(t: Try[Unit]) : Try[ClusterContext] = { + t.map(_ => clusterContext) + } override def processCommandRequest(request: CommandRequest): Unit = { request match { @@ -373,7 +384,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) bl <- eventualBrokerList tds <- eventualDescriptions rp <- eventualReassignPartitions - tis = tds.descriptions.map(TopicIdentity.from(bl, _, None,cmConfig.clusterConfig)) + tis = tds.descriptions.map(TopicIdentity.from(bl, _, None,clusterContext)) } yield { bl.list.map(_.id.toInt) // check if any topic undergoing reassignment got selected for reassignment @@ -404,7 +415,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) assignments.map { case (topic, assignment) => updateAssignmentInZk(topic, assignment.toMap) } - } toIndexedSeq + }.toIndexedSeq } result.map(CMCommandResults.apply) pipeTo sender() @@ -415,7 +426,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) val preferredLeaderElections = for { bl <- eventualBrokerList tds <- eventualDescriptions - tis = tds.descriptions.map(TopicIdentity.from(bl, _, None, cmConfig.clusterConfig)) + tis = tds.descriptions.map(TopicIdentity.from(bl, _, None, clusterContext)) toElect = tis.map(ti => ti.partitionsIdentity.values.filter(!_.isPreferredLeader).map(tpi => TopicAndPartition(ti.topic, tpi.partNum))).flatten.toSet } yield toElect preferredLeaderElections.map { toElect => @@ -431,7 +442,7 @@ class ClusterManagerActor(cmConfig: ClusterManagerActorConfig) val topicsAndReassignments = for { bl <- eventualBrokerList tds <- eventualDescriptions - tis = tds.descriptions.map(TopicIdentity.from(bl, _, None, cmConfig.clusterConfig)) + tis = tds.descriptions.map(TopicIdentity.from(bl, _, None, clusterContext)) } yield { val reassignments = tis.map { ti => val topicZkPath = zkPathFrom(baseTopicsZkPath, ti.topic) diff --git a/app/kafka/manager/KafkaCommandActor.scala b/app/kafka/manager/KafkaCommandActor.scala index fb6c8bd36..9926a83fa 100644 --- a/app/kafka/manager/KafkaCommandActor.scala +++ b/app/kafka/manager/KafkaCommandActor.scala @@ -9,6 +9,7 @@ import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, ThreadPoolExecutor} import akka.pattern._ import akka.util.Timeout +import kafka.manager.features.KMDeleteTopicFeature import kafka.manager.utils.zero81.{ReassignPartitionCommand, PreferredReplicaLeaderElectionCommand} import org.apache.curator.framework.CuratorFramework import kafka.manager.utils.{AdminUtils, ZkUtils} @@ -26,14 +27,13 @@ import ActorModel._ case class KafkaCommandActorConfig(curator: CuratorFramework, longRunningPoolConfig: LongRunningPoolConfig, askTimeoutMillis: Long = 400, - version: KafkaVersion) + clusterContext: ClusterContext, + adminUtils: AdminUtils) class KafkaCommandActor(kafkaCommandActorConfig: KafkaCommandActorConfig) extends BaseCommandActor with LongRunningPoolActor { //private[this] val askTimeout: Timeout = kafkaCommandActorConfig.askTimeoutMillis.milliseconds - private[this] val adminUtils = new AdminUtils(kafkaCommandActorConfig.version) - - private[this] val reassignPartitionCommand = new ReassignPartitionCommand(adminUtils) + private[this] val reassignPartitionCommand = new ReassignPartitionCommand(kafkaCommandActorConfig.adminUtils) @scala.throws[Exception](classOf[Exception]) override def preStart() = { @@ -68,26 +68,25 @@ class KafkaCommandActor(kafkaCommandActorConfig: KafkaCommandActorConfig) extend implicit val ec = longRunningExecutionContext request match { case KCDeleteTopic(topic) => - kafkaCommandActorConfig.version match { - case Kafka_0_8_1_1 => - val result : KCCommandResult = KCCommandResult(Failure(new UnsupportedOperationException( - s"Delete topic not supported for kafka version ${kafkaCommandActorConfig.version}"))) - sender ! result - case Kafka_0_8_2_0 | Kafka_0_8_2_1 => - longRunning { - Future { - KCCommandResult(Try { - adminUtils.deleteTopic(kafkaCommandActorConfig.curator, topic) //this should work in 0.8.2 - kafkaCommandActorConfig.curator.delete().deletingChildrenIfNeeded().forPath(ZkUtils.getTopicPath(topic)) - }) - } + if(kafkaCommandActorConfig.clusterContext.clusterFeatures.features(KMDeleteTopicFeature)) { + longRunning { + Future { + KCCommandResult(Try { + kafkaCommandActorConfig.adminUtils.deleteTopic(kafkaCommandActorConfig.curator, topic) //this should work in 0.8.2 + kafkaCommandActorConfig.curator.delete().deletingChildrenIfNeeded().forPath(ZkUtils.getTopicPath(topic)) + }) } + } + } else { + val result : KCCommandResult = KCCommandResult(Failure(new UnsupportedOperationException( + s"Delete topic not supported for kafka version ${kafkaCommandActorConfig.clusterContext.config.version}"))) + sender ! result } case KCCreateTopic(topic, brokers, partitions, replicationFactor, config) => longRunning { Future { KCCommandResult(Try { - adminUtils.createTopic(kafkaCommandActorConfig.curator, brokers, topic, partitions, replicationFactor, config) + kafkaCommandActorConfig.adminUtils.createTopic(kafkaCommandActorConfig.curator, brokers, topic, partitions, replicationFactor, config) }) } } @@ -95,7 +94,7 @@ class KafkaCommandActor(kafkaCommandActorConfig: KafkaCommandActorConfig) extend longRunning { Future { KCCommandResult(Try { - adminUtils.addPartitions(kafkaCommandActorConfig.curator, topic, partitions, partitionReplicaList, brokers, readVersion) + kafkaCommandActorConfig.adminUtils.addPartitions(kafkaCommandActorConfig.curator, topic, partitions, partitionReplicaList, brokers, readVersion) }) } } @@ -103,7 +102,7 @@ class KafkaCommandActor(kafkaCommandActorConfig: KafkaCommandActorConfig) extend longRunning { Future { KCCommandResult(Try { - adminUtils.addPartitionsToTopics(kafkaCommandActorConfig.curator, topicsAndReplicas, partitions, brokers, readVersion) + kafkaCommandActorConfig.adminUtils.addPartitionsToTopics(kafkaCommandActorConfig.curator, topicsAndReplicas, partitions, brokers, readVersion) }) } } @@ -111,7 +110,7 @@ class KafkaCommandActor(kafkaCommandActorConfig: KafkaCommandActorConfig) extend longRunning { Future { KCCommandResult(Try { - adminUtils.changeTopicConfig(kafkaCommandActorConfig.curator, topic, config, readVersion) + kafkaCommandActorConfig.adminUtils.changeTopicConfig(kafkaCommandActorConfig.curator, topic, config, readVersion) }) } } diff --git a/app/kafka/manager/KafkaManager.scala b/app/kafka/manager/KafkaManager.scala index 3d59b83cc..8a95b5150 100644 --- a/app/kafka/manager/KafkaManager.scala +++ b/app/kafka/manager/KafkaManager.scala @@ -11,11 +11,9 @@ import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, ThreadPoolExecutor} import akka.actor.{ActorPath, ActorSystem, Props} import akka.util.Timeout import com.typesafe.config.{ConfigFactory, Config} -import controllers.Topic import kafka.manager.ActorModel._ import org.slf4j.{LoggerFactory, Logger} -import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ import scala.reflect.ClassTag @@ -24,8 +22,8 @@ import scala.util.{Success, Failure, Try} /** * @author hiral */ -case class TopicListExtended(list: IndexedSeq[(String, Option[TopicIdentity])], deleteSet: Set[String], underReassignments: IndexedSeq[String]) -case class BrokerListExtended(list: IndexedSeq[BrokerIdentity], metrics: Map[Int,BrokerMetrics], combinedMetric: Option[BrokerMetrics], clusterConfig: ClusterConfig) +case class TopicListExtended(list: IndexedSeq[(String, Option[TopicIdentity])], deleteSet: Set[String], underReassignments: IndexedSeq[String], clusterContext: ClusterContext) +case class BrokerListExtended(list: IndexedSeq[BrokerIdentity], metrics: Map[Int,BrokerMetrics], combinedMetric: Option[BrokerMetrics], clusterContext: ClusterContext) case class LogkafkaListExtended(list: IndexedSeq[(String, Option[LogkafkaIdentity])], deleteSet: Set[String]) case class ApiError(msg: String) object ApiError { @@ -208,7 +206,7 @@ class KafkaManager(akkaConfig: Config) } } - def runPreferredLeaderElection(clusterName: String, topics: Set[String]): Future[ApiError \/ Unit] = { + def runPreferredLeaderElection(clusterName: String, topics: Set[String]): Future[ApiError \/ ClusterContext] = { implicit val ec = apiExecutionContext withKafkaManagerActor( KMClusterCommandRequest( @@ -291,7 +289,7 @@ class KafkaManager(akkaConfig: Config) partitions: Int, replication: Int, config: Properties = new Properties - ): Future[ApiError \/ Unit] = + ): Future[ApiError \/ ClusterContext] = { implicit val ec = apiExecutionContext withKafkaManagerActor(KMClusterCommandRequest(clusterName, CMCreateTopic(topic, partitions, replication, config))) { @@ -306,7 +304,7 @@ class KafkaManager(akkaConfig: Config) brokers: Seq[Int], partitions: Int, readVersion: Int - ): Future[ApiError \/ Unit] = + ): Future[ApiError \/ ClusterContext] = { implicit val ec = apiExecutionContext getTopicIdentity(clusterName, topic).flatMap { topicIdentityOrError => @@ -333,7 +331,7 @@ class KafkaManager(akkaConfig: Config) brokers: Seq[Int], partitions: Int, readVersions: Map[String, Int] - ): Future[ApiError \/ Unit] = + ): Future[ApiError \/ ClusterContext] = { implicit val ec = apiExecutionContext getTopicListExtended(clusterName).flatMap { tleOrError => @@ -360,7 +358,7 @@ class KafkaManager(akkaConfig: Config) topic: String, config: Properties, readVersion: Int - ): Future[ApiError \/ Unit] = + ): Future[ApiError \/ ClusterContext] = { implicit val ec = apiExecutionContext withKafkaManagerActor( @@ -377,7 +375,7 @@ class KafkaManager(akkaConfig: Config) def deleteTopic( clusterName: String, topic: String - ): Future[ApiError \/ Unit] = + ): Future[ApiError \/ ClusterContext] = { implicit val ec = apiExecutionContext withKafkaManagerActor(KMClusterCommandRequest(clusterName, CMDeleteTopic(topic))) { @@ -391,7 +389,7 @@ class KafkaManager(akkaConfig: Config) hostname: String, log_path: String, config: Properties = new Properties - ): Future[ApiError \/ Unit] = + ): Future[ApiError \/ ClusterContext] = { implicit val ec = apiExecutionContext withKafkaManagerActor(KMClusterCommandRequest(clusterName, CMCreateLogkafka(hostname, log_path, config))) { @@ -405,7 +403,7 @@ class KafkaManager(akkaConfig: Config) hostname: String, log_path: String, config: Properties - ): Future[ApiError \/ Unit] = + ): Future[ApiError \/ ClusterContext] = { implicit val ec = apiExecutionContext withKafkaManagerActor( @@ -423,7 +421,7 @@ class KafkaManager(akkaConfig: Config) clusterName: String, hostname: String, log_path: String - ): Future[ApiError \/ Unit] = + ): Future[ApiError \/ ClusterContext] = { implicit val ec = apiExecutionContext withKafkaManagerActor(KMClusterCommandRequest(clusterName, CMDeleteLogkafka(hostname, log_path))) { @@ -438,6 +436,12 @@ class KafkaManager(akkaConfig: Config) result.result.get } } + + def getClusterContext(clusterName: String): Future[ApiError \/ ClusterContext] = { + tryWithKafkaManagerActor(KMClusterQueryRequest(clusterName, CMGetClusterContext))( + identity[ClusterContext] + ) + } def getClusterList: Future[ApiError \/ KMClusterList] = { tryWithKafkaManagerActor(KMGetAllClusters)(identity[KMClusterList]) @@ -466,7 +470,7 @@ class KafkaManager(akkaConfig: Config) tl <- errOrTl rap <- errOrRap } yield { - TopicListExtended(tl.list.map(t => (t, ti.get(t))).sortBy(_._1), tl.deleteSet, rap) + TopicListExtended(tl.list.map(t => (t, ti.get(t))).sortBy(_._1), tl.deleteSet, rap, tl.clusterContext) } } } @@ -504,7 +508,7 @@ class KafkaManager(akkaConfig: Config) bl.list, bm, if(bm.isEmpty) None else Option(bm.values.foldLeft(BrokerMetrics.DEFAULT)((acc, m) => acc + m)), - bl.clusterConfig + bl.clusterContext )) } }) diff --git a/app/kafka/manager/KafkaManagerActor.scala b/app/kafka/manager/KafkaManagerActor.scala index 2b1e4d6a1..56b4bec0d 100644 --- a/app/kafka/manager/KafkaManagerActor.scala +++ b/app/kafka/manager/KafkaManagerActor.scala @@ -11,6 +11,7 @@ import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, ThreadPoolExecutor} import akka.pattern._ import akka.actor.{Props, ActorPath} import ActorModel.CMShutdown +import kafka.manager.features.ClusterFeatures import org.apache.curator.framework.CuratorFramework import org.apache.curator.framework.recipes.cache.{PathChildrenCacheEvent, PathChildrenCacheListener, PathChildrenCache} import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode @@ -154,6 +155,7 @@ object ClusterConfig { } +case class ClusterContext(clusterFeatures: ClusterFeatures, config: ClusterConfig) case class ClusterConfig (name: String, curatorConfig : CuratorConfig, enabled: Boolean, version: KafkaVersion, jmxEnabled: Boolean, logkafkaEnabled: Boolean) object KafkaManagerActor { diff --git a/app/kafka/manager/KafkaStateActor.scala b/app/kafka/manager/KafkaStateActor.scala index 6abbf9d3a..90d09ce13 100644 --- a/app/kafka/manager/KafkaStateActor.scala +++ b/app/kafka/manager/KafkaStateActor.scala @@ -5,6 +5,7 @@ package kafka.manager +import kafka.manager.features.KMDeleteTopicFeature import kafka.manager.utils.zero81.{ReassignPartitionCommand, PreferredReplicaLeaderElectionCommand} import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode import org.apache.curator.framework.recipes.cache._ @@ -22,8 +23,7 @@ import ActorModel._ import kafka.manager.utils._ import scala.collection.JavaConverters._ class KafkaStateActor(curator: CuratorFramework, - deleteSupported: Boolean, - clusterConfig: ClusterConfig) extends BaseQueryCommandActor { + clusterContext: ClusterContext) extends BaseQueryCommandActor { // e.g. /brokers/topics/analytics_content/partitions/0/state private[this] val topicsTreeCache = new TreeCache(curator,ZkUtils.BrokerTopicsPath) @@ -175,7 +175,7 @@ class KafkaStateActor(curator: CuratorFramework, Option(topicsTreeCache.getCurrentData(statePath)).map(cd => (part, asString(cd.getData))) } config = getTopicConfigString(topic) - } yield TopicDescription(topic, description, Option(states),config, deleteSupported) + } yield TopicDescription(topic, description, Option(states),config) } override def processActorResponse(response: ActorResponse): Unit = { @@ -194,7 +194,7 @@ class KafkaStateActor(curator: CuratorFramework, request match { case KSGetTopics => val deleteSet: Set[String] = { - if(deleteSupported) { + if(clusterContext.clusterFeatures.features(KMDeleteTopicFeature)) { val deleteTopicsData: mutable.Buffer[ChildData] = deleteTopicsPathCache.getCurrentData.asScala deleteTopicsData.map { cd => nodeFromPath(cd.getPath) @@ -206,9 +206,9 @@ class KafkaStateActor(curator: CuratorFramework, withTopicsTreeCache { cache => cache.getCurrentChildren(ZkUtils.BrokerTopicsPath) }.fold { - sender ! TopicList(IndexedSeq.empty, deleteSet) + sender ! TopicList(IndexedSeq.empty, deleteSet, clusterContext) } { data: java.util.Map[String, ChildData] => - sender ! TopicList(data.asScala.map(kv => kv._1).toIndexedSeq, deleteSet) + sender ! TopicList(data.asScala.map(kv => kv._1).toIndexedSeq, deleteSet, clusterContext) } case KSGetTopicConfig(topic) => @@ -250,7 +250,7 @@ class KafkaStateActor(curator: CuratorFramework, }.collect { case scalaz.Success(bi) => bi }.toIndexedSeq.sortBy(_.id) - sender ! BrokerList(result, clusterConfig) + sender ! BrokerList(result, clusterContext) case KSGetPreferredLeaderElection => sender ! preferredLeaderElection @@ -269,7 +269,7 @@ class KafkaStateActor(curator: CuratorFramework, val s: Set[TopicAndPartition] = PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(json) preferredLeaderElection.fold { //nothing there, add as new - preferredLeaderElection = Some(PreferredReplicaElection(getDateTime(millis), s, None)) + preferredLeaderElection = Some(PreferredReplicaElection(getDateTime(millis), s, None, clusterContext)) } { existing => existing.endTime.fold { @@ -277,7 +277,7 @@ class KafkaStateActor(curator: CuratorFramework, preferredLeaderElection = Some(existing.copy(topicAndPartition = existing.topicAndPartition ++ s)) } { _ => //new op started - preferredLeaderElection = Some(PreferredReplicaElection(getDateTime(millis), s, None)) + preferredLeaderElection = Some(PreferredReplicaElection(getDateTime(millis), s, None, clusterContext)) } } } @@ -286,7 +286,7 @@ class KafkaStateActor(curator: CuratorFramework, val m : Map[TopicAndPartition, Seq[Int]] = ReassignPartitionCommand.parsePartitionReassignmentZkData(json) reassignPartitions.fold { //nothing there, add as new - reassignPartitions = Some(ReassignPartitions(getDateTime(millis),m, None)) + reassignPartitions = Some(ReassignPartitions(getDateTime(millis),m, None, clusterContext)) } { existing => existing.endTime.fold { @@ -294,7 +294,7 @@ class KafkaStateActor(curator: CuratorFramework, reassignPartitions = Some(existing.copy(partitionsToBeReassigned = existing.partitionsToBeReassigned ++ m)) } { _ => //new op started - reassignPartitions = Some(ReassignPartitions(getDateTime(millis),m, None)) + reassignPartitions = Some(ReassignPartitions(getDateTime(millis),m, None, clusterContext)) } } } diff --git a/app/kafka/manager/LogkafkaCommandActor.scala b/app/kafka/manager/LogkafkaCommandActor.scala index e76576f14..6dcf367e5 100644 --- a/app/kafka/manager/LogkafkaCommandActor.scala +++ b/app/kafka/manager/LogkafkaCommandActor.scala @@ -9,6 +9,7 @@ import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, ThreadPoolExecutor} import akka.pattern._ import akka.util.Timeout +import kafka.manager.features.KMDeleteTopicFeature import org.apache.curator.framework.CuratorFramework import kafka.manager.utils.{LogkafkaAdminUtils, ZkUtils} @@ -25,12 +26,12 @@ import ActorModel._ case class LogkafkaCommandActorConfig(curator: CuratorFramework, longRunningPoolConfig: LongRunningPoolConfig, askTimeoutMillis: Long = 400, - version: KafkaVersion) + clusterContext: ClusterContext) class LogkafkaCommandActor(logkafkaCommandActorConfig: LogkafkaCommandActorConfig) extends BaseCommandActor with LongRunningPoolActor { //private[this] val askTimeout: Timeout = logkafkaCommandActorConfig.askTimeoutMillis.milliseconds - private[this] val logkafkaAdminUtils = new LogkafkaAdminUtils(logkafkaCommandActorConfig.version) + private[this] val logkafkaAdminUtils = new LogkafkaAdminUtils(logkafkaCommandActorConfig.clusterContext.config.version) @scala.throws[Exception](classOf[Exception]) override def preStart() = { @@ -65,19 +66,18 @@ class LogkafkaCommandActor(logkafkaCommandActorConfig: LogkafkaCommandActorConfi implicit val ec = longRunningExecutionContext request match { case LKCDeleteLogkafka(hostname, log_path, logkafkaConfig) => - logkafkaCommandActorConfig.version match { - case Kafka_0_8_1_1 => - val result : LKCCommandResult = LKCCommandResult(Failure(new UnsupportedOperationException( - s"Delete logkafka not supported for kafka version ${logkafkaCommandActorConfig.version}"))) - sender ! result - case Kafka_0_8_2_0 | Kafka_0_8_2_1 => - longRunning { - Future { - LKCCommandResult(Try { - logkafkaAdminUtils.deleteLogkafka(logkafkaCommandActorConfig.curator, hostname, log_path, logkafkaConfig) - }) - } + if(logkafkaCommandActorConfig.clusterContext.clusterFeatures.features(KMDeleteTopicFeature)) { + longRunning { + Future { + LKCCommandResult(Try { + logkafkaAdminUtils.deleteLogkafka(logkafkaCommandActorConfig.curator, hostname, log_path, logkafkaConfig) + }) } + } + } else { + val result : LKCCommandResult = LKCCommandResult(Failure(new UnsupportedOperationException( + s"Delete logkafka not supported for kafka version ${logkafkaCommandActorConfig.clusterContext.config.version}"))) + sender ! result } case LKCCreateLogkafka(hostname, log_path, config, logkafkaConfig) => longRunning { diff --git a/app/kafka/manager/LogkafkaStateActor.scala b/app/kafka/manager/LogkafkaStateActor.scala index 133a5ecb1..ef4fc7ac2 100644 --- a/app/kafka/manager/LogkafkaStateActor.scala +++ b/app/kafka/manager/LogkafkaStateActor.scala @@ -5,24 +5,21 @@ package kafka.manager -import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode +import kafka.manager.features.KMLogKafkaFeature import org.apache.curator.framework.recipes.cache._ import org.apache.curator.framework.CuratorFramework import org.joda.time.{DateTimeZone, DateTime} import kafka.manager.utils.{LogkafkaZkUtils} -import scala.collection.mutable import scala.util.{Success, Failure, Try} /** * @author hiral */ import ActorModel._ -import kafka.manager.utils._ import scala.collection.JavaConverters._ class LogkafkaStateActor(curator: CuratorFramework, - deleteSupported: Boolean, - clusterConfig: ClusterConfig) extends BaseQueryCommandActor { + clusterContext: ClusterContext) extends BaseQueryCommandActor { private[this] val logkafkaConfigTreeCache = new TreeCache(curator,LogkafkaZkUtils.LogkafkaConfigPath) @@ -60,7 +57,7 @@ class LogkafkaStateActor(curator: CuratorFramework, @scala.throws[Exception](classOf[Exception]) override def preStart() = { - if (clusterConfig.logkafkaEnabled) { + if (clusterContext.clusterFeatures.features(KMLogKafkaFeature)) { log.info("Started actor %s".format(self.path)) log.info("Starting logkafka config tree cache...") logkafkaConfigTreeCache.start() diff --git a/app/kafka/manager/LogkafkaViewCacheActor.scala b/app/kafka/manager/LogkafkaViewCacheActor.scala index 146ba8006..a70d3cb8c 100644 --- a/app/kafka/manager/LogkafkaViewCacheActor.scala +++ b/app/kafka/manager/LogkafkaViewCacheActor.scala @@ -6,6 +6,7 @@ package kafka.manager import akka.actor.{ActorRef, Cancellable, ActorPath} +import kafka.manager.features.KMLogKafkaFeature import scala.collection.mutable import scala.concurrent.Future import scala.concurrent.duration._ @@ -16,7 +17,7 @@ import scala.util.Try */ import ActorModel._ case class LogkafkaViewCacheActorConfig(logkafkaStateActorPath: ActorPath, - clusterConfig: ClusterConfig, + clusterContext: ClusterContext, longRunningPoolConfig: LongRunningPoolConfig, updatePeriod: FiniteDuration = 10 seconds) class LogkafkaViewCacheActor(config: LogkafkaViewCacheActorConfig) extends LongRunningPoolActor { @@ -32,7 +33,7 @@ class LogkafkaViewCacheActor(config: LogkafkaViewCacheActorConfig) extends LongR private[this] var logkafkaClientsOption : Option[LogkafkaClients] = None override def preStart() = { - if (config.clusterConfig.logkafkaEnabled) { + if (config.clusterContext.clusterFeatures.features(KMLogKafkaFeature)) { log.info("Started actor %s".format(self.path)) log.info("Scheduling updater for %s".format(config.updatePeriod)) cancellable = Some( diff --git a/app/kafka/manager/features/KMFeature.scala b/app/kafka/manager/features/KMFeature.scala new file mode 100644 index 000000000..71923fe70 --- /dev/null +++ b/app/kafka/manager/features/KMFeature.scala @@ -0,0 +1,71 @@ +/** + * Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 + * See accompanying LICENSE file. + */ +package kafka.manager.features + +import kafka.manager.{Kafka_0_8_1_1, ClusterConfig} +import org.slf4j.LoggerFactory + +import scala.collection.mutable.ListBuffer +import scala.util.{Success, Failure, Try} + +/** + * Created by hiral on 8/22/15. + */ + +trait KMFeature + +sealed trait ClusterFeature extends KMFeature + +case object KMLogKafkaFeature extends ClusterFeature +case object KMDeleteTopicFeature extends ClusterFeature +case object KMJMXMetricsFeature extends ClusterFeature + +object ClusterFeature { + private lazy val log = LoggerFactory.getLogger(classOf[ClusterFeature]) + import scala.reflect.runtime.universe + + val runtimeMirror = universe.runtimeMirror(getClass.getClassLoader) + + def from(s: String) : Option[ClusterFeature] = { + Try { + val clazz = s"features.$s" + val module = runtimeMirror.staticModule(clazz) + val obj = runtimeMirror.reflectModule(module) + obj.instance match { + case f: ClusterFeature => + f + case _ => + throw new IllegalArgumentException(s"Unknown application feature $s") + } + } match { + case Failure(t) => + log.error(s"Unknown application feature $s") + None + case Success(f) => Option(f) + } + } + +} + +case class ClusterFeatures(features: Set[ClusterFeature]) + +object ClusterFeatures { + val default = ClusterFeatures(Set()) + + def from(clusterConfig: ClusterConfig) : ClusterFeatures = { + val buffer = new ListBuffer[ClusterFeature] + + if(clusterConfig.logkafkaEnabled) + buffer+=KMLogKafkaFeature + + if(clusterConfig.jmxEnabled) + buffer+=KMJMXMetricsFeature + + if(clusterConfig.version != Kafka_0_8_1_1) + buffer+=KMDeleteTopicFeature + + ClusterFeatures(buffer.toSet) + } +} diff --git a/app/kafka/manager/utils/AdminUtils.scala b/app/kafka/manager/utils/AdminUtils.scala index 15997c70a..f82dd1548 100644 --- a/app/kafka/manager/utils/AdminUtils.scala +++ b/app/kafka/manager/utils/AdminUtils.scala @@ -91,13 +91,6 @@ class AdminUtils(version: KafkaVersion) { (firstReplicaIndex + shift) % nBrokers } - def isDeleteSupported : Boolean = { - version match { - case Kafka_0_8_2_0 => true - case _ => false - } - } - def deleteTopic(curator: CuratorFramework, topic: String): Unit = { checkCondition(topicExists(curator, topic),TopicErrors.TopicDoesNotExist(topic)) ZkUtils.createPersistentPath(curator,ZkUtils.getDeleteTopicPath(topic)) diff --git a/app/kafka/manager/utils/zero81/ReassignPartitionCommand.scala b/app/kafka/manager/utils/zero81/ReassignPartitionCommand.scala index 6ded61713..2c9904f54 100644 --- a/app/kafka/manager/utils/zero81/ReassignPartitionCommand.scala +++ b/app/kafka/manager/utils/zero81/ReassignPartitionCommand.scala @@ -59,8 +59,7 @@ class ReassignPartitionCommand(adminUtils: AdminUtils) { currentTopicIdentity.numBrokers, currentTopicIdentity.configReadVersion, currentTopicIdentity.config, - currentTopicIdentity.deleteSupported, - currentTopicIdentity.clusterConfig + currentTopicIdentity.clusterContext ) } } diff --git a/app/models/navigation/Menus.scala b/app/models/navigation/Menus.scala index 1a7e7d2b7..c251fd755 100644 --- a/app/models/navigation/Menus.scala +++ b/app/models/navigation/Menus.scala @@ -5,34 +5,85 @@ package models.navigation +import features.{KMTopicManagerFeature, KMClusterManagerFeature, ApplicationFeatures} +import kafka.manager.features.{KMLogKafkaFeature, ClusterFeatures} + /** * @author hiral */ object Menus { import models.navigation.QuickRoutes._ - def clusterMenus(cluster: String) : IndexedSeq[Menu] = IndexedSeq( - Menu("Cluster",IndexedSeq( - "Summary".clusterRouteMenuItem(cluster), - "List".baseRouteMenuItem, - "Add Cluster".baseRouteMenuItem), - None), - "Brokers".clusterMenu(cluster), - Menu("Topic",IndexedSeq( - "List".clusterRouteMenuItem(cluster), - "Create".clusterRouteMenuItem(cluster)), - None), - "Preferred Replica Election".clusterMenu(cluster), - "Reassign Partitions".clusterMenu(cluster), - Menu("Logkafka",IndexedSeq( - "List Logkafka".clusterRouteMenuItem(cluster), - "Create Logkafka".clusterRouteMenuItem(cluster)), - None) - ) + + private[this] def clusterMenu(cluster: String, applicationFeatures: ApplicationFeatures) : Option[Menu] = { + val defaultItems = IndexedSeq("Summary".clusterRouteMenuItem(cluster), + "List".baseRouteMenuItem) + val items = { + if(applicationFeatures.features(KMClusterManagerFeature)) + defaultItems.+:("Add Cluster".baseRouteMenuItem) + else + defaultItems + } + + Option(Menu("Cluster", items, None)) + } + + private[this] def topicMenu(cluster: String, applicationFeatures: ApplicationFeatures) : Option[Menu] = { + val defaultItems = IndexedSeq("List".clusterRouteMenuItem(cluster)) + + val items = { + if(applicationFeatures.features(KMTopicManagerFeature)) + defaultItems.+:("Create".clusterRouteMenuItem(cluster)) + else + defaultItems + } - def indexMenu : IndexedSeq[Menu] = IndexedSeq( - Menu("Cluster",IndexedSeq( - "List".baseRouteMenuItem, - "Add Cluster".baseRouteMenuItem), - None) - ) + Option(Menu("Topic", items, None)) + } + + private[this] def brokersMenu(cluster: String, applicationFeatures: ApplicationFeatures) : Option[Menu] = { + Option("Brokers".clusterMenu(cluster)) + } + + private[this] def preferredReplicaElectionMenu(cluster: String, applicationFeatures: ApplicationFeatures) : Option[Menu] = { + Option("Preferred Replica Election".clusterMenu(cluster)) + } + + private[this] def reassignPartitionsMenu(cluster: String, applicationFeatures: ApplicationFeatures) : Option[Menu] = { + Option("Reassign Partitions".clusterMenu(cluster)) + } + + private[this] def logKafkaMenu(cluster: String, + applicationFeatures: ApplicationFeatures, + clusterFeatures: ClusterFeatures) : Option[Menu] = { + if (clusterFeatures.features(KMLogKafkaFeature)) { + Option(Menu("Logkafka", IndexedSeq( + "List Logkafka".clusterRouteMenuItem(cluster), + "Create Logkafka".clusterRouteMenuItem(cluster)), + None)) + } else None + } + + def clusterMenus(cluster: String) + (implicit applicationFeatures: ApplicationFeatures, + clusterFeatures: ClusterFeatures) : IndexedSeq[Menu] = { + IndexedSeq( + clusterMenu(cluster, applicationFeatures), + brokersMenu(cluster, applicationFeatures), + topicMenu(cluster, applicationFeatures), + preferredReplicaElectionMenu(cluster, applicationFeatures), + reassignPartitionsMenu(cluster, applicationFeatures), + logKafkaMenu(cluster, applicationFeatures, clusterFeatures) + ).flatten + } + + val indexMenu = { + val defaultItems = IndexedSeq("List".baseRouteMenuItem) + val items = { + if(ApplicationFeatures.features.features(KMClusterManagerFeature)) + defaultItems.+:("Add Cluster".baseRouteMenuItem) + else + defaultItems + } + IndexedSeq(Menu("Cluster", items, None)) + } } diff --git a/app/models/navigation/QuickRoutes.scala b/app/models/navigation/QuickRoutes.scala index da118aef0..c15bff833 100644 --- a/app/models/navigation/QuickRoutes.scala +++ b/app/models/navigation/QuickRoutes.scala @@ -20,8 +20,8 @@ object QuickRoutes { ) val clusterRoutes : Map[String, String => Call] = Map( "Update Cluster" -> controllers.routes.Cluster.updateCluster, - "Summary" -> controllers.routes.Application.cluster, - "Brokers" -> controllers.routes.Application.brokers, + "Summary" -> controllers.routes.Cluster.cluster, + "Brokers" -> controllers.routes.Cluster.brokers, "Topics" -> controllers.routes.Topic.topics, "List" -> controllers.routes.Topic.topics, "Create" -> controllers.routes.Topic.createTopic, diff --git a/app/views/broker/brokerList.scala.html b/app/views/broker/brokerList.scala.html index 73ddd1a16..e505b5214 100644 --- a/app/views/broker/brokerList.scala.html +++ b/app/views/broker/brokerList.scala.html @@ -4,14 +4,18 @@ *@ @import kafka.manager.ActorModel.BrokerIdentity @import scalaz.{\/} -@(cluster:String, errorOrBrokers: kafka.manager.ApiError \/ kafka.manager.BrokerListExtended) +@(cluster:String, errorOrBrokers: kafka.manager.ApiError \/ kafka.manager.BrokerListExtended +)(implicit af: features.ApplicationFeatures) @theMenu = { - @views.html.navigation.clusterMenu(cluster,"Brokers","",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Brokers","",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrBrokers.toOption.map(_.clusterContext.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) + } @renderBrokerMetrics(bl: kafka.manager.BrokerListExtended) = { - @if(bl.clusterConfig.jmxEnabled) { + @if(bl.clusterContext.clusterFeatures.features(kafka.manager.features.KMJMXMetricsFeature)) { @views.html.common.brokerMetrics(bl.combinedMetric) } else { } diff --git a/app/views/logkafka/logkafkaView.scala.html b/app/views/logkafka/logkafkaView.scala.html index bfd37bdc7..7bf41d51d 100644 --- a/app/views/logkafka/logkafkaView.scala.html +++ b/app/views/logkafka/logkafkaView.scala.html @@ -6,11 +6,13 @@ @(cluster:String, hostname: String, log_path: String, - errorOrLogkafkaIdentity: kafka.manager.ApiError \/ kafka.manager.ActorModel.LogkafkaIdentity -) + errorOrLogkafkaIdentity: kafka.manager.ApiError \/ (kafka.manager.ActorModel.LogkafkaIdentity, kafka.manager.ClusterContext) +)(implicit af: features.ApplicationFeatures) @theMenu = { - @views.html.navigation.clusterMenu(cluster,"Logkafka","",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Logkafka","",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrLogkafkaIdentity.toOption.map(_._2.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @main( @@ -22,7 +24,7 @@

@hostname
@log_path

- @errorOrLogkafkaIdentity.fold(views.html.errors.onApiError(_),views.html.logkafka.logkafkaViewContent(cluster,hostname,log_path,_)) + @errorOrLogkafkaIdentity.fold(views.html.errors.onApiError(_), t => views.html.logkafka.logkafkaViewContent(cluster,hostname,log_path,t._1)) } diff --git a/app/views/logkafka/logkafkaViewContent.scala.html b/app/views/logkafka/logkafkaViewContent.scala.html index 241ba8a3d..1cb8d66bd 100644 --- a/app/views/logkafka/logkafkaViewContent.scala.html +++ b/app/views/logkafka/logkafkaViewContent.scala.html @@ -49,7 +49,6 @@ - @if(logkafkaIdentity.deleteSupported) { - } @features.app(features.KMReassignPartitionsFeature) { } From 1948c931ca4ef46d67dc9df0909512ea7f7a373d Mon Sep 17 00:00:00 2001 From: patelh Date: Tue, 25 Aug 2015 21:58:14 -0700 Subject: [PATCH 13/17] Add references --- app/views/cluster/addCluster.scala.html | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/app/views/cluster/addCluster.scala.html b/app/views/cluster/addCluster.scala.html index 23b2129da..b84acea39 100644 --- a/app/views/cluster/addCluster.scala.html +++ b/app/views/cluster/addCluster.scala.html @@ -33,6 +33,13 @@ @drawForm(addClusterForm) +
+
References
+
    +
  1. Kafka Quickstart
  2. +
  3. LogKafka
  4. +
+
} From 3cdde7587d100b1203561655b25add6ab609ca6f Mon Sep 17 00:00:00 2001 From: patelh Date: Tue, 25 Aug 2015 22:16:35 -0700 Subject: [PATCH 14/17] update readme --- README.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/README.md b/README.md index 52cd410ea..ef3a5a405 100644 --- a/README.md +++ b/README.md @@ -73,6 +73,15 @@ Alternatively, use the environment variable `ZK_HOSTS` if you don't want to hard ZK_HOSTS="my.zookeeper.host.com:2181" +You can optionally enable/disable the following functionality by modifying the default list in application.conf : + + application.features=["KMClusterManagerFeature","KMTopicManagerFeature","KMPreferredReplicaElectionFeature","KMReassignPartitionsFeature"] + + - KMClusterManagerFeature - allows adding, updating, deleting cluster from Kafka Manager + - KMTopicManagerFeature - allows adding, updating, deleting topic from a Kafka cluster + - KMPreferredReplicaElectionFeature - allows running of preferred replica election for a Kafka cluster + - KMReassignPartitionsFeature - allows generating partition assignments and reassigning partitions + Deployment ---------- From 347862ce03899d31cc70d83a71338bedcce51cac Mon Sep 17 00:00:00 2001 From: patelh Date: Tue, 25 Aug 2015 22:22:55 -0700 Subject: [PATCH 15/17] Add config references --- app/views/cluster/addCluster.scala.html | 8 +------- app/views/cluster/configReferences.scala.html | 13 +++++++++++++ app/views/cluster/updateCluster.scala.html | 1 + 3 files changed, 15 insertions(+), 7 deletions(-) create mode 100644 app/views/cluster/configReferences.scala.html diff --git a/app/views/cluster/addCluster.scala.html b/app/views/cluster/addCluster.scala.html index b84acea39..f6acd2232 100644 --- a/app/views/cluster/addCluster.scala.html +++ b/app/views/cluster/addCluster.scala.html @@ -33,13 +33,7 @@ @drawForm(addClusterForm) -
-
References
-
    -
  1. Kafka Quickstart
  2. -
  3. LogKafka
  4. -
-
+ @configReferences } diff --git a/app/views/cluster/configReferences.scala.html b/app/views/cluster/configReferences.scala.html new file mode 100644 index 000000000..6caa9ebcb --- /dev/null +++ b/app/views/cluster/configReferences.scala.html @@ -0,0 +1,13 @@ +@* +* Copyright 2015 Yahoo Inc. Licensed under the Apache License, Version 2.0 +* See accompanying LICENSE file. +*@ + +
+
References
+
    +
  1. Kafka Quickstart
  2. +
  3. LogKafka
  4. +
+
+ diff --git a/app/views/cluster/updateCluster.scala.html b/app/views/cluster/updateCluster.scala.html index f67ff483a..ea21b201f 100644 --- a/app/views/cluster/updateCluster.scala.html +++ b/app/views/cluster/updateCluster.scala.html @@ -39,6 +39,7 @@ @errorOrForm.fold(views.html.errors.onApiError(_), drawForm(_)) + @configReferences } From 7add672f25907303bce1bb3e61577f3d5127e09b Mon Sep 17 00:00:00 2001 From: patelh Date: Tue, 25 Aug 2015 22:31:19 -0700 Subject: [PATCH 16/17] Update view --- app/views/cluster/addCluster.scala.html | 2 +- app/views/cluster/updateCluster.scala.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/app/views/cluster/addCluster.scala.html b/app/views/cluster/addCluster.scala.html index f6acd2232..e243c7564 100644 --- a/app/views/cluster/addCluster.scala.html +++ b/app/views/cluster/addCluster.scala.html @@ -33,7 +33,7 @@ @drawForm(addClusterForm) - @configReferences + @configReferences() } diff --git a/app/views/cluster/updateCluster.scala.html b/app/views/cluster/updateCluster.scala.html index ea21b201f..ae102a6aa 100644 --- a/app/views/cluster/updateCluster.scala.html +++ b/app/views/cluster/updateCluster.scala.html @@ -39,7 +39,7 @@ @errorOrForm.fold(views.html.errors.onApiError(_), drawForm(_)) - @configReferences + @configReferences() } From acacac283129080940ca3a02f7160318c7a01421 Mon Sep 17 00:00:00 2001 From: patelh Date: Wed, 26 Aug 2015 08:37:37 -0700 Subject: [PATCH 17/17] Increment version --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 2a6d7e7a5..734f949ff 100644 --- a/build.sbt +++ b/build.sbt @@ -5,7 +5,7 @@ name := """kafka-manager""" /* For packaging purposes, -SNAPSHOT MUST contain a digit */ -version := "1.2.7" +version := "1.2.8" scalaVersion := "2.11.5"
@b3.form(routes.Logkafka.handleDeleteLogkafka(cluster, hostname, log_path)) {
@@ -59,7 +58,6 @@
}
diff --git a/app/views/logkafka/updateConfig.scala.html b/app/views/logkafka/updateConfig.scala.html index 4a75a6f9a..4f6682513 100644 --- a/app/views/logkafka/updateConfig.scala.html +++ b/app/views/logkafka/updateConfig.scala.html @@ -3,14 +3,18 @@ * See accompanying LICENSE file. *@ @import scalaz.{\/} -@(cluster: String, hostname: String, log_path: String, errorOrForm: kafka.manager.ApiError \/ Form[models.form.UpdateLogkafkaConfig])(implicit request: RequestHeader) +@(cluster: String, hostname: String, log_path: String, +errorOrForm: kafka.manager.ApiError \/ (Form[models.form.UpdateLogkafkaConfig], kafka.manager.ClusterContext) +)(implicit af: features.ApplicationFeatures) @import helper._ @import b3.vertical.fieldConstructor @import controllers.routes @theMenu = { - @views.html.navigation.clusterMenu(cluster,"Logkafka","Update Config",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Logkafka","Update Config",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrForm.toOption.map(_._2.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @renderForm(updateLogkafkaConfigForm: Form[models.form.UpdateLogkafkaConfig]) = { @@ -45,7 +49,7 @@

Update Config

- @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) + @errorOrForm.fold( views.html.errors.onApiError(_), t => renderForm(t._1))
diff --git a/app/views/preferredReplicaElection.scala.html b/app/views/preferredReplicaElection.scala.html index c855d2839..922222920 100644 --- a/app/views/preferredReplicaElection.scala.html +++ b/app/views/preferredReplicaElection.scala.html @@ -7,10 +7,12 @@ @(cluster:String, errorOrStatus: kafka.manager.ApiError \/ Option[kafka.manager.ActorModel.PreferredReplicaElection], operationForm: Form[models.form.PreferredReplicaElectionOperation] -) +)(implicit af: features.ApplicationFeatures) @theMenu = { -@views.html.navigation.clusterMenu(cluster,"Preferred Replica Election","",models.navigation.Menus.clusterMenus(cluster)) +@views.html.navigation.clusterMenu(cluster,"Preferred Replica Election","",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrStatus.toOption.flatten.map(_.clusterContext.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @renderView(c: String, viewOption: Option[kafka.manager.ActorModel.PreferredReplicaElection]) = { @@ -66,14 +68,16 @@

Preferred Replica Election

-
- @b3.form(routes.PreferredReplicaElection.handleRunElection(cluster)) { -
- - @b3.submit('class -> "btn btn-primary"){ Run Preferred Replica Election } -
- } -
+ @features.app(features.KMPreferredReplicaElectionFeature) { +
+ @b3.form(routes.PreferredReplicaElection.handleRunElection(cluster)) { +
+ + @b3.submit('class -> "btn btn-primary"){ Run Preferred Replica Election } +
+ } +
+ } @errorOrStatus.fold[Html](views.html.errors.onApiError(_), renderView(cluster, _))
diff --git a/app/views/reassignPartitions.scala.html b/app/views/reassignPartitions.scala.html index 666fd0e31..c18a37d1c 100644 --- a/app/views/reassignPartitions.scala.html +++ b/app/views/reassignPartitions.scala.html @@ -4,10 +4,13 @@ *@ @import scalaz.{\/} @import b3.vertical.fieldConstructor -@(cluster:String, errorOrStatus: kafka.manager.ApiError \/ Option[kafka.manager.ActorModel.ReassignPartitions]) +@(cluster:String, errorOrStatus: kafka.manager.ApiError \/ Option[kafka.manager.ActorModel.ReassignPartitions] +)(implicit af: features.ApplicationFeatures) @theMenu = { -@views.html.navigation.clusterMenu(cluster,"Reassign Partitions","",models.navigation.Menus.clusterMenus(cluster)) +@views.html.navigation.clusterMenu(cluster,"Reassign Partitions","",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrStatus.toOption.flatten.map(_.clusterContext.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @renderView(c: String, viewOption: Option[kafka.manager.ActorModel.ReassignPartitions]) = { diff --git a/app/views/topic/addPartitions.scala.html b/app/views/topic/addPartitions.scala.html index 46343b1db..895c7dc27 100644 --- a/app/views/topic/addPartitions.scala.html +++ b/app/views/topic/addPartitions.scala.html @@ -3,14 +3,17 @@ * See accompanying LICENSE file. *@ @import scalaz.{\/} -@(cluster: String, topic: String, errorOrForm: kafka.manager.ApiError \/ Form[models.form.AddTopicPartitions])(implicit request: RequestHeader) +@(cluster: String, topic: String, errorOrForm: kafka.manager.ApiError \/ (Form[models.form.AddTopicPartitions], kafka.manager.ClusterContext) +)(implicit af: features.ApplicationFeatures) @import helper._ @import b3.vertical.fieldConstructor @import controllers.routes @theMenu = { - @views.html.navigation.clusterMenu(cluster,"Topic","Add Partitions",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Topic","Add Partitions",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrForm.toOption.map(_._2.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @renderForm(addTopicPartitionsForm: Form[models.form.AddTopicPartitions]) = { @@ -55,7 +58,7 @@

Add Partitions

- @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) + @errorOrForm.fold( views.html.errors.onApiError(_), t => renderForm(t._1))
diff --git a/app/views/topic/addPartitionsToMultipleTopics.scala.html b/app/views/topic/addPartitionsToMultipleTopics.scala.html index 52503c3b1..66b369e34 100644 --- a/app/views/topic/addPartitionsToMultipleTopics.scala.html +++ b/app/views/topic/addPartitionsToMultipleTopics.scala.html @@ -3,14 +3,17 @@ * See accompanying LICENSE file. *@ @import scalaz.{\/} -@(cluster: String, errorOrForm: kafka.manager.ApiError \/ Form[models.form.AddMultipleTopicsPartitions])(implicit request: RequestHeader) +@(cluster: String, errorOrForm: kafka.manager.ApiError \/ (Form[models.form.AddMultipleTopicsPartitions], kafka.manager.ClusterContext) +)(implicit af: features.ApplicationFeatures) @import helper._ @import b3.vertical.fieldConstructor @import controllers.routes @theMenu = { -@views.html.navigation.clusterMenu(cluster,"Topics","Add Partitions to Topics",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Topics","Add Partitions to Topics",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrForm.toOption.map(_._2.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @checkboxWithLink(field: play.api.data.Field, topic: String) = { @@ -81,7 +84,7 @@

Add Partitions

- @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) + @errorOrForm.fold( views.html.errors.onApiError(_), t => renderForm(t._1))
diff --git a/app/views/topic/confirmAssignment.scala.html b/app/views/topic/confirmAssignment.scala.html index 17fc42e30..129686802 100644 --- a/app/views/topic/confirmAssignment.scala.html +++ b/app/views/topic/confirmAssignment.scala.html @@ -6,11 +6,13 @@ @import b3.vertical.fieldConstructor @(cluster: String, topic: String, - errorOrForm: kafka.manager.ApiError \/ Form[models.form.GenerateAssignment] -) + errorOrForm: kafka.manager.ApiError \/ (Form[models.form.GenerateAssignment], kafka.manager.ClusterContext) +)(implicit af: features.ApplicationFeatures) @theMenu = { -@views.html.navigation.clusterMenu(cluster,"Topic","Confirm Assignment",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Topic","Confirm Assignment",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrForm.toOption.map(_._2.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @renderForm(confirmForm: Form[models.form.GenerateAssignment]) = { @@ -51,7 +53,7 @@

Choose brokers to reassign topic @topic to:

Confirm Assignment

- @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) + @errorOrForm.fold( views.html.errors.onApiError(_), t => renderForm(t._1))
diff --git a/app/views/topic/confirmMultipleAssignments.scala.html b/app/views/topic/confirmMultipleAssignments.scala.html index 0c9e68df5..85b11ee05 100644 --- a/app/views/topic/confirmMultipleAssignments.scala.html +++ b/app/views/topic/confirmMultipleAssignments.scala.html @@ -5,11 +5,13 @@ @import scalaz.{\/} @import b3.vertical.fieldConstructor @(cluster: String, - errorOrForm: kafka.manager.ApiError \/ Form[models.form.GenerateMultipleAssignments] -) + errorOrForm: kafka.manager.ApiError \/ (Form[models.form.GenerateMultipleAssignments], kafka.manager.ClusterContext) +)(implicit af: features.ApplicationFeatures) @theMenu = { -@views.html.navigation.clusterMenu(cluster,"Topic","Confirm Assignments",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Topic","Confirm Assignments",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrForm.toOption.map(_._2.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @checkboxWithLink(field: play.api.data.Field, topic: String) = { @@ -73,7 +75,7 @@

Choose topics and brokers for reassignments:

Confirm Assignments

- @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) + @errorOrForm.fold( views.html.errors.onApiError(_), t => renderForm(t._1))
diff --git a/app/views/topic/createTopic.scala.html b/app/views/topic/createTopic.scala.html index 1fa9fa5d6..5a3d694e9 100644 --- a/app/views/topic/createTopic.scala.html +++ b/app/views/topic/createTopic.scala.html @@ -3,14 +3,17 @@ * See accompanying LICENSE file. *@ @import scalaz.{\/} -@(cluster: String, errorOrForm: kafka.manager.ApiError \/ Form[models.form.CreateTopic])(implicit request: RequestHeader) +@(cluster: String, errorOrForm: kafka.manager.ApiError \/ (Form[models.form.CreateTopic], kafka.manager.ClusterContext) +)(implicit af: features.ApplicationFeatures) @import helper._ @import b3.vertical.fieldConstructor @import controllers.routes @theMenu = { - @views.html.navigation.clusterMenu(cluster,"Topic","Create",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Topic","Create",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrForm.toOption.map(_._2.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @renderForm(createTopicForm: Form[models.form.CreateTopic]) = { @@ -46,7 +49,7 @@

Create Topic

- @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) + @errorOrForm.fold( views.html.errors.onApiError(_), t => renderForm(t._1))
} diff --git a/app/views/topic/manualMultipleAssignments.scala.html b/app/views/topic/manualAssignments.scala.html similarity index 96% rename from app/views/topic/manualMultipleAssignments.scala.html rename to app/views/topic/manualAssignments.scala.html index 2843670a8..19220069c 100644 --- a/app/views/topic/manualMultipleAssignments.scala.html +++ b/app/views/topic/manualAssignments.scala.html @@ -12,11 +12,12 @@ @import models.form._ @(cluster: String, + topic: String, assignForm: Form[List[(String, List[(Int,List[Int])])]], brokers: BrokerListExtended, brokersViews: Seq[BVView], formErrors: Seq[FormError] -) +)(implicit af: features.ApplicationFeatures, cf: kafka.manager.features.ClusterFeatures) @theMenu = { @views.html.navigation.clusterMenu(cluster,"Topic","Manual Partition Assignments",models.navigation.Menus.clusterMenus(cluster)) @@ -125,7 +126,7 @@

Broker @idx

@if(true) { - @helper.form(action = routes.ReassignPartitions.handleManualAssignment(cluster)) { + @helper.form(action = routes.ReassignPartitions.handleManualAssignment(cluster, topic)) { @partitionsAssignmentsPane } } else { diff --git a/app/views/topic/runMultipleAssignments.scala.html b/app/views/topic/runMultipleAssignments.scala.html index ec7839690..23696c9ca 100644 --- a/app/views/topic/runMultipleAssignments.scala.html +++ b/app/views/topic/runMultipleAssignments.scala.html @@ -5,11 +5,13 @@ @import scalaz.{\/} @import b3.vertical.fieldConstructor @(cluster: String, - errorOrForm: kafka.manager.ApiError \/ Form[models.form.RunMultipleAssignments] -) + errorOrForm: kafka.manager.ApiError \/ (Form[models.form.RunMultipleAssignments], kafka.manager.ClusterContext) +)(implicit af: features.ApplicationFeatures) @theMenu = { -@views.html.navigation.clusterMenu(cluster,"Topic","Confirm Assignments",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Topic","Confirm Assignments",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrForm.toOption.map(_._2.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @checkboxWithLink(field: play.api.data.Field, topic: String) = { @@ -60,7 +62,7 @@

Choose topics to reassign:

Run Assignments

- @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) + @errorOrForm.fold( views.html.errors.onApiError(_), t => renderForm(t._1))
diff --git a/app/views/topic/topicList.scala.html b/app/views/topic/topicList.scala.html index 002d5d239..9aa1e150e 100644 --- a/app/views/topic/topicList.scala.html +++ b/app/views/topic/topicList.scala.html @@ -3,10 +3,13 @@ * See accompanying LICENSE file. *@ @import scalaz.{\/} -@(cluster:String, errorOrTopics: kafka.manager.ApiError \/ kafka.manager.TopicListExtended) +@(cluster:String, errorOrTopics: kafka.manager.ApiError \/ kafka.manager.TopicListExtended +)(implicit af: features.ApplicationFeatures) @theMenu = { - @views.html.navigation.clusterMenu(cluster,"Topic","List",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Topic","List",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrTopics.toOption.map(_.clusterContext.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @topicScripts = { @@ -23,19 +26,19 @@ breadcrumbs=views.html.navigation.breadCrumbs(models.navigation.BreadCrumbs.withViewAndCluster("Topics",cluster)), scripts=topicScripts) {
+ @features.app(features.KMTopicManagerFeature) {

Operations

+ @features.app(features.KMReassignPartitionsFeature) { - + } @@ -43,6 +46,7 @@
Generate Partition Assignments - Manually Set Partition Assignments - Run Partition Assignments Add Partitions
+ }
diff --git a/app/views/topic/topicView.scala.html b/app/views/topic/topicView.scala.html index be356919b..4ec9be5ed 100644 --- a/app/views/topic/topicView.scala.html +++ b/app/views/topic/topicView.scala.html @@ -6,10 +6,12 @@ @(cluster:String, topic: String, errorOrTopicIdentity: kafka.manager.ApiError \/ kafka.manager.ActorModel.TopicIdentity -) +)(implicit af: features.ApplicationFeatures) @theMenu = { - @views.html.navigation.clusterMenu(cluster,"Topic","",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Topic","",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrTopicIdentity.toOption.map(_.clusterContext.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @main( diff --git a/app/views/topic/topicViewContent.scala.html b/app/views/topic/topicViewContent.scala.html index 0f6bf1c40..eb6878496 100644 --- a/app/views/topic/topicViewContent.scala.html +++ b/app/views/topic/topicViewContent.scala.html @@ -3,7 +3,8 @@ * See accompanying LICENSE file. *@ @import b3.vertical.fieldConstructor -@(cluster:String,topic: String, topicIdentity: kafka.manager.ActorModel.TopicIdentity) +@(cluster:String,topic: String, topicIdentity: kafka.manager.ActorModel.TopicIdentity +)(implicit af: features.ApplicationFeatures) @getUnderReplicatedLevel(percentage: Int) = { @percentage match { @@ -65,7 +66,7 @@ } @renderTopicMetrics = { - @if(topicIdentity.clusterConfig.jmxEnabled) { + @if(topicIdentity.clusterContext.clusterFeatures.features(kafka.manager.features.KMJMXMetricsFeature)) { @views.html.common.brokerMetrics(topicIdentity.metrics) } else {
+ @features.app(features.KMTopicManagerFeature) {

Operations

- @if(topicIdentity.deleteSupported) { + @if(topicIdentity.clusterContext.clusterFeatures.features(kafka.manager.features.KMDeleteTopicFeature)) { } + @features.app(features.KMReassignPartitionsFeature) { + + }
@b3.form(routes.Topic.handleDeleteTopic(cluster, topic)) {
@@ -158,6 +160,7 @@ }
@b3.form(routes.ReassignPartitions.handleOperation(cluster,topic)) {
@@ -170,6 +173,10 @@
Generate Partition Assignments + Manually Set Partition Assignments +
@@ -182,6 +189,7 @@
+ }

Partitions by Broker

@@ -191,7 +199,7 @@ @for(btp <- topicIdentity.partitionsByBroker) { - + diff --git a/app/views/topic/updateConfig.scala.html b/app/views/topic/updateConfig.scala.html index 165d611e9..b653db69a 100644 --- a/app/views/topic/updateConfig.scala.html +++ b/app/views/topic/updateConfig.scala.html @@ -3,14 +3,17 @@ * See accompanying LICENSE file. *@ @import scalaz.{\/} -@(cluster: String, topic: String, errorOrForm: kafka.manager.ApiError \/ Form[models.form.UpdateTopicConfig])(implicit request: RequestHeader) +@(cluster: String, topic: String, errorOrForm: kafka.manager.ApiError \/ (Form[models.form.UpdateTopicConfig], kafka.manager.ClusterContext) +)(implicit af: features.ApplicationFeatures) @import helper._ @import b3.vertical.fieldConstructor @import controllers.routes @theMenu = { - @views.html.navigation.clusterMenu(cluster,"Topic","Update Config",models.navigation.Menus.clusterMenus(cluster)) + @views.html.navigation.clusterMenu(cluster,"Topic","Update Config",models.navigation.Menus.clusterMenus(cluster)( + af, + errorOrForm.toOption.map(_._2.clusterFeatures).getOrElse(kafka.manager.features.ClusterFeatures.default))) } @renderForm(updateTopicConfigForm: Form[models.form.UpdateTopicConfig]) = { @@ -45,7 +48,7 @@

Update Config

- @errorOrForm.fold( views.html.errors.onApiError(_), renderForm(_)) + @errorOrForm.fold( views.html.errors.onApiError(_), t => renderForm(t._1))
diff --git a/conf/application.conf b/conf/application.conf index 8ed2809e4..982df57c7 100644 --- a/conf/application.conf +++ b/conf/application.conf @@ -64,4 +64,5 @@ kafka-manager.zkhosts="kafka-manager-zookeeper:2181" kafka-manager.zkhosts=${?ZK_HOSTS} pinned-dispatcher.type="PinnedDispatcher" pinned-dispatcher.executor="thread-pool-executor" +application.features=["KMClusterManagerFeature","KMTopicManagerFeature","KMPreferredReplicaElectionFeature","KMReassignPartitionsFeature"] diff --git a/conf/routes b/conf/routes index 23e1cdc9a..d64347774 100644 --- a/conf/routes +++ b/conf/routes @@ -7,15 +7,15 @@ # Home page GET / controllers.Application.index -GET /clusters/:c controllers.Application.cluster(c:String) +GET /clusters/:c controllers.Cluster.cluster(c:String) GET /clusters/:c/topics controllers.Topic.topics(c:String) GET /clusters/:c/topics/addPartitions controllers.Topic.addPartitionsToMultipleTopics(c:String) POST /clusters/:c/topics/addPartitions controllers.Topic.handleAddPartitionsToMultipleTopics(c:String) GET /clusters/:c/topics/:t controllers.Topic.topic(c:String, t:String) GET /clusters/:c/logkafkas controllers.Logkafka.logkafkas(c:String) GET /clusters/:c/logkafkas/:h/:l controllers.Logkafka.logkafka(c:String, h:String, l:String) -GET /clusters/:c/brokers controllers.Application.brokers(c: String) -GET /clusters/:c/brokers/:b controllers.Application.broker(c: String, b:Int) +GET /clusters/:c/brokers controllers.Cluster.brokers(c: String) +GET /clusters/:c/brokers/:b controllers.Cluster.broker(c: String, b:Int) GET /clusters/:c/leader controllers.PreferredReplicaElection.preferredReplicaElection(c:String) POST /clusters/:c/leader controllers.PreferredReplicaElection.handleRunElection(c:String) GET /clusters/:c/assignment controllers.ReassignPartitions.reassignPartitions(c:String) @@ -24,8 +24,8 @@ GET /clusters/:c/assignment/confirm controllers.ReassignPa POST /clusters/:c/assignment/generate controllers.ReassignPartitions.handleGenerateAssignment(c:String,t:String) GET /clusters/:c/assignments/confirm controllers.ReassignPartitions.confirmMultipleAssignments(c:String) POST /clusters/:c/assignments/generate controllers.ReassignPartitions.handleGenerateMultipleAssignments(c:String) -GET /clusters/:c/assignments/manual controllers.ReassignPartitions.manualMultipleAssignments(c:String) -POST /clusters/:c/assignments/manual controllers.ReassignPartitions.handleManualAssignment(c:String) +#GET /clusters/:c/assignments/manual controllers.ReassignPartitions.manualMultipleAssignments(c:String) +#POST /clusters/:c/assignments/manual controllers.ReassignPartitions.handleManualAssignment(c:String) GET /clusters/:c/assignments/run controllers.ReassignPartitions.runMultipleAssignments(c:String) POST /clusters/:c/assignments/run controllers.ReassignPartitions.handleRunMultipleAssignments(c:String) GET /addCluster controllers.Cluster.addCluster @@ -39,11 +39,13 @@ GET /clusters/:c/topics/:t/addPartitions controllers.Topic.addP POST /clusters/:c/topics/:t/addPartitions controllers.Topic.handleAddPartitions(c:String,t:String) GET /clusters/:c/topics/:t/updateConfig controllers.Topic.updateConfig(c:String,t:String) POST /clusters/:c/topics/:t/updateConfig controllers.Topic.handleUpdateConfig(c:String,t: String) +GET /clusters/:c/topics/:t/assignments/manual controllers.ReassignPartitions.manualAssignments(c:String, t:String) +POST /clusters/:c/topics/:t/assignments/manual controllers.ReassignPartitions.handleManualAssignment(c:String, t: String) GET /clusters/:c/createLogkafka controllers.Logkafka.createLogkafka(c:String) POST /clusters/:c/logkafkas/create controllers.Logkafka.handleCreateLogkafka(c:String) POST /clusters/:c/logkafkas/delete controllers.Logkafka.handleDeleteLogkafka(c:String, h:String, l:String) -GET /clusters/:c/logkafkas/:h/:l/updateConfig controllers.Logkafka.updateConfig(c:String, h:String, l:String) -POST /clusters/:c/logkafkas/:h/:l/updateConfig controllers.Logkafka.handleUpdateConfig(c:String, h:String, l:String) +GET /clusters/:c/logkafkas/:h/:l/updateConfig controllers.Logkafka.updateConfig(c:String, h:String, l:String) +POST /clusters/:c/logkafkas/:h/:l/updateConfig controllers.Logkafka.handleUpdateConfig(c:String, h:String, l:String) GET /api/status/:c/availableBrokers controllers.api.KafkaHealthCheck.availableBrokers(c:String) GET /api/status/:c/:t/underReplicatedPartitions controllers.api.KafkaHealthCheck.underReplicatedPartitions(c:String,t:String) GET /api/status/:c/:t/unavailablePartitions controllers.api.KafkaHealthCheck.unavailablePartitions(c:String,t:String) diff --git a/test/kafka/manager/TestBrokerViewCacheActor.scala b/test/kafka/manager/TestBrokerViewCacheActor.scala index 083f2016d..0d1c3dc44 100644 --- a/test/kafka/manager/TestBrokerViewCacheActor.scala +++ b/test/kafka/manager/TestBrokerViewCacheActor.scala @@ -10,6 +10,7 @@ import akka.actor.{ActorRef, ActorSystem, Kill, Props} import akka.pattern._ import akka.util.Timeout import com.typesafe.config.{Config, ConfigFactory} +import kafka.manager.features.ClusterFeatures import kafka.manager.utils.KafkaServerInTest import ActorModel._ import kafka.test.SeededBroker @@ -35,15 +36,17 @@ class TestBrokerViewCacheActor extends KafkaServerInTest { private[this] var brokerViewCacheActor : Option[ActorRef] = None private[this] val defaultClusterConfig = ClusterConfig("test","0.8.2.0","localhost:2818",100,false) + private[this] val defaultClusterContext = ClusterContext(ClusterFeatures.from(defaultClusterConfig), defaultClusterConfig) override protected def beforeAll(): Unit = { super.beforeAll() val clusterConfig = ClusterConfig("dev","0.8.2.0",kafkaServerZkPath, jmxEnabled = false) - val props = Props(classOf[KafkaStateActor],sharedCurator, true, defaultClusterConfig) + val clusterContext = ClusterContext(ClusterFeatures.from(clusterConfig), clusterConfig) + val props = Props(classOf[KafkaStateActor],sharedCurator, defaultClusterContext) kafkaStateActor = Some(system.actorOf(props.withDispatcher("pinned-dispatcher"),"ksa")) - val bvConfig = BrokerViewCacheActorConfig(kafkaStateActor.get.path, clusterConfig, LongRunningPoolConfig(2,100), FiniteDuration(10, SECONDS)) + val bvConfig = BrokerViewCacheActorConfig(kafkaStateActor.get.path, clusterContext, LongRunningPoolConfig(2,100), FiniteDuration(10, SECONDS)) val bvcProps = Props(classOf[BrokerViewCacheActor],bvConfig) brokerViewCacheActor = Some(system.actorOf(bvcProps,"broker-view")) diff --git a/test/kafka/manager/TestClusterManagerActor.scala b/test/kafka/manager/TestClusterManagerActor.scala index 4c021fa72..d1a0b527f 100644 --- a/test/kafka/manager/TestClusterManagerActor.scala +++ b/test/kafka/manager/TestClusterManagerActor.scala @@ -118,7 +118,7 @@ class TestClusterManagerActor extends CuratorAwareTest { descriptions foreach println withClusterManagerActor(KSGetBrokers) { brokerList: BrokerList => - val topicIdentityList : IndexedSeq[TopicIdentity] = descriptions.flatten.map(td => TopicIdentity.from(brokerList,td, None, brokerList.clusterConfig)) + val topicIdentityList : IndexedSeq[TopicIdentity] = descriptions.flatten.map(td => TopicIdentity.from(brokerList,td, None, brokerList.clusterContext)) topicIdentityList foreach println } } diff --git a/test/kafka/manager/TestKafkaManager.scala b/test/kafka/manager/TestKafkaManager.scala index f7fd1f39d..514388d0c 100644 --- a/test/kafka/manager/TestKafkaManager.scala +++ b/test/kafka/manager/TestKafkaManager.scala @@ -7,6 +7,7 @@ package kafka.manager import java.util.Properties import com.typesafe.config.{Config, ConfigFactory} +import kafka.manager.features.KMDeleteTopicFeature import kafka.manager.utils.CuratorAwareTest import ActorModel.TopicList import kafka.test.SeededBroker @@ -131,6 +132,13 @@ class TestKafkaManager extends CuratorAwareTest { val result = Await.result(future,duration) assert(result.isRight === true) } + + test("get cluster context") { + val future = kafkaManager.getClusterContext("dev") + val result = Await.result(future,duration) + assert(result.isRight === true, s"Failed : ${result}") + assert(result.toOption.get.clusterFeatures.features(KMDeleteTopicFeature)) + } test("run preferred leader election") { val topicList = getTopicList() diff --git a/test/kafka/manager/TestKafkaStateActor.scala b/test/kafka/manager/TestKafkaStateActor.scala index f6191b555..a2b014ef1 100644 --- a/test/kafka/manager/TestKafkaStateActor.scala +++ b/test/kafka/manager/TestKafkaStateActor.scala @@ -11,6 +11,7 @@ import akka.pattern._ import akka.util.Timeout import akka.util.Timeout._ import com.typesafe.config.{Config, ConfigFactory} +import kafka.manager.features.ClusterFeatures import kafka.manager.utils.KafkaServerInTest import ActorModel._ import kafka.test.SeededBroker @@ -35,10 +36,11 @@ class TestKafkaStateActor extends KafkaServerInTest { private[this] var kafkaStateActor : Option[ActorRef] = None private[this] implicit val timeout: Timeout = 10.seconds private[this] val defaultClusterConfig = ClusterConfig("test","0.8.2.0","localhost:2818",100,false) + private[this] val defaultClusterContext = ClusterContext(ClusterFeatures.from(defaultClusterConfig), defaultClusterConfig) override protected def beforeAll(): Unit = { super.beforeAll() - val props = Props(classOf[KafkaStateActor],sharedCurator, true, defaultClusterConfig) + val props = Props(classOf[KafkaStateActor],sharedCurator, defaultClusterContext) kafkaStateActor = Some(system.actorOf(props.withDispatcher("pinned-dispatcher"),"ksa")) } @@ -89,7 +91,7 @@ class TestKafkaStateActor extends KafkaServerInTest { descriptions foreach println withKafkaStateActor(KSGetBrokers) { brokerList: BrokerList => - val topicIdentityList : IndexedSeq[TopicIdentity] = descriptions.flatten.map(td => TopicIdentity.from(brokerList,td, None, brokerList.clusterConfig)) + val topicIdentityList : IndexedSeq[TopicIdentity] = descriptions.flatten.map(td => TopicIdentity.from(brokerList,td, None, brokerList.clusterContext)) topicIdentityList foreach println } } diff --git a/test/kafka/manager/TestLogkafkaStateActor.scala b/test/kafka/manager/TestLogkafkaStateActor.scala index 18e90d896..5960b88ff 100644 --- a/test/kafka/manager/TestLogkafkaStateActor.scala +++ b/test/kafka/manager/TestLogkafkaStateActor.scala @@ -11,6 +11,7 @@ import akka.pattern._ import akka.util.Timeout import akka.util.Timeout._ import com.typesafe.config.{Config, ConfigFactory} +import kafka.manager.features.ClusterFeatures import kafka.manager.utils.KafkaServerInTest import ActorModel._ import kafka.test.SeededBroker @@ -35,10 +36,11 @@ class TestLogkafkaStateActor extends KafkaServerInTest { private[this] var logkafkaStateActor : Option[ActorRef] = None private[this] implicit val timeout: Timeout = 10.seconds private[this] val defaultClusterConfig = ClusterConfig("test","0.8.2.0","localhost:2818",100,false,true) + private[this] val defaultClusterContext = ClusterContext(ClusterFeatures.from(defaultClusterConfig), defaultClusterConfig) override protected def beforeAll(): Unit = { super.beforeAll() - val props = Props(classOf[LogkafkaStateActor],sharedCurator, true, defaultClusterConfig) + val props = Props(classOf[LogkafkaStateActor],sharedCurator, defaultClusterContext) logkafkaStateActor = Some(system.actorOf(props.withDispatcher("pinned-dispatcher"),"lksa")) } diff --git a/test/kafka/manager/TestLogkafkaViewCacheActor.scala b/test/kafka/manager/TestLogkafkaViewCacheActor.scala index 1712696fd..12a61318b 100644 --- a/test/kafka/manager/TestLogkafkaViewCacheActor.scala +++ b/test/kafka/manager/TestLogkafkaViewCacheActor.scala @@ -10,6 +10,7 @@ import akka.actor.{ActorRef, ActorSystem, Kill, Props} import akka.pattern._ import akka.util.Timeout import com.typesafe.config.{Config, ConfigFactory} +import kafka.manager.features.ClusterFeatures import kafka.manager.utils.KafkaServerInTest import ActorModel._ import kafka.test.SeededBroker @@ -35,15 +36,17 @@ class TestLogkafkaViewCacheActor extends KafkaServerInTest { private[this] var logkafkaViewCacheActor : Option[ActorRef] = None private[this] val defaultClusterConfig = ClusterConfig("test","0.8.2.0","localhost:2818",100,false,true) + private[this] val defaultClusterContext = ClusterContext(ClusterFeatures.from(defaultClusterConfig), defaultClusterConfig) override protected def beforeAll(): Unit = { super.beforeAll() val clusterConfig = ClusterConfig("dev","0.8.2.0",kafkaServerZkPath, jmxEnabled = false, logkafkaEnabled = true) - val props = Props(classOf[KafkaStateActor],sharedCurator, true, defaultClusterConfig) + val clusterContext = ClusterContext(ClusterFeatures.from(clusterConfig), clusterConfig) + val props = Props(classOf[KafkaStateActor],sharedCurator, defaultClusterContext) logkafkaStateActor = Some(system.actorOf(props.withDispatcher("pinned-dispatcher"),"lksa")) - val lkvConfig = LogkafkaViewCacheActorConfig(logkafkaStateActor.get.path, clusterConfig, LongRunningPoolConfig(2,100), FiniteDuration(10, SECONDS)) + val lkvConfig = LogkafkaViewCacheActorConfig(logkafkaStateActor.get.path, clusterContext, LongRunningPoolConfig(2,100), FiniteDuration(10, SECONDS)) val lkvcProps = Props(classOf[LogkafkaViewCacheActor],lkvConfig) logkafkaViewCacheActor = Some(system.actorOf(lkvcProps,"logkafka-view")) diff --git a/test/kafka/manager/utils/TestCreateTopic.scala b/test/kafka/manager/utils/TestCreateTopic.scala index a7b8a6d5c..9d9b2bff0 100644 --- a/test/kafka/manager/utils/TestCreateTopic.scala +++ b/test/kafka/manager/utils/TestCreateTopic.scala @@ -8,7 +8,8 @@ import java.util.Properties import TopicErrors._ import kafka.manager.ActorModel.{TopicIdentity, TopicDescription} -import kafka.manager.{ClusterConfig, Kafka_0_8_2_0} +import kafka.manager.features.ClusterFeatures +import kafka.manager.{ClusterContext, ClusterConfig, Kafka_0_8_2_0} import org.apache.zookeeper.data.Stat /** @@ -18,6 +19,7 @@ class TestCreateTopic extends CuratorAwareTest { private[this] val adminUtils = new AdminUtils(Kafka_0_8_2_0) private[this] val defaultClusterConfig = ClusterConfig("test","0.8.2.0","localhost:2818",100,false) + private[this] val defaultClusterContext = ClusterContext(ClusterFeatures.from(defaultClusterConfig), defaultClusterConfig) test("create topic with empty name") { checkError[TopicNameEmpty] { @@ -94,7 +96,7 @@ class TestCreateTopic extends CuratorAwareTest { val stat = new Stat() val json:String = curator.getData.storingStatIn(stat).forPath(ZkUtils.getTopicPath("mytopic")) val configJson : String = curator.getData.forPath(ZkUtils.getTopicConfigPath("mytopic")) - val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion(),json),None,Option((-1,configJson)),false),None,defaultClusterConfig) + val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion(),json),None,Option((-1,configJson))),None,defaultClusterContext) assert(td.partitions == 10) assert(td.replicationFactor == 3) } @@ -118,7 +120,7 @@ class TestCreateTopic extends CuratorAwareTest { val stat = new Stat val json:String = curator.getData.storingStatIn(stat).forPath(ZkUtils.getTopicPath("mytopic")) val configJson : String = curator.getData.forPath(ZkUtils.getTopicConfigPath("mytopic")) - val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion,json),None,Option((-1,configJson)),false),None, defaultClusterConfig) + val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion,json),None,Option((-1,configJson))),None, defaultClusterContext) val numPartitions = td.partitions adminUtils.addPartitions(curator, td.topic, numPartitions, td.partitionsIdentity.mapValues(_.replicas.toSeq),brokerList, stat.getVersion) } @@ -132,7 +134,7 @@ class TestCreateTopic extends CuratorAwareTest { val stat = new Stat val json:String = curator.getData.storingStatIn(stat).forPath(ZkUtils.getTopicPath("mytopic")) val configJson : String = curator.getData.forPath(ZkUtils.getTopicConfigPath("mytopic")) - val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion,json),None,Option((-1,configJson)),false),None, defaultClusterConfig) + val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion,json),None,Option((-1,configJson))),None, defaultClusterContext) val numPartitions = td.partitions + 2 adminUtils.addPartitions(curator, td.topic, numPartitions, td.partitionsIdentity.mapValues(_.replicas.toSeq),brokerList,stat.getVersion) } @@ -145,7 +147,7 @@ class TestCreateTopic extends CuratorAwareTest { val stat = new Stat val json:String = curator.getData.storingStatIn(stat).forPath(ZkUtils.getTopicPath("mytopic")) val configJson : String = curator.getData.forPath(ZkUtils.getTopicConfigPath("mytopic")) - val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion,json),None,Option((-1,configJson)),false),None, defaultClusterConfig) + val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion,json),None,Option((-1,configJson))),None, defaultClusterContext) val numPartitions = td.partitions + 2 adminUtils.addPartitions(curator, td.topic, numPartitions, td.partitionsIdentity.mapValues(_.replicas.toSeq),brokerList,stat.getVersion) @@ -153,7 +155,7 @@ class TestCreateTopic extends CuratorAwareTest { { val json: String = curator.getData.forPath(ZkUtils.getTopicPath("mytopic")) val configJson: String = curator.getData.forPath(ZkUtils.getTopicConfigPath("mytopic")) - val td = TopicIdentity.from(3, TopicDescription("mytopic", (-1,json), None, Option((-1,configJson)), false),None, defaultClusterConfig) + val td = TopicIdentity.from(3, TopicDescription("mytopic", (-1,json), None, Option((-1,configJson))),None, defaultClusterContext) assert(td.partitions === numPartitions, "Failed to add partitions!") assert(td.config.toMap.apply(kafka.manager.utils.zero82.LogConfig.RententionMsProp) === "1800000") } @@ -168,7 +170,7 @@ class TestCreateTopic extends CuratorAwareTest { val configStat = new Stat val configJson : String = curator.getData.storingStatIn(configStat).forPath(ZkUtils.getTopicConfigPath("mytopic")) val configReadVersion = configStat.getVersion - val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion,json),None,Option((configReadVersion,configJson)),false),None, defaultClusterConfig) + val td = TopicIdentity.from(3,TopicDescription("mytopic",(stat.getVersion,json),None,Option((configReadVersion,configJson))),None, defaultClusterContext) val properties = new Properties() td.config.foreach { case (k,v) => properties.put(k,v)} properties.setProperty(kafka.manager.utils.zero82.LogConfig.RententionMsProp,"3600000") @@ -179,7 +181,7 @@ class TestCreateTopic extends CuratorAwareTest { val json: String = curator.getData.forPath(ZkUtils.getTopicPath("mytopic")) val configStat = new Stat val configJson : String = curator.getData.storingStatIn(configStat).forPath(ZkUtils.getTopicConfigPath("mytopic")) - val td = TopicIdentity.from(3, TopicDescription("mytopic", (-1,json), None, Option((configStat.getVersion,configJson)), false),None, defaultClusterConfig) + val td = TopicIdentity.from(3, TopicDescription("mytopic", (-1,json), None, Option((configStat.getVersion,configJson))),None, defaultClusterContext) assert(td.config.toMap.apply(kafka.manager.utils.zero82.LogConfig.RententionMsProp) === "3600000") assert(configReadVersion != configStat.getVersion) } diff --git a/test/kafka/manager/utils/TestReassignPartitions.scala b/test/kafka/manager/utils/TestReassignPartitions.scala index 4b57fe4db..9f4b67059 100644 --- a/test/kafka/manager/utils/TestReassignPartitions.scala +++ b/test/kafka/manager/utils/TestReassignPartitions.scala @@ -7,7 +7,8 @@ package kafka.manager.utils import java.util.Properties import kafka.manager.ActorModel._ -import kafka.manager.{ClusterConfig, Kafka_0_8_2_0} +import kafka.manager.features.ClusterFeatures +import kafka.manager.{ClusterContext, ClusterConfig, Kafka_0_8_2_0} import kafka.manager.utils.zero81._ import org.apache.zookeeper.data.Stat @@ -25,7 +26,8 @@ class TestReassignPartitions extends CuratorAwareTest { private[this] val brokerList = IndexedSeq(1,2,3) private[this] val defaultClusterConfig = ClusterConfig("test","0.8.2.0","localhost:2818",100,false) - + private[this] val defaultClusterContext = ClusterContext(ClusterFeatures.from(defaultClusterConfig), defaultClusterConfig) + private[this] def mytopic1 : TopicIdentity = getTopicIdentity("mytopic1") private[this] def mytopic2 : TopicIdentity = getTopicIdentity("mytopic2") private[this] def mytopic3 : TopicIdentity = getTopicIdentity("mytopic3") @@ -47,8 +49,8 @@ class TestReassignPartitions extends CuratorAwareTest { val json : String = curator.getData.storingStatIn(stat).forPath(ZkUtils.getTopicPath(topic)) val configStat = new Stat val configJson : String = curator.getData.storingStatIn(configStat).forPath(ZkUtils.getTopicConfigPath(topic)) - val td: TopicDescription = TopicDescription(topic,(stat.getVersion,json),None,Option((configStat.getVersion,configJson)),false) - TopicIdentity.from(brokerList.size,td,None, defaultClusterConfig) + val td: TopicDescription = TopicDescription(topic,(stat.getVersion,json),None,Option((configStat.getVersion,configJson))) + TopicIdentity.from(brokerList.size,td,None, defaultClusterContext) } } From 34be57486fca6bb672111dfcf39a97a919a40062 Mon Sep 17 00:00:00 2001 From: patelh Date: Tue, 25 Aug 2015 21:29:26 -0700 Subject: [PATCH 11/17] Move button on topic view --- app/views/topic/topicViewContent.scala.html | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/app/views/topic/topicViewContent.scala.html b/app/views/topic/topicViewContent.scala.html index eb6878496..5cce38c02 100644 --- a/app/views/topic/topicViewContent.scala.html +++ b/app/views/topic/topicViewContent.scala.html @@ -173,9 +173,6 @@ - } @@ -185,6 +182,11 @@ + @features.app(features.KMReassignPartitionsFeature) { + + }
@btp.id@btp.id @btp.partitions.size @btp.partitions.mkString("(",",",")") @@ -217,7 +225,7 @@
@tpi.partNum - @tpi.leader + @tpi.leader @tpi.replicas.mkString("(",",",")") @tpi.isr.mkString("(",",",")") Generate Partition Assignments - Manually Set Partition Assignments -
Update Config + Manually Set Partition Assignments +
From eafb0a43edb4b882a9807bde9a4be8b2b9148b31 Mon Sep 17 00:00:00 2001 From: patelh Date: Tue, 25 Aug 2015 21:36:28 -0700 Subject: [PATCH 12/17] Update button text on topic view --- app/views/topic/topicViewContent.scala.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/app/views/topic/topicViewContent.scala.html b/app/views/topic/topicViewContent.scala.html index 5cce38c02..cb4971869 100644 --- a/app/views/topic/topicViewContent.scala.html +++ b/app/views/topic/topicViewContent.scala.html @@ -184,7 +184,7 @@
- Manually Set Partition Assignments + Manual Partition Assignments