diff --git a/site2/website-next/versioned_docs/version-2.3.1/develop-binary-protocol.md b/site2/website-next/versioned_docs/version-2.3.1/develop-binary-protocol.md new file mode 100644 index 0000000000000..b233f10530af6 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/develop-binary-protocol.md @@ -0,0 +1,581 @@ +--- +id: develop-binary-protocol +title: Pulsar binary protocol specification +sidebar_label: "Binary protocol" +original_id: develop-binary-protocol +--- + +Pulsar uses a custom binary protocol for communications between producers/consumers and brokers. This protocol is designed to support required features, such as acknowledgements and flow control, while ensuring maximum transport and implementation efficiency. + +Clients and brokers exchange *commands* with each other. Commands are formatted as binary [protocol buffer](https://developers.google.com/protocol-buffers/) (aka *protobuf*) messages. The format of protobuf commands is specified in the [`PulsarApi.proto`](https://github.com/apache/pulsar/blob/master/pulsar-common/src/main/proto/PulsarApi.proto) file and also documented in the [Protobuf interface](#protobuf-interface) section below. + +> ### Connection sharing +> Commands for different producers and consumers can be interleaved and sent through the same connection without restriction. + +All commands associated with Pulsar's protocol are contained in a [`BaseCommand`](#pulsar.proto.BaseCommand) protobuf message that includes a [`Type`](#pulsar.proto.Type) [enum](https://developers.google.com/protocol-buffers/docs/proto#enum) with all possible subcommands as optional fields. `BaseCommand` messages can specify only one subcommand. + +## Framing + +Since protobuf doesn't provide any sort of message frame, all messages in the Pulsar protocol are prepended with a 4-byte field that specifies the size of the frame. The maximum allowable size of a single frame is 5 MB. + +The Pulsar protocol allows for two types of commands: + +1. **Simple commands** that do not carry a message payload. +2. **Payload commands** that bear a payload that is used when publishing or delivering messages. In payload commands, the protobuf command data is followed by protobuf [metadata](#message-metadata) and then the payload, which is passed in raw format outside of protobuf. All sizes are passed as 4-byte unsigned big endian integers. + +> Message payloads are passed in raw format rather than protobuf format for efficiency reasons. + +### Simple commands + +Simple (payload-free) commands have this basic structure: + +| Component | Description | Size (in bytes) | +|:------------|:----------------------------------------------------------------------------------------|:----------------| +| totalSize | The size of the frame, counting everything that comes after it (in bytes) | 4 | +| commandSize | The size of the protobuf-serialized command | 4 | +| message | The protobuf message serialized in a raw binary format (rather than in protobuf format) | | + +### Payload commands + +Payload commands have this basic structure: + +| Component | Description | Size (in bytes) | +|:-------------|:--------------------------------------------------------------------------------------------|:----------------| +| totalSize | The size of the frame, counting everything that comes after it (in bytes) | 4 | +| commandSize | The size of the protobuf-serialized command | 4 | +| message | The protobuf message serialized in a raw binary format (rather than in protobuf format) | | +| magicNumber | A 2-byte byte array (`0x0e01`) identifying the current format | 2 | +| checksum | A [CRC32-C checksum](http://www.evanjones.ca/crc32c.html) of everything that comes after it | 4 | +| metadataSize | The size of the message [metadata](#message-metadata) | 4 | +| metadata | The message [metadata](#message-metadata) stored as a binary protobuf message | | +| payload | Anything left in the frame is considered the payload and can include any sequence of bytes | | + +## Message metadata + +Message metadata is stored alongside the application-specified payload as a serialized protobuf message. Metadata is created by the producer and passed on unchanged to the consumer. + +| Field | Description | +|:-------------------------------------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `producer_name` | The name of the producer that published the message | +| `sequence_id` | The sequence ID of the message, assigned by producer | +| `publish_time` | The publish timestamp in Unix time (i.e. as the number of milliseconds since January 1st, 1970 in UTC) | +| `properties` | A sequence of key/value pairs (using the [`KeyValue`](https://github.com/apache/pulsar/blob/master/pulsar-common/src/main/proto/PulsarApi.proto#L32) message). These are application-defined keys and values with no special meaning to Pulsar. | +| `replicated_from` *(optional)* | Indicates that the message has been replicated and specifies the name of the [cluster](reference-terminology.md#cluster) where the message was originally published | +| `partition_key` *(optional)* | While publishing on a partition topic, if the key is present, the hash of the key is used to determine which partition to choose | +| `compression` *(optional)* | Signals that payload has been compressed and with which compression library | +| `uncompressed_size` *(optional)* | If compression is used, the producer must fill the uncompressed size field with the original payload size | +| `num_messages_in_batch` *(optional)* | If this message is really a [batch](#batch-messages) of multiple entries, this field must be set to the number of messages in the batch | + +### Batch messages + +When using batch messages, the payload will be containing a list of entries, +each of them with its individual metadata, defined by the `SingleMessageMetadata` +object. + + +For a single batch, the payload format will look like this: + + +| Field | Description | +|:--------------|:------------------------------------------------------------| +| metadataSizeN | The size of the single message metadata serialized Protobuf | +| metadataN | Single message metadata | +| payloadN | Message payload passed by application | + +Each metadata field looks like this; + +| Field | Description | +|:---------------------------|:--------------------------------------------------------| +| properties | Application-defined properties | +| partition key *(optional)* | Key to indicate the hashing to a particular partition | +| payload_size | Size of the payload for the single message in the batch | + +When compression is enabled, the whole batch will be compressed at once. + +## Interactions + +### Connection establishment + +After opening a TCP connection to a broker, typically on port 6650, the client +is responsible to initiate the session. + +![Connect interaction](/assets/binary-protocol-connect.png) + +After receiving a `Connected` response from the broker, the client can +consider the connection ready to use. Alternatively, if the broker doesn't +validate the client authentication, it will reply with an `Error` command and +close the TCP connection. + +Example: + +```protobuf + +message CommandConnect { + "client_version" : "Pulsar-Client-Java-v1.15.2", + "auth_method_name" : "my-authentication-plugin", + "auth_data" : "my-auth-data", + "protocol_version" : 6 +} + +``` + +Fields: + * `client_version` → String based identifier. Format is not enforced + * `auth_method_name` → *(optional)* Name of the authentication plugin if auth + enabled + * `auth_data` → *(optional)* Plugin specific authentication data + * `protocol_version` → Indicates the protocol version supported by the + client. Broker will not send commands introduced in newer revisions of the + protocol. Broker might be enforcing a minimum version + +```protobuf + +message CommandConnected { + "server_version" : "Pulsar-Broker-v1.15.2", + "protocol_version" : 6 +} + +``` + +Fields: + * `server_version` → String identifier of broker version + * `protocol_version` → Protocol version supported by the broker. Client + must not attempt to send commands introduced in newer revisions of the + protocol + +### Keep Alive + +To identify prolonged network partitions between clients and brokers or cases +in which a machine crashes without interrupting the TCP connection on the remote +end (eg: power outage, kernel panic, hard reboot...), we have introduced a +mechanism to probe for the availability status of the remote peer. + +Both clients and brokers are sending `Ping` commands periodically and they will +close the socket if a `Pong` response is not received within a timeout (default +used by broker is 60s). + +A valid implementation of a Pulsar client is not required to send the `Ping` +probe, though it is required to promptly reply after receiving one from the +broker in order to prevent the remote side from forcibly closing the TCP connection. + + +### Producer + +In order to send messages, a client needs to establish a producer. When creating +a producer, the broker will first verify that this particular client is +authorized to publish on the topic. + +Once the client gets confirmation of the producer creation, it can publish +messages to the broker, referring to the producer id negotiated before. + +![Producer interaction](/assets/binary-protocol-producer.png) + +##### Command Producer + +```protobuf + +message CommandProducer { + "topic" : "persistent://my-property/my-cluster/my-namespace/my-topic", + "producer_id" : 1, + "request_id" : 1 +} + +``` + +Parameters: + * `topic` → Complete topic name to where you want to create the producer on + * `producer_id` → Client generated producer identifier. Needs to be unique + within the same connection + * `request_id` → Identifier for this request. Used to match the response with + the originating request. Needs to be unique within the same connection + * `producer_name` → *(optional)* If a producer name is specified, the name will + be used, otherwise the broker will generate a unique name. Generated + producer name is guaranteed to be globally unique. Implementations are + expected to let the broker generate a new producer name when the producer + is initially created, then reuse it when recreating the producer after + reconnections. + +The broker will reply with either `ProducerSuccess` or `Error` commands. + +##### Command ProducerSuccess + +```protobuf + +message CommandProducerSuccess { + "request_id" : 1, + "producer_name" : "generated-unique-producer-name" +} + +``` + +Parameters: + * `request_id` → Original id of the `CreateProducer` request + * `producer_name` → Generated globally unique producer name or the name + specified by the client, if any. + +##### Command Send + +Command `Send` is used to publish a new message within the context of an +already existing producer. This command is used in a frame that includes command +as well as message payload, for which the complete format is specified in the [payload commands](#payload-commands) section. + +```protobuf + +message CommandSend { + "producer_id" : 1, + "sequence_id" : 0, + "num_messages" : 1 +} + +``` + +Parameters: + * `producer_id` → id of an existing producer + * `sequence_id` → each message has an associated sequence id which is expected + to be implemented with a counter starting at 0. The `SendReceipt` that + acknowledges the effective publishing of a messages will refer to it by + its sequence id. + * `num_messages` → *(optional)* Used when publishing a batch of messages at + once. + +##### Command SendReceipt + +After a message has been persisted on the configured number of replicas, the +broker will send the acknowledgment receipt to the producer. + +```protobuf + +message CommandSendReceipt { + "producer_id" : 1, + "sequence_id" : 0, + "message_id" : { + "ledgerId" : 123, + "entryId" : 456 + } +} + +``` + +Parameters: + * `producer_id` → id of producer originating the send request + * `sequence_id` → sequence id of the published message + * `message_id` → message id assigned by the system to the published message + Unique within a single cluster. Message id is composed of 2 longs, `ledgerId` + and `entryId`, that reflect that this unique id is assigned when appending + to a BookKeeper ledger + + +##### Command CloseProducer + +**Note**: *This command can be sent by either producer or broker*. + +When receiving a `CloseProducer` command, the broker will stop accepting any +more messages for the producer, wait until all pending messages are persisted +and then reply `Success` to the client. + +The broker can send a `CloseProducer` command to client when it's performing +a graceful failover (eg: broker is being restarted, or the topic is being unloaded +by load balancer to be transferred to a different broker). + +When receiving the `CloseProducer`, the client is expected to go through the +service discovery lookup again and recreate the producer again. The TCP +connection is not affected. + +### Consumer + +A consumer is used to attach to a subscription and consume messages from it. +After every reconnection, a client needs to subscribe to the topic. If a +subscription is not already there, a new one will be created. + +![Consumer](/assets/binary-protocol-consumer.png) + +#### Flow control + +After the consumer is ready, the client needs to *give permission* to the +broker to push messages. This is done with the `Flow` command. + +A `Flow` command gives additional *permits* to send messages to the consumer. +A typical consumer implementation will use a queue to accumulate these messages +before the application is ready to consume them. + +After the application has dequeued half of the messages in the queue, the consumer +sends permits to the broker to ask for more messages (equals to half of the messages in the queue). + +For example, if the queue size is 1000 and the consumer consumes 500 messages in the queue. +Then the consumer sends permits to the broker to ask for 500 messages. + +##### Command Subscribe + +```protobuf + +message CommandSubscribe { + "topic" : "persistent://my-property/my-cluster/my-namespace/my-topic", + "subscription" : "my-subscription-name", + "subType" : "Exclusive", + "consumer_id" : 1, + "request_id" : 1 +} + +``` + +Parameters: + * `topic` → Complete topic name to where you want to create the consumer on + * `subscription` → Subscription name + * `subType` → Subscription type: Exclusive, Shared, Failover, Key_Shared + * `consumer_id` → Client generated consumer identifier. Needs to be unique + within the same connection + * `request_id` → Identifier for this request. Used to match the response with + the originating request. Needs to be unique within the same connection + * `consumer_name` → *(optional)* Clients can specify a consumer name. This + name can be used to track a particular consumer in the stats. Also, in + Failover subscription type, the name is used to decide which consumer is + elected as *master* (the one receiving messages): consumers are sorted by + their consumer name and the first one is elected master. + +##### Command Flow + +```protobuf + +message CommandFlow { + "consumer_id" : 1, + "messagePermits" : 1000 +} + +``` + +Parameters: +* `consumer_id` → Id of an already established consumer +* `messagePermits` → Number of additional permits to grant to the broker for + pushing more messages + +##### Command Message + +Command `Message` is used by the broker to push messages to an existing consumer, +within the limits of the given permits. + + +This command is used in a frame that includes the message payload as well, for +which the complete format is specified in the [payload commands](#payload-commands) +section. + +```protobuf + +message CommandMessage { + "consumer_id" : 1, + "message_id" : { + "ledgerId" : 123, + "entryId" : 456 + } +} + +``` + +##### Command Ack + +An `Ack` is used to signal to the broker that a given message has been +successfully processed by the application and can be discarded by the broker. + +In addition, the broker will also maintain the consumer position based on the +acknowledged messages. + +```protobuf + +message CommandAck { + "consumer_id" : 1, + "ack_type" : "Individual", + "message_id" : { + "ledgerId" : 123, + "entryId" : 456 + } +} + +``` + +Parameters: + * `consumer_id` → Id of an already established consumer + * `ack_type` → Type of acknowledgment: `Individual` or `Cumulative` + * `message_id` → Id of the message to acknowledge + * `validation_error` → *(optional)* Indicates that the consumer has discarded + the messages due to: `UncompressedSizeCorruption`, + `DecompressionError`, `ChecksumMismatch`, `BatchDeSerializeError` + +##### Command CloseConsumer + +***Note***: **This command can be sent by either producer or broker*. + +This command behaves the same as [`CloseProducer`](#command-closeproducer) + +##### Command RedeliverUnacknowledgedMessages + +A consumer can ask the broker to redeliver some or all of the pending messages +that were pushed to that particular consumer and not yet acknowledged. + +The protobuf object accepts a list of message ids that the consumer wants to +be redelivered. If the list is empty, the broker will redeliver all the +pending messages. + +On redelivery, messages can be sent to the same consumer or, in the case of a +shared subscription, spread across all available consumers. + + +##### Command ReachedEndOfTopic + +This is sent by a broker to a particular consumer, whenever the topic +has been "terminated" and all the messages on the subscription were +acknowledged. + +The client should use this command to notify the application that no more +messages are coming from the consumer. + +##### Command ConsumerStats + +This command is sent by the client to retrieve Subscriber and Consumer level +stats from the broker. +Parameters: + * `request_id` → Id of the request, used to correlate the request + and the response. + * `consumer_id` → Id of an already established consumer. + +##### Command ConsumerStatsResponse + +This is the broker's response to ConsumerStats request by the client. +It contains the Subscriber and Consumer level stats of the `consumer_id` sent in the request. +If the `error_code` or the `error_message` field is set it indicates that the request has failed. + +##### Command Unsubscribe + +This command is sent by the client to unsubscribe the `consumer_id` from the associated topic. +Parameters: + * `request_id` → Id of the request. + * `consumer_id` → Id of an already established consumer which needs to unsubscribe. + + +## Service discovery + +### Topic lookup + +Topic lookup needs to be performed each time a client needs to create or +reconnect a producer or a consumer. Lookup is used to discover which particular +broker is serving the topic we are about to use. + +Lookup can be done with a REST call as described in the [admin API](admin-api-topics.md#lookup-of-topic) +docs. + +Since Pulsar-1.16 it is also possible to perform the lookup within the binary +protocol. + +For the sake of example, let's assume we have a service discovery component +running at `pulsar://broker.example.com:6650` + +Individual brokers will be running at `pulsar://broker-1.example.com:6650`, +`pulsar://broker-2.example.com:6650`, ... + +A client can use a connection to the discovery service host to issue a +`LookupTopic` command. The response can either be a broker hostname to +connect to, or a broker hostname to which retry the lookup. + +The `LookupTopic` command has to be used in a connection that has already +gone through the `Connect` / `Connected` initial handshake. + +![Topic lookup](/assets/binary-protocol-topic-lookup.png) + +```protobuf + +message CommandLookupTopic { + "topic" : "persistent://my-property/my-cluster/my-namespace/my-topic", + "request_id" : 1, + "authoritative" : false +} + +``` + +Fields: + * `topic` → Topic name to lookup + * `request_id` → Id of the request that will be passed with its response + * `authoritative` → Initial lookup request should use false. When following a + redirect response, client should pass the same value contained in the + response + +##### LookupTopicResponse + +Example of response with successful lookup: + +```protobuf + +message CommandLookupTopicResponse { + "request_id" : 1, + "response" : "Connect", + "brokerServiceUrl" : "pulsar://broker-1.example.com:6650", + "brokerServiceUrlTls" : "pulsar+ssl://broker-1.example.com:6651", + "authoritative" : true +} + +``` + +Example of lookup response with redirection: + +```protobuf + +message CommandLookupTopicResponse { + "request_id" : 1, + "response" : "Redirect", + "brokerServiceUrl" : "pulsar://broker-2.example.com:6650", + "brokerServiceUrlTls" : "pulsar+ssl://broker-2.example.com:6651", + "authoritative" : true +} + +``` + +In this second case, we need to reissue the `LookupTopic` command request +to `broker-2.example.com` and this broker will be able to give a definitive +answer to the lookup request. + +### Partitioned topics discovery + +Partitioned topics metadata discovery is used to find out if a topic is a +"partitioned topic" and how many partitions were set up. + +If the topic is marked as "partitioned", the client is expected to create +multiple producers or consumers, one for each partition, using the `partition-X` +suffix. + +This information only needs to be retrieved the first time a producer or +consumer is created. There is no need to do this after reconnections. + +The discovery of partitioned topics metadata works very similar to the topic +lookup. The client send a request to the service discovery address and the +response will contain actual metadata. + +##### Command PartitionedTopicMetadata + +```protobuf + +message CommandPartitionedTopicMetadata { + "topic" : "persistent://my-property/my-cluster/my-namespace/my-topic", + "request_id" : 1 +} + +``` + +Fields: + * `topic` → the topic for which to check the partitions metadata + * `request_id` → Id of the request that will be passed with its response + + +##### Command PartitionedTopicMetadataResponse + +Example of response with metadata: + +```protobuf + +message CommandPartitionedTopicMetadataResponse { + "request_id" : 1, + "response" : "Success", + "partitions" : 32 +} + +``` + +## Protobuf interface + +All Pulsar's Protobuf definitions can be found {@inject: github:here:/pulsar-common/src/main/proto/PulsarApi.proto}. diff --git a/site2/website-next/versioned_docs/version-2.3.1/develop-cpp.md b/site2/website-next/versioned_docs/version-2.3.1/develop-cpp.md new file mode 100644 index 0000000000000..9da7a3a4131a4 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/develop-cpp.md @@ -0,0 +1,114 @@ +--- +id: develop-cpp +title: Building Pulsar C++ client +sidebar_label: "Building Pulsar C++ client" +original_id: develop-cpp +--- + +## Supported platforms + +The Pulsar C++ client has been successfully tested on **MacOS** and **Linux**. + +## System requirements + +You need to have the following installed to use the C++ client: + +* [CMake](https://cmake.org/) +* [Boost](http://www.boost.org/) +* [Protocol Buffers](https://developers.google.com/protocol-buffers/) 2.6 +* [Log4CXX](https://logging.apache.org/log4cxx) +* [libcurl](https://curl.haxx.se/libcurl/) +* [Google Test](https://github.com/google/googletest) +* [JsonCpp](https://github.com/open-source-parsers/jsoncpp) + +## Compilation + +There are separate compilation instructions for [MacOS](#macos) and [Linux](#linux). For both systems, start by cloning the Pulsar repository: + +```shell + +$ git clone https://github.com/apache/pulsar + +``` + +### Linux + +First, install all of the necessary dependencies: + +```shell + +$ apt-get install cmake libssl-dev libcurl4-openssl-dev liblog4cxx-dev \ + libprotobuf-dev protobuf-compiler libboost-all-dev google-mock libgtest-dev libjsoncpp-dev + +``` + +Then compile and install [Google Test](https://github.com/google/googletest): + +```shell + +# libgtest-dev version is 1.18.0 or above +$ cd /usr/src/googletest +$ sudo cmake . +$ sudo make +$ sudo cp ./googlemock/libgmock.a ./googlemock/gtest/libgtest.a /usr/lib/ + +# less than 1.18.0 +$ cd /usr/src/gtest +$ sudo cmake . +$ sudo make +$ sudo cp libgtest.a /usr/lib + +$ cd /usr/src/gmock +$ sudo cmake . +$ sudo make +$ sudo cp libgmock.a /usr/lib + +``` + +Finally, compile the Pulsar client library for C++ inside the Pulsar repo: + +```shell + +$ cd pulsar-client-cpp +$ cmake . +$ make + +``` + +The resulting files, `libpulsar.so` and `libpulsar.a`, will be placed in the `lib` folder of the repo while two tools, `perfProducer` and `perfConsumer`, will be placed in the `perf` directory. + +### MacOS + +First, install all of the necessary dependencies: + +```shell + +# OpenSSL installation +$ brew install openssl +$ export OPENSSL_INCLUDE_DIR=/usr/local/opt/openssl/include/ +$ export OPENSSL_ROOT_DIR=/usr/local/opt/openssl/ + +# Protocol Buffers installation +$ brew tap homebrew/versions +$ brew install protobuf260 +$ brew install boost +$ brew install log4cxx + +# Google Test installation +$ git clone https://github.com/google/googletest.git +$ cd googletest +$ cmake . +$ make install + +``` + +Then compile the Pulsar client library in the repo that you cloned: + +```shell + +$ cd pulsar-client-cpp +$ cmake . +$ make + +``` + diff --git a/site2/website-next/versioned_docs/version-2.3.1/develop-load-manager.md b/site2/website-next/versioned_docs/version-2.3.1/develop-load-manager.md new file mode 100644 index 0000000000000..509209b6a852d --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/develop-load-manager.md @@ -0,0 +1,227 @@ +--- +id: develop-load-manager +title: Modular load manager +sidebar_label: "Modular load manager" +original_id: develop-load-manager +--- + +The *modular load manager*, implemented in [`ModularLoadManagerImpl`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java), is a flexible alternative to the previously implemented load manager, [`SimpleLoadManagerImpl`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java), which attempts to simplify how load is managed while also providing abstractions so that complex load management strategies may be implemented. + +## Usage + +There are two ways that you can enable the modular load manager: + +1. Change the value of the `loadManagerClassName` parameter in `conf/broker.conf` from `org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl` to `org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl`. +2. Using the `pulsar-admin` tool. Here's an example: + + ```shell + + $ pulsar-admin update-dynamic-config \ + --config loadManagerClassName \ + --value org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl + + ``` + + You can use the same method to change back to the original value. In either case, any mistake in specifying the load manager will cause Pulsar to default to `SimpleLoadManagerImpl`. + +## Verification + +There are a few different ways to determine which load manager is being used: + +1. Use `pulsar-admin` to examine the `loadManagerClassName` element: + + ```shell + + $ bin/pulsar-admin brokers get-all-dynamic-config + { + "loadManagerClassName" : "org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl" + } + + ``` + + If there is no `loadManagerClassName` element, then the default load manager is used. + +2. Consult a ZooKeeper load report. With the module load manager, the load report in `/loadbalance/brokers/...` will have many differences. for example the `systemResourceUsage` sub-elements (`bandwidthIn`, `bandwidthOut`, etc.) are now all at the top level. Here is an example load report from the module load manager: + + ```json + + { + "bandwidthIn": { + "limit": 10240000.0, + "usage": 4.256510416666667 + }, + "bandwidthOut": { + "limit": 10240000.0, + "usage": 5.287239583333333 + }, + "bundles": [], + "cpu": { + "limit": 2400.0, + "usage": 5.7353247655435915 + }, + "directMemory": { + "limit": 16384.0, + "usage": 1.0 + } + } + + ``` + + With the simple load manager, the load report in `/loadbalance/brokers/...` will look like this: + + ```json + + { + "systemResourceUsage": { + "bandwidthIn": { + "limit": 10240000.0, + "usage": 0.0 + }, + "bandwidthOut": { + "limit": 10240000.0, + "usage": 0.0 + }, + "cpu": { + "limit": 2400.0, + "usage": 0.0 + }, + "directMemory": { + "limit": 16384.0, + "usage": 1.0 + }, + "memory": { + "limit": 8192.0, + "usage": 3903.0 + } + } + } + + ``` + +3. The command-line [broker monitor](reference-cli-tools.md#monitor-brokers) will have a different output format depending on which load manager implementation is being used. + + Here is an example from the modular load manager: + + ``` + + =================================================================================================================== + ||SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.00 |48.33 |0.01 |0.00 |0.00 |48.33 || + ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || + || |4 |4 |0 |2 |4 |0 || + ||LATEST |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + ||SHORT |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + ||LONG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + =================================================================================================================== + + ``` + + Here is an example from the simple load manager: + + ``` + + =================================================================================================================== + ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || + || |4 |4 |0 |2 |0 |0 || + ||RAW SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.25 |47.94 |0.01 |0.00 |0.00 |47.94 || + ||ALLOC SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.20 |1.89 | |1.27 |3.21 |3.21 || + ||RAW MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.01 |0.01 |0.01 || + ||ALLOC MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |54.84 |134.48 |189.31 |126.54 |320.96 |447.50 || + =================================================================================================================== + + ``` + +It is important to note that the module load manager is _centralized_, meaning that all requests to assign a bundle---whether it's been seen before or whether this is the first time---only get handled by the _lead_ broker (which can change over time). To determine the current lead broker, examine the `/loadbalance/leader` node in ZooKeeper. + +## Implementation + +### Data + +The data monitored by the modular load manager is contained in the [`LoadData`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadData.java) class. +Here, the available data is subdivided into the bundle data and the broker data. + +#### Broker + +The broker data is contained in the [`BrokerData`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/BrokerData.java) class. It is further subdivided into two parts, +one being the local data which every broker individually writes to ZooKeeper, and the other being the historical broker +data which is written to ZooKeeper by the leader broker. + +##### Local Broker Data +The local broker data is contained in the class [`LocalBrokerData`](https://github.com/apache/pulsar/blob/master/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerData.java) and provides information about the following resources: + +* CPU usage +* JVM heap memory usage +* Direct memory usage +* Bandwidth in/out usage +* Most recent total message rate in/out across all bundles +* Total number of topics, bundles, producers, and consumers +* Names of all bundles assigned to this broker +* Most recent changes in bundle assignments for this broker + +The local broker data is updated periodically according to the service configuration +"loadBalancerReportUpdateMaxIntervalMinutes". After any broker updates their local broker data, the leader broker will +receive the update immediately via a ZooKeeper watch, where the local data is read from the ZooKeeper node +`/loadbalance/brokers/` + +##### Historical Broker Data + +The historical broker data is contained in the [`TimeAverageBrokerData`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/TimeAverageBrokerData.java) class. + +In order to reconcile the need to make good decisions in a steady-state scenario and make reactive decisions in a critical scenario, the historical data is split into two parts: the short-term data for reactive decisions, and the long-term data for steady-state decisions. Both time frames maintain the following information: + +* Message rate in/out for the entire broker +* Message throughput in/out for the entire broker + +Unlike the bundle data, the broker data does not maintain samples for the global broker message rates and throughputs, which is not expected to remain steady as new bundles are removed or added. Instead, this data is aggregated over the short-term and long-term data for the bundles. See the section on bundle data to understand how that data is collected and maintained. + +The historical broker data is updated for each broker in memory by the leader broker whenever any broker writes their local data to ZooKeeper. Then, the historical data is written to ZooKeeper by the leader broker periodically according to the configuration `loadBalancerResourceQuotaUpdateIntervalMinutes`. + +##### Bundle Data + +The bundle data is contained in the [`BundleData`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/BundleData.java). Like the historical broker data, the bundle data is split into a short-term and a long-term time frame. The information maintained in each time frame: + +* Message rate in/out for this bundle +* Message Throughput In/Out for this bundle +* Current number of samples for this bundle + +The time frames are implemented by maintaining the average of these values over a set, limited number of samples, where +the samples are obtained through the message rate and throughput values in the local data. Thus, if the update interval +for the local data is 2 minutes, the number of short samples is 10 and the number of long samples is 1000, the +short-term data is maintained over a period of `10 samples * 2 minutes / sample = 20 minutes`, while the long-term +data is similarly over a period of 2000 minutes. Whenever there are not enough samples to satisfy a given time frame, +the average is taken only over the existing samples. When no samples are available, default values are assumed until +they are overwritten by the first sample. Currently, the default values are + +* Message rate in/out: 50 messages per second both ways +* Message throughput in/out: 50KB per second both ways + +The bundle data is updated in memory on the leader broker whenever any broker writes their local data to ZooKeeper. +Then, the bundle data is written to ZooKeeper by the leader broker periodically at the same time as the historical +broker data, according to the configuration `loadBalancerResourceQuotaUpdateIntervalMinutes`. + +### Traffic Distribution + +The modular load manager uses the abstraction provided by [`ModularLoadManagerStrategy`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategy.java) to make decisions about bundle assignment. The strategy makes a decision by considering the service configuration, the entire load data, and the bundle data for the bundle to be assigned. Currently, the only supported strategy is [`LeastLongTermMessageRate`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java), though soon users will have the ability to inject their own strategies if desired. + +#### Least Long Term Message Rate Strategy + +As its name suggests, the least long term message rate strategy attempts to distribute bundles across brokers so that +the message rate in the long-term time window for each broker is roughly the same. However, simply balancing load based +on message rate does not handle the issue of asymmetric resource burden per message on each broker. Thus, the system +resource usages, which are CPU, memory, direct memory, bandwidth in, and bandwidth out, are also considered in the +assignment process. This is done by weighting the final message rate according to +`1 / (overload_threshold - max_usage)`, where `overload_threshold` corresponds to the configuration +`loadBalancerBrokerOverloadedThresholdPercentage` and `max_usage` is the maximum proportion among the system resources +that is being utilized by the candidate broker. This multiplier ensures that machines with are being more heavily taxed +by the same message rates will receive less load. In particular, it tries to ensure that if one machine is overloaded, +then all machines are approximately overloaded. In the case in which a broker's max usage exceeds the overload +threshold, that broker is not considered for bundle assignment. If all brokers are overloaded, the bundle is randomly +assigned. + diff --git a/site2/website-next/versioned_docs/version-2.3.1/develop-schema.md b/site2/website-next/versioned_docs/version-2.3.1/develop-schema.md new file mode 100644 index 0000000000000..e71c04ef60dc1 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/develop-schema.md @@ -0,0 +1,62 @@ +--- +id: develop-schema +title: Custom schema storage +sidebar_label: "Custom schema storage" +original_id: develop-schema +--- + +By default, Pulsar stores data type [schemas](concepts-schema-registry) in [Apache BookKeeper](https://bookkeeper.apache.org) (which is deployed alongside Pulsar). You can, however, use another storage system if you wish. This doc walks you through creating your own schema storage implementation. + +In order to use a non-default (i.e. non-BookKeeper) storage system for Pulsar schemas, you need to implement two Java interfaces: [`SchemaStorage`](#schemastorage-interface) and [`SchemaStorageFactory`](#schemastoragefactory-interface). + +## SchemaStorage interface + +The `SchemaStorage` interface has the following methods: + +```java + +public interface SchemaStorage { + // How schemas are updated + CompletableFuture put(String key, byte[] value, byte[] hash); + + // How schemas are fetched from storage + CompletableFuture get(String key, SchemaVersion version); + + // How schemas are deleted + CompletableFuture delete(String key); + + // Utility method for converting a schema version byte array to a SchemaVersion object + SchemaVersion versionFromBytes(byte[] version); + + // Startup behavior for the schema storage client + void start() throws Exception; + + // Shutdown behavior for the schema storage client + void close() throws Exception; +} + +``` + +> For a full-fledged example schema storage implementation, see the [`BookKeeperSchemaStorage`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java) class. + +## SchemaStorageFactory interface + +```java + +public interface SchemaStorageFactory { + @NotNull + SchemaStorage create(PulsarService pulsar) throws Exception; +} + +``` + +> For a full-fledged example schema storage factory implementation, see the [`BookKeeperSchemaStorageFactory`](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java) class. + +## Deployment + +In order to use your custom schema storage implementation, you'll need to: + +1. Package the implementation in a [JAR](https://docs.oracle.com/javase/tutorial/deployment/jar/basicsindex.html) file. +1. Add that jar to the `lib` folder in your Pulsar [binary or source distribution](getting-started-standalone.md#installing-pulsar). +1. Change the `schemaRegistryStorageClassName` configuration in [`broker.conf`](reference-configuration.md#broker) to your custom factory class (i.e. the `SchemaStorageFactory` implementation, not the `SchemaStorage` implementation). +1. Start up Pulsar. diff --git a/site2/website-next/versioned_docs/version-2.3.1/develop-tools.md b/site2/website-next/versioned_docs/version-2.3.1/develop-tools.md new file mode 100644 index 0000000000000..b5457790b8081 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/develop-tools.md @@ -0,0 +1,111 @@ +--- +id: develop-tools +title: Simulation tools +sidebar_label: "Simulation tools" +original_id: develop-tools +--- + +It is sometimes necessary create an test environment and incur artificial load to observe how well load managers +handle the load. The load simulation controller, the load simulation client, and the broker monitor were created as an +effort to make create this load and observe the effects on the managers more easily. + +## Simulation Client +The simulation client is a machine which will create and subscribe to topics with configurable message rates and sizes. +Because it is sometimes necessary in simulating large load to use multiple client machines, the user does not interact +with the simulation client directly, but instead delegates their requests to the simulation controller, which will then +send signals to clients to start incurring load. The client implementation is in the class +`org.apache.pulsar.testclient.LoadSimulationClient`. + +### Usage +To Start a simulation client, use the `pulsar-perf` script with the command `simulation-client` as follows: + +``` + +pulsar-perf simulation-client --port --service-url + +``` + +The client will then be ready to receive controller commands. +## Simulation Controller +The simulation controller send signals to the simulation clients, requesting them to create new topics, stop old +topics, change the load incurred by topics, as well as several other tasks. It is implemented in the class +`org.apache.pulsar.testclient.LoadSimulationController` and presents a shell to the user as an interface to send +command with. + +### Usage +To start a simulation controller, use the `pulsar-perf` script with the command `simulation-controller` as follows: + +``` + +pulsar-perf simulation-controller --cluster --client-port +--clients + +``` + +The clients should already be started before the controller is started. You will then be presented with a simple prompt, +where you can issue commands to simulation clients. Arguments often refer to tenant names, namespace names, and topic +names. In all cases, the BASE name of the tenants, namespaces, and topics are used. For example, for the topic +`persistent://my_tenant/my_cluster/my_namespace/my_topic`, the tenant name is `my_tenant`, the namespace name is +`my_namespace`, and the topic name is `my_topic`. The controller can perform the following actions: + +* Create a topic with a producer and a consumer + * `trade [--rate ] + [--rand-rate ,] + [--size ]` +* Create a group of topics with a producer and a consumer + * `trade_group [--rate ] + [--rand-rate ,] + [--separation ] [--size ] + [--topics-per-namespace ]` +* Change the configuration of an existing topic + * `change [--rate ] + [--rand-rate ,] + [--size ]` +* Change the configuration of a group of topics + * `change_group [--rate ] [--rand-rate ,] + [--size ] [--topics-per-namespace ]` +* Shutdown a previously created topic + * `stop ` +* Shutdown a previously created group of topics + * `stop_group ` +* Copy the historical data from one ZooKeeper to another and simulate based on the message rates and sizes in that history + * `copy [--rate-multiplier value]` +* Simulate the load of the historical data on the current ZooKeeper (should be same ZooKeeper being simulated on) + * `simulate [--rate-multiplier value]` +* Stream the latest data from the given active ZooKeeper to simulate the real-time load of that ZooKeeper. + * `stream [--rate-multiplier value]` + +The "group" arguments in these commands allow the user to create or affect multiple topics at once. Groups are created +when calling the `trade_group` command, and all topics from these groups may be subsequently modified or stopped +with the `change_group` and `stop_group` commands respectively. All ZooKeeper arguments are of the form +`zookeeper_host:port`. + +### Difference Between Copy, Simulate, and Stream +The commands `copy`, `simulate`, and `stream` are very similar but have significant differences. `copy` is used when +you want to simulate the load of a static, external ZooKeeper on the ZooKeeper you are simulating on. Thus, +`source zookeeper` should be the ZooKeeper you want to copy and `target zookeeper` should be the ZooKeeper you are +simulating on, and then it will get the full benefit of the historical data of the source in both load manager +implementations. `simulate` on the other hand takes in only one ZooKeeper, the one you are simulating on. It assumes +that you are simulating on a ZooKeeper that has historical data for `SimpleLoadManagerImpl` and creates equivalent +historical data for `ModularLoadManagerImpl`. Then, the load according to the historical data is simulated by the +clients. Finally, `stream` takes in an active ZooKeeper different than the ZooKeeper being simulated on and streams +load data from it and simulates the real-time load. In all cases, the optional `rate-multiplier` argument allows the +user to simulate some proportion of the load. For instance, using `--rate-multiplier 0.05` will cause messages to +be sent at only `5%` of the rate of the load that is being simulated. + +## Broker Monitor +To observe the behavior of the load manager in these simulations, one may utilize the broker monitor, which is +implemented in `org.apache.pulsar.testclient.BrokerMonitor`. The broker monitor will print tabular load data to the +console as it is updated using watchers. + +### Usage +To start a broker monitor, use the `monitor-brokers` command in the `pulsar-perf` script: + +``` + +pulsar-perf monitor-brokers --connect-string + +``` + +The console will then continuously print load data until it is interrupted. + diff --git a/site2/website-next/versioned_docs/version-2.3.1/io-cdc-debezium.md b/site2/website-next/versioned_docs/version-2.3.1/io-cdc-debezium.md new file mode 100644 index 0000000000000..01d6bfb2d16c5 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/io-cdc-debezium.md @@ -0,0 +1,208 @@ +--- +id: io-cdc-debezium +title: CDC Debezium Connector +sidebar_label: "CDC Debezium Connector" +original_id: io-cdc-debezium +--- + +### Source Configuration Options + +The Configuration is mostly related to Debezium task config, besides this we should provides the service URL of Pulsar cluster, and topic names that used to store offset and history. + +| Name | Required | Default | Description | +|------|----------|---------|-------------| +| `task.class` | `true` | `null` | A source task class that implemented in Debezium. | +| `database.hostname` | `true` | `null` | The address of the Database server. | +| `database.port` | `true` | `null` | The port number of the Database server.. | +| `database.user` | `true` | `null` | The name of the Database user that has the required privileges. | +| `database.password` | `true` | `null` | The password for the Database user that has the required privileges. | +| `database.server.id` | `true` | `null` | The connector’s identifier that must be unique within the Database cluster and similar to Database’s server-id configuration property. | +| `database.server.name` | `true` | `null` | The logical name of the Database server/cluster, which forms a namespace and is used in all the names of the Kafka topics to which the connector writes, the Kafka Connect schema names, and the namespaces of the corresponding Avro schema when the Avro Connector is used. | +| `database.whitelist` | `false` | `null` | A list of all databases hosted by this server that this connector will monitor. This is optional, and there are other properties for listing the databases and tables to include or exclude from monitoring. | +| `key.converter` | `true` | `null` | The converter provided by Kafka Connect to convert record key. | +| `value.converter` | `true` | `null` | The converter provided by Kafka Connect to convert record value. | +| `database.history` | `true` | `null` | The name of the database history class name. | +| `database.history.pulsar.topic` | `true` | `null` | The name of the database history topic where the connector will write and recover DDL statements. This topic is for internal use only and should not be used by consumers. | +| `database.history.pulsar.service.url` | `true` | `null` | Pulsar cluster service url for history topic. | +| `pulsar.service.url` | `true` | `null` | Pulsar cluster service URL for the offset topic used in Debezium. You can use the `bin/pulsar-admin --admin-url http://pulsar:8080 sources localrun --source-config-file configs/pg-pulsar-config.yaml` command to point to the target Pulsar cluster. | +| `offset.storage.topic` | `true` | `null` | Record the last committed offsets that the connector successfully completed. | + +### Configuration Example + +Here is a configuration Json example: + +```$json + +{ + "tenant": "public", + "namespace": "default", + "name": "debezium-kafka-source", + "className": "org.apache.pulsar.io.kafka.connect.KafkaConnectSource" , + "topicName": "kafka-connect-topic", + "configs": + { + "task.class": "io.debezium.connector.mysql.MySqlConnectorTask", + "database.hostname": "localhost", + "database.port": "3306", + "database.user": "debezium", + "database.password": "dbz", + "database.server.id": "184054", + "database.server.name": "dbserver1", + "database.whitelist": "inventory", + "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", + "database.history.pulsar.topic": "history-topic", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "pulsar.service.url": "pulsar://127.0.0.1:6650", + "offset.storage.topic": "offset-topic" + }, + "archive": "connectors/pulsar-io-kafka-connect-adaptor-2.3.1-SNAPSHOT.nar" +} + +``` + +You could also find the yaml example in this [file](https://github.com/apache/pulsar/blob/master/pulsar-io/kafka-connect-adaptor/src/main/resources/debezium-mysql-source-config.yaml), which has similar content below: + +```$yaml + +tenant: "public" +namespace: "default" +name: "debezium-kafka-source" +topicName: "kafka-connect-topic" +archive: "connectors/pulsar-io-kafka-connect-adaptor-2.3.1-SNAPSHOT.nar" + +##autoAck: true +parallelism: 1 + +configs: + ## sourceTask + task.class: "io.debezium.connector.mysql.MySqlConnectorTask" + + ## config for mysql, docker image: debezium/example-mysql:0.8 + database.hostname: "localhost" + database.port: "3306" + database.user: "debezium" + database.password: "dbz" + database.server.id: "184054" + database.server.name: "dbserver1" + database.whitelist: "inventory" + + database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" + database.history.pulsar.topic: "history-topic" + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG + key.converter: "org.apache.kafka.connect.json.JsonConverter" + value.converter: "org.apache.kafka.connect.json.JsonConverter" + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + ## OFFSET_STORAGE_TOPIC_CONFIG + offset.storage.topic: "offset-topic" + +``` + +### Usage example + +Here is a simple example to store MySQL change data using above example config. + +- Start a MySQL server with an example database, from which Debezium can capture changes. + +```$bash + + docker run -it --rm --name mysql -p 3306:3306 -e MYSQL_ROOT_PASSWORD=debezium -e MYSQL_USER=mysqluser -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 + +``` + +- Start a Pulsar service locally in standalone mode. + +```$bash + + bin/pulsar standalone + +``` + +- Start pulsar debezium connector, with local run mode, and using above yaml config file. Please make sure that the nar file is available as configured in path `connectors/pulsar-io-kafka-connect-adaptor-2.3.1-SNAPSHOT.nar`. + +```$bash + + bin/pulsar-admin source localrun --sourceConfigFile debezium-mysql-source-config.yaml + +``` + +- Subscribe the topic for table `inventory.products`. + +``` + + bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + +``` + +- start a MySQL cli docker connector, and use it we could change to the table `products` in MySQL server. + +```$bash + +$docker run -it --rm --name mysqlterm --link mysql --rm mysql:5.7 sh -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' + +``` + +This command will pop out MySQL cli, in this cli, we could do a change in table products, use commands below to change the name of 2 items in table products: + +``` + +mysql> use inventory; +mysql> show tables; +mysql> SELECT * FROM products ; +mysql> UPDATE products SET name='1111111111' WHERE id=101; +mysql> UPDATE products SET name='1111111111' WHERE id=107; + +``` + +- In above subscribe topic terminal tab, we could find that 2 changes has been kept into products topic. + +## FAQ + +### Debezium postgres connector will hang when create snap + +```$xslt + +#18 prio=5 os_prio=31 tid=0x00007fd83096f800 nid=0xa403 waiting on condition [0x000070000f534000] + java.lang.Thread.State: WAITING (parking) + at sun.misc.Unsafe.park(Native Method) + - parking to wait for <0x00000007ab025a58> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) + at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) + at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039) + at java.util.concurrent.LinkedBlockingDeque.putLast(LinkedBlockingDeque.java:396) + at java.util.concurrent.LinkedBlockingDeque.put(LinkedBlockingDeque.java:649) + at io.debezium.connector.base.ChangeEventQueue.enqueue(ChangeEventQueue.java:132) + at io.debezium.connector.postgresql.PostgresConnectorTask$Lambda$203/385424085.accept(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer.sendCurrentRecord(RecordsSnapshotProducer.java:402) + at io.debezium.connector.postgresql.RecordsSnapshotProducer.readTable(RecordsSnapshotProducer.java:321) + at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$takeSnapshot$6(RecordsSnapshotProducer.java:226) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$240/1347039967.accept(Unknown Source) + at io.debezium.jdbc.JdbcConnection.queryWithBlockingConsumer(JdbcConnection.java:535) + at io.debezium.connector.postgresql.RecordsSnapshotProducer.takeSnapshot(RecordsSnapshotProducer.java:224) + at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$start$0(RecordsSnapshotProducer.java:87) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$206/589332928.run(Unknown Source) + at java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:705) + at java.util.concurrent.CompletableFuture.uniRunStage(CompletableFuture.java:717) + at java.util.concurrent.CompletableFuture.thenRun(CompletableFuture.java:2010) + at io.debezium.connector.postgresql.RecordsSnapshotProducer.start(RecordsSnapshotProducer.java:87) + at io.debezium.connector.postgresql.PostgresConnectorTask.start(PostgresConnectorTask.java:126) + at io.debezium.connector.common.BaseSourceTask.start(BaseSourceTask.java:47) + at org.apache.pulsar.io.kafka.connect.KafkaConnectSource.open(KafkaConnectSource.java:127) + at org.apache.pulsar.io.debezium.DebeziumSource.open(DebeziumSource.java:100) + at org.apache.pulsar.functions.instance.JavaInstanceRunnable.setupInput(JavaInstanceRunnable.java:690) + at org.apache.pulsar.functions.instance.JavaInstanceRunnable.setupJavaInstance(JavaInstanceRunnable.java:200) + at org.apache.pulsar.functions.instance.JavaInstanceRunnable.run(JavaInstanceRunnable.java:230) + at java.lang.Thread.run(Thread.java:748) + +``` + +If you encounter the above problems in synchronizing data, please refer to [this](https://github.com/apache/pulsar/issues/4075) and add the following configuration to the configuration file: + +```$xslt + +max.queue.size= + +``` + diff --git a/site2/website-next/versioned_docs/version-2.3.1/io-elasticsearch.md b/site2/website-next/versioned_docs/version-2.3.1/io-elasticsearch.md new file mode 100644 index 0000000000000..e9ce0a353d1a6 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/io-elasticsearch.md @@ -0,0 +1,22 @@ +--- +id: io-elasticsearch +title: ElasticSearch Connector +sidebar_label: "ElasticSearch Connector" +original_id: io-elasticsearch +--- + +## Sink + +The ElasticSearch Sink Connector is used to pull messages from Pulsar topics and persist the messages +to a index. + +## Sink Configuration Options + +| Name | Default | Required | Description | +|------|---------|----------|-------------| +| `elasticSearchUrl` | `null` | `true` | The url of elastic search cluster that the connector connects to. | +| `indexName` | `null` | `true` | The index name that the connector writes messages to. | +| `indexNumberOfShards` | `1` | `false` | The number of shards of the index. | +| `indexNumberOfReplicas` | `1` | `false` | The number of replicas of the index. | +| `username` | `null` | `false` | The username used by the connector to connect to the elastic search cluster. If username is set, a password should also be provided. | +| `password` | `null` | `false` | The password used by the connector to connect to the elastic search cluster. If password is set, a username should also be provided. | \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.3.1/io-file.md b/site2/website-next/versioned_docs/version-2.3.1/io-file.md new file mode 100644 index 0000000000000..a0b0f7cad4c48 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/io-file.md @@ -0,0 +1,28 @@ +--- +id: io-file +title: File Connector +sidebar_label: "File Connector" +original_id: io-file +--- + +## Source + +The File Source Connector is used to pull messages from files in a directory and persist the messages +to a Pulsar topic. + +### Source Configuration Options + +| Name | Required | Default | Description | +|------|----------|---------|-------------| +| inputDirectory | `true` | `null` | The input directory from which to pull files. | +| recurse | `false` | `true` | Indicates whether or not to pull files from sub-directories. | +| keepFile | `false` | `false` | If true, the file is not deleted after it has been processed and causes the file to be picked up continually. | +| fileFilter | `false` | `[^\\.].*` | Only files whose names match the given regular expression will be picked up. | +| pathFilter | `false` | `null` | When 'recurse' property is true, then only sub-directories whose path matches the given regular expression will be scanned. | +| minimumFileAge | `false` | `0` | The minimum age that a file must be in order to be processed; any file younger than this amount of time (according to last modification date) will be ignored. | +| maximumFileAge | `false` | `Long.MAX_VALUE` | The maximum age that a file must be in order to be processed; any file older than this amount of time (according to last modification date) will be ignored. | +| minimumSize | `false` | `1` | The minimum size (in bytes) that a file must be in order to be processed. | +| maximumSize | `false` | `Double.MAX_VALUE` | The maximum size (in bytes) that a file can be in order to be processed. | +| ignoreHiddenFiles | `false` | `true` | Indicates whether or not hidden files should be ignored or not. | +| pollingInterval | `false` | `10000` | Indicates how long to wait before performing a directory listing. | +| numWorkers | `false` | `1` | The number of worker threads that will be processing the files. This allows you to process a larger number of files concurrently. However, setting this to a value greater than 1 will result in the data from multiple files being "intermingled" in the target topic. | \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.3.1/io-hdfs.md b/site2/website-next/versioned_docs/version-2.3.1/io-hdfs.md new file mode 100644 index 0000000000000..a7b8eee77bdc4 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/io-hdfs.md @@ -0,0 +1,27 @@ +--- +id: io-hdfs +title: Hdfs Connector +sidebar_label: "Hdfs Connector" +original_id: io-hdfs +--- + +## Sink + +The Hdfs Sink Connector is used to pull messages from Pulsar topics and persist the messages +to an hdfs file. + +## Sink Configuration Options + +| Name | Default | Required | Description | +|------|---------|----------|-------------| +| `hdfsConfigResources` | `null` | `true` | A file or comma separated list of files which contains the Hadoop file system configuration, e.g. 'core-site.xml', 'hdfs-site.xml'. | +| `directory` | `null` | `true` | The HDFS directory from which files should be read from or written to. | +| `encoding` | `null` | `false` | The character encoding for the files, e.g. UTF-8, ASCII, etc. | +| `compression` | `null` | `false` | The compression codec used to compress/de-compress the files on HDFS. | +| `kerberosUserPrincipal` | `null` | `false` | The Kerberos user principal account to use for authentication. | +| `keytab` | `null` | `false` | The full pathname to the Kerberos keytab file to use for authentication. | +| `filenamePrefix` | `null` | `false` | The prefix of the files to create inside the HDFS directory, i.e. a value of "topicA" will result in files named topicA-, topicA-, etc being produced. | +| `fileExtension` | `null` | `false` | The extension to add to the files written to HDFS, e.g. '.txt', '.seq', etc. | +| `separator` | `null` | `false` | The character to use to separate records in a text file. If no value is provided then the content from all of the records will be concatenated together in one continuous byte array. | +| `syncInterval` | `null` | `false` | The interval (in milliseconds) between calls to flush data to HDFS disk. | +| `maxPendingRecords` | `Integer.MAX_VALUE` | `false` | The maximum number of records that we hold in memory before acking. Default is `Integer.MAX_VALUE`. Setting this value to one, results in every record being sent to disk before the record is acked, while setting it to a higher values allows us to buffer records before flushing them all to disk. | \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.3.1/io-influxdb.md b/site2/website-next/versioned_docs/version-2.3.1/io-influxdb.md new file mode 100644 index 0000000000000..e4160fe1e2281 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/io-influxdb.md @@ -0,0 +1,26 @@ +--- +id: io-influxdb +title: InfluxDB Connector +sidebar_label: "InfluxDB Connector" +original_id: io-influxdb +--- + +## Sink + +The InfluxDB Sink Connector is used to pull messages from Pulsar topics and persist the messages +to an InfluxDB database. + +## Sink Configuration Options + +| Name | Default | Required | Description | +|------|---------|----------|-------------| +| `influxdbUrl` | `null` | `true` | The url of the InfluxDB instance to connect to. | +| `username` | `null` | `false` | The username used to authenticate to InfluxDB. | +| `password` | `null` | `false` | The password used to authenticate to InfluxDB. | +| `database` | `null` | `true` | The InfluxDB database to write to. | +| `consistencyLevel` | `ONE` | `false` | The consistency level for writing data to InfluxDB. Possible values [ALL, ANY, ONE, QUORUM]. | +| `logLevel` | `NONE` | `false` | The log level for InfluxDB request and response. Possible values [NONE, BASIC, HEADERS, FULL]. | +| `retentionPolicy` | `autogen` | `false` | The retention policy for the InfluxDB database. | +| `gzipEnable` | `false` | `false` | Flag to determine if gzip should be enabled. | +| `batchTimeMs` | `1000` | `false` | The InfluxDB operation time in milliseconds. | +| `batchSize` | `200` | `false` | The batch size of write to InfluxDB database. | diff --git a/site2/website-next/versioned_docs/version-2.3.1/io-mongo.md b/site2/website-next/versioned_docs/version-2.3.1/io-mongo.md new file mode 100644 index 0000000000000..c36a9f3245202 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/io-mongo.md @@ -0,0 +1,21 @@ +--- +id: io-mongo +title: MongoDB Connector +sidebar_label: "MongoDB Connector" +original_id: io-mongo +--- + +## Sink + +The MongoDB Sink Connector is used to pull messages from Pulsar topics and persist the messages +to a collection. + +## Sink Configuration Options + +| Name | Default | Required | Description | +|------|---------|----------|-------------| +| `mongoUri` | `null` | `true` | The uri of mongodb that the connector connects to (see: https://docs.mongodb.com/manual/reference/connection-string/). | +| `database` | `null` | `true` | The name of the database to which the collection belongs to. | +| `collection` | `null` | `true` | The collection name that the connector writes messages to. | +| `batchSize` | `100` | `false` | The batch size of write to the collection. | +| `batchTimeMs` | `1000` | `false` | The batch operation interval in milliseconds. | \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.3.1/io-rabbitmq.md b/site2/website-next/versioned_docs/version-2.3.1/io-rabbitmq.md new file mode 100644 index 0000000000000..1ed7eda1c3ef4 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/io-rabbitmq.md @@ -0,0 +1,55 @@ +--- +id: io-rabbitmq +title: RabbitMQ Connector +sidebar_label: "RabbitMQ Connector" +original_id: io-rabbitmq +--- + +## Source + +The RabbitMQ Source connector is used for receiving messages from a RabbitMQ cluster and writing +messages to Pulsar topics. + +### Source Configuration Options + +| Name | Required | Default | Description | +|------|----------|---------|-------------| +| `connectionName` | `true` | `null` | The connection name used for connecting to RabbitMQ. | +| `host` | `true` | `null` | The RabbitMQ host to connect to. | +| `port` | `true` | `5672` | The RabbitMQ port to connect to. | +| `virtualHost` | `true` | `/` | The virtual host used for connecting to RabbitMQ. | +| `username` | `false` | `guest` | The username used to authenticate to RabbitMQ. | +| `password` | `false` | `guest` | The password used to authenticate to RabbitMQ. | +| `queueName` | `true` | `null` | The RabbitMQ queue name from which messages should be read from or written to. | +| `requestedChannelMax` | `false` | `0` | Initially requested maximum channel number. 0 for unlimited. | +| `requestedFrameMax` | `false` | `0` | Initially requested maximum frame size, in octets. 0 for unlimited. | +| `connectionTimeout` | `false` | `60000` | Connection TCP establishment timeout in milliseconds. 0 for infinite. | +| `handshakeTimeout` | `false` | `10000` | The AMQP0-9-1 protocol handshake timeout in milliseconds. | +| `requestedHeartbeat` | `false` | `60` | The requested heartbeat timeout in seconds. | +| `prefetchCount` | `false` | `0` | Maximum number of messages that the server will deliver, 0 for unlimited. | +| `prefetchGlobal` | `false` | `false` | Set true if the settings should be applied to the entire channel rather than each consumer. | + +## Sink + +The RabbitMQ Sink connector is used to pull messages from Pulsar topics and persist the messages +to a RabbitMQ queue. + +### Sink Configuration Options + +| Name | Required | Default | Description | +|------|----------|---------|-------------| +| `connectionName` | `true` | `null` | The connection name used for connecting to RabbitMQ. | +| `host` | `true` | `null` | The RabbitMQ host to connect to. | +| `port` | `true` | `5672` | The RabbitMQ port to connect to. | +| `virtualHost` | `true` | `/` | The virtual host used for connecting to RabbitMQ. | +| `username` | `false` | `guest` | The username used to authenticate to RabbitMQ. | +| `password` | `false` | `guest` | The password used to authenticate to RabbitMQ. | +| `queueName` | `true` | `null` | The RabbitMQ queue name from which messages should be read from or written to. | +| `requestedChannelMax` | `false` | `0` | Initially requested maximum channel number. 0 for unlimited. | +| `requestedFrameMax` | `false` | `0` | Initially requested maximum frame size, in octets. 0 for unlimited. | +| `connectionTimeout` | `false` | `60000` | Connection TCP establishment timeout in milliseconds. 0 for infinite. | +| `handshakeTimeout` | `false` | `10000` | The AMQP0-9-1 protocol handshake timeout in milliseconds. | +| `requestedHeartbeat` | `false` | `60` | The requested heartbeat timeout in seconds. | +| `exchangeName` | `true` | `null` | The exchange to publish the messages on. | +| `routingKey` | `true` | `null` | The routing key used for publishing the messages. | + diff --git a/site2/website-next/versioned_docs/version-2.3.1/io-solr.md b/site2/website-next/versioned_docs/version-2.3.1/io-solr.md new file mode 100644 index 0000000000000..f0050517d3fac --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/io-solr.md @@ -0,0 +1,22 @@ +--- +id: io-solr +title: solr Connector +sidebar_label: "solr Connector" +original_id: io-solr +--- + +## Sink + +The solr Sink Connector is used to pull messages from Pulsar topics and persist the messages +to a solr collection. + +## Sink Configuration Options + +| Name | Default | Required | Description | +|------|---------|----------|-------------| +| `solrUrl` | `null` | `true` | Comma separated zookeeper hosts with chroot used in SolrCloud mode (eg: localhost:2181,localhost:2182/chroot) or Url to connect to solr used in Standalone mode (e.g. localhost:8983/solr). | +| `solrMode` | `SolrCloud` | `true` | The client mode to use when interacting with the Solr cluster. Possible values [Standalone, SolrCloud]. | +| `solrCollection` | `null` | `true` | Solr collection name to which records need to be written. | +| `solrCommitWithinMs` | `10` | `false` | Commit within milli seconds for solr update, if none passes defaults to 10 ms. | +| `username` | `null` | `false` | The username to use for basic authentication. | +| `password` | `null` | `false` | The password to use for basic authentication. | \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.3.1/reference-cli-tools.md b/site2/website-next/versioned_docs/version-2.3.1/reference-cli-tools.md new file mode 100644 index 0000000000000..902f47bb408c8 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/reference-cli-tools.md @@ -0,0 +1,857 @@ +--- +id: reference-cli-tools +title: Pulsar command-line tools +sidebar_label: "Pulsar CLI tools" +original_id: reference-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](getting-started-standalone). The following tools are currently documented: + +* [`pulsar`](#pulsar) +* [`pulsar-client`](#pulsar-client) +* [`pulsar-daemon`](#pulsar-daemon) +* [`pulsar-perf`](#pulsar-perf) +* [`bookkeeper`](#bookkeeper) + +> ### 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: + +> ```shell +> +> $ 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: + +```bash + +$ pulsar command + +``` + +Commands: +* `bookie` +* `broker` +* `compact-topic` +* `discovery` +* `configuration-store` +* `initialize-cluster-metadata` +* `proxy` +* `standalone` +* `websocket` +* `zookeeper` +* `zookeeper-shell` + +Example: + +```bash + +$ 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. + +|Variable|Description|Default| +|---|---|---| +|`PULSAR_LOG_CONF`|Log4j configuration file|`conf/log4j2.yaml`| +|`PULSAR_BROKER_CONF`|Configuration file for broker|`conf/broker.conf`| +|`PULSAR_BOOKKEEPER_CONF`|description: Configuration file for bookie|`conf/bookkeeper.conf`| +|`PULSAR_ZK_CONF`|Configuration file for zookeeper|`conf/zookeeper.conf`| +|`PULSAR_CONFIGURATION_STORE_CONF`|Configuration file for the configuration store|`conf/global_zookeeper.conf`| +|`PULSAR_DISCOVERY_CONF`|Configuration file for discovery service|`conf/discovery.conf`| +|`PULSAR_WEBSOCKET_CONF`|Configuration file for websocket proxy|`conf/websocket.conf`| +|`PULSAR_STANDALONE_CONF`|Configuration file for standalone|`conf/standalone.conf`| +|`PULSAR_EXTRA_OPTS`|Extra options to be passed to the jvm|| +|`PULSAR_EXTRA_CLASSPATH`|Extra paths for Pulsar's classpath|| +|`PULSAR_PID_DIR`|Folder where the pulsar server PID file should be stored|| +|`PULSAR_STOP_TIMEOUT`|Wait time before forcefully killing the Bookie server instance if attempts to stop it are not successful|| + + + +### `bookie` + +Starts up a bookie server + +Usage: + +```bash + +$ pulsar bookie options + +``` + +Options + +|Option|Description|Default| +|---|---|---| +|`-readOnly`|Force start a read-only bookie server|false| +|`-withAutoRecovery`|Start auto-recover service bookie server|false| + + +Example + +```bash + +$ PULSAR_BOOKKEEPER_CONF=/path/to/bookkeeper.conf pulsar bookie \ + -readOnly \ + -withAutoRecovery + +``` + +### `broker` + +Starts up a Pulsar broker + +Usage + +```bash + +$ 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 + +```bash + +$ PULSAR_BROKER_CONF=/path/to/broker.conf pulsar broker + +``` + +### `compact-topic` + +Run compaction against a Pulsar topic (in a new process) + +Usage + +```bash + +$ pulsar compact-topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-t` , `--topic`|The Pulsar topic that you would like to compact|| + +Example + +```bash + +$ pulsar compact-topic --topic topic-to-compact + +``` + +### `discovery` + +Run a discovery server + +Usage + +```bash + +$ pulsar discovery + +``` + +Example + +```bash + +$ PULSAR_DISCOVERY_CONF=/path/to/discovery.conf pulsar discovery + +``` + +### `configuration-store` + +Starts up the Pulsar configuration store + +Usage + +```bash + +$ pulsar configuration-store + +``` + +Example + +```bash + +$ PULSAR_CONFIGURATION_STORE_CONF=/path/to/configuration_store.conf pulsar configuration-store + +``` + +### `initialize-cluster-metadata` + +One-time cluster metadata initialization + +Usage + +```bash + +$ 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|| +|`--configuration-store`|The configuration store quorum connection string|| +|`-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|| + + +### `proxy` + +Manages the Pulsar proxy + +Usage + +```bash + +$ pulsar proxy options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--configuration-store`|Configuration store connection string|| +|`-zk` , `--zookeeper-servers`|Local ZooKeeper connection string|| + +Example + +```bash + +$ 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 + +```bash + +$ pulsar standalone options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-a` , `--advertised-address`|The standalone broker advertised address|| +|`--bookkeeper-dir`|Local bookies’ base data directory|data/standalone/bookkeeper| +|`--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 + +```bash + +$ PULSAR_STANDALONE_CONF=/path/to/standalone.conf pulsar standalone + +``` + +### `websocket` + +Usage + +```bash + +$ pulsar websocket + +``` + +Example + +```bash + +$ PULSAR_WEBSOCKET_CONF=/path/to/websocket.conf pulsar websocket + +``` + +### `zookeeper` + +Starts up a ZooKeeper cluster + +Usage + +```bash + +$ pulsar zookeeper + +``` + +Example + +```bash + +$ PULSAR_ZK_CONF=/path/to/zookeeper.conf pulsar zookeeper + +``` + +### `zookeeper-shell` + +Connects to a running ZooKeeper cluster using the ZooKeeper shell + +Usage + +```bash + +$ pulsar zookeeper-shell options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-c`, `--conf`|Configuration file for ZooKeeper|| + + + +## `pulsar-client` + +The pulsar-client tool + +Usage + +```bash + +$ 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\"}"|| +|`--auth-plugin`|Authentication plugin class name|| +|`--url`|Broker URL to which to connect|pulsar://localhost:6650/| +|`-h`, `--help`|Show this help + + +### `produce` +Send a message or messages to a specific broker and topic + +Usage + +```bash + +$ 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 + +```bash + +$ 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.|0| +|`-r`, `--rate`|Rate (in messages per second) at which to consume; a value 0 means to consume messages as fast as possible|0.0| +|`-s`, `--subscription-name`|Subscription name|| +|`-t`, `--subscription-type`|The type of the subscription. Possible values: Exclusive, Shared, Failover.|Exclusive| + + + +## `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 + +```bash + +$ pulsar-daemon command + +``` + +Commands +* `start` +* `stop` + + +### `start` +Start a service in the background using nohup. + +Usage + +```bash + +$ pulsar-daemon start service + +``` + +### `stop` +Stop a service that’s already been started using start. + +Usage + +```bash + +$ 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 + +```bash + +$ 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. + +|Variable|Description|Default| +|---|---|---| +|`PULSAR_LOG_CONF`|Log4j configuration file|conf/log4j2.yaml| +|`PULSAR_CLIENT_CONF`|Configuration file for the client|conf/client.conf| +|`PULSAR_EXTRA_OPTS`|Extra options to be passed to the JVM|| +|`PULSAR_EXTRA_CLASSPATH`|Extra paths for Pulsar's classpath|| + + +### `consume` +Run a consumer + +Usage + +``` + +$ pulsar-perf consume 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|| +|`--acks-delay-millis`|Acknowlegments grouping delay in millis|100| +|`-k`, `--encryption-key-name`|The private key name to decrypt payload|| +|`-v`, `--encryption-key-value-file`|The file which contains the private key to decrypt payload|| +|`-h`, `--help`|Help message|false| +|`--conf-file`|Configuration file|| +|`-c`, `--max-connections`|Max number of TCP connections to a single broker|100| +|`-n`, `--num-consumers`|Number of consumers (per topic)|1| +|`-t`, `--num-topic`|The number of topics|1| +|`-r`, `--rate`|Simulate a slow message consumer (rate in msg/s)|0| +|`-q`, `--receiver-queue-size`|Size of the receiver queue|1000| +|`-u`, `--service-url`|Pulsar service URL|| +|`-i`, `--stats-interval-seconds`|Statistics interval seconds. If 0, statistics will be disabled|0| +|`-s`, `--subscriber-name`|Subscriber name prefix|sub| +|`-st`, `--subscription-type`|Subscriber name prefix. Possible values are Exclusive, Shared, Failover.|Exclusive| +|`--trust-cert-file`|Path for the trusted TLS certificate file|| + + +### `produce` +Run a producer + +Usage + +```bash + +$ pulsar-perf produce 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|| +|`-b`, `--batch-time-window`|Batch messages in a window of the specified number of milliseconds|1| +|`-z`, `--compression`|Compress messages’ payload. Possible values are NONE, LZ4, ZLIB, ZSTD or SNAPPY.|| +|`--conf-file`|Configuration file|| +|`-k`, `--encryption-key-name`|The public key name to encrypt payload|| +|`-v`, `--encryption-key-value-file`|The file which contains the public key to encrypt payload|| +|`-h`, `--help`|Help message|false| +|`-c`, `--max-connections`|Max number of TCP connections to a single broker|100| +|`-o`, `--max-outstanding`|Max number of outstanding messages|1000| +|`-m`, `--num-messages`|Number of messages to publish in total. If set to 0, it will keep publishing.|0| +|`-n`, `--num-producers`|The number of producers (per topic)|1| +|`-t`, `--num-topic`|The number of topics|1| +|`-f`, `--payload-file`|Use payload from a file instead of an empty buffer|| +|`-r`, `--rate`|Publish rate msg/s across topics|100| +|`-u`, `--service-url`|Pulsar service URL|| +|`-s`, `--size`|Message size (in bytes)|1024| +|`-i`, `--stats-interval-seconds`|Statistics interval seconds. If 0, statistics will be disabled.|0| +|`-time`, `--test-duration`|Test duration in secs. If set to 0, it will keep publishing.|0| +|`--trust-cert-file`|Path for the trusted TLS certificate file|| +|`--warmup-time`|Warm-up time in seconds|1| + + +### `read` +Run a topic reader + +Usage + +```bash + +$ 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|| +|`--conf-file`|Configuration file|| +|`-h`, `--help`|Help message|false| +|`-c`, `--max-connections`|Max number of TCP connections to a single broker|100| +|`-t`, `--num-topic`|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| + + +### `websocket-producer` +Run a websocket producer + +Usage + +```bash + +$ 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 the value is 0 or smaller than 0, it keeps publishing messages|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 seconds). If the value is 0 or smaller than 0, it keeps publishing messages|0| + + +### `managed-ledger` +Write directly on managed-ledgers + +Usage + +```bash + +$ 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 the value is 0 or smaller than 0, it keeps publishing messages|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 seconds). If the value is 0 or smaller than 0, it keeps publishing messages|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 + +```bash + +$ 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 + +```bash + +$ 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 + +```bash + +$ 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 + +```bash + +$ pulsar-perf help + +``` + +## `bookkeeper` +A tool for managing BookKeeper. + +Usage + +```bash + +$ bookkeeper command + +``` + +Commands +* `autorecovery` +* `bookie` +* `localbookie` +* `upgrade` +* `shell` + + +Environment variables + +The table below lists the environment variables that you can use to configure the bookkeeper tool. + +|Variable|Description|Default| +|---|---|---| +|BOOKIE_LOG_CONF|Log4j configuration file|conf/log4j2.yaml| +|BOOKIE_CONF|BookKeeper configuration file|conf/bk_server.conf| +|BOOKIE_EXTRA_OPTS|Extra options to be passed to the JVM|| +|BOOKIE_EXTRA_CLASSPATH|Extra paths for BookKeeper's classpath|| +|ENTRY_FORMATTER_CLASS|The Java class used to format entries|| +|BOOKIE_PID_DIR|Folder where the BookKeeper server PID file should be stored|| +|BOOKIE_STOP_TIMEOUT|Wait time before forcefully killing the Bookie server instance if attempts to stop it are not successful|| + + +### `autorecovery` +Runs an auto-recovery service + +Usage + +```bash + +$ bookkeeper autorecovery options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-c`, `--conf`|Configuration for the auto-recovery|| + + +### `bookie` +Starts up a BookKeeper server (aka bookie) + +Usage + +```bash + +$ bookkeeper bookie options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-c`, `--conf`|Configuration for the auto-recovery|| +|-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 + +```bash + +$ bookkeeper localbookie N + +``` + +### `upgrade` +Upgrade the bookie’s filesystem + +Usage + +```bash + +$ bookkeeper upgrade options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-c`, `--conf`|Configuration for the auto-recovery|| +|`-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 + +```bash + +$ bookkeeper shell + +``` + +Example + +```bash + +$ bookkeeper shell bookiesanity + +``` + diff --git a/site2/website-next/versioned_docs/version-2.3.1/reference-configuration.md b/site2/website-next/versioned_docs/version-2.3.1/reference-configuration.md new file mode 100644 index 0000000000000..6b8e1226663ee --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/reference-configuration.md @@ -0,0 +1,482 @@ +--- +id: reference-configuration +title: Pulsar configuration +sidebar_label: "Pulsar configuration" +original_id: reference-configuration +--- + + + + +Pulsar configuration can be managed either via a series of configuration files contained in the [`conf`](https://github.com/apache/pulsar/tree/master/conf) directory of a Pulsar [installation](getting-started-standalone) + +* [BookKeeper](#bookkeeper) +* [Broker](#broker) +* [Client](#client) +* [Service discovery](#service-discovery) +* [Log4j](#log4j) +* [Log4j shell](#log4j-shell) +* [Standalone](#standalone) +* [WebSocket](#websocket) +* [ZooKeeper](#zookeeper) + +## BookKeeper + +BookKeeper is a replicated log storage system that Pulsar uses for durable storage of all messages. + + +|Name|Description|Default| +|---|---|---| +|bookiePort|The port on which the bookie server listens.|3181| +|allowLoopback|Whether the bookie is allowed to use a loopback interface as its primary interface (i.e. the interface used to establish its identity). By default, loopback interfaces are not allowed as the primary interface. Using a loopback interface as the primary interface usually indicates a configuration error. For example, it’s fairly common in some VPS setups to not configure a hostname or to have the hostname resolve to `127.0.0.1`. If this is the case, then all bookies in the cluster will establish their identities as `127.0.0.1:3181` and only one will be able to join the cluster. For VPSs configured like this, you should explicitly set the listening interface.|false| +|listeningInterface|The network interface on which the bookie listens. If not set, the bookie will listen on all interfaces.|eth0| +|journalDirectory|The directory where Bookkeeper outputs its write-ahead log (WAL)|data/bookkeeper/journal| +|ledgerDirectories|The directory where Bookkeeper outputs ledger snapshots. This could define multiple directories to store snapshots separated by comma, for example `ledgerDirectories=/tmp/bk1-data,/tmp/bk2-data`. Ideally, ledger dirs and the journal dir are each in a different device, which reduces the contention between random I/O and sequential write. It is possible to run with a single disk, but performance will be significantly lower.|data/bookkeeper/ledgers| +|ledgerManagerType|The type of ledger manager used to manage how ledgers are stored, managed, and garbage collected. See [BookKeeper Internals](http://bookkeeper.apache.org/docs/latest/getting-started/concepts) for more info.|hierarchical| +|zkLedgersRootPath|The root ZooKeeper path used to store ledger metadata. This parameter is used by the ZooKeeper-based ledger manager as a root znode to store all ledgers.|/ledgers| +|ledgerStorageClass|Ledger storage implementation class|org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage| +|entryLogFilePreallocationEnabled|Enable or disable entry logger preallocation|true| +|logSizeLimit|Max file size of the entry logger, in bytes. A new entry log file will be created when the old one reaches the file size limitation.|2147483648| +|minorCompactionThreshold|Threshold of minor compaction. Entry log files whose remaining size percentage reaches below this threshold will be compacted in a minor compaction. If set to less than zero, the minor compaction is disabled.|0.2| +|minorCompactionInterval|Time interval to run minor compaction, in seconds. If set to less than zero, the minor compaction is disabled.|3600| +|majorCompactionThreshold|The threshold of major compaction. Entry log files whose remaining size percentage reaches below this threshold will be compacted in a major compaction. Those entry log files whose remaining size percentage is still higher than the threshold will never be compacted. If set to less than zero, the minor compaction is disabled.|0.5| +|majorCompactionInterval|The time interval to run major compaction, in seconds. If set to less than zero, the major compaction is disabled.|86400| +|compactionMaxOutstandingRequests|Sets the maximum number of entries that can be compacted without flushing. When compacting, the entries are written to the entrylog and the new offsets are cached in memory. Once the entrylog is flushed the index is updated with the new offsets. This parameter controls the number of entries added to the entrylog before a flush is forced. A higher value for this parameter means more memory will be used for offsets. Each offset consists of 3 longs. This parameter should not be modified unless you’re fully aware of the consequences.|100000| +|compactionRate|The rate at which compaction will read entries, in adds per second.|1000| +|isThrottleByBytes|Throttle compaction by bytes or by entries.|false| +|compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| +|compactionRateByBytes|Set the rate at which compaction reads entries. The unit is bytes added per second.|1000000| +|journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| +|journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| +|journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| +|journalAdaptiveGroupWrites|Whether to group journal force writes, which optimizes group commit for higher throughput.|true| +|journalMaxGroupWaitMSec|The maximum latency to impose on a journal write to achieve grouping.|1| +|journalAlignmentSize|All the journal writes and commits should be aligned to given size|4096| +|journalBufferedWritesThreshold|Maximum writes to buffer to achieve grouping|524288| +|journalFlushWhenQueueEmpty|If we should flush the journal when journal queue is empty|false| +|numJournalCallbackThreads|The number of threads that should handle journal callbacks|8| +|rereplicationEntryBatchSize|The number of max entries to keep in fragment for re-replication|5000| +|gcWaitTime|How long the interval to trigger next garbage collection, in milliseconds. Since garbage collection is running in background, too frequent gc will heart performance. It is better to give a higher number of gc interval if there is enough disk capacity.|900000| +|gcOverreplicatedLedgerWaitTime|How long the interval to trigger next garbage collection of overreplicated ledgers, in milliseconds. This should not be run very frequently since we read the metadata for all the ledgers on the bookie from zk.|86400000| +|flushInterval|How long the interval to flush ledger index pages to disk, in milliseconds. Flushing index files will introduce much random disk I/O. If separating journal dir and ledger dirs each on different devices, flushing would not affect performance. But if putting journal dir and ledger dirs on same device, performance degrade significantly on too frequent flushing. You can consider increment flush interval to get better performance, but you need to pay more time on bookie server restart after failure.|60000| +|bookieDeathWatchInterval|Interval to watch whether bookie is dead or not, in milliseconds|1000| +|zkServers|A list of one of more servers on which zookeeper is running. The server list can be comma separated values, for example: zkServers=zk1:2181,zk2:2181,zk3:2181.|localhost:2181| +|zkTimeout|ZooKeeper client session timeout in milliseconds Bookie server will exit if it received SESSION_EXPIRED because it was partitioned off from ZooKeeper for more than the session timeout JVM garbage collection, disk I/O will cause SESSION_EXPIRED. Increment this value could help avoiding this issue|30000| +|serverTcpNoDelay|This settings is used to enabled/disabled Nagle’s algorithm, which is a means of improving the efficiency of TCP/IP networks by reducing the number of packets that need to be sent over the network. If you are sending many small messages, such that more than one can fit in a single IP packet, setting server.tcpnodelay to false to enable Nagle algorithm can provide better performance.|true| +|openFileLimit|Max number of ledger index files could be opened in bookie server If number of ledger index files reaches this limitation, bookie server started to swap some ledgers from memory to disk. Too frequent swap will affect performance. You can tune this number to gain performance according your requirements.|0| +|pageSize|Size of a index page in ledger cache, in bytes A larger index page can improve performance writing page to disk, which is efficient when you have small number of ledgers and these ledgers have similar number of entries. If you have large number of ledgers and each ledger has fewer entries, smaller index page would improve memory usage.|8192| +|pageLimit|How many index pages provided in ledger cache If number of index pages reaches this limitation, bookie server starts to swap some ledgers from memory to disk. You can increment this value when you found swap became more frequent. But make sure pageLimit*pageSize should not more than JVM max memory limitation, otherwise you would got OutOfMemoryException. In general, incrementing pageLimit, using smaller index page would gain better performance in lager number of ledgers with fewer entries case If pageLimit is -1, bookie server will use 1/3 of JVM memory to compute the limitation of number of index pages.|0| +|readOnlyModeEnabled|If all ledger directories configured are full, then support only read requests for clients. If “readOnlyModeEnabled=true” then on all ledger disks full, bookie will be converted to read-only mode and serve only read requests. Otherwise the bookie will be shutdown. By default this will be disabled.|true| +|diskUsageThreshold|For each ledger dir, maximum disk space which can be used. Default is 0.95f. i.e. 95% of disk can be used at most after which nothing will be written to that partition. If all ledger dir partitions are full, then bookie will turn to readonly mode if ‘readOnlyModeEnabled=true’ is set, else it will shutdown. Valid values should be in between 0 and 1 (exclusive).|0.95| +|diskCheckInterval|Disk check interval in milli seconds, interval to check the ledger dirs usage.|10000| +|auditorPeriodicCheckInterval|Interval at which the auditor will do a check of all ledgers in the cluster. By default this runs once a week. The interval is set in seconds. To disable the periodic check completely, set this to 0. Note that periodic checking will put extra load on the cluster, so it should not be run more frequently than once a day.|604800| +|auditorPeriodicBookieCheckInterval|The interval between auditor bookie checks. The auditor bookie check, checks ledger metadata to see which bookies should contain entries for each ledger. If a bookie which should contain entries is unavailable, thea the ledger containing that entry is marked for recovery. Setting this to 0 disabled the periodic check. Bookie checks will still run when a bookie fails. The interval is specified in seconds.|86400| +|numAddWorkerThreads|number of threads that should handle write requests. if zero, the writes would be handled by netty threads directly.|0| +|numReadWorkerThreads|number of threads that should handle read requests. if zero, the reads would be handled by netty threads directly.|8| +|maxPendingReadRequestsPerThread|If read workers threads are enabled, limit the number of pending requests, to avoid the executor queue to grow indefinitely.|2500| +|readBufferSizeBytes|The number of bytes we should use as capacity for BufferedReadChannel.|4096| +|writeBufferSizeBytes|The number of bytes used as capacity for the write buffer|65536| +|useHostNameAsBookieID|Whether the bookie should use its hostname to register with the coordination service (e.g.: zookeeper service). When false, bookie will use its ipaddress for the registration.|false| +|statsProviderClass||org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider| +|prometheusStatsHttpPort||8000| +|dbStorage_writeCacheMaxSizeMb|Size of Write Cache. Memory is allocated from JVM direct memory. Write cache is used to buffer entries before flushing into the entry log For good performance, it should be big enough to hold a substantial amount of entries in the flush interval.|25% of direct memory| +|dbStorage_readAheadCacheMaxSizeMb|Size of Read cache. Memory is allocated from JVM direct memory. This read cache is pre-filled doing read-ahead whenever a cache miss happens|256| +|dbStorage_readAheadCacheBatchSize|How many entries to pre-fill in cache after a read cache miss|1000| +|dbStorage_rocksDB_blockCacheSize|Size of RocksDB block-cache. For best performance, this cache should be big enough to hold a significant portion of the index database which can reach ~2GB in some cases|268435456| +|dbStorage_rocksDB_writeBufferSizeMB||64| +|dbStorage_rocksDB_sstSizeInMB||64| +|dbStorage_rocksDB_blockSize||65536| +|dbStorage_rocksDB_bloomFilterBitsPerKey||10| +|dbStorage_rocksDB_numLevels||-1| +|dbStorage_rocksDB_numFilesInLevel0||4| +|dbStorage_rocksDB_maxSizeInLevel1MB||256| + + + +## Broker + +Pulsar brokers are responsible for handling incoming messages from producers, dispatching messages to consumers, replicating data between clusters, and more. + +|Name|Description|Default| +|---|---|---| +|enablePersistentTopics| Whether persistent topics are enabled on the broker |true| +|enableNonPersistentTopics| Whether non-persistent topics are enabled on the broker |true| +|functionsWorkerEnabled| Whether the Pulsar Functions worker service is enabled in the broker |false| +|zookeeperServers| Zookeeper quorum connection string || +|configurationStoreServers| Configuration store connection string (as a comma-separated list) || +|brokerServicePort| Broker data port |6650| +|brokerServicePortTls| Broker data port for TLS |6651| +|webServicePort| Port to use to server HTTP request |8080| +|webServicePortTls| Port to use to server HTTPS request |8443| +|webSocketServiceEnabled| Enable the WebSocket API service in broker |false| +|bindAddress| Hostname or IP address the service binds on, default is 0.0.0.0. |0.0.0.0| +|advertisedAddress| Hostname or IP address the service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostName()` is used. || +|clusterName| Name of the cluster to which this broker belongs to || +|brokerDeduplicationEnabled| Sets the default behavior for message deduplication in the broker. If enabled, the broker will reject messages that were already stored in the topic. This setting can be overridden on a per-namespace basis. |false| +|brokerDeduplicationMaxNumberOfProducers| The maximum number of producers for which information will be stored for deduplication purposes. |10000| +|brokerDeduplicationEntriesInterval| The number of entries after which a deduplication informational snapshot is taken. A larger interval will lead to fewer snapshots being taken, though this would also lengthen the topic recovery time (the time required for entries published after the snapshot to be replayed). |1000| +|brokerDeduplicationProducerInactivityTimeoutMinutes| The time of inactivity (in minutes) after which the broker will discard deduplication information related to a disconnected producer. |360| +|zooKeeperSessionTimeoutMillis| Zookeeper session timeout in milliseconds |30000| +|brokerShutdownTimeoutMs| Time to wait for broker graceful shutdown. After this time elapses, the process will be killed |60000| +|backlogQuotaCheckEnabled| Enable backlog quota check. Enforces action on topic when the quota is reached |true| +|backlogQuotaCheckIntervalInSeconds| How often to check for topics that have reached the quota |60| +|backlogQuotaDefaultLimitGB| Default per-topic backlog quota limit |10| +|brokerDeleteInactiveTopicsEnabled| Enable the deletion of inactive topics. If topics are not consumed for some while, these inactive topics might be cleaned up. Deleting inactive topics is enabled by default. The default period is 1 minute. |true| +|brokerDeleteInactiveTopicsFrequencySeconds| How often to check for inactive topics |60| +|messageExpiryCheckIntervalInMinutes| How frequently to proactively check and purge expired messages |5| +|brokerServiceCompactionMonitorIntervalInSeconds| Interval between checks to see if topics with compaction policies need to be compacted |60| +|activeConsumerFailoverDelayTimeMillis| How long to delay rewinding cursor and dispatching messages when active consumer is changed. |1000| +|clientLibraryVersionCheckEnabled| Enable check for minimum allowed client library version |false| +|clientLibraryVersionCheckAllowUnversioned| Allow client libraries with no version information |true| +|statusFilePath| Path for the file used to determine the rotation status for the broker when responding to service discovery health checks || +|preferLaterVersions| If true, (and ModularLoadManagerImpl is being used), the load manager will attempt to use only brokers running the latest software version (to minimize impact to bundles) |false| +|tlsEnabled| Enable TLS |false| +|tlsCertificateFilePath| Path for the TLS certificate file || +|tlsKeyFilePath| Path for the TLS private key file || +|tlsTrustCertsFilePath| Path for the trusted TLS certificate file || +|tlsAllowInsecureConnection| Accept untrusted TLS certificate from client |false| +|tlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLSv1.2```, ```TLSv1.1```, ```TLSv1``` || +|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256```|| +|tokenSecretKey| Configure the secret key to be used to validate auth tokens. The key can be specified like: `tokenSecretKey=data:base64,xxxxxxxxx` or `tokenSecretKey=file:///my/secret.key`|| +|tokenPublicKey| Configure the public key to be used to validate auth tokens. The key can be specified like: `tokenPublicKey=data:base64,xxxxxxxxx` or `tokenPublicKey=file:///my/secret.key`|| +|tokenAuthClaim| Specify which of the token's claims will be used as the authentication "principal" or "role". The default "sub" claim will be used if this is left blank || +|maxUnackedMessagesPerConsumer| Max number of unacknowledged messages allowed to receive messages by a consumer on a shared subscription. Broker will stop sending messages to consumer once, this limit reaches until consumer starts acknowledging messages back. Using a value of 0, is disabling unackeMessage limit check and consumer can receive messages without any restriction |50000| +|maxUnackedMessagesPerSubscription| Max number of unacknowledged messages allowed per shared subscription. Broker will stop dispatching messages to all consumers of the subscription once this limit reaches until consumer starts acknowledging messages back and unack count reaches to limit/2. Using a value of 0, is disabling unackedMessage-limit check and dispatcher can dispatch messages without any restriction |200000| +|subscriptionRedeliveryTrackerEnabled| Enable subscription message redelivery tracker |true| +|maxConcurrentLookupRequest| Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic |50000| +|maxConcurrentTopicLoadRequest| Max number of concurrent topic loading request broker allows to control number of zk-operations |5000| +|authenticationEnabled| Enable authentication |false| +|authenticationProviders| Authentication provider name list, which is comma separated list of class names || +|authorizationEnabled| Enforce authorization |false| +|superUserRoles| Role names that are treated as “super-user”, meaning they will be able to do all admin operations and publish/consume from all topics || +|brokerClientAuthenticationPlugin| Authentication settings of the broker itself. Used when the broker connects to other brokers, either in same or other clusters || +|brokerClientAuthenticationParameters||| +|athenzDomainNames| Supported Athenz provider domain names(comma separated) for authentication || +|bookkeeperClientAuthenticationPlugin| Authentication plugin to use when connecting to bookies || +|bookkeeperClientAuthenticationParametersName| BookKeeper auth plugin implementation specifics parameters name and values || +|bookkeeperClientAuthenticationParameters||| +|bookkeeperClientTimeoutInSeconds| Timeout for BK add / read operations |30| +|bookkeeperClientSpeculativeReadTimeoutInMillis| Speculative reads are initiated if a read request doesn’t complete within a certain time Using a value of 0, is disabling the speculative reads |0| +|bookkeeperClientHealthCheckEnabled| Enable bookies health check. Bookies that have more than the configured number of failure within the interval will be quarantined for some time. During this period, new ledgers won’t be created on these bookies |true| +|bookkeeperClientHealthCheckIntervalSeconds||60| +|bookkeeperClientHealthCheckErrorThresholdPerInterval||5| +|bookkeeperClientHealthCheckQuarantineTimeInSeconds ||1800| +|bookkeeperClientRackawarePolicyEnabled| Enable rack-aware bookie selection policy. BK will chose bookies from different racks when forming a new bookie ensemble |true| +|bookkeeperClientRegionawarePolicyEnabled| Enable region-aware bookie selection policy. BK will chose bookies from different regions and racks when forming a new bookie ensemble. If enabled, the value of bookkeeperClientRackawarePolicyEnabled is ignored |false| +|bookkeeperClientReorderReadSequenceEnabled| Enable/disable reordering read sequence on reading entries. |false| +|bookkeeperClientIsolationGroups| Enable bookie isolation by specifying a list of bookie groups to choose from. Any bookie outside the specified groups will not be used by the broker || +|bookkeeperEnableStickyReads | Enable/disable having read operations for a ledger to be sticky to a single bookie. If this flag is enabled, the client will use one single bookie (by preference) to read all entries for a ledger. | true | +|managedLedgerDefaultEnsembleSize| Number of bookies to use when creating a ledger |2| +|managedLedgerDefaultWriteQuorum| Number of copies to store for each message |2| +|managedLedgerDefaultAckQuorum| Number of guaranteed copies (acks to wait before write is complete) |2| +|managedLedgerCacheSizeMB| Amount of memory to use for caching data payload in managed ledger. This memory is allocated from JVM direct memory and it’s shared across all the topics running in the same broker. By default, uses 1/5th of available direct memory || +|managedLedgerCacheEvictionWatermark| Threshold to which bring down the cache level when eviction is triggered |0.9| +|managedLedgerDefaultMarkDeleteRateLimit| Rate limit the amount of writes per second generated by consumer acking the messages |1.0| +|managedLedgerMaxEntriesPerLedger| The max number of entries to append to a ledger before triggering a rollover. A ledger rollover is triggered after the min rollover time has passed and one of the following conditions is true:
  • The max rollover time has been reached
  • The max entries have been written to the ledger
  • The max ledger size has been written to the ledger
|50000| +|managedLedgerMinLedgerRolloverTimeMinutes| Minimum time between ledger rollover for a topic |10| +|managedLedgerMaxLedgerRolloverTimeMinutes| Maximum time before forcing a ledger rollover for a topic |240| +|managedLedgerCursorMaxEntriesPerLedger| Max number of entries to append to a cursor ledger |50000| +|managedLedgerCursorRolloverTimeInSeconds| Max time before triggering a rollover on a cursor ledger |14400| +|managedLedgerMaxUnackedRangesToPersist| Max number of “acknowledgment holes” that are going to be persistently stored. When acknowledging out of order, a consumer will leave holes that are supposed to be quickly filled by acking all the messages. The information of which messages are acknowledged is persisted by compressing in “ranges” of messages that were acknowledged. After the max number of ranges is reached, the information will only be tracked in memory and messages will be redelivered in case of crashes. |1000| +|autoSkipNonRecoverableData| Skip reading non-recoverable/unreadable data-ledger under managed-ledger’s list.It helps when data-ledgers gets corrupted at bookkeeper and managed-cursor is stuck at that ledger. |false| +|loadBalancerEnabled| Enable load balancer |true| +|loadBalancerPlacementStrategy| Strategy to assign a new bundle weightedRandomSelection || +|loadBalancerReportUpdateThresholdPercentage| Percentage of change to trigger load report update |10| +|loadBalancerReportUpdateMaxIntervalMinutes| maximum interval to update load report |15| +|loadBalancerHostUsageCheckIntervalMinutes| Frequency of report to collect |1| +|loadBalancerSheddingIntervalMinutes| Load shedding interval. Broker periodically checks whether some traffic should be offload from some over-loaded broker to other under-loaded brokers |30| +|loadBalancerSheddingGracePeriodMinutes| Prevent the same topics to be shed and moved to other broker more than once within this timeframe |30| +|loadBalancerBrokerMaxTopics| Usage threshold to allocate max number of topics to broker |50000| +|loadBalancerBrokerUnderloadedThresholdPercentage| Usage threshold to determine a broker as under-loaded |1| +|loadBalancerBrokerOverloadedThresholdPercentage| Usage threshold to determine a broker as over-loaded |85| +|loadBalancerResourceQuotaUpdateIntervalMinutes| Interval to update namespace bundle resource quota |15| +|loadBalancerBrokerComfortLoadLevelPercentage| Usage threshold to determine a broker is having just right level of load |65| +|loadBalancerAutoBundleSplitEnabled| enable/disable namespace bundle auto split |false| +|loadBalancerNamespaceBundleMaxTopics| maximum topics in a bundle, otherwise bundle split will be triggered |1000| +|loadBalancerNamespaceBundleMaxSessions| maximum sessions (producers + consumers) in a bundle, otherwise bundle split will be triggered |1000| +|loadBalancerNamespaceBundleMaxMsgRate| maximum msgRate (in + out) in a bundle, otherwise bundle split will be triggered |1000| +|loadBalancerNamespaceBundleMaxBandwidthMbytes| maximum bandwidth (in + out) in a bundle, otherwise bundle split will be triggered |100| +|loadBalancerNamespaceMaximumBundles| maximum number of bundles in a namespace |128| +|replicationMetricsEnabled| Enable replication metrics |true| +|replicationConnectionsPerBroker| Max number of connections to open for each broker in a remote cluster More connections host-to-host lead to better throughput over high-latency links. |16| +|replicationProducerQueueSize| Replicator producer queue size |1000| +|replicatorPrefix| Replicator prefix used for replicator producer name and cursor name pulsar.repl|| +|replicationTlsEnabled| Enable TLS when talking with other clusters to replicate messages |false| +|defaultRetentionTimeInMinutes| Default message retention time || +|defaultRetentionSizeInMB| Default retention size |0| +|keepAliveIntervalSeconds| How often to check whether the connections are still alive |30| +|brokerServicePurgeInactiveFrequencyInSeconds| How often broker checks for inactive topics to be deleted (topics with no subscriptions and no one connected) |60| +|loadManagerClassName| Name of load manager to use |org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl| +|managedLedgerOffloadDriver| Driver to use to offload old data to long term storage (Possible values: S3) || +|managedLedgerOffloadMaxThreads| Maximum number of thread pool threads for ledger offloading |2| +|s3ManagedLedgerOffloadRegion| For Amazon S3 ledger offload, AWS region || +|s3ManagedLedgerOffloadBucket| For Amazon S3 ledger offload, Bucket to place offloaded ledger into || +|s3ManagedLedgerOffloadServiceEndpoint| For Amazon S3 ledger offload, Alternative endpoint to connect to (useful for testing) || +|s3ManagedLedgerOffloadMaxBlockSizeInBytes| For Amazon S3 ledger offload, Max block size in bytes. (64MB by default, 5MB minimum) |67108864| +|s3ManagedLedgerOffloadReadBufferSizeInBytes| For Amazon S3 ledger offload, Read buffer size in bytes (1MB by default) |1048576| + + + + +## Client + +The [`pulsar-client`](reference-cli-tools.md#pulsar-client) CLI tool can be used to publish messages to Pulsar and consume messages from Pulsar topics. This tool can be used in lieu of a client library. + +|Name|Description|Default| +|---|---|---| +|webServiceUrl| The web URL for the cluster. |http://localhost:8080/| +|brokerServiceUrl| The Pulsar protocol URL for the cluster. |pulsar://localhost:6650/| +|authPlugin| The authentication plugin. || +|authParams| The authentication parameters for the cluster, as a comma-separated string. || +|useTls| Whether or not TLS authentication will be enforced in the cluster. |false| +|tlsAllowInsecureConnection||| +|tlsTrustCertsFilePath||| + + +## Service discovery + +|Name|Description|Default| +|---|---|---| +|zookeeperServers| Zookeeper quorum connection string (comma-separated) || +|configurationStoreServers| Configuration store connection string (as a comma-separated list) || +|zookeeperSessionTimeoutMs| ZooKeeper session timeout |30000| +|servicePort| Port to use to server binary-proto request |6650| +|servicePortTls| Port to use to server binary-proto-tls request |6651| +|webServicePort| Port that discovery service listen on |8080| +|webServicePortTls| Port to use to server HTTPS request |8443| +|bindOnLocalhost| Control whether to bind directly on localhost rather than on normal hostname |false| +|authenticationEnabled| Enable authentication |false| +|authenticationProviders| Authentication provider name list, which is comma separated list of class names (comma-separated) || +|authorizationEnabled| Enforce authorization |false| +|superUserRoles| Role names that are treated as “super-user”, meaning they will be able to do all admin operations and publish/consume from all topics (comma-separated) || +|tlsEnabled| Enable TLS |false| +|tlsCertificateFilePath| Path for the TLS certificate file || +|tlsKeyFilePath| Path for the TLS private key file || + + + +## Log4j + + +|Name|Default| +|---|---| +|pulsar.root.logger| WARN,CONSOLE| +|pulsar.log.dir| logs| +|pulsar.log.file| pulsar.log| +|log4j.rootLogger| ${pulsar.root.logger}| +|log4j.appender.CONSOLE| org.apache.log4j.ConsoleAppender| +|log4j.appender.CONSOLE.Threshold| DEBUG| +|log4j.appender.CONSOLE.layout| org.apache.log4j.PatternLayout| +|log4j.appender.CONSOLE.layout.ConversionPattern| %d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n| +|log4j.appender.ROLLINGFILE| org.apache.log4j.DailyRollingFileAppender| +|log4j.appender.ROLLINGFILE.Threshold| DEBUG| +|log4j.appender.ROLLINGFILE.File| ${pulsar.log.dir}/${pulsar.log.file}| +|log4j.appender.ROLLINGFILE.layout| org.apache.log4j.PatternLayout| +|log4j.appender.ROLLINGFILE.layout.ConversionPattern| %d{ISO8601} - %-5p [%t:%C{1}@%L] - %m%n| +|log4j.appender.TRACEFILE| org.apache.log4j.FileAppender| +|log4j.appender.TRACEFILE.Threshold| TRACE| +|log4j.appender.TRACEFILE.File| pulsar-trace.log| +|log4j.appender.TRACEFILE.layout| org.apache.log4j.PatternLayout| +|log4j.appender.TRACEFILE.layout.ConversionPattern| %d{ISO8601} - %-5p [%t:%C{1}@%L][%x] - %m%n| + + +## Log4j shell + +|Name|Default| +|---|---| +|bookkeeper.root.logger| ERROR,CONSOLE| +|log4j.rootLogger| ${bookkeeper.root.logger}| +|log4j.appender.CONSOLE| org.apache.log4j.ConsoleAppender| +|log4j.appender.CONSOLE.Threshold| DEBUG| +|log4j.appender.CONSOLE.layout| org.apache.log4j.PatternLayout| +|log4j.appender.CONSOLE.layout.ConversionPattern| %d{ABSOLUTE} %-5p %m%n| +|log4j.logger.org.apache.zookeeper| ERROR| +|log4j.logger.org.apache.bookkeeper| ERROR| +|log4j.logger.org.apache.bookkeeper.bookie.BookieShell| INFO| + + +## Standalone + +|Name|Description|Default| +|---|---|---| +|zookeeperServers| The quorum connection string for local ZooKeeper || +|configurationStoreServers| Configuration store connection string (as a comma-separated list) || +|brokerServicePort| The port on which the standalone broker listens for connections |6650| +|webServicePort| The port used by the standalone broker for HTTP requests |8080| +|bindAddress| The hostname or IP address on which the standalone service binds |0.0.0.0| +|advertisedAddress| The hostname or IP address that the standalone service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostName()` is used. || +|clusterName| The name of the cluster that this broker belongs to. |standalone| +|zooKeeperSessionTimeoutMillis| The ZooKeeper session timeout, in milliseconds. |30000| +|brokerShutdownTimeoutMs| The time to wait for graceful broker shutdown. After this time elapses, the process will be killed. |60000| +|backlogQuotaCheckEnabled| Enable the backlog quota check, which enforces a specified action when the quota is reached. |true| +|backlogQuotaCheckIntervalInSeconds| How often to check for topics that have reached the backlog quota. |60| +|backlogQuotaDefaultLimitGB| The default per-topic backlog quota limit. |10| +|ttlDurationDefaultInSeconds| The default Time to Live (TTL) for namespaces if the TTL is not configured at namespace policies. When the value is set to `0`, TTL is disabled. By default, TTL is disabled. |0| +|brokerDeleteInactiveTopicsEnabled| Enable the deletion of inactive topics. If topics are not consumed for some while, these inactive topics might be cleaned up. Deleting inactive topics is enabled by default. The default period is 1 minute. |true| +|brokerDeleteInactiveTopicsFrequencySeconds| How often to check for inactive topics, in seconds. |60| +|messageExpiryCheckIntervalInMinutes| How often to proactively check and purged expired messages. |5| +|activeConsumerFailoverDelayTimeMillis| How long to delay rewinding cursor and dispatching messages when active consumer is changed. |1000| +|clientLibraryVersionCheckEnabled| Enable checks for minimum allowed client library version. |false| +|clientLibraryVersionCheckAllowUnversioned| Allow client libraries with no version information |true| +|statusFilePath| The path for the file used to determine the rotation status for the broker when responding to service discovery health checks |/usr/local/apache/htdocs| +|maxUnackedMessagesPerConsumer| The maximum number of unacknowledged messages allowed to be received by consumers on a shared subscription. The broker will stop sending messages to a consumer once this limit is reached or until the consumer begins acknowledging messages. A value of 0 disables the unacked message limit check and thus allows consumers to receive messages without any restrictions. |50000| +|maxUnackedMessagesPerSubscription| The same as above, except per subscription rather than per consumer. |200000| +|authenticationEnabled| Enable authentication for the broker. |false| +|authenticationProviders| A comma-separated list of class names for authentication providers. |false| +|authorizationEnabled| Enforce authorization in brokers. |false| +|superUserRoles| Role names that are treated as “superusers.” Superusers are authorized to perform all admin tasks. || +|brokerClientAuthenticationPlugin| The authentication settings of the broker itself. Used when the broker connects to other brokers either in the same cluster or from other clusters. || +|brokerClientAuthenticationParameters| The parameters that go along with the plugin specified using brokerClientAuthenticationPlugin. || +|athenzDomainNames| Supported Athenz authentication provider domain names as a comma-separated list. || +|bookkeeperClientAuthenticationPlugin| Authentication plugin to be used when connecting to bookies (BookKeeper servers). || +|bookkeeperClientAuthenticationParametersName| BookKeeper authentication plugin implementation parameters and values. || +|bookkeeperClientAuthenticationParameters| Parameters associated with the bookkeeperClientAuthenticationParametersName || +|bookkeeperClientTimeoutInSeconds| Timeout for BookKeeper add and read operations. |30| +|bookkeeperClientSpeculativeReadTimeoutInMillis| Speculative reads are initiated if a read request doesn’t complete within a certain time. A value of 0 disables speculative reads. |0| +|bookkeeperClientHealthCheckEnabled| Enable bookie health checks. |true| +|bookkeeperClientHealthCheckIntervalSeconds| The time interval, in seconds, at which health checks are performed. New ledgers are not created during health checks. |60| +|bookkeeperClientHealthCheckErrorThresholdPerInterval| Error threshold for health checks. |5| +|bookkeeperClientHealthCheckQuarantineTimeInSeconds| If bookies have more than the allowed number of failures within the time interval specified by bookkeeperClientHealthCheckIntervalSeconds |1800| +|bookkeeperClientRackawarePolicyEnabled| |true| +|bookkeeperClientRegionawarePolicyEnabled| |false| +|bookkeeperClientReorderReadSequenceEnabled| |false| +|bookkeeperClientIsolationGroups||| +|managedLedgerDefaultEnsembleSize| |1| +|managedLedgerDefaultWriteQuorum| |1| +|managedLedgerDefaultAckQuorum| |1| +|managedLedgerCacheSizeMB| |1024| +|managedLedgerCacheEvictionWatermark| |0.9| +|managedLedgerDefaultMarkDeleteRateLimit| |0.1| +|managedLedgerMaxEntriesPerLedger| |50000| +|managedLedgerMinLedgerRolloverTimeMinutes| |10| +|managedLedgerMaxLedgerRolloverTimeMinutes| |240| +|managedLedgerCursorMaxEntriesPerLedger| |50000| +|managedLedgerCursorRolloverTimeInSeconds| |14400| +|autoSkipNonRecoverableData| |false| +|loadBalancerEnabled| |false| +|loadBalancerPlacementStrategy| |weightedRandomSelection| +|loadBalancerReportUpdateThresholdPercentage| |10| +|loadBalancerReportUpdateMaxIntervalMinutes| |15| +|loadBalancerHostUsageCheckIntervalMinutes| |1| +|loadBalancerSheddingIntervalMinutes| |30| +|loadBalancerSheddingGracePeriodMinutes| |30| +|loadBalancerBrokerMaxTopics| |50000| +|loadBalancerBrokerUnderloadedThresholdPercentage| |1| +|loadBalancerBrokerOverloadedThresholdPercentage| |85| +|loadBalancerResourceQuotaUpdateIntervalMinutes| |15| +|loadBalancerBrokerComfortLoadLevelPercentage| |65| +|loadBalancerAutoBundleSplitEnabled| |false| +|loadBalancerNamespaceBundleMaxTopics| |1000| +|loadBalancerNamespaceBundleMaxSessions| |1000| +|loadBalancerNamespaceBundleMaxMsgRate| |1000| +|loadBalancerNamespaceBundleMaxBandwidthMbytes| |100| +|loadBalancerNamespaceMaximumBundles| |128| +|replicationMetricsEnabled| |true| +|replicationConnectionsPerBroker| |16| +|replicationProducerQueueSize| |1000| +|defaultRetentionTimeInMinutes| |0| +|defaultRetentionSizeInMB| |0| +|keepAliveIntervalSeconds| |30| +|brokerServicePurgeInactiveFrequencyInSeconds| |60| + + + + + +## WebSocket + +|Name|Description|Default| +|---|---|---| +|configurationStoreServers ||| +|zooKeeperSessionTimeoutMillis| |30000| +|serviceUrl||| +|serviceUrlTls||| +|brokerServiceUrl||| +|brokerServiceUrlTls||| +|webServicePort||8080| +|webServicePortTls||8443| +|bindAddress||0.0.0.0| +|clusterName ||| +|authenticationEnabled||false| +|authenticationProviders||| +|authorizationEnabled||false| +|superUserRoles ||| +|brokerClientAuthenticationPlugin||| +|brokerClientAuthenticationParameters||| +|tlsEnabled||false| +|tlsAllowInsecureConnection||false| +|tlsCertificateFilePath||| +|tlsKeyFilePath ||| +|tlsTrustCertsFilePath||| + + +## Pulsar proxy + +The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be configured in the `conf/proxy.conf` file. + + +|Name|Description|Default| +|---|---|---| +|zookeeperServers| The ZooKeeper quorum connection string (as a comma-separated list) || +|configurationStoreServers| Configuration store connection string (as a comma-separated list) || +|zookeeperSessionTimeoutMs| ZooKeeper session timeout (in milliseconds) |30000| +|servicePort| The port to use for server binary Protobuf requests |6650| +|servicePortTls| The port to use to server binary Protobuf TLS requests |6651| +|statusFilePath| Path for the file used to determine the rotation status for the proxy instance when responding to service discovery health checks || +|authenticationEnabled| Whether authentication is enabled for the Pulsar proxy |false| +|authenticationProviders| Authentication provider name list (a comma-separated list of class names) || +|authorizationEnabled| Whether authorization is enforced by the Pulsar proxy |false| +|authorizationProvider| Authorization provider as a fully qualified class name |org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider| +|brokerClientAuthenticationPlugin| The authentication plugin used by the Pulsar proxy to authenticate with Pulsar brokers || +|brokerClientAuthenticationParameters| The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers || +|brokerClientTrustCertsFilePath| The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers || +|superUserRoles| Role names that are treated as “super-users,” meaning that they will be able to perform all admin || +|forwardAuthorizationCredentials| Whether client authorization credentials are forwarded to the broker for re-authorization. Authentication must be enabled via authenticationEnabled=true for this to take effect. |false| +|maxConcurrentInboundConnections| Max concurrent inbound connections. The proxy will reject requests beyond that. |10000| +|maxConcurrentLookupRequests| Max concurrent outbound connections. The proxy will error out requests beyond that. |50000| +|tlsEnabledInProxy| Whether TLS is enabled for the proxy |false| +|tlsEnabledWithBroker| Whether TLS is enabled when communicating with Pulsar brokers |false| +|tlsCertificateFilePath| Path for the TLS certificate file || +|tlsKeyFilePath| Path for the TLS private key file || +|tlsTrustCertsFilePath| Path for the trusted TLS certificate pem file || +|tlsHostnameVerificationEnabled| Whether the hostname is validated when the proxy creates a TLS connection with brokers |false| +|tlsRequireTrustedClientCertOnConnect| Whether client certificates are required for TLS. Connections are rejected if the client certificate isn’t trusted. |false| +|tlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLSv1.2```, ```TLSv1.1```, ```TLSv1``` || +|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256```|| +|tokenSecretKey| Configure the secret key to be used to validate auth tokens. The key can be specified like: `tokenSecretKey=data:base64,xxxxxxxxx` or `tokenSecretKey=file:///my/secret.key`|| +|tokenPublicKey| Configure the public key to be used to validate auth tokens. The key can be specified like: `tokenPublicKey=data:base64,xxxxxxxxx` or `tokenPublicKey=file:///my/secret.key`|| +|tokenAuthClaim| Specify the token claim that will be used as the authentication "principal" or "role". The "subject" field will be used if this is left blank || + +## ZooKeeper + +ZooKeeper handles a broad range of essential configuration- and coordination-related tasks for Pulsar. The default configuration file for ZooKeeper is in the `conf/zookeeper.conf` file in your Pulsar installation. The following parameters are available: + + +|Name|Description|Default| +|---|---|---| +|tickTime| The tick is the basic unit of time in ZooKeeper, measured in milliseconds and used to regulate things like heartbeats and timeouts. tickTime is the length of a single tick. |2000| +|initLimit| The maximum time, in ticks, that the leader ZooKeeper server allows follower ZooKeeper servers to successfully connect and sync. The tick time is set in milliseconds using the tickTime parameter. |10| +|syncLimit| The maximum time, in ticks, that a follower ZooKeeper server is allowed to sync with other ZooKeeper servers. The tick time is set in milliseconds using the tickTime parameter. |5| +|dataDir| The location where ZooKeeper will store in-memory database snapshots as well as the transaction log of updates to the database. |data/zookeeper| +|clientPort| The port on which the ZooKeeper server will listen for connections. |2181| +|autopurge.snapRetainCount| In ZooKeeper, auto purge determines how many recent snapshots of the database stored in dataDir to retain within the time interval specified by autopurge.purgeInterval (while deleting the rest). |3| +|autopurge.purgeInterval| The time interval, in hours, by which the ZooKeeper database purge task is triggered. Setting to a non-zero number will enable auto purge; setting to 0 will disable. Read this guide before enabling auto purge. |1| +|maxClientCnxns| The maximum number of client connections. Increase this if you need to handle more ZooKeeper clients. |60| + + + + +In addition to the parameters in the table above, configuring ZooKeeper for Pulsar involves adding +a `server.N` line to the `conf/zookeeper.conf` file for each node in the ZooKeeper cluster, where `N` is the number of the ZooKeeper node. Here's an example for a three-node ZooKeeper cluster: + +```properties + +server.1=zk1.us-west.example.com:2888:3888 +server.2=zk2.us-west.example.com:2888:3888 +server.3=zk3.us-west.example.com:2888:3888 + +``` + +> We strongly recommend consulting the [ZooKeeper Administrator's Guide](https://zookeeper.apache.org/doc/current/zookeeperAdmin.html) for a more thorough and comprehensive introduction to ZooKeeper configuration diff --git a/site2/website-next/versioned_docs/version-2.3.1/reference-pulsar-admin.md b/site2/website-next/versioned_docs/version-2.3.1/reference-pulsar-admin.md new file mode 100644 index 0000000000000..8a7ff0ef335fb --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/reference-pulsar-admin.md @@ -0,0 +1,2814 @@ +--- +id: pulsar-admin +title: Pulsar admin CLI +sidebar_label: "Pulsar Admin CLI" +original_id: pulsar-admin +--- + +The `pulsar-admin` tool enables you to manage Pulsar installations, including clusters, brokers, namespaces, tenants, and more. + +Usage + +```bash + +$ pulsar-admin command + +``` + +Commands +* `broker-stats` +* `brokers` +* `clusters` +* `functions` +* `namespaces` +* `ns-isolation-policy` +* `sink` +* `source` +* `topics` +* `tenants` +* `resource-quotas` +* `schemas` + +## `broker-stats` + +Operations to collect broker statistics + +```bash + +$ pulsar-admin broker-stats subcommand + +``` + +Subcommands +* `allocator-stats` +* `topics(destinations)` +* `mbeans` +* `monitoring-metrics` +* `load-report` + + +### `allocator-stats` + +Dump allocator stats + +Usage + +```bash + +$ pulsar-admin broker-stats allocator-stats allocator-name + +``` + +### `topics(destinations)` + +Dump topic stats + +Usage + +```bash + +$ pulsar-admin broker-stats topics options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-i`, `--indent`|Indent JSON output|false| + +### `mbeans` + +Dump Mbean stats + +Usage + +```bash + +$ pulsar-admin broker-stats mbeans options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-i`, `--indent`|Indent JSON output|false| + + +### `monitoring-metrics` + +Dump metrics for monitoring + +Usage + +```bash + +$ pulsar-admin broker-stats monitoring-metrics options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-i`, `--indent`|Indent JSON output|false| + + +### `load-report` + +Dump broker load-report + +Usage + +```bash + +$ pulsar-admin broker-stats load-report + +``` + +## `brokers` + +Operations about brokers + +```bash + +$ pulsar-admin brokers subcommand + +``` + +Subcommands +* `list` +* `namespaces` +* `update-dynamic-config` +* `list-dynamic-config` +* `get-all-dynamic-config` +* `get-internal-config` +* `get-runtime-config` +* `healthcheck` + +### `list` +List active brokers of the cluster + +Usage + +```bash + +$ pulsar-admin brokers list cluster-name + +``` + +### `namespaces` +List namespaces owned by the broker + +Usage + +```bash + +$ pulsar-admin brokers namespaces cluster-name options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--url`|The URL for the broker|| + + +### `update-dynamic-config` +Update a broker's dynamic service configuration + +Usage + +```bash + +$ pulsar-admin brokers update-dynamic-config options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--config`|Service configuration parameter name|| +|`--value`|Value for the configuration parameter value specified using the `--config` flag|| + + +### `list-dynamic-config` +Get list of updatable configuration name + +Usage + +```bash + +$ pulsar-admin brokers list-dynamic-config + +``` + +### `get-all-dynamic-config` +Get all overridden dynamic-configuration values + +Usage + +```bash + +$ pulsar-admin brokers get-all-dynamic-config + +``` + +### `get-internal-config` +Get internal configuration information + +Usage + +```bash + +$ pulsar-admin brokers get-internal-config + +``` + +### `get-runtime-config` +Get runtime configuration values + +Usage + +```bash + +$ pulsar-admin brokers get-runtime-config + +``` + +### `healthcheck` +Run a health check against the broker + +Usage + +```bash + +$ pulsar-admin brokers healthcheck + +``` + +## `clusters` +Operations about clusters + +Usage + +```bash + +$ pulsar-admin clusters subcommand + +``` + +Subcommands +* `get` +* `create` +* `update` +* `delete` +* `list` +* `update-peer-clusters` +* `get-peer-clusters` +* `get-failure-domain` +* `create-failure-domain` +* `update-failure-domain` +* `delete-failure-domain` +* `list-failure-domains` + + +### `get` +Get the configuration data for the specified cluster + +Usage + +```bash + +$ pulsar-admin clusters get cluster-name + +``` + +### `create` +Provisions a new cluster. This operation requires Pulsar super-user privileges. + +Usage + +```bash + +$ pulsar-admin clusters create cluster-name options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--broker-url`|The URL for the broker service.|| +|`--broker-url-secure`|The broker service URL for a secure connection|| +|`--url`|service-url|| +|`--url-secure`|service-url for secure connection|| + + +### `update` +Update the configuration for a cluster + +Usage + +```bash + +$ pulsar-admin clusters update cluster-name options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--broker-url`|The URL for the broker service.|| +|`--broker-url-secure`|The broker service URL for a secure connection|| +|`--url`|service-url|| +|`--url-secure`|service-url for secure connection|| + + +### `delete` +Deletes an existing cluster + +Usage + +```bash + +$ pulsar-admin clusters delete cluster-name + +``` + +### `list` +List the existing clusters + +Usage + +```bash + +$ pulsar-admin clusters list + +``` + +### `update-peer-clusters` +Update peer cluster names + +Usage + +```bash + +$ pulsar-admin clusters update-peer-clusters cluster-name options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--peer-clusters`|Comma separated peer cluster names (Pass empty string "" to delete list)|| + +### `get-peer-clusters` +Get list of peer clusters + +Usage + +```bash + +$ pulsar-admin clusters get-peer-clusters + +``` + +### `get-failure-domain` +Get the configuration brokers of a failure domain + +Usage + +```bash + +$ pulsar-admin clusters get-failure-domain cluster-name options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--domain-name`|The failure domain name, which is a logical domain under a Pulsar cluster|| + +### `create-failure-domain` +Create a new failure domain for a cluster (updates it if already created) + +Usage + +```bash + +$ pulsar-admin clusters create-failure-domain cluster-name options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--broker-list`|Comma separated broker list|| +|`--domain-name`|The failure domain name, which is a logical domain under a Pulsar cluster|| + +### `update-failure-domain` +Update failure domain for a cluster (creates a new one if not exist) + +Usage + +```bash + +$ pulsar-admin clusters update-failure-domain cluster-name options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--broker-list`|Comma separated broker list|| +|`--domain-name`|The failure domain name, which is a logical domain under a Pulsar cluster|| + +### `delete-failure-domain` +Delete an existing failure domain + +Usage + +```bash + +$ pulsar-admin clusters delete-failure-domain cluster-name options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--domain-name`|The failure domain name, which is a logical domain under a Pulsar cluster|| + +### `list-failure-domains` +List the existing failure domains for a cluster + +Usage + +```bash + +$ pulsar-admin clusters list-failure-domains cluster-name + +``` + +## `functions` + +A command-line interface for Pulsar Functions + +Usage + +```bash + +$ pulsar-admin functions subcommand + +``` + +Subcommands +* `localrun` +* `create` +* `delete` +* `update` +* `get` +* `getstatus` +* `list` +* `querystate` +* `trigger` + + +### `localrun` +Run a Pulsar Function locally + + +Usage + +```bash + +$ pulsar-admin functions localrun options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--cpu`|The CPU to allocate to each function instance (in number of cores)|| +|`--ram`|The RAM to allocate to each function instance (in bytes)|| +|`--disk`|The disk space to allocate to each function instance (in bytes)|| +|`--auto-ack`|Let the functions framework manage acking|| +|`--subs-name`|Pulsar source subscription name if user wants a specific subscription-name for input-topic consumer|| +|`--broker-service-url `|The URL of the Pulsar broker|| +|`--classname`|The name of the function’s class|| +|`--custom-serde-inputs`|A map of the input topic to SerDe name|| +|`--custom-schema-inputs`|A map of the input topic to Schema class name|| +|`--client-auth-params`|Client Authentication Params|| +|`--function-config-file`|The path of the YAML config file used to configure the function|| +|`--hostname-verification-enabled`|Enable Hostname verification|| +|`--instance-id-offset`|Instance ids will be assigned starting from this offset|| +|`--inputs`|The input topics for the function (as a comma-separated list if more than one topic is desired)|| +|`--log-topic`|The topic to which logs from this function are published|| +|`--jar`|A path to the JAR file for the function (if the function is written in Java)|| +|`--name`|The name of the function|| +|`--namespace`|The function’s namespace|| +|`--output`|The name of the topic to which the function publishes its output (if any)|| +|`--output-serde-classname`|The SerDe class used for the function’s output|| +|`--parallelism`|The function’s parallelism factor, i.e. the number of instances of the function to run|1| +|`--processing-guarantees`|The processing guarantees applied to the function. Can be one of: ATLEAST_ONCE, ATMOST_ONCE, or EFFECTIVELY_ONCE|ATLEAST_ONCE| +|`--py`|The path of the Python file containing the function’s processing logic (if the function is written in Python)|| +|`--schema-type`|Schema Type to be used for storing output messages|| +|`--sliding-interval-count`|Number of messages after which the window ends|| +|`--sliding-interval-duration-ms`|The time duration after which the window slides|| +|`--state-storage-service-url`|The service URL for the function’s state storage (if the function uses a storage system different from the Apache BookKeeper cluster used by Pulsar). This service URL must be added manually when the Pulsar Function runs locally.|| +|`--tenant`|The function’s tenant|| +|`--topics-pattern`|The topic pattern to consume from list of topics under a namespace that match the pattern|| +|`--user-config`|A user-supplied config value, set as a key/value pair. You can set multiple user config values.|| +|`--window-length-count`|The number of messages per window.|| +|`--window-length-duration-ms`|The time duration of the window in milliseconds.|| + + +### `create` +Creates a new Pulsar Function on the target infrastructure + +Usage + +``` + +$ pulsar-admin functions create options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--cpu`|The CPU to allocate to each function instance (in number of cores)|| +|`--ram`|The RAM to allocate to each function instance (in bytes)|| +|`--disk`|The disk space to allocate to each function instance (in bytes)|| +|`--auto-ack`|Let the functions framework manage acking|| +|`--subs-name`|Pulsar source subscription name if user wants a specific subscription-name for input-topic consumer|| +|`--classname`|The name of the function’s class|| +|`--custom-serde-inputs`|A map of the input topic to SerDe name|| +|`--custom-schema-inputs`|A map of the input topic to Schema class name|| +|`--function-config-file`|The path of the YAML config file used to configure the function|| +|`--inputs`|The input topics for the function (as a comma-separated list if more than one topic is desired)|| +|`--log-topic`|The topic to which logs from this function are published|| +|`--jar`|A path to the JAR file for the function (if the function is written in Java)|| +|`--name`|The name of the function|| +|`--namespace`|The function’s namespace|| +|`--output`|The name of the topic to which the function publishes its output (if any)|| +|`--output-serde-classname`|The SerDe class used for the function’s output|| +|`--parallelism`|The function’s parallelism factor, i.e. the number of instances of the function to run|1| +|`--processing-guarantees`|The processing guarantees applied to the function. Can be one of: ATLEAST_ONCE, ATMOST_ONCE, or EFFECTIVELY_ONCE|ATLEAST_ONCE| +|`--py`|The path of the Python file containing the function’s processing logic (if the function is written in Python)|| +|`--schema-type`|Schema Type to be used for storing output messages|| +|`--sliding-interval-count`|Number of messages after which the window ends|| +|`--sliding-interval-duration-ms`|The time duration after which the window slides|| +|`--tenant`|The function’s tenant|| +|`--topics-pattern`|The topic pattern to consume from list of topics under a namespace that match the pattern|| +|`--user-config`|A user-supplied config value, set as a key/value pair. You can set multiple user config values.|| +|`--window-length-count`|The number of messages per window.|| +|`--window-length-duration-ms`|The time duration of the window in milliseconds.|| + + +### `delete` +Deletes an existing Pulsar Function + +Usage + +```bash + +$ pulsar-admin functions delete options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--name`|The name of the function to delete|| +|`--namespace`|The namespace of the function to delete|| +|`--tenant`|The tenant of the function to delete|| + + +### `update` +Updates an existing Pulsar Function + +Usage + +```bash + +$ pulsar-admin functions update options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--cpu`|The CPU to allocate to each function instance (in number of cores)|| +|`--ram`|The RAM to allocate to each function instance (in bytes)|| +|`--disk`|The disk space to allocate to each function instance (in bytes)|| +|`--auto-ack`|Let the functions framework manage acking|| +|`--subs-name`|Pulsar source subscription name if user wants a specific subscription-name for input-topic consumer|| +|`--classname`|The name of the function’s class|| +|`--custom-serde-inputs`|A map of the input topic to SerDe name|| +|`--custom-schema-inputs`|A map of the input topic to Schema class name|| +|`--function-config-file`|The path of the YAML config file used to configure the function|| +|`--inputs`|The input topics for the function (as a comma-separated list if more than one topic is desired)|| +|`--log-topic`|The topic to which logs from this function are published|| +|`--jar`|A path to the JAR file for the function (if the function is written in Java)|| +|`--name`|The name of the function|| +|`--namespace`|The function’s namespace|| +|`--output`|The name of the topic to which the function publishes its output (if any)|| +|`--output-serde-classname`|The SerDe class used for the function’s output|| +|`--parallelism`|The function’s parallelism factor, i.e. the number of instances of the function to run|1| +|`--processing-guarantees`|The processing guarantees applied to the function. Can be one of: ATLEAST_ONCE, ATMOST_ONCE, or EFFECTIVELY_ONCE|ATLEAST_ONCE| +|`--py`|The path of the Python file containing the function’s processing logic (if the function is written in Python)|| +|`--schema-type`|Schema Type to be used for storing output messages|| +|`--sliding-interval-count`|Number of messages after which the window ends|| +|`--sliding-interval-duration-ms`|The time duration after which the window slides|| +|`--tenant`|The function’s tenant|| +|`--topics-pattern`|The topic pattern to consume from list of topics under a namespace that match the pattern|| +|`--user-config`|A user-supplied config value, set as a key/value pair. You can set multiple user config values.|| +|`--window-length-count`|The number of messages per window.|| +|`--window-length-duration-ms`|The time duration of the window in milliseconds.|| + + +### `get` +Fetch information about an existing Pulsar Function + +Usage + +```bash + +$ pulsar-admin functions get options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--name`|The name of the function|| +|`--namespace`|The namespace of the function|| +|`--tenant`|The tenant of the function|| + + +### `restart` +Restarts either all instances or one particular instance of a function + +Usage + +```bash + +$ pulsar-admin functions restart options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--name`|The name of the function|| +|`--namespace`|The namespace of the function|| +|`--tenant`|The tenant of the function|| +|`--instance-id`|The function instanceId; restart all instances if instance-id is not provided|| + + +### `stop` +Temporary stops function instance. (If worker restarts then it reassigns and starts functiona again) + +Usage + +```bash + +$ pulsar-admin functions stop options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--name`|The name of the function|| +|`--namespace`|The namespace of the function|| +|`--tenant`|The tenant of the function|| +|`--instance-id`|The function instanceId; stop all instances if instance-id is not provided|| + + +### `getstatus` +Get the status of an existing Pulsar Function + +Usage + +```bash + +$ pulsar-admin functions getstatus options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--name`|The name of the function|| +|`--namespace`|The namespace of the function|| +|`--tenant`|The tenant of the function|| +|`--instance-id`|The function instanceId; get status of all instances if instance-id is not provided|| + +### `list` +List all Pulsar Functions for a specific tenant and namespace + +Usage + +```bash + +$ pulsar-admin functions list options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--namespace`|The namespace of the function|| +|`--tenant`|The tenant of the function|| + + +### `querystate` +Retrieve the current state of a Pulsar Function by key + +Usage + +```bash + +$ pulsar-admin functions querystate options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-k`, `--key`|The key for the state you want to fetch|| +|`--name`|The name of the function whose state you want to query|| +|`--namespace`|The namespace of the function whose state you want to query|| +|`--tenant`|The tenant of the function whose state you want to query|| +|`-u`, `--storage-service-url`|The service URL for the function’s state storage (if the function uses a storage system different from the Apache BookKeeper cluster used by Pulsar)|| +|`-w`, `--watch`|If set, watching for state changes is enabled|false| + + +### `trigger` +Triggers the specified Pulsar Function with a supplied value or file data + +Usage + +```bash + +$ pulsar-admin functions trigger options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--name`|The name of the Pulsar Function to trigger|| +|`--namespace`|The namespace of the Pulsar Function to trigger|| +|`--tenant`|The tenant of the Pulsar Function to trigger|| +|`--trigger-file`|The path to the file containing the data with which the Pulsar Function is to be triggered|| +|`--trigger-value`|The value with which the Pulsar Function is to be triggered|| + + +## `namespaces` + +Operations for managing namespaces + +```bash + +$ pulsar-admin namespaces subcommand + +``` + +Subcommands +* `list` +* `topics` +* `policies` +* `create` +* `delete` +* `set-deduplication` +* `permissions` +* `grant-permission` +* `revoke-permission` +* `grant-subscription-permission` +* `revoke-subscription-permission` +* `set-clusters` +* `get-clusters` +* `get-backlog-quotas` +* `set-backlog-quota` +* `remove-backlog-quota` +* `get-persistence` +* `set-persistence` +* `get-message-ttl` +* `set-message-ttl` +* `get-anti-affinity-group` +* `set-anti-affinity-group` +* `get-anti-affinity-namespaces` +* `delete-anti-affinity-group` +* `get-retention` +* `set-retention` +* `unload` +* `split-bundle` +* `set-dispatch-rate` +* `get-dispatch-rate` +* `set-subscribe-rate` +* `get-subscribe-rate` +* `set-subscription-dispatch-rate` +* `get-subscription-dispatch-rate` +* `clear-backlog` +* `unsubscribe` +* `set-encryption-required` +* `set-subscription-auth-mode` +* `get-max-producers-per-topic` +* `set-max-producers-per-topic` +* `get-max-consumers-per-topic` +* `set-max-consumers-per-topic` +* `get-max-consumers-per-subscription` +* `set-max-consumers-per-subscription` +* `get-compaction-threshold` +* `set-compaction-threshold` +* `get-offload-threshold` +* `set-offload-threshold` +* `get-offload-deletion-lag` +* `set-offload-deletion-lag` +* `clear-offload-deletion-lag` +* `get-schema-autoupdate-strategy` +* `set-schema-autoupdate-strategy` + + +### `list` +Get the namespaces for a tenant + +Usage + +```bash + +$ pulsar-admin namespaces list tenant-name + +``` + +### `topics` +Get the list of topics for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces topics tenant/namespace + +``` + +### `policies` +Get the configuration policies of a namespace + +Usage + +```bash + +$ pulsar-admin namespaces policies tenant/namespace + +``` + +### `create` +Create a new namespace + +Usage + +```bash + +$ pulsar-admin namespaces create tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-b`, `--bundles`|The number of bundles to activate|0| +|`-c`, `--clusters`|List of clusters this namespace will be assigned|| + + +### `delete` +Deletes a namespace. The namespace needs to be empty + +Usage + +```bash + +$ pulsar-admin namespaces delete tenant/namespace + +``` + +### `set-deduplication` +Enable or disable message deduplication on a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-deduplication tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--enable`, `-e`|Enable message deduplication on the specified namespace|false| +|`--disable`, `-d`|Disable message deduplication on the specified namespace|false| + + +### `permissions` +Get the permissions on a namespace + +Usage + +```bash + +$ pulsar-admin namespaces permissions tenant/namespace + +``` + +### `grant-permission` +Grant permissions on a namespace + +Usage + +```bash + +$ pulsar-admin namespaces grant-permission tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--actions`|Actions to be granted (`produce` or `consume`)|| +|`--role`|The client role to which to grant the permissions|| + + +### `revoke-permission` +Revoke permissions on a namespace + +Usage + +```bash + +$ pulsar-admin namespaces revoke-permission tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--role`|The client role to which to revoke the permissions|| + +### `grant-subscription-permission` +Grant permissions to access subscription admin-api + +Usage + +```bash + +$ pulsar-admin namespaces grant-subscription-permission tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--roles`|The client roles to which to grant the permissions (comma separated roles)|| +|`--subscription`|The subscription name for which permission will be granted to roles|| + +### `revoke-subscription-permission` +Revoke permissions to access subscription admin-api + +Usage + +```bash + +$ pulsar-admin namespaces revoke-subscription-permission tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--role`|The client role to which to revoke the permissions|| +|`--subscription`|The subscription name for which permission will be revoked to roles|| + +### `set-clusters` +Set replication clusters for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-clusters tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-c`, `--clusters`|Replication clusters ID list (comma-separated values)|| + + +### `get-clusters` +Get replication clusters for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-clusters tenant/namespace + +``` + +### `get-backlog-quotas` +Get the backlog quota policies for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-backlog-quotas tenant/namespace + +``` + +### `set-backlog-quota` +Set a backlog quota policy for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-backlog-quota tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-l`, `--limit`|The backlog size limit (for example `10M` or `16G`)|| +|`-p`, `--policy`|The retention policy to enforce when the limit is reached. The valid options are: `producer_request_hold`, `producer_exception` or `consumer_backlog_eviction`| + +Example + +```bash + +$ pulsar-admin namespaces set-backlog-quota my-tenant/my-ns \ +--limit 2G \ +--policy producer_request_hold + +``` + +### `remove-backlog-quota` +Remove a backlog quota policy from a namespace + +Usage + +```bash + +$ pulsar-admin namespaces remove-backlog-quota tenant/namespace + +``` + +### `get-persistence` +Get the persistence policies for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-persistence tenant/namespace + +``` + +### `set-persistence` +Set the persistence policies for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-persistence tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-a`, `--bookkeeper-ack-quorum`|The number of acks (guaranteed copies) to wait for each entry|0| +|`-e`, `--bookkeeper-ensemble`|The number of bookies to use for a topic|0| +|`-w`, `--bookkeeper-write-quorum`|How many writes to make of each entry|0| +|`-r`, `--ml-mark-delete-max-rate`|Throttling rate of mark-delete operation (0 means no throttle)|| + + +### `get-message-ttl` +Get the message TTL for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-message-ttl tenant/namespace + +``` + +### `set-message-ttl` +Set the message TTL for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-message-ttl tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-ttl`, `--messageTTL`|Message TTL in seconds. When the value is set to `0`, TTL is disabled. TTL is disabled by default.|0| + +### `get-anti-affinity-group` +Get Anti-affinity group name for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-anti-affinity-group tenant/namespace + +``` + +### `set-anti-affinity-group` +Set Anti-affinity group name for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-anti-affinity-group tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-g`, `--group`|Anti-affinity group name|| + +### `get-anti-affinity-namespaces` +Get Anti-affinity namespaces grouped with the given anti-affinity group name + +Usage + +```bash + +$ pulsar-admin namespaces get-anti-affinity-namespaces options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-c`, `--cluster`|Cluster name|| +|`-g`, `--group`|Anti-affinity group name|| +|`-p`, `--tenant`|Tenant is only used for authorization. Client has to be admin of any of the tenant to access this api|| + +### `delete-anti-affinity-group` +Remove Anti-affinity group name for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces delete-anti-affinity-group tenant/namespace + +``` + +### `get-retention` +Get the retention policy for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-retention tenant/namespace + +``` + +### `set-retention` +Set the retention policy for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-retention tenant/namespace + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-s`, `--size`|The retention size limits (for example 10M, 16G or 3T). 0 means no retention and -1 means infinite size retention|| +|`-t`, `--time`|The retention time in minutes, hours, days, or weeks. Examples: 100m, 13h, 2d, 5w. 0 means no retention and -1 means infinite time retention|| + + +### `unload` +Unload a namespace or namespace bundle from the current serving broker. + +Usage + +```bash + +$ pulsar-admin namespaces unload tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-b`, `--bundle`|{start-boundary}_{end-boundary} (e.g. 0x00000000_0xffffffff)|| + +### `split-bundle` +Split a namespace-bundle from the current serving broker + +Usage + +```bash + +$ pulsar-admin namespaces split-bundle tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-b`, `--bundle`|{start-boundary}_{end-boundary} (e.g. 0x00000000_0xffffffff)|| +|`-u`, `--unload`|Unload newly split bundles after splitting old bundle|false| + +### `set-dispatch-rate` +Set message-dispatch-rate for all topics of the namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-dispatch-rate tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-bd`, `--byte-dispatch-rate`|The byte dispatch rate (default -1 will be overwrite if not passed)|-1| +|`-dt`, `--dispatch-rate-period`|The dispatch rate period in second type (default 1 second will be overwrite if not passed)|1| +|`-md`, `--msg-dispatch-rate`|The message dispatch rate (default -1 will be overwrite if not passed)|-1| + +### `get-dispatch-rate` +Get configured message-dispatch-rate for all topics of the namespace (Disabled if value < 0) + +Usage + +```bash + +$ pulsar-admin namespaces get-dispatch-rate tenant/namespace + +``` + +### `set-subscribe-rate` +Set subscribe-rate per consumer for all topics of the namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-subscribe-rate tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-sr`, `--subscribe-rate`|The subscribe rate (default -1 will be overwrite if not passed)|-1| +|`-st`, `--subscribe-rate-period`|The subscribe rate period in second type (default 30 second will be overwrite if not passed)|30| + +### `get-subscribe-rate` +Get configured subscribe-rate per consumer for all topics of the namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-subscribe-rate tenant/namespace + +``` + +### `set-subscription-dispatch-rate` +Set subscription message-dispatch-rate for all subscription of the namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-subscription-dispatch-rate tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-bd`, `--byte-dispatch-rate`|The byte dispatch rate (default -1 will be overwrite if not passed)|-1| +|`-dt`, `--dispatch-rate-period`|The dispatch rate period in second type (default 1 second will be overwrite if not passed)|1| +|`-md`, `--sub-msg-dispatch-rate`|The message dispatch rate (default -1 will be overwrite if not passed)|-1| + +### `get-subscription-dispatch-rate` +Get subscription configured message-dispatch-rate for all topics of the namespace (Disabled if value < 0) + +Usage + +```bash + +$ pulsar-admin namespaces get-subscription-dispatch-rate tenant/namespace + +``` + +### `clear-backlog` +Clear the backlog for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces clear-backlog tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-b`, `--bundle`|{start-boundary}_{end-boundary} (e.g. 0x00000000_0xffffffff)|| +|`-force`, `--force`|Whether to force a clear backlog without prompt|false| +|`-s`, `--sub`|The subscription name|| + + +### `unsubscribe` +Unsubscribe the given subscription on all destinations on a namespace + +Usage + +```bash + +$ pulsar-admin namespaces unsubscribe tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-b`, `--bundle`|{start-boundary}_{end-boundary} (e.g. 0x00000000_0xffffffff)|| +|`-s`, `--sub`|The subscription name|| + +### `set-encryption-required` +Enable or disable message encryption required for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-encryption-required tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-d`, `--disable`|Disable message encryption required|false| +|`-e`, `--enable`|Enable message encryption required|false| + +### `set-subscription-auth-mode` +Set subscription auth mode on a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-subscription-auth-mode tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-m`, `--subscription-auth-mode`|Subscription authorization mode for Pulsar policies. Valid options are: [None, Prefix]|| + +### `get-max-producers-per-topic` +Get maxProducersPerTopic for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-max-producers-per-topic tenant/namespace + +``` + +### `set-max-producers-per-topic` +Set maxProducersPerTopic for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-max-producers-per-topic tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-p`, `--max-producers-per-topic`|maxProducersPerTopic for a namespace|0| + +### `get-max-consumers-per-topic` +Get maxConsumersPerTopic for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-max-consumers-per-topic tenant/namespace + +``` + +### `set-max-consumers-per-topic` +Set maxConsumersPerTopic for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-max-consumers-per-topic tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-c`, `--max-consumers-per-topic`|maxConsumersPerTopic for a namespace|0| + +### `get-max-consumers-per-subscription` +Get maxConsumersPerSubscription for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-max-consumers-per-subscription tenant/namespace + +``` + +### `set-max-consumers-per-subscription` +Set maxConsumersPerSubscription for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-max-consumers-per-subscription tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-c`, `--max-consumers-per-subscription`|maxConsumersPerSubscription for a namespace|0| + + +### `get-compaction-threshold` +Get compactionThreshold for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-compaction-threshold tenant/namespace + +``` + +### `set-compaction-threshold` +Set compactionThreshold for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-compaction-threshold tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-t`, `--threshold`|Maximum number of bytes in a topic backlog before compaction is triggered (eg: 10M, 16G, 3T). 0 disables automatic compaction|0| + + +### `get-offload-threshold` +Get offloadThreshold for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-offload-threshold tenant/namespace + +``` + +### `set-offload-threshold` +Set offloadThreshold for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-offload-threshold tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-s`, `--size`|Maximum number of bytes stored in the pulsar cluster for a topic before data will start being automatically offloaded to longterm storage (eg: 10M, 16G, 3T, 100). Negative values disable automatic offload. 0 triggers offloading as soon as possible.|-1| + +### `get-offload-deletion-lag` +Get offloadDeletionLag, in minutes, for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-offload-deletion-lag tenant/namespace + +``` + +### `set-offload-deletion-lag` +Set offloadDeletionLag for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-offload-deletion-lag tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-l`, `--lag`|Duration to wait after offloading a ledger segment, before deleting the copy of that segment from cluster local storage. (eg: 10m, 5h, 3d, 2w).|-1| + +### `clear-offload-deletion-lag` +Clear offloadDeletionLag for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces clear-offload-deletion-lag tenant/namespace + +``` + +### `get-schema-autoupdate-strategy` +Get the schema auto-update strategy for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces get-schema-autoupdate-strategy tenant/namespace + +``` + +### `set-schema-autoupdate-strategy` +Set the schema auto-update strategy for a namespace + +Usage + +```bash + +$ pulsar-admin namespaces set-schema-autoupdate-strategy tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-c`, `--compatibility`|Compatibility level required for new schemas created via a Producer. Possible values (Full, Backward, Forward, None).|Full| +|`-d`, `--disabled`|Disable automatic schema updates.|false| + + +## `ns-isolation-policy` +Operations for managing namespace isolation policies. + +Usage + +```bash + +$ pulsar-admin ns-isolation-policy subcommand + +``` + +Subcommands +* `set` +* `get` +* `list` +* `delete` +* `brokers` +* `broker` + +### `set` +Create/update a namespace isolation policy for a cluster. This operation requires Pulsar superuser privileges. + +Usage + +```bash + +$ pulsar-admin ns-isolation-policy set cluster-name policy-name options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--auto-failover-policy-params`|Comma-separated name=value auto failover policy parameters|[]| +|`--auto-failover-policy-type`|Auto failover policy type name. Currently available options: min_available.|[]| +|`--namespaces`|Comma-separated namespaces regex list|[]| +|`--primary`|Comma-separated primary broker regex list|[]| +|`--secondary`|Comma-separated secondary broker regex list|[]| + + +### `get` +Get the namespace isolation policy of a cluster. This operation requires Pulsar superuser privileges. + +Usage + +```bash + +$ pulsar-admin ns-isolation-policy get cluster-name policy-name + +``` + +### `list` +List all namespace isolation policies of a cluster. This operation requires Pulsar superuser privileges. + +Usage + +```bash + +$ pulsar-admin ns-isolation-policy list cluster-name + +``` + +### `delete` +Delete namespace isolation policy of a cluster. This operation requires superuser privileges. + +Usage + +```bash + +$ pulsar-admin ns-isolation-policy delete + +``` + +### `brokers` +List all brokers with namespace-isolation policies attached to it. This operation requires Pulsar super-user privileges. + +Usage + +```bash + +$ pulsar-admin ns-isolation-policy brokers cluster-name + +``` + +### `broker` +Get broker with namespace-isolation policies attached to it. This operation requires Pulsar super-user privileges. + +Usage + +```bash + +$ pulsar-admin ns-isolation-policy broker cluster-name options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--broker`|Broker name to get namespace-isolation policies attached to it|| + + +## `sink` + +An interface for managing Pulsar IO sinks (egress data from Pulsar) + +Usage + +```bash + +$ pulsar-admin sink subcommand + +``` + +Subcommands +* `create` +* `update` +* `delete` +* `localrun` +* `available-sinks` + + +### `create` +Submit a Pulsar IO sink connector to run in a Pulsar cluster + +Usage + +```bash + +$ pulsar-admin sink create options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--classname`|The sink’s Java class name|| +|`--cpu`|The CPU (in cores) that needs to be allocated per sink instance (applicable only to the Docker runtime)|| +|`--custom-serde-inputs`|The map of input topics to SerDe class names (as a JSON string)|| +|`--custom-schema-inputs`|The map of input topics to Schema types or class names (as a JSON string)|| +|`--disk`|The disk (in bytes) that needs to be allocated per sink instance (applicable only to the Docker runtime)|| +|`--inputs`|The sink’s input topic(s) (multiple topics can be specified as a comma-separated list)|| +|`--archive`|Path to the archive file for the sink|| +|`--name`|The sink’s name|| +|`--namespace`|The sink’s namespace|| +|`--parallelism`|“The sink’s parallelism factor (i.e. the number of sink instances to run).”|| +|`--processing-guarantees`|“The processing guarantees (aka delivery semantics) applied to the sink. Available values: ATLEAST_ONCE, ATMOST_ONCE, EFFECTIVELY_ONCE.”|| +|`--ram`|The RAM (in bytes) that needs to be allocated per sink instance (applicable only to the Docker runtime)|| +|`--sink-config`|Sink config key/values|| +|`--sink-config-file`|The path to a YAML config file specifying the sink’s configuration|| +|`--sink-type`|The built-in sinks's connector provider. The `sink-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file.|| +|`--topics-pattern`|TopicsPattern to consume from list of topics under a namespace that match the pattern.|| +|`--tenant`|The sink’s tenant|| +|`--auto-ack`|Let the functions framework manage acking|| +|`--timeout-ms`|The message timeout in milliseconds|| + + +### `update` +Submit a Pulsar IO sink connector to run in a Pulsar cluster + +Usage + +```bash + +$ pulsar-admin sink update options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--classname`|The sink’s Java class name|| +|`--cpu`|The CPU (in cores) that needs to be allocated per sink instance (applicable only to the Docker runtime)|| +|`--custom-serde-inputs`|The map of input topics to SerDe class names (as a JSON string)|| +|`--custom-schema-inputs`|The map of input topics to Schema types or class names (as a JSON string)|| +|`--disk`|The disk (in bytes) that needs to be allocated per sink instance (applicable only to the Docker runtime)|| +|`--inputs`|The sink’s input topic(s) (multiple topics can be specified as a comma-separated list)|| +|`--archive`|Path to the archive file for the sink|| +|`--name`|The sink’s name|| +|`--namespace`|The sink’s namespace|| +|`--parallelism`|“The sink’s parallelism factor (i.e. the number of sink instances to run).”|| +|`--processing-guarantees`|“The processing guarantees (aka delivery semantics) applied to the sink. Available values: ATLEAST_ONCE, ATMOST_ONCE, EFFECTIVELY_ONCE.”|| +|`--ram`|The RAM (in bytes) that needs to be allocated per sink instance (applicable only to the Docker runtime)|| +|`--sink-config`|Sink config key/values|| +|`--sink-config-file`|The path to a YAML config file specifying the sink’s configuration|| +|`--sink-type`|The built-in sinks's connector provider. The `sink-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file.|| +|`--topics-pattern`|TopicsPattern to consume from list of topics under a namespace that match the pattern.|| +|`--tenant`|The sink’s tenant|| + + +### `delete` +Stops a Pulsar IO sink + +Usage + +```bash + +$ pulsar-admin sink delete options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--name`|The name of the function to delete|| +|`--namespace`|The namespace of the function to delete|| +|`--tenant`|The tenant of the function to delete|| + + +### `localrun` +Run the Pulsar sink locally (rather than in the Pulsar cluster) + +Usage + +```bash + +$ pulsar-admin sink localrun options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--broker-service-url`|The URL for the Pulsar broker|| +|`--classname`|The sink’s Java class name|| +|`--cpu`|The CPU (in cores) that needs to be allocated per sink instance (applicable only to the Docker runtime)|| +|`--custom-serde-inputs`|The map of input topics to SerDe class names (as a JSON string)|| +|`--custom-schema-inputs`|The map of input topics to Schema types or class names (as a JSON string)|| +|`--disk`|The disk (in bytes) that needs to be allocated per sink instance (applicable only to the Docker runtime)|| +|`--inputs`|The sink’s input topic(s) (multiple topics can be specified as a comma-separated list)|| +|`--archive`|Path to the archive file for the sink|| +|`--name`|The sink’s name|| +|`--namespace`|The sink’s namespace|| +|`--parallelism`|“The sink’s parallelism factor (i.e. the number of sink instances to run).”|| +|`--processing-guarantees`|“The processing guarantees (aka delivery semantics) applied to the sink. Available values: ATLEAST_ONCE, ATMOST_ONCE, EFFECTIVELY_ONCE.”|| +|`--ram`|The RAM (in bytes) that needs to be allocated per sink instance (applicable only to the Docker runtime)|| +|`--sink-config`|Sink config key/values|| +|`--sink-config-file`|The path to a YAML config file specifying the sink’s configuration|| +|`--sink-type`|The built-in sinks's connector provider. The `sink-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file.|| +|`--topics-pattern`|TopicsPattern to consume from list of topics under a namespace that match the pattern.|| +|`--tenant`|The sink’s tenant|| +|`--auto-ack`|Let the functions framework manage acking|| +|`--timeout-ms`|The message timeout in milliseconds|| + + +### `available-sinks` +Get a list of all built-in sink connectors + +Usage + +```bash + +$ pulsar-admin sink available-sinks + +``` + +## `source` +An interface for managing Pulsar IO sources (ingress data into Pulsar) + +Usage + +```bash + +$ pulsar-admin source subcommand + +``` + +Subcommands +* `create` +* `update` +* `delete` +* `localrun` +* `available-sources` + + +### `create` +Submit a Pulsar IO source connector to run in a Pulsar cluster + +Usage + +```bash + +$ pulsar-admin source create options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--classname`|The source’s Java class name|| +|`--cpu`|The CPU (in cores) that needs to be allocated per source instance (applicable only to the Docker runtime)|| +|`--deserialization-classname`|The SerDe classname for the source|| +|`--destination-topic-name`|The Pulsar topic to which data is sent|| +|`--disk`|The disk (in bytes) that needs to be allocated per source instance (applicable only to the Docker runtime)|| +|`--archive`|The path to the NAR archive for the Source|| +|`--name`|The source’s name|| +|`--namespace`|The source’s namespace|| +|`--parallelism`|The source’s parallelism factor (i.e. the number of source instances to run).|| +|`--processing-guarantees`|“The processing guarantees (aka delivery semantics) applied to the source. Available values: ATLEAST_ONCE, ATMOST_ONCE, EFFECTIVELY_ONCE.”|| +|`--ram`|The RAM (in bytes) that needs to be allocated per source instance (applicable only to the Docker runtime)|| +|`--schema-type`|The schema type (either a builtin schema like 'avro', 'json', etc, or custom Schema class name to be used to encode messages emitted from the source|| +|`--source-type`|One of the built-in source's connector provider. The source's connector provider. The `source-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file.|| +|`--source-config`|Source config key/values|| +|`--source-config-file`|The path to a YAML config file specifying the source’s configuration|| +|`--tenant`|The source’s tenant|| + + +### `update` +Update a already submitted Pulsar IO source connector + +Usage + +```bash + +$ pulsar-admin source update options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--classname`|The source’s Java class name|| +|`--cpu`|The CPU (in cores) that needs to be allocated per source instance (applicable only to the Docker runtime)|| +|`--deserialization-classname`|The SerDe classname for the source|| +|`--destination-topic-name`|The Pulsar topic to which data is sent|| +|`--disk`|The disk (in bytes) that needs to be allocated per source instance (applicable only to the Docker runtime)|| +|`--archive`|The path to the NAR archive for the Source|| +|`--name`|The source’s name|| +|`--namespace`|The source’s namespace|| +|`--parallelism`|The source’s parallelism factor (i.e. the number of source instances to run).|| +|`--processing-guarantees`|“The processing guarantees (aka delivery semantics) applied to the source. Available values: ATLEAST_ONCE, ATMOST_ONCE, EFFECTIVELY_ONCE.”|| +|`--ram`|The RAM (in bytes) that needs to be allocated per source instance (applicable only to the Docker runtime)|| +|`--schema-type`|The schema type (either a builtin schema like 'avro', 'json', etc, or custom Schema class name to be used to encode messages emitted from the source|| +|`--source-type`|One of the built-in source's connector provider. The source's connector provider. The `source-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file.|| +|`--source-config`|Source config key/values|| +|`--source-config-file`|The path to a YAML config file specifying the source’s configuration|| +|`--tenant`|The source’s tenant|| + + +### `delete` +Stops a Pulsar IO source + +Usage + +```bash + +$ pulsar-admin source delete options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--name`|The name of the function to delete|| +|`--namespace`|The namespace of the function to delete|| +|`--tenant`|The tenant of the function to delete|| + + +### `localrun` +Run the Pulsar source locally (rather than in the Pulsar cluster) + +Usage + +```bash + +$ pulsar-admin source localrun options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--classname`|The source’s Java class name|| +|`--cpu`|The CPU (in cores) that needs to be allocated per source instance (applicable only to the Docker runtime)|| +|`--deserialization-classname`|The SerDe classname for the source|| +|`--destination-topic-name`|The Pulsar topic to which data is sent|| +|`--disk`|The disk (in bytes) that needs to be allocated per source instance (applicable only to the Docker runtime)|| +|`--archive`|The path to the NAR archive for the Source|| +|`--name`|The source’s name|| +|`--namespace`|The source’s namespace|| +|`--parallelism`|The source’s parallelism factor (i.e. the number of source instances to run).|| +|`--processing-guarantees`|“The processing guarantees (aka delivery semantics) applied to the source. Available values: ATLEAST_ONCE, ATMOST_ONCE, EFFECTIVELY_ONCE.”|| +|`--ram`|The RAM (in bytes) that needs to be allocated per source instance (applicable only to the Docker runtime)|| +|`--schema-type`|The schema type (either a builtin schema like 'avro', 'json', etc, or custom Schema class name to be used to encode messages emitted from the source|| +|`--source-type`|One of the built-in source's connector provider. The source's connector provider. The `source-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file.|| +|`--source-config`|Source config key/values|| +|`--source-config-file`|The path to a YAML config file specifying the source’s configuration|| +|`--tenant`|The source’s tenant|| + + +### `available-sources` +Get a list of all built-in source connectors + +Usage + +```bash + +$ pulsar-admin source available-sources + +``` + +## `topics` +Operations for managing Pulsar topics (both persistent and non persistent) + +Usage + +```bash + +$ pulsar-admin topics subcommand + +``` + +Subcommands +* `compact` +* `compaction-status` +* `offload` +* `offload-status` +* `create-partitioned-topic` +* `delete-partitioned-topic` +* `create` +* `get-partitioned-topic-metadata` +* `update-partitioned-topic` +* `list` +* `list-in-bundle` +* `terminate` +* `permissions` +* `grant-permission` +* `revoke-permission` +* `lookup` +* `bundle-range` +* `delete` +* `unload` +* `subscriptions` +* `unsubscribe` +* `stats` +* `stats-internal` +* `info-internal` +* `partitioned-stats` +* `skip` +* `skip-all` +* `expire-messages` +* `expire-messages-all-subscriptions` +* `peek-messages` +* `reset-cursor` + + +### `compact` +Run compaction on the specified topic (persistent topics only) + +Usage + +``` + +$ pulsar-admin topics compact persistent://tenant/namespace/topic + +``` + +### `compaction-status` +Check the status of a topic compaction (persistent topics only) + +Usage + +```bash + +$ pulsar-admin topics compaction-status persistent://tenant/namespace/topic + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-w`, `--wait-complete`|Wait for compaction to complete|false| + + +### `offload` +Trigger offload of data from a topic to long-term storage (e.g. Amazon S3) + +Usage + +```bash + +$ pulsar-admin topics offload persistent://tenant/namespace/topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-s`, `--size-threshold`|The maximum amount of data to keep in BookKeeper for the specific topic|| + + +### `offload-status` +Check the status of data offloading from a topic to long-term storage + +Usage + +```bash + +$ pulsar-admin topics offload-status persistent://tenant/namespace/topic op + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-w`, `--wait-complete`|Wait for compaction to complete|false| + + +### `create-partitioned-topic` +Create a partitioned topic. A partitioned topic must be created before producers can publish to it. + +Usage + +```bash + +$ pulsar-admin topics create-partitioned-topic {persistent|non-persistent}://tenant/namespace/topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-p`, `--partitions`|The number of partitions for the topic|0| + +### `delete-partitioned-topic` +Delete a partitioned topic. This will also delete all the partitions of the topic if they exist. + +Usage + +```bash + +$ pulsar-admin topics delete-partitioned-topic {persistent|non-persistent} + +``` + +### `create` +Creates a non-partitioned topic. A non-partitioned topic must explicitly be created by the user if allowAutoTopicCreation or createIfMissing is disabled. + +Usage + +```bash + +$ pulsar-admin topics create {persistent|non-persistent}://tenant/namespace/topic + +``` + +### `get-partitioned-topic-metadata` +Get the partitioned topic metadata. If the topic is not created or is a non-partitioned topic, this will return an empty topic with zero partitions. + +Usage + +```bash + +$ pulsar-admin topics get-partitioned-topic-metadata {persistent|non-persistent}://tenant/namespace/topic + +``` + +### `update-partitioned-topic` +Update existing non-global partitioned topic. New updating number of partitions must be greater than existing number of partitions. + +Usage + +```bash + +$ pulsar-admin topics update-partitioned-topic {persistent|non-persistent}://tenant/namespace/topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-p`, `--partitions`|The number of partitions for the topic|0| + +### `list` +Get the list of topics under a namespace + +Usage + +``` + +$ pulsar-admin topics list tenant/cluster/namespace + +``` + +### `list-in-bundle` +Get a list of non-persistent topics present under a namespace bundle + +Usage + +``` + +$ pulsar-admin topics list-in-bundle tenant/namespace options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-b`, `--bundle`|The bundle range|| + + +### `terminate` +Terminate a persistent topic (disallow further messages from being published on the topic) + +Usage + +```bash + +$ pulsar-admin topics terminate persistent://tenant/namespace/topic + +``` + +### `permissions` +Get the permissions on a topic. Retrieve the effective permissions for a destination. These permissions are defined by the permissions set at the namespace level combined (union) with any eventual specific permissions set on the topic. + +Usage + +```bash + +$ pulsar-admin topics permissions topic + +``` + +### `grant-permission` +Grant a new permission to a client role on a single topic + +Usage + +```bash + +$ pulsar-admin topics grant-permission {persistent|non-persistent}://tenant/namespace/topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--actions`|Actions to be granted (`produce` or `consume`)|| +|`--role`|The client role to which to grant the permissions|| + + +### `revoke-permission` +Revoke permissions to a client role on a single topic. If the permission was not set at the topic level, but rather at the namespace level, this operation will return an error (HTTP status code 412). + +Usage + +```bash + +$ pulsar-admin topics revoke-permission topic + +``` + +### `lookup` +Look up a topic from the current serving broker + +Usage + +```bash + +$ pulsar-admin topics lookup topic + +``` + +### `bundle-range` +Get the namespace bundle which contains the given topic + +Usage + +```bash + +$ pulsar-admin topics bundle-range topic + +``` + +### `delete` +Delete a topic. The topic cannot be deleted if there are any active subscriptions or producers connected to the topic. + +Usage + +```bash + +$ pulsar-admin topics delete topic + +``` + +### `unload` +Unload a topic + +Usage + +```bash + +$ pulsar-admin topics unload topic + +``` + +### `subscriptions` +Get the list of subscriptions on the topic + +Usage + +```bash + +$ pulsar-admin topics subscriptions topic + +``` + +### `unsubscribe` +Delete a durable subscriber from a topic + +Usage + +```bash + +$ pulsar-admin topics unsubscribe topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-s`, `--subscription`|The subscription to delete|| + + +### `stats` +Get the stats for the topic and its connected producers and consumers. All rates are computed over a 1-minute window and are relative to the last completed 1-minute period. + +Usage + +```bash + +$ pulsar-admin topics stats topic + +``` + +### `stats-internal` +Get the internal stats for the topic + +Usage + +```bash + +$ pulsar-admin topics stats-internal topic + +``` + +### `info-internal` +Get the internal metadata info for the topic + +Usage + +```bash + +$ pulsar-admin topics info-internal topic + +``` + +### `partitioned-stats` +Get the stats for the partitioned topic and its connected producers and consumers. All rates are computed over a 1-minute window and are relative to the last completed 1-minute period. + +Usage + +```bash + +$ pulsar-admin topics partitioned-stats topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`--per-partition`|Get per-partition stats|false| + + +### `skip` +Skip some messages for the subscription + +Usage + +```bash + +$ pulsar-admin topics skip topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-n`, `--count`|The number of messages to skip|0| +|`-s`, `--subscription`|The subscription on which to skip messages|| + + +### `skip-all` +Skip all the messages for the subscription + +Usage + +```bash + +$ pulsar-admin topics skip-all topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-s`, `--subscription`|The subscription to clear|| + + +### `expire-messages` +Expire messages that are older than the given expiry time (in seconds) for the subscription. + +Usage + +```bash + +$ pulsar-admin topics expire-messages topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-t`, `--expireTime`|Expire messages older than the time (in seconds)|0| +|`-s`, `--subscription`|The subscription to skip messages on|| + + +### `expire-messages-all-subscriptions` +Expire messages older than the given expiry time (in seconds) for all subscriptions + +Usage + +```bash + +$ pulsar-admin topics expire-messages-all-subscriptions topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-t`, `--expireTime`|Expire messages older than the time (in seconds)|0| + + +### `peek-messages` +Peek some messages for the subscription. + +Usage + +```bash + +$ pulsar-admin topics peek-messages topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-n`, `--count`|The number of messages|0| +|`-s`, `--subscription`|Subscription to get messages from|| + + +### `reset-cursor` +Reset position for subscription to closest to timestamp + +Usage + +```bash + +$ pulsar-admin topics reset-cursor topic options + +``` + +Options + +|Flag|Description|Default| +|---|---|---| +|`-s`, `--subscription`|Subscription to reset position on|| +|`-t`, `--time`|The time, in minutes, to reset back to (or minutes, hours, days, weeks, etc.). Examples: `100m`, `3h`, `2d`, `5w`.|| + + + +## `tenants` +Operations for managing tenants + +Usage + +```bash + +$ pulsar-admin tenants subcommand + +``` + +Subcommands +* `list` +* `get` +* `create` +* `update` +* `delete` + +### `list` +List the existing tenants + +Usage + +```bash + +$ pulsar-admin tenants list + +``` + +### `get` +Gets the configuration of a tenant + +Usage + +```bash + +$ pulsar-admin tenants get tenant-name + +``` + +### `create` +Creates a new tenant + +Usage + +```bash + +$ pulsar-admin tenants create tenant-name options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-r`, `--admin-roles`|Comma-separated admin roles|| +|`-c`, `--allowed-clusters`|Comma-separated allowed clusters|| + +### `update` +Updates a tenant + +Usage + +```bash + +$ pulsar-admin tenants update tenant-name options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-r`, `--admin-roles`|Comma-separated admin roles|| +|`-c`, `--allowed-clusters`|Comma-separated allowed clusters|| + + +### `delete` +Deletes an existing tenant + +Usage + +```bash + +$ pulsar-admin tenants delete tenant-name + +``` + +## `resource-quotas` +Operations for managing resource quotas + +Usage + +```bash + +$ pulsar-admin resource-quotas subcommand + +``` + +Subcommands +* `get` +* `set` +* `reset-namespace-bundle-quota` + + +### `get` +Get the resource quota for a specified namespace bundle, or default quota if no namespace/bundle is specified. + +Usage + +```bash + +$ pulsar-admin resource-quotas get options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-b`, `--bundle`|A bundle of the form {start-boundary}_{end_boundary}. This must be specified together with -n/--namespace.|| +|`-n`, `--namespace`|The namespace|| + + +### `set` +Set the resource quota for the specified namespace bundle, or default quota if no namespace/bundle is specified. + +Usage + +```bash + +$ pulsar-admin resource-quotas set options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-bi`, `--bandwidthIn`|The expected inbound bandwidth (in bytes/second)|0| +|`-bo`, `--bandwidthOut`|Expected outbound bandwidth (in bytes/second)0| +|`-b`, `--bundle`|A bundle of the form {start-boundary}_{end_boundary}. This must be specified together with -n/--namespace.|| +|`-d`, `--dynamic`|Allow to be dynamically re-calculated (or not)|false| +|`-mem`, `--memory`|Expectred memory usage (in megabytes)|0| +|`-mi`, `--msgRateIn`|Expected incoming messages per second|0| +|`-mo`, `--msgRateOut`|Expected outgoing messages per second|0| +|`-n`, `--namespace`|The namespace as tenant/namespace, for example my-tenant/my-ns. Must be specified together with -b/--bundle.|| + + +### `reset-namespace-bundle-quota` +Reset the specified namespace bundle's resource quota to a default value. + +Usage + +```bash + +$ pulsar-admin resource-quotas reset-namespace-bundle-quota options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-b`, `--bundle`|A bundle of the form {start-boundary}_{end_boundary}. This must be specified together with -n/--namespace.|| +|`-n`, `--namespace`|The namespace|| + + + +## `schemas` +Operations related to Schemas associated with Pulsar topics. + +Usage + +``` + +$ pulsar-admin schemas subcommand + +``` + +Subcommands +* `upload` +* `delete` +* `get` +* `extract` + + +### `upload` +Upload the schema definition for a topic + +Usage + +```bash + +$ pulsar-admin schemas upload persistent://tenant/namespace/topic options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--filename`|The path to the schema definition file. An example schema file is available under conf directory.|| + + +### `delete` +Delete the schema definition associated with a topic + +Usage + +```bash + +$ pulsar-admin schemas delete persistent://tenant/namespace/topic + +``` + +### `get` +Retrieve the schema definition associated with a topic (at a given version if version is supplied). + +Usage + +```bash + +$ pulsar-admin schemas get persistent://tenant/namespace/topic options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`--version`|The version of the schema definition to retrieve for a topic.|| + +### `extract` +Provide the schema definition for a topic via Java class name contained in a JAR file + +Usage + +```bash + +$ pulsar-admin schemas extract persistent://tenant/namespace/topic options + +``` + +Options + +|Flag|Description|Default| +|----|---|---| +|`-c`, `--classname`|The Java class name|| +|`-j`, `--jar`|A path to the JAR file which contains the above Java class|| +|`-t`, `--type`|The type of the schema (avro or json)|| + + diff --git a/site2/website-next/versioned_docs/version-2.3.1/reference-terminology.md b/site2/website-next/versioned_docs/version-2.3.1/reference-terminology.md new file mode 100644 index 0000000000000..8d9e9192381b0 --- /dev/null +++ b/site2/website-next/versioned_docs/version-2.3.1/reference-terminology.md @@ -0,0 +1,167 @@ +--- +id: reference-terminology +title: Pulsar Terminology +sidebar_label: "Terminology" +original_id: reference-terminology +--- + +Here is a glossary of terms related to Apache Pulsar: + +### Concepts + +#### Pulsar + +Pulsar is a distributed messaging system originally created by Yahoo but now under the stewardship of the Apache Software Foundation. + +#### Message + +Messages are the basic unit of Pulsar. They're what [producers](#producer) publish to [topics](#topic) +and what [consumers](#consumer) then consume from topics. + +#### Topic + +A named channel used to pass messages published by [producers](#producer) to [consumers](#consumer) who +process those [messages](#message). + +#### Partitioned Topic + +A topic that is served by multiple Pulsar [brokers](#broker), which enables higher throughput. + +#### Namespace + +A grouping mechanism for related [topics](#topic). + +#### Namespace Bundle + +A virtual group of [topics](#topic) that belong to the same [namespace](#namespace). A namespace bundle +is defined as a range between two 32-bit hashes, such as 0x00000000 and 0xffffffff. + +#### Tenant + +An administrative unit for allocating capacity and enforcing an authentication/authorization scheme. + +#### Subscription + +A lease on a [topic](#topic) established by a group of [consumers](#consumer). Pulsar has three subscription +modes (exclusive, shared, and failover). + +#### Pub-Sub + +A messaging pattern in which [producer](#producer) processes publish messages on [topics](#topic) that +are then consumed (processed) by [consumer](#consumer) processes. + +#### Producer + +A process that publishes [messages](#message) to a Pulsar [topic](#topic). + +#### Consumer + +A process that establishes a subscription to a Pulsar [topic](#topic) and processes messages published +to that topic by [producers](#producer). + +#### Reader + +Pulsar readers are message processors much like Pulsar [consumers](#consumer) but with two crucial differences: + +- you can specify *where* on a topic readers begin processing messages (consumers always begin with the latest + available unacked message); +- readers don't retain data or acknowledge messages. + +#### Cursor + +The subscription position for a [consumer](#consumer). + +#### Acknowledgment (ack) + +A message sent to a Pulsar broker by a [consumer](#consumer) that a message has been successfully processed. +An acknowledgement (ack) is Pulsar's way of knowing that the message can be deleted from the system; +if no acknowledgement, then the message will be retained until it's processed. + +#### Negative Acknowledgment (nack) + +When an application fails to process a particular message, it can send a "negative ack" to Pulsar +to signal that the message should be replayed at a later timer. (By default, failed messages are +replayed after a 1 minute delay) + +#### Unacknowledged + +A message that has been delivered to a consumer for processing but not yet confirmed as processed by the consumer. + +#### Retention Policy + +Size and/or time limits that you can set on a [namespace](#namespace) to configure retention of [messages](#message) +that have already been [acknowledged](#acknowledgement-ack). + +#### Multi-Tenancy + +The ability to isolate [namespaces](#namespace), specify quotas, and configure authentication and authorization +on a per-[tenant](#tenant) basis. + +### Architecture + +#### Standalone + +A lightweight Pulsar broker in which all components run in a single Java Virtual Machine (JVM) process. Standalone +clusters can be run on a single machine and are useful for development purposes. + +#### Cluster + +A set of Pulsar [brokers](#broker) and [BookKeeper](#bookkeeper) servers (aka [bookies](#bookie)). +Clusters can reside in different geographical regions and replicate messages to one another +in a process called [geo-replication](#geo-replication). + +#### Instance + +A group of Pulsar [clusters](#cluster) that act together as a single unit. + +#### Geo-Replication + +Replication of messages across Pulsar [clusters](#cluster), potentially in different datacenters +or geographical regions. + +#### Configuration Store + +Pulsar's configuration store (previously known as configuration store) is a ZooKeeper quorum that +is used for configuration-specific tasks. A multi-cluster Pulsar installation requires just one +configuration store across all [clusters](#cluster). + +#### Topic Lookup + +A service provided by Pulsar [brokers](#broker) that enables connecting clients to automatically determine +which Pulsar [cluster](#cluster) is responsible for a [topic](#topic) (and thus where message traffic for +the topic needs to be routed). + +#### Service Discovery + +A mechanism provided by Pulsar that enables connecting clients to use just a single URL to interact +with all the [brokers](#broker) in a [cluster](#cluster). + +#### Broker + +A stateless component of Pulsar [clusters](#cluster) that runs two other components: an HTTP server +exposing a REST interface for administration and topic lookup and a [dispatcher](#dispatcher) that +handles all message transfers. Pulsar clusters typically consist of multiple brokers. + +#### Dispatcher + +An asynchronous TCP server used for all data transfers in-and-out a Pulsar [broker](#broker). The Pulsar +dispatcher uses a custom binary protocol for all communications. + +### Storage + +#### BookKeeper + +[Apache BookKeeper](http://bookkeeper.apache.org/) is a scalable, low-latency persistent log storage +service that Pulsar uses to store data. + +#### Bookie + +Bookie is the name of an individual BookKeeper server. It is effectively the storage server of Pulsar. + +#### Ledger + +An append-only data structure in [BookKeeper](#bookkeeper) that is used to persistently store messages in Pulsar [topics](#topic). + +### Functions + +Pulsar Functions are lightweight functions that can consume messages from Pulsar topics, apply custom processing logic, and, if desired, publish results to topics. diff --git a/site2/website-next/versioned_sidebars/version-2.3.1-sidebars.json b/site2/website-next/versioned_sidebars/version-2.3.1-sidebars.json index e094d7a94f410..a0983c900a360 100644 --- a/site2/website-next/versioned_sidebars/version-2.3.1-sidebars.json +++ b/site2/website-next/versioned_sidebars/version-2.3.1-sidebars.json @@ -387,6 +387,54 @@ "id": "version-2.3.1/cookbooks-bookkeepermetadata" } ] + }, + { + "type": "category", + "label": "Development", + "items": [ + { + "type": "doc", + "id": "version-2.3.1/develop-tools" + }, + { + "type": "doc", + "id": "version-2.3.1/develop-binary-protocol" + }, + { + "type": "doc", + "id": "version-2.3.1/develop-schema" + }, + { + "type": "doc", + "id": "version-2.3.1/develop-load-manager" + }, + { + "type": "doc", + "id": "version-2.3.1/develop-cpp" + } + ] + }, + { + "type": "category", + "label": "Reference", + "items": [ + { + "type": "doc", + "id": "version-2.3.1/reference-terminology" + }, + { + "type": "doc", + "id": "version-2.3.1/reference-cli-tools" + }, + { + "type": "doc", + "id": "version-2.3.1/pulsar-admin" + }, + { + "type": "doc", + "id": "version-2.3.1/reference-configuration" + } + ] } ] } \ No newline at end of file