Skip to content

Commit eeb6a0d

Browse files
authored
KAFKA-19618 the record-size and throughputarguments don't work in TestRaftServer (#20379)
The `record-size` and `throughput` arguments don’t work in `TestRaftServer`. The `recordsPerSec` and `recordSize` values are always hard-coded. - Fix `recordsPerSec` and `recordSize` values hard-coded issue - Add "Required" description to command-line options to make it clear to users. Reviewers: Chia-Ping Tsai <[email protected]>
1 parent 0202721 commit eeb6a0d

File tree

1 file changed

+8
-12
lines changed

1 file changed

+8
-12
lines changed

core/src/main/scala/kafka/tools/TestRaftServer.scala

Lines changed: 8 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@ import kafka.network.SocketServer
2525
import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager, RaftManager}
2626
import kafka.server.{KafkaConfig, KafkaRequestHandlerPool}
2727
import kafka.utils.{CoreUtils, Logging}
28-
import org.apache.kafka.common.errors.InvalidConfigurationException
2928
import org.apache.kafka.common.message.ApiMessageType.ListenerType
3029
import org.apache.kafka.common.metrics.Metrics
3130
import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing
@@ -115,8 +114,8 @@ class TestRaftServer(
115114
workloadGenerator = new RaftWorkloadGenerator(
116115
raftManager,
117116
time,
118-
recordsPerSec = 20000,
119-
recordSize = 256
117+
recordsPerSec = throughput,
118+
recordSize = recordSize
120119
)
121120

122121
val requestHandler = new TestRaftRequestHandler(
@@ -428,7 +427,7 @@ object TestRaftServer extends Logging {
428427
}
429428

430429
private class TestRaftServerOptions(args: Array[String]) extends CommandDefaultOptions(args) {
431-
val configOpt: OptionSpec[String] = parser.accepts("config", "Required configured file")
430+
val configOpt: OptionSpec[String] = parser.accepts("config", "REQUIRED: The configured file")
432431
.withRequiredArg
433432
.describedAs("filename")
434433
.ofType(classOf[String])
@@ -446,29 +445,26 @@ object TestRaftServer extends Logging {
446445
.ofType(classOf[Int])
447446
.defaultsTo(256)
448447

449-
val directoryId: OptionSpec[String] = parser.accepts("replica-directory-id", "The directory id of the replica")
448+
val directoryId: OptionSpec[String] = parser.accepts("replica-directory-id", "REQUIRED: The directory id of the replica")
450449
.withRequiredArg
451450
.describedAs("directory id")
452451
.ofType(classOf[String])
453452

454453
options = parser.parse(args : _*)
454+
455+
def checkArgs(): Unit = CommandLineUtils.checkRequiredArgs(parser, options, configOpt, directoryId)
455456
}
456457

457458
def main(args: Array[String]): Unit = {
458459
val opts = new TestRaftServerOptions(args)
459460
try {
460461
CommandLineUtils.maybePrintHelpOrVersion(opts,
461462
"Standalone raft server for performance testing")
463+
opts.checkArgs()
462464

463465
val configFile = opts.options.valueOf(opts.configOpt)
464-
if (configFile == null) {
465-
throw new InvalidConfigurationException("Missing configuration file. Should specify with '--config'")
466-
}
467-
468466
val directoryIdAsString = opts.options.valueOf(opts.directoryId)
469-
if (directoryIdAsString == null) {
470-
throw new InvalidConfigurationException("Missing replica directory id. Should specify with --replica-directory-id")
471-
}
467+
472468
val serverProps = Utils.loadProps(configFile)
473469

474470
// KafkaConfig requires either `process.roles` or `zookeeper.connect`. Neither are

0 commit comments

Comments
 (0)