id: reference-cli-tools title: Pulsar command-line tools sidebar_label: Pulsar CLI tools

Pulsar offers several command-line tools that you can use for managing Pulsar installations, performance testing, using command-line producers and consumers, and more.

All Pulsar command-line tools can be run from the bin directory of your installed Pulsar package. The following tools are currently documented:

Getting help

You can get help for any CLI tool, command, or subcommand using the --help flag, or -h for short. Here's an example:

$ bin/pulsar broker --help

pulsar

The pulsar tool is used to start Pulsar components, such as bookies and ZooKeeper, in the foreground.

These processes can also be started in the background, using nohup, using the pulsar-daemon tool, which has the same command interface as pulsar.

Usage:

$ pulsar command

Commands:

  • bookie
  • broker
  • compact-topic
  • discovery
  • configuration-store
  • initialize-cluster-metadata
  • proxy
  • standalone
  • websocket
  • zookeeper
  • zookeeper-shell

Example:

$ PULSAR_BROKER_CONF=/path/to/broker.conf pulsar broker

The table below lists the environment variables that you can use to configure the pulsar tool.

VariableDescriptionDefault
PULSAR_LOG_CONFLog4j configuration fileconf/log4j2.yaml
PULSAR_BROKER_CONFConfiguration file for brokerconf/broker.conf
PULSAR_BOOKKEEPER_CONFdescription: Configuration file for bookieconf/bookkeeper.conf
PULSAR_ZK_CONFConfiguration file for zookeeperconf/zookeeper.conf
PULSAR_CONFIGURATION_STORE_CONFConfiguration file for the configuration storeconf/global_zookeeper.conf
PULSAR_DISCOVERY_CONFConfiguration file for discovery serviceconf/discovery.conf
PULSAR_WEBSOCKET_CONFConfiguration file for websocket proxyconf/websocket.conf
PULSAR_STANDALONE_CONFConfiguration file for standaloneconf/standalone.conf
PULSAR_EXTRA_OPTSExtra options to be passed to the jvm
PULSAR_EXTRA_CLASSPATHExtra paths for Pulsar's classpath
PULSAR_PID_DIRFolder where the pulsar server PID file should be stored
PULSAR_STOP_TIMEOUTWait time before forcefully killing the Bookie server instance if attempts to stop it are not successful

bookie

Starts up a bookie server

Usage:

$ pulsar bookie options

Options

OptionDescriptionDefault
-readOnlyForce start a read-only bookie serverfalse
-withAutoRecoveryStart auto-recover service bookie serverfalse

Example

$ PULSAR_BOOKKEEPER_CONF=/path/to/bookkeeper.conf pulsar bookie \
  -readOnly \
  -withAutoRecovery

broker

Starts up a Pulsar broker

Usage

$ pulsar broker options

Options |Option|Description|Default| |---|---|---| |-bc , --bookie-conf|Configuration file for BookKeeper|| |-rb , --run-bookie|Run a BookKeeper bookie on the same host as the Pulsar broker|false| |-ra , --run-bookie-autorecovery|Run a BookKeeper autorecovery daemon on the same host as the Pulsar broker|false|

Example

$ PULSAR_BROKER_CONF=/path/to/broker.conf pulsar broker

compact-topic

Run compaction against a Pulsar topic (in a new process)

Usage

$ pulsar compact-topic options

Options |Flag|Description|Default| |---|---|---| |-t , --topic|The Pulsar topic that you would like to compact||

Example

$ pulsar compact-topic --topic topic-to-compact

discovery

Run a discovery server

Usage

$ pulsar discovery

Example

$ PULSAR_DISCOVERY_CONF=/path/to/discovery.conf pulsar discovery

configuration-store

Starts up the Pulsar configuration store

Usage

$ pulsar configuration-store

Example

$ PULSAR_CONFIGURATION_STORE_CONF=/path/to/configuration_store.conf pulsar configuration-store

initialize-cluster-metadata

One-time cluster metadata initialization

