Docs Self-Managed Develop Transactions This is documentation for Self-Managed v23.3, which is no longer supported. To view the latest available version of the docs, see v24.3. Transactions Redpanda supports Apache Kafka®-compatible transaction semantics and APIs. For example, you can fetch messages starting from the last consumed offset and transactionally process them one by one, updating the last consumed offset and producing events at the same time. If a producer is sending multiple messages to the same or different partitions, and network connectivity or broker failure cause the transaction to fail, then it’s guaranteed that either all messages are written to the partitions or none. This is important for applications that require strict guarantees, like financial services transactions. Transactions guarantee both exactly-once semantics (EOS) and atomicity. EOS helps developers avoid the anomalies of at-most-once processing (with potential lost events) and at-least-once processing (with potential duplicated events). Redpanda supports EOS when transactions are used in combination with idempotent producers. Atomicity additionally commits a set of messages across partitions as a unit: either all messages are committed or none. Encapsulated data received or sent across multiple topics in a single operation can only succeed or fail globally. Atomicity of transactions is not guaranteed when remote recovery is used. Use transactions By default, the enable_transactions cluster configuration property is set to true. However, in the following use cases, clients must explicitly use the Transactions API to perform operations within a transaction. The required transactional.id property acts as a producer identity. It enables reliability semantics that span multiple producer sessions by allowing the client to guarantee that all transactions issued by the client with the same ID have completed prior to starting any new transactions. The two primary use cases for transactions are: Atomic (all or nothing) publishing of multiple messages Exactly-once stream processing Transaction usage tips A transaction can span partitions from different topics, and a topic can be deleted while there are active transactions on one or more of its partitions. In-flight transactions can detect deletion events, remove the deleted partitions (and related messages) from the transaction scope, and commit changes to the remaining partitions. Ongoing transactions can prevent consumers from advancing. To avoid this, don’t set transaction timeout (transaction.timeout.ms in Java client) to high values: the higher the timeout, the longer consumers may be blocked. By default, it’s about a minute, but it’s a client setting that depends on the client. When upgrading, make sure to use maintenance mode with a rolling upgrade. When running transactional workloads from clients, tune max_transactions_per_coordinator to the number of active transactions that you expect your clients to run at any given time (if your client transaction IDs are not reused). The total number of transactions in the cluster at any one time is max_transactions_per_coordinator * transaction_coordinator_partitions (default is 50). When the threshold is exceeded, Redpanda terminates old sessions. If an idle producer corresponding to the terminated session wakes up and produces, its batches are rejected with the message invalid producer epoch or invalid_producer_id_mapping, depending on where it is in the transaction execution phase. Be aware that if you keep the default as 50 and your clients create a new ID for every transaction, the total continues to accumulate, which bloats memory. Atomic publishing of multiple messages With its event sourcing microservice architecture, a banking IT system illustrates the necessity for transactions well. A bank has multiple branches, and each branch is an independent microservice that manages its own non-intersecting set of accounts. Each branch keeps its own ledger, which is represented as a Redpanda partition. When a branch representing a microservice starts, it replays its ledger to reconstruct the actual state. Financial transactions (money transfers) require the following guarantees: A sender can’t withdraw more than the account withdrawal limit. A recipient receives exactly the same amount sent. A transaction is fast and is run at most once. If a transaction fails, the system rolls back to the initial state. Without withdrawals and deposits, the amount of money in the system remains constant with any history of money transfers. These requirements are easy to satisfy when the sender and the recipient of a financial transaction are hosted by the same branch. The operation doesn’t leave the consistency domain, and all checks and locks can be performed within a single service (ledger). Things get more complex with cross-branch financial transactions, because they involve several ledgers, and the operations should be performed atomically (all or nothing). The default approach (saga pattern) breaks a transaction into a sequence of reversible idempotent steps; however, this violates the isolation principle and adds complexity, making the application responsible for orchestrating the steps. Redpanda natively supports transactions, so it’s possible to atomically update several ledgers at the same time. For example: Properties props = new Properties(); props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "..."); props.put(ProducerConfig.ACKS_CONFIG, "all"); props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true); props.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "app-id"); Producer<String, String> producer = null; while (true) { // waiting for somebody to initiate a financial transaction var sender_branch = ...; var sender_account = ...; var recipient_branch = ...; var recipient_account = ...; var amount = 42; if (producer == null) { try { producer = new KafkaProducer<>(props); producer.initTransactions(); } catch (Exception e1) { // TIP: log error for further analysis try { if (producer != null) { producer.close(); } } catch(Exception e2) { } producer = null; // TIP: notify the initiator of a transaction about the failure continue; } } producer.beginTransaction(); try { var f1 = producer.send(new ProducerRecord<String, String>("ledger", sender_branch, sender_account, "" + (-amount))); var f2 = producer.send(new ProducerRecord<String, String>("ledger", recipient_branch, recipient_account, "" + amount)); f1.get(); f2.get(); } catch (Exception e1) { // TIP: log error for further analysis try { producer.abortTransaction(); } catch (Exception e2) { // TIP: log error for further analysis try { producer.close(); } catch (Exception e3) { } producer = null; } // TIP: notify the initiator of a transaction about the failure continue; } try { producer.commitTransaction(); } catch (Exception e1) { try { producer.close(); } catch (Exception e3) {} producer = null; // TIP: notify the initiator of a transaction about the failure continue; } // TIP: notify the initiator of a transaction about the success } When a transaction fails before a commitTransaction attempt completes, you can assume that it is not executed. When a transaction fails after a commitTransaction attempt completes, the true transaction status is unknown. Redpanda only guarantees that there isn’t a partial result: either the transaction is committed and complete, or it is fully rolled back. Exactly-once stream processing Redpanda is commonly used as a pipe connecting different applications and storage systems. An application could use an OLTP database and then rely on change data capture to deliver the changes to a data warehouse. Redpanda transactions let you use streams as a smart pipe in your applications, building complex atomic operations that transform, aggregate, or otherwise process data transiting between external applications and storage systems. For example, here is the regular pipe flow: Postgresql -> topic -> warehouse Here is the smart pipe flow, with a transformation in topic(1) -> topic(2): Postgresql -> topic(1) transform topic(2) -> warehouse The transformation reads a record from topic(1), processes it, and writes it to topic(2). Without transactions, an intermittent error can cause a message to be lost or processed several times. With transactions, Redpanda guarantees exactly-once semantics. For example: var source = "source-topic"; var target = "target-topic" Properties pprops = new Properties(); pprops.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "..."); pprops.put(ProducerConfig.ACKS_CONFIG, "all"); pprops.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true); pprops.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, UUID.newUUID()); Properties cprops = new Properties(); cprops.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "..."); cprops.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); cprops.put(ConsumerConfig.GROUP_ID_CONFIG, "app-id"); cprops.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); cprops.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); Consumer<String, String> consumer = null; Producer<String, String> producer = null; boolean should_reset = false; while (true) { if (should_reset) { should_reset = false; if (consumer != null) { try { consumer.close(); } catch(Exception e) {} consumer = null; } if (producer != null) { try { producer.close(); } catch (Exception e2) {} producer = null; } } try { if (consumer == null) { consumer = new KafkaConsumer<>(cprops); consumer.subscribe(Collections.singleton(source)); } } catch (Exception e1) { // TIP: log error for further analysis should_reset = true; continue; } try { if (producer == null) { producer = new KafkaProducer<>(pprops); producer.initTransactions(); } } catch (Exception e1) { // TIP: log error for further analysis should_reset = true; continue; } ConsumerRecords<String, String> records = null; try { records = consumer.poll(Duration.ofMillis(10000)); } catch (Exception e1) { // TIP: log error for further analysis should_reset = true; continue; } var it = records.iterator(); while (it.hasNext()) { var record = it.next(); // transformation var old_value = record.value(); var new_value = old_value.toUpperCase(); try { producer.beginTransaction(); producer.send(new ProducerRecord<String, String>(target, record.key(), new_value)); var offsets = new HashMap<TopicPartition, OffsetAndMetadata>(); offsets.put(new TopicPartition(source, record.partition()), new OffsetAndMetadata(record.offset() + 1)); producer.sendOffsetsToTransaction(offsets, consumer.groupMetadata()); } catch (Exception e1) { // TIP: log error for further analysis try { producer.abortTransaction(); } catch (Exception e2) { } should_reset = true; break; } try { producer.commitTransaction(); } catch (Exception e1) { // TIP: log error for further analysis should_reset = true; break; } } } Different transactions require different approaches to handling failures within the application. Consider the approaches to failed or timed-out transactions in the provided use cases. Publishing of multiple messages: The request came from outside the system, and it is the application’s responsibility to discover the true status of a timed-out transaction. (This example doesn’t use consumer groups to distribute partitions between consumers.) Exactly-once streaming (consume-transform-loop): This is a closed system. Upon re-initialization of the consumer and producer, the system automatically discovers the moment it was interrupted and continues from that place. Additionally, this automatically scales by the number of partitions. Run another instance of the application, and it starts processing its share of partitions in the source topic. Enabling exactly-once processing The default configuration of Redpanda supports exactly-once processing. Preserving this capability requires maintaining the following settings: enable_idempotence = true enable_transactions = true transaction_coordinator_delete_retention_ms >= transactional_id_expiration_ms Transactions with compacted segments Transactions are supported on topics with compaction configured. When log_cleanup_policy or cleanup.policy are set to either compact or compact,delete, the compaction process removes the aborted transaction’s data and all transactional control markers from the log. The resulting compacted segment contains only committed data batches (and potentially harmless gaps in the offsets due to skipped batches). Suggested reading Kafka-compatible fast distributed transactions Suggested labs Stream Stock Market Data from a CSV file Using Node.jsStream Stock Market Data from a CSV file Using PythonBuild a Chat Room Application with Redpanda and GolangBuild a Chat Room Application with Redpanda and JavaBuild a Chat Room Application with Redpanda and PythonBuild a Chat Room Application with Redpanda and Node.jsBuild a Chat Room Application with Redpanda and RustSee moreSearch all labs Back to top × Simple online edits For simple changes, such as fixing a typo, you can edit the content directly on GitHub. Edit on GitHub Or, open an issue to let us know about something that you want us to change. Open an issue Contribution guide For extensive content updates, or if you prefer to work locally, read our contribution guide . Was this helpful? thumb_up thumb_down group Ask in the community mail Share your feedback group_add Make a contribution Run Data Transforms in Kubernetes Deploy