diff --git a/bin/pulsar-perf b/bin/pulsar-perf index bdc1dc1ed8b8c..1f6ce97476b4e 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -84,37 +84,6 @@ add_maven_deps_to_classpath() { fi PULSAR_CLASSPATH=${CLASSPATH}:`cat "${f}"` } -pulsar_help() { - cat < -where command is one of: - produce Run a producer - consume Run a consumer - transaction Run a transaction repeatedly - read Run a topic reader - - websocket-producer Run a websocket producer - - managed-ledger Write directly on managed-ledgers - monitor-brokers Continuously receive broker data and/or load reports - simulation-client Run a simulation server acting as a Pulsar client - simulation-controller Run a simulation controller to give commands to servers - - gen-doc Generate documentation automatically. - - help This help message - -or command is the full name of a class with a defined main() method. - -Environment variables: - PULSAR_LOG_CONF Log4j configuration file (default $DEFAULT_LOG_CONF) - PULSAR_CLIENT_CONF Configuration file for client (default: $DEFAULT_CLIENT_CONF) - PULSAR_EXTRA_OPTS Extra options to be passed to the jvm - PULSAR_EXTRA_CLASSPATH Add extra paths to the pulsar classpath - -These variable can also be set in conf/pulsar_env.sh -EOF -} if [ -d "$PULSAR_HOME/lib" ]; then PULSAR_CLASSPATH="$PULSAR_CLASSPATH:$PULSAR_HOME/lib/*" @@ -162,36 +131,4 @@ OPTS="$OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE" #Change to PULSAR_HOME to support relative paths cd "$PULSAR_HOME" -# if no args specified, show usage -if [ $# = 0 ]; then - pulsar_help; - exit 1; -fi - -# get arguments -COMMAND=$1 -shift - -if [ "$COMMAND" == "produce" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.PerformanceProducer --conf-file $PULSAR_PERFTEST_CONF "$@" -elif [ "$COMMAND" == "consume" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.PerformanceConsumer --conf-file $PULSAR_PERFTEST_CONF "$@" -elif [ "$COMMAND" == "transaction" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.PerformanceTransaction --conf-file $PULSAR_PERFTEST_CONF "$@" -elif [ "$COMMAND" == "read" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.PerformanceReader --conf-file $PULSAR_PERFTEST_CONF "$@" -elif [ "$COMMAND" == "monitor-brokers" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.BrokerMonitor "$@" -elif [ "$COMMAND" == "simulation-client" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.LoadSimulationClient "$@" -elif [ "$COMMAND" == "simulation-controller" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.LoadSimulationController "$@" -elif [ "$COMMAND" == "websocket-producer" ]; then - exec $JAVA $OPTS org.apache.pulsar.proxy.socket.client.PerformanceClient "$@" -elif [ "$COMMAND" == "managed-ledger" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.ManagedLedgerWriter "$@" -elif [ "$COMMAND" == "gen-doc" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.CmdGenerateDocumentation "$@" -else - pulsar_help; -fi +exec $JAVA $OPTS org.apache.pulsar.testclient.PulsarPerfTestTool $PULSAR_PERFTEST_CONF "$@" diff --git a/bin/pulsar-perf.cmd b/bin/pulsar-perf.cmd index cf6c25b77e59d..f2b33ef6eb88e 100644 --- a/bin/pulsar-perf.cmd +++ b/bin/pulsar-perf.cmd @@ -72,67 +72,7 @@ set "OPTS=%OPTS% -Dpulsar.log.level=%PULSAR_LOG_LEVEL%" set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_ROOT_LEVEL%" set "OPTS=%OPTS% -Dpulsar.log.immediateFlush=%PULSAR_LOG_IMMEDIATE_FLUSH%" -set "COMMAND=%1" - -for /f "tokens=1,* delims= " %%a in ("%*") do set "_args=%%b" - -if "%COMMAND%" == "produce" ( - call :execCmdWithConfigFile org.apache.pulsar.testclient.PerformanceProducer - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "consume" ( - call :execCmdWithConfigFile org.apache.pulsar.testclient.PerformanceConsumer - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "transaction" ( - call :execCmdWithConfigFile org.apache.pulsar.testclient.PerformanceTransaction - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "read" ( - call :execCmdWithConfigFile org.apache.pulsar.testclient.PerformanceReader - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "monitor-brokers" ( - call :execCmd org.apache.pulsar.testclient.BrokerMonitor - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "simulation-client" ( - call :execCmd org.apache.pulsar.testclient.LoadSimulationClient - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "simulation-controller" ( - call :execCmd org.apache.pulsar.testclient.LoadSimulationController - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "websocket-producer" ( - call :execCmd org.apache.pulsar.proxy.socket.client.PerformanceClient - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "managed-ledger" ( - call :execCmd org.apache.pulsar.testclient.ManagedLedgerWriter - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "gen-doc" ( - call :execCmd org.apache.pulsar.testclient.CmdGenerateDocumentation - exit /B %ERROR_CODE% -) - -call :usage -exit /B %ERROR_CODE% - -:execCmdWithConfigFile -"%JAVACMD%" %OPTS% %1 --conf-file "%PULSAR_PERFTEST_CONF%" %_args% -if ERRORLEVEL 1 ( - call :error -) -goto :eof - -:execCmd -"%JAVACMD%" %OPTS% %1 %_args% -if ERRORLEVEL 1 ( - call :error -) -goto :eof +"%JAVACMD%" %OPTS% org.apache.pulsar.testclient.PulsarPerfTestTool "%PULSAR_PERFTEST_CONF%" %* @@ -142,25 +82,3 @@ goto :eof - -:usage -echo Usage: pulsar-perf COMMAND -echo where command is one of: -echo produce Run a producer -echo consume Run a consumer -echo transaction Run a transaction repeatedly -echo read Run a topic reader -echo websocket-producer Run a websocket producer -echo managed-ledger Write directly on managed-ledgers -echo monitor-brokers Continuously receive broker data and/or load reports -echo simulation-client Run a simulation server acting as a Pulsar client -echo simulation-controller Run a simulation controller to give commands to servers -echo gen-doc Generate documentation automatically. -echo help This help message -echo or command is the full name of a class with a defined main() method. -echo Environment variables: -echo PULSAR_LOG_CONF Log4j configuration file (default %PULSAR_HOME%\logs) -echo PULSAR_CLIENT_CONF Configuration file for client (default: %PULSAR_HOME%\conf\client.conf) -echo PULSAR_EXTRA_OPTS Extra options to be passed to the jvm -echo PULSAR_EXTRA_CLASSPATH Add extra paths to the pulsar classpath -goto error diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java index 9d95d0b74a284..4d73fd9f9b4e3 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java @@ -49,6 +49,7 @@ import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.testclient.CmdBase; import org.apache.pulsar.testclient.IMessageFormatter; import org.apache.pulsar.testclient.PerfClientUtils; import org.apache.pulsar.testclient.PositiveNumberParameterConvert; @@ -60,173 +61,159 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine; import picocli.CommandLine.Command; +import picocli.CommandLine.Model.CommandSpec; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; import picocli.CommandLine.Parameters; +import picocli.CommandLine.Spec; -public class PerformanceClient { +@Command(name = "websocket-producer", description = "Test pulsar websocket producer performance.") +public class PerformanceClient extends CmdBase { private static final LongAdder messagesSent = new LongAdder(); private static final LongAdder bytesSent = new LongAdder(); private static final LongAdder totalMessagesSent = new LongAdder(); private static final LongAdder totalBytesSent = new LongAdder(); private static IMessageFormatter messageFormatter = null; - private CommandLine commander; - @Command(description = "Test pulsar websocket producer performance.") - static class Arguments { + @Option(names = { "-cf", "--conf-file" }, description = "Configuration file") + public String confFile; - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; + @Option(names = { "-u", "--proxy-url" }, description = "Pulsar Proxy URL, e.g., \"ws://localhost:8080/\"") + public String proxyURL; - @Option(names = { "-cf", "--conf-file" }, description = "Configuration file") - public String confFile; + @Parameters(description = "persistent://tenant/ns/my-topic", arity = "1") + public List topics; - @Option(names = { "-u", "--proxy-url" }, description = "Pulsar Proxy URL, e.g., \"ws://localhost:8080/\"") - public String proxyURL; + @Option(names = { "-r", "--rate" }, description = "Publish rate msg/s across topics") + public int msgRate = 100; - @Parameters(description = "persistent://tenant/ns/my-topic", arity = "1") - public List topics; + @Option(names = { "-s", "--size" }, description = "Message size in byte") + public int msgSize = 1024; - @Option(names = { "-r", "--rate" }, description = "Publish rate msg/s across topics") - public int msgRate = 100; + @Option(names = { "-t", "--num-topic" }, description = "Number of topics", + converter = PositiveNumberParameterConvert.class + ) + public int numTopics = 1; - @Option(names = { "-s", "--size" }, description = "Message size in byte") - public int msgSize = 1024; + @Option(names = { "--auth_plugin" }, description = "Authentication plugin class name", hidden = true) + public String deprecatedAuthPluginClassName; - @Option(names = { "-t", "--num-topic" }, description = "Number of topics", - converter = PositiveNumberParameterConvert.class - ) - public int numTopics = 1; + @Option(names = { "--auth-plugin" }, description = "Authentication plugin class name") + public String authPluginClassName; - @Option(names = { "--auth_plugin" }, description = "Authentication plugin class name", hidden = true) - public String deprecatedAuthPluginClassName; + @Option( + names = { "--auth-params" }, + description = "Authentication parameters, whose format is determined by the implementation " + + "of method `configure` in authentication plugin class, for example \"key1:val1,key2:val2\" " + + "or \"{\"key1\":\"val1\",\"key2\":\"val2\"}\".") + public String authParams; - @Option(names = { "--auth-plugin" }, description = "Authentication plugin class name") - public String authPluginClassName; + @Option(names = { "-m", + "--num-messages" }, description = "Number of messages to publish in total. If <= 0, it will keep" + + " publishing") + public long numMessages = 0; - @Option( - names = { "--auth-params" }, - description = "Authentication parameters, whose format is determined by the implementation " - + "of method `configure` in authentication plugin class, for example \"key1:val1,key2:val2\" " - + "or \"{\"key1\":\"val1\",\"key2\":\"val2\"}\".") - public String authParams; + @Option(names = { "-f", "--payload-file" }, description = "Use payload from a file instead of empty buffer") + public String payloadFilename = null; - @Option(names = { "-m", - "--num-messages" }, description = "Number of messages to publish in total. If <= 0, it will keep" - + " publishing") - public long numMessages = 0; + @Option(names = { "-e", "--payload-delimiter" }, + description = "The delimiter used to split lines when using payload from a file") + // here escaping \n since default value will be printed with the help text + public String payloadDelimiter = "\\n"; - @Option(names = { "-f", "--payload-file" }, description = "Use payload from a file instead of empty buffer") - public String payloadFilename = null; + @Option(names = { "-fp", "--format-payload" }, + description = "Format %%i as a message index in the stream from producer and/or %%t as the timestamp" + + " nanoseconds") + public boolean formatPayload = false; - @Option(names = { "-e", "--payload-delimiter" }, - description = "The delimiter used to split lines when using payload from a file") - // here escaping \n since default value will be printed with the help text - public String payloadDelimiter = "\\n"; + @Option(names = {"-fc", "--format-class"}, description = "Custom Formatter class name") + public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter"; - @Option(names = { "-fp", "--format-payload" }, - description = "Format %i as a message index in the stream from producer and/or %t as the timestamp" - + " nanoseconds") - public boolean formatPayload = false; + @Option(names = { "-time", + "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") + public long testTime = 0; - @Option(names = {"-fc", "--format-class"}, description = "Custom Formatter class name") - public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter"; - - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") - public long testTime = 0; + public PerformanceClient() { + super("websocket-producer"); } - public Arguments loadArguments(String[] args) { - Arguments arguments = new Arguments(); - commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf websocket-producer"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } - if (arguments.help) { - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Spec + CommandSpec spec; - if (isBlank(arguments.authPluginClassName) && !isBlank(arguments.deprecatedAuthPluginClassName)) { - arguments.authPluginClassName = arguments.deprecatedAuthPluginClassName; + public void loadArguments() { + CommandLine commander = spec.commandLine(); + + if (isBlank(this.authPluginClassName) && !isBlank(this.deprecatedAuthPluginClassName)) { + this.authPluginClassName = this.deprecatedAuthPluginClassName; } - if (arguments.topics.size() != 1) { + if (this.topics.size() != 1) { System.err.println("Only one topic name is allowed"); commander.usage(commander.getOut()); PerfClientUtils.exit(1); } - if (arguments.confFile != null) { + if (this.confFile != null) { Properties prop = new Properties(System.getProperties()); try { - prop.load(new FileInputStream(arguments.confFile)); + prop.load(new FileInputStream(this.confFile)); } catch (IOException e) { log.error("Error in loading config file"); commander.usage(commander.getOut()); PerfClientUtils.exit(1); } - if (isBlank(arguments.proxyURL)) { + if (isBlank(this.proxyURL)) { String webSocketServiceUrl = prop.getProperty("webSocketServiceUrl"); if (isNotBlank(webSocketServiceUrl)) { - arguments.proxyURL = webSocketServiceUrl; + this.proxyURL = webSocketServiceUrl; } else { String webServiceUrl = isNotBlank(prop.getProperty("webServiceUrl")) ? prop.getProperty("webServiceUrl") : prop.getProperty("serviceUrl"); if (isNotBlank(webServiceUrl)) { if (webServiceUrl.startsWith("ws://") || webServiceUrl.startsWith("wss://")) { - arguments.proxyURL = webServiceUrl; + this.proxyURL = webServiceUrl; } else if (webServiceUrl.startsWith("http://") || webServiceUrl.startsWith("https://")) { - arguments.proxyURL = webServiceUrl.replaceFirst("^http", "ws"); + this.proxyURL = webServiceUrl.replaceFirst("^http", "ws"); } } } } - if (arguments.authPluginClassName == null) { - arguments.authPluginClassName = prop.getProperty("authPlugin", null); + if (this.authPluginClassName == null) { + this.authPluginClassName = prop.getProperty("authPlugin", null); } - if (arguments.authParams == null) { - arguments.authParams = prop.getProperty("authParams", null); + if (this.authParams == null) { + this.authParams = prop.getProperty("authParams", null); } } - if (isBlank(arguments.proxyURL)) { - arguments.proxyURL = "ws://localhost:8080/"; + if (isBlank(this.proxyURL)) { + this.proxyURL = "ws://localhost:8080/"; } - if (!arguments.proxyURL.endsWith("/")) { - arguments.proxyURL += "/"; + if (!this.proxyURL.endsWith("/")) { + this.proxyURL += "/"; } - return arguments; - } - public void runPerformanceTest(Arguments arguments) throws InterruptedException, IOException { + public void runPerformanceTest() throws InterruptedException, IOException { // Read payload data from file if needed - final byte[] payloadBytes = new byte[arguments.msgSize]; + final byte[] payloadBytes = new byte[this.msgSize]; Random random = new Random(0); List payloadByteList = new ArrayList<>(); - if (arguments.payloadFilename != null) { - Path payloadFilePath = Paths.get(arguments.payloadFilename); + if (this.payloadFilename != null) { + Path payloadFilePath = Paths.get(this.payloadFilename); if (Files.notExists(payloadFilePath) || Files.size(payloadFilePath) == 0) { throw new IllegalArgumentException("Payload file doesn't exist or it is empty."); } // here escaping the default payload delimiter to correct value - String delimiter = arguments.payloadDelimiter.equals("\\n") ? "\n" : arguments.payloadDelimiter; + String delimiter = this.payloadDelimiter.equals("\\n") ? "\n" : this.payloadDelimiter; String[] payloadList = new String(Files.readAllBytes(payloadFilePath), StandardCharsets.UTF_8) .split(delimiter); log.info("Reading payloads from {} and {} records read", payloadFilePath.toAbsolutePath(), @@ -235,8 +222,8 @@ public void runPerformanceTest(Arguments arguments) throws InterruptedException, payloadByteList.add(payload.getBytes(StandardCharsets.UTF_8)); } - if (arguments.formatPayload) { - messageFormatter = getMessageFormatter(arguments.formatterClass); + if (this.formatPayload) { + messageFormatter = getMessageFormatter(this.formatterClass); } } else { for (int i = 0; i < payloadBytes.length; ++i) { @@ -248,21 +235,21 @@ public void runPerformanceTest(Arguments arguments) throws InterruptedException, ExecutorService executor = Executors.newCachedThreadPool( new DefaultThreadFactory("pulsar-perf-producer-exec")); HashMap producersMap = new HashMap<>(); - String topicName = arguments.topics.get(0); + String topicName = this.topics.get(0); String restPath = TopicName.get(topicName).getRestPath(); String produceBaseEndPoint = TopicName.get(topicName).isV2() - ? arguments.proxyURL + "ws/v2/producer/" + restPath : arguments.proxyURL + "ws/producer/" + restPath; - for (int i = 0; i < arguments.numTopics; i++) { - String topic = arguments.numTopics > 1 ? produceBaseEndPoint + i : produceBaseEndPoint; + ? this.proxyURL + "ws/v2/producer/" + restPath : this.proxyURL + "ws/producer/" + restPath; + for (int i = 0; i < this.numTopics; i++) { + String topic = this.numTopics > 1 ? produceBaseEndPoint + i : produceBaseEndPoint; URI produceUri = URI.create(topic); WebSocketClient produceClient = new WebSocketClient(new SslContextFactory(true)); ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - if (StringUtils.isNotBlank(arguments.authPluginClassName) && StringUtils.isNotBlank(arguments.authParams)) { + if (StringUtils.isNotBlank(this.authPluginClassName) && StringUtils.isNotBlank(this.authParams)) { try { - Authentication auth = AuthenticationFactory.create(arguments.authPluginClassName, - arguments.authParams); + Authentication auth = AuthenticationFactory.create(this.authPluginClassName, + this.authParams); auth.start(); AuthenticationDataProvider authData = auth.getAuthData(); if (authData.hasDataForHttp()) { @@ -296,23 +283,23 @@ public void runPerformanceTest(Arguments arguments) throws InterruptedException, executor.submit(() -> { try { - RateLimiter rateLimiter = RateLimiter.create(arguments.msgRate); + RateLimiter rateLimiter = RateLimiter.create(this.msgRate); long startTime = System.nanoTime(); - long testEndTime = startTime + (long) (arguments.testTime * 1e9); + long testEndTime = startTime + (long) (this.testTime * 1e9); // Send messages on all topics/producers long totalSent = 0; while (true) { for (String topic : producersMap.keySet()) { - if (arguments.testTime > 0 && System.nanoTime() > testEndTime) { + if (this.testTime > 0 && System.nanoTime() > testEndTime) { log.info("------------- DONE (reached the maximum duration: [{} seconds] of production) " - + "--------------", arguments.testTime); + + "--------------", this.testTime); PerfClientUtils.exit(0); } - if (arguments.numMessages > 0) { - if (totalSent >= arguments.numMessages) { + if (this.numMessages > 0) { + if (totalSent >= this.numMessages) { log.trace("------------- DONE (reached the maximum number: [{}] of production) " - + "--------------", arguments.numMessages); + + "--------------", this.numMessages); Thread.sleep(10000); PerfClientUtils.exit(0); } @@ -326,7 +313,7 @@ public void runPerformanceTest(Arguments arguments) throws InterruptedException, } byte[] payloadData; - if (arguments.payloadFilename != null) { + if (this.payloadFilename != null) { if (messageFormatter != null) { payloadData = messageFormatter.formatMessage("", totalSent, payloadByteList.get(random.nextInt(payloadByteList.size()))); @@ -416,16 +403,16 @@ static IMessageFormatter getMessageFormatter(String formatterClass) { } } - public static void main(String[] args) throws Exception { - PerformanceClient test = new PerformanceClient(); - Arguments arguments = test.loadArguments(args); + @Override + public void run() throws Exception { + loadArguments(); PerfClientUtils.printJVMInformation(log); long start = System.nanoTime(); Runtime.getRuntime().addShutdownHook(new Thread(() -> { printAggregatedThroughput(start); printAggregatedStats(); })); - test.runPerformanceTest(arguments); + runPerformanceTest(); } private class Tuple { diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java index d195e8fd45695..a2f5b382c7b8f 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java @@ -46,17 +46,17 @@ import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; -import picocli.CommandLine.ScopeType; /** * Monitors brokers and prints to the console information about their system resource usages, their topic and bundle * counts, their message rates, and other metrics. */ -public class BrokerMonitor { +@Command(name = "monitor-brokers", + description = "Monitors brokers and prints to the console information about their system " + + "resource usages, \ntheir topic and bundle counts, their message rates, and other metrics.") +public class BrokerMonitor extends CmdBase { private static final Logger log = LoggerFactory.getLogger(BrokerMonitor.class); private static final String BROKER_ROOT = "/loadbalance/brokers"; @@ -88,6 +88,7 @@ public class BrokerMonitor { private Map loadData; private static final FixedColumnLengthTableMaker localTableMaker = new FixedColumnLengthTableMaker(); + static { // Makes the table length about 120. localTableMaker.elementLength = 14; @@ -95,6 +96,7 @@ public class BrokerMonitor { } private static final FixedColumnLengthTableMaker globalTableMaker = new FixedColumnLengthTableMaker(); + static { globalTableMaker.decimalFormatter = "%.2f"; globalTableMaker.topBorder = '*'; @@ -126,7 +128,7 @@ private static void initRow(final Object[] row, final Object... elements) { // Helper method to initialize rows which hold message data. private static void initMessageRow(final Object[] row, final double messageRateIn, final double messageRateOut, - final double messageThroughputIn, final double messageThroughputOut) { + final double messageThroughputIn, final double messageThroughputOut) { initRow(row, messageRateIn, messageRateOut, messageRateIn + messageRateOut, messageThroughputIn / 1024, messageThroughputOut / 1024, (messageThroughputIn + messageThroughputOut) / 1024); @@ -391,7 +393,7 @@ private synchronized void printLoadReport(final String broker, final LoadReport // Print the broker data in a tabular form for a broker using ModularLoadManagerImpl. private synchronized void printBrokerData(final String broker, final LocalBrokerData localBrokerData, - final TimeAverageBrokerData timeAverageData) { + final TimeAverageBrokerData timeAverageData) { loadData.put(broker, localBrokerData); // Initialize the constant rows. @@ -435,19 +437,15 @@ private synchronized void printBrokerData(final String broker, final LocalBroker } } - // picocli arguments class. - @Command(description = "Monitors brokers and prints to the console information about their system " - + "resource usages, \ntheir topic and bundle counts, their message rates, and other metrics.", - showDefaultValues = true, scope = ScopeType.INHERIT) - private static class Arguments { - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; + @Option(names = {"--connect-string"}, description = "Zookeeper or broker connect string", required = true) + public String connectString = null; + + @Option(names = {"--extensions"}, description = "true to monitor Load Balance Extensions.") + boolean extensions = false; - @Option(names = { "--connect-string" }, description = "Zookeeper or broker connect string", required = true) - public String connectString = null; - @Option(names = { "--extensions" }, description = "true to monitor Load Balance Extensions.") - boolean extensions = false; + public BrokerMonitor() { + super("monitor-brokers"); } /** @@ -456,6 +454,7 @@ private static class Arguments { * @param zkClient Client to create this from. */ public BrokerMonitor(final ZooKeeper zkClient) { + super("monitor-brokers"); loadData = new ConcurrentHashMap<>(); this.zkClient = zkClient; } @@ -479,6 +478,7 @@ public void start() { private TableView brokerLoadDataTableView; private BrokerMonitor(String brokerServiceUrl) { + super("monitor-brokers"); try { PulsarClient client = PulsarClient.builder() .memoryLimit(0, SizeUnit.BYTES) @@ -541,32 +541,16 @@ private void startBrokerLoadDataStoreMonitor() { } } - /** - * Run a monitor from command line arguments. - * - * @param args Arguments for the monitor. - */ - public static void main(String[] args) throws Exception { - final Arguments arguments = new Arguments(); - final CommandLine commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf monitor-brokers"); - - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } - - - if (arguments.extensions) { - final BrokerMonitor monitor = new BrokerMonitor(arguments.connectString); + @Override + public void run() throws Exception { + if (this.extensions) { + final BrokerMonitor monitor = new BrokerMonitor(this.connectString); monitor.startBrokerLoadDataStoreMonitor(); } else { - final ZooKeeper zkClient = new ZooKeeper(arguments.connectString, ZOOKEEPER_TIMEOUT_MILLIS, null); + final ZooKeeper zkClient = new ZooKeeper(this.connectString, ZOOKEEPER_TIMEOUT_MILLIS, null); final BrokerMonitor monitor = new BrokerMonitor(zkClient); monitor.start(); } } + } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdBase.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdBase.java new file mode 100644 index 0000000000000..6d5796ad5dda7 --- /dev/null +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdBase.java @@ -0,0 +1,80 @@ +/* + * 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 org.apache.pulsar.testclient; + +import java.util.concurrent.Callable; +import picocli.CommandLine; + +public abstract class CmdBase implements Callable { + private final CommandLine commander; + + public CmdBase(String cmdName) { + commander = new CommandLine(this); + commander.setCommandName(cmdName); + } + + public boolean run(String[] args) { + return commander.execute(args) == 0; + } + + public void parse(String[] args) { + commander.parseArgs(args); + } + + /** + * Validate the CLI arguments. Default implementation provides validation for the common arguments. + * Each subclass should call super.validate() and provide validation code specific to the sub-command. + * @throws Exception + */ + public void validate() throws Exception { + } + + // Picocli entrypoint. + @Override + public Integer call() throws Exception { + validate(); + run(); + return 0; + } + + public abstract void run() throws Exception; + + + protected CommandLine getCommander() { + return commander; + } + + protected void addCommand(String name, Object cmd) { + commander.addSubcommand(name, cmd); + } + + protected void addCommand(String name, Object cmd, String... aliases) { + commander.addSubcommand(name, cmd, aliases); + } + + protected class ParameterException extends CommandLine.ParameterException { + public ParameterException(String msg) { + super(commander, msg); + } + + public ParameterException(String msg, Throwable e) { + super(commander, msg, e); + } + } +} diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdGenerateDocumentation.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdGenerateDocumentation.java index 6ff0ab296a684..d2e08e2cc8664 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdGenerateDocumentation.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdGenerateDocumentation.java @@ -24,57 +24,41 @@ import java.util.List; import java.util.Map; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.proxy.socket.client.PerformanceClient; import picocli.CommandLine; import picocli.CommandLine.Command; +import picocli.CommandLine.Model.CommandSpec; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; -import picocli.CommandLine.ScopeType; +import picocli.CommandLine.Spec; @Slf4j -public class CmdGenerateDocumentation { +@Command(name = "gen-doc", description = "Generate documentation automatically.") +public class CmdGenerateDocumentation extends CmdBase{ - @Command(description = "Generate documentation automatically.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments { - - @Option(names = {"-h", "--help"}, description = "Help message", help = true) - boolean help; - - @Option(names = {"-n", "--command-names"}, description = "List of command names") - private List commandNames = new ArrayList<>(); + @Option(names = {"-n", "--command-names"}, description = "List of command names") + private List commandNames = new ArrayList<>(); + public CmdGenerateDocumentation() { + super("gen-doc"); } - public static void main(String[] args) throws Exception { - final Arguments arguments = new Arguments(); - CommandLine commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf gen-doc"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Spec + CommandSpec spec; - - if (arguments.help) { - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Override + public void run() throws Exception { + CommandLine commander = spec.commandLine(); Map> cmdClassMap = new LinkedHashMap<>(); - cmdClassMap.put("produce", Class.forName("org.apache.pulsar.testclient.PerformanceProducer$Arguments")); - cmdClassMap.put("consume", Class.forName("org.apache.pulsar.testclient.PerformanceConsumer$Arguments")); - cmdClassMap.put("transaction", Class.forName("org.apache.pulsar.testclient.PerformanceTransaction$Arguments")); - cmdClassMap.put("read", Class.forName("org.apache.pulsar.testclient.PerformanceReader$Arguments")); - cmdClassMap.put("monitor-brokers", Class.forName("org.apache.pulsar.testclient.BrokerMonitor$Arguments")); - cmdClassMap.put("simulation-client", - Class.forName("org.apache.pulsar.testclient.LoadSimulationClient$MainArguments")); - cmdClassMap.put("simulation-controller", - Class.forName("org.apache.pulsar.testclient.LoadSimulationController$MainArguments")); - cmdClassMap.put("websocket-producer", - Class.forName("org.apache.pulsar.proxy.socket.client.PerformanceClient$Arguments")); - cmdClassMap.put("managed-ledger", Class.forName("org.apache.pulsar.testclient.ManagedLedgerWriter$Arguments")); + cmdClassMap.put("produce", PerformanceProducer.class); + cmdClassMap.put("consume", PerformanceConsumer.class); + cmdClassMap.put("transaction", PerformanceTransaction.class); + cmdClassMap.put("read", PerformanceReader.class); + cmdClassMap.put("monitor-brokers", BrokerMonitor.class); + cmdClassMap.put("simulation-client", LoadSimulationClient.class); + cmdClassMap.put("simulation-controller", LoadSimulationController.class); + cmdClassMap.put("websocket-producer", PerformanceClient.class); + cmdClassMap.put("managed-ledger", ManagedLedgerWriter.class); for (Map.Entry> entry : cmdClassMap.entrySet()) { String cmd = entry.getKey(); @@ -84,12 +68,12 @@ public static void main(String[] args) throws Exception { commander.addSubcommand(cmd, constructor.newInstance()); } - if (arguments.commandNames.size() == 0) { + if (this.commandNames.size() == 0) { for (Map.Entry cmd : commander.getSubcommands().entrySet()) { generateDocument(cmd.getKey(), commander); } } else { - for (String commandName : arguments.commandNames) { + for (String commandName : this.commandNames) { generateDocument(commandName, commander); } } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java index 42d2f0dd5143e..c58de64056a66 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java @@ -41,20 +41,20 @@ import org.apache.pulsar.client.api.MessageListener; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SizeUnit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; -import picocli.CommandLine.ScopeType; /** * LoadSimulationClient is used to simulate client load by maintaining producers and consumers for topics. Instances of * this class are controlled across a network via LoadSimulationController. */ -public class LoadSimulationClient { +@Command(name = "simulation-client", + description = "Simulate client load by maintaining producers and consumers for topics.") +public class LoadSimulationClient extends CmdBase{ private static final Logger log = LoggerFactory.getLogger(LoadSimulationClient.class); // Values for command encodings. @@ -65,7 +65,7 @@ public class LoadSimulationClient { public static final byte STOP_GROUP_COMMAND = 4; public static final byte FIND_COMMAND = 5; - private final ExecutorService executor; + private ExecutorService executor; // Map from a message size to a cached byte[] of that size. private final Map payloadCache; @@ -73,12 +73,10 @@ public class LoadSimulationClient { private final Map topicsToTradeUnits; // Pulsar admin to create namespaces with. - private final PulsarAdmin admin; + private PulsarAdmin admin; // Pulsar client to create producers and consumers with. - private final PulsarClient client; - - private final int port; + private PulsarClient client; // A TradeUnit is a Consumer and Producer pair. The rate of message // consumption as well as size may be changed at @@ -172,22 +170,17 @@ public void start() throws Exception { } // picocli arguments for starting a LoadSimulationClient. - @Command(description = "Simulate client load by maintaining producers and consumers for topics.", - showDefaultValues = true, scope = ScopeType.INHERIT) - private static class MainArguments { - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; - @Option(names = { "--port" }, description = "Port to listen on for controller", required = true) - public int port; + @Option(names = { "--port" }, description = "Port to listen on for controller", required = true) + public int port; - @Option(names = { "--service-url" }, description = "Pulsar Service URL", required = true) - public String serviceURL; + @Option(names = { "--service-url" }, description = "Pulsar Service URL", required = true) + public String serviceURL; + + @Option(names = { "-ml", "--memory-limit", }, description = "Configure the Pulsar client memory limit " + + "(eg: 32M, 64M)", converter = ByteUnitToLongConverter.class) + public long memoryLimit = 0L; - @Option(names = { "-ml", "--memory-limit", }, description = "Configure the Pulsar client memory limit " - + "(eg: 32M, 64M)", converter = ByteUnitToLongConverter.class) - public long memoryLimit = 0L; - } // Configuration class for initializing or modifying TradeUnits. private static class TradeConfiguration { @@ -312,54 +305,40 @@ private void handle(final byte command, final DataInputStream inputStream, final private static final MessageListener ackListener = Consumer::acknowledgeAsync; /** - * Create a LoadSimulationClient with the given picocli arguments. + * Create a LoadSimulationClient with the given picocli this. * - * @param arguments - * Arguments to configure this from. */ - public LoadSimulationClient(final MainArguments arguments) throws Exception { + public LoadSimulationClient() throws PulsarClientException { + super("simulation-client"); payloadCache = new ConcurrentHashMap<>(); topicsToTradeUnits = new ConcurrentHashMap<>(); - - admin = PulsarAdmin.builder() - .serviceHttpUrl(arguments.serviceURL) - .build(); - client = PulsarClient.builder() - .memoryLimit(arguments.memoryLimit, SizeUnit.BYTES) - .serviceUrl(arguments.serviceURL) - .connectionsPerBroker(4) - .ioThreads(Runtime.getRuntime().availableProcessors()) - .statsInterval(0, TimeUnit.SECONDS) - .build(); - port = arguments.port; - executor = Executors.newCachedThreadPool(new DefaultThreadFactory("test-client")); } /** - * Start a client with command line arguments. + * Start a client with command line this. * - * @param args - * Command line arguments to pass in. */ - public static void main(String[] args) throws Exception { - final MainArguments mainArguments = new MainArguments(); - CommandLine commander = new CommandLine(mainArguments); - commander.setCommandName("pulsar-perf simulation-client"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Override + public void run() throws Exception { + admin = PulsarAdmin.builder() + .serviceHttpUrl(this.serviceURL) + .build(); + client = PulsarClient.builder() + .memoryLimit(this.memoryLimit, SizeUnit.BYTES) + .serviceUrl(this.serviceURL) + .connectionsPerBroker(4) + .ioThreads(Runtime.getRuntime().availableProcessors()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + executor = Executors.newCachedThreadPool(new DefaultThreadFactory("test-client")); PerfClientUtils.printJVMInformation(log); - (new LoadSimulationClient(mainArguments)).run(); + this.start(); } /** * Start listening for controller commands to create producers and consumers. */ - public void run() throws Exception { + public void start() throws Exception { final ServerSocket serverSocket = new ServerSocket(port); while (true) { diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java index 94186c581ebe4..99f443f26d7d2 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java @@ -55,51 +55,41 @@ import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; import picocli.CommandLine.Parameters; -import picocli.CommandLine.ScopeType; /** * This class provides a shell for the user to dictate how simulation clients should incur load. */ -public class LoadSimulationController { +@Command(name = "simulation-controller", + description = "Provides a shell for the user to dictate how simulation clients should " + + "incur load.") +public class LoadSimulationController extends CmdBase{ private static final Logger log = LoggerFactory.getLogger(LoadSimulationController.class); // Input streams for each client to send commands through. - private final DataInputStream[] inputStreams; + private DataInputStream[] inputStreams; // Output streams for each client to receive information from. - private final DataOutputStream[] outputStreams; + private DataOutputStream[] outputStreams; // client host names. - private final String[] clients; + private String[] clients; - // Port clients are listening on. - private final int clientPort; - - // The ZooKeeper cluster to run on. - private final String cluster; - - private final Random random; + private Random random; private static final ExecutorService threadPool = Executors.newCachedThreadPool(); // picocli arguments for starting a controller via main. - @Command(description = "Provides a shell for the user to dictate how simulation clients should " - + "incur load.", showDefaultValues = true, scope = ScopeType.INHERIT) - private static class MainArguments { - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; - @Option(names = { "--cluster" }, description = "Cluster to test on", required = true) - String cluster; + @Option(names = { "--cluster" }, description = "Cluster to test on", required = true) + String cluster; - @Option(names = { "--clients" }, description = "Comma separated list of client hostnames", required = true) - String clientHostNames; + @Option(names = { "--clients" }, description = "Comma separated list of client hostnames", required = true) + String clientHostNames; + + @Option(names = { "--client-port" }, description = "Port that the clients are listening on", required = true) + int clientPort; - @Option(names = { "--client-port" }, description = "Port that the clients are listening on", required = true) - int clientPort; - } // picocli arguments for accepting user input. private static class ShellArguments { @@ -216,24 +206,9 @@ public synchronized void process(final WatchedEvent event) { /** * Create a LoadSimulationController with the given picocli arguments. * - * @param arguments - * Arguments to create from. */ - public LoadSimulationController(final MainArguments arguments) throws Exception { - random = new Random(); - clientPort = arguments.clientPort; - cluster = arguments.cluster; - clients = arguments.clientHostNames.split(","); - final Socket[] sockets = new Socket[clients.length]; - inputStreams = new DataInputStream[clients.length]; - outputStreams = new DataOutputStream[clients.length]; - log.info("Found {} clients:", clients.length); - for (int i = 0; i < clients.length; ++i) { - sockets[i] = new Socket(clients[i], clientPort); - inputStreams[i] = new DataInputStream(sockets[i].getInputStream()); - outputStreams[i] = new DataOutputStream(sockets[i].getOutputStream()); - log.info("Connected to {}", clients[i]); - } + public LoadSimulationController() throws Exception { + super("simulation-controller"); } // Check that the expected number of application arguments matches the @@ -700,7 +675,7 @@ private void read(final String[] args) { /** * Create a shell for the user to send commands to clients. */ - public void run() throws Exception { + public void start() throws Exception { BufferedReader inReader = new BufferedReader(new InputStreamReader(System.in)); while (true) { // Print the very simple prompt. @@ -713,20 +688,21 @@ public void run() throws Exception { /** * Start a controller with command line arguments. * - * @param args - * Arguments to pass in. */ - public static void main(String[] args) throws Exception { - final MainArguments arguments = new MainArguments(); - final CommandLine commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf simulation-controller"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); + @Override + public void run() throws Exception { + random = new Random(); + clients = this.clientHostNames.split(","); + final Socket[] sockets = new Socket[clients.length]; + inputStreams = new DataInputStream[clients.length]; + outputStreams = new DataOutputStream[clients.length]; + log.info("Found {} clients:", clients.length); + for (int i = 0; i < clients.length; ++i) { + sockets[i] = new Socket(clients[i], clientPort); + inputStreams[i] = new DataInputStream(sockets[i].getInputStream()); + outputStreams[i] = new DataOutputStream(sockets[i].getOutputStream()); + log.info("Connected to {}", clients[i]); } - (new LoadSimulationController(arguments)).run(); + start(); } } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java index bad8e56a638b6..8913d17474279 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java @@ -62,11 +62,12 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine; import picocli.CommandLine.Command; +import picocli.CommandLine.Model.CommandSpec; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; -import picocli.CommandLine.ScopeType; +import picocli.CommandLine.Spec; -public class ManagedLedgerWriter { +@Command(name = "managed-ledger", description = "Write directly on managed-ledgers") +public class ManagedLedgerWriter extends CmdBase{ private static final ExecutorService executor = Executors .newCachedThreadPool(new DefaultThreadFactory("pulsar-perf-managed-ledger-exec")); @@ -79,86 +80,72 @@ public class ManagedLedgerWriter { private static Recorder recorder = new Recorder(TimeUnit.SECONDS.toMillis(120000), 5); private static Recorder cumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMillis(120000), 5); - @Command(description = "Write directly on managed-ledgers", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments { - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; + @Option(names = { "-r", "--rate" }, description = "Write rate msg/s across managed ledgers") + public int msgRate = 100; - @Option(names = { "-r", "--rate" }, description = "Write rate msg/s across managed ledgers") - public int msgRate = 100; + @Option(names = { "-s", "--size" }, description = "Message size") + public int msgSize = 1024; - @Option(names = { "-s", "--size" }, description = "Message size") - public int msgSize = 1024; + @Option(names = { "-t", "--num-topic" }, + description = "Number of managed ledgers", converter = PositiveNumberParameterConvert.class) + public int numManagedLedgers = 1; - @Option(names = { "-t", "--num-topic" }, - description = "Number of managed ledgers", converter = PositiveNumberParameterConvert.class) - public int numManagedLedgers = 1; + @Option(names = { "--threads" }, + description = "Number of threads writing", converter = PositiveNumberParameterConvert.class) + public int numThreads = 1; - @Option(names = { "--threads" }, - description = "Number of threads writing", converter = PositiveNumberParameterConvert.class) - public int numThreads = 1; + @Deprecated + @Option(names = {"-zk", "--zookeeperServers"}, + description = "ZooKeeper connection string", + hidden = true) + public String zookeeperServers; - @Deprecated - @Option(names = {"-zk", "--zookeeperServers"}, - description = "ZooKeeper connection string", - hidden = true) - public String zookeeperServers; + @Option(names = {"-md", + "--metadata-store"}, description = "Metadata store service URL. For example: zk:my-zk:2181") + private String metadataStoreUrl; - @Option(names = {"-md", - "--metadata-store"}, description = "Metadata store service URL. For example: zk:my-zk:2181") - private String metadataStoreUrl; + @Option(names = { "-o", "--max-outstanding" }, description = "Max number of outstanding requests") + public int maxOutstanding = 1000; - @Option(names = { "-o", "--max-outstanding" }, description = "Max number of outstanding requests") - public int maxOutstanding = 1000; + @Option(names = { "-c", + "--max-connections" }, description = "Max number of TCP connections to a single bookie") + public int maxConnections = 1; - @Option(names = { "-c", - "--max-connections" }, description = "Max number of TCP connections to a single bookie") - public int maxConnections = 1; + @Option(names = { "-m", + "--num-messages" }, + description = "Number of messages to publish in total. If <= 0, it will keep publishing") + public long numMessages = 0; - @Option(names = { "-m", - "--num-messages" }, - description = "Number of messages to publish in total. If <= 0, it will keep publishing") - public long numMessages = 0; + @Option(names = { "-e", "--ensemble-size" }, description = "Ledger ensemble size") + public int ensembleSize = 1; - @Option(names = { "-e", "--ensemble-size" }, description = "Ledger ensemble size") - public int ensembleSize = 1; + @Option(names = { "-w", "--write-quorum" }, description = "Ledger write quorum") + public int writeQuorum = 1; - @Option(names = { "-w", "--write-quorum" }, description = "Ledger write quorum") - public int writeQuorum = 1; + @Option(names = { "-a", "--ack-quorum" }, description = "Ledger ack quorum") + public int ackQuorum = 1; - @Option(names = { "-a", "--ack-quorum" }, description = "Ledger ack quorum") - public int ackQuorum = 1; + @Option(names = { "-dt", "--digest-type" }, description = "BookKeeper digest type") + public DigestType digestType = DigestType.CRC32C; - @Option(names = { "-dt", "--digest-type" }, description = "BookKeeper digest type") - public DigestType digestType = DigestType.CRC32C; - - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") - public long testTime = 0; + @Option(names = { "-time", + "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") + public long testTime = 0; + public ManagedLedgerWriter() { + super("managed-ledger"); } - public static void main(String[] args) throws Exception { - - final Arguments arguments = new Arguments(); - CommandLine commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf managed-ledger"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Spec + CommandSpec spec; - if (arguments.help) { - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Override + public void run() throws Exception { + CommandLine commander = spec.commandLine(); - if (arguments.metadataStoreUrl == null && arguments.zookeeperServers == null) { + if (this.metadataStoreUrl == null && this.zookeeperServers == null) { System.err.println("Metadata store address argument is required (--metadata-store)"); commander.usage(commander.getOut()); PerfClientUtils.exit(1); @@ -168,17 +155,17 @@ public static void main(String[] args) throws Exception { PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar managed-ledger perf writer with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar managed-ledger perf writer with config: {}", w.writeValueAsString(this)); - byte[] payloadData = new byte[arguments.msgSize]; - ByteBuf payloadBuffer = PulsarByteBufAllocator.DEFAULT.directBuffer(arguments.msgSize); - payloadBuffer.writerIndex(arguments.msgSize); + byte[] payloadData = new byte[this.msgSize]; + ByteBuf payloadBuffer = PulsarByteBufAllocator.DEFAULT.directBuffer(this.msgSize); + payloadBuffer.writerIndex(this.msgSize); // Now processing command line arguments String managedLedgerPrefix = "test-" + DigestUtils.sha1Hex(UUID.randomUUID().toString()).substring(0, 5); - if (arguments.metadataStoreUrl == null) { - arguments.metadataStoreUrl = arguments.zookeeperServers; + if (this.metadataStoreUrl == null) { + this.metadataStoreUrl = this.zookeeperServers; } ClientConfiguration bkConf = new ClientConfiguration(); @@ -186,31 +173,31 @@ public static void main(String[] args) throws Exception { bkConf.setAddEntryTimeout(30); bkConf.setReadEntryTimeout(30); bkConf.setThrottleValue(0); - bkConf.setNumChannelsPerBookie(arguments.maxConnections); - bkConf.setMetadataServiceUri(arguments.metadataStoreUrl); + bkConf.setNumChannelsPerBookie(this.maxConnections); + bkConf.setMetadataServiceUri(this.metadataStoreUrl); ManagedLedgerFactoryConfig mlFactoryConf = new ManagedLedgerFactoryConfig(); mlFactoryConf.setMaxCacheSize(0); @Cleanup - MetadataStoreExtended metadataStore = MetadataStoreExtended.create(arguments.metadataStoreUrl, + MetadataStoreExtended metadataStore = MetadataStoreExtended.create(this.metadataStoreUrl, MetadataStoreConfig.builder().metadataStoreName(MetadataStoreConfig.METADATA_STORE).build()); ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkConf, mlFactoryConf); ManagedLedgerConfig mlConf = new ManagedLedgerConfig(); - mlConf.setEnsembleSize(arguments.ensembleSize); - mlConf.setWriteQuorumSize(arguments.writeQuorum); - mlConf.setAckQuorumSize(arguments.ackQuorum); + mlConf.setEnsembleSize(this.ensembleSize); + mlConf.setWriteQuorumSize(this.writeQuorum); + mlConf.setAckQuorumSize(this.ackQuorum); mlConf.setMinimumRolloverTime(10, TimeUnit.MINUTES); - mlConf.setMetadataEnsembleSize(arguments.ensembleSize); - mlConf.setMetadataWriteQuorumSize(arguments.writeQuorum); - mlConf.setMetadataAckQuorumSize(arguments.ackQuorum); - mlConf.setDigestType(arguments.digestType); + mlConf.setMetadataEnsembleSize(this.ensembleSize); + mlConf.setMetadataWriteQuorumSize(this.writeQuorum); + mlConf.setMetadataAckQuorumSize(this.ackQuorum); + mlConf.setDigestType(this.digestType); mlConf.setMaxSizePerLedgerMb(2048); List> futures = new ArrayList<>(); - for (int i = 0; i < arguments.numManagedLedgers; i++) { + for (int i = 0; i < this.numManagedLedgers; i++) { String name = String.format("%s-%03d", managedLedgerPrefix, i); CompletableFuture future = new CompletableFuture<>(); futures.add(future); @@ -242,23 +229,23 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { AtomicBoolean isDone = new AtomicBoolean(); Map> managedLedgersPerThread = allocateToThreads(managedLedgers, - arguments.numThreads); + this.numThreads); - for (int i = 0; i < arguments.numThreads; i++) { + for (int i = 0; i < this.numThreads; i++) { List managedLedgersForThisThread = managedLedgersPerThread.get(i); int nunManagedLedgersForThisThread = managedLedgersForThisThread.size(); - long numMessagesForThisThread = arguments.numMessages / arguments.numThreads; - int maxOutstandingForThisThread = arguments.maxOutstanding; + long numMessagesForThisThread = this.numMessages / this.numThreads; + int maxOutstandingForThisThread = this.maxOutstanding; executor.submit(() -> { try { - final double msgRate = arguments.msgRate / (double) arguments.numThreads; + final double msgRate = this.msgRate / (double) this.numThreads; final RateLimiter rateLimiter = RateLimiter.create(msgRate); // Acquire 1 sec worth of messages to have a slower ramp-up rateLimiter.acquire((int) msgRate); final long startTime = System.nanoTime(); - final long testEndTime = startTime + (long) (arguments.testTime * 1e9); + final long testEndTime = startTime + (long) (this.testTime * 1e9); final Semaphore semaphore = new Semaphore(maxOutstandingForThisThread); @@ -289,10 +276,10 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { long totalSent = 0; while (true) { for (int j = 0; j < nunManagedLedgersForThisThread; j++) { - if (arguments.testTime > 0) { + if (this.testTime > 0) { if (System.nanoTime() > testEndTime) { log.info("------------- DONE (reached the maximum duration: [{} seconds] of " - + "production) --------------", arguments.testTime); + + "production) --------------", this.testTime); isDone.set(true); Thread.sleep(5000); PerfClientUtils.exit(0); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java index d320cafc1a08f..3c4b831332281 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java @@ -19,53 +19,44 @@ package org.apache.pulsar.testclient; import static org.apache.commons.lang3.StringUtils.isBlank; -import static org.apache.pulsar.testclient.PerfClientUtils.exit; -import java.io.File; -import java.io.FileInputStream; -import java.util.Properties; -import lombok.SneakyThrows; -import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.cli.converters.picocli.ByteUnitToLongConverter; import org.apache.pulsar.client.api.ProxyProtocol; -import picocli.CommandLine; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; /** * PerformanceBaseArguments contains common CLI arguments and parsing logic available to all sub-commands. * Sub-commands should create Argument subclasses and override the `validate` method as necessary. */ -public abstract class PerformanceBaseArguments { +public abstract class PerformanceBaseArguments extends CmdBase{ - @Option(names = { "-h", "--help" }, description = "Print help message", help = true) - boolean help; - @Option(names = { "-cf", "--conf-file" }, description = "Pulsar configuration file") - public String confFile; - - @Option(names = { "-u", "--service-url" }, description = "Pulsar Service URL") + @Option(names = { "-u", "--service-url" }, description = "Pulsar Service URL", descriptionKey = "brokerServiceUrl") public String serviceURL; - @Option(names = { "--auth-plugin" }, description = "Authentication plugin class name") + @Option(names = { "--auth-plugin" }, description = "Authentication plugin class name", + descriptionKey = "authPlugin") public String authPluginClassName; @Option( names = { "--auth-params" }, description = "Authentication parameters, whose format is determined by the implementation " + "of method `configure` in authentication plugin class, for example \"key1:val1,key2:val2\" " - + "or \"{\"key1\":\"val1\",\"key2\":\"val2\"}\".") + + "or \"{\"key1\":\"val1\",\"key2\":\"val2\"}\".", descriptionKey = "authParams") public String authParams; @Option(names = { - "--trust-cert-file" }, description = "Path for the trusted TLS certificate file") + "--trust-cert-file" }, description = "Path for the trusted TLS certificate file", + descriptionKey = "tlsTrustCertsFilePath") public String tlsTrustCertsFilePath = ""; @Option(names = { - "--tls-allow-insecure" }, description = "Allow insecure TLS connection") + "--tls-allow-insecure" }, description = "Allow insecure TLS connection", + descriptionKey = "tlsAllowInsecureConnection") public Boolean tlsAllowInsecureConnection = null; @Option(names = { - "--tls-enable-hostname-verification" }, description = "Enable TLS hostname verification") + "--tls-enable-hostname-verification" }, description = "Enable TLS hostname verification", + descriptionKey = "tlsEnableHostnameVerification") public Boolean tlsHostnameVerificationEnable = null; @Option(names = { "-c", @@ -95,10 +86,12 @@ public abstract class PerformanceBaseArguments { + "on each broker connection to prevent overloading a broker") public int maxLookupRequest = 50000; - @Option(names = { "--proxy-url" }, description = "Proxy-server URL to which to connect.") + @Option(names = { "--proxy-url" }, description = "Proxy-server URL to which to connect.", + descriptionKey = "proxyServiceUrl") String proxyServiceURL = null; - @Option(names = { "--proxy-protocol" }, description = "Proxy protocol to select type of routing at proxy.") + @Option(names = { "--proxy-protocol" }, description = "Proxy protocol to select type of routing at proxy.", + descriptionKey = "proxyProtocol", converter = ProxyProtocolConverter.class) ProxyProtocol proxyProtocol = null; @Option(names = { "--auth_plugin" }, description = "Authentication plugin class name", hidden = true) @@ -107,129 +100,23 @@ public abstract class PerformanceBaseArguments { @Option(names = { "-ml", "--memory-limit", }, description = "Configure the Pulsar client memory limit " + "(eg: 32M, 64M)", converter = ByteUnitToLongConverter.class) public long memoryLimit; - - public abstract void fillArgumentsFromProperties(Properties prop); - - @SneakyThrows - public void fillArgumentsFromProperties() { - if (confFile == null) { - return; - } - - Properties prop = new Properties(System.getProperties()); - try (FileInputStream fis = new FileInputStream(confFile)) { - prop.load(fis); - } - - if (serviceURL == null) { - serviceURL = prop.getProperty("brokerServiceUrl"); - } - - if (serviceURL == null) { - serviceURL = prop.getProperty("webServiceUrl"); - } - - // fallback to previous-version serviceUrl property to maintain backward-compatibility - if (serviceURL == null) { - serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/"); - } - - if (authPluginClassName == null) { - authPluginClassName = prop.getProperty("authPlugin", null); - } - - if (authParams == null) { - authParams = prop.getProperty("authParams", null); - } - - if (isBlank(tlsTrustCertsFilePath)) { - tlsTrustCertsFilePath = prop.getProperty("tlsTrustCertsFilePath", ""); - } - - if (tlsAllowInsecureConnection == null) { - tlsAllowInsecureConnection = Boolean.parseBoolean(prop - .getProperty("tlsAllowInsecureConnection", "")); - } - - if (tlsHostnameVerificationEnable == null) { - tlsHostnameVerificationEnable = Boolean.parseBoolean(prop - .getProperty("tlsEnableHostnameVerification", "")); - - } - - if (proxyServiceURL == null) { - proxyServiceURL = StringUtils.trimToNull(prop.getProperty("proxyServiceUrl")); - } - - if (proxyProtocol == null) { - String proxyProtocolString = null; - try { - proxyProtocolString = StringUtils.trimToNull(prop.getProperty("proxyProtocol")); - if (proxyProtocolString != null) { - proxyProtocol = ProxyProtocol.valueOf(proxyProtocolString.toUpperCase()); - } - } catch (IllegalArgumentException e) { - System.out.println("Incorrect proxyProtocol name '" + proxyProtocolString + "'"); - e.printStackTrace(); - exit(1); - } - - } - - fillArgumentsFromProperties(prop); + public PerformanceBaseArguments(String cmdName) { + super(cmdName); } - /** - * Validate the CLI arguments. Default implementation provides validation for the common arguments. - * Each subclass should call super.validate() and provide validation code specific to the sub-command. - * @throws Exception - */ + @Override public void validate() throws Exception { - if (confFile != null && !confFile.isBlank()) { - File configFile = new File(confFile); - if (!configFile.exists()) { - throw new Exception("config file '" + confFile + "', does not exist"); - } - if (configFile.isDirectory()) { - throw new Exception("config file '" + confFile + "', is a directory"); - } - } + parseCLI(); } /** * Parse the command line args. - * @param cmdName used for the help message - * @param args String[] of CLI args * @throws ParameterException If there is a problem parsing the arguments */ - public void parseCLI(String cmdName, String[] args) { - CommandLine commander = new CommandLine(this); - commander.setCommandName(cmdName); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } - - if (help) { - commander.usage(commander.getOut()); - PerfClientUtils.exit(0); - } - - fillArgumentsFromProperties(); - + public void parseCLI() { if (isBlank(authPluginClassName) && !isBlank(deprecatedAuthPluginClassName)) { authPluginClassName = deprecatedAuthPluginClassName; } - - try { - validate(); - } catch (Exception e) { - System.out.println("error: " + e.getMessage()); - PerfClientUtils.exit(1); - } } } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java index 7a2bc4382fd14..5126eefd9ca1e 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Properties; import java.util.concurrent.Future; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; @@ -57,9 +56,9 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ScopeType; -public class PerformanceConsumer { +@Command(name = "consume", description = "Test pulsar consumer performance.") +public class PerformanceConsumer extends PerformanceTopicListArguments{ private static final LongAdder messagesReceived = new LongAdder(); private static final LongAdder bytesReceived = new LongAdder(); private static final DecimalFormat intFormat = new PaddingDecimalFormat("0", 7); @@ -83,325 +82,319 @@ public class PerformanceConsumer { private static final Recorder recorder = new Recorder(MAX_LATENCY, 5); private static final Recorder cumulativeRecorder = new Recorder(MAX_LATENCY, 5); - @Command(description = "Test pulsar consumer performance.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments extends PerformanceTopicListArguments { + @Option(names = { "-n", "--num-consumers" }, description = "Number of consumers (per subscription), only " + + "one consumer is allowed when subscriptionType is Exclusive", + converter = PositiveNumberParameterConvert.class + ) + public int numConsumers = 1; - @Option(names = { "-n", "--num-consumers" }, description = "Number of consumers (per subscription), only " - + "one consumer is allowed when subscriptionType is Exclusive", - converter = PositiveNumberParameterConvert.class - ) - public int numConsumers = 1; + @Option(names = { "-ns", "--num-subscriptions" }, description = "Number of subscriptions (per topic)", + converter = PositiveNumberParameterConvert.class + ) + public int numSubscriptions = 1; - @Option(names = { "-ns", "--num-subscriptions" }, description = "Number of subscriptions (per topic)", - converter = PositiveNumberParameterConvert.class - ) - public int numSubscriptions = 1; + @Option(names = { "-s", "--subscriber-name" }, description = "Subscriber name prefix", hidden = true) + public String subscriberName; - @Option(names = { "-s", "--subscriber-name" }, description = "Subscriber name prefix", hidden = true) - public String subscriberName; + @Option(names = { "-ss", "--subscriptions" }, + description = "A list of subscriptions to consume (for example, sub1,sub2)") + public List subscriptions = Collections.singletonList("sub"); - @Option(names = { "-ss", "--subscriptions" }, - description = "A list of subscriptions to consume (for example, sub1,sub2)") - public List subscriptions = Collections.singletonList("sub"); + @Option(names = { "-st", "--subscription-type" }, description = "Subscription type") + public SubscriptionType subscriptionType = SubscriptionType.Exclusive; - @Option(names = { "-st", "--subscription-type" }, description = "Subscription type") - public SubscriptionType subscriptionType = SubscriptionType.Exclusive; + @Option(names = { "-sp", "--subscription-position" }, description = "Subscription position") + private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest; - @Option(names = { "-sp", "--subscription-position" }, description = "Subscription position") - private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest; + @Option(names = { "-r", "--rate" }, description = "Simulate a slow message consumer (rate in msg/s)") + public double rate = 0; - @Option(names = { "-r", "--rate" }, description = "Simulate a slow message consumer (rate in msg/s)") - public double rate = 0; + @Option(names = { "-q", "--receiver-queue-size" }, description = "Size of the receiver queue") + public int receiverQueueSize = 1000; - @Option(names = { "-q", "--receiver-queue-size" }, description = "Size of the receiver queue") - public int receiverQueueSize = 1000; + @Option(names = { "-p", "--receiver-queue-size-across-partitions" }, + description = "Max total size of the receiver queue across partitions") + public int maxTotalReceiverQueueSizeAcrossPartitions = 50000; - @Option(names = { "-p", "--receiver-queue-size-across-partitions" }, - description = "Max total size of the receiver queue across partitions") - public int maxTotalReceiverQueueSizeAcrossPartitions = 50000; + @Option(names = {"-aq", "--auto-scaled-receiver-queue-size"}, + description = "Enable autoScaledReceiverQueueSize") + public boolean autoScaledReceiverQueueSize = false; - @Option(names = {"-aq", "--auto-scaled-receiver-queue-size"}, - description = "Enable autoScaledReceiverQueueSize") - public boolean autoScaledReceiverQueueSize = false; + @Option(names = {"-rs", "--replicated" }, + description = "Whether the subscription status should be replicated") + public boolean replicatedSubscription = false; - @Option(names = {"-rs", "--replicated" }, - description = "Whether the subscription status should be replicated") - public boolean replicatedSubscription = false; + @Option(names = { "--acks-delay-millis" }, description = "Acknowledgements grouping delay in millis") + public int acknowledgmentsGroupingDelayMillis = 100; - @Option(names = { "--acks-delay-millis" }, description = "Acknowledgements grouping delay in millis") - public int acknowledgmentsGroupingDelayMillis = 100; + @Option(names = {"-m", + "--num-messages"}, + description = "Number of messages to consume in total. If <= 0, it will keep consuming") + public long numMessages = 0; - @Option(names = {"-m", - "--num-messages"}, - description = "Number of messages to consume in total. If <= 0, it will keep consuming") - public long numMessages = 0; + @Option(names = { "-mc", "--max_chunked_msg" }, description = "Max pending chunk messages") + private int maxPendingChunkedMessage = 0; - @Option(names = { "-mc", "--max_chunked_msg" }, description = "Max pending chunk messages") - private int maxPendingChunkedMessage = 0; + @Option(names = { "-ac", + "--auto_ack_chunk_q_full" }, description = "Auto ack for oldest message on queue is full") + private boolean autoAckOldestChunkedMessageOnQueueFull = false; - @Option(names = { "-ac", - "--auto_ack_chunk_q_full" }, description = "Auto ack for oldest message on queue is full") - private boolean autoAckOldestChunkedMessageOnQueueFull = false; + @Option(names = { "-e", + "--expire_time_incomplete_chunked_messages" }, + description = "Expire time in ms for incomplete chunk messages") + private long expireTimeOfIncompleteChunkedMessageMs = 0; - @Option(names = { "-e", - "--expire_time_incomplete_chunked_messages" }, - description = "Expire time in ms for incomplete chunk messages") - private long expireTimeOfIncompleteChunkedMessageMs = 0; + @Option(names = { "-v", + "--encryption-key-value-file" }, + description = "The file which contains the private key to decrypt payload") + public String encKeyFile = null; - @Option(names = { "-v", - "--encryption-key-value-file" }, - description = "The file which contains the private key to decrypt payload") - public String encKeyFile = null; + @Option(names = { "-time", + "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep consuming") + public long testTime = 0; - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep consuming") - public long testTime = 0; + @Option(names = {"--batch-index-ack" }, description = "Enable or disable the batch index acknowledgment") + public boolean batchIndexAck = false; - @Option(names = {"--batch-index-ack" }, description = "Enable or disable the batch index acknowledgment") - public boolean batchIndexAck = false; + @Option(names = { "-pm", "--pool-messages" }, description = "Use the pooled message", arity = "1") + private boolean poolMessages = true; - @Option(names = { "-pm", "--pool-messages" }, description = "Use the pooled message", arity = "1") - private boolean poolMessages = true; + @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," + + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") + public long transactionTimeout = 10; - @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," - + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") - public long transactionTimeout = 10; + @Option(names = {"-nmt", "--numMessage-perTransaction"}, + description = "The number of messages acknowledged by a transaction. " + + "(After --txn-enable setting to true, -numMessage-perTransaction takes effect") + public int numMessagesPerTransaction = 50; - @Option(names = {"-nmt", "--numMessage-perTransaction"}, - description = "The number of messages acknowledged by a transaction. " - + "(After --txn-enable setting to true, -numMessage-perTransaction takes effect") - public int numMessagesPerTransaction = 50; + @Option(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction") + public boolean isEnableTransaction = false; - @Option(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction") - public boolean isEnableTransaction = false; + @Option(names = {"-ntxn"}, description = "The number of opened transactions, 0 means keeping open." + + "(After --txn-enable setting to true, -ntxn takes effect.)") + public long totalNumTxn = 0; - @Option(names = {"-ntxn"}, description = "The number of opened transactions, 0 means keeping open." - + "(After --txn-enable setting to true, -ntxn takes effect.)") - public long totalNumTxn = 0; + @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-enable " + + "setting to true, -abort takes effect)") + public boolean isAbortTransaction = false; - @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-enable " - + "setting to true, -abort takes effect)") - public boolean isAbortTransaction = false; + @Option(names = { "--histogram-file" }, description = "HdrHistogram output file") + public String histogramFile = null; - @Option(names = { "--histogram-file" }, description = "HdrHistogram output file") - public String histogramFile = null; + public PerformanceConsumer() { + super("consume"); + } - @Override - public void fillArgumentsFromProperties(Properties prop) { - } - @Override - public void validate() throws Exception { - super.validate(); - if (subscriptionType == SubscriptionType.Exclusive && numConsumers > 1) { - throw new Exception("Only one consumer is allowed when subscriptionType is Exclusive"); - } + @Override + public void validate() throws Exception { + super.validate(); + if (subscriptionType == SubscriptionType.Exclusive && numConsumers > 1) { + throw new Exception("Only one consumer is allowed when subscriptionType is Exclusive"); + } - if (subscriptions != null && subscriptions.size() != numSubscriptions) { - // keep compatibility with the previous version - if (subscriptions.size() == 1) { - if (subscriberName == null) { - subscriberName = subscriptions.get(0); - } - List defaultSubscriptions = new ArrayList<>(); - for (int i = 0; i < numSubscriptions; i++) { - defaultSubscriptions.add(String.format("%s-%d", subscriberName, i)); - } - subscriptions = defaultSubscriptions; - } else { - throw new Exception("The size of subscriptions list should be equal to --num-subscriptions"); + if (subscriptions != null && subscriptions.size() != numSubscriptions) { + // keep compatibility with the previous version + if (subscriptions.size() == 1) { + if (subscriberName == null) { + subscriberName = subscriptions.get(0); } + List defaultSubscriptions = new ArrayList<>(); + for (int i = 0; i < numSubscriptions; i++) { + defaultSubscriptions.add(String.format("%s-%d", subscriberName, i)); + } + subscriptions = defaultSubscriptions; + } else { + throw new Exception("The size of subscriptions list should be equal to --num-subscriptions"); } } } - - public static void main(String[] args) throws Exception { - final Arguments arguments = new Arguments(); - arguments.parseCLI("pulsar-perf consume", args); - + @Override + public void run() throws Exception { // Dump config variables PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar performance consumer with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar performance consumer with config: {}", w.writeValueAsString(this)); - final Recorder qRecorder = arguments.autoScaledReceiverQueueSize - ? new Recorder(arguments.receiverQueueSize, 5) : null; - final RateLimiter limiter = arguments.rate > 0 ? RateLimiter.create(arguments.rate) : null; + final Recorder qRecorder = this.autoScaledReceiverQueueSize + ? new Recorder(this.receiverQueueSize, 5) : null; + final RateLimiter limiter = this.rate > 0 ? RateLimiter.create(this.rate) : null; long startTime = System.nanoTime(); - long testEndTime = startTime + (long) (arguments.testTime * 1e9); + long testEndTime = startTime + (long) (this.testTime * 1e9); - ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTransaction(arguments.isEnableTransaction); + ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(this) + .enableTransaction(this.isEnableTransaction); PulsarClient pulsarClient = clientBuilder.build(); AtomicReference atomicReference; - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { atomicReference = new AtomicReference<>(pulsarClient.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS).build().get()); + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS).build().get()); } else { atomicReference = new AtomicReference<>(null); } AtomicLong messageAckedCount = new AtomicLong(); - Semaphore messageReceiveLimiter = new Semaphore(arguments.numMessagesPerTransaction); + Semaphore messageReceiveLimiter = new Semaphore(this.numMessagesPerTransaction); Thread thread = Thread.currentThread(); MessageListener listener = (consumer, msg) -> { - if (arguments.testTime > 0) { - if (System.nanoTime() > testEndTime) { - log.info("------------------- DONE -----------------------"); - PerfClientUtils.exit(0); - thread.interrupt(); - } - } - if (arguments.totalNumTxn > 0) { - if (totalEndTxnOpFailNum.sum() + totalEndTxnOpSuccessNum.sum() >= arguments.totalNumTxn) { - log.info("------------------- DONE -----------------------"); - PerfClientUtils.exit(0); - thread.interrupt(); - } - } - if (qRecorder != null) { - qRecorder.recordValue(((ConsumerBase) consumer).getTotalIncomingMessages()); + if (this.testTime > 0) { + if (System.nanoTime() > testEndTime) { + log.info("------------------- DONE -----------------------"); + PerfClientUtils.exit(0); + thread.interrupt(); } - messagesReceived.increment(); - bytesReceived.add(msg.size()); - - totalMessagesReceived.increment(); - totalBytesReceived.add(msg.size()); - - if (arguments.numMessages > 0 && totalMessagesReceived.sum() >= arguments.numMessages) { + } + if (this.totalNumTxn > 0) { + if (totalEndTxnOpFailNum.sum() + totalEndTxnOpSuccessNum.sum() >= this.totalNumTxn) { log.info("------------------- DONE -----------------------"); PerfClientUtils.exit(0); thread.interrupt(); } + } + if (qRecorder != null) { + qRecorder.recordValue(((ConsumerBase) consumer).getTotalIncomingMessages()); + } + messagesReceived.increment(); + bytesReceived.add(msg.size()); - if (limiter != null) { - limiter.acquire(); - } + totalMessagesReceived.increment(); + totalBytesReceived.add(msg.size()); - long latencyMillis = System.currentTimeMillis() - msg.getPublishTime(); - if (latencyMillis >= 0) { - if (latencyMillis >= MAX_LATENCY) { - latencyMillis = MAX_LATENCY; - } - recorder.recordValue(latencyMillis); - cumulativeRecorder.recordValue(latencyMillis); - } - if (arguments.isEnableTransaction) { - try { - messageReceiveLimiter.acquire(); - } catch (InterruptedException e){ - log.error("Got error: ", e); - } - consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> { - totalMessageAck.increment(); - messageAck.increment(); - }).exceptionally(throwable ->{ - log.error("Ack message {} failed with exception", msg, throwable); - totalMessageAckFailed.increment(); - return null; - }); - } else { - consumer.acknowledgeAsync(msg).thenRun(()->{ - totalMessageAck.increment(); - messageAck.increment(); - } - ).exceptionally(throwable ->{ - log.error("Ack message {} failed with exception", msg, throwable); - totalMessageAckFailed.increment(); - return null; - } - ); + if (this.numMessages > 0 && totalMessagesReceived.sum() >= this.numMessages) { + log.info("------------------- DONE -----------------------"); + PerfClientUtils.exit(0); + thread.interrupt(); + } + + if (limiter != null) { + limiter.acquire(); + } + + long latencyMillis = System.currentTimeMillis() - msg.getPublishTime(); + if (latencyMillis >= 0) { + if (latencyMillis >= MAX_LATENCY) { + latencyMillis = MAX_LATENCY; } - if (arguments.poolMessages) { - msg.release(); + recorder.recordValue(latencyMillis); + cumulativeRecorder.recordValue(latencyMillis); + } + if (this.isEnableTransaction) { + try { + messageReceiveLimiter.acquire(); + } catch (InterruptedException e){ + log.error("Got error: ", e); } - if (arguments.isEnableTransaction - && messageAckedCount.incrementAndGet() == arguments.numMessagesPerTransaction) { - Transaction transaction = atomicReference.get(); - if (!arguments.isAbortTransaction) { - transaction.commit() - .thenRun(() -> { - if (log.isDebugEnabled()) { - log.debug("Commit transaction {}", transaction.getTxnID()); - } - totalEndTxnOpSuccessNum.increment(); - numTxnOpSuccess.increment(); - }) - .exceptionally(exception -> { - log.error("Commit transaction failed with exception : ", exception); - totalEndTxnOpFailNum.increment(); - return null; - }); - } else { - transaction.abort().thenRun(() -> { - if (log.isDebugEnabled()) { - log.debug("Abort transaction {}", transaction.getTxnID()); - } - totalEndTxnOpSuccessNum.increment(); - numTxnOpSuccess.increment(); - }).exceptionally(exception -> { - log.error("Abort transaction {} failed with exception", - transaction.getTxnID().toString(), - exception); - totalEndTxnOpFailNum.increment(); + consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> { + totalMessageAck.increment(); + messageAck.increment(); + }).exceptionally(throwable ->{ + log.error("Ack message {} failed with exception", msg, throwable); + totalMessageAckFailed.increment(); + return null; + }); + } else { + consumer.acknowledgeAsync(msg).thenRun(()->{ + totalMessageAck.increment(); + messageAck.increment(); + } + ).exceptionally(throwable ->{ + log.error("Ack message {} failed with exception", msg, throwable); + totalMessageAckFailed.increment(); return null; - }); - } - while (true) { - try { - Transaction newTransaction = pulsarClient.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS) - .build().get(); - atomicReference.compareAndSet(transaction, newTransaction); - totalNumTxnOpenSuccess.increment(); - messageAckedCount.set(0); - messageReceiveLimiter.release(arguments.numMessagesPerTransaction); - break; - } catch (Exception e) { - log.error("Failed to new transaction with exception:", e); - totalNumTxnOpenFail.increment(); } + ); + } + if (this.poolMessages) { + msg.release(); + } + if (this.isEnableTransaction + && messageAckedCount.incrementAndGet() == this.numMessagesPerTransaction) { + Transaction transaction = atomicReference.get(); + if (!this.isAbortTransaction) { + transaction.commit() + .thenRun(() -> { + if (log.isDebugEnabled()) { + log.debug("Commit transaction {}", transaction.getTxnID()); + } + totalEndTxnOpSuccessNum.increment(); + numTxnOpSuccess.increment(); + }) + .exceptionally(exception -> { + log.error("Commit transaction failed with exception : ", exception); + totalEndTxnOpFailNum.increment(); + return null; + }); + } else { + transaction.abort().thenRun(() -> { + if (log.isDebugEnabled()) { + log.debug("Abort transaction {}", transaction.getTxnID()); + } + totalEndTxnOpSuccessNum.increment(); + numTxnOpSuccess.increment(); + }).exceptionally(exception -> { + log.error("Abort transaction {} failed with exception", + transaction.getTxnID().toString(), + exception); + totalEndTxnOpFailNum.increment(); + return null; + }); + } + while (true) { + try { + Transaction newTransaction = pulsarClient.newTransaction() + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS) + .build().get(); + atomicReference.compareAndSet(transaction, newTransaction); + totalNumTxnOpenSuccess.increment(); + messageAckedCount.set(0); + messageReceiveLimiter.release(this.numMessagesPerTransaction); + break; + } catch (Exception e) { + log.error("Failed to new transaction with exception:", e); + totalNumTxnOpenFail.increment(); } } + } }; List>> futures = new ArrayList<>(); ConsumerBuilder consumerBuilder = pulsarClient.newConsumer(Schema.BYTEBUFFER) // .messageListener(listener) // - .receiverQueueSize(arguments.receiverQueueSize) // - .maxTotalReceiverQueueSizeAcrossPartitions(arguments.maxTotalReceiverQueueSizeAcrossPartitions) - .acknowledgmentGroupTime(arguments.acknowledgmentsGroupingDelayMillis, TimeUnit.MILLISECONDS) // - .subscriptionType(arguments.subscriptionType) - .subscriptionInitialPosition(arguments.subscriptionInitialPosition) - .autoAckOldestChunkedMessageOnQueueFull(arguments.autoAckOldestChunkedMessageOnQueueFull) - .enableBatchIndexAcknowledgment(arguments.batchIndexAck) - .poolMessages(arguments.poolMessages) - .replicateSubscriptionState(arguments.replicatedSubscription) - .autoScaledReceiverQueueSizeEnabled(arguments.autoScaledReceiverQueueSize); - if (arguments.maxPendingChunkedMessage > 0) { - consumerBuilder.maxPendingChunkedMessage(arguments.maxPendingChunkedMessage); + .receiverQueueSize(this.receiverQueueSize) // + .maxTotalReceiverQueueSizeAcrossPartitions(this.maxTotalReceiverQueueSizeAcrossPartitions) + .acknowledgmentGroupTime(this.acknowledgmentsGroupingDelayMillis, TimeUnit.MILLISECONDS) // + .subscriptionType(this.subscriptionType) + .subscriptionInitialPosition(this.subscriptionInitialPosition) + .autoAckOldestChunkedMessageOnQueueFull(this.autoAckOldestChunkedMessageOnQueueFull) + .enableBatchIndexAcknowledgment(this.batchIndexAck) + .poolMessages(this.poolMessages) + .replicateSubscriptionState(this.replicatedSubscription) + .autoScaledReceiverQueueSizeEnabled(this.autoScaledReceiverQueueSize); + if (this.maxPendingChunkedMessage > 0) { + consumerBuilder.maxPendingChunkedMessage(this.maxPendingChunkedMessage); } - if (arguments.expireTimeOfIncompleteChunkedMessageMs > 0) { - consumerBuilder.expireTimeOfIncompleteChunkedMessage(arguments.expireTimeOfIncompleteChunkedMessageMs, + if (this.expireTimeOfIncompleteChunkedMessageMs > 0) { + consumerBuilder.expireTimeOfIncompleteChunkedMessage(this.expireTimeOfIncompleteChunkedMessageMs, TimeUnit.MILLISECONDS); } - if (isNotBlank(arguments.encKeyFile)) { - consumerBuilder.defaultCryptoKeyReader(arguments.encKeyFile); + if (isNotBlank(this.encKeyFile)) { + consumerBuilder.defaultCryptoKeyReader(this.encKeyFile); } - for (int i = 0; i < arguments.numTopics; i++) { - final TopicName topicName = TopicName.get(arguments.topics.get(i)); + for (int i = 0; i < this.numTopics; i++) { + final TopicName topicName = TopicName.get(this.topics.get(i)); - log.info("Adding {} consumers per subscription on topic {}", arguments.numConsumers, topicName); + log.info("Adding {} consumers per subscription on topic {}", this.numConsumers, topicName); - for (int j = 0; j < arguments.numSubscriptions; j++) { - String subscriberName = arguments.subscriptions.get(j); - for (int k = 0; k < arguments.numConsumers; k++) { + for (int j = 0; j < this.numSubscriptions; j++) { + String subscriberName = this.subscriptions.get(j); + for (int k = 0; k < this.numConsumers; k++) { futures.add(consumerBuilder.clone().topic(topicName.toString()).subscriptionName(subscriberName) .subscribeAsync()); } @@ -410,13 +403,13 @@ public static void main(String[] args) throws Exception { for (Future> future : futures) { future.get(); } - log.info("Start receiving from {} consumers per subscription on {} topics", arguments.numConsumers, - arguments.numTopics); + log.info("Start receiving from {} consumers per subscription on {} topics", this.numConsumers, + this.numTopics); long start = System.nanoTime(); Runtime.getRuntime().addShutdownHook(new Thread(() -> { - printAggregatedThroughput(start, arguments); + printAggregatedThroughput(start); printAggregatedStats(); })); @@ -427,8 +420,8 @@ public static void main(String[] args) throws Exception { Histogram qHistogram = null; HistogramLogWriter histogramLogWriter = null; - if (arguments.histogramFile != null) { - String statsFileName = arguments.histogramFile; + if (this.histogramFile != null) { + String statsFileName = this.histogramFile; log.info("Dumping latency stats to {}", statsFileName); PrintStream histogramLog = new PrintStream(new FileOutputStream(statsFileName), false); @@ -457,7 +450,7 @@ public static void main(String[] args) throws Exception { double rateOpenTxn = 0; reportHistogram = recorder.getIntervalHistogram(reportHistogram); - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { totalTxnOpSuccessNum = totalEndTxnOpSuccessNum.sum(); totalTxnOpFailNum = totalEndTxnOpFailNum.sum(); rateOpenTxn = numTxnOpSuccess.sumThenReset() / elapsed; @@ -478,7 +471,7 @@ public static void main(String[] args) throws Exception { reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9), reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue()); - if (arguments.autoScaledReceiverQueueSize && log.isDebugEnabled() && qRecorder != null) { + if (this.autoScaledReceiverQueueSize && log.isDebugEnabled() && qRecorder != null) { qHistogram = qRecorder.getIntervalHistogram(qHistogram); log.debug("ReceiverQueueUsage: cnt={},mean={}, min={},max={},25pct={},50pct={},75pct={}", qHistogram.getTotalCount(), dec.format(qHistogram.getMean()), @@ -507,7 +500,7 @@ public static void main(String[] args) throws Exception { reportHistogram.reset(); oldTime = now; - if (arguments.testTime > 0) { + if (this.testTime > 0) { if (now > testEndTime) { log.info("------------------- DONE -----------------------"); PerfClientUtils.exit(0); @@ -519,7 +512,7 @@ public static void main(String[] args) throws Exception { pulsarClient.close(); } - private static void printAggregatedThroughput(long start, Arguments arguments) { + private void printAggregatedThroughput(long start) { double elapsed = (System.nanoTime() - start) / 1e9; double rate = totalMessagesReceived.sum() / elapsed; double throughput = totalBytesReceived.sum() / elapsed * 8 / 1024 / 1024; @@ -530,7 +523,7 @@ private static void printAggregatedThroughput(long start, Arguments arguments) { long totalnumMessageAckFailed = 0; double rateAck = totalMessageAck.sum() / elapsed; double rateOpenTxn = 0; - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { totalEndTxnSuccess = totalEndTxnOpSuccessNum.sum(); totalEndTxnFail = totalEndTxnOpFailNum.sum(); rateOpenTxn = (totalEndTxnSuccess + totalEndTxnFail) / elapsed; diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java index 0eb8d02f31efa..ba5be3a3c4566 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java @@ -20,7 +20,6 @@ import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.NANOSECONDS; -import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_BATCHING_MAX_MESSAGES; import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_MAX_PENDING_MESSAGES; @@ -40,7 +39,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Properties; import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -75,13 +73,13 @@ import picocli.CommandLine.Command; import picocli.CommandLine.ITypeConverter; import picocli.CommandLine.Option; -import picocli.CommandLine.ScopeType; import picocli.CommandLine.TypeConversionException; /** * A client program to test pulsar producer performance. */ -public class PerformanceProducer { +@Command(name = "produce", description = "Test pulsar producer performance.") +public class PerformanceProducer extends PerformanceTopicListArguments{ private static final ExecutorService executor = Executors .newCachedThreadPool(new DefaultThreadFactory("pulsar-perf-producer-exec")); @@ -105,192 +103,171 @@ public class PerformanceProducer { private static IMessageFormatter messageFormatter = null; - @Command(description = "Test pulsar producer performance.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments extends PerformanceTopicListArguments { + @Option(names = { "-threads", "--num-test-threads" }, description = "Number of test threads", + converter = PositiveNumberParameterConvert.class + ) + public int numTestThreads = 1; - @Option(names = { "-threads", "--num-test-threads" }, description = "Number of test threads", - converter = PositiveNumberParameterConvert.class - ) - public int numTestThreads = 1; + @Option(names = { "-r", "--rate" }, description = "Publish rate msg/s across topics") + public int msgRate = 100; - @Option(names = { "-r", "--rate" }, description = "Publish rate msg/s across topics") - public int msgRate = 100; + @Option(names = { "-s", "--size" }, description = "Message size (bytes)") + public int msgSize = 1024; - @Option(names = { "-s", "--size" }, description = "Message size (bytes)") - public int msgSize = 1024; + @Option(names = { "-n", "--num-producers" }, description = "Number of producers (per topic)", + converter = PositiveNumberParameterConvert.class + ) + public int numProducers = 1; - @Option(names = { "-n", "--num-producers" }, description = "Number of producers (per topic)", - converter = PositiveNumberParameterConvert.class - ) - public int numProducers = 1; + @Option(names = {"--separator"}, description = "Separator between the topic and topic number") + public String separator = "-"; - @Option(names = {"--separator"}, description = "Separator between the topic and topic number") - public String separator = "-"; + @Option(names = {"--send-timeout"}, description = "Set the sendTimeout value default 0 to keep " + + "compatibility with previous version of pulsar-perf") + public int sendTimeout = 0; - @Option(names = {"--send-timeout"}, description = "Set the sendTimeout value default 0 to keep " - + "compatibility with previous version of pulsar-perf") - public int sendTimeout = 0; + @Option(names = { "-pn", "--producer-name" }, description = "Producer Name") + public String producerName = null; - @Option(names = { "-pn", "--producer-name" }, description = "Producer Name") - public String producerName = null; + @Option(names = { "-au", "--admin-url" }, description = "Pulsar Admin URL", descriptionKey = "webServiceUrl") + public String adminURL; - @Option(names = { "-au", "--admin-url" }, description = "Pulsar Admin URL") - public String adminURL; + @Option(names = { "-ch", + "--chunking" }, description = "Should split the message and publish in chunks if message size is " + + "larger than allowed max size") + private boolean chunkingAllowed = false; - @Option(names = { "-ch", - "--chunking" }, description = "Should split the message and publish in chunks if message size is " - + "larger than allowed max size") - private boolean chunkingAllowed = false; + @Option(names = { "-o", "--max-outstanding" }, description = "Max number of outstanding messages") + public int maxOutstanding = DEFAULT_MAX_PENDING_MESSAGES; - @Option(names = { "-o", "--max-outstanding" }, description = "Max number of outstanding messages") - public int maxOutstanding = DEFAULT_MAX_PENDING_MESSAGES; + @Option(names = { "-p", "--max-outstanding-across-partitions" }, description = "Max number of outstanding " + + "messages across partitions") + public int maxPendingMessagesAcrossPartitions = DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; - @Option(names = { "-p", "--max-outstanding-across-partitions" }, description = "Max number of outstanding " - + "messages across partitions") - public int maxPendingMessagesAcrossPartitions = DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; + @Option(names = { "-np", "--partitions" }, description = "Create partitioned topics with the given number " + + "of partitions, set 0 to not try to create the topic") + public Integer partitions = null; - @Option(names = { "-np", "--partitions" }, description = "Create partitioned topics with the given number " - + "of partitions, set 0 to not try to create the topic") - public Integer partitions = null; + @Option(names = { "-m", + "--num-messages" }, description = "Number of messages to publish in total. If <= 0, it will keep " + + "publishing") + public long numMessages = 0; - @Option(names = { "-m", - "--num-messages" }, description = "Number of messages to publish in total. If <= 0, it will keep " - + "publishing") - public long numMessages = 0; + @Option(names = { "-z", "--compression" }, description = "Compress messages payload") + public CompressionType compression = CompressionType.NONE; - @Option(names = { "-z", "--compression" }, description = "Compress messages payload") - public CompressionType compression = CompressionType.NONE; + @Option(names = { "-f", "--payload-file" }, description = "Use payload from an UTF-8 encoded text file and " + + "a payload will be randomly selected when publishing messages") + public String payloadFilename = null; - @Option(names = { "-f", "--payload-file" }, description = "Use payload from an UTF-8 encoded text file and " - + "a payload will be randomly selected when publishing messages") - public String payloadFilename = null; + @Option(names = { "-e", "--payload-delimiter" }, description = "The delimiter used to split lines when " + + "using payload from a file") + // here escaping \n since default value will be printed with the help text + public String payloadDelimiter = "\\n"; - @Option(names = { "-e", "--payload-delimiter" }, description = "The delimiter used to split lines when " - + "using payload from a file") - // here escaping \n since default value will be printed with the help text - public String payloadDelimiter = "\\n"; + @Option(names = { "-b", + "--batch-time-window" }, description = "Batch messages in 'x' ms window (Default: 1ms)") + public double batchTimeMillis = 1.0; - @Option(names = { "-b", - "--batch-time-window" }, description = "Batch messages in 'x' ms window (Default: 1ms)") - public double batchTimeMillis = 1.0; + @Option(names = { "-db", + "--disable-batching" }, description = "Disable batching if true") + public boolean disableBatching; - @Option(names = { "-db", - "--disable-batching" }, description = "Disable batching if true") - public boolean disableBatching; - - @Option(names = { + @Option(names = { "-bm", "--batch-max-messages" - }, description = "Maximum number of messages per batch") - public int batchMaxMessages = DEFAULT_BATCHING_MAX_MESSAGES; + }, description = "Maximum number of messages per batch") + public int batchMaxMessages = DEFAULT_BATCHING_MAX_MESSAGES; - @Option(names = { + @Option(names = { "-bb", "--batch-max-bytes" - }, description = "Maximum number of bytes per batch") - public int batchMaxBytes = 4 * 1024 * 1024; - - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") - public long testTime = 0; - - @Option(names = "--warmup-time", description = "Warm-up time in seconds (Default: 1 sec)") - public double warmupTimeSeconds = 1.0; + }, description = "Maximum number of bytes per batch") + public int batchMaxBytes = 4 * 1024 * 1024; - @Option(names = { "-k", "--encryption-key-name" }, description = "The public key name to encrypt payload") - public String encKeyName = null; + @Option(names = { "-time", + "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") + public long testTime = 0; - @Option(names = { "-v", - "--encryption-key-value-file" }, - description = "The file which contains the public key to encrypt payload") - public String encKeyFile = null; + @Option(names = "--warmup-time", description = "Warm-up time in seconds (Default: 1 sec)") + public double warmupTimeSeconds = 1.0; - @Option(names = { "-d", - "--delay" }, description = "Mark messages with a given delay in seconds") - public long delay = 0; + @Option(names = { "-k", "--encryption-key-name" }, description = "The public key name to encrypt payload") + public String encKeyName = null; - @Option(names = { "-dr", "--delay-range"}, description = "Mark messages with a given delay by a random" - + " number of seconds. this value between the specified origin (inclusive) and the specified bound" - + " (exclusive). e.g. 1,300", converter = RangeConvert.class) - public Range delayRange = null; + @Option(names = { "-v", + "--encryption-key-value-file" }, + description = "The file which contains the public key to encrypt payload") + public String encKeyFile = null; - @Option(names = { "-set", - "--set-event-time" }, description = "Set the eventTime on messages") - public boolean setEventTime = false; + @Option(names = { "-d", + "--delay" }, description = "Mark messages with a given delay in seconds") + public long delay = 0; - @Option(names = { "-ef", - "--exit-on-failure" }, description = "Exit from the process on publish failure (default: disable)") - public boolean exitOnFailure = false; + @Option(names = { "-dr", "--delay-range"}, description = "Mark messages with a given delay by a random" + + " number of seconds. this value between the specified origin (inclusive) and the specified bound" + + " (exclusive). e.g. 1,300", converter = RangeConvert.class) + public Range delayRange = null; - @Option(names = {"-mk", "--message-key-generation-mode"}, description = "The generation mode of message key" - + ", valid options are: [autoIncrement, random]") - public String messageKeyGenerationMode = null; + @Option(names = { "-set", + "--set-event-time" }, description = "Set the eventTime on messages") + public boolean setEventTime = false; - @Option(names = { "-am", "--access-mode" }, description = "Producer access mode") - public ProducerAccessMode producerAccessMode = ProducerAccessMode.Shared; + @Option(names = { "-ef", + "--exit-on-failure" }, description = "Exit from the process on publish failure (default: disable)") + public boolean exitOnFailure = false; - @Option(names = { "-fp", "--format-payload" }, - description = "Format %%i as a message index in the stream from producer and/or %%t as the timestamp" - + " nanoseconds.") - public boolean formatPayload = false; + @Option(names = {"-mk", "--message-key-generation-mode"}, description = "The generation mode of message key" + + ", valid options are: [autoIncrement, random]", descriptionKey = "messageKeyGenerationMode") + public String messageKeyGenerationMode = null; - @Option(names = {"-fc", "--format-class"}, description = "Custom Formatter class name") - public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter"; + @Option(names = { "-am", "--access-mode" }, description = "Producer access mode") + public ProducerAccessMode producerAccessMode = ProducerAccessMode.Shared; - @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," - + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") - public long transactionTimeout = 10; + @Option(names = { "-fp", "--format-payload" }, + description = "Format %%i as a message index in the stream from producer and/or %%t as the timestamp" + + " nanoseconds.") + public boolean formatPayload = false; - @Option(names = {"-nmt", "--numMessage-perTransaction"}, - description = "The number of messages sent by a transaction. " - + "(After --txn-enable setting to true, -nmt takes effect)") - public int numMessagesPerTransaction = 50; + @Option(names = {"-fc", "--format-class"}, description = "Custom Formatter class name") + public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter"; - @Option(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction") - public boolean isEnableTransaction = false; + @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," + + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") + public long transactionTimeout = 10; - @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-enable " - + "setting to true, -abort takes effect)") - public boolean isAbortTransaction = false; - - @Option(names = { "--histogram-file" }, description = "HdrHistogram output file") - public String histogramFile = null; - - @Override - public void fillArgumentsFromProperties(Properties prop) { - if (adminURL == null) { - adminURL = prop.getProperty("webServiceUrl"); - } - if (adminURL == null) { - adminURL = prop.getProperty("adminURL", "http://localhost:8080/"); - } + @Option(names = {"-nmt", "--numMessage-perTransaction"}, + description = "The number of messages sent by a transaction. " + + "(After --txn-enable setting to true, -nmt takes effect)") + public int numMessagesPerTransaction = 50; - if (isBlank(messageKeyGenerationMode)) { - messageKeyGenerationMode = prop.getProperty("messageKeyGenerationMode", null); - } - } - } + @Option(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction") + public boolean isEnableTransaction = false; - public static void main(String[] args) throws Exception { + @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-enable " + + "setting to true, -abort takes effect)") + public boolean isAbortTransaction = false; - final Arguments arguments = new Arguments(); - arguments.parseCLI("pulsar-perf produce", args); + @Option(names = { "--histogram-file" }, description = "HdrHistogram output file") + public String histogramFile = null; + @Override + public void run() throws Exception { // Dump config variables PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar perf producer with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar perf producer with config: {}", w.writeValueAsString(this)); // Read payload data from file if needed - final byte[] payloadBytes = new byte[arguments.msgSize]; + final byte[] payloadBytes = new byte[msgSize]; Random random = new Random(0); List payloadByteList = new ArrayList<>(); - if (arguments.payloadFilename != null) { - Path payloadFilePath = Paths.get(arguments.payloadFilename); + if (this.payloadFilename != null) { + Path payloadFilePath = Paths.get(this.payloadFilename); if (Files.notExists(payloadFilePath) || Files.size(payloadFilePath) == 0) { throw new IllegalArgumentException("Payload file doesn't exist or it is empty."); } // here escaping the default payload delimiter to correct value - String delimiter = arguments.payloadDelimiter.equals("\\n") ? "\n" : arguments.payloadDelimiter; + String delimiter = this.payloadDelimiter.equals("\\n") ? "\n" : this.payloadDelimiter; String[] payloadList = new String(Files.readAllBytes(payloadFilePath), StandardCharsets.UTF_8).split(delimiter); log.info("Reading payloads from {} and {} records read", payloadFilePath.toAbsolutePath(), @@ -299,8 +276,8 @@ public static void main(String[] args) throws Exception { payloadByteList.add(payload.getBytes(StandardCharsets.UTF_8)); } - if (arguments.formatPayload) { - messageFormatter = getMessageFormatter(arguments.formatterClass); + if (this.formatPayload) { + messageFormatter = getMessageFormatter(this.formatterClass); } } else { for (int i = 0; i < payloadBytes.length; ++i) { @@ -312,29 +289,29 @@ public static void main(String[] args) throws Exception { Runtime.getRuntime().addShutdownHook(new Thread(() -> { executorShutdownNow(); - printAggregatedThroughput(start, arguments); + printAggregatedThroughput(start); printAggregatedStats(); })); - if (arguments.partitions != null) { + if (this.partitions != null) { final PulsarAdminBuilder adminBuilder = PerfClientUtils - .createAdminBuilderFromArguments(arguments, arguments.adminURL); + .createAdminBuilderFromArguments(this, this.adminURL); try (PulsarAdmin adminClient = adminBuilder.build()) { - for (String topic : arguments.topics) { - log.info("Creating partitioned topic {} with {} partitions", topic, arguments.partitions); + for (String topic : this.topics) { + log.info("Creating partitioned topic {} with {} partitions", topic, this.partitions); try { - adminClient.topics().createPartitionedTopic(topic, arguments.partitions); + adminClient.topics().createPartitionedTopic(topic, this.partitions); } catch (PulsarAdminException.ConflictException alreadyExists) { if (log.isDebugEnabled()) { log.debug("Topic {} already exists: {}", topic, alreadyExists); } PartitionedTopicMetadata partitionedTopicMetadata = adminClient.topics() .getPartitionedTopicMetadata(topic); - if (partitionedTopicMetadata.partitions != arguments.partitions) { + if (partitionedTopicMetadata.partitions != this.partitions) { log.error("Topic {} already exists but it has a wrong number of partitions: {}, " + "expecting {}", - topic, partitionedTopicMetadata.partitions, arguments.partitions); + topic, partitionedTopicMetadata.partitions, this.partitions); PerfClientUtils.exit(1); } } @@ -342,23 +319,23 @@ public static void main(String[] args) throws Exception { } } - CountDownLatch doneLatch = new CountDownLatch(arguments.numTestThreads); + CountDownLatch doneLatch = new CountDownLatch(this.numTestThreads); - final long numMessagesPerThread = arguments.numMessages / arguments.numTestThreads; - final int msgRatePerThread = arguments.msgRate / arguments.numTestThreads; + final long numMessagesPerThread = this.numMessages / this.numTestThreads; + final int msgRatePerThread = this.msgRate / this.numTestThreads; - for (int i = 0; i < arguments.numTestThreads; i++) { + for (int i = 0; i < this.numTestThreads; i++) { final int threadIdx = i; executor.submit(() -> { log.info("Started performance test thread {}", threadIdx); runProducer( - threadIdx, - arguments, - numMessagesPerThread, - msgRatePerThread, - payloadByteList, - payloadBytes, - doneLatch + threadIdx, + this, + numMessagesPerThread, + msgRatePerThread, + payloadByteList, + payloadBytes, + doneLatch ); }); } @@ -369,8 +346,8 @@ public static void main(String[] args) throws Exception { Histogram reportHistogram = null; HistogramLogWriter histogramLogWriter = null; - if (arguments.histogramFile != null) { - String statsFileName = arguments.histogramFile; + if (this.histogramFile != null) { + String statsFileName = this.histogramFile; log.info("Dumping latency stats to {}", statsFileName); PrintStream histogramLog = new PrintStream(new FileOutputStream(statsFileName), false); @@ -404,7 +381,7 @@ public static void main(String[] args) throws Exception { reportHistogram = recorder.getIntervalHistogram(reportHistogram); - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { totalTxnOpSuccess = totalEndTxnOpSuccessNum.sum(); totalTxnOpFail = totalEndTxnOpFailNum.sum(); rateOpenTxn = numTxnOpSuccess.sumThenReset() / elapsed; @@ -435,7 +412,9 @@ public static void main(String[] args) throws Exception { oldTime = now; } - PerfClientUtils.exit(0); + } + public PerformanceProducer() { + super("produce"); } private static void executorShutdownNow() { @@ -460,49 +439,49 @@ static IMessageFormatter getMessageFormatter(String formatterClass) { } } - static ProducerBuilder createProducerBuilder(PulsarClient client, Arguments arguments, int producerId) { + ProducerBuilder createProducerBuilder(PulsarClient client, int producerId) { ProducerBuilder producerBuilder = client.newProducer() // - .sendTimeout(arguments.sendTimeout, TimeUnit.SECONDS) // - .compressionType(arguments.compression) // - .maxPendingMessages(arguments.maxOutstanding) // - .accessMode(arguments.producerAccessMode) + .sendTimeout(this.sendTimeout, TimeUnit.SECONDS) // + .compressionType(this.compression) // + .maxPendingMessages(this.maxOutstanding) // + .accessMode(this.producerAccessMode) // enable round robin message routing if it is a partitioned topic .messageRoutingMode(MessageRoutingMode.RoundRobinPartition); - if (arguments.maxPendingMessagesAcrossPartitions > 0) { - producerBuilder.maxPendingMessagesAcrossPartitions(arguments.maxPendingMessagesAcrossPartitions); + if (this.maxPendingMessagesAcrossPartitions > 0) { + producerBuilder.maxPendingMessagesAcrossPartitions(this.maxPendingMessagesAcrossPartitions); } - if (arguments.producerName != null) { - String producerName = String.format("%s%s%d", arguments.producerName, arguments.separator, producerId); + if (this.producerName != null) { + String producerName = String.format("%s%s%d", this.producerName, this.separator, producerId); producerBuilder.producerName(producerName); } - if (arguments.disableBatching || (arguments.batchTimeMillis <= 0.0 && arguments.batchMaxMessages <= 0)) { + if (this.disableBatching || (this.batchTimeMillis <= 0.0 && this.batchMaxMessages <= 0)) { producerBuilder.enableBatching(false); } else { - long batchTimeUsec = (long) (arguments.batchTimeMillis * 1000); + long batchTimeUsec = (long) (this.batchTimeMillis * 1000); producerBuilder.batchingMaxPublishDelay(batchTimeUsec, TimeUnit.MICROSECONDS).enableBatching(true); } - if (arguments.batchMaxMessages > 0) { - producerBuilder.batchingMaxMessages(arguments.batchMaxMessages); + if (this.batchMaxMessages > 0) { + producerBuilder.batchingMaxMessages(this.batchMaxMessages); } - if (arguments.batchMaxBytes > 0) { - producerBuilder.batchingMaxBytes(arguments.batchMaxBytes); + if (this.batchMaxBytes > 0) { + producerBuilder.batchingMaxBytes(this.batchMaxBytes); } // Block if queue is full else we will start seeing errors in sendAsync producerBuilder.blockIfQueueFull(true); - if (isNotBlank(arguments.encKeyName) && isNotBlank(arguments.encKeyFile)) { - producerBuilder.addEncryptionKey(arguments.encKeyName); - producerBuilder.defaultCryptoKeyReader(arguments.encKeyFile); + if (isNotBlank(this.encKeyName) && isNotBlank(this.encKeyFile)) { + producerBuilder.addEncryptionKey(this.encKeyName); + producerBuilder.defaultCryptoKeyReader(this.encKeyFile); } return producerBuilder; } - private static void runProducer(int producerId, - Arguments arguments, + private void runProducer(int producerId, + PerformanceProducer arguments, long numMessages, int msgRate, List payloadByteList, @@ -516,31 +495,31 @@ private static void runProducer(int producerId, ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTransaction(arguments.isEnableTransaction); + .enableTransaction(this.isEnableTransaction); client = clientBuilder.build(); - ProducerBuilder producerBuilder = createProducerBuilder(client, arguments, producerId); + ProducerBuilder producerBuilder = createProducerBuilder(client, producerId); AtomicReference transactionAtomicReference; - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { producerBuilder.sendTimeout(0, TimeUnit.SECONDS); transactionAtomicReference = new AtomicReference<>(client.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS) + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS) .build() .get()); } else { transactionAtomicReference = new AtomicReference<>(null); } - for (int i = 0; i < arguments.numTopics; i++) { + for (int i = 0; i < this.numTopics; i++) { - String topic = arguments.topics.get(i); - log.info("Adding {} publishers on topic {}", arguments.numProducers, topic); + String topic = this.topics.get(i); + log.info("Adding {} publishers on topic {}", this.numProducers, topic); - for (int j = 0; j < arguments.numProducers; j++) { + for (int j = 0; j < this.numProducers; j++) { ProducerBuilder prodBuilder = producerBuilder.clone().topic(topic); - if (arguments.chunkingAllowed) { + if (this.chunkingAllowed) { prodBuilder.enableChunking(true); prodBuilder.enableBatching(false); } @@ -559,12 +538,12 @@ private static void runProducer(int producerId, RateLimiter rateLimiter = RateLimiter.create(msgRate); long startTime = System.nanoTime(); - long warmupEndTime = startTime + (long) (arguments.warmupTimeSeconds * 1e9); - long testEndTime = startTime + (long) (arguments.testTime * 1e9); + long warmupEndTime = startTime + (long) (this.warmupTimeSeconds * 1e9); + long testEndTime = startTime + (long) (this.testTime * 1e9); MessageKeyGenerationMode msgKeyMode = null; - if (isNotBlank(arguments.messageKeyGenerationMode)) { + if (isNotBlank(this.messageKeyGenerationMode)) { try { - msgKeyMode = MessageKeyGenerationMode.valueOf(arguments.messageKeyGenerationMode); + msgKeyMode = MessageKeyGenerationMode.valueOf(this.messageKeyGenerationMode); } catch (IllegalArgumentException e) { throw new IllegalArgumentException("messageKeyGenerationMode only support [autoIncrement, random]"); } @@ -572,16 +551,16 @@ private static void runProducer(int producerId, // Send messages on all topics/producers AtomicLong totalSent = new AtomicLong(0); AtomicLong numMessageSend = new AtomicLong(0); - Semaphore numMsgPerTxnLimit = new Semaphore(arguments.numMessagesPerTransaction); + Semaphore numMsgPerTxnLimit = new Semaphore(this.numMessagesPerTransaction); while (true) { if (produceEnough) { break; } for (Producer producer : producers) { - if (arguments.testTime > 0) { + if (this.testTime > 0) { if (System.nanoTime() > testEndTime) { log.info("------------- DONE (reached the maximum duration: [{} seconds] of production) " - + "--------------", arguments.testTime); + + "--------------", this.testTime); doneLatch.countDown(); produceEnough = true; break; @@ -604,9 +583,9 @@ private static void runProducer(int producerId, byte[] payloadData; - if (arguments.payloadFilename != null) { + if (this.payloadFilename != null) { if (messageFormatter != null) { - payloadData = messageFormatter.formatMessage(arguments.producerName, totalSent.get(), + payloadData = messageFormatter.formatMessage(this.producerName, totalSent.get(), payloadByteList.get(ThreadLocalRandom.current().nextInt(payloadByteList.size()))); } else { payloadData = payloadByteList.get( @@ -616,8 +595,8 @@ private static void runProducer(int producerId, payloadData = payloadBytes; } TypedMessageBuilder messageBuilder; - if (arguments.isEnableTransaction) { - if (arguments.numMessagesPerTransaction > 0) { + if (this.isEnableTransaction) { + if (this.numMessagesPerTransaction > 0) { try { numMsgPerTxnLimit.acquire(); } catch (InterruptedException exception){ @@ -630,14 +609,14 @@ private static void runProducer(int producerId, messageBuilder = producer.newMessage() .value(payloadData); } - if (arguments.delay > 0) { - messageBuilder.deliverAfter(arguments.delay, TimeUnit.SECONDS); - } else if (arguments.delayRange != null) { + if (this.delay > 0) { + messageBuilder.deliverAfter(this.delay, TimeUnit.SECONDS); + } else if (this.delayRange != null) { final long deliverAfter = ThreadLocalRandom.current() - .nextLong(arguments.delayRange.lowerEndpoint(), arguments.delayRange.upperEndpoint()); + .nextLong(this.delayRange.lowerEndpoint(), this.delayRange.upperEndpoint()); messageBuilder.deliverAfter(deliverAfter, TimeUnit.SECONDS); } - if (arguments.setEventTime) { + if (this.setEventTime) { messageBuilder.eventTime(System.currentTimeMillis()); } //generate msg key @@ -668,14 +647,14 @@ private static void runProducer(int producerId, } log.warn("Write message error with exception", ex); messagesFailed.increment(); - if (arguments.exitOnFailure) { + if (this.exitOnFailure) { PerfClientUtils.exit(1); } return null; }); - if (arguments.isEnableTransaction - && numMessageSend.incrementAndGet() == arguments.numMessagesPerTransaction) { - if (!arguments.isAbortTransaction) { + if (this.isEnableTransaction + && numMessageSend.incrementAndGet() == this.numMessagesPerTransaction) { + if (!this.isAbortTransaction) { transaction.commit() .thenRun(() -> { if (log.isDebugEnabled()) { @@ -709,11 +688,11 @@ private static void runProducer(int producerId, while (true) { try { Transaction newTransaction = pulsarClient.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS).build().get(); transactionAtomicReference.compareAndSet(transaction, newTransaction); numMessageSend.set(0); - numMsgPerTxnLimit.release(arguments.numMessagesPerTransaction); + numMsgPerTxnLimit.release(this.numMessagesPerTransaction); totalNumTxnOpenTxnSuccess.increment(); break; } catch (Exception e){ @@ -740,7 +719,7 @@ private static void runProducer(int producerId, } } - private static void printAggregatedThroughput(long start, Arguments arguments) { + private void printAggregatedThroughput(long start) { double elapsed = (System.nanoTime() - start) / 1e9; double rate = totalMessagesSent.sum() / elapsed; double throughput = totalBytesSent.sum() / elapsed / 1024 / 1024 * 8; @@ -750,7 +729,7 @@ private static void printAggregatedThroughput(long start, Arguments arguments) { long numTransactionOpenFailed = 0; long numTransactionOpenSuccess = 0; - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { totalTxnSuccess = totalEndTxnOpSuccessNum.sum(); totalTxnFail = totalEndTxnOpFailNum.sum(); rateOpenTxn = elapsed / (totalTxnFail + totalTxnSuccess); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceReader.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceReader.java index 3572cbde43cb7..3c6940b262f44 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceReader.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceReader.java @@ -24,7 +24,6 @@ import java.text.DecimalFormat; import java.util.ArrayList; import java.util.List; -import java.util.Properties; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.CompletableFuture; @@ -46,9 +45,9 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ScopeType; -public class PerformanceReader { +@Command(name = "read", description = "Test pulsar reader performance.") +public class PerformanceReader extends PerformanceTopicListArguments { private static final LongAdder messagesReceived = new LongAdder(); private static final LongAdder bytesReceived = new LongAdder(); private static final DecimalFormat intFormat = new PaddingDecimalFormat("0", 7); @@ -60,62 +59,53 @@ public class PerformanceReader { private static Recorder recorder = new Recorder(TimeUnit.DAYS.toMillis(10), 5); private static Recorder cumulativeRecorder = new Recorder(TimeUnit.DAYS.toMillis(10), 5); - @Command(description = "Test pulsar reader performance.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments extends PerformanceTopicListArguments { + @Option(names = {"-r", "--rate"}, description = "Simulate a slow message reader (rate in msg/s)") + public double rate = 0; - @Option(names = { "-r", "--rate" }, description = "Simulate a slow message reader (rate in msg/s)") - public double rate = 0; + @Option(names = {"-m", + "--start-message-id"}, description = "Start message id. This can be either 'earliest', " + + "'latest' or a specific message id by using 'lid:eid'") + public String startMessageId = "earliest"; - @Option(names = { "-m", - "--start-message-id" }, description = "Start message id. This can be either 'earliest', " - + "'latest' or a specific message id by using 'lid:eid'") - public String startMessageId = "earliest"; + @Option(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue") + public int receiverQueueSize = 1000; - @Option(names = { "-q", "--receiver-queue-size" }, description = "Size of the receiver queue") - public int receiverQueueSize = 1000; + @Option(names = {"-n", + "--num-messages"}, description = "Number of messages to consume in total. If <= 0, " + + "it will keep consuming") + public long numMessages = 0; - @Option(names = {"-n", - "--num-messages"}, description = "Number of messages to consume in total. If <= 0, " - + "it will keep consuming") - public long numMessages = 0; + @Option(names = { + "--use-tls"}, description = "Use TLS encryption on the connection", descriptionKey = "useTls") + public boolean useTls; - @Option(names = { - "--use-tls" }, description = "Use TLS encryption on the connection") - public boolean useTls; - - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep consuming") - public long testTime = 0; + @Option(names = {"-time", + "--test-duration"}, description = "Test duration in secs. If <= 0, it will keep consuming") + public long testTime = 0; + public PerformanceReader() { + super("read"); + } - @Override - public void fillArgumentsFromProperties(Properties prop) { - if (!useTls) { - useTls = Boolean.parseBoolean(prop.getProperty("useTls")); - } - } - @Override - public void validate() throws Exception { - super.validate(); - if (startMessageId != "earliest" && startMessageId != "latest" - && (startMessageId.split(":")).length != 2) { - String errMsg = String.format("invalid start message ID '%s', must be either either 'earliest', " - + "'latest' or a specific message id by using 'lid:eid'", startMessageId); - throw new Exception(errMsg); - } + @Override + public void validate() throws Exception { + super.validate(); + if (startMessageId != "earliest" && startMessageId != "latest" + && (startMessageId.split(":")).length != 2) { + String errMsg = String.format("invalid start message ID '%s', must be either either 'earliest', " + + "'latest' or a specific message id by using 'lid:eid'", startMessageId); + throw new Exception(errMsg); } } - public static void main(String[] args) throws Exception { - final Arguments arguments = new Arguments(); - arguments.parseCLI("pulsar-perf read", args); - + @Override + public void run() throws Exception { // Dump config variables PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar performance reader with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar performance reader with config: {}", w.writeValueAsString(this)); - final RateLimiter limiter = arguments.rate > 0 ? RateLimiter.create(arguments.rate) : null; + final RateLimiter limiter = this.rate > 0 ? RateLimiter.create(this.rate) : null; ReaderListener listener = (reader, msg) -> { messagesReceived.increment(); bytesReceived.add(msg.getData().length); @@ -123,9 +113,9 @@ public static void main(String[] args) throws Exception { totalMessagesReceived.increment(); totalBytesReceived.add(msg.getData().length); - if (arguments.numMessages > 0 && totalMessagesReceived.sum() >= arguments.numMessages) { + if (this.numMessages > 0 && totalMessagesReceived.sum() >= this.numMessages) { log.info("------------- DONE (reached the maximum number: [{}] of consumption) --------------", - arguments.numMessages); + this.numMessages); PerfClientUtils.exit(0); } @@ -140,37 +130,37 @@ public static void main(String[] args) throws Exception { } }; - ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTls(arguments.useTls); + ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(this) + .enableTls(this.useTls); PulsarClient pulsarClient = clientBuilder.build(); List>> futures = new ArrayList<>(); MessageId startMessageId; - if ("earliest".equals(arguments.startMessageId)) { + if ("earliest".equals(this.startMessageId)) { startMessageId = MessageId.earliest; - } else if ("latest".equals(arguments.startMessageId)) { + } else if ("latest".equals(this.startMessageId)) { startMessageId = MessageId.latest; } else { - String[] parts = arguments.startMessageId.split(":"); + String[] parts = this.startMessageId.split(":"); startMessageId = new MessageIdImpl(Long.parseLong(parts[0]), Long.parseLong(parts[1]), -1); } ReaderBuilder readerBuilder = pulsarClient.newReader() // .readerListener(listener) // - .receiverQueueSize(arguments.receiverQueueSize) // + .receiverQueueSize(this.receiverQueueSize) // .startMessageId(startMessageId); - for (int i = 0; i < arguments.numTopics; i++) { - final TopicName topicName = TopicName.get(arguments.topics.get(i)); + for (int i = 0; i < this.numTopics; i++) { + final TopicName topicName = TopicName.get(this.topics.get(i)); futures.add(readerBuilder.clone().topic(topicName.toString()).createAsync()); } FutureUtil.waitForAll(futures).get(); - log.info("Start reading from {} topics", arguments.numTopics); + log.info("Start reading from {} topics", this.numTopics); final long start = System.nanoTime(); Runtime.getRuntime().addShutdownHook(new Thread(() -> { @@ -178,17 +168,17 @@ public static void main(String[] args) throws Exception { printAggregatedStats(); })); - if (arguments.testTime > 0) { + if (this.testTime > 0) { TimerTask timoutTask = new TimerTask() { @Override public void run() { log.info("------------- DONE (reached the maximum duration: [{} seconds] of consumption) " - + "--------------", arguments.testTime); + + "--------------", testTime); PerfClientUtils.exit(0); } }; Timer timer = new Timer(); - timer.schedule(timoutTask, arguments.testTime * 1000); + timer.schedule(timoutTask, this.testTime * 1000); } long oldTime = System.nanoTime(); @@ -223,7 +213,6 @@ public void run() { pulsarClient.close(); } - private static void printAggregatedThroughput(long start) { double elapsed = (System.nanoTime() - start) / 1e9; double rate = totalMessagesReceived.sum() / elapsed; diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTopicListArguments.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTopicListArguments.java index 9ac99d0abcca5..e4771c3652fb1 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTopicListArguments.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTopicListArguments.java @@ -39,6 +39,10 @@ public abstract class PerformanceTopicListArguments extends PerformanceBaseArgum ) public int numTopics = 1; + public PerformanceTopicListArguments(String cmdName) { + super(cmdName); + } + @Override public void validate() throws Exception { super.validate(); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java index 02e50ab4e2bb9..943cfaf451032 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java @@ -23,14 +23,12 @@ import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.util.concurrent.RateLimiter; import java.io.FileOutputStream; -import java.io.IOException; import java.io.PrintStream; import java.text.DecimalFormat; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Properties; import java.util.Random; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -65,9 +63,9 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ScopeType; -public class PerformanceTransaction { +@Command(name = "transaction", description = "Test pulsar transaction performance.") +public class PerformanceTransaction extends PerformanceBaseArguments{ private static final LongAdder totalNumEndTxnOpFailed = new LongAdder(); private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder(); @@ -90,132 +88,120 @@ public class PerformanceTransaction { private static final Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5); - @Command(description = "Test pulsar transaction performance.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments extends PerformanceBaseArguments { + @Option(names = "--topics-c", description = "All topics that need ack for a transaction", required = + true) + public List consumerTopic = Collections.singletonList("test-consume"); - @Option(names = "--topics-c", description = "All topics that need ack for a transaction", required = - true) - public List consumerTopic = Collections.singletonList("test-consume"); + @Option(names = "--topics-p", description = "All topics that need produce for a transaction", + required = true) + public List producerTopic = Collections.singletonList("test-produce"); - @Option(names = "--topics-p", description = "All topics that need produce for a transaction", - required = true) - public List producerTopic = Collections.singletonList("test-produce"); + @Option(names = {"-threads", "--num-test-threads"}, description = "Number of test threads." + + "This thread is for a new transaction to ack messages from consumer topics and produce message to " + + "producer topics, and then commit or abort this transaction. " + + "Increasing the number of threads increases the parallelism of the performance test, " + + "thereby increasing the intensity of the stress test.") + public int numTestThreads = 1; - @Option(names = {"-threads", "--num-test-threads"}, description = "Number of test threads." - + "This thread is for a new transaction to ack messages from consumer topics and produce message to " - + "producer topics, and then commit or abort this transaction. " - + "Increasing the number of threads increases the parallelism of the performance test, " - + "thereby increasing the intensity of the stress test.") - public int numTestThreads = 1; + @Option(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL", descriptionKey = "webServiceUrl") + public String adminURL; - @Option(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL") - public String adminURL; + @Option(names = {"-np", + "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means" + + "not trying to create a topic") + public Integer partitions = null; - @Option(names = {"-np", - "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means" - + "not trying to create a topic") - public Integer partitions = null; + @Option(names = {"-time", + "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing") + public long testTime = 0; - @Option(names = {"-time", - "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing") - public long testTime = 0; + @Option(names = {"-ss", + "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)") + public List subscriptions = Collections.singletonList("sub"); - @Option(names = {"-ss", - "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)") - public List subscriptions = Collections.singletonList("sub"); + @Option(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)") + public int numSubscriptions = 1; - @Option(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)") - public int numSubscriptions = 1; + @Option(names = {"-sp", "--subscription-position"}, description = "Subscription position") + private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest; - @Option(names = {"-sp", "--subscription-position"}, description = "Subscription position") - private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest; + @Option(names = {"-st", "--subscription-type"}, description = "Subscription type") + public SubscriptionType subscriptionType = SubscriptionType.Shared; - @Option(names = {"-st", "--subscription-type"}, description = "Subscription type") - public SubscriptionType subscriptionType = SubscriptionType.Shared; + @Option(names = {"-rs", "--replicated" }, + description = "Whether the subscription status should be replicated") + private boolean replicatedSubscription = false; - @Option(names = {"-rs", "--replicated" }, - description = "Whether the subscription status should be replicated") - private boolean replicatedSubscription = false; + @Option(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue") + public int receiverQueueSize = 1000; - @Option(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue") - public int receiverQueueSize = 1000; + @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," + + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") + public long transactionTimeout = 5; - @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," - + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") - public long transactionTimeout = 5; + @Option(names = {"-ntxn", + "--number-txn"}, description = "Set the number of transaction. 0 means keeping open." + + "If transaction disabled, it means the number of tasks. The task or transaction produces or " + + "consumes a specified number of messages.") + public long numTransactions = 0; - @Option(names = {"-ntxn", - "--number-txn"}, description = "Set the number of transaction. 0 means keeping open." - + "If transaction disabled, it means the number of tasks. The task or transaction produces or " - + "consumes a specified number of messages.") - public long numTransactions = 0; + @Option(names = {"-nmp", "--numMessage-perTransaction-produce"}, + description = "Set the number of messages produced in a transaction." + + "If transaction disabled, it means the number of messages produced in a task.") + public int numMessagesProducedPerTransaction = 1; - @Option(names = {"-nmp", "--numMessage-perTransaction-produce"}, - description = "Set the number of messages produced in a transaction." - + "If transaction disabled, it means the number of messages produced in a task.") - public int numMessagesProducedPerTransaction = 1; + @Option(names = {"-nmc", "--numMessage-perTransaction-consume"}, + description = "Set the number of messages consumed in a transaction." + + "If transaction disabled, it means the number of messages consumed in a task.") + public int numMessagesReceivedPerTransaction = 1; - @Option(names = {"-nmc", "--numMessage-perTransaction-consume"}, - description = "Set the number of messages consumed in a transaction." - + "If transaction disabled, it means the number of messages consumed in a task.") - public int numMessagesReceivedPerTransaction = 1; + @Option(names = {"--txn-disable"}, description = "Disable transaction") + public boolean isDisableTransaction = false; - @Option(names = {"--txn-disable"}, description = "Disable transaction") - public boolean isDisableTransaction = false; + @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable " + + "setting to false, -abort takes effect)") + public boolean isAbortTransaction = false; - @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable " - + "setting to false, -abort takes effect)") - public boolean isAbortTransaction = false; - - @Option(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit") - public int openTxnRate = 0; - - @Override - public void fillArgumentsFromProperties(Properties prop) { - if (adminURL == null) { - adminURL = prop.getProperty("webServiceUrl"); - } - if (adminURL == null) { - adminURL = prop.getProperty("adminURL", "http://localhost:8080/"); - } - } + @Option(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit") + public int openTxnRate = 0; + public PerformanceTransaction() { + super("transaction"); } - public static void main(String[] args) - throws IOException, PulsarAdminException, ExecutionException, InterruptedException { - final Arguments arguments = new Arguments(); - arguments.parseCLI("pulsar-perf transaction", args); + @Override + public void run() throws Exception { + super.parseCLI(); // Dump config variables PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(this)); final byte[] payloadBytes = new byte[1024]; Random random = new Random(0); for (int i = 0; i < payloadBytes.length; ++i) { payloadBytes[i] = (byte) (random.nextInt(26) + 65); } - if (arguments.partitions != null) { + if (this.partitions != null) { final PulsarAdminBuilder adminBuilder = PerfClientUtils - .createAdminBuilderFromArguments(arguments, arguments.adminURL); + .createAdminBuilderFromArguments(this, this.adminURL); try (PulsarAdmin adminClient = adminBuilder.build()) { - for (String topic : arguments.producerTopic) { - log.info("Creating produce partitioned topic {} with {} partitions", topic, arguments.partitions); + for (String topic : this.producerTopic) { + log.info("Creating produce partitioned topic {} with {} partitions", topic, this.partitions); try { - adminClient.topics().createPartitionedTopic(topic, arguments.partitions); + adminClient.topics().createPartitionedTopic(topic, this.partitions); } catch (PulsarAdminException.ConflictException alreadyExists) { if (log.isDebugEnabled()) { log.debug("Topic {} already exists: {}", topic, alreadyExists); } PartitionedTopicMetadata partitionedTopicMetadata = adminClient.topics().getPartitionedTopicMetadata(topic); - if (partitionedTopicMetadata.partitions != arguments.partitions) { + if (partitionedTopicMetadata.partitions != this.partitions) { log.error( "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}", - topic, partitionedTopicMetadata.partitions, arguments.partitions); + topic, partitionedTopicMetadata.partitions, this.partitions); PerfClientUtils.exit(1); } } @@ -223,21 +209,21 @@ public static void main(String[] args) } } - ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTransaction(!arguments.isDisableTransaction); + ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(this) + .enableTransaction(!this.isDisableTransaction); try (PulsarClient client = clientBuilder.build()) { - ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads, - arguments.numTestThreads, + ExecutorService executorService = new ThreadPoolExecutor(this.numTestThreads, + this.numTestThreads, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>()); long startTime = System.nanoTime(); - long testEndTime = startTime + (long) (arguments.testTime * 1e9); + long testEndTime = startTime + (long) (this.testTime * 1e9); Runtime.getRuntime().addShutdownHook(new Thread(() -> { - if (!arguments.isDisableTransaction) { + if (!this.isDisableTransaction) { printTxnAggregatedThroughput(startTime); } else { printAggregatedThroughput(startTime); @@ -248,10 +234,10 @@ public static void main(String[] args) // start perf test AtomicBoolean executing = new AtomicBoolean(true); - RateLimiter rateLimiter = arguments.openTxnRate > 0 - ? RateLimiter.create(arguments.openTxnRate) + RateLimiter rateLimiter = this.openTxnRate > 0 + ? RateLimiter.create(this.openTxnRate) : null; - for (int i = 0; i < arguments.numTestThreads; i++) { + for (int i = 0; i < this.numTestThreads; i++) { executorService.submit(() -> { //The producer and consumer clients are built in advance, and then this thread is //responsible for the production and consumption tasks of the transaction through the loop. @@ -260,11 +246,11 @@ public static void main(String[] args) List>> consumers = null; AtomicReference atomicReference = null; try { - producers = buildProducers(client, arguments); - consumers = buildConsumer(client, arguments); - if (!arguments.isDisableTransaction) { + producers = buildProducers(client); + consumers = buildConsumer(client); + if (!this.isDisableTransaction) { atomicReference = new AtomicReference<>(client.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS) + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS) .build() .get()); } else { @@ -278,11 +264,11 @@ public static void main(String[] args) //The while loop has no break, and finally ends the execution through the shutdownNow of //the executorService while (true) { - if (arguments.numTransactions > 0) { + if (this.numTransactions > 0) { if (totalNumTxnOpenTxnFail.sum() - + totalNumTxnOpenTxnSuccess.sum() >= arguments.numTransactions) { + + totalNumTxnOpenTxnSuccess.sum() >= this.numTransactions) { if (totalNumEndTxnOpFailed.sum() - + totalNumEndTxnOpSuccess.sum() < arguments.numTransactions) { + + totalNumEndTxnOpSuccess.sum() < this.numTransactions) { continue; } log.info("------------------- DONE -----------------------"); @@ -292,7 +278,7 @@ public static void main(String[] args) break; } } - if (arguments.testTime > 0) { + if (this.testTime > 0) { if (System.nanoTime() > testEndTime) { log.info("------------------- DONE -----------------------"); executing.compareAndSet(true, false); @@ -304,7 +290,7 @@ public static void main(String[] args) Transaction transaction = atomicReference.get(); for (List> subscriptions : consumers) { for (Consumer consumer : subscriptions) { - for (int j = 0; j < arguments.numMessagesReceivedPerTransaction; j++) { + for (int j = 0; j < this.numMessagesReceivedPerTransaction; j++) { Message message = null; try { message = consumer.receive(); @@ -314,7 +300,7 @@ public static void main(String[] args) PerfClientUtils.exit(1); } long receiveTime = System.nanoTime(); - if (!arguments.isDisableTransaction) { + if (!this.isDisableTransaction) { consumer.acknowledgeAsync(message.getMessageId(), transaction) .thenRun(() -> { long latencyMicros = NANOSECONDS.toMicros( @@ -355,9 +341,9 @@ public static void main(String[] args) } for (Producer producer : producers) { - for (int j = 0; j < arguments.numMessagesProducedPerTransaction; j++) { + for (int j = 0; j < this.numMessagesProducedPerTransaction; j++) { long sendTime = System.nanoTime(); - if (!arguments.isDisableTransaction) { + if (!this.isDisableTransaction) { producer.newMessage(transaction).value(payloadBytes) .sendAsync().thenRun(() -> { long latencyMicros = NANOSECONDS.toMicros( @@ -397,8 +383,8 @@ public static void main(String[] args) if (rateLimiter != null) { rateLimiter.tryAcquire(); } - if (!arguments.isDisableTransaction) { - if (!arguments.isAbortTransaction) { + if (!this.isDisableTransaction) { + if (!this.isAbortTransaction) { transaction.commit() .thenRun(() -> { numTxnOpSuccess.increment(); @@ -431,7 +417,7 @@ public static void main(String[] args) while (true) { try { Transaction newTransaction = client.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS) + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS) .build() .get(); atomicReference.compareAndSet(transaction, newTransaction); @@ -483,7 +469,7 @@ public static void main(String[] args) double rate = numTxnOpSuccess.sumThenReset() / elapsed; reportSendHistogram = messageSendRecorder.getIntervalHistogram(reportSendHistogram); reportAckHistogram = messageAckRecorder.getIntervalHistogram(reportAckHistogram); - String txnOrTaskLog = !arguments.isDisableTransaction + String txnOrTaskLog = !this.isDisableTransaction ? "Throughput transaction: {} transaction executes --- {} transaction/s" : "Throughput task: {} task executes --- {} task/s"; log.info( @@ -609,24 +595,24 @@ private static void printAggregatedStats() { private static final Logger log = LoggerFactory.getLogger(PerformanceTransaction.class); - private static List>> buildConsumer(PulsarClient client, Arguments arguments) + private List>> buildConsumer(PulsarClient client) throws ExecutionException, InterruptedException { ConsumerBuilder consumerBuilder = client.newConsumer(Schema.BYTES) - .subscriptionType(arguments.subscriptionType) - .receiverQueueSize(arguments.receiverQueueSize) - .subscriptionInitialPosition(arguments.subscriptionInitialPosition) - .replicateSubscriptionState(arguments.replicatedSubscription); + .subscriptionType(this.subscriptionType) + .receiverQueueSize(this.receiverQueueSize) + .subscriptionInitialPosition(this.subscriptionInitialPosition) + .replicateSubscriptionState(this.replicatedSubscription); - Iterator consumerTopicsIterator = arguments.consumerTopic.iterator(); - List>> consumers = new ArrayList<>(arguments.consumerTopic.size()); + Iterator consumerTopicsIterator = this.consumerTopic.iterator(); + List>> consumers = new ArrayList<>(this.consumerTopic.size()); while (consumerTopicsIterator.hasNext()){ String topic = consumerTopicsIterator.next(); - final List> subscriptions = new ArrayList<>(arguments.numSubscriptions); + final List> subscriptions = new ArrayList<>(this.numSubscriptions); final List>> subscriptionFutures = - new ArrayList<>(arguments.numSubscriptions); + new ArrayList<>(this.numSubscriptions); log.info("Create subscriptions for topic {}", topic); - for (int j = 0; j < arguments.numSubscriptions; j++) { - String subscriberName = arguments.subscriptions.get(j); + for (int j = 0; j < this.numSubscriptions; j++) { + String subscriberName = this.subscriptions.get(j); subscriptionFutures .add(consumerBuilder.clone().topic(topic).subscriptionName(subscriberName) .subscribeAsync()); @@ -639,14 +625,14 @@ private static List>> buildConsumer(PulsarClient client, return consumers; } - private static List> buildProducers(PulsarClient client, Arguments arguments) + private List> buildProducers(PulsarClient client) throws ExecutionException, InterruptedException { ProducerBuilder producerBuilder = client.newProducer(Schema.BYTES) .sendTimeout(0, TimeUnit.SECONDS); final List>> producerFutures = new ArrayList<>(); - for (String topic : arguments.producerTopic) { + for (String topic : this.producerTopic) { log.info("Create producer for topic {}", topic); producerFutures.add(producerBuilder.clone().topic(topic).createAsync()); } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ProxyProtocolConverter.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ProxyProtocolConverter.java new file mode 100644 index 0000000000000..6cccc8ce480ae --- /dev/null +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ProxyProtocolConverter.java @@ -0,0 +1,35 @@ +/* + * 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 org.apache.pulsar.testclient; + +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.api.ProxyProtocol; +import picocli.CommandLine.ITypeConverter; + +public class ProxyProtocolConverter implements ITypeConverter { + + @Override + public ProxyProtocol convert(String value) throws Exception { + String proxyProtocolString = StringUtils.trimToNull(value); + if (proxyProtocolString != null) { + return ProxyProtocol.valueOf(proxyProtocolString.toUpperCase()); + } + return null; + } +} diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PulsarPerfTestTool.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PulsarPerfTestTool.java new file mode 100644 index 0000000000000..826060dc6b799 --- /dev/null +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PulsarPerfTestTool.java @@ -0,0 +1,133 @@ +/* + * 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 org.apache.pulsar.testclient; + +import static org.apache.commons.lang3.StringUtils.isBlank; +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import java.io.FileInputStream; +import java.lang.reflect.Constructor; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import org.apache.pulsar.proxy.socket.client.PerformanceClient; +import picocli.CommandLine; + +@CommandLine.Command(name = "pulsar-perf", + scope = CommandLine.ScopeType.INHERIT, + mixinStandardHelpOptions = true, + showDefaultValues = true +) +public class PulsarPerfTestTool { + + protected Map> commandMap; + protected final CommandLine commander; + + public PulsarPerfTestTool() { + this.commander = new CommandLine(this); + commandMap = new HashMap<>(); + } + + private String[] initCommander(String[] args) throws Exception { + commandMap.put("produce", PerformanceProducer.class); + commandMap.put("consume", PerformanceConsumer.class); + commandMap.put("transaction", PerformanceTransaction.class); + commandMap.put("read", PerformanceReader.class); + commandMap.put("monitor-brokers", BrokerMonitor.class); + commandMap.put("simulation-client", LoadSimulationClient.class); + commandMap.put("simulation-controller", LoadSimulationController.class); + commandMap.put("websocket-producer", PerformanceClient.class); + commandMap.put("managed-ledger", ManagedLedgerWriter.class); + commandMap.put("gen-doc", CmdGenerateDocumentation.class); + if (args.length == 0) { + System.out.println("Usage: pulsar-perf CONF_FILE_PATH [options] [command] [command options]"); + PerfClientUtils.exit(0); + } + String configFile = args[0]; + Properties prop = new Properties(System.getProperties()); + if (configFile != null) { + try (FileInputStream fis = new FileInputStream(configFile)) { + prop.load(fis); + } + } + commander.setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + + for (Map.Entry> c : commandMap.entrySet()) { + Constructor constructor = c.getValue().getDeclaredConstructor(); + constructor.setAccessible(true); + addCommand(c.getKey(), constructor.newInstance()); + } + + // Remove the first argument, it's the config file path + return Arrays.copyOfRange(args, 1, args.length); + } + + private void addCommand(String name, Object o) { + if (o instanceof CmdBase) { + commander.addSubcommand(name, ((CmdBase) o).getCommander()); + } else { + commander.addSubcommand(o); + } + } + + public static void main(String[] args) throws Exception { + PulsarPerfTestTool tool = new PulsarPerfTestTool(); + args = tool.initCommander(args); + + if (tool.run(args)) { + PerfClientUtils.exit(0); + } else { + PerfClientUtils.exit(1); + } + } + + protected boolean run(String[] args) { + return commander.execute(args) == 0; + } + +} + +class PulsarPerfTestPropertiesProvider extends CommandLine.PropertiesDefaultProvider{ + private static final String brokerServiceUrlKey = "brokerServiceUrl"; + private static final String webServiceUrlKey = "webServiceUrl"; + private final Properties properties; + + public PulsarPerfTestPropertiesProvider(Properties properties) { + super(properties); + this.properties = properties; + } + + static PulsarPerfTestPropertiesProvider create(Properties properties) { + if (isBlank(properties.getProperty(brokerServiceUrlKey))) { + String webServiceUrl = properties.getProperty("webServiceUrl"); + if (isNotBlank(webServiceUrl)) { + properties.put(brokerServiceUrlKey, webServiceUrl); + } else if (isNotBlank(properties.getProperty("serviceUrl"))) { + properties.put(brokerServiceUrlKey, properties.getProperty("serviceUrl", "http://localhost:8080/")); + } + } + + // Used for produce and transaction to fill parameters. + if (isBlank(properties.getProperty(webServiceUrlKey))) { + properties.put(webServiceUrlKey, properties.getProperty("adminURL", "http://localhost:8080/")); + } + + return new PulsarPerfTestPropertiesProvider(properties); + } +} diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/proxy/socket/client/PerformanceClientTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/proxy/socket/client/PerformanceClientTest.java index f623662e0e946..d45c3e8f3a4e7 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/proxy/socket/client/PerformanceClientTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/proxy/socket/client/PerformanceClientTest.java @@ -31,29 +31,34 @@ public void testLoadArguments() throws Exception { PerformanceClient client = new PerformanceClient(); // "--proxy-url" has the highest priority - PerformanceClient.Arguments arguments = client.loadArguments( - getArgs("ws://broker0.pulsar.apache.org:8080/", "./src/test/resources/websocket_client1.conf")); - assertEquals(arguments.proxyURL, "ws://broker0.pulsar.apache.org:8080/"); + client.parse(getArgs("ws://broker0.pulsar.apache.org:8080/", "./src/test/resources/websocket_client1.conf")); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://broker0.pulsar.apache.org:8080/"); // "webSocketServiceUrl" written in the conf file has the second priority - arguments = client.loadArguments(getArgs(null, "./src/test/resources/websocket_client1.conf")); - assertEquals(arguments.proxyURL, "ws://broker1.pulsar.apache.org:8080/"); + client.parse(getArgs(null, "./src/test/resources/websocket_client1.conf")); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://broker1.pulsar.apache.org:8080/"); // "webServiceUrl" written in the conf file has the third priority - arguments = client.loadArguments(getArgs(null, "./src/test/resources/websocket_client2.conf")); - assertEquals(arguments.proxyURL, "ws://broker2.pulsar.apache.org:8080/"); + client.parse(getArgs(null, "./src/test/resources/websocket_client2.conf")); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://broker2.pulsar.apache.org:8080/"); // "serviceUrl" written in the conf file has the fourth priority - arguments = client.loadArguments(getArgs(null, "./src/test/resources/websocket_client3.conf")); - assertEquals(arguments.proxyURL, "wss://broker3.pulsar.apache.org:8443/"); + client.parse(getArgs(null, "./src/test/resources/websocket_client3.conf")); + client.loadArguments(); + assertEquals(client.proxyURL, "wss://broker3.pulsar.apache.org:8443/"); // The default value is "ws://localhost:8080/" - arguments = client.loadArguments(getArgs(null, null)); - assertEquals(arguments.proxyURL, "ws://localhost:8080/"); + client.parse(getArgs(null, null)); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://localhost:8080/"); // If the URL does not end with "/", it will be added - arguments = client.loadArguments(getArgs("ws://broker0.pulsar.apache.org:8080", null)); - assertEquals(arguments.proxyURL, "ws://broker0.pulsar.apache.org:8080/"); + client.parse(getArgs("ws://broker0.pulsar.apache.org:8080", null)); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://broker0.pulsar.apache.org:8080/"); } private String[] getArgs(String proxyUrl, String confFile) { diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/GenerateDocumentionTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/GenerateDocumentionTest.java index 73d7751e33343..e76e0cca0cb76 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/GenerateDocumentionTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/GenerateDocumentionTest.java @@ -26,13 +26,13 @@ public class GenerateDocumentionTest { @Test public void testGenerateDocumention() throws Exception { - CmdGenerateDocumentation.main(new String[]{}); + new CmdGenerateDocumentation().run(new String[]{}); } @Test public void testSpecifyModuleName() throws Exception { String[] args = new String[]{"-n", "produce", "-n", "consume"}; - CmdGenerateDocumentation.main(args); + new CmdGenerateDocumentation().run(args); } private static final String DESC = "desc"; diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/Oauth2PerformanceTransactionTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/Oauth2PerformanceTransactionTest.java index f1be515e9c7f8..e8eeb3bf51993 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/Oauth2PerformanceTransactionTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/Oauth2PerformanceTransactionTest.java @@ -186,7 +186,7 @@ public void testTransactionPerf() throws Exception { Thread thread = new Thread(() -> { try { - PerformanceTransaction.main(args.split(" ")); + new PerformanceTransaction().run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerfClientUtilsTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerfClientUtilsTest.java index a7aa3b5a976e3..ed0d055ce1188 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerfClientUtilsTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerfClientUtilsTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.testclient; +import java.io.FileInputStream; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; @@ -55,7 +56,7 @@ public void close() throws IOException { @Test public void testClientCreation() throws Exception { - final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(); + final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(""); args.tlsHostnameVerificationEnable = true; args.authPluginClassName = MyAuth.class.getName(); @@ -97,7 +98,7 @@ public void testClientCreation() throws Exception { @Test public void testClientCreationWithProxy() throws Exception { - final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(); + final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(""); args.serviceURL = "pulsar+ssl://my-pulsar:6651"; args.proxyServiceURL = "pulsar+ssl://my-proxy-pulsar:4443"; @@ -120,11 +121,13 @@ public void testClientCreationWithProxyDefinedInConfFile() throws Exception { + "proxyServiceUrl=pulsar+ssl://my-proxy-pulsar:4443\n" + "proxyProtocol=SNI"); - final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(); - - args.confFile = testConf.toString(); - args.fillArgumentsFromProperties(); - + final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(""); + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(testConf.toString())) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + args.parse(new String[]{}); final ClientBuilderImpl builder = (ClientBuilderImpl) PerfClientUtils.createClientBuilderFromArguments(args); final ClientConfigurationData conf = builder.getClientConfigurationData(); @@ -145,16 +148,19 @@ public void testClientCreationWithEmptyProxyPropertyInConfFile() throws Exceptio + "proxyServiceUrl=\n" + "proxyProtocol="); - final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(); - - args.confFile = testConf.toString(); - args.fillArgumentsFromProperties(); + final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(""); + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(testConf.toString())) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + args.parse(new String[]{}); final ClientBuilderImpl builder = (ClientBuilderImpl) PerfClientUtils.createClientBuilderFromArguments(args); final ClientConfigurationData conf = builder.getClientConfigurationData(); - Assert.assertNull(conf.getProxyServiceUrl()); + Assert.assertEquals(conf.getProxyServiceUrl(),""); Assert.assertNull(conf.getProxyProtocol()); } finally { Files.deleteIfExists(testConf); @@ -163,7 +169,13 @@ public void testClientCreationWithEmptyProxyPropertyInConfFile() throws Exceptio } class PerformanceArgumentsTestDefault extends PerformanceBaseArguments { + public PerformanceArgumentsTestDefault(String cmdName) { + super(cmdName); + } + + @Override - public void fillArgumentsFromProperties(Properties prop) { + public void run() throws Exception { + } } diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceBaseArgumentsTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceBaseArgumentsTest.java index 699f138bfdaa8..9b54fa510cee2 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceBaseArgumentsTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceBaseArgumentsTest.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.fail; import java.io.File; +import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; import java.util.List; @@ -31,23 +32,28 @@ import org.testng.Assert; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import picocli.CommandLine; public class PerformanceBaseArgumentsTest { @Test - public void testReadFromConfigFile() { - - AtomicBoolean called = new AtomicBoolean(); - - final PerformanceBaseArguments args = new PerformanceBaseArguments() { + public void testReadFromConfigFile() throws Exception { + final PerformanceBaseArguments args = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - called.set(true); + public void run() throws Exception { + } }; - args.confFile = "./src/test/resources/perf_client1.conf"; - args.fillArgumentsFromProperties(); - Assert.assertTrue(called.get()); + + String confFile = "./src/test/resources/perf_client1.conf"; + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + args.parse(new String[]{}); + + Assert.assertEquals(args.serviceURL, "https://my-pulsar:8443/"); Assert.assertEquals(args.authPluginClassName, "org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth"); @@ -62,37 +68,41 @@ public void fillArgumentsFromProperties(Properties prop) { @Test public void testReadFromConfigFileWithoutProxyUrl() { - AtomicBoolean called = new AtomicBoolean(); - final PerformanceBaseArguments args = new PerformanceBaseArguments() { + final PerformanceBaseArguments args = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - called.set(true); + public void run() throws Exception { + } + }; + String confFile = "./src/test/resources/performance_client2.conf"; - File tempConfigFile = new File("./src/test/resources/performance_client2.conf"); + File tempConfigFile = new File(confFile); if (tempConfigFile.exists()) { tempConfigFile.delete(); } try { Properties props = new Properties(); - - Map configs = Map.of("brokerServiceUrl","https://my-pulsar:8443/", - "authPlugin","org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth", - "authParams", "myparams", - "tlsTrustCertsFilePath", "./path", - "tlsAllowInsecureConnection","true", - "tlsEnableHostnameVerification", "true" + + Map configs = Map.of("brokerServiceUrl", "https://my-pulsar:8443/", + "authPlugin", "org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth", + "authParams", "myparams", + "tlsTrustCertsFilePath", "./path", + "tlsAllowInsecureConnection", "true", + "tlsEnableHostnameVerification", "true" ); props.putAll(configs); FileOutputStream out = new FileOutputStream(tempConfigFile); props.store(out, "properties file"); out.close(); - args.confFile = "./src/test/resources/performance_client2.conf"; + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + args.parse(new String[]{}); - args.fillArgumentsFromProperties(); - Assert.assertTrue(called.get()); Assert.assertEquals(args.serviceURL, "https://my-pulsar:8443/"); Assert.assertEquals(args.authPluginClassName, "org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth"); @@ -100,7 +110,7 @@ public void fillArgumentsFromProperties(Properties prop) { Assert.assertEquals(args.tlsTrustCertsFilePath, "./path"); Assert.assertTrue(args.tlsAllowInsecureConnection); Assert.assertTrue(args.tlsHostnameVerificationEnable); - + } catch (IOException e) { e.printStackTrace(); fail("Error while updating/reading config file"); @@ -112,27 +122,27 @@ public void fillArgumentsFromProperties(Properties prop) { @Test public void testReadFromConfigFileProxyProtocolException() { - AtomicBoolean calledVar1 = new AtomicBoolean(); AtomicBoolean calledVar2 = new AtomicBoolean(); - final PerformanceBaseArguments args = new PerformanceBaseArguments() { + final PerformanceBaseArguments args = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - calledVar1.set(true); + public void run() throws Exception { + } }; - File tempConfigFile = new File("./src/test/resources/performance_client3.conf"); + String confFile = "./src/test/resources/performance_client3.conf"; + File tempConfigFile = new File(confFile); if (tempConfigFile.exists()) { tempConfigFile.delete(); } try { Properties props = new Properties(); - Map configs = Map.of("brokerServiceUrl","https://my-pulsar:8443/", - "authPlugin","org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth", + Map configs = Map.of("brokerServiceUrl", "https://my-pulsar:8443/", + "authPlugin", "org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth", "authParams", "myparams", "tlsTrustCertsFilePath", "./path", - "tlsAllowInsecureConnection","true", + "tlsAllowInsecureConnection", "true", "tlsEnableHostnameVerification", "true", "proxyServiceURL", "https://my-proxy-pulsar:4443/", "proxyProtocol", "TEST" @@ -141,15 +151,17 @@ public void fillArgumentsFromProperties(Properties prop) { FileOutputStream out = new FileOutputStream(tempConfigFile); props.store(out, "properties file"); out.close(); - args.confFile = "./src/test/resources/performance_client3.conf"; - PerfClientUtils.setExitProcedure(code -> { - calledVar2.set(true); - Assert.assertEquals(code, 1, "Incorrect exit code"); - }); - args.confFile = "./src/test/resources/performance_client3.conf"; - args.fillArgumentsFromProperties(); - Assert.assertTrue(calledVar1.get()); + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + try { + args.parse(new String[]{}); + }catch (CommandLine.ParameterException e){ + calledVar2.set(true); + } Assert.assertTrue(calledVar2.get()); } catch (IOException e) { e.printStackTrace(); @@ -161,15 +173,15 @@ public void fillArgumentsFromProperties(Properties prop) { @DataProvider(name = "memoryLimitCliArgumentProvider") public Object[][] memoryLimitCliArgumentProvider() { - return new Object[][] { - { new String[]{"-ml","1"}, 1L}, - { new String[]{"-ml","1K"}, 1024L}, - { new String[]{"--memory-limit", "1G"}, 1024 * 1024 * 1024} + return new Object[][]{ + {new String[]{"-ml", "1"}, 1L}, + {new String[]{"-ml", "1K"}, 1024L}, + {new String[]{"--memory-limit", "1G"}, 1024 * 1024 * 1024} }; } @Test(dataProvider = "memoryLimitCliArgumentProvider") - public void testMemoryLimitCliArgument(String[] cliArgs, long expectedMemoryLimit) { + public void testMemoryLimitCliArgument(String[] cliArgs, long expectedMemoryLimit) throws Exception { for (String cmd : List.of( "pulsar-perf read", "pulsar-perf produce", @@ -177,17 +189,24 @@ public void testMemoryLimitCliArgument(String[] cliArgs, long expectedMemoryLimi "pulsar-perf transaction" )) { // Arrange - AtomicBoolean called = new AtomicBoolean(); - final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments() { + final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - called.set(true); + public void run() throws Exception { + } + }; - baseArgument.confFile = "./src/test/resources/perf_client1.conf"; + String confFile = "./src/test/resources/perf_client1.conf"; + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + baseArgument.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + baseArgument.parse(new String[]{}); // Act - baseArgument.parseCLI(cmd, cliArgs); + baseArgument.parseCLI(); + baseArgument.getCommander().execute(cliArgs); // Assert assertEquals(baseArgument.memoryLimit, expectedMemoryLimit); @@ -196,15 +215,15 @@ public void fillArgumentsFromProperties(Properties prop) { @DataProvider(name = "invalidMemoryLimitCliArgumentProvider") public Object[][] invalidMemoryLimitCliArgumentProvider() { - return new Object[][] { - { new String[]{"-ml","-1"}}, - { new String[]{"-ml","1C"}}, - { new String[]{"--memory-limit", "1Q"}} + return new Object[][]{ + {new String[]{"-ml", "-1"}}, + {new String[]{"-ml", "1C"}}, + {new String[]{"--memory-limit", "1Q"}} }; } @Test - public void testMemoryLimitCliArgumentDefault() { + public void testMemoryLimitCliArgumentDefault() throws Exception { for (String cmd : List.of( "pulsar-perf read", "pulsar-perf produce", @@ -212,17 +231,23 @@ public void testMemoryLimitCliArgumentDefault() { "pulsar-perf transaction" )) { // Arrange - AtomicBoolean called = new AtomicBoolean(); - final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments() { + final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - called.set(true); + public void run() throws Exception { + } + }; - baseArgument.confFile = "./src/test/resources/perf_client1.conf"; + String confFile = "./src/test/resources/perf_client1.conf"; + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + baseArgument.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + baseArgument.parse(new String[]{}); // Act - baseArgument.parseCLI(cmd, new String[]{}); + baseArgument.parseCLI(); // Assert assertEquals(baseArgument.memoryLimit, 0L); diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java index 20679d8367677..d0b25c6971697 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java @@ -85,7 +85,8 @@ public void testMsgKey() throws Exception { String args = String.format(argString, topic, pulsar.getBrokerServiceUrl()); Thread thread = new Thread(() -> { try { - PerformanceProducer.main(args.split(" ")); + PerformanceProducer producer = new PerformanceProducer(); + producer.run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -131,7 +132,8 @@ public void testMsgKey() throws Exception { String newArgs = String.format(newArgString, topic2, pulsar.getBrokerServiceUrl()); Thread thread2 = new Thread(() -> { try { - PerformanceProducer.main(newArgs.split(" ")); + PerformanceProducer producer = new PerformanceProducer(); + producer.run(newArgs.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -169,23 +171,23 @@ public void testMsgKey() throws Exception { @Test(timeOut = 20000) public void testBatchingDisabled() throws Exception { - PerformanceProducer.Arguments arguments = new PerformanceProducer.Arguments(); + PerformanceProducer producer = new PerformanceProducer(); int producerId = 0; String topic = testTopic + UUID.randomUUID(); - arguments.topics = List.of(topic); - arguments.msgRate = 10; - arguments.serviceURL = pulsar.getBrokerServiceUrl(); - arguments.numMessages = 500; - arguments.disableBatching = true; - - ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTransaction(arguments.isEnableTransaction); + producer.topics = List.of(topic); + producer.msgRate = 10; + producer.serviceURL = pulsar.getBrokerServiceUrl(); + producer.numMessages = 500; + producer.disableBatching = true; + + ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(producer) + .enableTransaction(producer.isEnableTransaction); @Cleanup PulsarClient client = clientBuilder.build(); - - ProducerBuilderImpl builder = (ProducerBuilderImpl) PerformanceProducer.createProducerBuilder(client, arguments, producerId); + ProducerBuilderImpl builder = (ProducerBuilderImpl) producer.createProducerBuilder(client, + producerId); Assert.assertFalse(builder.getConf().isBatchingEnabled()); } @@ -196,7 +198,8 @@ public void testCreatePartitions() throws Exception { String args = String.format(argString, topic, pulsar.getBrokerServiceUrl(), pulsar.getWebServiceAddress()); Thread thread = new Thread(() -> { try { - PerformanceProducer.main(args.split(" ")); + PerformanceProducer producer = new PerformanceProducer(); + producer.run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -227,7 +230,8 @@ public void testMaxOutstanding() throws Exception { .subscriptionType(SubscriptionType.Key_Shared).subscribe(); new Thread(() -> { try { - PerformanceProducer.main(args.split(" ")); + PerformanceProducer producer = new PerformanceProducer(); + producer.run(args.split(" ")); } catch (Exception e) { log.error("Failed to start perf producer"); } diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java index 12f457587f685..c8d71d98e701b 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java @@ -133,7 +133,7 @@ public void testTxnPerf() throws Exception { Thread thread = new Thread(() -> { try { - PerformanceTransaction.main(args.split(" ")); + new PerformanceTransaction().run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -184,7 +184,7 @@ public void testProduceTxnMessage() throws InterruptedException, PulsarClientExc .subscribe(); Thread thread = new Thread(() -> { try { - PerformanceProducer.main(args.split(" ")); + new PerformanceProducer().run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -237,7 +237,7 @@ public void testConsumeTxnMessage() throws Exception { Thread thread = new Thread(() -> { try { log.info(""); - PerformanceConsumer.main(args.split(" ")); + new PerformanceConsumer().run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); }