Usage

$ pulsar initialize-cluster-metadata options

Options |Flag|Description|Default| |---|---|---| |-ub , --broker-service-url|The broker service URL for the new cluster|| |-tb , --broker-service-url-tls|The broker service URL for the new cluster with TLS encryption|| |-c , --cluster|Cluster name|| |-cs , --configuration-store|The configuration store quorum connection string|| |--existing-bk-metadata-service-uri|The metadata service URI of the existing BookKeeper cluster that you want to use|| |-h , --help|Cluster name|false| |--initial-num-stream-storage-containers|The number of storage containers of BookKeeper stream storage|16| |--initial-num-transaction-coordinators|The number of transaction coordinators assigned in a cluster|16| |-uw , --web-service-url|The web service URL for the new cluster|| |-tw , --web-service-url-tls|The web service URL for the new cluster with TLS encryption|| |-zk , --zookeeper|The local ZooKeeper quorum connection string|| |--zookeeper-session-timeout-ms|The local ZooKeeper session timeout. The time unit is in millisecond(ms)|30000|

proxy

Manages the Pulsar proxy

Usage

$ pulsar proxy options

Options |Flag|Description|Default| |---|---|---| |--configuration-store|Configuration store connection string|| |-zk , --zookeeper-servers|Local ZooKeeper connection string||

Example

$ PULSAR_PROXY_CONF=/path/to/proxy.conf pulsar proxy \
  --zookeeper-servers zk-0,zk-1,zk2 \
  --configuration-store zk-0,zk-1,zk-2

standalone

Run a broker service with local bookies and local ZooKeeper

Usage

$ pulsar standalone options

Options |Flag|Description|Default| |---|---|---| |-a , --advertised-address|The standalone broker advertised address|| |--bookkeeper-dir|Local bookies’ base data directory|data/standalone/bookeeper| |--bookkeeper-port|Local bookies’ base port|3181| |--no-broker|Only start ZooKeeper and BookKeeper services, not the broker|false| |--num-bookies|The number of local bookies|1| |--only-broker|Only start the Pulsar broker service (not ZooKeeper or BookKeeper)|| |--wipe-data|Clean up previous ZooKeeper/BookKeeper data|| |--zookeeper-dir|Local ZooKeeper’s data directory|data/standalone/zookeeper| |--zookeeper-port |Local ZooKeeper’s port|2181|

Example

$ PULSAR_STANDALONE_CONF=/path/to/standalone.conf pulsar standalone

websocket

Usage

$ pulsar websocket

Example

$ PULSAR_WEBSOCKET_CONF=/path/to/websocket.conf pulsar websocket

zookeeper

Starts up a ZooKeeper cluster

Usage

$ pulsar zookeeper

Example

$ PULSAR_ZK_CONF=/path/to/zookeeper.conf pulsar zookeeper

zookeeper-shell

Connects to a running ZooKeeper cluster using the ZooKeeper shell

Usage

$ pulsar zookeeper-shell options

Options |Flag|Description|Default| |---|---|---| |-c, --conf|Configuration file for ZooKeeper||

pulsar-client

The pulsar-client tool

Usage

$ pulsar-client command

Commands

  • produce
  • consume

Options |Flag|Description|Default| |---|---|---| |--auth-params|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"}”|{“saslJaasClientSectionName”:“PulsarClient”, “serverType”:“broker”}| |--auth-plugin|Authentication plugin class name|org.apache.pulsar.client.impl.auth.AuthenticationSasl| |--listener-name|Listener name for the broker|| |--url|Broker URL to which to connect|pulsar://localhost:6650/ ws://localhost:8080 |

produce

Send a message or messages to a specific broker and topic

Usage

$ pulsar-client produce topic options

Options |Flag|Description|Default| |---|---|---| |-f, --files|Comma-separated file paths to send; either -m or -f must be specified|[]| |-m, --messages|Comma-separated string of messages to send; either -m or -f must be specified|[]| |-n, --num-produce|The number of times to send the message(s); the count of messages/files * num-produce should be below 1000|1| |-r, --rate|Rate (in messages per second) at which to produce; a value 0 means to produce messages as fast as possible|0.0|

