Skip to content

Commit

Permalink
[feat][txn]Implement transactional consumer/producer API (#1002)
Browse files Browse the repository at this point in the history
* [feat][txn]Implement transactional consumer/producer API

* rollback

* fix test

* optimize code

* state check

* fix callback

* golangci-lint

* add more test
  • Loading branch information
liangyepianzhou authored May 6, 2023
1 parent fc4f289 commit 2437caa
Show file tree
Hide file tree
Showing 19 changed files with 573 additions and 85 deletions.
11 changes: 11 additions & 0 deletions pulsar/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,17 @@ type Client interface {
// {@link Consumer} or {@link Producer} instances directly on a particular partition.
TopicPartitions(topic string) ([]string, error)

// NewTransaction creates a new Transaction instance.
//
// This function is used to initiate a new transaction for performing
// atomic operations on the message broker. It returns a Transaction
// object that can be used to produce, consume and commit messages in a
// transactional manner.
//
// In case of any errors while creating the transaction, an error will
// be returned.
NewTransaction(duration time.Duration) (Transaction, error)

// Close Closes the Client and free associated resources
Close()
}
Expand Down
8 changes: 8 additions & 0 deletions pulsar/client_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,14 @@ func newClient(options ClientOptions) (Client, error) {
return c, nil
}

func (c *client) NewTransaction(timeout time.Duration) (Transaction, error) {
id, err := c.tcClient.newTransaction(timeout)
if err != nil {
return nil, err
}
return newTransaction(*id, c.tcClient, timeout), nil
}

func (c *client) CreateProducer(options ProducerOptions) (Producer, error) {
producer, err := newProducer(c, &options)
if err == nil {
Expand Down
3 changes: 3 additions & 0 deletions pulsar/consumer.go
Original file line number Diff line number Diff line change
Expand Up @@ -265,6 +265,9 @@ type Consumer interface {
// AckID the consumption of a single message, identified by its MessageID
AckID(MessageID) error

// AckWithTxn the consumption of a single message with a transaction
AckWithTxn(Message, Transaction) error

// AckCumulative the reception of all the messages in the stream up to (and including)
// the provided message.
AckCumulative(msg Message) error
Expand Down
10 changes: 10 additions & 0 deletions pulsar/consumer_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ type acker interface {
// AckID does not handle errors returned by the Broker side, so no need to wait for doneCh to finish.
AckID(id MessageID) error
AckIDWithResponse(id MessageID) error
AckIDWithTxn(msgID MessageID, txn Transaction) error
AckIDCumulative(msgID MessageID) error
AckIDWithResponseCumulative(msgID MessageID) error
NackID(id MessageID)
Expand Down Expand Up @@ -478,6 +479,15 @@ func (c *consumer) Receive(ctx context.Context) (message Message, err error) {
}
}

func (c *consumer) AckWithTxn(msg Message, txn Transaction) error {
msgID := msg.ID()
if err := c.checkMsgIDPartition(msgID); err != nil {
return err
}

return c.consumers[msgID.PartitionIdx()].AckIDWithTxn(msgID, txn)
}

// Chan return the message chan to users
func (c *consumer) Chan() <-chan ConsumerMessage {
return c.messageCh
Expand Down
17 changes: 17 additions & 0 deletions pulsar/consumer_multitopic.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,23 @@ func (c *multiTopicConsumer) AckID(msgID MessageID) error {
return mid.consumer.AckID(msgID)
}

// AckWithTxn the consumption of a single message with a transaction
func (c *multiTopicConsumer) AckWithTxn(msg Message, txn Transaction) error {
msgID := msg.ID()
if !checkMessageIDType(msgID) {
c.log.Warnf("invalid message id type %T", msgID)
return errors.New("invalid message id type in multi_consumer")
}
mid := toTrackingMessageID(msgID)

if mid.consumer == nil {
c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID)
return errors.New("unable to ack message because consumer is nil")
}

return mid.consumer.AckIDWithTxn(msgID, txn)
}

// AckCumulative the reception of all the messages in the stream up to (and including)
// the provided message
func (c *multiTopicConsumer) AckCumulative(msg Message) error {
Expand Down
174 changes: 133 additions & 41 deletions pulsar/consumer_partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -417,6 +417,118 @@ func (pc *partitionConsumer) Unsubscribe() error {
return req.err
}

// ackIDCommon handles common logic for acknowledging messages with or without transactions.
// withTxn should be set to true when dealing with transactions.
func (pc *partitionConsumer) ackIDCommon(msgID MessageID, withResponse bool, txn Transaction) error {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer")
return errors.New("consumer state is closed")
}

if cmid, ok := msgID.(*chunkMessageID); ok {
return pc.unAckChunksTracker.ack(cmid)
}

trackingID := toTrackingMessageID(msgID)

if trackingID != nil && trackingID.ack() {
// All messages in the same batch have been acknowledged, we only need to acknowledge the
// MessageID that represents the entry that stores the whole batch
trackingID = &trackingMessageID{
messageID: &messageID{
ledgerID: trackingID.ledgerID,
entryID: trackingID.entryID,
},
}
pc.metrics.AcksCounter.Inc()
pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9)
} else if !pc.options.enableBatchIndexAck {
return nil
}

var err error
if withResponse {
if txn != nil {
ackReq := pc.sendIndividualAckWithTxn(trackingID, txn.(*transaction))
<-ackReq.doneCh
err = ackReq.err
} else {
ackReq := pc.sendIndividualAck(trackingID)
<-ackReq.doneCh
err = ackReq.err
}
} else {
pc.ackGroupingTracker.add(trackingID)
}
pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID)
return err
}

// AckIDWithTxn acknowledges the consumption of a message with transaction.
func (pc *partitionConsumer) AckIDWithTxn(msgID MessageID, txn Transaction) error {
return pc.ackIDCommon(msgID, true, txn)
}

// ackID acknowledges the consumption of a message and optionally waits for response from the broker.
func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error {
return pc.ackIDCommon(msgID, withResponse, nil)
}

func (pc *partitionConsumer) internalAckWithTxn(req *ackWithTxnRequest) {
defer close(req.doneCh)
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer")
req.err = newError(ConsumerClosed, "Failed to ack by closing or closed consumer")
return
}
if req.Transaction.state != TxnOpen {
pc.log.WithField("state", req.Transaction.state).Error("Failed to ack by a non-open transaction.")
req.err = newError(InvalidStatus, "Failed to ack by a non-open transaction.")
return
}
msgID := req.msgID

messageIDs := make([]*pb.MessageIdData, 1)
messageIDs[0] = &pb.MessageIdData{
LedgerId: proto.Uint64(uint64(msgID.ledgerID)),
EntryId: proto.Uint64(uint64(msgID.entryID)),
}
if pc.options.enableBatchIndexAck && msgID.tracker != nil {
ackSet := msgID.tracker.toAckSet()
if ackSet != nil {
messageIDs[0].AckSet = ackSet
}
}

reqID := pc.client.rpcClient.NewRequestID()
txnID := req.Transaction.GetTxnID()
cmdAck := &pb.CommandAck{
ConsumerId: proto.Uint64(pc.consumerID),
MessageId: messageIDs,
AckType: pb.CommandAck_Individual.Enum(),
TxnidMostBits: proto.Uint64(txnID.MostSigBits),
TxnidLeastBits: proto.Uint64(txnID.LeastSigBits),
}

if err := req.Transaction.registerAckTopic(pc.options.topic, pc.options.subscription); err != nil {
req.err = err
return
}

if err := req.Transaction.registerSendOrAckOp(); err != nil {
req.err = err
return
}

cmdAck.RequestId = proto.Uint64(reqID)
_, err := pc.client.rpcClient.RequestOnCnx(pc._getConn(), reqID, pb.BaseCommand_ACK, cmdAck)
if err != nil {
pc.log.WithError(err).Error("Ack with response error")
}
req.Transaction.endSendOrAckOp(err)
req.err = err
}

func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) {
defer close(unsub.doneCh)

Expand Down Expand Up @@ -488,47 +600,6 @@ func (pc *partitionConsumer) requestGetLastMessageID() (*trackingMessageID, erro
return convertToMessageID(id), nil
}

func (pc *partitionConsumer) ackID(msgID MessageID, withResponse bool) error {
if state := pc.getConsumerState(); state == consumerClosed || state == consumerClosing {
pc.log.WithField("state", state).Error("Failed to ack by closing or closed consumer")
return errors.New("consumer state is closed")
}

if cmid, ok := msgID.(*chunkMessageID); ok {
return pc.unAckChunksTracker.ack(cmid)
}

trackingID := toTrackingMessageID(msgID)

if trackingID != nil && trackingID.ack() {
// All messages in the same batch have been acknowledged, we only need to acknowledge the
// MessageID that represents the entry that stores the whole batch
trackingID = &trackingMessageID{
messageID: &messageID{
ledgerID: trackingID.ledgerID,
entryID: trackingID.entryID,
},
}
pc.metrics.AcksCounter.Inc()
pc.metrics.ProcessingTime.Observe(float64(time.Now().UnixNano()-trackingID.receivedTime.UnixNano()) / 1.0e9)
} else if !pc.options.enableBatchIndexAck {
return nil
}

var ackReq *ackRequest
if withResponse {
ackReq := pc.sendIndividualAck(trackingID)
<-ackReq.doneCh
} else {
pc.ackGroupingTracker.add(trackingID)
}
pc.options.interceptors.OnAcknowledge(pc.parentConsumer, msgID)
if ackReq == nil {
return nil
}
return ackReq.err
}

func (pc *partitionConsumer) sendIndividualAck(msgID MessageID) *ackRequest {
ackReq := &ackRequest{
doneCh: make(chan struct{}),
Expand All @@ -539,6 +610,17 @@ func (pc *partitionConsumer) sendIndividualAck(msgID MessageID) *ackRequest {
return ackReq
}

func (pc *partitionConsumer) sendIndividualAckWithTxn(msgID MessageID, txn *transaction) *ackWithTxnRequest {
ackReq := &ackWithTxnRequest{
Transaction: txn,
doneCh: make(chan struct{}),
ackType: individualAck,
msgID: *msgID.(*trackingMessageID),
}
pc.eventsCh <- ackReq
return ackReq
}

func (pc *partitionConsumer) AckIDWithResponse(msgID MessageID) error {
if !checkMessageIDType(msgID) {
pc.log.Errorf("invalid message id type %T", msgID)
Expand Down Expand Up @@ -1389,6 +1471,14 @@ type ackRequest struct {
err error
}

type ackWithTxnRequest struct {
doneCh chan struct{}
msgID trackingMessageID
Transaction *transaction
ackType int
err error
}

type unsubscribeRequest struct {
doneCh chan struct{}
err error
Expand Down Expand Up @@ -1444,6 +1534,8 @@ func (pc *partitionConsumer) runEventsLoop() {
switch v := i.(type) {
case *ackRequest:
pc.internalAck(v)
case *ackWithTxnRequest:
pc.internalAckWithTxn(v)
case []*pb.MessageIdData:
pc.internalAckList(v)
case *redeliveryRequest:
Expand Down
18 changes: 18 additions & 0 deletions pulsar/consumer_regex.go
Original file line number Diff line number Diff line change
Expand Up @@ -193,6 +193,24 @@ func (c *regexConsumer) AckID(msgID MessageID) error {
return mid.consumer.AckID(msgID)
}

// AckID the consumption of a single message, identified by its MessageID
func (c *regexConsumer) AckWithTxn(msg Message, txn Transaction) error {
msgID := msg.ID()
if !checkMessageIDType(msgID) {
c.log.Warnf("invalid message id type %T", msgID)
return fmt.Errorf("invalid message id type %T", msgID)
}

mid := toTrackingMessageID(msgID)

if mid.consumer == nil {
c.log.Warnf("unable to ack messageID=%+v can not determine topic", msgID)
return errors.New("consumer is nil in consumer_regex")
}

return mid.consumer.AckIDWithTxn(msgID, txn)
}

// AckCumulative the reception of all the messages in the stream up to (and including)
// the provided message.
func (c *regexConsumer) AckCumulative(msg Message) error {
Expand Down
2 changes: 1 addition & 1 deletion pulsar/helper_for_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ func topicStats(topic string) (map[string]interface{}, error) {

func transactionStats(id *TxnID) (map[string]interface{}, error) {
var metadata map[string]interface{}
path := fmt.Sprintf("admin/v3/transactions/transactionMetadata/%d/%d", id.mostSigBits, id.leastSigBits)
path := fmt.Sprintf("admin/v3/transactions/transactionMetadata/%d/%d", id.MostSigBits, id.LeastSigBits)
err := httpGet(path, &metadata)
return metadata, err
}
Expand Down
8 changes: 8 additions & 0 deletions pulsar/internal/batch_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,9 @@ type BatchBuilder interface {
payload []byte,
callback interface{}, replicateTo []string, deliverAt time.Time,
schemaVersion []byte, multiSchemaEnabled bool,
useTxn bool,
mostSigBits uint64,
leastSigBits uint64,
) bool

// Flush all the messages buffered in the client and wait until all messages have been successfully persisted.
Expand Down Expand Up @@ -185,6 +188,7 @@ func (bc *batchContainer) Add(
payload []byte,
callback interface{}, replicateTo []string, deliverAt time.Time,
schemaVersion []byte, multiSchemaEnabled bool,
useTxn bool, mostSigBits uint64, leastSigBits uint64,
) bool {

if replicateTo != nil && bc.numMessages != 0 {
Expand Down Expand Up @@ -223,6 +227,10 @@ func (bc *batchContainer) Add(
}

bc.cmdSend.Send.SequenceId = proto.Uint64(sequenceID)
if useTxn {
bc.cmdSend.Send.TxnidMostBits = proto.Uint64(mostSigBits)
bc.cmdSend.Send.TxnidLeastBits = proto.Uint64(leastSigBits)
}
}
addSingleMessageToBatch(bc.buffer, metadata, payload)

Expand Down
12 changes: 9 additions & 3 deletions pulsar/internal/commands.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,10 @@ import (
"errors"
"fmt"

"google.golang.org/protobuf/proto"

"github.com/apache/pulsar-client-go/pulsar/internal/compression"
"github.com/apache/pulsar-client-go/pulsar/internal/crypto"
pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto"
"google.golang.org/protobuf/proto"
)

const (
Expand Down Expand Up @@ -332,7 +331,10 @@ func SingleSend(wb Buffer,
msgMetadata *pb.MessageMetadata,
compressedPayload Buffer,
encryptor crypto.Encryptor,
maxMassageSize uint32) error {
maxMassageSize uint32,
useTxn bool,
mostSigBits uint64,
leastSigBits uint64) error {
cmdSend := baseCommand(
pb.BaseCommand_SEND,
&pb.CommandSend{
Expand All @@ -344,6 +346,10 @@ func SingleSend(wb Buffer,
isChunk := true
cmdSend.Send.IsChunk = &isChunk
}
if useTxn {
cmdSend.Send.TxnidMostBits = proto.Uint64(mostSigBits)
cmdSend.Send.TxnidLeastBits = proto.Uint64(leastSigBits)
}
// payload has been compressed so compressionProvider can be nil
return serializeMessage(wb, cmdSend, msgMetadata, compressedPayload,
nil, encryptor, maxMassageSize, false)
Expand Down
Loading

0 comments on commit 2437caa

Please sign in to comment.