This is an automated email from the ASF dual-hosted git repository.

penghui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/master by this push:
     new 4a3936e  [Doc] Add guides for Transactions (#11235)
4a3936e is described below

commit 4a3936e30d72adef0b479a65785626ee43ed0559
Author: Anonymitaet <[email protected]>
AuthorDate: Thu Jul 8 09:05:42 2021 +0800

    [Doc] Add guides for Transactions (#11235)
    
    Pulsar 2.8 has been released and one of the highlights is transaction API, 
which supports atomic writes and acknowledgments across multiple topics. It 
completes our vision of making Pulsar a complete unified messaging and 
streaming platform.
    
    While the previous transaction docs are incomplete and need to be improved, 
so I've re-architectured it and added missing contents.
    
    This PR fixes https://github.com/apache/pulsar/issues/10971 and it is based 
on the [google 
doc](https://docs.google.com/document/d/1rChR2ZScYxCDKUmkQTg6inbuhyfrz_jJfdQOFtwO8P0/edit#)
 I collaborated with @congbobo184 and @D-2-Ed, so it should be pretty good 
already.
---
 site2/docs/assets/txn-1.png | Bin 0 -> 146508 bytes
 site2/docs/assets/txn-2.png | Bin 0 -> 57117 bytes
 site2/docs/assets/txn-3.png | Bin 0 -> 249164 bytes
 site2/docs/assets/txn-4.png | Bin 0 -> 197983 bytes
 site2/docs/assets/txn-5.png | Bin 0 -> 202533 bytes
 site2/docs/assets/txn-6.png | Bin 0 -> 188856 bytes
 site2/docs/assets/txn-7.png | Bin 0 -> 216642 bytes
 site2/docs/assets/txn-8.png | Bin 0 -> 190633 bytes
 site2/docs/assets/txn-9.png | Bin 0 -> 469713 bytes
 site2/docs/txn-how.md       | 318 ++++++++++++++++++++++++++++++++++++++++++++
 site2/docs/txn-monitor.md   |   9 ++
 site2/docs/txn-use.md       | 124 +++++++++++++++++
 site2/docs/txn-what.md      |  59 ++++++++
 site2/docs/txn-why.md       |  44 ++++++
 site2/website/sidebars.json |   8 +-
 15 files changed, 559 insertions(+), 3 deletions(-)

diff --git a/site2/docs/assets/txn-1.png b/site2/docs/assets/txn-1.png
new file mode 100644
index 0000000..357099b
Binary files /dev/null and b/site2/docs/assets/txn-1.png differ
diff --git a/site2/docs/assets/txn-2.png b/site2/docs/assets/txn-2.png
new file mode 100644
index 0000000..2ab46e1
Binary files /dev/null and b/site2/docs/assets/txn-2.png differ
diff --git a/site2/docs/assets/txn-3.png b/site2/docs/assets/txn-3.png
new file mode 100644
index 0000000..33236ee
Binary files /dev/null and b/site2/docs/assets/txn-3.png differ
diff --git a/site2/docs/assets/txn-4.png b/site2/docs/assets/txn-4.png
new file mode 100644
index 0000000..fca20aa
Binary files /dev/null and b/site2/docs/assets/txn-4.png differ
diff --git a/site2/docs/assets/txn-5.png b/site2/docs/assets/txn-5.png
new file mode 100644
index 0000000..0de425c
Binary files /dev/null and b/site2/docs/assets/txn-5.png differ
diff --git a/site2/docs/assets/txn-6.png b/site2/docs/assets/txn-6.png
new file mode 100644
index 0000000..83a7151
Binary files /dev/null and b/site2/docs/assets/txn-6.png differ
diff --git a/site2/docs/assets/txn-7.png b/site2/docs/assets/txn-7.png
new file mode 100644
index 0000000..0c71859
Binary files /dev/null and b/site2/docs/assets/txn-7.png differ
diff --git a/site2/docs/assets/txn-8.png b/site2/docs/assets/txn-8.png
new file mode 100644
index 0000000..96654ba
Binary files /dev/null and b/site2/docs/assets/txn-8.png differ
diff --git a/site2/docs/assets/txn-9.png b/site2/docs/assets/txn-9.png
new file mode 100644
index 0000000..1a24056
Binary files /dev/null and b/site2/docs/assets/txn-9.png differ
diff --git a/site2/docs/txn-how.md b/site2/docs/txn-how.md
new file mode 100644
index 0000000..3ae0a59
--- /dev/null
+++ b/site2/docs/txn-how.md
@@ -0,0 +1,318 @@
+---
+id: txn-how
+title: How transactions work?
+sidebar_label: How transactions work?
+---
+
+This section describes transaction components and how the components work 
together. For the complete design details, see [PIP-31: Transactional 
Streaming](https://docs.google.com/document/d/145VYp09JKTw9jAT-7yNyFU255FptB2_B2Fye100ZXDI/edit#heading=h.bm5ainqxosrx).
+
+## Key concept
+
+It is important to know the following key concepts, which is a prerequisite 
for understanding how transactions work.
+
+### Transaction coordinator
+
+The transaction coordinator (TC) is a module running inside a Pulsar broker. 
+
+* It maintains the entire life cycle of transactions and prevents a 
transaction from getting into an incorrect status. 
+  
+* It handles transaction timeout, and ensures that the transaction is aborted 
after a transaction timeout.
+
+### Transaction log
+
+All the transaction metadata persists in the transaction log. The transaction 
log is backed by a Pulsar topic. If the transaction coordinator crashes, it can 
restore the transaction metadata from the transaction log.
+
+The transaction log stores the transaction status rather than actual messages 
in the transaction (the actual messages are stored in the actual topic 
partitions). 
+
+### Transaction buffer
+
+Messages produced to a topic partition within a transaction are stored in the 
transaction buffer (TB) of that topic partition. The messages in the 
transaction buffer are not visible to consumers until the transactions are 
committed. The messages in the transaction buffer are discarded when the 
transactions are aborted. 
+
+Transaction buffer stores all ongoing and aborted transactions in memory. All 
messages are sent to the actual partitioned Pulsar topics.  After transactions 
are committed, the messages in the transaction buffer are materialized 
(visible) to consumers. When the transactions are aborted, the messages in the 
transaction buffer are discarded.
+
+### Transaction ID
+
+Transaction ID (TxnID) identifies a unique transaction in Pulsar. The 
transaction ID is 128-bit. The highest 16 bits are reserved for the ID of the 
transaction coordinator, and the remaining bits are used for monotonically 
increasing numbers in each transaction coordinator. It is easy to locate the 
transaction crash with the TxnID.
+
+### Pending acknowledge state
+
+Pending acknowledge state maintains message acknowledgments within a 
transaction before a transaction completes. If a message is in the pending 
acknowledge state, the message cannot be acknowledged by other transactions 
until the message is removed from the pending acknowledge state.
+
+The pending acknowledge state is persisted to the pending acknowledge log 
(cursor ledger). A new broker can restore the state from the pending 
acknowledge log to ensure the acknowledgement is not lost.    
+
+## Data flow
+
+At a high level, the data flow can be split into several steps:
+
+1. Begin a transaction.
+   
+2. Publish messages with a transaction.
+   
+3. Acknowledge messages with a transaction.
+   
+4. End a transaction.
+
+To help you debug or tune the transaction for better performance, review the 
following diagrams and descriptions. 
+
+### 1. Begin a transaction
+
+Before introducing the transaction in Pulsar, a producer is created and then 
messages are sent to brokers and stored in data logs. 
+
+![](assets/txn-3.png)
+
+Let’s walk through the steps for _beginning a transaction_.
+
+<table>
+  <tr>
+   <td>Step
+   </td>
+   <td>Description
+   </td>
+  </tr>
+  <tr>
+   <td>1.1<br>New Txn
+   </td>
+   <td>The first step is that the Pulsar client finds the transaction 
coordinator.
+   </td>
+  </tr>
+  <tr>
+   <td>1.2<br>Allocate Txn ID
+   </td>
+   <td>The transaction coordinator allocates a transaction ID for the 
transaction. In the transaction log, the transaction is logged with its 
transaction ID and status (OPEN), which ensures the transaction status is 
persisted regardless of transaction coordinator crashes. 
+   </td>
+  </tr>
+  <tr>
+   <td>1.3<br>Send Result
+   </td>
+   <td>The transaction log sends the result of persisting the transaction ID 
to the transaction coordinator.
+   </td>
+  </tr>
+  <tr>
+   <td>1.4<br>Bring Txn ID
+   </td>
+   <td>After the transaction status entry is logged, the transaction 
coordinator brings the transaction ID back to the Pulsar client.
+   </td>
+  </tr>
+</table>
+
+### 2. Publish messages with a transaction
+
+In this stage, the Pulsar client enters a transaction loop, repeating the 
`consume-process-produce` operation for all the messages that comprise the 
transaction. This is a long phase and is potentially composed of multiple 
produce and acknowledgement requests. 
+
+![](assets/txn-4.png)
+
+Let’s walk through the steps for _publishing messages with a transaction_.
+
+<table>
+  <tr>
+   <td>Step
+   </td>
+   <td>Description
+   </td>
+  </tr>
+  <tr>
+   <td>2.1.1<br>Add Produced Partitions to Txn
+   </td>
+   <td>Before the Pulsar client produces messages to a new topic partition, it 
sends a request to the transaction coordinator to add the partition to the 
transaction.
+   </td>
+  </tr>
+  <tr>
+   <td>2.1.2<br>Log Partition Changes of Txn
+   </td>
+   <td>The transaction coordinator logs the partition changes of the 
transaction into the transaction log for durability, which ensures the 
transaction coordinator knows all the partitions that a transaction is 
handling. The transaction coordinator can commit or abort changes on each 
partition at the end-partition phase.
+   </td>
+  </tr>
+  <tr>
+   <td>2.1.3<br>Send Result
+   </td>
+   <td>The transaction log sends the result of logging the new partition (used 
for producing messages) to the transaction coordinator.
+   </td>
+  </tr>
+  <tr>
+   <td>2.1.4<br>Send Result
+   </td>
+   <td>The transaction coordinator sends the result of adding a new produced 
partition to the transaction.
+   </td>
+  </tr>
+  <tr>
+   <td>2.2.1<br>Produce Msgs to Partitions w/Txn
+   </td>
+   <td>The Pulsar client starts producing messages to partitions. The flow of 
this part is the same as the normal flow of producing messages except that the 
batch of messages produced by a transaction contains transaction IDs. 
+   </td>
+  </tr>
+  <tr>
+   <td>2.2.2<br>Write Msgs
+   </td>
+   <td>The broker writes messages to a partition.
+   </td>
+  </tr>
+</table>
+
+### 3. Acknowledge messages with a transaction
+
+In this phase, the Pulsar client sends a request to the transaction 
coordinator and a new subscription is acknowledged as a part of a transaction.
+
+![](assets/txn-5.png)
+
+Let’s walk through the steps for _acknowledging messages with a transaction_.
+
+<table>
+  <tr>
+   <td>Step
+   </td>
+   <td>Description
+   </td>
+  </tr>
+  <tr>
+   <td>3.1.1<br>Send Request
+   </td>
+   <td>The Pulsar client sends a request to add an acknowledged subscription 
to the transaction coordinator.
+   </td>
+  </tr>
+  <tr>
+   <td>3.1.2<br>Log Subscription
+   </td>
+   <td>The transaction coordinator logs the addition of subscription, which 
ensures that it knows all subscriptions handled by a transaction and can commit 
or abort changes on each subscription at the end phase.
+   </td>
+  </tr>
+  <tr>
+   <td>3.1.3<br>Send Result
+   </td>
+   <td>The transaction log sends the result of logging the new partition (used 
for acknowledging messages) to the transaction coordinator.
+   </td>
+  </tr>
+  <tr>
+   <td>3.1.4<br>Send Result
+   </td>
+   <td>The transaction coordinator sends the result of adding the new 
acknowledged partition to the transaction.
+   </td>
+  </tr>
+  <tr>
+   <td>3.2<br>Ack Msgs w/ Txn
+   </td>
+   <td>The Pulsar client acknowledges messages on the subscription. The flow 
of this part is the same as the normal flow of acknowledging messages except 
that the acknowledged request carries a transaction ID. 
+   </td>
+  </tr>
+  <tr>
+   <td>3.3<br>Check Ack
+   </td>
+   <td>The broker receiving the acknowledgement request checks if the 
acknowledgment belongs to a transaction or not.<br>If it belongs to a 
transaction, the broker marks the message as in PENDING_ACK status, which means 
the message can not be acknowledged or negative-acknowledged by other consumers 
using the same subscription until the acknowledgement is committed or 
aborted.<br>If there are two transactions attempting to acknowledge on one 
message with the same subscription, only one tr [...]
+   </td>
+  </tr>
+</table>
+
+### 4. End a transaction
+
+At the end of a transaction, the Pulsar client decides to commit or abort the 
transaction. The transaction can be aborted when a conflict is detected on 
acknowledging messages. 
+
+#### 4.1 End transaction request
+
+When the Pulsar client finishes a transaction, it issues an end transaction 
request.
+
+![](assets/txn-6.png)
+
+Let’s walk through the steps for _ending the transaction_.
+
+<table>
+  <tr>
+   <td>Step
+   </td>
+   <td>Description
+   </td>
+  </tr>
+  <tr>
+   <td>4.1.1<br>End Txn request
+   </td>
+   <td>The Pulsar client issues an end transaction request (with a field 
indicating whether the transaction is to be committed or aborted) to the 
transaction coordinator. 
+   </td>
+  </tr>
+  <tr>
+   <td>4.1.2<br>Committing Txn
+   </td>
+   <td>The transaction coordinator writes a COMMITTING or ABORTING message to 
its transaction log.
+   </td>
+  </tr>
+  <tr>
+   <td>4.1.3<br>Send Results
+   </td>
+   <td>The transaction log sends the result of logging the committing or 
aborting status.
+   </td>
+  </tr>
+</table>
+
+#### 4.2 Finalize a transaction
+
+The transaction coordinator starts the process of committing or aborting 
messages to all the partitions involved in this transaction. 
+
+![](assets/txn-7.png)
+
+Let’s walk through the steps for _finalizing a transaction_.
+
+<table>
+  <tr>
+   <td>Step
+   </td>
+   <td>Description
+   </td>
+  </tr>
+  <tr>
+   <td>4.2.1<br>Commit Txn on Subscriptions
+   </td>
+   <td>The transaction coordinator commits transactions on subscriptions and 
commits transactions on partitions at the same time.
+   </td>
+  </tr>
+  <tr>
+   <td>4.2.2<br>Write Marker
+   </td>
+   <td>The broker (produce) writes produced committed markers to the actual 
partitions. At the same time, the broker (ack) writes acked committed marks to 
the subscription pending ack partitions.
+   </td>
+  </tr>
+  <tr>
+   <td>4.2.3<br>Send Result
+   </td>
+   <td>The data log sends the result of writing produced committed marks to 
the broker. At the same time, pending ack data log sends the result of writing 
acked committed marks to the broker. The cursor moves to the next position.
+<ul>
+
+<li>If the transaction is committed, the PENDING ACK status becomes `ACK` 
status.
+
+<li>If the transaction is aborted, the PENDING ACK status becomes UNACK 
status. (Aborting an acknowledgment results in the message being re-delivered 
to other consumers.)
+</li>
+</ul>
+   </td>
+  </tr>
+</table>
+
+#### 4.3 Mark a transaction as COMMITTED or ABORTED
+
+The transaction coordinator writes the final transaction status to the 
transaction log to complete the transaction.
+
+![](assets/txn-8.png)
+
+Let’s walk through the steps for _marking a transaction as COMMITTED or 
ABORTED_.
+
+<table>
+  <tr>
+   <td>Step
+   </td>
+   <td>Description
+   </td>
+  </tr>
+  <tr>
+   <td>4.3.1<br>Commit Txn
+   </td>
+   <td>After all produced messages and acknowledgements to all partitions 
involved in this transaction have been successfully committed or aborted, the 
transaction coordinator writes the final COMMITTED or ABORTED transaction 
status messages to its transaction log, indicating that the transaction is 
complete. All the messages associated with the transaction in its transaction 
log can be safely removed.
+   </td>
+  </tr>
+  <tr>
+   <td>4.3.2<br>Send Result
+   </td>
+   <td>The transaction log sends the result of the committed transaction to 
the transaction coordinator.
+   </td>
+  </tr>
+  <tr>
+   <td>4.3.3<br>Send Result
+   </td>
+   <td>The transaction coordinator sends the result of the committed 
transaction to the Pulsar client.
+   </td>
+  </tr>
+</table>
diff --git a/site2/docs/txn-monitor.md b/site2/docs/txn-monitor.md
new file mode 100644
index 0000000..bdd9900
--- /dev/null
+++ b/site2/docs/txn-monitor.md
@@ -0,0 +1,9 @@
+---
+id: txn-monitor
+title: How to monitor transactions?
+sidebar_label: How to monitor transactions?
+---
+
+You can monitor the status of the transactions in Prometheus and Grafana using 
the [transaction 
metrics](https://pulsar.apache.org/docs/en/next/reference-metrics/#pulsar-transaction).
 
+
+For how to configure Prometheus and Grafana, see 
[here](https://pulsar.apache.org/docs/en/next/deploy-monitoring).
diff --git a/site2/docs/txn-use.md b/site2/docs/txn-use.md
new file mode 100644
index 0000000..d527e3e
--- /dev/null
+++ b/site2/docs/txn-use.md
@@ -0,0 +1,124 @@
+---
+id: txn-use
+title: How to use transactions?
+sidebar_label: How to use transactions?
+---
+
+## Transaction API
+
+The transaction feature is primarily a server-side and protocol-level feature. 
You can use the transaction feature via the [transaction 
API](https://pulsar.apache.org/api/admin/), which is available in **Pulsar 
2.8.0 or later**. 
+
+To use the transaction API, you do not need any additional settings in the 
Pulsar client. **By default**, transactions is **disabled**. 
+
+Currently, transaction API is only available for **Java** clients. Support for 
other language clients will be added in the future releases.
+
+## Quick start
+
+This section provides an example of how to use the transaction API to send and 
receive messages in a Java client. 
+
+1. Start Pulsar 2.8.0 or later. 
+
+2. Enable transaction. 
+
+    Change the configuration in the `broker.conf` file.
+
+    ```
+    transactionCoordinatorEnabled=true
+    ```
+
+    If you want to enable batch messages in transactions, follow the steps 
below.
+
+    Set `acknowledgmentAtBatchIndexLevelEnabled` to `true` in the 
`broker.conf` or `standalone.conf` file.
+
+      ```
+      acknowledgmentAtBatchIndexLevelEnabled=true
+      ```
+
+3. Initialize transaction coordinator metadata.
+
+    The transaction coordinator can leverage the advantages of partitioned 
topics (such as load balance).
+
+    **Input**
+
+    ```
+    bin/pulsar initialize-transaction-coordinator-metadata -cs 127.0.0.1:2181 
-c standalone
+    ```
+
+    **Output**
+
+    ```
+    Transaction coordinator metadata setup success
+    ```
+
+4. Initialize a Pulsar client.
+
+    ```
+    PulsarClient client = PulsarClient.builder()
+
+    .serviceUrl(“pulsar://localhost:6650”)
+
+    .enableTransaction(true)
+
+    .build();
+    ```
+
+Now you can start using the transaction API to send and receive messages. 
Below is an example of a `consume-process-produce` application written in Java.
+
+![](assets/txn-9.png)
+
+Let’s walk through this example step by step.
+
+<table>
+  <tr>
+   <td>Step
+   </td>
+   <td>Description
+   </td>
+  </tr>
+  <tr>
+   <td>1. Start a transaction.
+   </td>
+   <td>The application opens a new transaction by calling 
PulsarClient.newTransaction. It specifics the transaction timeout as 1 minute. 
If the transaction is not committed within 1 minute, the transaction is 
automatically aborted.
+   </td>
+  </tr>
+  <tr>
+   <td>2. Receive messages from topics.
+   </td>
+   <td>The application creates two normal consumers to receive messages from 
topic input-topic-1 and input-topic-2 respectively.<br><br>If you want to 
enable batch messages ack in transactions, call the 
enableBatchIndexAcknowledgment(true) method in the consumer builder. For the 
example, see [1] below this table.
+   </td>
+  </tr>
+  <tr>
+   <td>3. Publish messages to topics with the transaction.
+   </td>
+   <td>The application creates two producers to produce the resulting messages 
to the output topic _output-topic-1_ and output-topic-2 respectively. The 
application applies the processing logic and generates two output messages. The 
application sends those two output messages as part of the transaction opened 
in the first step via Producer.newMessage(Transaction).
+   </td>
+  </tr>
+  <tr>
+   <td>4. Acknowledge the messages with the transaction.
+   </td>
+   <td>In the same transaction, the application acknowledges the two input 
messages.
+   </td>
+  </tr>
+  <tr>
+   <td>5. Commit the transaction.
+   </td>
+   <td>The application commits the transaction by calling Transaction.commit() 
on the open transaction. The commit operation ensures the two input messages 
are marked as acknowledged and the two output messages are written successfully 
to the output topics. 
+   <br><br>Tip: You can also call Transaction.abort() to abort the open 
transaction.
+   </td>
+  </tr>
+</table>
+
+[1] Example of enabling batch messages ack in transactions in the consumer 
builder.
+
+```
+Consumer<byte[]> sinkConsumer = pulsarClient
+    .newConsumer()
+    .topic(transferTopic)
+    .subscriptionName("sink-topic")
+
+.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+    .subscriptionType(SubscriptionType.Shared)
+    .enableBatchIndexAcknowledgment(true) // enable batch index acknowledgement
+    .subscribe();
+```
+
diff --git a/site2/docs/txn-what.md b/site2/docs/txn-what.md
new file mode 100644
index 0000000..f3eb3f8
--- /dev/null
+++ b/site2/docs/txn-what.md
@@ -0,0 +1,59 @@
+---
+id: txn-what
+title: What are transactions?
+sidebar_label: What are transactions?
+---
+
+Transactions strengthen the message delivery semantics of Apache Pulsar and 
[processing guarantees of Pulsar 
Functions](https://pulsar.apache.org/docs/en/next/functions-overview/#processing-guarantees).
 The Pulsar Transaction API supports atomic writes and acknowledgments across 
multiple topics. 
+
+Transactions allow:
+
+- A producer to send a batch of messages to multiple topics where all messages 
in the batch are eventually visible to any consumer, or none are ever visible 
to consumers. 
+
+- End-to-end exactly-once semantics (execute a `consume-process-produce` 
operation exactly once).
+
+## Transaction semantics
+
+Pulsar transactions have the following semantics: 
+
+* All operations within a transaction are committed as a single unit.
+   
+  * Either all messages are committed, or none of them are. 
+
+  * Each message is written or processed exactly once, without data loss or 
duplicates (even in the event of failures). 
+
+  * If a transaction is aborted, all the writes and acknowledgments in this 
transaction rollback.
+  
+* A group of messages in a transaction can be received from, produced to, and 
acknowledged by multiple partitions.
+  
+  * Consumers are only allowed to read committed (acked) messages. In other 
words, the broker does not deliver transactional messages which are part of an 
open transaction or messages which are part of an aborted transaction.
+    
+  * Message writes across multiple partitions are atomic.
+    
+  * Message acks across multiple subscriptions are atomic. A message is acked 
successfully only once by a consumer under the subscription when acknowledging 
the message with the transaction ID.
+
+## Transactions and stream processing
+
+Stream processing on Pulsar is a `consume-process-produce` operation on Pulsar 
topics:
+
+* `Consume`: a source operator that runs a Pulsar consumer reads messages from 
one or multiple Pulsar topics.
+  
+* `Process`: a processing operator transforms the messages. 
+  
+* `Produce`: a sink operator that runs a Pulsar producer writes the resulting 
messages to one or multiple Pulsar topics.
+
+![](assets/txn-2.png)
+
+Pulsar transactions support end-to-end exactly-once stream processing, which 
means messages are not lost from a source operator and messages are not 
duplicated to a sink operator.
+
+## Use case
+
+Prior to Pulsar 2.8.0, there was no easy way to build stream processing 
applications with Pulsar to achieve exactly-once processing guarantees. With 
the transaction introduced in Pulsar 2.8.0, the following services support 
exactly-once semantics:
+
+* [Pulsar Flink 
connector](https://flink.apache.org/2021/01/07/pulsar-flink-connector-270.html)
+
+    Prior to Pulsar 2.8.0, if you want to build stream applications using 
Pulsar and Flink, the Pulsar Flink connector only supported exactly-once source 
connector and at-least-once sink connector, which means the highest processing 
guarantee for end-to-end was at-least-once, there was possibility that the 
resulting messages from streaming applications produce duplicated messages to 
the resulting topics in Pulsar.
+
+    With the transaction introduced in Pulsar 2.8.0, the Pulsar Flink sink 
connector can support exactly-once semantics by implementing the designated 
`TwoPhaseCommitSinkFunction` and hooking up the Flink sink message lifecycle 
with Pulsar transaction API. 
+
+* Support for Pulsar Functions and other connectors will be added in the 
future releases.
diff --git a/site2/docs/txn-why.md b/site2/docs/txn-why.md
new file mode 100644
index 0000000..73d9f8a
--- /dev/null
+++ b/site2/docs/txn-why.md
@@ -0,0 +1,44 @@
+---
+id: txn-why
+title: Why transactions?
+sidebar_label: Why transactions?
+---
+
+Pulsar transactions (txn) enable event streaming applications to consume, 
process, and produce messages in one atomic operation. The reason for 
developing this feature can be summarized as below.
+
+## Demand of stream processing
+
+The demand for stream processing applications with stronger processing 
guarantees has grown along with the rise of stream processing. For example, in 
the financial industry, financial institutions use stream processing engines to 
process debits and credits for users. This type of use case requires that every 
message is processed exactly once, without exception.
+
+In other words, if a stream processing application consumes message A and
+produces the result as a message B (B = f(A)), then exactly-once processing
+guarantee means that A can only be marked as consumed if and only if B is
+successfully produced, and vice versa.
+
+![](assets/txn-1.png)
+
+The Pulsar transactions API strengthens the message delivery semantics and the 
processing guarantees for stream processing. It enables stream processing 
applications to consume, process, and produce messages in one atomic operation. 
That means, a batch of messages in a transaction can be received from, produced 
to and acknowledged by many topic partitions. All the operations involved in a 
transaction succeed or fail as one single until.
+
+## Limitation of idempotent producer
+
+Avoiding data loss or duplication can be achieved by using the Pulsar 
idempotent producer, but it does not provide guarantees for writes across 
multiple partitions. 
+
+In Pulsar, the highest level of message delivery guarantee is using an 
[idempotent 
producer](https://pulsar.apache.org/docs/en/next/concepts-messaging/#producer-idempotency)
 with the exactly once semantic at one single partition, that is, each message 
is persisted exactly once without data loss and duplication. However, there are 
some limitations in this solution:
+
+- Due to the monotonic increasing sequence ID, this solution only works on a 
single partition and within a single producer session (that is, for producing 
one message), so there is no atomicity when producing multiple messages to one 
or multiple partitions. 
+  
+  In this case, if there are some failures  (for example, client / broker / 
bookie crashes, network failure, and more) in the process of producing and 
receiving messages, messages are re-processed and re-delivered, which may cause 
data loss or data duplication: 
+
+  - For the producer: if the producer retry sending messages, some messages 
are persisted multiple times; if the producer does not retry sending messages, 
some messages are persisted once and other messages are lost. 
+  
+  - For the consumer: since the consumer does not know whether the broker has 
received messages or not, the consumer may not retry sending acks, which causes 
it to receive duplicate messages.  
+
+- Similarly, for Pulsar Function, it only guarantees exactly once semantics 
for an idempotent function on a single event rather than processing multiple 
events or producing multiple results that can happen exactly. 
+
+    For example, if a function accepts multiple events and produces one result 
(for example, window function), the function may fail between producing the 
result and acknowledging the incoming messages, or even between acknowledging 
individual events, which causes all (or some) incoming messages to be 
re-delivered and reprocessed, and a new result is generated.
+
+    However, many scenarios need atomic guarantees across multiple partitions 
and sessions.
+
+- Consumers need to rely on more mechanisms to acknowledge (ack) messages 
once. 
+  
+  For example, consumers are required to store the MessgeID along with its 
acked state. After the topic is unloaded, the subscription can recover the 
acked state of this MessgeID in memory when the topic is loaded again.
\ No newline at end of file
diff --git a/site2/website/sidebars.json b/site2/website/sidebars.json
index 7723606..3526362 100644
--- a/site2/website/sidebars.json
+++ b/site2/website/sidebars.json
@@ -59,9 +59,11 @@
      "tiered-storage-aliyun"
   ],
   "Transactions": [
-    "transactions",
-    "transactions-guarantee",
-    "transactions-api"
+    "txn-why",
+    "txn-what",
+    "txn-how",
+    "txn-use",
+    "txn-monitor"
   ],
     "Kubernetes (Helm)": [
       "helm-overview",

Reply via email to