consume

Consume messages from a specific broker and topic

Usage

$ pulsar-client consume topic options

Options |Flag|Description|Default| |---|---|---| |--hex|Display binary messages in hexadecimal format.|false| |-n, --num-messages|Number of messages to consume, 0 means to consume forever.|1| |-r, --rate|Rate (in messages per second) at which to consume; a value 0 means to consume messages as fast as possible|0.0| |--regex|Indicate the topic name is a regex pattern|false| |-s, --subscription-name|Subscription name|| |-t, --subscription-type|The type of the subscription. Possible values: Exclusive, Shared, Failover, Key_Shared.|Exclusive| |-p, --subscription-position|The position of the subscription. Possible values: Latest, Earliest.|Latest|

pulsar-daemon

A wrapper around the pulsar tool that’s used to start and stop processes, such as ZooKeeper, bookies, and Pulsar brokers, in the background using nohup.

pulsar-daemon has a similar interface to the pulsar command but adds start and stop commands for various services. For a listing of those services, run pulsar-daemon to see the help output or see the documentation for the pulsar command.

Usage

$ pulsar-daemon command

Commands

  • start
  • stop

start

Start a service in the background using nohup.

Usage

$ pulsar-daemon start service

stop

Stop a service that’s already been started using start.

Usage

$ pulsar-daemon stop service options

Options |Flag|Description|Default| |---|---|---| |-force|Stop the service forcefully if not stopped by normal shutdown.|false|

pulsar-perf

A tool for performance testing a Pulsar broker.

Usage

$ pulsar-perf command

Commands

  • consume
  • produce
  • read
  • websocket-producer
  • managed-ledger
  • monitor-brokers
  • simulation-client
  • simulation-controller
  • help

Environment variables

The table below lists the environment variables that you can use to configure the pulsar-perf tool.

VariableDescriptionDefault
PULSAR_LOG_CONFLog4j configuration fileconf/log4j2.yaml
PULSAR_CLIENT_CONFConfiguration file for the clientconf/client.conf
PULSAR_EXTRA_OPTSExtra options to be passed to the JVM
PULSAR_EXTRA_CLASSPATHExtra paths for Pulsar's classpath

consume

Run a consumer

Usage

$ pulsar-perf consume options

Options

FlagDescriptionDefault
--auth_paramsAuthentication 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”}.
--auth_pluginAuthentication plugin class name
--listener-nameListener name for the broker
--acks-delay-millisAcknowledgements grouping delay in millis100
-k, --encryption-key-nameThe private key name to decrypt payload
-v, --encryption-key-value-fileThe file which contains the private key to decrypt payload
-h, --helpHelp messagefalse
--conf-fileConfiguration file
-c, --max-connectionsMax number of TCP connections to a single broker100
-n, --num-consumersNumber of consumers (per topic)1
-t, --num-topicsThe number of topics1
-r, --rateSimulate a slow message consumer (rate in msg/s)0
-q, --receiver-queue-sizeSize of the receiver queue1000
-u, --service-urlPulsar service URL
-i, --stats-interval-secondsStatistics interval seconds. If 0, statistics will be disabled0
-ss, --subscriptionsA list of subscriptions to consume on (e.g. sub1,sub2)sub
-st, --subscription-typeSubscriber type. Possible values are Exclusive, Shared, Failover, Key_Shared.Exclusive
-sp, --subscription-positionSubscriber position. Possible values are Latest, Earliest.Latest
--trust-cert-filePath for the trusted TLS certificate file
--tls-allow-insecureAllow insecure TLS connection

produce

Run a producer

Usage

$ pulsar-perf produce options

Options

