forked from segmentio/kafka-go
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add AddOffsetsToTxn function to Client (segmentio#739)
Implement the AddOffsetsToTxn protocol and request types. Add Client function AddOffsetsToTxn.
- Loading branch information
Showing
5 changed files
with
249 additions
and
1 deletion.
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,67 @@ | ||
package kafka | ||
|
||
import ( | ||
"context" | ||
"fmt" | ||
"net" | ||
"time" | ||
|
||
"github.com/segmentio/kafka-go/protocol/addoffsetstotxn" | ||
) | ||
|
||
// AddOffsetsToTxnRequest is the request structure for the AddOffsetsToTxn function. | ||
type AddOffsetsToTxnRequest 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 | ||
|
||
// The unique group identifier. | ||
GroupID string | ||
} | ||
|
||
// AddOffsetsToTxnResponse is the response structure for the AddOffsetsToTxn function. | ||
type AddOffsetsToTxnResponse struct { | ||
// The amount of time that the broker throttled the request. | ||
Throttle time.Duration | ||
|
||
// An error that may have occured when attempting to add the offsets | ||
// 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 | ||
} | ||
|
||
// AddOffsetsToTnx sends an add offsets to txn request to a kafka broker and returns the response. | ||
func (c *Client) AddOffsetsToTxn( | ||
ctx context.Context, | ||
req *AddOffsetsToTxnRequest, | ||
) (*AddOffsetsToTxnResponse, error) { | ||
m, err := c.roundTrip(ctx, req.Addr, &addoffsetstotxn.Request{ | ||
TransactionalID: req.TransactionalID, | ||
ProducerID: int64(req.ProducerID), | ||
ProducerEpoch: int16(req.ProducerEpoch), | ||
GroupID: req.GroupID, | ||
}) | ||
if err != nil { | ||
return nil, fmt.Errorf("kafka.(*Client).AddOffsetsToTxn: %w", err) | ||
} | ||
|
||
r := m.(*addoffsetstotxn.Response) | ||
|
||
res := &AddOffsetsToTxnResponse{ | ||
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
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,122 @@ | ||
package kafka | ||
|
||
import ( | ||
"context" | ||
"log" | ||
"net" | ||
"os" | ||
"strconv" | ||
"testing" | ||
"time" | ||
|
||
ktesting "github.com/segmentio/kafka-go/testing" | ||
) | ||
|
||
func TestClientAddOffsetsToTxn(t *testing.T) { | ||
if !ktesting.KafkaIsAtLeast("0.11.0") { | ||
t.Skip("Skipping test because kafka version is not high enough.") | ||
} | ||
topic := makeTopic() | ||
transactionalID := makeTransactionalID() | ||
client, shutdown := newLocalClient() | ||
defer shutdown() | ||
|
||
err := clientCreateTopic(client, topic, 3) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) | ||
defer cancel() | ||
respc, err := waitForCoordinatorIndefinitely(ctx, client, &FindCoordinatorRequest{ | ||
Addr: client.Addr, | ||
Key: transactionalID, | ||
KeyType: CoordinatorKeyTypeConsumer, | ||
}) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
if respc.Error != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
groupID := makeGroupID() | ||
|
||
group, err := NewConsumerGroup(ConsumerGroupConfig{ | ||
ID: groupID, | ||
Topics: []string{topic}, | ||
Brokers: []string{"localhost:9092"}, | ||
HeartbeatInterval: 2 * time.Second, | ||
RebalanceTimeout: 2 * time.Second, | ||
RetentionTime: time.Hour, | ||
Logger: log.New(os.Stdout, "cg-test: ", 0), | ||
}) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
defer group.Close() | ||
|
||
ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) | ||
defer cancel() | ||
_, err = group.Next(ctx) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
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.ProducerEpoch, | ||
Committed: false, | ||
}) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
}() | ||
|
||
ctx, cancel = context.WithTimeout(context.Background(), time.Second*30) | ||
defer cancel() | ||
|
||
resp, err := client.AddOffsetsToTxn(ctx, &AddOffsetsToTxnRequest{ | ||
TransactionalID: transactionalID, | ||
ProducerID: ipResp.Producer.ProducerID, | ||
ProducerEpoch: ipResp.Producer.ProducerEpoch, | ||
GroupID: groupID, | ||
}) | ||
if err != nil { | ||
t.Fatal(err) | ||
} | ||
|
||
if resp.Error != nil { | ||
t.Fatal(err) | ||
} | ||
} |
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,31 @@ | ||
package addoffsetstotxn | ||
|
||
import "github.com/segmentio/kafka-go/protocol" | ||
|
||
func init() { | ||
protocol.Register(&Request{}, &Response{}) | ||
} | ||
|
||
type Request struct { | ||
// We need at least one tagged field to indicate that this is a "flexible" message | ||
// type. | ||
_ struct{} `kafka:"min=v3,max=v3,tag"` | ||
|
||
TransactionalID string `kafka:"min=v0,max=v2|min=v3,max=v3,compact"` | ||
ProducerID int64 `kafka:"min=v0,max=v3"` | ||
ProducerEpoch int16 `kafka:"min=v0,max=v3"` | ||
GroupID string `kafka:"min=v0,max=v3|min=v3,max=v3,compact"` | ||
} | ||
|
||
func (r *Request) ApiKey() protocol.ApiKey { return protocol.AddOffsetsToTxn } | ||
|
||
type Response struct { | ||
// We need at least one tagged field to indicate that this is a "flexible" message | ||
// type. | ||
_ struct{} `kafka:"min=v3,max=v3,tag"` | ||
|
||
ThrottleTimeMs int32 `kafka:"min=v0,max=v3"` | ||
ErrorCode int16 `kafka:"min=v0,max=v3"` | ||
} | ||
|
||
func (r *Response) ApiKey() protocol.ApiKey { return protocol.AddOffsetsToTxn } |
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,28 @@ | ||
package addoffsetstotxn_test | ||
|
||
import ( | ||
"testing" | ||
|
||
"github.com/segmentio/kafka-go/protocol/addoffsetstotxn" | ||
"github.com/segmentio/kafka-go/protocol/prototest" | ||
) | ||
|
||
func TestAddOffsetsToTxnRequest(t *testing.T) { | ||
for _, version := range []int16{0, 1, 2, 3} { | ||
prototest.TestRequest(t, version, &addoffsetstotxn.Request{ | ||
TransactionalID: "transactional-id-0", | ||
ProducerID: 1, | ||
ProducerEpoch: 10, | ||
GroupID: "group-id-0", | ||
}) | ||
} | ||
} | ||
|
||
func TestAddOffsetsToTxnResponse(t *testing.T) { | ||
for _, version := range []int16{0, 1, 2, 3} { | ||
prototest.TestResponse(t, version, &addoffsetstotxn.Response{ | ||
ThrottleTimeMs: 10, | ||
ErrorCode: 1, | ||
}) | ||
} | ||
} |