From 7988d30e3c2d092dc1132d5087dd0a4cdcef3254 Mon Sep 17 00:00:00 2001 From: DCjanus Date: Wed, 26 Nov 2025 17:12:18 +0800 Subject: [PATCH 01/17] feat(admin): add describe cluster api support Signed-off-by: DCjanus --- admin.go | 63 ++++++++++- api_versions.go | 1 + broker.go | 11 ++ describe_cluster_request.go | 107 ++++++++++++++++++ describe_cluster_request_test.go | 54 +++++++++ describe_cluster_response.go | 176 ++++++++++++++++++++++++++++++ describe_cluster_response_test.go | 28 +++++ request.go | 2 + request_test.go | 4 +- 9 files changed, 444 insertions(+), 2 deletions(-) create mode 100644 describe_cluster_request.go create mode 100644 describe_cluster_request_test.go create mode 100644 describe_cluster_response.go create mode 100644 describe_cluster_response_test.go diff --git a/admin.go b/admin.go index 2568d14ce..f32a4fb90 100644 --- a/admin.go +++ b/admin.go @@ -6,6 +6,7 @@ import ( "io" "maps" "math/rand" + "net" "strconv" "sync" "time" @@ -308,6 +309,51 @@ func (ca *clusterAdmin) DescribeTopics(topics []string) (metadata []*TopicMetada } func (ca *clusterAdmin) DescribeCluster() (brokers []*Broker, controllerID int32, err error) { + if ca.conf.Version.IsAtLeast(V3_0_0_0) { + brokers, controllerID, err = ca.describeClusterUsingAPI() + if err == nil { + return brokers, controllerID, nil + } + if !errors.Is(err, ErrUnsupportedVersion) { + return nil, 0, err + } + } + return ca.describeClusterUsingMetadata() +} + +func (ca *clusterAdmin) describeClusterUsingAPI() (brokers []*Broker, controllerID int32, err error) { + var response *DescribeClusterResponse + err = ca.retryOnError(isRetriableControllerError, func() error { + controller, err := ca.Controller() + if err != nil { + return err + } + + request := NewDescribeClusterRequest(ca.conf.Version) + response, err = controller.DescribeCluster(request) + if err != nil { + return err + } + if !errors.Is(response.Err, ErrNoError) { + if isRetriableControllerError(response.Err) { + _, _ = ca.refreshController() + } + if response.ErrorMessage != nil && *response.ErrorMessage != "" { + return fmt.Errorf("%w: %s", response.Err, *response.ErrorMessage) + } + return response.Err + } + return nil + }) + if err != nil { + return nil, 0, err + } + + brokers = convertDescribeClusterBrokers(response.Brokers) + return brokers, response.ControllerID, nil +} + +func (ca *clusterAdmin) describeClusterUsingMetadata() (brokers []*Broker, controllerID int32, err error) { var response *MetadataResponse err = ca.retryOnError(isRetriableControllerError, func() error { controller, err := ca.Controller() @@ -323,12 +369,27 @@ func (ca *clusterAdmin) DescribeCluster() (brokers []*Broker, controllerID int32 return err }) if err != nil { - return nil, int32(0), err + return nil, 0, err } return response.Brokers, response.ControllerID, nil } +func convertDescribeClusterBrokers(entries []*DescribeClusterBroker) []*Broker { + if len(entries) == 0 { + return nil + } + result := make([]*Broker, 0, len(entries)) + for _, info := range entries { + addr := net.JoinHostPort(info.Host, strconv.Itoa(int(info.Port))) + b := NewBroker(addr) + b.id = info.BrokerID + b.rack = info.Rack + result = append(result, b) + } + return result +} + func (ca *clusterAdmin) findBroker(id int32) (*Broker, error) { brokers := ca.client.Brokers() for _, b := range brokers { diff --git a/api_versions.go b/api_versions.go index e993432d0..cf0d0b871 100644 --- a/api_versions.go +++ b/api_versions.go @@ -81,4 +81,5 @@ const ( apiKeyAlterClientQuotas = 49 apiKeyDescribeUserScramCredentials = 50 apiKeyAlterUserScramCredentials = 51 + apiKeyDescribeCluster = 60 ) diff --git a/broker.go b/broker.go index 7c559cfcb..c56146eb5 100644 --- a/broker.go +++ b/broker.go @@ -400,6 +400,17 @@ func (b *Broker) GetMetadata(request *MetadataRequest) (*MetadataResponse, error return response, nil } +func (b *Broker) DescribeCluster(request *DescribeClusterRequest) (*DescribeClusterResponse, error) { + response := new(DescribeClusterResponse) + response.Version = request.Version + + if err := b.sendAndReceive(request, response); err != nil { + return nil, err + } + + return response, nil +} + // GetConsumerMetadata send a consumer metadata request and returns a consumer metadata response or error func (b *Broker) GetConsumerMetadata(request *ConsumerMetadataRequest) (*ConsumerMetadataResponse, error) { response := new(ConsumerMetadataResponse) diff --git a/describe_cluster_request.go b/describe_cluster_request.go new file mode 100644 index 000000000..189583c12 --- /dev/null +++ b/describe_cluster_request.go @@ -0,0 +1,107 @@ +package sarama + +// DescribeClusterEndpointType enumerates the DescribeCluster endpoint types +// defined by the Kafka protocol. +const ( + DescribeClusterEndpointTypeBrokers int8 = 1 + DescribeClusterEndpointTypeControllers int8 = 2 +) + +type DescribeClusterRequest struct { + Version int16 + + IncludeClusterAuthorizedOperations bool + EndpointType int8 + IncludeFencedBrokers bool +} + +func NewDescribeClusterRequest(version KafkaVersion) *DescribeClusterRequest { + req := &DescribeClusterRequest{} + + switch { + case version.IsAtLeast(V4_0_0_0): + req.Version = 2 + case version.IsAtLeast(V3_7_0_0): + req.Version = 1 + default: + req.Version = 0 + } + + if req.Version >= 1 { + req.EndpointType = DescribeClusterEndpointTypeBrokers + } + if req.Version >= 2 { + req.IncludeFencedBrokers = true + } + + return req +} + +func (r *DescribeClusterRequest) encode(pe packetEncoder) error { + if r.Version < 0 || r.Version > 2 { + return PacketEncodingError{"invalid or unsupported DescribeClusterRequest version"} + } + + pe.putBool(r.IncludeClusterAuthorizedOperations) + if r.Version >= 1 { + pe.putInt8(r.EndpointType) + } + if r.Version >= 2 { + pe.putBool(r.IncludeFencedBrokers) + } + pe.putEmptyTaggedFieldArray() + + return nil +} + +func (r *DescribeClusterRequest) decode(pd packetDecoder, version int16) error { + r.Version = version + if r.Version < 0 || r.Version > 2 { + return PacketDecodingError{"invalid or unsupported DescribeClusterRequest version"} + } + + var err error + if r.IncludeClusterAuthorizedOperations, err = pd.getBool(); err != nil { + return err + } + if r.Version >= 1 { + if r.EndpointType, err = pd.getInt8(); err != nil { + return err + } + } + if r.Version >= 2 { + if r.IncludeFencedBrokers, err = pd.getBool(); err != nil { + return err + } + } + + _, err = pd.getEmptyTaggedFieldArray() + return err +} + +func (r *DescribeClusterRequest) key() int16 { return apiKeyDescribeCluster } + +func (r *DescribeClusterRequest) version() int16 { return r.Version } + +func (r *DescribeClusterRequest) setVersion(v int16) { r.Version = v } + +func (r *DescribeClusterRequest) headerVersion() int16 { return 2 } + +func (r *DescribeClusterRequest) isValidVersion() bool { + return r.Version >= 0 && r.Version <= 2 +} + +func (r *DescribeClusterRequest) isFlexible() bool { return true } + +func (r *DescribeClusterRequest) isFlexibleVersion(version int16) bool { return version >= 0 } + +func (r *DescribeClusterRequest) requiredVersion() KafkaVersion { + switch r.Version { + case 2: + return V4_0_0_0 + case 1: + return V3_7_0_0 + default: + return V3_0_0_0 + } +} diff --git a/describe_cluster_request_test.go b/describe_cluster_request_test.go new file mode 100644 index 000000000..cc4b0ba4a --- /dev/null +++ b/describe_cluster_request_test.go @@ -0,0 +1,54 @@ +//go:build !functional + +package sarama + +import "testing" + +func TestDescribeClusterRequestVersions(t *testing.T) { + t.Run("v0", func(t *testing.T) { + req := &DescribeClusterRequest{Version: 0, IncludeClusterAuthorizedOperations: true} + testRequestWithoutByteComparison(t, "DescribeClusterRequest v0", req) + }) + + t.Run("v1", func(t *testing.T) { + req := &DescribeClusterRequest{ + Version: 1, + EndpointType: DescribeClusterEndpointTypeControllers, + } + testRequestWithoutByteComparison(t, "DescribeClusterRequest v1", req) + }) + + t.Run("v2", func(t *testing.T) { + req := &DescribeClusterRequest{ + Version: 2, + EndpointType: DescribeClusterEndpointTypeBrokers, + IncludeFencedBrokers: true, + IncludeClusterAuthorizedOperations: true, + } + testRequestWithoutByteComparison(t, "DescribeClusterRequest v2", req) + }) +} + +func TestNewDescribeClusterRequest(t *testing.T) { + testCases := []struct { + version KafkaVersion + expectedVer int16 + }{ + {V3_0_0_0, 0}, + {V3_7_0_0, 1}, + {V4_0_0_0, 2}, + } + + for _, tc := range testCases { + req := NewDescribeClusterRequest(tc.version) + if req.Version != tc.expectedVer { + t.Fatalf("version mismatch for %v: got %d", tc.version, req.Version) + } + if req.Version >= 1 && req.EndpointType == 0 { + t.Fatalf("endpoint type not set for version %d", req.Version) + } + if req.Version < 2 && req.IncludeFencedBrokers { + t.Fatalf("include fenced brokers unexpectedly enabled for version %d", req.Version) + } + } +} diff --git a/describe_cluster_response.go b/describe_cluster_response.go new file mode 100644 index 000000000..7f0b5c685 --- /dev/null +++ b/describe_cluster_response.go @@ -0,0 +1,176 @@ +package sarama + +import "time" + +type DescribeClusterResponse struct { + Version int16 + + ThrottleTimeMs int32 + Err KError + ErrorMessage *string + EndpointType int8 + ClusterID string + ControllerID int32 + Brokers []*DescribeClusterBroker + + ClusterAuthorizedOperations int32 +} + +type DescribeClusterBroker struct { + BrokerID int32 + Host string + Port int32 + Rack *string + IsFenced bool +} + +func (r *DescribeClusterResponse) encode(pe packetEncoder) error { + if r.Version < 0 || r.Version > 2 { + return PacketEncodingError{"invalid or unsupported DescribeClusterResponse version"} + } + + pe.putInt32(r.ThrottleTimeMs) + pe.putInt16(int16(r.Err)) + if err := pe.putNullableString(r.ErrorMessage); err != nil { + return err + } + if r.Version >= 1 { + pe.putInt8(r.EndpointType) + } + if err := pe.putString(r.ClusterID); err != nil { + return err + } + pe.putInt32(r.ControllerID) + + if err := pe.putArrayLength(len(r.Brokers)); err != nil { + return err + } + for _, broker := range r.Brokers { + if err := broker.encode(pe, r.Version); err != nil { + return err + } + } + + pe.putInt32(r.ClusterAuthorizedOperations) + pe.putEmptyTaggedFieldArray() + return nil +} + +func (r *DescribeClusterResponse) decode(pd packetDecoder, version int16) error { + r.Version = version + if r.Version < 0 || r.Version > 2 { + return PacketDecodingError{"invalid or unsupported DescribeClusterResponse version"} + } + + var err error + if r.ThrottleTimeMs, err = pd.getInt32(); err != nil { + return err + } + if r.Err, err = pd.getKError(); err != nil { + return err + } + if r.ErrorMessage, err = pd.getNullableString(); err != nil { + return err + } + if r.Version >= 1 { + if r.EndpointType, err = pd.getInt8(); err != nil { + return err + } + } + if r.ClusterID, err = pd.getString(); err != nil { + return err + } + if r.ControllerID, err = pd.getInt32(); err != nil { + return err + } + + brokerCount, err := pd.getArrayLength() + if err != nil { + return err + } + r.Brokers = make([]*DescribeClusterBroker, brokerCount) + for i := 0; i < brokerCount; i++ { + broker := &DescribeClusterBroker{} + if err := broker.decode(pd, r.Version); err != nil { + return err + } + r.Brokers[i] = broker + } + + if r.ClusterAuthorizedOperations, err = pd.getInt32(); err != nil { + return err + } + + _, err = pd.getEmptyTaggedFieldArray() + return err +} + +func (r *DescribeClusterResponse) key() int16 { return apiKeyDescribeCluster } + +func (r *DescribeClusterResponse) version() int16 { return r.Version } + +func (r *DescribeClusterResponse) setVersion(v int16) { r.Version = v } + +func (r *DescribeClusterResponse) headerVersion() int16 { return 1 } + +func (r *DescribeClusterResponse) isValidVersion() bool { + return r.Version >= 0 && r.Version <= 2 +} + +func (r *DescribeClusterResponse) isFlexible() bool { return true } + +func (r *DescribeClusterResponse) isFlexibleVersion(version int16) bool { return version >= 0 } + +func (r *DescribeClusterResponse) requiredVersion() KafkaVersion { + switch r.Version { + case 2: + return V4_0_0_0 + case 1: + return V3_7_0_0 + default: + return V3_0_0_0 + } +} + +func (r *DescribeClusterResponse) throttleTime() time.Duration { + return time.Duration(r.ThrottleTimeMs) * time.Millisecond +} + +func (b *DescribeClusterBroker) encode(pe packetEncoder, version int16) error { + pe.putInt32(b.BrokerID) + if err := pe.putString(b.Host); err != nil { + return err + } + pe.putInt32(b.Port) + if err := pe.putNullableString(b.Rack); err != nil { + return err + } + if version >= 2 { + pe.putBool(b.IsFenced) + } + pe.putEmptyTaggedFieldArray() + return nil +} + +func (b *DescribeClusterBroker) decode(pd packetDecoder, version int16) error { + var err error + if b.BrokerID, err = pd.getInt32(); err != nil { + return err + } + if b.Host, err = pd.getString(); err != nil { + return err + } + if b.Port, err = pd.getInt32(); err != nil { + return err + } + if b.Rack, err = pd.getNullableString(); err != nil { + return err + } + if version >= 2 { + if b.IsFenced, err = pd.getBool(); err != nil { + return err + } + } + _, err = pd.getEmptyTaggedFieldArray() + return err +} diff --git a/describe_cluster_response_test.go b/describe_cluster_response_test.go new file mode 100644 index 000000000..13fbcc1e4 --- /dev/null +++ b/describe_cluster_response_test.go @@ -0,0 +1,28 @@ +//go:build !functional + +package sarama + +import "testing" + +func TestDescribeClusterResponse(t *testing.T) { + rib := &DescribeClusterBroker{ + BrokerID: 1, + Host: "localhost", + Port: 9092, + Rack: nullString("rack-a"), + IsFenced: true, + } + + resp := &DescribeClusterResponse{ + Version: 2, + ThrottleTimeMs: 10, + Err: ErrNoError, + EndpointType: DescribeClusterEndpointTypeBrokers, + ClusterID: "cluster-1", + ControllerID: 1, + Brokers: []*DescribeClusterBroker{rib}, + ClusterAuthorizedOperations: 7, + } + + testResponse(t, "DescribeClusterResponse", resp, nil) +} diff --git a/request.go b/request.go index 83a1c466f..861815452 100644 --- a/request.go +++ b/request.go @@ -217,6 +217,8 @@ func allocateBody(key, version int16) protocolBody { return &DescribeUserScramCredentialsRequest{Version: version} case apiKeyAlterUserScramCredentials: return &AlterUserScramCredentialsRequest{Version: version} + case apiKeyDescribeCluster: + return &DescribeClusterRequest{Version: version} // 52: VoteRequest // 53: BeginQuorumEpochRequest // 54: EndQuorumEpochRequest diff --git a/request_test.go b/request_test.go index 581d08232..5dc6e7c08 100644 --- a/request_test.go +++ b/request_test.go @@ -73,7 +73,7 @@ var names = map[int16]string{ 57: "UpdateFeaturesRequest", 58: "EnvelopeRequest", 59: "FetchSnapshotRequest", - 60: "DescribeClusterRequest", + apiKeyDescribeCluster: "DescribeClusterRequest", 61: "DescribeProducersRequest", 62: "BrokerRegistrationRequest", 63: "BrokerHeartbeatRequest", @@ -174,6 +174,8 @@ func allocateResponseBody(req protocolBody) protocolBody { return &DescribeUserScramCredentialsResponse{Version: version} case apiKeyAlterUserScramCredentials: return &AlterUserScramCredentialsResponse{Version: version} + case apiKeyDescribeCluster: + return &DescribeClusterResponse{Version: version} } return nil } From 80c0becfcfc99b17be87a1b5ad8416445b2be2c6 Mon Sep 17 00:00:00 2001 From: DCjanus Date: Wed, 26 Nov 2025 17:41:37 +0800 Subject: [PATCH 02/17] feat(describe-cluster): add example and extend admin support Signed-off-by: DCjanus --- admin.go | 2 +- examples/README.md | 4 + examples/describe_cluster/README.md | 25 +++ examples/describe_cluster/go.mod | 30 +++ examples/describe_cluster/go.sum | 97 ++++++++++ examples/describe_cluster/main.go | 226 ++++++++++++++++++++++ examples/describe_cluster/scram_client.go | 29 +++ 7 files changed, 412 insertions(+), 1 deletion(-) create mode 100644 examples/describe_cluster/README.md create mode 100644 examples/describe_cluster/go.mod create mode 100644 examples/describe_cluster/go.sum create mode 100644 examples/describe_cluster/main.go create mode 100644 examples/describe_cluster/scram_client.go diff --git a/admin.go b/admin.go index f32a4fb90..06baa49c7 100644 --- a/admin.go +++ b/admin.go @@ -309,7 +309,7 @@ func (ca *clusterAdmin) DescribeTopics(topics []string) (metadata []*TopicMetada } func (ca *clusterAdmin) DescribeCluster() (brokers []*Broker, controllerID int32, err error) { - if ca.conf.Version.IsAtLeast(V3_0_0_0) { + if ca.conf.Version.IsAtLeast(V2_8_0_0) { brokers, controllerID, err = ca.describeClusterUsingAPI() if err == nil { return brokers, controllerID, nil diff --git a/examples/README.md b/examples/README.md index baded0116..bb17b53fa 100644 --- a/examples/README.md +++ b/examples/README.md @@ -6,6 +6,10 @@ This folder contains example applications to demonstrate the use of Sarama. For [http_server](./http_server) is a simple HTTP server uses both the sync producer to produce data as part of the request handling cycle, as well as the async producer to maintain an access log. It also uses the [mocks subpackage](https://pkg.go.dev/github.com/IBM/sarama/mocks) to test both. +#### Describe cluster + +[describe_cluster](./describe_cluster) shows how to connect to the controller, issue the DescribeCluster API (key 60), and print the cluster ID, controller ID, authorized operations, and brokers. + #### Interceptors Basic example to use a producer interceptor that produces [OpenTelemetry](https://github.com/open-telemetry/opentelemetry-go/) spans and add some headers for each intercepted message. diff --git a/examples/describe_cluster/README.md b/examples/describe_cluster/README.md new file mode 100644 index 000000000..bd3674840 --- /dev/null +++ b/examples/describe_cluster/README.md @@ -0,0 +1,25 @@ +# Describe cluster example + +This example shows how to invoke Kafka's [DescribeCluster](https://kafka.apache.org/protocol.html#The_Messages_DescribeCluster) API (key 60) directly through Sarama. It connects to the cluster controller, issues the request, and prints the cluster ID, controller ID, endpoint type, authorized operations, and the broker list (including the fenced bit on Kafka 4.0+). + +在仓库根目录下运行: + +```bash +cd examples/describe_cluster +go run . \ + -brokers="localhost:9092" \ + -version="3.7.0.0" \ + -endpoint-type=brokers \ + -include-cluster-ops +``` + +Flags: + +- `-brokers`: Bootstrap broker list (comma separated). Required. +- `-version`: Kafka protocol version used for request negotiation. Determines the DescribeCluster version. +- `-endpoint-type`: `brokers` or `controllers` (Kafka 3.7+). +- `-include-cluster-ops`: Request the cluster authorized operations bitfield. +- `-include-fenced-brokers`: Whether to include fenced brokers when the cluster supports DescribeCluster v2 (Kafka 4.0+). +- `-sasl-mechanism`: Set to `PLAIN`, `SCRAM-SHA-256`, or `SCRAM-SHA-512` when the cluster requires SASL authentication (provide `-sasl-user` and `-sasl-password` as well). +- `-sasl-user`, `-sasl-password`, `-sasl-authzid`: Credentials used when SASL is enabled. +- `-verbose`: Enable Sarama's internal logging for quick troubleshooting. diff --git a/examples/describe_cluster/go.mod b/examples/describe_cluster/go.mod new file mode 100644 index 000000000..a57f2511d --- /dev/null +++ b/examples/describe_cluster/go.mod @@ -0,0 +1,30 @@ +module github.com/IBM/sarama/examples/describe_cluster + +go 1.24.0 + +replace github.com/IBM/sarama => ../../ + +require github.com/IBM/sarama v1.45.0 + +require ( + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/eapache/go-resiliency v1.7.0 // indirect + github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3 // indirect + github.com/eapache/queue v1.1.0 // indirect + github.com/golang/snappy v0.0.4 // indirect + github.com/hashicorp/go-uuid v1.0.3 // indirect + github.com/jcmturner/aescts/v2 v2.0.0 // indirect + github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect + github.com/jcmturner/gofork v1.7.6 // indirect + github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect + github.com/jcmturner/rpc/v2 v2.0.3 // indirect + github.com/klauspost/compress v1.18.1 // indirect + github.com/pierrec/lz4/v4 v4.1.22 // indirect + github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect + github.com/xdg-go/pbkdf2 v1.0.0 // indirect + github.com/xdg-go/scram v1.1.2 // indirect + github.com/xdg-go/stringprep v1.0.4 // indirect + golang.org/x/crypto v0.45.0 // indirect + golang.org/x/net v0.47.0 // indirect + golang.org/x/text v0.31.0 // indirect +) diff --git a/examples/describe_cluster/go.sum b/examples/describe_cluster/go.sum new file mode 100644 index 000000000..a42a7a7d4 --- /dev/null +++ b/examples/describe_cluster/go.sum @@ -0,0 +1,97 @@ +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/eapache/go-resiliency v1.7.0 h1:n3NRTnBn5N0Cbi/IeOHuQn9s2UwVUH7Ga0ZWcP+9JTA= +github.com/eapache/go-resiliency v1.7.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= +github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3 h1:Oy0F4ALJ04o5Qqpdz8XLIpNA3WM/iSIXqxtqo7UGVws= +github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3/go.mod h1:YvSRo5mw33fLEx1+DlK6L2VV43tJt5Eyel9n9XBcR+0= +github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= +github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= +github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= +github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= +github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= +github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= +github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= +github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= +github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= +github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= +github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= +github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM= +github.com/jcmturner/gofork v1.7.6 h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVETg= +github.com/jcmturner/gofork v1.7.6/go.mod h1:1622LH6i/EZqLloHfE7IeZ0uEJwMSUyQ/nDd82IeqRo= +github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= +github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= +github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh687T8= +github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= +github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= +github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= +github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= +github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= +github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= +github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 h1:bsUq1dX0N8AOIL7EB/X911+m4EHsnWEHeJ0c+3TTBrg= +github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu7U= +github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= +github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= +github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.1.2 h1:FHX5I5B4i4hKRVRBCFRxq1iQRej7WO3hhBuJf+UUySY= +github.com/xdg-go/scram v1.1.2/go.mod h1:RT/sEzTbU5y00aCK8UOx6R7YryM0iF1N2MOmC3kKLN4= +github.com/xdg-go/stringprep v1.0.4 h1:XLI/Ng3O1Atzq0oBs3TWm+5ZVgkq2aqdlvP9JtoZ6c8= +github.com/xdg-go/stringprep v1.0.4/go.mod h1:mPGuuIYwz7CmR2bT9j4GbQqutWS1zV24gijq1dTyGkM= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= +golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= +golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.18.0 h1:kr88TuHDroi+UVf+0hZnirlk8o8T+4MrK6mr60WkH/I= +golang.org/x/sync v0.18.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= +golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.31.0 h1:aC8ghyu4JhP8VojJ2lEHBnochRno1sgL6nEi9WGFGMM= +golang.org/x/text v0.31.0/go.mod h1:tKRAlv61yKIjGGHX/4tP1LTbc13YSec1pxVEWXzfoeM= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/examples/describe_cluster/main.go b/examples/describe_cluster/main.go new file mode 100644 index 000000000..afa04b046 --- /dev/null +++ b/examples/describe_cluster/main.go @@ -0,0 +1,226 @@ +package main + +import ( + "crypto/sha256" + "crypto/sha512" + "errors" + "flag" + "fmt" + "log" + "os" + "strings" + + "github.com/IBM/sarama" +) + +var ( + brokers = flag.String("brokers", os.Getenv("KAFKA_PEERS"), "Kafka bootstrap brokers to connect to, as a comma separated list") + kafkaVer = flag.String("version", sarama.DefaultVersion.String(), "Kafka cluster version") + clientID = flag.String("client-id", "describe-cluster-example", "Client identifier") + endpoint = flag.String("endpoint-type", "brokers", "DescribeCluster endpoint type to request: brokers or controllers") + includeOps = flag.Bool("include-cluster-ops", false, "Include cluster authorized operations in the response") + includeFsvc = flag.Bool("include-fenced-brokers", true, "Include fenced brokers when supported (Kafka >= 4.0)") + verbose = flag.Bool("verbose", false, "Enable Sarama debug logging") + + saslMechanism = flag.String("sasl-mechanism", "", "SASL mechanism (PLAIN, SCRAM-SHA-256, SCRAM-SHA-512)") + saslUser = flag.String("sasl-user", "", "SASL username") + saslPassword = flag.String("sasl-password", "", "SASL password") + saslAuthzID = flag.String("sasl-authzid", "", "Optional SASL authorization identity (authzid)") +) + +func main() { + flag.Parse() + + if *brokers == "" { + flag.PrintDefaults() + os.Exit(1) + } + + if *verbose { + sarama.Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags) + } + + version, err := sarama.ParseKafkaVersion(*kafkaVer) + if err != nil { + log.Fatalf("invalid Kafka version %q: %v", *kafkaVer, err) + } + + config := sarama.NewConfig() + config.Version = version + config.ClientID = *clientID + + if err := configureSASL(config); err != nil { + log.Fatalf("invalid SASL configuration: %v", err) + } + + adminClient, err := sarama.NewClient(strings.Split(*brokers, ","), config) + if err != nil { + log.Fatalf("failed to create Sarama client: %v", err) + } + defer func() { + if err := adminClient.Close(); err != nil { + log.Printf("failed to close client: %v", err) + } + }() + + controller, err := adminClient.Controller() + if err != nil { + log.Fatalf("failed to get controller: %v", err) + } + defer func() { + if err := controller.Close(); err != nil && !errors.Is(err, sarama.ErrNotConnected) { + log.Printf("failed to close controller connection: %v", err) + } + }() + + if err := controller.Open(adminClient.Config()); err != nil && !errors.Is(err, sarama.ErrAlreadyConnected) { + log.Fatalf("failed to open controller connection: %v", err) + } + + request := sarama.NewDescribeClusterRequest(config.Version) + request.IncludeClusterAuthorizedOperations = *includeOps + + if request.Version >= 1 { + endpointType, err := parseEndpointType(*endpoint) + if err != nil { + log.Fatal(err) + } + request.EndpointType = endpointType + } else if *endpoint != "brokers" { + log.Printf("endpoint type flag ignored on Kafka versions older than 3.7 (request v%d)", request.Version) + } + + if request.Version >= 2 { + request.IncludeFencedBrokers = *includeFsvc + } else if !*includeFsvc { + log.Printf("include-fenced-brokers flag ignored on Kafka versions older than 4.0 (request v%d)", request.Version) + } + + response, err := controller.DescribeCluster(request) + if err != nil { + log.Fatalf("describe cluster call failed: %v", err) + } + if response.Err != sarama.ErrNoError { + if response.ErrorMessage != nil && *response.ErrorMessage != "" { + log.Fatalf("describe cluster returned %s: %s", response.Err, *response.ErrorMessage) + } + log.Fatalf("describe cluster returned %s", response.Err) + } + + printClusterInfo(response) +} + +func parseEndpointType(value string) (int8, error) { + switch strings.ToLower(value) { + case "", "broker", "brokers": + return sarama.DescribeClusterEndpointTypeBrokers, nil + case "controller", "controllers": + return sarama.DescribeClusterEndpointTypeControllers, nil + default: + return 0, fmt.Errorf("unsupported endpoint type %q (use brokers or controllers)", value) + } +} + +func endpointTypeLabel(value int8) string { + switch value { + case sarama.DescribeClusterEndpointTypeBrokers: + return "brokers" + case sarama.DescribeClusterEndpointTypeControllers: + return "controllers" + default: + return fmt.Sprintf("unknown(%d)", value) + } +} + +func printClusterInfo(resp *sarama.DescribeClusterResponse) { + fmt.Printf("Cluster ID: %s\n", resp.ClusterID) + fmt.Printf("Controller ID: %d\n", resp.ControllerID) + if resp.Version >= 1 { + fmt.Printf("Endpoint type: %s\n", endpointTypeLabel(resp.EndpointType)) + } + if resp.ClusterAuthorizedOperations != 0 { + fmt.Printf("Cluster operations: %s\n", strings.Join(formatAuthorizedOperations(resp.ClusterAuthorizedOperations), ", ")) + } else { + fmt.Println("Cluster operations: not requested") + } + + if len(resp.Brokers) == 0 { + fmt.Println("No brokers returned.") + return + } + + fmt.Println("\nBrokers:") + for _, broker := range resp.Brokers { + rack := "" + if broker.Rack != nil && *broker.Rack != "" { + rack = *broker.Rack + } + fmt.Printf(" - id=%d host=%s port=%d rack=%s", broker.BrokerID, broker.Host, broker.Port, rack) + if resp.Version >= 2 { + fmt.Printf(" fenced=%t", broker.IsFenced) + } + fmt.Println() + } +} + +func formatAuthorizedOperations(mask int32) []string { + if mask == 0 { + return nil + } + if mask < 0 { + return []string{"All"} + } + + var operations []string + for op := sarama.AclOperationRead; op <= sarama.AclOperationIdempotentWrite; op++ { + if mask&(1< Date: Wed, 26 Nov 2025 17:43:47 +0800 Subject: [PATCH 03/17] chore(deps): update dependencies to latest versions Signed-off-by: DCjanus --- examples/consumergroup/go.mod | 6 +++--- examples/consumergroup/go.sum | 3 +++ examples/exactly_once/go.mod | 6 +++--- examples/exactly_once/go.sum | 3 +++ examples/http_server/go.mod | 6 +++--- examples/http_server/go.sum | 3 +++ examples/interceptors/go.mod | 8 ++++---- examples/interceptors/go.sum | 4 ++++ examples/sasl_scram_client/go.mod | 8 ++++---- examples/sasl_scram_client/go.sum | 4 ++++ examples/txn_producer/go.mod | 6 +++--- examples/txn_producer/go.sum | 3 +++ 12 files changed, 40 insertions(+), 20 deletions(-) diff --git a/examples/consumergroup/go.mod b/examples/consumergroup/go.mod index c07f82874..723ff7c58 100644 --- a/examples/consumergroup/go.mod +++ b/examples/consumergroup/go.mod @@ -16,11 +16,11 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.0 // indirect + github.com/klauspost/compress v1.18.1 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect - golang.org/x/crypto v0.42.0 // indirect - golang.org/x/net v0.44.0 // indirect + golang.org/x/crypto v0.45.0 // indirect + golang.org/x/net v0.47.0 // indirect ) replace github.com/IBM/sarama => ../../ diff --git a/examples/consumergroup/go.sum b/examples/consumergroup/go.sum index e05fdaf7b..1f8665b2d 100644 --- a/examples/consumergroup/go.sum +++ b/examples/consumergroup/go.sum @@ -30,6 +30,7 @@ github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZ github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -51,6 +52,7 @@ golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5y golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -60,6 +62,7 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= diff --git a/examples/exactly_once/go.mod b/examples/exactly_once/go.mod index 7075489ac..99fedd3a1 100644 --- a/examples/exactly_once/go.mod +++ b/examples/exactly_once/go.mod @@ -16,11 +16,11 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.0 // indirect + github.com/klauspost/compress v1.18.1 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect - golang.org/x/crypto v0.42.0 // indirect - golang.org/x/net v0.44.0 // indirect + golang.org/x/crypto v0.45.0 // indirect + golang.org/x/net v0.47.0 // indirect ) replace github.com/IBM/sarama => ../../ diff --git a/examples/exactly_once/go.sum b/examples/exactly_once/go.sum index e05fdaf7b..1f8665b2d 100644 --- a/examples/exactly_once/go.sum +++ b/examples/exactly_once/go.sum @@ -30,6 +30,7 @@ github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZ github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -51,6 +52,7 @@ golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5y golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -60,6 +62,7 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= diff --git a/examples/http_server/go.mod b/examples/http_server/go.mod index 739ccc53e..2f5c2452b 100644 --- a/examples/http_server/go.mod +++ b/examples/http_server/go.mod @@ -16,11 +16,11 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.0 // indirect + github.com/klauspost/compress v1.18.1 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect - golang.org/x/crypto v0.42.0 // indirect - golang.org/x/net v0.44.0 // indirect + golang.org/x/crypto v0.45.0 // indirect + golang.org/x/net v0.47.0 // indirect ) replace github.com/IBM/sarama => ../../ diff --git a/examples/http_server/go.sum b/examples/http_server/go.sum index e05fdaf7b..1f8665b2d 100644 --- a/examples/http_server/go.sum +++ b/examples/http_server/go.sum @@ -30,6 +30,7 @@ github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZ github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -51,6 +52,7 @@ golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5y golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -60,6 +62,7 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= diff --git a/examples/interceptors/go.mod b/examples/interceptors/go.mod index 395f3e952..e4ec20c81 100644 --- a/examples/interceptors/go.mod +++ b/examples/interceptors/go.mod @@ -25,14 +25,14 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.0 // indirect + github.com/klauspost/compress v1.18.1 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect go.opentelemetry.io/otel/metric v1.29.0 // indirect go.opentelemetry.io/otel/sdk/metric v1.29.0 // indirect - golang.org/x/crypto v0.42.0 // indirect - golang.org/x/net v0.44.0 // indirect - golang.org/x/sys v0.36.0 // indirect + golang.org/x/crypto v0.45.0 // indirect + golang.org/x/net v0.47.0 // indirect + golang.org/x/sys v0.38.0 // indirect ) replace github.com/IBM/sarama => ../../ diff --git a/examples/interceptors/go.sum b/examples/interceptors/go.sum index dc0d9fd86..09f6d1de8 100644 --- a/examples/interceptors/go.sum +++ b/examples/interceptors/go.sum @@ -39,6 +39,7 @@ github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZ github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -72,6 +73,7 @@ golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5y golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -81,6 +83,7 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= @@ -93,6 +96,7 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.36.0 h1:KVRy2GtZBrk1cBYA7MKu5bEZFxQk4NIDV6RLVcC8o0k= golang.org/x/sys v0.36.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= +golang.org/x/sys v0.38.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= diff --git a/examples/sasl_scram_client/go.mod b/examples/sasl_scram_client/go.mod index a3e0c17e3..471dc1280 100644 --- a/examples/sasl_scram_client/go.mod +++ b/examples/sasl_scram_client/go.mod @@ -19,14 +19,14 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.0 // indirect + github.com/klauspost/compress v1.18.1 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect github.com/xdg-go/pbkdf2 v1.0.0 // indirect github.com/xdg-go/stringprep v1.0.4 // indirect - golang.org/x/crypto v0.42.0 // indirect - golang.org/x/net v0.44.0 // indirect - golang.org/x/text v0.29.0 // indirect + golang.org/x/crypto v0.45.0 // indirect + golang.org/x/net v0.47.0 // indirect + golang.org/x/text v0.31.0 // indirect ) replace github.com/IBM/sarama => ../../ diff --git a/examples/sasl_scram_client/go.sum b/examples/sasl_scram_client/go.sum index 457153b3b..10ea8b4c2 100644 --- a/examples/sasl_scram_client/go.sum +++ b/examples/sasl_scram_client/go.sum @@ -30,6 +30,7 @@ github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZ github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -57,6 +58,7 @@ golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5y golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -66,6 +68,7 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= @@ -86,6 +89,7 @@ golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.29.0 h1:1neNs90w9YzJ9BocxfsQNHKuAT4pkghyXc4nhZ6sJvk= golang.org/x/text v0.29.0/go.mod h1:7MhJOA9CD2qZyOKYazxdYMF85OwPdEr9jTtBpO7ydH4= +golang.org/x/text v0.31.0/go.mod h1:tKRAlv61yKIjGGHX/4tP1LTbc13YSec1pxVEWXzfoeM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= diff --git a/examples/txn_producer/go.mod b/examples/txn_producer/go.mod index a8fc0dcb3..316dce1c1 100644 --- a/examples/txn_producer/go.mod +++ b/examples/txn_producer/go.mod @@ -19,10 +19,10 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.0 // indirect + github.com/klauspost/compress v1.18.1 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect - golang.org/x/crypto v0.42.0 // indirect - golang.org/x/net v0.44.0 // indirect + golang.org/x/crypto v0.45.0 // indirect + golang.org/x/net v0.47.0 // indirect ) replace github.com/IBM/sarama => ../../ diff --git a/examples/txn_producer/go.sum b/examples/txn_producer/go.sum index e05fdaf7b..1f8665b2d 100644 --- a/examples/txn_producer/go.sum +++ b/examples/txn_producer/go.sum @@ -30,6 +30,7 @@ github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZ github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -51,6 +52,7 @@ golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5y golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -60,6 +62,7 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= From 1f85f51b6b26b3e6dd89f2686fa8eca6cb744b1f Mon Sep 17 00:00:00 2001 From: DCjanus Date: Wed, 26 Nov 2025 17:56:32 +0800 Subject: [PATCH 04/17] feat(describe-cluster): align example with admin helper Signed-off-by: DCjanus --- admin.go | 1 + examples/README.md | 2 +- examples/describe_cluster/README.md | 14 +-- examples/describe_cluster/main.go | 146 ++++++---------------------- 4 files changed, 38 insertions(+), 125 deletions(-) diff --git a/admin.go b/admin.go index 06baa49c7..c16b54127 100644 --- a/admin.go +++ b/admin.go @@ -309,6 +309,7 @@ func (ca *clusterAdmin) DescribeTopics(topics []string) (metadata []*TopicMetada } func (ca *clusterAdmin) DescribeCluster() (brokers []*Broker, controllerID int32, err error) { + fmt.Println("ClusterAdmin DescribeCluster called: checking version", ca.conf.Version) if ca.conf.Version.IsAtLeast(V2_8_0_0) { brokers, controllerID, err = ca.describeClusterUsingAPI() if err == nil { diff --git a/examples/README.md b/examples/README.md index bb17b53fa..dcb5d7fcd 100644 --- a/examples/README.md +++ b/examples/README.md @@ -8,7 +8,7 @@ This folder contains example applications to demonstrate the use of Sarama. For #### Describe cluster -[describe_cluster](./describe_cluster) shows how to connect to the controller, issue the DescribeCluster API (key 60), and print the cluster ID, controller ID, authorized operations, and brokers. +[describe_cluster](./describe_cluster) creates a `ClusterAdmin`, calls `DescribeCluster`, and prints the controller ID and brokers so you can validate Sarama's DescribeCluster support (API key 60). #### Interceptors diff --git a/examples/describe_cluster/README.md b/examples/describe_cluster/README.md index bd3674840..f30c66599 100644 --- a/examples/describe_cluster/README.md +++ b/examples/describe_cluster/README.md @@ -1,25 +1,21 @@ # Describe cluster example -This example shows how to invoke Kafka's [DescribeCluster](https://kafka.apache.org/protocol.html#The_Messages_DescribeCluster) API (key 60) directly through Sarama. It connects to the cluster controller, issues the request, and prints the cluster ID, controller ID, endpoint type, authorized operations, and the broker list (including the fenced bit on Kafka 4.0+). +This example uses Sarama's `ClusterAdmin.DescribeCluster` helper to exercise the logic added in `admin.go`. When the configured Kafka version is at least 2.8.0 it will use the DescribeCluster API (key 60); otherwise it transparently falls back to the Metadata API, just like the library call. -在仓库根目录下运行: +Run it from the repository root with: ```bash cd examples/describe_cluster go run . \ -brokers="localhost:9092" \ - -version="3.7.0.0" \ - -endpoint-type=brokers \ - -include-cluster-ops + -version="3.7.0" ``` Flags: - `-brokers`: Bootstrap broker list (comma separated). Required. -- `-version`: Kafka protocol version used for request negotiation. Determines the DescribeCluster version. -- `-endpoint-type`: `brokers` or `controllers` (Kafka 3.7+). -- `-include-cluster-ops`: Request the cluster authorized operations bitfield. -- `-include-fenced-brokers`: Whether to include fenced brokers when the cluster supports DescribeCluster v2 (Kafka 4.0+). +- `-version`: Kafka protocol version negotiated by Sarama. Determines whether DescribeCluster is available. +- `-client-id`: Client identifier sent to the cluster. - `-sasl-mechanism`: Set to `PLAIN`, `SCRAM-SHA-256`, or `SCRAM-SHA-512` when the cluster requires SASL authentication (provide `-sasl-user` and `-sasl-password` as well). - `-sasl-user`, `-sasl-password`, `-sasl-authzid`: Credentials used when SASL is enabled. - `-verbose`: Enable Sarama's internal logging for quick troubleshooting. diff --git a/examples/describe_cluster/main.go b/examples/describe_cluster/main.go index afa04b046..0bebdc043 100644 --- a/examples/describe_cluster/main.go +++ b/examples/describe_cluster/main.go @@ -14,13 +14,10 @@ import ( ) var ( - brokers = flag.String("brokers", os.Getenv("KAFKA_PEERS"), "Kafka bootstrap brokers to connect to, as a comma separated list") - kafkaVer = flag.String("version", sarama.DefaultVersion.String(), "Kafka cluster version") - clientID = flag.String("client-id", "describe-cluster-example", "Client identifier") - endpoint = flag.String("endpoint-type", "brokers", "DescribeCluster endpoint type to request: brokers or controllers") - includeOps = flag.Bool("include-cluster-ops", false, "Include cluster authorized operations in the response") - includeFsvc = flag.Bool("include-fenced-brokers", true, "Include fenced brokers when supported (Kafka >= 4.0)") - verbose = flag.Bool("verbose", false, "Enable Sarama debug logging") + brokers = flag.String("brokers", os.Getenv("KAFKA_PEERS"), "Kafka bootstrap brokers to connect to, as a comma separated list") + kafkaVer = flag.String("version", sarama.DefaultVersion.String(), "Kafka cluster version") + clientID = flag.String("client-id", "describe-cluster-example", "Client identifier") + verbose = flag.Bool("verbose", false, "Enable Sarama debug logging") saslMechanism = flag.String("sasl-mechanism", "", "SASL mechanism (PLAIN, SCRAM-SHA-256, SCRAM-SHA-512)") saslUser = flag.String("sasl-user", "", "SASL username") @@ -53,138 +50,57 @@ func main() { log.Fatalf("invalid SASL configuration: %v", err) } - adminClient, err := sarama.NewClient(strings.Split(*brokers, ","), config) - if err != nil { - log.Fatalf("failed to create Sarama client: %v", err) + brokerList := parseBrokerList(*brokers) + if len(brokerList) == 0 { + log.Fatalf("no valid broker addresses provided in %q", *brokers) } - defer func() { - if err := adminClient.Close(); err != nil { - log.Printf("failed to close client: %v", err) - } - }() - controller, err := adminClient.Controller() + admin, err := sarama.NewClusterAdmin(brokerList, config) if err != nil { - log.Fatalf("failed to get controller: %v", err) + log.Fatalf("failed to create cluster admin: %v", err) } defer func() { - if err := controller.Close(); err != nil && !errors.Is(err, sarama.ErrNotConnected) { - log.Printf("failed to close controller connection: %v", err) + if err := admin.Close(); err != nil { + log.Printf("failed to close cluster admin: %v", err) } }() - if err := controller.Open(adminClient.Config()); err != nil && !errors.Is(err, sarama.ErrAlreadyConnected) { - log.Fatalf("failed to open controller connection: %v", err) - } - - request := sarama.NewDescribeClusterRequest(config.Version) - request.IncludeClusterAuthorizedOperations = *includeOps - - if request.Version >= 1 { - endpointType, err := parseEndpointType(*endpoint) - if err != nil { - log.Fatal(err) - } - request.EndpointType = endpointType - } else if *endpoint != "brokers" { - log.Printf("endpoint type flag ignored on Kafka versions older than 3.7 (request v%d)", request.Version) - } - - if request.Version >= 2 { - request.IncludeFencedBrokers = *includeFsvc - } else if !*includeFsvc { - log.Printf("include-fenced-brokers flag ignored on Kafka versions older than 4.0 (request v%d)", request.Version) - } - - response, err := controller.DescribeCluster(request) + brokers, controllerID, err := admin.DescribeCluster() if err != nil { log.Fatalf("describe cluster call failed: %v", err) } - if response.Err != sarama.ErrNoError { - if response.ErrorMessage != nil && *response.ErrorMessage != "" { - log.Fatalf("describe cluster returned %s: %s", response.Err, *response.ErrorMessage) - } - log.Fatalf("describe cluster returned %s", response.Err) - } - printClusterInfo(response) + printClusterInfo(brokers, controllerID) } -func parseEndpointType(value string) (int8, error) { - switch strings.ToLower(value) { - case "", "broker", "brokers": - return sarama.DescribeClusterEndpointTypeBrokers, nil - case "controller", "controllers": - return sarama.DescribeClusterEndpointTypeControllers, nil - default: - return 0, fmt.Errorf("unsupported endpoint type %q (use brokers or controllers)", value) - } -} - -func endpointTypeLabel(value int8) string { - switch value { - case sarama.DescribeClusterEndpointTypeBrokers: - return "brokers" - case sarama.DescribeClusterEndpointTypeControllers: - return "controllers" - default: - return fmt.Sprintf("unknown(%d)", value) +func parseBrokerList(list string) []string { + parts := strings.Split(list, ",") + result := make([]string, 0, len(parts)) + for _, part := range parts { + trimmed := strings.TrimSpace(part) + if trimmed == "" { + continue + } + result = append(result, trimmed) } + return result } -func printClusterInfo(resp *sarama.DescribeClusterResponse) { - fmt.Printf("Cluster ID: %s\n", resp.ClusterID) - fmt.Printf("Controller ID: %d\n", resp.ControllerID) - if resp.Version >= 1 { - fmt.Printf("Endpoint type: %s\n", endpointTypeLabel(resp.EndpointType)) - } - if resp.ClusterAuthorizedOperations != 0 { - fmt.Printf("Cluster operations: %s\n", strings.Join(formatAuthorizedOperations(resp.ClusterAuthorizedOperations), ", ")) - } else { - fmt.Println("Cluster operations: not requested") - } - - if len(resp.Brokers) == 0 { +func printClusterInfo(brokers []*sarama.Broker, controllerID int32) { + fmt.Printf("Controller ID: %d\n", controllerID) + if len(brokers) == 0 { fmt.Println("No brokers returned.") return } fmt.Println("\nBrokers:") - for _, broker := range resp.Brokers { - rack := "" - if broker.Rack != nil && *broker.Rack != "" { - rack = *broker.Rack - } - fmt.Printf(" - id=%d host=%s port=%d rack=%s", broker.BrokerID, broker.Host, broker.Port, rack) - if resp.Version >= 2 { - fmt.Printf(" fenced=%t", broker.IsFenced) - } - fmt.Println() - } -} - -func formatAuthorizedOperations(mask int32) []string { - if mask == 0 { - return nil - } - if mask < 0 { - return []string{"All"} - } - - var operations []string - for op := sarama.AclOperationRead; op <= sarama.AclOperationIdempotentWrite; op++ { - if mask&(1< Date: Wed, 26 Nov 2025 17:57:51 +0800 Subject: [PATCH 05/17] feat(admin): remove debug print from DescribeCluster method Signed-off-by: DCjanus --- admin.go | 1 - 1 file changed, 1 deletion(-) diff --git a/admin.go b/admin.go index c16b54127..06baa49c7 100644 --- a/admin.go +++ b/admin.go @@ -309,7 +309,6 @@ func (ca *clusterAdmin) DescribeTopics(topics []string) (metadata []*TopicMetada } func (ca *clusterAdmin) DescribeCluster() (brokers []*Broker, controllerID int32, err error) { - fmt.Println("ClusterAdmin DescribeCluster called: checking version", ca.conf.Version) if ca.conf.Version.IsAtLeast(V2_8_0_0) { brokers, controllerID, err = ca.describeClusterUsingAPI() if err == nil { From cf965ca8788cb95c20078a8933951d6863a3f245 Mon Sep 17 00:00:00 2001 From: DCjanus Date: Wed, 26 Nov 2025 18:12:50 +0800 Subject: [PATCH 06/17] fix(describe-cluster): align min version Signed-off-by: DCjanus --- describe_cluster_request.go | 2 +- describe_cluster_request_test.go | 2 +- describe_cluster_response.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/describe_cluster_request.go b/describe_cluster_request.go index 189583c12..abd0ebe68 100644 --- a/describe_cluster_request.go +++ b/describe_cluster_request.go @@ -102,6 +102,6 @@ func (r *DescribeClusterRequest) requiredVersion() KafkaVersion { case 1: return V3_7_0_0 default: - return V3_0_0_0 + return V2_8_0_0 } } diff --git a/describe_cluster_request_test.go b/describe_cluster_request_test.go index cc4b0ba4a..6549714a7 100644 --- a/describe_cluster_request_test.go +++ b/describe_cluster_request_test.go @@ -34,7 +34,7 @@ func TestNewDescribeClusterRequest(t *testing.T) { version KafkaVersion expectedVer int16 }{ - {V3_0_0_0, 0}, + {V2_8_0_0, 0}, {V3_7_0_0, 1}, {V4_0_0_0, 2}, } diff --git a/describe_cluster_response.go b/describe_cluster_response.go index 7f0b5c685..28ae76c0b 100644 --- a/describe_cluster_response.go +++ b/describe_cluster_response.go @@ -128,7 +128,7 @@ func (r *DescribeClusterResponse) requiredVersion() KafkaVersion { case 1: return V3_7_0_0 default: - return V3_0_0_0 + return V2_8_0_0 } } From 61bbd2553dbbe93697301b5b526f2147dfed69fa Mon Sep 17 00:00:00 2001 From: DCjanus Date: Thu, 27 Nov 2025 10:27:42 +0800 Subject: [PATCH 07/17] fix: document describe cluster gaps and sasl casing Signed-off-by: DCjanus --- admin.go | 4 ++++ examples/describe_cluster/main.go | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/admin.go b/admin.go index 06baa49c7..42f6fa0c4 100644 --- a/admin.go +++ b/admin.go @@ -376,6 +376,10 @@ func (ca *clusterAdmin) describeClusterUsingMetadata() (brokers []*Broker, contr } func convertDescribeClusterBrokers(entries []*DescribeClusterBroker) []*Broker { + // TODO: DescribeCluster brokers currently drop DescribeCluster-specific fields + // such as IsFenced (KIP-1073) and ClusterAuthorizedOperations because Broker + // has no equivalents yet. This keeps API parity with MetadataResponse for now, + // but the richer fields need to be surfaced in a future change. if len(entries) == 0 { return nil } diff --git a/examples/describe_cluster/main.go b/examples/describe_cluster/main.go index 0bebdc043..0899a4f80 100644 --- a/examples/describe_cluster/main.go +++ b/examples/describe_cluster/main.go @@ -122,7 +122,7 @@ func configureSASL(config *sarama.Config) error { config.Net.SASL.AuthIdentity = *saslAuthzID switch mech { - case "PLAIN", "PLAINtext": + case "PLAIN", "PLAINTEXT": config.Net.SASL.Mechanism = sarama.SASLTypePlaintext case "SCRAM-SHA-256": config.Net.SASL.Mechanism = sarama.SASLTypeSCRAMSHA256 From 9df3b047caff22d696e8c0e4189de75b14b53d5f Mon Sep 17 00:00:00 2001 From: DCjanus Date: Mon, 1 Dec 2025 10:54:27 +0800 Subject: [PATCH 08/17] chore(examples): remove describe_cluster sample Signed-off-by: DCjanus --- examples/README.md | 4 - examples/describe_cluster/README.md | 21 ---- examples/describe_cluster/go.mod | 30 ----- examples/describe_cluster/go.sum | 97 --------------- examples/describe_cluster/main.go | 142 ---------------------- examples/describe_cluster/scram_client.go | 29 ----- 6 files changed, 323 deletions(-) delete mode 100644 examples/describe_cluster/README.md delete mode 100644 examples/describe_cluster/go.mod delete mode 100644 examples/describe_cluster/go.sum delete mode 100644 examples/describe_cluster/main.go delete mode 100644 examples/describe_cluster/scram_client.go diff --git a/examples/README.md b/examples/README.md index dcb5d7fcd..baded0116 100644 --- a/examples/README.md +++ b/examples/README.md @@ -6,10 +6,6 @@ This folder contains example applications to demonstrate the use of Sarama. For [http_server](./http_server) is a simple HTTP server uses both the sync producer to produce data as part of the request handling cycle, as well as the async producer to maintain an access log. It also uses the [mocks subpackage](https://pkg.go.dev/github.com/IBM/sarama/mocks) to test both. -#### Describe cluster - -[describe_cluster](./describe_cluster) creates a `ClusterAdmin`, calls `DescribeCluster`, and prints the controller ID and brokers so you can validate Sarama's DescribeCluster support (API key 60). - #### Interceptors Basic example to use a producer interceptor that produces [OpenTelemetry](https://github.com/open-telemetry/opentelemetry-go/) spans and add some headers for each intercepted message. diff --git a/examples/describe_cluster/README.md b/examples/describe_cluster/README.md deleted file mode 100644 index f30c66599..000000000 --- a/examples/describe_cluster/README.md +++ /dev/null @@ -1,21 +0,0 @@ -# Describe cluster example - -This example uses Sarama's `ClusterAdmin.DescribeCluster` helper to exercise the logic added in `admin.go`. When the configured Kafka version is at least 2.8.0 it will use the DescribeCluster API (key 60); otherwise it transparently falls back to the Metadata API, just like the library call. - -Run it from the repository root with: - -```bash -cd examples/describe_cluster -go run . \ - -brokers="localhost:9092" \ - -version="3.7.0" -``` - -Flags: - -- `-brokers`: Bootstrap broker list (comma separated). Required. -- `-version`: Kafka protocol version negotiated by Sarama. Determines whether DescribeCluster is available. -- `-client-id`: Client identifier sent to the cluster. -- `-sasl-mechanism`: Set to `PLAIN`, `SCRAM-SHA-256`, or `SCRAM-SHA-512` when the cluster requires SASL authentication (provide `-sasl-user` and `-sasl-password` as well). -- `-sasl-user`, `-sasl-password`, `-sasl-authzid`: Credentials used when SASL is enabled. -- `-verbose`: Enable Sarama's internal logging for quick troubleshooting. diff --git a/examples/describe_cluster/go.mod b/examples/describe_cluster/go.mod deleted file mode 100644 index a57f2511d..000000000 --- a/examples/describe_cluster/go.mod +++ /dev/null @@ -1,30 +0,0 @@ -module github.com/IBM/sarama/examples/describe_cluster - -go 1.24.0 - -replace github.com/IBM/sarama => ../../ - -require github.com/IBM/sarama v1.45.0 - -require ( - github.com/davecgh/go-spew v1.1.1 // indirect - github.com/eapache/go-resiliency v1.7.0 // indirect - github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3 // indirect - github.com/eapache/queue v1.1.0 // indirect - github.com/golang/snappy v0.0.4 // indirect - github.com/hashicorp/go-uuid v1.0.3 // indirect - github.com/jcmturner/aescts/v2 v2.0.0 // indirect - github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect - github.com/jcmturner/gofork v1.7.6 // indirect - github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect - github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.1 // indirect - github.com/pierrec/lz4/v4 v4.1.22 // indirect - github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect - github.com/xdg-go/pbkdf2 v1.0.0 // indirect - github.com/xdg-go/scram v1.1.2 // indirect - github.com/xdg-go/stringprep v1.0.4 // indirect - golang.org/x/crypto v0.45.0 // indirect - golang.org/x/net v0.47.0 // indirect - golang.org/x/text v0.31.0 // indirect -) diff --git a/examples/describe_cluster/go.sum b/examples/describe_cluster/go.sum deleted file mode 100644 index a42a7a7d4..000000000 --- a/examples/describe_cluster/go.sum +++ /dev/null @@ -1,97 +0,0 @@ -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= -github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/eapache/go-resiliency v1.7.0 h1:n3NRTnBn5N0Cbi/IeOHuQn9s2UwVUH7Ga0ZWcP+9JTA= -github.com/eapache/go-resiliency v1.7.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= -github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3 h1:Oy0F4ALJ04o5Qqpdz8XLIpNA3WM/iSIXqxtqo7UGVws= -github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3/go.mod h1:YvSRo5mw33fLEx1+DlK6L2VV43tJt5Eyel9n9XBcR+0= -github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= -github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= -github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= -github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= -github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= -github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= -github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= -github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= -github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= -github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= -github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= -github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM= -github.com/jcmturner/gofork v1.7.6 h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVETg= -github.com/jcmturner/gofork v1.7.6/go.mod h1:1622LH6i/EZqLloHfE7IeZ0uEJwMSUyQ/nDd82IeqRo= -github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= -github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= -github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh687T8= -github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= -github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= -github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= -github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= -github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= -github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= -github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 h1:bsUq1dX0N8AOIL7EB/X911+m4EHsnWEHeJ0c+3TTBrg= -github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= -github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu7U= -github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= -github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= -github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= -github.com/xdg-go/scram v1.1.2 h1:FHX5I5B4i4hKRVRBCFRxq1iQRej7WO3hhBuJf+UUySY= -github.com/xdg-go/scram v1.1.2/go.mod h1:RT/sEzTbU5y00aCK8UOx6R7YryM0iF1N2MOmC3kKLN4= -github.com/xdg-go/stringprep v1.0.4 h1:XLI/Ng3O1Atzq0oBs3TWm+5ZVgkq2aqdlvP9JtoZ6c8= -github.com/xdg-go/stringprep v1.0.4/go.mod h1:mPGuuIYwz7CmR2bT9j4GbQqutWS1zV24gijq1dTyGkM= -github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= -golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= -golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= -golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= -golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.18.0 h1:kr88TuHDroi+UVf+0hZnirlk8o8T+4MrK6mr60WkH/I= -golang.org/x/sync v0.18.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= -golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= -golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= -golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.31.0 h1:aC8ghyu4JhP8VojJ2lEHBnochRno1sgL6nEi9WGFGMM= -golang.org/x/text v0.31.0/go.mod h1:tKRAlv61yKIjGGHX/4tP1LTbc13YSec1pxVEWXzfoeM= -golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= -golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= -gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/examples/describe_cluster/main.go b/examples/describe_cluster/main.go deleted file mode 100644 index 0899a4f80..000000000 --- a/examples/describe_cluster/main.go +++ /dev/null @@ -1,142 +0,0 @@ -package main - -import ( - "crypto/sha256" - "crypto/sha512" - "errors" - "flag" - "fmt" - "log" - "os" - "strings" - - "github.com/IBM/sarama" -) - -var ( - brokers = flag.String("brokers", os.Getenv("KAFKA_PEERS"), "Kafka bootstrap brokers to connect to, as a comma separated list") - kafkaVer = flag.String("version", sarama.DefaultVersion.String(), "Kafka cluster version") - clientID = flag.String("client-id", "describe-cluster-example", "Client identifier") - verbose = flag.Bool("verbose", false, "Enable Sarama debug logging") - - saslMechanism = flag.String("sasl-mechanism", "", "SASL mechanism (PLAIN, SCRAM-SHA-256, SCRAM-SHA-512)") - saslUser = flag.String("sasl-user", "", "SASL username") - saslPassword = flag.String("sasl-password", "", "SASL password") - saslAuthzID = flag.String("sasl-authzid", "", "Optional SASL authorization identity (authzid)") -) - -func main() { - flag.Parse() - - if *brokers == "" { - flag.PrintDefaults() - os.Exit(1) - } - - if *verbose { - sarama.Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags) - } - - version, err := sarama.ParseKafkaVersion(*kafkaVer) - if err != nil { - log.Fatalf("invalid Kafka version %q: %v", *kafkaVer, err) - } - - config := sarama.NewConfig() - config.Version = version - config.ClientID = *clientID - - if err := configureSASL(config); err != nil { - log.Fatalf("invalid SASL configuration: %v", err) - } - - brokerList := parseBrokerList(*brokers) - if len(brokerList) == 0 { - log.Fatalf("no valid broker addresses provided in %q", *brokers) - } - - admin, err := sarama.NewClusterAdmin(brokerList, config) - if err != nil { - log.Fatalf("failed to create cluster admin: %v", err) - } - defer func() { - if err := admin.Close(); err != nil { - log.Printf("failed to close cluster admin: %v", err) - } - }() - - brokers, controllerID, err := admin.DescribeCluster() - if err != nil { - log.Fatalf("describe cluster call failed: %v", err) - } - - printClusterInfo(brokers, controllerID) -} - -func parseBrokerList(list string) []string { - parts := strings.Split(list, ",") - result := make([]string, 0, len(parts)) - for _, part := range parts { - trimmed := strings.TrimSpace(part) - if trimmed == "" { - continue - } - result = append(result, trimmed) - } - return result -} - -func printClusterInfo(brokers []*sarama.Broker, controllerID int32) { - fmt.Printf("Controller ID: %d\n", controllerID) - if len(brokers) == 0 { - fmt.Println("No brokers returned.") - return - } - - fmt.Println("\nBrokers:") - for _, broker := range brokers { - rack := broker.Rack() - if rack == "" { - rack = "" - } - fmt.Printf(" - id=%d addr=%s rack=%s\n", broker.ID(), broker.Addr(), rack) - } -} - -func configureSASL(config *sarama.Config) error { - mech := strings.ToUpper(strings.ReplaceAll(strings.TrimSpace(*saslMechanism), "_", "-")) - if mech == "" { - if *saslUser != "" || *saslPassword != "" { - return errors.New("sasl-mechanism must be provided when username or password is set") - } - return nil - } - - if *saslUser == "" || *saslPassword == "" { - return errors.New("sasl-user and sasl-password must be provided when SASL is enabled") - } - - config.Net.SASL.Enable = true - config.Net.SASL.User = *saslUser - config.Net.SASL.Password = *saslPassword - config.Net.SASL.AuthIdentity = *saslAuthzID - - switch mech { - case "PLAIN", "PLAINTEXT": - config.Net.SASL.Mechanism = sarama.SASLTypePlaintext - case "SCRAM-SHA-256": - config.Net.SASL.Mechanism = sarama.SASLTypeSCRAMSHA256 - config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { - return &XDGSCRAMClient{HashGeneratorFcn: sha256.New} - } - case "SCRAM-SHA-512": - config.Net.SASL.Mechanism = sarama.SASLTypeSCRAMSHA512 - config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { - return &XDGSCRAMClient{HashGeneratorFcn: sha512.New} - } - default: - return fmt.Errorf("unsupported SASL mechanism %q", mech) - } - - return nil -} diff --git a/examples/describe_cluster/scram_client.go b/examples/describe_cluster/scram_client.go deleted file mode 100644 index 8ffee9f31..000000000 --- a/examples/describe_cluster/scram_client.go +++ /dev/null @@ -1,29 +0,0 @@ -package main - -import ( - "github.com/xdg-go/scram" -) - -type XDGSCRAMClient struct { - *scram.Client - *scram.ClientConversation - scram.HashGeneratorFcn -} - -func (x *XDGSCRAMClient) Begin(userName, password, authzID string) error { - client, err := x.HashGeneratorFcn.NewClient(userName, password, authzID) - if err != nil { - return err - } - x.Client = client - x.ClientConversation = client.NewConversation() - return nil -} - -func (x *XDGSCRAMClient) Step(challenge string) (string, error) { - return x.ClientConversation.Step(challenge) -} - -func (x *XDGSCRAMClient) Done() bool { - return x.ClientConversation.Done() -} From 1b0a1b2b599f1711de5582077c4fbc92e55b12de Mon Sep 17 00:00:00 2001 From: DCjanus Date: Mon, 1 Dec 2025 10:55:49 +0800 Subject: [PATCH 09/17] chore(deps): tidy go.sum Signed-off-by: DCjanus --- go.sum | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/go.sum b/go.sum index 111cbc65b..dcf07d123 100644 --- a/go.sum +++ b/go.sum @@ -29,8 +29,6 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= -github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -62,10 +60,6 @@ github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5t golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= -golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= -golang.org/x/crypto v0.43.0 h1:dduJYIi3A3KOfdGOHX8AVZ/jGiyPa3IbBozJ5kNuE04= -golang.org/x/crypto v0.43.0/go.mod h1:BFbav4mRNlXJL4wNeejLpWxB7wMbc79PdRGhWKncxR0= golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= @@ -75,16 +69,10 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= -golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= -golang.org/x/net v0.46.0 h1:giFlY12I07fugqwPuWJi68oOnpfqFnJIJzaIIm2JVV4= -golang.org/x/net v0.46.0/go.mod h1:Q9BGdFy1y4nkUwiLvT5qtyhAnEHgnQ/zd8PfU6nc210= golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= -golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= golang.org/x/sync v0.18.0 h1:kr88TuHDroi+UVf+0hZnirlk8o8T+4MrK6mr60WkH/I= golang.org/x/sync v0.18.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= From 2276139b575a228faea49d0c5bf969ea73213c1f Mon Sep 17 00:00:00 2001 From: DCjanus Date: Mon, 1 Dec 2025 11:00:09 +0800 Subject: [PATCH 10/17] chore(examples): tidy go.sum in submodules Signed-off-by: DCjanus --- examples/consumergroup/go.sum | 17 +++++++---------- examples/exactly_once/go.sum | 17 +++++++---------- examples/http_server/go.sum | 17 +++++++---------- examples/interceptors/go.sum | 20 ++++++++------------ examples/sasl_scram_client/go.sum | 20 ++++++++------------ examples/txn_producer/go.sum | 17 +++++++---------- 6 files changed, 44 insertions(+), 64 deletions(-) diff --git a/examples/consumergroup/go.sum b/examples/consumergroup/go.sum index 1f8665b2d..ff7614215 100644 --- a/examples/consumergroup/go.sum +++ b/examples/consumergroup/go.sum @@ -28,8 +28,7 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= -github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -44,14 +43,13 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.11.0 h1:ib4sjIrwZKxE5u/Japgo/7SJV3PvgjGiRNAvTVGqQl8= -github.com/stretchr/testify v1.11.0/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= +github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu7U= +github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= -golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -60,13 +58,12 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= -golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= -golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= +golang.org/x/sync v0.18.0 h1:kr88TuHDroi+UVf+0hZnirlk8o8T+4MrK6mr60WkH/I= +golang.org/x/sync v0.18.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= diff --git a/examples/exactly_once/go.sum b/examples/exactly_once/go.sum index 1f8665b2d..ff7614215 100644 --- a/examples/exactly_once/go.sum +++ b/examples/exactly_once/go.sum @@ -28,8 +28,7 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= -github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -44,14 +43,13 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.11.0 h1:ib4sjIrwZKxE5u/Japgo/7SJV3PvgjGiRNAvTVGqQl8= -github.com/stretchr/testify v1.11.0/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= +github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu7U= +github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= -golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -60,13 +58,12 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= -golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= -golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= +golang.org/x/sync v0.18.0 h1:kr88TuHDroi+UVf+0hZnirlk8o8T+4MrK6mr60WkH/I= +golang.org/x/sync v0.18.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= diff --git a/examples/http_server/go.sum b/examples/http_server/go.sum index 1f8665b2d..ff7614215 100644 --- a/examples/http_server/go.sum +++ b/examples/http_server/go.sum @@ -28,8 +28,7 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= -github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -44,14 +43,13 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.11.0 h1:ib4sjIrwZKxE5u/Japgo/7SJV3PvgjGiRNAvTVGqQl8= -github.com/stretchr/testify v1.11.0/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= +github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu7U= +github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= -golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -60,13 +58,12 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= -golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= -golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= +golang.org/x/sync v0.18.0 h1:kr88TuHDroi+UVf+0hZnirlk8o8T+4MrK6mr60WkH/I= +golang.org/x/sync v0.18.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= diff --git a/examples/interceptors/go.sum b/examples/interceptors/go.sum index 09f6d1de8..aa2f9bdd8 100644 --- a/examples/interceptors/go.sum +++ b/examples/interceptors/go.sum @@ -37,8 +37,7 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= -github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -53,8 +52,8 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.11.0 h1:ib4sjIrwZKxE5u/Japgo/7SJV3PvgjGiRNAvTVGqQl8= -github.com/stretchr/testify v1.11.0/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= +github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu7U= +github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.opentelemetry.io/otel v1.29.0 h1:PdomN/Al4q/lN6iBJEN3AwPvUiHPMlt93c8bqTG5Llw= go.opentelemetry.io/otel v1.29.0/go.mod h1:N/WtXPs1CNCUEx+Agz5uouwCba+i+bJGFicT8SR4NP8= @@ -71,8 +70,7 @@ go.opentelemetry.io/otel/trace v1.29.0/go.mod h1:eHl3w0sp3paPkYstJOmAimxhiFXPg+M golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= -golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -81,21 +79,19 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= -golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= -golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= +golang.org/x/sync v0.18.0 h1:kr88TuHDroi+UVf+0hZnirlk8o8T+4MrK6mr60WkH/I= +golang.org/x/sync v0.18.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.36.0 h1:KVRy2GtZBrk1cBYA7MKu5bEZFxQk4NIDV6RLVcC8o0k= -golang.org/x/sys v0.36.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= +golang.org/x/sys v0.38.0 h1:3yZWxaJjBmCWXqhN1qh02AkOnCQ1poK6oF+a7xWL6Gc= golang.org/x/sys v0.38.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= diff --git a/examples/sasl_scram_client/go.sum b/examples/sasl_scram_client/go.sum index 10ea8b4c2..a42a7a7d4 100644 --- a/examples/sasl_scram_client/go.sum +++ b/examples/sasl_scram_client/go.sum @@ -28,8 +28,7 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= -github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -44,8 +43,8 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.11.0 h1:ib4sjIrwZKxE5u/Japgo/7SJV3PvgjGiRNAvTVGqQl8= -github.com/stretchr/testify v1.11.0/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= +github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu7U= +github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= github.com/xdg-go/scram v1.1.2 h1:FHX5I5B4i4hKRVRBCFRxq1iQRej7WO3hhBuJf+UUySY= @@ -56,8 +55,7 @@ github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5t golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= -golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -66,13 +64,12 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= -golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= -golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= +golang.org/x/sync v0.18.0 h1:kr88TuHDroi+UVf+0hZnirlk8o8T+4MrK6mr60WkH/I= +golang.org/x/sync v0.18.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -87,8 +84,7 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.29.0 h1:1neNs90w9YzJ9BocxfsQNHKuAT4pkghyXc4nhZ6sJvk= -golang.org/x/text v0.29.0/go.mod h1:7MhJOA9CD2qZyOKYazxdYMF85OwPdEr9jTtBpO7ydH4= +golang.org/x/text v0.31.0 h1:aC8ghyu4JhP8VojJ2lEHBnochRno1sgL6nEi9WGFGMM= golang.org/x/text v0.31.0/go.mod h1:tKRAlv61yKIjGGHX/4tP1LTbc13YSec1pxVEWXzfoeM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= diff --git a/examples/txn_producer/go.sum b/examples/txn_producer/go.sum index 1f8665b2d..ff7614215 100644 --- a/examples/txn_producer/go.sum +++ b/examples/txn_producer/go.sum @@ -28,8 +28,7 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= -github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= @@ -44,14 +43,13 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.11.0 h1:ib4sjIrwZKxE5u/Japgo/7SJV3PvgjGiRNAvTVGqQl8= -github.com/stretchr/testify v1.11.0/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= +github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu7U= +github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= -golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -60,13 +58,12 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= -golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= -golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= -golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= +golang.org/x/sync v0.18.0 h1:kr88TuHDroi+UVf+0hZnirlk8o8T+4MrK6mr60WkH/I= +golang.org/x/sync v0.18.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= From 9efa7b37184dcae9d6fbed5dfaef95d03e2479fe Mon Sep 17 00:00:00 2001 From: DCjanus Date: Mon, 1 Dec 2025 11:06:39 +0800 Subject: [PATCH 11/17] chore(examples): bump sarama dep but keep local replace Signed-off-by: DCjanus --- examples/consumergroup/go.mod | 4 ++-- examples/consumergroup/go.sum | 4 ++-- examples/exactly_once/go.mod | 2 +- examples/http_server/go.mod | 2 +- examples/interceptors/go.mod | 2 +- examples/sasl_scram_client/go.mod | 2 +- examples/txn_producer/go.mod | 2 +- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/examples/consumergroup/go.mod b/examples/consumergroup/go.mod index 723ff7c58..7b0ca562f 100644 --- a/examples/consumergroup/go.mod +++ b/examples/consumergroup/go.mod @@ -2,14 +2,14 @@ module github.com/IBM/sarama/examples/consumer go 1.24.0 -require github.com/IBM/sarama v1.45.0 +require github.com/IBM/sarama v1.46.3 require ( github.com/davecgh/go-spew v1.1.1 // indirect github.com/eapache/go-resiliency v1.7.0 // indirect github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3 // indirect github.com/eapache/queue v1.1.0 // indirect - github.com/golang/snappy v0.0.4 // indirect + github.com/golang/snappy v1.0.0 // indirect github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/jcmturner/aescts/v2 v2.0.0 // indirect github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect diff --git a/examples/consumergroup/go.sum b/examples/consumergroup/go.sum index ff7614215..b8e525726 100644 --- a/examples/consumergroup/go.sum +++ b/examples/consumergroup/go.sum @@ -9,8 +9,8 @@ github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= -github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= -github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v1.0.0 h1:Oy607GVXHs7RtbggtPBnr2RmDArIsAefDwvrdWvRhGs= +github.com/golang/snappy v1.0.0/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= diff --git a/examples/exactly_once/go.mod b/examples/exactly_once/go.mod index 99fedd3a1..0976d1c3d 100644 --- a/examples/exactly_once/go.mod +++ b/examples/exactly_once/go.mod @@ -2,7 +2,7 @@ module github.com/IBM/sarama/examples/exactly_once go 1.24.0 -require github.com/IBM/sarama v1.45.0 +require github.com/IBM/sarama v1.46.3 require ( github.com/davecgh/go-spew v1.1.1 // indirect diff --git a/examples/http_server/go.mod b/examples/http_server/go.mod index 2f5c2452b..c9101b0b5 100644 --- a/examples/http_server/go.mod +++ b/examples/http_server/go.mod @@ -2,7 +2,7 @@ module github.com/IBM/sarama/examples/http_server go 1.24.0 -require github.com/IBM/sarama v1.45.0 +require github.com/IBM/sarama v1.46.3 require ( github.com/davecgh/go-spew v1.1.1 // indirect diff --git a/examples/interceptors/go.mod b/examples/interceptors/go.mod index e4ec20c81..27e17a156 100644 --- a/examples/interceptors/go.mod +++ b/examples/interceptors/go.mod @@ -3,7 +3,7 @@ module github.com/IBM/sarama/examples/interceptors go 1.24.0 require ( - github.com/IBM/sarama v1.45.0 + github.com/IBM/sarama v1.46.3 go.opentelemetry.io/otel v1.29.0 go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.29.0 go.opentelemetry.io/otel/sdk v1.29.0 diff --git a/examples/sasl_scram_client/go.mod b/examples/sasl_scram_client/go.mod index 471dc1280..c5b81490e 100644 --- a/examples/sasl_scram_client/go.mod +++ b/examples/sasl_scram_client/go.mod @@ -3,7 +3,7 @@ module github.com/IBM/sarama/examples/sasl_scram_client go 1.24.0 require ( - github.com/IBM/sarama v1.45.0 + github.com/IBM/sarama v1.46.3 github.com/xdg-go/scram v1.1.2 ) diff --git a/examples/txn_producer/go.mod b/examples/txn_producer/go.mod index 316dce1c1..4c2fad1b6 100644 --- a/examples/txn_producer/go.mod +++ b/examples/txn_producer/go.mod @@ -3,7 +3,7 @@ module github.com/IBM/sarama/examples/txn_producer go 1.24.0 require ( - github.com/IBM/sarama v1.45.0 + github.com/IBM/sarama v1.46.3 github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 ) From c73d2a2f2d09a1f9994a69604486ac6e44603774 Mon Sep 17 00:00:00 2001 From: Dominic Evans <8060970+dnwe@users.noreply.github.com> Date: Thu, 27 Nov 2025 16:12:01 +0000 Subject: [PATCH 12/17] fix(client): add nilguards to updateBroker (#3393) This _should_ be a rare edge case as the updateBroker func is only called from updateMetadata which does already do an up-front `client.Close()` check under read-lock before then acquiring the write lock. However, there's potentially a small window of opportunity that if client.Close() was called whilst metadata refresh was in-flight and for whatever reason the updateMetadata goroutine gets pre-empted in-between the readlock release and the write lock acquire then the client could have been closed and so `client.brokers` will be nil. I wouldn't have expected this to ever happen, but it was reported by a user in an older Sarama version in #3391 and there's no harm in adding the nilguard just in case. Signed-off-by: Dominic Evans Signed-off-by: DCjanus --- client.go | 7 ++++++ client_test.go | 68 ++++++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 75 insertions(+) diff --git a/client.go b/client.go index 0dc29e225..bc5e646b6 100644 --- a/client.go +++ b/client.go @@ -685,9 +685,16 @@ func (client *client) randomizeSeedBrokers(addrs []string) { } func (client *client) updateBroker(brokers []*Broker) { + if client.brokers == nil { + return + } + currentBroker := make(map[int32]*Broker, len(brokers)) for _, broker := range brokers { + if broker == nil { + continue + } currentBroker[broker.ID()] = broker if client.brokers[broker.ID()] == nil { // add new broker client.brokers[broker.ID()] = broker diff --git a/client_test.go b/client_test.go index 9d6242fa3..2fd819625 100644 --- a/client_test.go +++ b/client_test.go @@ -14,6 +14,7 @@ import ( "github.com/rcrowley/go-metrics" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestSimpleClient(t *testing.T) { @@ -1330,3 +1331,70 @@ func TestMetricsCleanup(t *testing.T) { t.Errorf("excepted 1 metric, found: %v", all) } } + +func TestUpdateBroker(t *testing.T) { + t.Run("closed client doesn't panic", func(t *testing.T) { + c := &client{} + fn := func() { + c.updateBroker(nil) + c.updateBroker([]*Broker{ + { + id: 0, + addr: "127.0.0.1:9092", + }, + }) + } + require.NotPanics(t, fn) + }) + + t.Run("open client adds new broker entries", func(t *testing.T) { + c := &client{ + brokers: make(map[int32]*Broker), + } + fn := func() { + c.updateBroker([]*Broker{ + { + id: 0, + addr: "127.0.0.1:9092", + }, + }) + } + require.NotPanics(t, fn) + require.Len(t, c.brokers, 1) + assert.Equal(t, 0, int(c.brokers[0].ID())) + assert.Equal(t, "127.0.0.1:9092", c.brokers[0].Addr()) + }) + + t.Run("open client adds, updates and removes broker entries", func(t *testing.T) { + c := &client{ + brokers: map[int32]*Broker{ + 0: { + id: 0, + addr: "127.0.0.1:9092", + }, + 1: { + id: 1, + addr: "127.0.0.1:9093", + }, + }, + } + fn := func() { + c.updateBroker([]*Broker{ + { + id: 1, + addr: "127.0.0.1:19093", // new addr for existing broker + }, + { + id: 2, + addr: "127.0.0.1:19094", + }, + }) + } + require.NotPanics(t, fn) + require.Len(t, c.brokers, 2) + assert.Equal(t, 1, int(c.brokers[1].ID())) + assert.Equal(t, "127.0.0.1:19093", c.brokers[1].Addr()) + assert.Equal(t, 2, int(c.brokers[2].ID())) + assert.Equal(t, "127.0.0.1:19094", c.brokers[2].Addr()) + }) +} From 01d79f5f98d239a4bd7ff1179763fcab0655b6bf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 27 Nov 2025 17:18:27 +0000 Subject: [PATCH 13/17] chore(ci): bump the actions group across 1 directory with 3 updates (#3388) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps the actions group with 3 updates in the / directory: [actions/setup-go](https://github.com/actions/setup-go), [actions/checkout](https://github.com/actions/checkout) and [actions/upload-artifact](https://github.com/actions/upload-artifact). Updates `actions/setup-go` from 6.0.0 to 6.1.0
Release notes

Sourced from actions/setup-go's releases.

v6.1.0

What's Changed

Enhancements

Dependency updates

New Contributors

Full Changelog: https://github.com/actions/setup-go/compare/v6...v6.1.0

Commits
  • 4dc6199 Bump semver and @​types/semver (#652)
  • f3787be Add comprehensive breaking changes documentation for v6 (#674)
  • 3a0c2c8 Bump actions/publish-action from 0.3.0 to 0.4.0 (#641)
  • faf5242 Add support for .tool-versions file in setup-go, update workflow (#673)
  • 7bc60db Fall back to downloading from go.dev/dl instead of storage.googleapis.com/gol...
  • c0137ca Bump eslint-config-prettier from 10.0.1 to 10.1.8 and document breaking chang...
  • See full diff in compare view

Updates `actions/checkout` from 5.0.1 to 6.0.0
Release notes

Sourced from actions/checkout's releases.

v6.0.0

What's Changed

Full Changelog: https://github.com/actions/checkout/compare/v5.0.0...v6.0.0

v6-beta

What's Changed

Updated persist-credentials to store the credentials under $RUNNER_TEMP instead of directly in the local git config.

This requires a minimum Actions Runner version of v2.329.0 to access the persisted credentials for Docker container action scenarios.

Changelog

Sourced from actions/checkout's changelog.

Changelog

V6.0.0

V5.0.1

V5.0.0

V4.3.1

V4.3.0

v4.2.2

v4.2.1

v4.2.0

v4.1.7

v4.1.6

v4.1.5

... (truncated)

Commits

Updates `actions/upload-artifact` from 4.6.2 to 5.0.0
Release notes

Sourced from actions/upload-artifact's releases.

v5.0.0

What's Changed

BREAKING CHANGE: this update supports Node v24.x. This is not a breaking change per-se but we're treating it as such.

New Contributors

Full Changelog: https://github.com/actions/upload-artifact/compare/v4...v5.0.0

Commits
  • 330a01c Merge pull request #734 from actions/danwkennedy/prepare-5.0.0
  • 03f2824 Update github.dep.yml
  • 905a1ec Prepare v5.0.0
  • 2d9f9cd Merge pull request #725 from patrikpolyak/patch-1
  • 9687587 Merge branch 'main' into patch-1
  • 2848b2c Merge pull request #727 from danwkennedy/patch-1
  • 9b51177 Spell out the first use of GHES
  • cd231ca Update GHES guidance to include reference to Node 20 version
  • de65e23 Merge pull request #712 from actions/nebuk89-patch-1
  • 8747d8c Update README.md
  • Additional commits viewable in compare view

Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore major version` will close this group update PR and stop Dependabot creating any more for the specific dependency's major version (unless you unignore this specific dependency's major version or upgrade to it yourself) - `@dependabot ignore minor version` will close this group update PR and stop Dependabot creating any more for the specific dependency's minor version (unless you unignore this specific dependency's minor version or upgrade to it yourself) - `@dependabot ignore ` will close this group update PR and stop Dependabot creating any more for the specific dependency (unless you unignore this specific dependency or upgrade to it yourself) - `@dependabot unignore ` will remove all of the ignore conditions of the specified dependency - `@dependabot unignore ` will remove the ignore condition of the specified dependency and ignore conditions
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Signed-off-by: DCjanus --- .github/workflows/apidiff.yml | 6 +++--- .github/workflows/ci.yml | 8 ++++---- .github/workflows/codeql-analysis.yml | 4 ++-- .github/workflows/dependency-review.yml | 2 +- .github/workflows/fuzz.yml | 4 ++-- .github/workflows/fvt.yml | 6 +++--- .github/workflows/i386.yml | 4 ++-- .github/workflows/renovate-config.yml | 2 +- .github/workflows/scorecard.yml | 4 ++-- 9 files changed, 20 insertions(+), 20 deletions(-) diff --git a/.github/workflows/apidiff.yml b/.github/workflows/apidiff.yml index 90de74ccc..8c0107af2 100644 --- a/.github/workflows/apidiff.yml +++ b/.github/workflows/apidiff.yml @@ -29,7 +29,7 @@ jobs: if: github.base_ref steps: - name: Setup Go - uses: actions/setup-go@44694675825211faa026b3c33043df3e48a5fa00 # v6.0.0 + uses: actions/setup-go@4dc6199c7b1a012772edbd06daecab0f50c9053c # v6.1.0 with: go-version: stable - name: Add GOBIN to PATH @@ -37,7 +37,7 @@ jobs: - name: Install apidiff cmd run: go install golang.org/x/exp/cmd/apidiff@v0.0.0-20250813145105-42675adae3e6 - name: Checkout base code - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: ref: ${{ github.base_ref }} path: "base" @@ -46,7 +46,7 @@ jobs: run: apidiff -m -w ../baseline.bin . working-directory: "base" - name: Checkout updated code - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: path: "updated" persist-credentials: false diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 039f055c9..ea48624b9 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -41,11 +41,11 @@ jobs: matrix: go-version: [stable] steps: - - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + - uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: persist-credentials: false - name: Setup Go - uses: actions/setup-go@44694675825211faa026b3c33043df3e48a5fa00 # v6.0.0 + uses: actions/setup-go@4dc6199c7b1a012772edbd06daecab0f50c9053c # v6.1.0 with: go-version: ${{ matrix.go-version }} - name: Staticcheck @@ -73,11 +73,11 @@ jobs: DEBUG: true GOFLAGS: -trimpath steps: - - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + - uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: persist-credentials: false - name: Setup Go - uses: actions/setup-go@44694675825211faa026b3c33043df3e48a5fa00 # v6.0.0 + uses: actions/setup-go@4dc6199c7b1a012772edbd06daecab0f50c9053c # v6.1.0 with: go-version: ${{ matrix.go-version }} - name: Test (Unit) diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index 08d51ca40..b4fe19686 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -35,7 +35,7 @@ jobs: language: ["actions", "go"] steps: - name: Checkout repository - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: persist-credentials: false - name: Initialize CodeQL @@ -43,7 +43,7 @@ jobs: with: languages: ${{ matrix.language }} - name: Setup Go - uses: actions/setup-go@44694675825211faa026b3c33043df3e48a5fa00 # v6.0.0 + uses: actions/setup-go@4dc6199c7b1a012772edbd06daecab0f50c9053c # v6.1.0 with: go-version: stable - name: Autobuild diff --git a/.github/workflows/dependency-review.yml b/.github/workflows/dependency-review.yml index c0e92ceac..6e653e9ee 100644 --- a/.github/workflows/dependency-review.yml +++ b/.github/workflows/dependency-review.yml @@ -26,7 +26,7 @@ jobs: runs-on: ubuntu-latest steps: - name: 'Checkout Repository' - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: persist-credentials: false - name: 'Dependency Review' diff --git a/.github/workflows/fuzz.yml b/.github/workflows/fuzz.yml index 7d67e88e9..c90a46810 100644 --- a/.github/workflows/fuzz.yml +++ b/.github/workflows/fuzz.yml @@ -30,11 +30,11 @@ jobs: env: GOFLAGS: -trimpath steps: - - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + - uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: persist-credentials: false - name: Setup Go - uses: actions/setup-go@44694675825211faa026b3c33043df3e48a5fa00 # v6.0.0 + uses: actions/setup-go@4dc6199c7b1a012772edbd06daecab0f50c9053c # v6.1.0 with: go-version: stable - name: Run any fuzzing tests diff --git a/.github/workflows/fvt.yml b/.github/workflows/fvt.yml index 126751b9c..f1586dbeb 100644 --- a/.github/workflows/fvt.yml +++ b/.github/workflows/fvt.yml @@ -36,7 +36,7 @@ jobs: KAFKA_VERSION: ${{ inputs.kafka-version }} SCALA_VERSION: ${{ inputs.scala-version }} steps: - - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + - uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: persist-credentials: false - name: Setup Docker @@ -53,7 +53,7 @@ jobs: *.cache-from=type=gha,scope=fvt-kafka-${{ inputs.kafka-version }} *.cache-to=type=gha,scope=fvt-kafka-${{ inputs.kafka-version }},mode=max - name: Setup Go - uses: actions/setup-go@44694675825211faa026b3c33043df3e48a5fa00 # v6.0.0 + uses: actions/setup-go@4dc6199c7b1a012772edbd06daecab0f50c9053c # v6.1.0 with: go-version: ${{ inputs.go-version }} - name: Setup Docker Compose @@ -92,7 +92,7 @@ jobs: if [ -f "fvt-kafka-${KAFKA_VERSION}.pcap" ]; then sudo chmod a+r "fvt-kafka-${KAFKA_VERSION}.pcap"; fi - name: Upload pcap file if: always() - uses: actions/upload-artifact@ea165f8d65b6e75b540449e92b4886f43607fa02 # v4.6.2 + uses: actions/upload-artifact@330a01c490aca151604b8cf639adc76d48f6c5d4 # v5.0.0 with: name: fvt-kafka-${{ inputs.kafka-version }}.pcap path: fvt-kafka-${{ inputs.kafka-version }}.pcap diff --git a/.github/workflows/i386.yml b/.github/workflows/i386.yml index 667616e97..1dc6747bf 100644 --- a/.github/workflows/i386.yml +++ b/.github/workflows/i386.yml @@ -30,11 +30,11 @@ jobs: pull-requests: read # for golangci/golangci-lint-action to fetch pull requests runs-on: ubuntu-latest steps: - - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + - uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: persist-credentials: false - name: Setup Go - uses: actions/setup-go@44694675825211faa026b3c33043df3e48a5fa00 # v6.0.0 + uses: actions/setup-go@4dc6199c7b1a012772edbd06daecab0f50c9053c # v6.1.0 with: go-version: stable - name: staticcheck diff --git a/.github/workflows/renovate-config.yml b/.github/workflows/renovate-config.yml index ad44407ec..7c80d7c21 100644 --- a/.github/workflows/renovate-config.yml +++ b/.github/workflows/renovate-config.yml @@ -12,7 +12,7 @@ jobs: validate: runs-on: ubuntu-latest steps: - - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + - uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: sparse-checkout: | .github/renovate.json5 diff --git a/.github/workflows/scorecard.yml b/.github/workflows/scorecard.yml index 5dd214f3c..85c252efb 100644 --- a/.github/workflows/scorecard.yml +++ b/.github/workflows/scorecard.yml @@ -38,7 +38,7 @@ jobs: steps: - name: "Checkout code" - uses: actions/checkout@93cb6efe18208431cddfb8368fd83d5badbf9bfd # v5.0.1 + uses: actions/checkout@1af3b93b6815bc44a9784bd300feb67ff0d1eeb3 # v6.0.0 with: persist-credentials: false @@ -65,7 +65,7 @@ jobs: # Upload the results as artifacts (optional). Commenting out will disable uploads of run results in SARIF # format to the repository Actions tab. - name: "Upload artifact" - uses: actions/upload-artifact@ea165f8d65b6e75b540449e92b4886f43607fa02 # v4.6.2 + uses: actions/upload-artifact@330a01c490aca151604b8cf639adc76d48f6c5d4 # v5.0.0 with: name: SARIF file path: results.sarif From 29e7bb08d4ec6d40bde08b2b21793f011174cc78 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 27 Nov 2025 17:18:52 +0000 Subject: [PATCH 14/17] chore(ci): bump github/codeql-action from 4.30.9 to 4.31.0 (#3364) Bumps [github/codeql-action](https://github.com/github/codeql-action) from 4.30.9 to 4.31.0.
Release notes

Sourced from github/codeql-action's releases.

v4.31.0

CodeQL Action Changelog

See the releases page for the relevant changes to the CodeQL CLI and language packs.

4.31.0 - 24 Oct 2025

  • Bump minimum CodeQL bundle version to 2.17.6. #3223
  • When SARIF files are uploaded by the analyze or upload-sarif actions, the CodeQL Action automatically performs post-processing steps to prepare the data for the upload. Previously, these post-processing steps were only performed before an upload took place. We are now changing this so that the post-processing steps will always be performed, even when the SARIF files are not uploaded. This does not change anything for the upload-sarif action. For analyze, this may affect Advanced Setup for CodeQL users who specify a value other than always for the upload input. #3222

See the full CHANGELOG.md for more information.

Changelog

Sourced from github/codeql-action's changelog.

CodeQL Action Changelog

See the releases page for the relevant changes to the CodeQL CLI and language packs.

[UNRELEASED]

No user facing changes.

4.31.0 - 24 Oct 2025

  • Bump minimum CodeQL bundle version to 2.17.6. #3223
  • When SARIF files are uploaded by the analyze or upload-sarif actions, the CodeQL Action automatically performs post-processing steps to prepare the data for the upload. Previously, these post-processing steps were only performed before an upload took place. We are now changing this so that the post-processing steps will always be performed, even when the SARIF files are not uploaded. This does not change anything for the upload-sarif action. For analyze, this may affect Advanced Setup for CodeQL users who specify a value other than always for the upload input. #3222

4.30.9 - 17 Oct 2025

  • Update default CodeQL bundle version to 2.23.3. #3205
  • Experimental: A new setup-codeql action has been added which is similar to init, except it only installs the CodeQL CLI and does not initialize a database. Do not use this in production as it is part of an internal experiment and subject to change at any time. #3204

4.30.8 - 10 Oct 2025

No user facing changes.

4.30.7 - 06 Oct 2025

  • [v4+ only] The CodeQL Action now runs on Node.js v24. #3169

3.30.6 - 02 Oct 2025

  • Update default CodeQL bundle version to 2.23.2. #3168

3.30.5 - 26 Sep 2025

  • We fixed a bug that was introduced in 3.30.4 with upload-sarif which resulted in files without a .sarif extension not getting uploaded. #3160

3.30.4 - 25 Sep 2025

  • We have improved the CodeQL Action's ability to validate that the workflow it is used in does not use different versions of the CodeQL Action for different workflow steps. Mixing different versions of the CodeQL Action in the same workflow is unsupported and can lead to unpredictable results. A warning will now be emitted from the codeql-action/init step if different versions of the CodeQL Action are detected in the workflow file. Additionally, an error will now be thrown by the other CodeQL Action steps if they load a configuration file that was generated by a different version of the codeql-action/init step. #3099 and #3100
  • We added support for reducing the size of dependency caches for Java analyses, which will reduce cache usage and speed up workflows. This will be enabled automatically at a later time. #3107
  • You can now run the latest CodeQL nightly bundle by passing tools: nightly to the init action. In general, the nightly bundle is unstable and we only recommend running it when directed by GitHub staff. #3130
  • Update default CodeQL bundle version to 2.23.1. #3118

3.30.3 - 10 Sep 2025

No user facing changes.

3.30.2 - 09 Sep 2025

  • Fixed a bug which could cause language autodetection to fail. #3084
  • Experimental: The quality-queries input that was added in 3.29.2 as part of an internal experiment is now deprecated and will be removed in an upcoming version of the CodeQL Action. It has been superseded by a new analysis-kinds input, which is part of the same internal experiment. Do not use this in production as it is subject to change at any time. #3064

... (truncated)

Commits
  • 4e94bd1 Merge pull request #3235 from github/update-v4.31.0-1d36546c1
  • 8f11182 Update changelog for v4.31.0
  • 1d36546 Merge pull request #3234 from github/mbg/changelog/post-processing
  • 08ada26 Add changelog entry for post-processing change
  • b843cbe Merge pull request #3233 from github/mbg/getOptionalEnvVar
  • 1ecd563 Use getOptionalEnvVar in writePostProcessedFiles
  • e576807 Merge pull request #3223 from github/henrymercer/bump-minimum
  • ad35676 Add getOptionalEnvVar function
  • d75645b Merge pull request #3222 from github/mbg/upload-lib/post-process
  • 710606c Check that outputPath is non-empty
  • Additional commits viewable in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=github/codeql-action&package-manager=github_actions&previous-version=4.30.9&new-version=4.31.0)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) You can trigger a rebase of this PR by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
> **Note** > Automatic rebases have been disabled on this pull request as it has been open for over 30 days. Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Signed-off-by: DCjanus --- .github/workflows/codeql-analysis.yml | 6 +++--- .github/workflows/scorecard.yml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index b4fe19686..1d7746612 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -39,7 +39,7 @@ jobs: with: persist-credentials: false - name: Initialize CodeQL - uses: github/codeql-action/init@014f16e7ab1402f30e7c3329d33797e7948572db # v4.31.3 + uses: github/codeql-action/init@e12f0178983d466f2f6028f5cc7a6d786fd97f4b # v4.31.4 with: languages: ${{ matrix.language }} - name: Setup Go @@ -47,6 +47,6 @@ jobs: with: go-version: stable - name: Autobuild - uses: github/codeql-action/autobuild@014f16e7ab1402f30e7c3329d33797e7948572db # v4.31.3 + uses: github/codeql-action/autobuild@e12f0178983d466f2f6028f5cc7a6d786fd97f4b # v4.31.4 - name: Perform CodeQL Analysis - uses: github/codeql-action/analyze@014f16e7ab1402f30e7c3329d33797e7948572db # v4.31.3 + uses: github/codeql-action/analyze@e12f0178983d466f2f6028f5cc7a6d786fd97f4b # v4.31.4 diff --git a/.github/workflows/scorecard.yml b/.github/workflows/scorecard.yml index 85c252efb..df55519d5 100644 --- a/.github/workflows/scorecard.yml +++ b/.github/workflows/scorecard.yml @@ -73,6 +73,6 @@ jobs: # Upload the results to GitHub's code scanning dashboard. - name: "Upload to code-scanning" - uses: github/codeql-action/upload-sarif@014f16e7ab1402f30e7c3329d33797e7948572db # v4.31.3 + uses: github/codeql-action/upload-sarif@e12f0178983d466f2f6028f5cc7a6d786fd97f4b # v4.31.4 with: sarif_file: results.sarif From abdd82c8130efd8210797690943def04558be984 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Thu, 27 Nov 2025 17:27:04 +0000 Subject: [PATCH 15/17] chore(deps): update docker/bake-action action to v6.10.0 (#3392) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR contains the following updates: | Package | Type | Update | Change | OpenSSF | |---|---|---|---|---| | [docker/bake-action](https://redirect.github.com/docker/bake-action) | action | minor | `v6.9.0` -> `v6.10.0` | [![OpenSSF Scorecard](https://api.securityscorecards.dev/projects/github.com/docker/bake-action/badge)](https://securityscorecards.dev/viewer/?uri=github.com/docker/bake-action) | --- ### Release Notes
docker/bake-action (docker/bake-action) ### [`v6.10.0`](https://redirect.github.com/docker/bake-action/releases/tag/v6.10.0) [Compare Source](https://redirect.github.com/docker/bake-action/compare/v6.9.0...v6.10.0) - Check provenance attestation set in bake definition before overriding by [@​crazy-max](https://redirect.github.com/crazy-max) in [#​359](https://redirect.github.com/docker/bake-action/pull/359) - Bump [@​docker/actions-toolkit](https://redirect.github.com/docker/actions-toolkit) from 0.63.0 to 0.68.0 in [#​360](https://redirect.github.com/docker/bake-action/pull/360) - Bump js-yaml from 3.14.1 to 3.14.2 in [#​357](https://redirect.github.com/docker/bake-action/pull/357) **Full Changelog**:
--- ### Configuration 📅 **Schedule**: Branch creation - At any time (no schedule defined), Automerge - At any time (no schedule defined). 🚦 **Automerge**: Disabled by config. Please merge this manually once you are satisfied. ♻ **Rebasing**: Whenever PR becomes conflicted, or you tick the rebase/retry checkbox. 🔕 **Ignore**: Close this PR and you won't be reminded about this update again. --- - [ ] If you want to rebase/retry this PR, check this box --- This PR was generated by [Mend Renovate](https://mend.io/renovate/). View the [repository job log](https://developer.mend.io/github/IBM/sarama). Signed-off-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Signed-off-by: DCjanus --- .github/workflows/fvt.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/fvt.yml b/.github/workflows/fvt.yml index f1586dbeb..20dd11eb8 100644 --- a/.github/workflows/fvt.yml +++ b/.github/workflows/fvt.yml @@ -43,7 +43,7 @@ jobs: uses: docker/setup-buildx-action@e468171a9de216ec08956ac3ada2f0791b6bd435 # v3.11.1 id: buildx - name: Build FVT Docker Image - uses: docker/bake-action@3acf805d94d93a86cce4ca44798a76464a75b88c # v6.9.0 + uses: docker/bake-action@5be5f02ff8819ecd3092ea6b2e6261c31774f2b4 # v6.10.0 with: builder: ${{ steps.buildx.outputs.name }} files: docker-compose.yml From b4f09896548d0599a5657b5db5806b708de99675 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Thu, 27 Nov 2025 17:32:12 +0000 Subject: [PATCH 16/17] chore(deps): update dependency golangci/golangci-lint to v2.6.2 (#3366) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR contains the following updates: | Package | Update | Change | OpenSSF | |---|---|---|---| | [golangci/golangci-lint](https://redirect.github.com/golangci/golangci-lint) | minor | `v2.5.0` -> `v2.6.2` | [![OpenSSF Scorecard](https://api.securityscorecards.dev/projects/github.com/golangci/golangci-lint/badge)](https://securityscorecards.dev/viewer/?uri=github.com/golangci/golangci-lint) | --- ### Release Notes
golangci/golangci-lint (golangci/golangci-lint) ### [`v2.6.2`](https://redirect.github.com/golangci/golangci-lint/blob/HEAD/CHANGELOG.md#v262) [Compare Source](https://redirect.github.com/golangci/golangci-lint/compare/v2.6.1...v2.6.2) 1. Bug fixes - `fmt` command with symlinks - use file depending on build configuration to invalidate cache 2. Linters bug fixes - `testableexamples`: from 1.0.0 to 1.0.1 - `testpackage`: from 1.1.1 to 1.1.2 ### [`v2.6.1`](https://redirect.github.com/golangci/golangci-lint/compare/v2.6.0...v2.6.1) [Compare Source](https://redirect.github.com/golangci/golangci-lint/compare/v2.6.0...v2.6.1) ### [`v2.6.0`](https://redirect.github.com/golangci/golangci-lint/blob/HEAD/CHANGELOG.md#v260) [Compare Source](https://redirect.github.com/golangci/golangci-lint/compare/v2.5.0...v2.6.0) 1. New linters - Add `modernize` analyzer suite 2. Linters new features or changes - `arangolint`: from 0.2.0 to 0.3.1 - `dupword`: from 0.1.6 to 0.1.7 (new option `comments-only`) - `go-critic`: from 0.13.0 to 0.14.0 (new rules/checkers: `zeroByteRepeat`, `dupOption`) - `gofumpt`: from 0.9.1 to 0.9.2 ("clothe" naked returns is now controlled by the `extra-rules` option) - `perfsprint`: from 0.9.1 to 0.10.0 (new options: `concat-loop`, `loop-other-ops`) - `wsl`: from 5.2.0 to 5.3.0 3. Linters bug fixes - `dupword`: from 0.1.6 to 0.1.7 - `durationcheck`: from 0.0.10 to 0.0.11 - `exptostd`: from 0.4.4 to 0.4.5 - `fatcontext`: from 0.8.1 to 0.9.0 - `forbidigo`: from 2.1.0 to 2.3.0 - `ginkgolinter`: from 0.21.0 to 0.21.2 - `godoc-lint`: from 0.10.0 to 0.10.1 - `gomoddirectives`: from 0.7.0 to 0.7.1 - `gosec`: from 2.22.8 to 2.22.10 - `makezero`: from 2.0.1 to 2.1.0 - `nilerr`: from 0.1.1 to 0.1.2 - `paralleltest`: from 1.0.14 to 1.0.15 - `protogetter`: from 0.3.16 to 0.3.17 - `unparam`: from [`0df0534`](https://redirect.github.com/golangci/golangci-lint/commit/0df0534333a4) to [`5beb8c8`](https://redirect.github.com/golangci/golangci-lint/commit/5beb8c8f8f15) 4. Misc. - fix: ignore some files to hash the version for custom build
--- ### Configuration 📅 **Schedule**: Branch creation - At any time (no schedule defined), Automerge - At any time (no schedule defined). 🚦 **Automerge**: Disabled by config. Please merge this manually once you are satisfied. ♻ **Rebasing**: Whenever PR becomes conflicted, or you tick the rebase/retry checkbox. 🔕 **Ignore**: Close this PR and you won't be reminded about this update again. --- - [ ] If you want to rebase/retry this PR, check this box --- This PR was generated by [Mend Renovate](https://mend.io/renovate/). View the [repository job log](https://developer.mend.io/github/IBM/sarama). --------- Signed-off-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Signed-off-by: Dominic Evans Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> Co-authored-by: Dominic Evans Signed-off-by: DCjanus --- .github/workflows/ci.yml | 2 +- admin.go | 1 + client_test.go | 2 +- config.go | 1 + 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index ea48624b9..48d842318 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -23,7 +23,7 @@ env: # Use the Go toolchain installed by setup-go GOTOOLCHAIN: local # renovate: datasource=github-releases depName=golangci/golangci-lint - GOLANGCI_LINT_VERSION: v2.5.0 + GOLANGCI_LINT_VERSION: v2.6.2 # renovate: datasource=github-releases depName=dominikh/go-tools STATICCHECK_VERSION: 2025.1.1 # renovate: datasource=github-releases depName=mfridman/tparse diff --git a/admin.go b/admin.go index 42f6fa0c4..da220ed2a 100644 --- a/admin.go +++ b/admin.go @@ -83,6 +83,7 @@ type ClusterAdmin interface { // This operation is not transactional so it may succeed or fail. // If you attempt to add an ACL that duplicates an existing ACL, no error will be raised, but // no changes will be made. This operation is supported by brokers with version 0.11.0.0 or higher. + // // Deprecated: Use CreateACLs instead. CreateACL(resource Resource, acl Acl) error diff --git a/client_test.go b/client_test.go index 2fd819625..72e825b22 100644 --- a/client_test.go +++ b/client_test.go @@ -595,7 +595,7 @@ func TestClientRefreshBrokers(t *testing.T) { newSeedBrokers := []string{"localhost:12345"} _ = client.RefreshBrokers(newSeedBrokers) - if client.seedBrokers[0].addr != newSeedBrokers[0] { + if len(client.seedBrokers) == 0 || client.seedBrokers[0].addr != newSeedBrokers[0] { t.Error("Seed broker not updated") } if len(client.Brokers()) != 0 { diff --git a/config.go b/config.go index 5bac2b50a..80758e9bb 100644 --- a/config.go +++ b/config.go @@ -327,6 +327,7 @@ type Config struct { } Rebalance struct { // Strategy for allocating topic partitions to members. + // // Deprecated: Strategy exists for historical compatibility // and should not be used. Please use GroupStrategies. Strategy BalanceStrategy From 4520b6c36ecc58e4df0a05266c10c752a76500bd Mon Sep 17 00:00:00 2001 From: DCjanus Date: Fri, 19 Dec 2025 14:47:34 +0800 Subject: [PATCH 17/17] chore(examples): tidy modules for compress v1.18.2 Signed-off-by: DCjanus --- examples/consumergroup/go.mod | 2 +- examples/consumergroup/go.sum | 4 ++-- examples/exactly_once/go.mod | 2 +- examples/exactly_once/go.sum | 4 ++-- examples/http_server/go.mod | 2 +- examples/http_server/go.sum | 4 ++-- examples/interceptors/go.mod | 2 +- examples/interceptors/go.sum | 4 ++-- examples/sasl_scram_client/go.mod | 2 +- examples/sasl_scram_client/go.sum | 4 ++-- examples/txn_producer/go.mod | 2 +- examples/txn_producer/go.sum | 4 ++-- 12 files changed, 18 insertions(+), 18 deletions(-) diff --git a/examples/consumergroup/go.mod b/examples/consumergroup/go.mod index 7b0ca562f..7fa672a9a 100644 --- a/examples/consumergroup/go.mod +++ b/examples/consumergroup/go.mod @@ -16,7 +16,7 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.1 // indirect + github.com/klauspost/compress v1.18.2 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect golang.org/x/crypto v0.45.0 // indirect diff --git a/examples/consumergroup/go.sum b/examples/consumergroup/go.sum index b8e525726..7486c1530 100644 --- a/examples/consumergroup/go.sum +++ b/examples/consumergroup/go.sum @@ -28,8 +28,8 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= -github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= +github.com/klauspost/compress v1.18.2 h1:iiPHWW0YrcFgpBYhsA6D1+fqHssJscY/Tm/y2Uqnapk= +github.com/klauspost/compress v1.18.2/go.mod h1:R0h/fSBs8DE4ENlcrlib3PsXS61voFxhIs2DeRhCvJ4= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= diff --git a/examples/exactly_once/go.mod b/examples/exactly_once/go.mod index 0976d1c3d..132b26bb0 100644 --- a/examples/exactly_once/go.mod +++ b/examples/exactly_once/go.mod @@ -16,7 +16,7 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.1 // indirect + github.com/klauspost/compress v1.18.2 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect golang.org/x/crypto v0.45.0 // indirect diff --git a/examples/exactly_once/go.sum b/examples/exactly_once/go.sum index ff7614215..5aa6fda55 100644 --- a/examples/exactly_once/go.sum +++ b/examples/exactly_once/go.sum @@ -28,8 +28,8 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= -github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= +github.com/klauspost/compress v1.18.2 h1:iiPHWW0YrcFgpBYhsA6D1+fqHssJscY/Tm/y2Uqnapk= +github.com/klauspost/compress v1.18.2/go.mod h1:R0h/fSBs8DE4ENlcrlib3PsXS61voFxhIs2DeRhCvJ4= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= diff --git a/examples/http_server/go.mod b/examples/http_server/go.mod index c9101b0b5..bae92128b 100644 --- a/examples/http_server/go.mod +++ b/examples/http_server/go.mod @@ -16,7 +16,7 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.1 // indirect + github.com/klauspost/compress v1.18.2 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect golang.org/x/crypto v0.45.0 // indirect diff --git a/examples/http_server/go.sum b/examples/http_server/go.sum index ff7614215..5aa6fda55 100644 --- a/examples/http_server/go.sum +++ b/examples/http_server/go.sum @@ -28,8 +28,8 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= -github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= +github.com/klauspost/compress v1.18.2 h1:iiPHWW0YrcFgpBYhsA6D1+fqHssJscY/Tm/y2Uqnapk= +github.com/klauspost/compress v1.18.2/go.mod h1:R0h/fSBs8DE4ENlcrlib3PsXS61voFxhIs2DeRhCvJ4= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= diff --git a/examples/interceptors/go.mod b/examples/interceptors/go.mod index 27e17a156..f10003ae8 100644 --- a/examples/interceptors/go.mod +++ b/examples/interceptors/go.mod @@ -25,7 +25,7 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.1 // indirect + github.com/klauspost/compress v1.18.2 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect go.opentelemetry.io/otel/metric v1.29.0 // indirect diff --git a/examples/interceptors/go.sum b/examples/interceptors/go.sum index aa2f9bdd8..b30ab660b 100644 --- a/examples/interceptors/go.sum +++ b/examples/interceptors/go.sum @@ -37,8 +37,8 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= -github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= +github.com/klauspost/compress v1.18.2 h1:iiPHWW0YrcFgpBYhsA6D1+fqHssJscY/Tm/y2Uqnapk= +github.com/klauspost/compress v1.18.2/go.mod h1:R0h/fSBs8DE4ENlcrlib3PsXS61voFxhIs2DeRhCvJ4= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= diff --git a/examples/sasl_scram_client/go.mod b/examples/sasl_scram_client/go.mod index c5b81490e..1b81aee54 100644 --- a/examples/sasl_scram_client/go.mod +++ b/examples/sasl_scram_client/go.mod @@ -19,7 +19,7 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.1 // indirect + github.com/klauspost/compress v1.18.2 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect github.com/rcrowley/go-metrics v0.0.0-20250401214520-65e299d6c5c9 // indirect github.com/xdg-go/pbkdf2 v1.0.0 // indirect diff --git a/examples/sasl_scram_client/go.sum b/examples/sasl_scram_client/go.sum index a42a7a7d4..0d8e35d5d 100644 --- a/examples/sasl_scram_client/go.sum +++ b/examples/sasl_scram_client/go.sum @@ -28,8 +28,8 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= -github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= +github.com/klauspost/compress v1.18.2 h1:iiPHWW0YrcFgpBYhsA6D1+fqHssJscY/Tm/y2Uqnapk= +github.com/klauspost/compress v1.18.2/go.mod h1:R0h/fSBs8DE4ENlcrlib3PsXS61voFxhIs2DeRhCvJ4= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= diff --git a/examples/txn_producer/go.mod b/examples/txn_producer/go.mod index 4c2fad1b6..ad95d3186 100644 --- a/examples/txn_producer/go.mod +++ b/examples/txn_producer/go.mod @@ -19,7 +19,7 @@ require ( github.com/jcmturner/gofork v1.7.6 // indirect github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect - github.com/klauspost/compress v1.18.1 // indirect + github.com/klauspost/compress v1.18.2 // indirect github.com/pierrec/lz4/v4 v4.1.22 // indirect golang.org/x/crypto v0.45.0 // indirect golang.org/x/net v0.47.0 // indirect diff --git a/examples/txn_producer/go.sum b/examples/txn_producer/go.sum index ff7614215..5aa6fda55 100644 --- a/examples/txn_producer/go.sum +++ b/examples/txn_producer/go.sum @@ -28,8 +28,8 @@ github.com/jcmturner/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh6 github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= -github.com/klauspost/compress v1.18.1 h1:bcSGx7UbpBqMChDtsF28Lw6v/G94LPrrbMbdC3JH2co= -github.com/klauspost/compress v1.18.1/go.mod h1:ZQFFVG+MdnR0P+l6wpXgIL4NTtwiKIdBnrBd8Nrxr+0= +github.com/klauspost/compress v1.18.2 h1:iiPHWW0YrcFgpBYhsA6D1+fqHssJscY/Tm/y2Uqnapk= +github.com/klauspost/compress v1.18.2/go.mod h1:R0h/fSBs8DE4ENlcrlib3PsXS61voFxhIs2DeRhCvJ4= github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU= github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=