FlagDescriptionDefault
--auth_paramsAuthentication 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”}.
--auth_pluginAuthentication plugin class name
--listener-nameListener name for the broker
-b, --batch-time-windowBatch messages in a window of the specified number of milliseconds1
-z, --compressionCompress messages’ payload. Possible values are NONE, LZ4, ZLIB, ZSTD or SNAPPY.
--conf-fileConfiguration file
-k, --encryption-key-nameThe public key name to encrypt payload
-v, --encryption-key-value-fileThe file which contains the public key to encrypt payload
-h, --helpHelp messagefalse
-c, --max-connectionsMax number of TCP connections to a single broker100
-o, --max-outstandingMax number of outstanding messages1000
-p, --max-outstanding-across-partitionsMax number of outstanding messages across partitions50000
-m, --num-messagesNumber of messages to publish in total. If set to 0, it will keep publishing.0
-n, --num-producersThe number of producers (per topic)1
-t, --num-topicThe number of topics1
-f, --payload-fileUse payload from an UTF-8 encoded text file and a payload will be randomly selected when publishing messages
-e, --payload-delimiterThe delimiter used to split lines when using payload from a file\n
-r, --ratePublish rate msg/s across topics100
-u, --service-urlPulsar service URL
-s, --sizeMessage size (in bytes)1024
-i, --stats-interval-secondsStatistics interval seconds. If 0, statistics will be disabled.0
-time, --test-durationTest duration in secs. If set to 0, it will keep publishing.0
--trust-cert-filePath for the trusted TLS certificate file
--warmup-timeWarm-up time in seconds1
--tls-allow-insecureAllow insecure TLS connection

read

Run a topic reader

Usage

$ pulsar-perf read options

Options |Flag|Description|Default| |---|---|---| |--auth_params|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”}.|| |--auth_plugin|Authentication plugin class name|| |--listener-name|Listener name for the broker|| |--conf-file|Configuration file|| |-h, --help|Help message|false| |-c, --max-connections|Max number of TCP connections to a single broker|100| |-t, --num-topics|The number of topics|1| |-r, --rate|Simulate a slow message reader (rate in msg/s)|0| |-q, --receiver-queue-size|Size of the receiver queue|1000| |-u, --service-url|Pulsar service URL|| |-m, --start-message-id|Start message id. This can be either ‘earliest’, ‘latest’ or a specific message id by using ‘lid:eid’|earliest| |-i, --stats-interval-seconds|Statistics interval seconds. If 0, statistics will be disabled.|0| |--trust-cert-file|Path for the trusted TLS certificate file|| |--use-tls|Use TLS encryption on the connection|false| |--tls-allow-insecure|Allow insecure TLS connection||

websocket-producer

Run a websocket producer

Usage

$ pulsar-perf websocket-producer options

Options |Flag|Description|Default| |---|---|---| |--auth_params|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”}.|| |--auth_plugin|Authentication plugin class name|| |--conf-file|Configuration file|| |-h, --help|Help message|false| |-m, --num-messages|Number of messages to publish in total. If 0, it will keep publishing|0| |-t, --num-topic|The number of topics|1| |-f, --payload-file|Use payload from a file instead of empty buffer|| |-u, --proxy-url|Pulsar Proxy URL, e.g., “ws://localhost:8080/”|| |-r, --rate|Publish rate msg/s across topics|100| |-s, --size|Message size in byte|1024| |-time, --test-duration|Test duration in secs. If 0, it will keep publishing|0|

managed-ledger

Write directly on managed-ledgers

Usage

$ pulsar-perf managed-ledger options

Options |Flag|Description|Default| |---|---|---| |-a, --ack-quorum|Ledger ack quorum|1| |-dt, --digest-type|BookKeeper digest type. Possible Values: [CRC32, MAC, CRC32C, DUMMY]|CRC32C| |-e, --ensemble-size|Ledger ensemble size|1| |-h, --help|Help message|false| |-c, --max-connections|Max number of TCP connections to a single bookie|1| |-o, --max-outstanding|Max number of outstanding requests|1000| |-m, --num-messages|Number of messages to publish in total. If 0, it will keep publishing|0| |-t, --num-topic|Number of managed ledgers|1| |-r, --rate|Write rate msg/s across managed ledgers|100| |-s, --size|Message size in byte|1024| |-time, --test-duration|Test duration in secs. If 0, it will keep publishing|0| |--threads|Number of threads writing|1| |-w, --write-quorum|Ledger write quorum|1| |-zk, --zookeeperServers|ZooKeeper connection string||

