-
Notifications
You must be signed in to change notification settings - Fork 798
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add AddPartitionsToTxn and EndTxn functions to client.
- Loading branch information
Showing
10 changed files
with
497 additions
and
10 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,108 @@ | ||
package kafka | ||
|
||
import ( | ||
"context" | ||
"fmt" | ||
"net" | ||
"time" | ||
|
||
"github.com/segmentio/kafka-go/protocol/addpartitionstotxn" | ||
) | ||
|
||
// AddPartitionToTxn represents a partition to be added | ||
// to a transaction. | ||
type AddPartitionToTxn struct { | ||
// Partition is the ID of a partition to add to the transaction. | ||
Partition int | ||
} | ||
|
||
// AddPartitionsToTxnRequest is the request structure fo the AddPartitionsToTxn function. | ||
type AddPartitionsToTxnRequest struct { | ||
// Address of the kafka broker to send the request to. | ||
Addr net.Addr | ||
|
||
// The transactional id key | ||
TransactionalID string | ||
|
||
// The Producer ID (PID) for the current producer session; | ||
// received from an InitProducerID request. | ||
ProducerID int | ||
|
||
// The epoch associated with the current producer session for the given PID | ||
ProducerEpoch int | ||
|
||
// Mappings of topic names to lists of partitions. | ||
Topics map[string][]AddPartitionToTxn | ||
} | ||
|
||
// AddPartitioinsToTxnResponse is the response structure for the AddPartitioinsToTxn function. | ||
type AddPartitioinsToTxnResponse struct { | ||
// The amount of time that the broker throttled the request. | ||
Throttle time.Duration | ||
|
||
// Mappings of topic names to partitions being added to a transactions. | ||
Topics map[string][]AddPartitionToTxnPartition | ||
} | ||
|
||
// AddPartitionToTxnPartition represents the state of a single partition | ||
// in response to adding to a transaction. | ||
type AddPartitionToTxnPartition struct { | ||
// The ID of the partition. | ||
Partition int | ||
|
||
// An error that may have occured when attempting to add the partition | ||
// to a transaction. | ||
// | ||
// The errors contain the kafka error code. Programs may use the standard | ||
// errors.Is function to test the error against kafka error codes. | ||
Error error | ||
} | ||
|
||
// AddPartitionsToTnx sends an add partitions to txn request to a kafka broker and returns the response. | ||
func (c *Client) AddPartitionsToTxn( | ||
ctx context.Context, | ||
req *AddPartitionsToTxnRequest, | ||
) (*AddPartitioinsToTxnResponse, error) { | ||
protoReq := &addpartitionstotxn.Request{ | ||
TransactionalID: req.TransactionalID, | ||
ProducerID: int64(req.ProducerID), | ||
ProducerEpoch: int16(req.ProducerEpoch), | ||
} | ||
protoReq.Topics = make([]addpartitionstotxn.RequestTopic, 0, len(req.Topics)) | ||
|
||
for topic, partitions := range req.Topics { | ||
reqTopic := addpartitionstotxn.RequestTopic{ | ||
Name: topic, | ||
Partitions: make([]int32, len(partitions)), | ||
} | ||
for i, partition := range partitions { | ||
reqTopic.Partitions[i] = int32(partition.Partition) | ||
} | ||
protoReq.Topics = append(protoReq.Topics, reqTopic) | ||
} | ||
|
||
m, err := c.roundTrip(ctx, req.Addr, protoReq) | ||
if err != nil { | ||
return nil, fmt.Errorf("kafka.(*Client).AddPartitionsToTxn: %w", err) | ||
} | ||
|
||
r := m.(*addpartitionstotxn.Response) | ||
|
||
res := &AddPartitioinsToTxnResponse{ | ||
Throttle: makeDuration(r.ThrottleTimeMs), | ||
Topics: make(map[string][]AddPartitionToTxnPartition, len(r.Results)), | ||
} | ||
|
||
for _, result := range r.Results { | ||
partitions := make([]AddPartitionToTxnPartition, 0, len(result.Results)) | ||
for _, rp := range result.Results { | ||
partitions = append(partitions, AddPartitionToTxnPartition{ | ||
Partition: int(rp.PartitionIndex), | ||
Error: makeError(rp.ErrorCode, ""), | ||
}) | ||
} | ||
res.Topics[result.Name] = partitions | ||
} | ||
|
||
return res, nil | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,126 @@ | ||
package kafka | ||
|
||
import ( | ||
"context" | ||
"net" | ||
"strconv" | ||
"testing" | ||
"time" | ||
|
||
ktesting "github.com/segmentio/kafka-go/testing" | ||
) | ||
|
||
func TestClientAddPartitionsToTxn(t *testing.T) { | ||
if !ktesting.KafkaIsAtLeast("0.11.0") { | ||
t.Skip("Skipping test because kafka version is not high enough.") | ||
} | ||
topic1 := makeTopic() | ||
topic2 := makeTopic() | ||
|
||
client, shutdown := newLocalClient() | ||
defer shutdown() | ||
|
||
err := clientCreateTopic(client, topic1, 3) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
err = clientCreateTopic(client, topic2, 3) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
transactionalID := makeTransactionalID() | ||
|
||
ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) | ||
defer cancel() | ||
respc, err := waitForCoordinatorIndefinitely(ctx, client, &FindCoordinatorRequest{ | ||
Addr: client.Addr, | ||
Key: transactionalID, | ||
KeyType: CoordinatorKeyTypeTransaction, | ||
}) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
transactionCoordinator := TCP(net.JoinHostPort(respc.Coordinator.Host, strconv.Itoa(int(respc.Coordinator.Port)))) | ||
client, shutdown = newClient(transactionCoordinator) | ||
defer shutdown() | ||
|
||
ipResp, err := client.InitProducerID(ctx, &InitProducerIDRequest{ | ||
TransactionalID: transactionalID, | ||
TransactionTimeoutMs: 10000, | ||
}) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
if ipResp.Error != nil { | ||
t.Fatal(ipResp.Error) | ||
} | ||
|
||
defer func() { | ||
err := clientEndTxn(client, &EndTxnRequest{ | ||
TransactionalID: transactionalID, | ||
ProducerID: ipResp.Producer.ProducerID, | ||
ProducerEpoch: ipResp.Producer.ProducerID, | ||
Committed: false, | ||
}) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
}() | ||
|
||
ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) | ||
defer cancel() | ||
resp, err := client.AddPartitionsToTxn(ctx, &AddPartitionsToTxnRequest{ | ||
TransactionalID: transactionalID, | ||
ProducerID: ipResp.Producer.ProducerID, | ||
ProducerEpoch: ipResp.Producer.ProducerEpoch, | ||
Topics: map[string][]AddPartitionToTxn{ | ||
topic1: { | ||
{ | ||
Partition: 0, | ||
}, | ||
{ | ||
Partition: 1, | ||
}, | ||
{ | ||
Partition: 2, | ||
}, | ||
}, | ||
topic2: { | ||
{ | ||
Partition: 0, | ||
}, | ||
{ | ||
Partition: 2, | ||
}, | ||
}, | ||
}, | ||
}) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
if len(resp.Topics) != 2 { | ||
t.Errorf("expected responses for 2 topics; got: %d", len(resp.Topics)) | ||
} | ||
for topic, partitions := range resp.Topics { | ||
if topic == topic1 { | ||
if len(partitions) != 3 { | ||
t.Errorf("expected 3 partitions in response for topic %s; got: %d", topic, len(partitions)) | ||
} | ||
} | ||
if topic == topic2 { | ||
if len(partitions) != 2 { | ||
t.Errorf("expected 2 partitions in response for topic %s; got: %d", topic, len(partitions)) | ||
} | ||
} | ||
for _, partition := range partitions { | ||
if partition.Error != nil { | ||
t.Error(partition.Error) | ||
} | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,61 @@ | ||
package kafka | ||
|
||
import ( | ||
"context" | ||
"fmt" | ||
"net" | ||
"time" | ||
|
||
"github.com/segmentio/kafka-go/protocol/endtxn" | ||
) | ||
|
||
// EndTxnRequest represets a request sent to a kafka broker to end a transaction. | ||
type EndTxnRequest struct { | ||
// Address of the kafka broker to send the request to. | ||
Addr net.Addr | ||
|
||
// The transactional id key. | ||
TransactionalID string | ||
|
||
// The Producer ID (PID) for the current producer session | ||
ProducerID int | ||
|
||
// The epoch associated with the current producer session for the given PID | ||
ProducerEpoch int | ||
|
||
// Committed should be set to true if the transaction was commited, false otherwise. | ||
Committed bool | ||
} | ||
|
||
// EndTxnResponse represents a resposne from a kafka broker to an end transaction request. | ||
type EndTxnResponse struct { | ||
// The amount of time that the broker throttled the request. | ||
Throttle time.Duration | ||
|
||
// Error is non-nil if an error occureda and contains the kafka error code. | ||
// Programs may use the standard errors.Is function to test the error | ||
// against kafka error codes. | ||
Error error | ||
} | ||
|
||
// EndTxn sends an EndTxn request to a kafka broker and returns its response. | ||
func (c *Client) EndTxn(ctx context.Context, req *EndTxnRequest) (*EndTxnResponse, error) { | ||
m, err := c.roundTrip(ctx, req.Addr, &endtxn.Request{ | ||
TransactionalID: req.TransactionalID, | ||
ProducerID: int64(req.ProducerID), | ||
ProducerEpoch: int16(req.ProducerEpoch), | ||
Committed: req.Committed, | ||
}) | ||
if err != nil { | ||
return nil, fmt.Errorf("kafka.(*Client).EndTxn: %w", err) | ||
} | ||
|
||
r := m.(*endtxn.Response) | ||
|
||
res := &EndTxnResponse{ | ||
Throttle: makeDuration(r.ThrottleTimeMs), | ||
Error: makeError(r.ErrorCode, ""), | ||
} | ||
|
||
return res, nil | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.