mirror of https://github.com/apache/kafka.git
KAFKA-4603: Disallow abbreviations in OptionParser constructor
KAFKA-4603 the command parsed error Using "new OptionParser" might result in parse error Change all the OptionParser constructor in Kafka into "new OptionParser(false)" Author: xinlihua <xin.lihua1@zte.com.cn> Author: unknown <00067310@A23338408.zte.intra> Author: auroraxlh <xin.lihua1@zte.com.cn> Author: xin <xin.lihua1@zte.com.cn> Reviewers: Damian Guy, Guozhang Wang Closes #2349 from auroraxlh/fix_OptionParser_bug
This commit is contained in:
parent
b3788d8dcb
commit
f0745cd514
|
@ -29,7 +29,7 @@ import scala.collection.JavaConverters._
|
||||||
object Kafka extends Logging {
|
object Kafka extends Logging {
|
||||||
|
|
||||||
def getPropsFromArgs(args: Array[String]): Properties = {
|
def getPropsFromArgs(args: Array[String]): Properties = {
|
||||||
val optionParser = new OptionParser
|
val optionParser = new OptionParser(false)
|
||||||
val overrideOpt = optionParser.accepts("override", "Optional property that should override values set in server.properties file")
|
val overrideOpt = optionParser.accepts("override", "Optional property that should override values set in server.properties file")
|
||||||
.withRequiredArg()
|
.withRequiredArg()
|
||||||
.ofType(classOf[String])
|
.ofType(classOf[String])
|
||||||
|
|
|
@ -268,7 +268,7 @@ object AclCommand {
|
||||||
}
|
}
|
||||||
|
|
||||||
class AclCommandOptions(args: Array[String]) {
|
class AclCommandOptions(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val authorizerOpt = parser.accepts("authorizer", "Fully qualified class name of the authorizer, defaults to kafka.security.auth.SimpleAclAuthorizer.")
|
val authorizerOpt = parser.accepts("authorizer", "Fully qualified class name of the authorizer, defaults to kafka.security.auth.SimpleAclAuthorizer.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("authorizer")
|
.describedAs("authorizer")
|
||||||
|
|
|
@ -64,7 +64,7 @@ object BrokerApiVersionsCommand {
|
||||||
val BootstrapServerDoc = "REQUIRED: The server to connect to."
|
val BootstrapServerDoc = "REQUIRED: The server to connect to."
|
||||||
val CommandConfigDoc = "A property file containing configs to be passed to Admin Client."
|
val CommandConfigDoc = "A property file containing configs to be passed to Admin Client."
|
||||||
|
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val commandConfigOpt = parser.accepts("command-config", CommandConfigDoc)
|
val commandConfigOpt = parser.accepts("command-config", CommandConfigDoc)
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("command config property file")
|
.describedAs("command config property file")
|
||||||
|
|
|
@ -278,7 +278,7 @@ object ConfigCommand extends Config {
|
||||||
}
|
}
|
||||||
|
|
||||||
class ConfigCommandOptions(args: Array[String]) {
|
class ConfigCommandOptions(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " +
|
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " +
|
||||||
"Multiple URLS can be given to allow fail-over.")
|
"Multiple URLS can be given to allow fail-over.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
|
|
|
@ -713,7 +713,7 @@ object ConsumerGroupCommand extends Logging {
|
||||||
val ResetToLatestDoc = "Reset offsets to latest offset."
|
val ResetToLatestDoc = "Reset offsets to latest offset."
|
||||||
val ResetShiftByDoc = "Reset offsets shifting current offset by 'n', where 'n' can be positive or negative"
|
val ResetShiftByDoc = "Reset offsets shifting current offset by 'n', where 'n' can be positive or negative"
|
||||||
|
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val zkConnectOpt = parser.accepts("zookeeper", ZkConnectDoc)
|
val zkConnectOpt = parser.accepts("zookeeper", ZkConnectDoc)
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("urls")
|
.describedAs("urls")
|
||||||
|
|
|
@ -97,7 +97,7 @@ object DeleteRecordsCommand {
|
||||||
"{\"partitions\":\n [{\"topic\": \"foo\", \"partition\": 1, \"offset\": 1}],\n \"version\":1\n}"
|
"{\"partitions\":\n [{\"topic\": \"foo\", \"partition\": 1, \"offset\": 1}],\n \"version\":1\n}"
|
||||||
val CommandConfigDoc = "A property file containing configs to be passed to Admin Client."
|
val CommandConfigDoc = "A property file containing configs to be passed to Admin Client."
|
||||||
|
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val bootstrapServerOpt = parser.accepts("bootstrap-server", BootstrapServerDoc)
|
val bootstrapServerOpt = parser.accepts("bootstrap-server", BootstrapServerDoc)
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("server(s) to use for bootstrapping")
|
.describedAs("server(s) to use for bootstrapping")
|
||||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.kafka.common.security.JaasUtils
|
||||||
object PreferredReplicaLeaderElectionCommand extends Logging {
|
object PreferredReplicaLeaderElectionCommand extends Logging {
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val jsonFileOpt = parser.accepts("path-to-json-file", "The JSON file with the list of partitions " +
|
val jsonFileOpt = parser.accepts("path-to-json-file", "The JSON file with the list of partitions " +
|
||||||
"for which preferred replica leader election should be done, in the following format - \n" +
|
"for which preferred replica leader election should be done, in the following format - \n" +
|
||||||
"{\"partitions\":\n\t[{\"topic\": \"foo\", \"partition\": 1},\n\t {\"topic\": \"foobar\", \"partition\": 2}]\n}\n" +
|
"{\"partitions\":\n\t[{\"topic\": \"foo\", \"partition\": 1},\n\t {\"topic\": \"foobar\", \"partition\": 2}]\n}\n" +
|
||||||
|
|
|
@ -279,7 +279,7 @@ object ReassignPartitionsCommand extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
class ReassignPartitionsCommandOptions(args: Array[String]) {
|
class ReassignPartitionsCommandOptions(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
|
|
||||||
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the " +
|
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the " +
|
||||||
"form host:port. Multiple URLS can be given to allow fail-over.")
|
"form host:port. Multiple URLS can be given to allow fail-over.")
|
||||||
|
|
|
@ -272,7 +272,7 @@ object TopicCommand extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
class TopicCommandOptions(args: Array[String]) {
|
class TopicCommandOptions(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " +
|
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " +
|
||||||
"Multiple URLS can be given to allow fail-over.")
|
"Multiple URLS can be given to allow fail-over.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
|
|
|
@ -65,7 +65,7 @@ object ZkSecurityMigrator extends Logging {
|
||||||
|
|
||||||
def run(args: Array[String]) {
|
def run(args: Array[String]) {
|
||||||
var jaasFile = System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)
|
var jaasFile = System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val zkAclOpt = parser.accepts("zookeeper.acl", "Indicates whether to make the Kafka znodes in ZooKeeper secure or unsecure."
|
val zkAclOpt = parser.accepts("zookeeper.acl", "Indicates whether to make the Kafka znodes in ZooKeeper secure or unsecure."
|
||||||
+ " The options are 'secure' and 'unsecure'").withRequiredArg().ofType(classOf[String])
|
+ " The options are 'secure' and 'unsecure'").withRequiredArg().ofType(classOf[String])
|
||||||
val zkUrlOpt = parser.accepts("zookeeper.connect", "Sets the ZooKeeper connect string (ensemble). This parameter " +
|
val zkUrlOpt = parser.accepts("zookeeper.connect", "Sets the ZooKeeper connect string (ensemble). This parameter " +
|
||||||
|
|
|
@ -205,7 +205,7 @@ object ConsoleConsumer extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
class ConsumerConfig(args: Array[String]) {
|
class ConsumerConfig(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val topicIdOpt = parser.accepts("topic", "The topic id to consume on.")
|
val topicIdOpt = parser.accepts("topic", "The topic id to consume on.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("topic")
|
.describedAs("topic")
|
||||||
|
|
|
@ -133,7 +133,7 @@ object ConsoleProducer {
|
||||||
}
|
}
|
||||||
|
|
||||||
class ProducerConfig(args: Array[String]) {
|
class ProducerConfig(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val topicOpt = parser.accepts("topic", "REQUIRED: The topic id to produce messages to.")
|
val topicOpt = parser.accepts("topic", "REQUIRED: The topic id to produce messages to.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("topic")
|
.describedAs("topic")
|
||||||
|
|
|
@ -102,7 +102,7 @@ object ConsumerOffsetChecker extends Logging {
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
warn("WARNING: ConsumerOffsetChecker is deprecated and will be dropped in releases following 0.9.0. Use ConsumerGroupCommand instead.")
|
warn("WARNING: ConsumerOffsetChecker is deprecated and will be dropped in releases following 0.9.0. Use ConsumerGroupCommand instead.")
|
||||||
|
|
||||||
val parser = new OptionParser()
|
val parser = new OptionParser(false)
|
||||||
|
|
||||||
val zkConnectOpt = parser.accepts("zookeeper", "ZooKeeper connect string.").
|
val zkConnectOpt = parser.accepts("zookeeper", "ZooKeeper connect string.").
|
||||||
withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String])
|
withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String])
|
||||||
|
|
|
@ -37,7 +37,7 @@ import scala.collection.JavaConverters._
|
||||||
object DumpLogSegments {
|
object DumpLogSegments {
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val printOpt = parser.accepts("print-data-log", "if set, printing the messages content when dumping data logs. Automatically set if any decoder option is specified.")
|
val printOpt = parser.accepts("print-data-log", "if set, printing the messages content when dumping data logs. Automatically set if any decoder option is specified.")
|
||||||
val verifyOpt = parser.accepts("verify-index-only", "if set, just verify the index log without printing its content.")
|
val verifyOpt = parser.accepts("verify-index-only", "if set, just verify the index log without printing its content.")
|
||||||
val indexSanityOpt = parser.accepts("index-sanity-check", "if set, just checks the index sanity without printing its content. " +
|
val indexSanityOpt = parser.accepts("index-sanity-check", "if set, just checks the index sanity without printing its content. " +
|
||||||
|
|
|
@ -46,7 +46,7 @@ import scala.collection.JavaConverters._
|
||||||
object ExportZkOffsets extends Logging {
|
object ExportZkOffsets extends Logging {
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
|
|
||||||
val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.")
|
val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.")
|
||||||
.withRequiredArg()
|
.withRequiredArg()
|
||||||
|
|
|
@ -29,7 +29,7 @@ import kafka.utils.{CommandLineUtils, Exit, ToolsUtils}
|
||||||
object GetOffsetShell {
|
object GetOffsetShell {
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
|
val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("hostname:port,...,hostname:port")
|
.describedAs("hostname:port,...,hostname:port")
|
||||||
|
|
|
@ -44,7 +44,7 @@ import org.apache.kafka.common.security.JaasUtils
|
||||||
object ImportZkOffsets extends Logging {
|
object ImportZkOffsets extends Logging {
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
|
|
||||||
val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.")
|
val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.")
|
||||||
.withRequiredArg()
|
.withRequiredArg()
|
||||||
|
|
|
@ -41,7 +41,7 @@ object JmxTool extends Logging {
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
// Parse command line
|
// Parse command line
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val objectNameOpt =
|
val objectNameOpt =
|
||||||
parser.accepts("object-name", "A JMX object name to use as a query. This can contain wild cards, and this option " +
|
parser.accepts("object-name", "A JMX object name to use as a query. This can contain wild cards, and this option " +
|
||||||
"can be given multiple times to specify more than one query. If no objects are specified " +
|
"can be given multiple times to specify more than one query. If no objects are specified " +
|
||||||
|
|
|
@ -87,7 +87,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
|
||||||
|
|
||||||
info("Starting mirror maker")
|
info("Starting mirror maker")
|
||||||
try {
|
try {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
|
|
||||||
val consumerConfigOpt = parser.accepts("consumer.config",
|
val consumerConfigOpt = parser.accepts("consumer.config",
|
||||||
"Embedded consumer config for consuming from the source cluster.")
|
"Embedded consumer config for consuming from the source cluster.")
|
||||||
|
|
|
@ -21,7 +21,7 @@ import joptsimple.OptionParser
|
||||||
|
|
||||||
|
|
||||||
class PerfConfig(args: Array[String]) {
|
class PerfConfig(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
|
val numMessagesOpt = parser.accepts("messages", "REQUIRED: The number of messages to send or consume")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("count")
|
.describedAs("count")
|
||||||
|
|
|
@ -61,7 +61,7 @@ object ReplayLogProducer extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
class Config(args: Array[String]) {
|
class Config(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " +
|
val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " +
|
||||||
"Multiple URLS can be given to allow fail-over.")
|
"Multiple URLS can be given to allow fail-over.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
|
|
|
@ -65,7 +65,7 @@ object ReplicaVerificationTool extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
|
val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("hostname:port,...,hostname:port")
|
.describedAs("hostname:port,...,hostname:port")
|
||||||
|
|
|
@ -38,7 +38,7 @@ object SimpleConsumerShell extends Logging {
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
|
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
|
val brokerListOpt = parser.accepts("broker-list", "REQUIRED: The list of hostname and port of the server to connect to.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("hostname:port,...,hostname:port")
|
.describedAs("hostname:port,...,hostname:port")
|
||||||
|
|
|
@ -60,7 +60,7 @@ object StateChangeLogMerger extends Logging {
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
|
|
||||||
// Parse input arguments.
|
// Parse input arguments.
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val filesOpt = parser.accepts("logs", "Comma separated list of state change logs or a regex for the log file names")
|
val filesOpt = parser.accepts("logs", "Comma separated list of state change logs or a regex for the log file names")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("file1,file2,...")
|
.describedAs("file1,file2,...")
|
||||||
|
|
|
@ -137,7 +137,8 @@ public class StreamsResetter {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void parseArguments(final String[] args) throws IOException {
|
private void parseArguments(final String[] args) throws IOException {
|
||||||
final OptionParser optionParser = new OptionParser();
|
|
||||||
|
final OptionParser optionParser = new OptionParser(false);
|
||||||
applicationIdOption = optionParser.accepts("application-id", "The Kafka Streams application ID (application.id).")
|
applicationIdOption = optionParser.accepts("application-id", "The Kafka Streams application ID (application.id).")
|
||||||
.withRequiredArg()
|
.withRequiredArg()
|
||||||
.ofType(String.class)
|
.ofType(String.class)
|
||||||
|
|
|
@ -23,7 +23,7 @@ import kafka.utils.{CommandLineUtils, Exit, Logging, ZKGroupTopicDirs, ZkUtils}
|
||||||
|
|
||||||
object VerifyConsumerRebalance extends Logging {
|
object VerifyConsumerRebalance extends Logging {
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val parser = new OptionParser()
|
val parser = new OptionParser(false)
|
||||||
|
|
||||||
val zkConnectOpt = parser.accepts("zookeeper.connect", "ZooKeeper connect string.").
|
val zkConnectOpt = parser.accepts("zookeeper.connect", "ZooKeeper connect string.").
|
||||||
withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String])
|
withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String])
|
||||||
|
|
|
@ -49,7 +49,7 @@ import scala.collection.JavaConverters._
|
||||||
object TestLogCleaning {
|
object TestLogCleaning {
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val numMessagesOpt = parser.accepts("messages", "The number of messages to send or consume.")
|
val numMessagesOpt = parser.accepts("messages", "The number of messages to send or consume.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("count")
|
.describedAs("count")
|
||||||
|
|
|
@ -38,7 +38,7 @@ import scala.math._
|
||||||
object TestLinearWriteSpeed {
|
object TestLinearWriteSpeed {
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val dirOpt = parser.accepts("dir", "The directory to write to.")
|
val dirOpt = parser.accepts("dir", "The directory to write to.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("path")
|
.describedAs("path")
|
||||||
|
|
|
@ -197,7 +197,7 @@ object TestOffsetManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val zookeeperOpt = parser.accepts("zookeeper", "The ZooKeeper connection URL.")
|
val zookeeperOpt = parser.accepts("zookeeper", "The ZooKeeper connection URL.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("ZooKeeper URL")
|
.describedAs("ZooKeeper URL")
|
||||||
|
|
|
@ -36,7 +36,7 @@ import scala.collection.JavaConverters._
|
||||||
object TestPurgatoryPerformance {
|
object TestPurgatoryPerformance {
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
val parser = new OptionParser
|
val parser = new OptionParser(false)
|
||||||
val keySpaceSizeOpt = parser.accepts("key-space-size", "The total number of possible keys")
|
val keySpaceSizeOpt = parser.accepts("key-space-size", "The total number of possible keys")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("total_num_possible_keys")
|
.describedAs("total_num_possible_keys")
|
||||||
|
|
|
@ -908,11 +908,11 @@
|
||||||
</ol>
|
</ol>
|
||||||
Here is an example of how to run the migration tool:
|
Here is an example of how to run the migration tool:
|
||||||
<pre>
|
<pre>
|
||||||
./bin/zookeeper-security-migration --zookeeper.acl=secure --zookeeper.connect=localhost:2181
|
./bin/zookeeper-security-migration.sh --zookeeper.acl=secure --zookeeper.connect=localhost:2181
|
||||||
</pre>
|
</pre>
|
||||||
<p>Run this to see the full list of parameters:</p>
|
<p>Run this to see the full list of parameters:</p>
|
||||||
<pre>
|
<pre>
|
||||||
./bin/zookeeper-security-migration --help
|
./bin/zookeeper-security-migration.sh --help
|
||||||
</pre>
|
</pre>
|
||||||
<h4><a id="zk_authz_ensemble" href="#zk_authz_ensemble">7.6.3 Migrating the ZooKeeper ensemble</a></h4>
|
<h4><a id="zk_authz_ensemble" href="#zk_authz_ensemble">7.6.3 Migrating the ZooKeeper ensemble</a></h4>
|
||||||
It is also necessary to enable authentication on the ZooKeeper ensemble. To do it, we need to perform a rolling restart of the server and set a few properties. Please refer to the ZooKeeper documentation for more detail:
|
It is also necessary to enable authentication on the ZooKeeper ensemble. To do it, we need to perform a rolling restart of the server and set a few properties. Please refer to the ZooKeeper documentation for more detail:
|
||||||
|
|
|
@ -340,7 +340,7 @@ public class ResetIntegrationTest {
|
||||||
if (intermediateUserTopic != null) {
|
if (intermediateUserTopic != null) {
|
||||||
parameters = new String[]{
|
parameters = new String[]{
|
||||||
"--application-id", APP_ID + testNo,
|
"--application-id", APP_ID + testNo,
|
||||||
"--bootstrap-server", CLUSTER.bootstrapServers(),
|
"--bootstrap-servers", CLUSTER.bootstrapServers(),
|
||||||
"--zookeeper", CLUSTER.zKConnectString(),
|
"--zookeeper", CLUSTER.zKConnectString(),
|
||||||
"--input-topics", INPUT_TOPIC,
|
"--input-topics", INPUT_TOPIC,
|
||||||
"--intermediate-topics", INTERMEDIATE_USER_TOPIC
|
"--intermediate-topics", INTERMEDIATE_USER_TOPIC
|
||||||
|
@ -348,7 +348,7 @@ public class ResetIntegrationTest {
|
||||||
} else {
|
} else {
|
||||||
parameters = new String[]{
|
parameters = new String[]{
|
||||||
"--application-id", APP_ID + testNo,
|
"--application-id", APP_ID + testNo,
|
||||||
"--bootstrap-server", CLUSTER.bootstrapServers(),
|
"--bootstrap-servers", CLUSTER.bootstrapServers(),
|
||||||
"--zookeeper", CLUSTER.zKConnectString(),
|
"--zookeeper", CLUSTER.zKConnectString(),
|
||||||
"--input-topics", INPUT_TOPIC
|
"--input-topics", INPUT_TOPIC
|
||||||
};
|
};
|
||||||
|
|
Loading…
Reference in New Issue