monitor-brokers

Continuously receive broker data and/or load reports

Usage

$ pulsar-perf monitor-brokers options

Options |Flag|Description|Default| |---|---|---| |--connect-string|A connection string for one or more ZooKeeper servers|| |-h, --help|Help message|false|

simulation-client

Run a simulation server acting as a Pulsar client. Uses the client configuration specified in conf/client.conf.

Usage

$ pulsar-perf simulation-client options

Options |Flag|Description|Default| |---|---|---| |--port|Port to listen on for controller|0| |--service-url|Pulsar Service URL|| |-h, --help|Help message|false|

simulation-controller

Run a simulation controller to give commands to servers

Usage

$ pulsar-perf simulation-controller options

Options |Flag|Description|Default| |---|---|---| |--client-port|The port that the clients are listening on|0| |--clients|Comma-separated list of client hostnames|| |--cluster|The cluster to test on|| |-h, --help|Help message|false|

help

This help message

Usage

$ pulsar-perf help

bookkeeper

A tool for managing BookKeeper.

Usage

$ bookkeeper command

Commands

  • auto-recovery
  • bookie
  • localbookie
  • upgrade
  • shell

Environment variables

The table below lists the environment variables that you can use to configure the bookkeeper tool.

VariableDescriptionDefault
BOOKIE_LOG_CONFLog4j configuration fileconf/log4j2.yaml
BOOKIE_CONFBookKeeper configuration fileconf/bk_server.conf
BOOKIE_EXTRA_OPTSExtra options to be passed to the JVM
BOOKIE_EXTRA_CLASSPATHExtra paths for BookKeeper's classpath
ENTRY_FORMATTER_CLASSThe Java class used to format entries
BOOKIE_PID_DIRFolder where the BookKeeper server PID file should be stored
BOOKIE_STOP_TIMEOUTWait time before forcefully killing the Bookie server instance if attempts to stop it are not successful

auto-recovery

Runs an auto-recovery service daemon

Usage

$ bookkeeper auto-recovery options

Options |Flag|Description|Default| |---|---|---| |-c, --conf|Configuration for the auto-recovery daemon||

bookie

Starts up a BookKeeper server (aka bookie)

Usage

$ bookkeeper bookie options

Options |Flag|Description|Default| |---|---|---| |-c, --conf|Configuration for the auto-recovery daemon|| |-readOnly|Force start a read-only bookie server|false| |-withAutoRecovery|Start auto-recovery service bookie server|false|

localbookie

Runs a test ensemble of N bookies locally

Usage

$ bookkeeper localbookie N

upgrade

Upgrade the bookie’s filesystem

Usage

$ bookkeeper upgrade options

Options |Flag|Description|Default| |---|---|---| |-c, --conf|Configuration for the auto-recovery daemon|| |-u, --upgrade|Upgrade the bookie’s directories||

shell

Run shell for admin commands. To see a full listing of those commands, run bookkeeper shell without an argument.

Usage

$ bookkeeper shell

Example

$ bookkeeper shell bookiesanity

broker-tool

The broker- tool is used for operations on a specific broker.

Usage

$ broker-tool command

Commands

  • load-report
  • help

Example Two ways to get more information about a command as below:

$ broker-tool help command
$ broker-tool command --help

load-report

Collect the load report of a specific broker. The command is run on a broker, and used for troubleshooting why broker can’t collect right load report.

Options |Flag|Description|Default| |---|---|---| |-i, --interval| Interval to collect load report, in milliseconds || |-h, --help| Display help information ||