shibd commented on code in PR #984:
URL: https://github.com/apache/pulsar-client-go/pull/984#discussion_r1142849790


##########
pulsar/transaction_impl.go:
##########
@@ -0,0 +1,232 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+       "context"
+       "sync"
+       "sync/atomic"
+       "time"
+
+       pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto"
+       "github.com/apache/pulsar-client-go/pulsar/log"
+)
+
+type subscription struct {
+       topic        string
+       subscription string
+}
+
+type transaction struct {
+       sync.Mutex
+       txnID                    TxnID
+       state                    TxnState
+       tcClient                 *transactionCoordinatorClient
+       registerPartitions       map[string]bool
+       registerAckSubscriptions map[subscription]bool
+       // opsFlow It has two effects:
+       // 1. Wait all the operations of sending and acking messages with the 
transaction complete
+       // by reading msg from the chan.
+       // 2. Prevent sending or acking messages with a committed or aborted 
transaction.
+       // opsCount is record the number of the uncompleted operations.
+       // opsFlow
+       //   Write:
+       //     1. When the transaction is created, a bool will be written to 
opsFlow chan.
+       //     2. When the opsCount decrement from 1 to 0, a new bool will be 
written to opsFlow chan.
+       //     3. When get a retryable error after committing or aborting the 
transaction,
+       //        a bool will be written to opsFlow chan.
+       //   Read:
+       //     1. When the transaction is committed or aborted, a bool will be 
read from opsFlow chan.
+       //     2. When the opsCount increment from 0 to 1, a bool will be read 
from opsFlow chan.
+       opsFlow   chan bool
+       opsCount  int32
+       opTimeout time.Duration
+       log       log.Logger
+}
+
+func newTransaction(id TxnID, tcClient *transactionCoordinatorClient, timeout 
time.Duration) *transaction {
+       transaction := &transaction{
+               txnID:                    id,
+               state:                    Open,
+               registerPartitions:       make(map[string]bool),
+               registerAckSubscriptions: make(map[subscription]bool),
+               opsFlow:                  make(chan bool, 1),
+               opTimeout:                5 * time.Second,
+               tcClient:                 tcClient,
+       }
+       //This means there are not pending requests with this transaction. The 
transaction can be committed or aborted.
+       transaction.opsFlow <- true
+       go func() {
+               //Set the state of the transaction to timeout after timeout
+               <-time.After(timeout)
+               atomic.CompareAndSwapInt32((*int32)(&transaction.state), 
int32(Open), int32(Timeout))
+       }()
+       transaction.log = tcClient.log.SubLogger(log.Fields{})
+       return transaction
+}
+
+func (txn *transaction) GetState() TxnState {
+       return txn.state
+}
+
+func (txn *transaction) Commit(ctx context.Context) error {
+       if !(atomic.CompareAndSwapInt32((*int32)(&txn.state), int32(Open), 
int32(Committing)) || txn.state == Committing) {
+               return newError(InvalidStatus, "Expect transaction state is 
Open but "+txn.state.string())
+       }
+
+       //Wait for all operations to complete
+       select {
+       case <-txn.opsFlow:
+       case <-time.After(txn.opTimeout):
+               return newError(TimeoutError, "There are some operations that 
are not completed after the timeout.")
+       }
+       //Send commit transaction command to transaction coordinator
+       err := txn.tcClient.endTxn(&txn.txnID, pb.TxnAction_COMMIT)
+       if err == nil {
+               atomic.StoreInt32((*int32)(&txn.state), int32(Committed))
+       } else {
+               if err.(*Error).Result() == 
Result(pb.ServerError_TransactionNotFound) || err.(*Error).Result() == 
InvalidStatus {
+                       atomic.StoreInt32((*int32)(&txn.state), int32(Errored))
+                       return err
+               }
+               txn.opsFlow <- true
+       }
+       return err
+}
+
+func (txn *transaction) Abort(ctx context.Context) error {
+       if !(atomic.CompareAndSwapInt32((*int32)(&txn.state), int32(Open), 
int32(Aborting)) || txn.state == Aborting) {
+               return newError(InvalidStatus, "Expect transaction state is 
Open but "+txn.state.string())
+       }
+
+       //Wait for all operations to complete
+       select {
+       case <-txn.opsFlow:
+       case <-time.After(txn.opTimeout):
+               return newError(TimeoutError, "There are some operations that 
are not completed after the timeout.")
+       }
+       //Send abort transaction command to transaction coordinator
+       err := txn.tcClient.endTxn(&txn.txnID, pb.TxnAction_ABORT)
+       if err == nil {
+               atomic.StoreInt32((*int32)(&txn.state), int32(Aborted))
+       } else {
+               if err.(*Error).Result() == 
Result(pb.ServerError_TransactionNotFound) ||
+                       err.(*Error).Result() == 
Result(pb.ServerError_InvalidTxnStatus) {
+                       atomic.StoreInt32((*int32)(&txn.state), int32(Errored))
+               } else {
+                       txn.opsFlow <- true
+               }
+       }
+       return err
+}
+
+func (txn *transaction) registerSendOrAckOp() error {
+       if atomic.AddInt32(&txn.opsCount, 1) == 1 {
+               //There are new operations that not completed
+               select {
+               case <-txn.opsFlow:
+                       return nil
+               case <-time.After(txn.opTimeout):
+                       if _, err := txn.checkIfOpen(); err != nil {
+                               return err
+                       }
+                       return newError(TimeoutError, "Failed to get the 
semaphore to register the send/ack operation")
+               }
+       }
+       return nil
+}
+
+func (txn *transaction) endSendOrAckOp(err error) {
+       if err != nil {
+               atomic.StoreInt32((*int32)(&txn.state), int32(Errored))
+       }
+       if atomic.AddInt32(&txn.opsCount, -1) == 0 {
+               //This means there are not pending send/ack requests
+               txn.opsFlow <- true
+       }
+}
+
+func (txn *transaction) registerProducerTopic(topic string) error {
+       isOpen, err := txn.checkIfOpen()
+       if !isOpen {
+               return err
+       }
+       _, ok := txn.registerPartitions[topic]

Review Comment:
   This map also needs to be protected with a lock, right?



##########
pulsar/transaction_impl.go:
##########
@@ -0,0 +1,235 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+       "context"
+       "sync"
+       "sync/atomic"
+       "time"
+
+       pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto"
+       "github.com/apache/pulsar-client-go/pulsar/log"
+       uAtomic "go.uber.org/atomic"
+)
+
+type TxnState int32
+type void struct{}
+type subscription struct {
+       topic        string
+       subscription string
+}
+
+type transaction struct {
+       sync.Mutex
+       txnID                    TxnID
+       state                    TxnState
+       tcClient                 *transactionCoordinatorClient
+       registerPartitions       map[string]void
+       registerAckSubscriptions map[subscription]void
+       /**
+       * opsFlow Wait all the operations of sending and acking messages with 
the transaction complete
+       * by reading msg from the chan.
+       * opsCount is record the number of the uncompleted operations.
+       * opsFlow
+       *        Write:
+       *                     1. When the transaction is created, a new empty 
struct{} will be written to opsFlow chan.
+       *                         2. When the opsCount decrement from 1 to 0, a 
new empty struct{} will be written to opsFlow chan.
+       *                         3. When get a retryable error after 
committing or aborting the transaction,
+       *                    a new empty struct{} will be written to opsFlow 
chan.
+       *        Read:
+       *                         1. When the transaction is committed or 
aborted, an empty struct{} will be read from opsFlow chan.
+       *                         2. When the opsCount increment from 0 to 1, 
an empty struct{} will be read from opsFlow chan.
+        */
+       opsFlow   chan void

Review Comment:
   oh, I see.
   
   When `registerProducerTopic` and `registerAckTopic` operations are being 
performed, commits and aborts are also not allowed, right?
   
   I see that the future of these Java operations is linked:
   - `registerAckedTopic` future -> `registerAckOp`
   - `registerProducedTopic` future -> `registerSendOp`
   
   
   I'm not quite sure what to do with the go client next, maybe you can add 
some more complete tests?
   
   `registerSendOrAckOp` and `endSendOrAckOp` and `registerProducerTopic` and 
`registerAckTopic`  these complete a test scenario overlay.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to