Browse Source
Author: Colin P. Mccabe <cmccabe@confluent.io> Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>, Apurva Mehta <apurva.1618@gmail.com>, Ismael Juma <ismael@juma.me.uk> Closes #2184 from cmccabe/KAFKA-4457pull/2389/merge
Colin P. Mccabe
8 years ago
committed by
Ismael Juma
7 changed files with 244 additions and 25 deletions
@ -0,0 +1,17 @@
@@ -0,0 +1,17 @@
|
||||
#!/bin/bash |
||||
# Licensed to the Apache Software Foundation (ASF) under one or more |
||||
# contributor license agreements. See the NOTICE file distributed with |
||||
# this work for additional information regarding copyright ownership. |
||||
# The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
# (the "License"); you may not use this file except in compliance with |
||||
# the License. You may obtain a copy of the License at |
||||
# |
||||
# http://www.apache.org/licenses/LICENSE-2.0 |
||||
# |
||||
# Unless required by applicable law or agreed to in writing, software |
||||
# distributed under the License is distributed on an "AS IS" BASIS, |
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
# See the License for the specific language governing permissions and |
||||
# limitations under the License. |
||||
|
||||
exec $(dirname $0)/kafka-run-class.sh kafka.admin.BrokerApiVersionsCommand "$@" |
@ -0,0 +1,81 @@
@@ -0,0 +1,81 @@
|
||||
/** |
||||
* 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.admin |
||||
|
||||
import java.io.PrintStream |
||||
import java.util.Properties |
||||
|
||||
import kafka.utils.CommandLineUtils |
||||
import org.apache.kafka.common.utils.Utils |
||||
import org.apache.kafka.clients.CommonClientConfigs |
||||
import joptsimple._ |
||||
|
||||
import scala.util.{Failure, Success} |
||||
|
||||
/** |
||||
* A command for retrieving broker version information. |
||||
*/ |
||||
object BrokerApiVersionsCommand { |
||||
|
||||
def main(args: Array[String]): Unit = { |
||||
execute(args, System.out) |
||||
} |
||||
|
||||
def execute(args: Array[String], out: PrintStream): Unit = { |
||||
val opts = new BrokerVersionCommandOptions(args) |
||||
val adminClient = createAdminClient(opts) |
||||
val brokerMap = adminClient.listAllBrokerVersionInfo() |
||||
brokerMap.foreach { case (broker, versionInfoOrError) => |
||||
versionInfoOrError match { |
||||
case Success(v) => out.print(s"${broker} -> ${v.toString(true)}\n") |
||||
case Failure(v) => out.print(s"${broker} -> ERROR: ${v}\n") |
||||
} |
||||
} |
||||
} |
||||
|
||||
private def createAdminClient(opts: BrokerVersionCommandOptions): AdminClient = { |
||||
val props = if (opts.options.has(opts.commandConfigOpt)) |
||||
Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) |
||||
else |
||||
new Properties() |
||||
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt)) |
||||
AdminClient.create(props) |
||||
} |
||||
|
||||
class BrokerVersionCommandOptions(args: Array[String]) { |
||||
val BootstrapServerDoc = "REQUIRED: The server to connect to." |
||||
val CommandConfigDoc = "A property file containing configs to be passed to Admin Client." |
||||
|
||||
val parser = new OptionParser |
||||
val commandConfigOpt = parser.accepts("command-config", CommandConfigDoc) |
||||
.withRequiredArg |
||||
.describedAs("command config property file") |
||||
.ofType(classOf[String]) |
||||
val bootstrapServerOpt = parser.accepts("bootstrap-server", BootstrapServerDoc) |
||||
.withRequiredArg |
||||
.describedAs("server(s) to use for bootstrapping") |
||||
.ofType(classOf[String]) |
||||
val options = parser.parse(args : _*) |
||||
checkArgs() |
||||
|
||||
def checkArgs() { |
||||
// check required args |
||||
CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,61 @@
@@ -0,0 +1,61 @@
|
||||
/* |
||||
* 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.admin |
||||
|
||||
import java.io.{ByteArrayOutputStream, PrintStream} |
||||
import java.nio.charset.StandardCharsets |
||||
|
||||
import kafka.integration.KafkaServerTestHarness |
||||
import kafka.server.KafkaConfig |
||||
import kafka.utils.TestUtils |
||||
import org.apache.kafka.clients.NodeApiVersions |
||||
import org.apache.kafka.common.protocol.ApiKeys |
||||
import org.apache.kafka.common.requests.ApiVersionsResponse |
||||
import org.junit.Assert.{assertEquals, assertFalse, assertTrue} |
||||
import org.junit.Test |
||||
|
||||
class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { |
||||
|
||||
def generateConfigs(): Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps) |
||||
|
||||
@Test |
||||
def checkBrokerApiVersionCommandOutput() { |
||||
val byteArrayOutputStream = new ByteArrayOutputStream |
||||
val printStream = new PrintStream(byteArrayOutputStream) |
||||
BrokerApiVersionsCommand.execute(Array("--bootstrap-server", brokerList), printStream) |
||||
val content = new String(byteArrayOutputStream.toByteArray, StandardCharsets.UTF_8) |
||||
val lineIter = content.split("\n").iterator |
||||
assertTrue(lineIter.hasNext) |
||||
assertEquals(s"$brokerList (id: 0 rack: null) -> (", lineIter.next) |
||||
val nodeApiVersions = new NodeApiVersions(ApiVersionsResponse.API_VERSIONS_RESPONSE.apiVersions) |
||||
for (apiKey <- ApiKeys.values) { |
||||
val apiVersion = nodeApiVersions.apiVersion(apiKey) |
||||
val versionRangeStr = |
||||
if (apiVersion.minVersion == apiVersion.maxVersion) apiVersion.minVersion.toString |
||||
else s"${apiVersion.minVersion} to ${apiVersion.maxVersion}" |
||||
val terminator = if (apiKey == ApiKeys.values.last) "" else "," |
||||
val usableVersion = nodeApiVersions.usableVersion(apiKey) |
||||
val line = s"\t${apiKey.name}(${apiKey.id}): $versionRangeStr [usable: $usableVersion]$terminator" |
||||
assertTrue(lineIter.hasNext) |
||||
assertEquals(line, lineIter.next) |
||||
} |
||||
assertTrue(lineIter.hasNext) |
||||
assertEquals(")", lineIter.next) |
||||
assertFalse(lineIter.hasNext) |
||||
} |
||||
} |
Loading…
Reference in new issue