Skip to content

Commit 6193fa9

Browse files
support userscramcredentials apis (#1168)
* userscramcredentials protocols * alteruserscramcredentials working * describeuserscramcredentials working * gofmt -s -w alteruserscramcredentials_test.go * fix typo * add tests for deletion * gofmt * improve test * separate alteruserscramcredentials_test and describeuserscramcredentials_test * add protocol tests * remove unused v1 constant * change iterations from int32 to int * keep errors with results
1 parent 861e102 commit 6193fa9

9 files changed

+739
-102
lines changed

Diff for: alteruserscramcredentials.go

+107
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,107 @@
1+
package kafka
2+
3+
import (
4+
"context"
5+
"fmt"
6+
"net"
7+
"time"
8+
9+
"github.com/segmentio/kafka-go/protocol/alteruserscramcredentials"
10+
)
11+
12+
// AlterUserScramCredentialsRequest represents a request sent to a kafka broker to
13+
// alter user scram credentials.
14+
type AlterUserScramCredentialsRequest struct {
15+
// Address of the kafka broker to send the request to.
16+
Addr net.Addr
17+
18+
// List of credentials to delete.
19+
Deletions []UserScramCredentialsDeletion
20+
21+
// List of credentials to upsert.
22+
Upsertions []UserScramCredentialsUpsertion
23+
}
24+
25+
type ScramMechanism int8
26+
27+
const (
28+
ScramMechanismUnknown ScramMechanism = iota // 0
29+
ScramMechanismSha256 // 1
30+
ScramMechanismSha512 // 2
31+
)
32+
33+
type UserScramCredentialsDeletion struct {
34+
Name string
35+
Mechanism ScramMechanism
36+
}
37+
38+
type UserScramCredentialsUpsertion struct {
39+
Name string
40+
Mechanism ScramMechanism
41+
Iterations int
42+
Salt []byte
43+
SaltedPassword []byte
44+
}
45+
46+
// AlterUserScramCredentialsResponse represents a response from a kafka broker to an alter user
47+
// credentials request.
48+
type AlterUserScramCredentialsResponse struct {
49+
// The amount of time that the broker throttled the request.
50+
Throttle time.Duration
51+
52+
// List of altered user scram credentials.
53+
Results []AlterUserScramCredentialsResponseUser
54+
}
55+
56+
type AlterUserScramCredentialsResponseUser struct {
57+
User string
58+
Error error
59+
}
60+
61+
// AlterUserScramCredentials sends user scram credentials alteration request to a kafka broker and returns
62+
// the response.
63+
func (c *Client) AlterUserScramCredentials(ctx context.Context, req *AlterUserScramCredentialsRequest) (*AlterUserScramCredentialsResponse, error) {
64+
deletions := make([]alteruserscramcredentials.RequestUserScramCredentialsDeletion, len(req.Deletions))
65+
upsertions := make([]alteruserscramcredentials.RequestUserScramCredentialsUpsertion, len(req.Upsertions))
66+
67+
for deletionIdx, deletion := range req.Deletions {
68+
deletions[deletionIdx] = alteruserscramcredentials.RequestUserScramCredentialsDeletion{
69+
Name: deletion.Name,
70+
Mechanism: int8(deletion.Mechanism),
71+
}
72+
}
73+
74+
for upsertionIdx, upsertion := range req.Upsertions {
75+
upsertions[upsertionIdx] = alteruserscramcredentials.RequestUserScramCredentialsUpsertion{
76+
Name: upsertion.Name,
77+
Mechanism: int8(upsertion.Mechanism),
78+
Iterations: int32(upsertion.Iterations),
79+
Salt: upsertion.Salt,
80+
SaltedPassword: upsertion.SaltedPassword,
81+
}
82+
}
83+
84+
m, err := c.roundTrip(ctx, req.Addr, &alteruserscramcredentials.Request{
85+
Deletions: deletions,
86+
Upsertions: upsertions,
87+
})
88+
if err != nil {
89+
return nil, fmt.Errorf("kafka.(*Client).AlterUserScramCredentials: %w", err)
90+
}
91+
92+
res := m.(*alteruserscramcredentials.Response)
93+
responseEntries := make([]AlterUserScramCredentialsResponseUser, len(res.Results))
94+
95+
for responseIdx, responseResult := range res.Results {
96+
responseEntries[responseIdx] = AlterUserScramCredentialsResponseUser{
97+
User: responseResult.User,
98+
Error: makeError(responseResult.ErrorCode, responseResult.ErrorMessage),
99+
}
100+
}
101+
ret := &AlterUserScramCredentialsResponse{
102+
Throttle: makeDuration(res.ThrottleTimeMs),
103+
Results: responseEntries,
104+
}
105+
106+
return ret, nil
107+
}

Diff for: alteruserscramcredentials_test.go

+73
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,73 @@
1+
package kafka
2+
3+
import (
4+
"context"
5+
"testing"
6+
7+
ktesting "github.com/segmentio/kafka-go/testing"
8+
)
9+
10+
func TestAlterUserScramCredentials(t *testing.T) {
11+
// https://issues.apache.org/jira/browse/KAFKA-10259
12+
if !ktesting.KafkaIsAtLeast("2.7.0") {
13+
return
14+
}
15+
16+
client, shutdown := newLocalClient()
17+
defer shutdown()
18+
19+
name := makeTopic()
20+
21+
createRes, err := client.AlterUserScramCredentials(context.Background(), &AlterUserScramCredentialsRequest{
22+
Upsertions: []UserScramCredentialsUpsertion{
23+
{
24+
Name: name,
25+
Mechanism: ScramMechanismSha512,
26+
Iterations: 15000,
27+
Salt: []byte("my-salt"),
28+
SaltedPassword: []byte("my-salted-password"),
29+
},
30+
},
31+
})
32+
33+
if err != nil {
34+
t.Fatal(err)
35+
}
36+
37+
if len(createRes.Results) != 1 {
38+
t.Fatalf("expected 1 createResult; got %d", len(createRes.Results))
39+
}
40+
41+
if createRes.Results[0].User != name {
42+
t.Fatalf("expected createResult with user: %s, got %s", name, createRes.Results[0].User)
43+
}
44+
45+
if createRes.Results[0].Error != nil {
46+
t.Fatalf("didn't expect an error in createResult, got %v", createRes.Results[0].Error)
47+
}
48+
49+
deleteRes, err := client.AlterUserScramCredentials(context.Background(), &AlterUserScramCredentialsRequest{
50+
Deletions: []UserScramCredentialsDeletion{
51+
{
52+
Name: name,
53+
Mechanism: ScramMechanismSha512,
54+
},
55+
},
56+
})
57+
58+
if err != nil {
59+
t.Fatal(err)
60+
}
61+
62+
if len(deleteRes.Results) != 1 {
63+
t.Fatalf("expected 1 deleteResult; got %d", len(deleteRes.Results))
64+
}
65+
66+
if deleteRes.Results[0].User != name {
67+
t.Fatalf("expected deleteResult with user: %s, got %s", name, deleteRes.Results[0].User)
68+
}
69+
70+
if deleteRes.Results[0].Error != nil {
71+
t.Fatalf("didn't expect an error in deleteResult, got %v", deleteRes.Results[0].Error)
72+
}
73+
}

Diff for: describeuserscramcredentials.go

+97
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,97 @@
1+
package kafka
2+
3+
import (
4+
"context"
5+
"fmt"
6+
"net"
7+
"time"
8+
9+
"github.com/segmentio/kafka-go/protocol/describeuserscramcredentials"
10+
)
11+
12+
// DescribeUserScramCredentialsRequest represents a request sent to a kafka broker to
13+
// describe user scram credentials.
14+
type DescribeUserScramCredentialsRequest struct {
15+
// Address of the kafka broker to send the request to.
16+
Addr net.Addr
17+
18+
// List of Scram users to describe
19+
Users []UserScramCredentialsUser
20+
}
21+
22+
type UserScramCredentialsUser struct {
23+
Name string
24+
}
25+
26+
// DescribeUserScramCredentialsResponse represents a response from a kafka broker to a describe user
27+
// credentials request.
28+
type DescribeUserScramCredentialsResponse struct {
29+
// The amount of time that the broker throttled the request.
30+
Throttle time.Duration
31+
32+
// Top level error that occurred while attempting to describe
33+
// the user scram credentials.
34+
//
35+
// The errors contain the kafka error code. Programs may use the standard
36+
// errors.Is function to test the error against kafka error codes.
37+
Error error
38+
39+
// List of described user scram credentials.
40+
Results []DescribeUserScramCredentialsResponseResult
41+
}
42+
43+
type DescribeUserScramCredentialsResponseResult struct {
44+
User string
45+
CredentialInfos []DescribeUserScramCredentialsCredentialInfo
46+
Error error
47+
}
48+
49+
type DescribeUserScramCredentialsCredentialInfo struct {
50+
Mechanism ScramMechanism
51+
Iterations int
52+
}
53+
54+
// DescribeUserScramCredentials sends a user scram credentials describe request to a kafka broker and returns
55+
// the response.
56+
func (c *Client) DescribeUserScramCredentials(ctx context.Context, req *DescribeUserScramCredentialsRequest) (*DescribeUserScramCredentialsResponse, error) {
57+
users := make([]describeuserscramcredentials.RequestUser, len(req.Users))
58+
59+
for userIdx, user := range req.Users {
60+
users[userIdx] = describeuserscramcredentials.RequestUser{
61+
Name: user.Name,
62+
}
63+
}
64+
65+
m, err := c.roundTrip(ctx, req.Addr, &describeuserscramcredentials.Request{
66+
Users: users,
67+
})
68+
if err != nil {
69+
return nil, fmt.Errorf("kafka.(*Client).DescribeUserScramCredentials: %w", err)
70+
}
71+
72+
res := m.(*describeuserscramcredentials.Response)
73+
responseResults := make([]DescribeUserScramCredentialsResponseResult, len(res.Results))
74+
75+
for responseIdx, responseResult := range res.Results {
76+
credentialInfos := make([]DescribeUserScramCredentialsCredentialInfo, len(responseResult.CredentialInfos))
77+
78+
for credentialInfoIdx, credentialInfo := range responseResult.CredentialInfos {
79+
credentialInfos[credentialInfoIdx] = DescribeUserScramCredentialsCredentialInfo{
80+
Mechanism: ScramMechanism(credentialInfo.Mechanism),
81+
Iterations: int(credentialInfo.Iterations),
82+
}
83+
}
84+
responseResults[responseIdx] = DescribeUserScramCredentialsResponseResult{
85+
User: responseResult.User,
86+
CredentialInfos: credentialInfos,
87+
Error: makeError(responseResult.ErrorCode, responseResult.ErrorMessage),
88+
}
89+
}
90+
ret := &DescribeUserScramCredentialsResponse{
91+
Throttle: makeDuration(res.ThrottleTimeMs),
92+
Error: makeError(res.ErrorCode, res.ErrorMessage),
93+
Results: responseResults,
94+
}
95+
96+
return ret, nil
97+
}

0 commit comments

Comments
 (0)