From d4df7a902ecec8a6e4e4f0beae9636923a5c1287 Mon Sep 17 00:00:00 2001 From: redwrasse Date: Tue, 3 Sep 2024 16:02:24 -0700 Subject: [PATCH 01/20] Replaces a number of error equality checks with errors.Is Signed-off-by: redwrasse --- client/pkg/fileutil/lock_flock.go | 3 ++- pkg/proxy/server.go | 5 ++-- server/etcdserver/api/v3compactor/periodic.go | 3 ++- .../etcdserver/api/v3discovery/discovery.go | 4 ++-- server/etcdserver/server.go | 13 ++++++----- server/etcdserver/v3_server.go | 17 +++++++------- tests/framework/e2e/etcd_process.go | 2 +- tests/integration/clientv3/kv_test.go | 23 ++++++++++--------- .../integration/clientv3/lease/lease_test.go | 11 +++++---- tests/integration/clientv3/metrics_test.go | 3 ++- tests/integration/clientv3/util.go | 7 +++--- tests/integration/v3_failover_test.go | 3 ++- 12 files changed, 52 insertions(+), 42 deletions(-) diff --git a/client/pkg/fileutil/lock_flock.go b/client/pkg/fileutil/lock_flock.go index a4e5707a6..178c987a4 100644 --- a/client/pkg/fileutil/lock_flock.go +++ b/client/pkg/fileutil/lock_flock.go @@ -17,6 +17,7 @@ package fileutil import ( + "errors" "os" "syscall" ) @@ -28,7 +29,7 @@ func flockTryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, err } if err = syscall.Flock(int(f.Fd()), syscall.LOCK_EX|syscall.LOCK_NB); err != nil { f.Close() - if err == syscall.EWOULDBLOCK { + if errors.Is(err, syscall.EWOULDBLOCK) { err = ErrLocked } return nil, err diff --git a/pkg/proxy/server.go b/pkg/proxy/server.go index 6d7931b4e..bc71c3a16 100644 --- a/pkg/proxy/server.go +++ b/pkg/proxy/server.go @@ -16,6 +16,7 @@ package proxy import ( "context" + "errors" "fmt" "io" mrand "math/rand" @@ -427,7 +428,7 @@ func (s *server) ioCopy(dst io.Writer, src io.Reader, ptype proxyType) { for { nr1, err := src.Read(buf) if err != nil { - if err == io.EOF { + if errors.Is(err, io.EOF) { return } // connection already closed @@ -545,7 +546,7 @@ func (s *server) ioCopy(dst io.Writer, src io.Reader, ptype proxyType) { var nw int nw, err = dst.Write(data) if err != nil { - if err == io.EOF { + if errors.Is(err, io.EOF) { return } select { diff --git a/server/etcdserver/api/v3compactor/periodic.go b/server/etcdserver/api/v3compactor/periodic.go index 98fbc381b..067f69ec3 100644 --- a/server/etcdserver/api/v3compactor/periodic.go +++ b/server/etcdserver/api/v3compactor/periodic.go @@ -16,6 +16,7 @@ package v3compactor import ( "context" + "errors" "sync" "time" @@ -139,7 +140,7 @@ func (pc *Periodic) Run() { ) startTime := pc.clock.Now() _, err := pc.c.Compact(pc.ctx, &pb.CompactionRequest{Revision: rev}) - if err == nil || err == mvcc.ErrCompacted { + if err == nil || errors.Is(err, mvcc.ErrCompacted) { pc.lg.Info( "completed auto periodic compaction", zap.Int64("revision", rev), diff --git a/server/etcdserver/api/v3discovery/discovery.go b/server/etcdserver/api/v3discovery/discovery.go index 3e01b7cf6..028876cd1 100644 --- a/server/etcdserver/api/v3discovery/discovery.go +++ b/server/etcdserver/api/v3discovery/discovery.go @@ -192,7 +192,7 @@ func newDiscovery(lg *zap.Logger, dcfg *DiscoveryConfig, id types.ID) (*discover func (d *discovery) getCluster() (string, error) { cls, clusterSize, rev, err := d.checkCluster() if err != nil { - if err == ErrFullCluster { + if errors.Is(err, ErrFullCluster) { return cls.getInitClusterStr(clusterSize) } return "", err @@ -303,7 +303,7 @@ func (d *discovery) checkClusterRetry() (*clusterInfo, int, int64, error) { func (d *discovery) checkCluster() (*clusterInfo, int, int64, error) { clusterSize, err := d.getClusterSize() if err != nil { - if err == ErrSizeNotFound || err == ErrBadSizeKey { + if errors.Is(err, ErrSizeNotFound) || errors.Is(err, ErrBadSizeKey) { return nil, 0, 0, err } diff --git a/server/etcdserver/server.go b/server/etcdserver/server.go index 0600a31b8..ba3a3f3ff 100644 --- a/server/etcdserver/server.go +++ b/server/etcdserver/server.go @@ -17,6 +17,7 @@ package etcdserver import ( "context" "encoding/json" + errorspkg "errors" "expvar" "fmt" "math" @@ -1445,7 +1446,7 @@ func (s *EtcdServer) PromoteMember(ctx context.Context, id uint64) ([]*membershi learnerPromoteSucceed.Inc() return resp, nil } - if err != errors.ErrNotLeader { + if !errorspkg.Is(err, errors.ErrNotLeader) { learnerPromoteFailed.WithLabelValues(err.Error()).Inc() return resp, err } @@ -1464,13 +1465,13 @@ func (s *EtcdServer) PromoteMember(ctx context.Context, id uint64) ([]*membershi return resp, nil } // If member promotion failed, return early. Otherwise keep retry. - if err == errors.ErrLearnerNotReady || err == membership.ErrIDNotFound || err == membership.ErrMemberNotLearner { + if errorspkg.Is(err, errors.ErrLearnerNotReady) || errorspkg.Is(err, membership.ErrIDNotFound) || errorspkg.Is(err, membership.ErrMemberNotLearner) { return nil, err } } } - if cctx.Err() == context.DeadlineExceeded { + if errorspkg.Is(cctx.Err(), context.DeadlineExceeded) { return nil, errors.ErrTimeout } return nil, errors.ErrCanceled @@ -1980,7 +1981,7 @@ func (s *EtcdServer) applyEntryNormal(e *raftpb.Entry, shouldApplyV3 membership. return } - if ar.Err != errors.ErrNoSpace || len(s.alarmStore.Get(pb.AlarmType_NOSPACE)) > 0 { + if !errorspkg.Is(ar.Err, errors.ErrNoSpace) || len(s.alarmStore.Get(pb.AlarmType_NOSPACE)) > 0 { s.w.Trigger(id, ar) return } @@ -2149,7 +2150,7 @@ func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) { if err != nil { // the snapshot was done asynchronously with the progress of raft. // raft might have already got a newer snapshot. - if err == raft.ErrSnapOutOfDate { + if errorspkg.Is(err, raft.ErrSnapOutOfDate) { return } lg.Panic("failed to create snapshot", zap.Error(err)) @@ -2190,7 +2191,7 @@ func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) { if err != nil { // the compaction was done asynchronously with the progress of raft. // raft log might already been compact. - if err == raft.ErrCompacted { + if errorspkg.Is(err, raft.ErrCompacted) { return } lg.Panic("failed to compact", zap.Error(err)) diff --git a/server/etcdserver/v3_server.go b/server/etcdserver/v3_server.go index 6c5eefcb3..803f931f5 100644 --- a/server/etcdserver/v3_server.go +++ b/server/etcdserver/v3_server.go @@ -19,6 +19,7 @@ import ( "context" "encoding/base64" "encoding/binary" + errorspkg "errors" "strconv" "time" @@ -296,7 +297,7 @@ func (s *EtcdServer) LeaseRenew(ctx context.Context, id lease.LeaseID) (int64, e if err == nil { // already requested to primary lessor(leader) return ttl, nil } - if err != lease.ErrNotPrimary { + if !errorspkg.Is(err, lease.ErrNotPrimary) { return -1, err } } @@ -313,7 +314,7 @@ func (s *EtcdServer) LeaseRenew(ctx context.Context, id lease.LeaseID) (int64, e for _, url := range leader.PeerURLs { lurl := url + leasehttp.LeasePrefix ttl, err := leasehttp.RenewHTTP(cctx, id, lurl, s.peerRt) - if err == nil || err == lease.ErrLeaseNotFound { + if err == nil || errorspkg.Is(err, lease.ErrLeaseNotFound) { return ttl, err } } @@ -321,7 +322,7 @@ func (s *EtcdServer) LeaseRenew(ctx context.Context, id lease.LeaseID) (int64, e time.Sleep(50 * time.Millisecond) } - if cctx.Err() == context.DeadlineExceeded { + if errorspkg.Is(cctx.Err(), context.DeadlineExceeded) { return -1, errors.ErrTimeout } return -1, errors.ErrCanceled @@ -402,13 +403,13 @@ func (s *EtcdServer) leaseTimeToLive(ctx context.Context, r *pb.LeaseTimeToLiveR if err == nil { return resp.LeaseTimeToLiveResponse, nil } - if err == lease.ErrLeaseNotFound { + if errorspkg.Is(err, lease.ErrLeaseNotFound) { return nil, err } } } - if cctx.Err() == context.DeadlineExceeded { + if errorspkg.Is(cctx.Err(), context.DeadlineExceeded) { return nil, errors.ErrTimeout } return nil, errors.ErrCanceled @@ -527,7 +528,7 @@ func (s *EtcdServer) Authenticate(ctx context.Context, r *pb.AuthenticateRequest for { checkedRevision, err := s.AuthStore().CheckPassword(r.Name, r.Password) if err != nil { - if err != auth.ErrAuthNotEnabled { + if !errorspkg.Is(err, auth.ErrAuthNotEnabled) { lg.Warn( "invalid authentication was requested", zap.String("user", r.Name), @@ -854,7 +855,7 @@ func (s *EtcdServer) linearizableReadLoop() { } func isStopped(err error) bool { - return err == raft.ErrStopped || err == errors.ErrStopped + return errorspkg.Is(err, raft.ErrStopped) || errorspkg.Is(err, errors.ErrStopped) } func (s *EtcdServer) requestCurrentIndex(leaderChangedNotifier <-chan struct{}, requestID uint64) (uint64, error) { @@ -942,7 +943,7 @@ func (s *EtcdServer) sendReadIndex(requestIndex uint64) error { cctx, cancel := context.WithTimeout(context.Background(), s.Cfg.ReqTimeout()) err := s.r.ReadIndex(cctx, ctxToSend) cancel() - if err == raft.ErrStopped { + if errorspkg.Is(err, raft.ErrStopped) { return err } if err != nil { diff --git a/tests/framework/e2e/etcd_process.go b/tests/framework/e2e/etcd_process.go index f913efffe..b55ef9e90 100644 --- a/tests/framework/e2e/etcd_process.go +++ b/tests/framework/e2e/etcd_process.go @@ -307,7 +307,7 @@ func (ep *EtcdServerProcess) IsRunning() bool { } exitCode, err := ep.proc.ExitCode() - if err == expect.ErrProcessRunning { + if errors.Is(err, expect.ErrProcessRunning) { return true } diff --git a/tests/integration/clientv3/kv_test.go b/tests/integration/clientv3/kv_test.go index 223058a03..5f66b4f90 100644 --- a/tests/integration/clientv3/kv_test.go +++ b/tests/integration/clientv3/kv_test.go @@ -17,6 +17,7 @@ package clientv3test import ( "bytes" "context" + "errors" "fmt" "os" "reflect" @@ -50,12 +51,12 @@ func TestKVPutError(t *testing.T) { ctx := context.TODO() _, err := kv.Put(ctx, "", "bar") - if err != rpctypes.ErrEmptyKey { + if !errors.Is(err, rpctypes.ErrEmptyKey) { t.Fatalf("expected %v, got %v", rpctypes.ErrEmptyKey, err) } _, err = kv.Put(ctx, "key", strings.Repeat("a", int(maxReqBytes+100))) - if err != rpctypes.ErrRequestTooLarge { + if !errors.Is(err, rpctypes.ErrRequestTooLarge) { t.Fatalf("expected %v, got %v", rpctypes.ErrRequestTooLarge, err) } @@ -67,7 +68,7 @@ func TestKVPutError(t *testing.T) { time.Sleep(1 * time.Second) // give enough time for commit _, err = kv.Put(ctx, "foo2", strings.Repeat("a", int(maxReqBytes-50))) - if err != rpctypes.ErrNoSpace { // over quota + if !errors.Is(err, rpctypes.ErrNoSpace) { // over quota t.Fatalf("expected %v, got %v", rpctypes.ErrNoSpace, err) } } @@ -118,7 +119,7 @@ func TestKVPutWithIgnoreValue(t *testing.T) { kv := clus.RandClient() _, err := kv.Put(context.TODO(), "foo", "", clientv3.WithIgnoreValue()) - if err != rpctypes.ErrKeyNotFound { + if !errors.Is(err, rpctypes.ErrKeyNotFound) { t.Fatalf("err expected %v, got %v", rpctypes.ErrKeyNotFound, err) } @@ -157,7 +158,7 @@ func TestKVPutWithIgnoreLease(t *testing.T) { t.Errorf("failed to create lease %v", err) } - if _, err := kv.Put(context.TODO(), "zoo", "bar", clientv3.WithIgnoreLease()); err != rpctypes.ErrKeyNotFound { + if _, err := kv.Put(context.TODO(), "zoo", "bar", clientv3.WithIgnoreLease()); !errors.Is(err, rpctypes.ErrKeyNotFound) { t.Fatalf("err expected %v, got %v", rpctypes.ErrKeyNotFound, err) } @@ -199,7 +200,7 @@ func TestKVPutWithRequireLeader(t *testing.T) { kv := clus.Client(0) _, err := kv.Put(clientv3.WithRequireLeader(context.Background()), "foo", "bar") - if err != rpctypes.ErrNoLeader { + if !errors.Is(err, rpctypes.ErrNoLeader) { t.Fatal(err) } @@ -413,12 +414,12 @@ func TestKVCompactError(t *testing.T) { } _, err = kv.Compact(ctx, 6) - if err != rpctypes.ErrCompacted { + if !errors.Is(err, rpctypes.ErrCompacted) { t.Fatalf("expected %v, got %v", rpctypes.ErrCompacted, err) } _, err = kv.Compact(ctx, 100) - if err != rpctypes.ErrFutureRev { + if !errors.Is(err, rpctypes.ErrFutureRev) { t.Fatalf("expected %v, got %v", rpctypes.ErrFutureRev, err) } } @@ -443,7 +444,7 @@ func TestKVCompact(t *testing.T) { t.Fatalf("couldn't compact kv space (%v)", err) } _, err = kv.Compact(ctx, 7) - if err == nil || err != rpctypes.ErrCompacted { + if err == nil || !errors.Is(err, rpctypes.ErrCompacted) { t.Fatalf("error got %v, want %v", err, rpctypes.ErrCompacted) } @@ -472,7 +473,7 @@ func TestKVCompact(t *testing.T) { } _, err = kv.Compact(ctx, 1000) - if err == nil || err != rpctypes.ErrFutureRev { + if err == nil || !errors.Is(err, rpctypes.ErrFutureRev) { t.Fatalf("error got %v, want %v", err, rpctypes.ErrFutureRev) } } @@ -750,7 +751,7 @@ func TestKVLargeRequests(t *testing.T) { _, err := cli.Put(context.TODO(), "foo", strings.Repeat("a", test.valueSize)) if _, ok := err.(rpctypes.EtcdError); ok { - if err != test.expectError { + if !errors.Is(err, test.expectError) { t.Errorf("#%d: expected %v, got %v", i, test.expectError, err) } } else if err != nil && !strings.HasPrefix(err.Error(), test.expectError.Error()) { diff --git a/tests/integration/clientv3/lease/lease_test.go b/tests/integration/clientv3/lease/lease_test.go index f23ed6728..1d9824e81 100644 --- a/tests/integration/clientv3/lease/lease_test.go +++ b/tests/integration/clientv3/lease/lease_test.go @@ -16,6 +16,7 @@ package lease_test import ( "context" + "errors" "fmt" "reflect" "sort" @@ -38,7 +39,7 @@ func TestLeaseNotFoundError(t *testing.T) { kv := clus.RandClient() _, err := kv.Put(context.TODO(), "foo", "bar", clientv3.WithLease(clientv3.LeaseID(500))) - if err != rpctypes.ErrLeaseNotFound { + if !errors.Is(err, rpctypes.ErrLeaseNotFound) { t.Fatalf("expected %v, got %v", rpctypes.ErrLeaseNotFound, err) } } @@ -54,7 +55,7 @@ func TestLeaseGrant(t *testing.T) { kv := clus.RandClient() _, merr := lapi.Grant(context.Background(), clientv3.MaxLeaseTTL+1) - if merr != rpctypes.ErrLeaseTTLTooLarge { + if !errors.Is(merr, rpctypes.ErrLeaseTTLTooLarge) { t.Fatalf("err = %v, want %v", merr, rpctypes.ErrLeaseTTLTooLarge) } @@ -90,7 +91,7 @@ func TestLeaseRevoke(t *testing.T) { } _, err = kv.Put(context.TODO(), "foo", "bar", clientv3.WithLease(resp.ID)) - if err != rpctypes.ErrLeaseNotFound { + if !errors.Is(err, rpctypes.ErrLeaseNotFound) { t.Fatalf("err = %v, want %v", err, rpctypes.ErrLeaseNotFound) } } @@ -114,7 +115,7 @@ func TestLeaseKeepAliveOnce(t *testing.T) { } _, err = lapi.KeepAliveOnce(context.Background(), clientv3.LeaseID(0)) - if err != rpctypes.ErrLeaseNotFound { + if !errors.Is(err, rpctypes.ErrLeaseNotFound) { t.Errorf("expected %v, got %v", rpctypes.ErrLeaseNotFound, err) } } @@ -761,7 +762,7 @@ func TestV3LeaseFailureOverlap(t *testing.T) { go func() { defer wg.Done() err := updown(n) - if err == nil || err == rpctypes.ErrTimeoutDueToConnectionLost { + if err == nil || errors.Is(err, rpctypes.ErrTimeoutDueToConnectionLost) { return } t.Error(err) diff --git a/tests/integration/clientv3/metrics_test.go b/tests/integration/clientv3/metrics_test.go index da112079e..081a2fc0b 100644 --- a/tests/integration/clientv3/metrics_test.go +++ b/tests/integration/clientv3/metrics_test.go @@ -17,6 +17,7 @@ package clientv3test import ( "bufio" "context" + "errors" "io" "net" "net/http" @@ -165,7 +166,7 @@ func getHTTPBodyAsLines(t *testing.T, url string) []string { for { line, err := reader.ReadString('\n') if err != nil { - if err == io.EOF { + if errors.Is(err, io.EOF) { break } else { t.Fatalf("error reading: %v", err) diff --git a/tests/integration/clientv3/util.go b/tests/integration/clientv3/util.go index 67d1fd59c..623e575be 100644 --- a/tests/integration/clientv3/util.go +++ b/tests/integration/clientv3/util.go @@ -16,6 +16,7 @@ package clientv3test import ( "context" + "errors" "fmt" "strings" "testing" @@ -64,7 +65,7 @@ func IsClientTimeout(err error) bool { if err == nil { return false } - if err == context.DeadlineExceeded { + if errors.Is(err, context.DeadlineExceeded) { return true } ev, ok := status.FromError(err) @@ -79,7 +80,7 @@ func IsCanceled(err error) bool { if err == nil { return false } - if err == context.Canceled { + if errors.Is(err, context.Canceled) { return true } ev, ok := status.FromError(err) @@ -94,7 +95,7 @@ func IsUnavailable(err error) bool { if err == nil { return false } - if err == context.Canceled { + if errors.Is(err, context.Canceled) { return true } ev, ok := status.FromError(err) diff --git a/tests/integration/v3_failover_test.go b/tests/integration/v3_failover_test.go index cf6c73f24..d71dd267b 100644 --- a/tests/integration/v3_failover_test.go +++ b/tests/integration/v3_failover_test.go @@ -18,6 +18,7 @@ import ( "bytes" "context" "crypto/tls" + "errors" "testing" "time" @@ -167,7 +168,7 @@ func getWithRetries(t *testing.T, cli *clientv3.Client, key, val string, retryCo func shouldRetry(err error) bool { if clientv3test.IsClientTimeout(err) || clientv3test.IsServerCtxTimeout(err) || - err == rpctypes.ErrTimeout || err == rpctypes.ErrTimeoutDueToLeaderFail { + errors.Is(err, rpctypes.ErrTimeout) || errors.Is(err, rpctypes.ErrTimeoutDueToLeaderFail) { return true } return false From 46cf1e6e8c6e9763a336dc08f725c5be21b36566 Mon Sep 17 00:00:00 2001 From: qsyqian Date: Thu, 5 Sep 2024 15:13:21 +0800 Subject: [PATCH 02/20] benchmark: support insecure-skip-tls-verify Signed-off-by: qsyqian --- tools/benchmark/cmd/root.go | 1 + 1 file changed, 1 insertion(+) diff --git a/tools/benchmark/cmd/root.go b/tools/benchmark/cmd/root.go index d4481acf8..c85beb9db 100644 --- a/tools/benchmark/cmd/root.go +++ b/tools/benchmark/cmd/root.go @@ -67,6 +67,7 @@ func init() { RootCmd.PersistentFlags().StringVar(&tls.CertFile, "cert", "", "identify HTTPS client using this SSL certificate file") RootCmd.PersistentFlags().StringVar(&tls.KeyFile, "key", "", "identify HTTPS client using this SSL key file") RootCmd.PersistentFlags().StringVar(&tls.TrustedCAFile, "cacert", "", "verify certificates of HTTPS-enabled servers using this CA bundle") + RootCmd.PersistentFlags().BoolVar(&tls.InsecureSkipVerify, "insecure-skip-tls-verify", false, "skip server certificate verification") RootCmd.PersistentFlags().StringVar(&user, "user", "", "provide username[:password] and prompt if password is not supplied.") RootCmd.PersistentFlags().DurationVar(&dialTimeout, "dial-timeout", 0, "dial timeout for client connections") From cb2f7b7a638dc4f337b40e7bb322c5a006f327b0 Mon Sep 17 00:00:00 2001 From: Benjamin Wang Date: Wed, 4 Sep 2024 12:49:12 +0100 Subject: [PATCH 03/20] Support scan-keys command to scan the key space starting a specific revision Signed-off-by: Benjamin Wang --- tools/etcd-dump-db/README.md | 10 +++ tools/etcd-dump-db/main.go | 28 +++++++ tools/etcd-dump-db/meta.go | 40 ++++++++++ tools/etcd-dump-db/page.go | 68 ++++++++++++++++ tools/etcd-dump-db/scan.go | 150 +++++++++++++++++++++++++++++++++++ tools/etcd-dump-db/utils.go | 27 ++++++- 6 files changed, 322 insertions(+), 1 deletion(-) create mode 100644 tools/etcd-dump-db/meta.go create mode 100644 tools/etcd-dump-db/page.go create mode 100644 tools/etcd-dump-db/scan.go diff --git a/tools/etcd-dump-db/README.md b/tools/etcd-dump-db/README.md index 2def87ba4..b176096f0 100644 --- a/tools/etcd-dump-db/README.md +++ b/tools/etcd-dump-db/README.md @@ -102,3 +102,13 @@ key="\x00\x00\x00\x00\x005@x_\x00\x00\x00\x00\x00\x00\x00\tt", value="\n\x153640 key="\x00\x00\x00\x00\x005@x_\x00\x00\x00\x00\x00\x00\x00\bt", value="\n\x153640412599896088633_8" key="\x00\x00\x00\x00\x005@x_\x00\x00\x00\x00\x00\x00\x00\at", value="\n\x153640412599896088633_7" ``` + +#### scan-keys [data dir or db file path] + +Scans all the key-value pairs starting from a specific revision in the key space. It works even the db is corrupted. + +``` +$ ./etcd-dump-db scan-keys ~/tmp/etcd/778/db.db 16589739 2>/dev/null | grep "/registry/configmaps/istio-system/istio-namespace-controller-election" +pageID=1306, index=5/5, rev={Revision:{Main:16589739 Sub:0} tombstone:false}, value=[key "/registry/configmaps/istio-system/istio-namespace-controller-election" | val "k8s\x00\n\x0f\n\x02v1\x12\tConfigMap\x12\xeb\x03\n\xe8\x03\n#istio-namespace-controller-election\x12\x00\x1a\fistio-system\"\x00*$bb696087-260d-4167-bf06-17d3361f9b5f2\x008\x00B\b\b\x9e\xbe\xed\xb5\x06\x10\x00b\xe6\x01\n(control-plane.alpha.kubernetes.io/leader\x12\xb9\x01{\"holderIdentity\":\"istiod-d56968787-txq2d\",\"holderKey\":\"default\",\"leaseDurationSeconds\":30,\"acquireTime\":\"2024-08-13T13:26:54Z\",\"renewTime\":\"2024-08-27T06:16:13Z\",\"leaderTransitions\":0}\x8a\x01\x90\x01\n\x0fpilot-discovery\x12\x06Update\x1a\x02v1\"\b\b\xad\u07b5\xb6\x06\x10\x002\bFieldsV1:[\nY{\"f:metadata\":{\"f:annotations\":{\".\":{},\"f:control-plane.alpha.kubernetes.io/leader\":{}}}}B\x00\x1a\x00\"\x00" | created 9612546 | mod 16589739 | ver 157604] +pageID=4737, index=4/4, rev={Revision:{Main:16589786 Sub:0} tombstone:false}, value=[key "/registry/configmaps/istio-system/istio-namespace-controller-election" | val "k8s\x00\n\x0f\n\x02v1\x12\tConfigMap\x12\xeb\x03\n\xe8\x03\n#istio-namespace-controller-election\x12\x00\x1a\fistio-system\"\x00*$bb696087-260d-4167-bf06-17d3361f9b5f2\x008\x00B\b\b\x9e\xbe\xed\xb5\x06\x10\x00b\xe6\x01\n(control-plane.alpha.kubernetes.io/leader\x12\xb9\x01{\"holderIdentity\":\"istiod-d56968787-txq2d\",\"holderKey\":\"default\",\"leaseDurationSeconds\":30,\"acquireTime\":\"2024-08-13T13:26:54Z\",\"renewTime\":\"2024-08-27T06:16:21Z\",\"leaderTransitions\":0}\x8a\x01\x90\x01\n\x0fpilot-discovery\x12\x06Update\x1a\x02v1\"\b\b\xb5\u07b5\xb6\x06\x10\x002\bFieldsV1:[\nY{\"f:metadata\":{\"f:annotations\":{\".\":{},\"f:control-plane.alpha.kubernetes.io/leader\":{}}}}B\x00\x1a\x00\"\x00" | created 9612546 | mod 16589786 | ver 157605] +``` \ No newline at end of file diff --git a/tools/etcd-dump-db/main.go b/tools/etcd-dump-db/main.go index 95e6c4392..89c2f2137 100644 --- a/tools/etcd-dump-db/main.go +++ b/tools/etcd-dump-db/main.go @@ -19,6 +19,7 @@ import ( "log" "os" "path/filepath" + "strconv" "strings" "time" @@ -40,6 +41,11 @@ var ( Short: "iterate-bucket lists key-value pairs in reverse order.", Run: iterateBucketCommandFunc, } + scanKeySpaceCommand = &cobra.Command{ + Use: "scan-keys [data dir or db file path] [start revision]", + Short: "scan-keys scans all the key-value pairs starting from a specific revision in the key space.", + Run: scanKeysCommandFunc, + } getHashCommand = &cobra.Command{ Use: "hash [data dir or db file path]", Short: "hash computes the hash of db file.", @@ -58,6 +64,7 @@ func init() { rootCommand.AddCommand(listBucketCommand) rootCommand.AddCommand(iterateBucketCommand) + rootCommand.AddCommand(scanKeySpaceCommand) rootCommand.AddCommand(getHashCommand) } @@ -107,6 +114,27 @@ func iterateBucketCommandFunc(_ *cobra.Command, args []string) { } } +func scanKeysCommandFunc(_ *cobra.Command, args []string) { + if len(args) != 2 { + log.Fatalf("Must provide 2 arguments (got %v)", args) + } + dp := args[0] + if !strings.HasSuffix(dp, "db") { + dp = filepath.Join(snapDir(dp), "db") + } + if !existFileOrDir(dp) { + log.Fatalf("%q does not exist", dp) + } + startRev, err := strconv.ParseInt(args[1], 10, 64) + if err != nil { + log.Fatal(err) + } + err = scanKeys(dp, startRev) + if err != nil { + log.Fatal(err) + } +} + func getHashCommandFunc(_ *cobra.Command, args []string) { if len(args) < 1 { log.Fatalf("Must provide at least 1 argument (got %v)", args) diff --git a/tools/etcd-dump-db/meta.go b/tools/etcd-dump-db/meta.go new file mode 100644 index 000000000..30860c8af --- /dev/null +++ b/tools/etcd-dump-db/meta.go @@ -0,0 +1,40 @@ +// Copyright 2024 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import "unsafe" + +const magic uint32 = 0xED0CDAED + +type inBucket struct { + root uint64 // page id of the bucket's root-level page + sequence uint64 // monotonically incrementing, used by NextSequence() +} + +type meta struct { + magic uint32 + version uint32 + pageSize uint32 + flags uint32 + root inBucket + freelist uint64 + pgid uint64 + txid uint64 + checksum uint64 +} + +func loadPageMeta(buf []byte) *meta { + return (*meta)(unsafe.Pointer(&buf[pageHeaderSize])) +} diff --git a/tools/etcd-dump-db/page.go b/tools/etcd-dump-db/page.go new file mode 100644 index 000000000..0b6c9c67b --- /dev/null +++ b/tools/etcd-dump-db/page.go @@ -0,0 +1,68 @@ +// Copyright 2024 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import "unsafe" + +const pageHeaderSize = unsafe.Sizeof(page{}) +const leafPageElementSize = unsafe.Sizeof(leafPageElement{}) +const pageMaxAllocSize = 0xFFFFFFF + +const ( + leafPageFlag = 0x02 +) + +type page struct { + id uint64 + flags uint16 + count uint16 + overflow uint32 +} + +func (p *page) isLeafPage() bool { + return p.flags == leafPageFlag +} + +func loadPage(buf []byte) *page { + return (*page)(unsafe.Pointer(&buf[0])) +} + +// leafPageElement retrieves the leaf node by index +func (p *page) leafPageElement(index uint16) *leafPageElement { + return (*leafPageElement)(unsafeIndex(unsafe.Pointer(p), unsafe.Sizeof(*p), + leafPageElementSize, int(index))) +} + +// leafPageElement represents a node on a leaf page. +type leafPageElement struct { + flags uint32 + pos uint32 + ksize uint32 + vsize uint32 +} + +// Key returns a byte slice of the node key. +func (n *leafPageElement) key() []byte { + i := int(n.pos) + j := i + int(n.ksize) + return unsafeByteSlice(unsafe.Pointer(n), 0, i, j) +} + +// Value returns a byte slice of the node value. +func (n *leafPageElement) value() []byte { + i := int(n.pos) + int(n.ksize) + j := i + int(n.vsize) + return unsafeByteSlice(unsafe.Pointer(n), 0, i, j) +} diff --git a/tools/etcd-dump-db/scan.go b/tools/etcd-dump-db/scan.go new file mode 100644 index 000000000..3898459d2 --- /dev/null +++ b/tools/etcd-dump-db/scan.go @@ -0,0 +1,150 @@ +// Copyright 2024 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "fmt" + "io" + "os" + + "go.etcd.io/etcd/server/v3/storage/mvcc" +) + +func scanKeys(dbPath string, startRev int64) error { + pgSize, hwm, err := readPageAndHWMSize(dbPath) + if err != nil { + return fmt.Errorf("failed to read page and HWM size: %w", err) + } + + for pageID := uint64(2); pageID < hwm; { + p, _, err := readPage(dbPath, pgSize, pageID) + if err != nil { + fmt.Fprintf(os.Stderr, "Reading page %d failed: %v. Continuting...\n", pageID, err) + pageID++ + continue + } + + if !p.isLeafPage() { + pageID++ + continue + } + + for i := uint16(0); i < p.count; i++ { + e := p.leafPageElement(i) + + rev, err := bytesToBucketKey(e.key()) + if err != nil { + if exceptionCheck(e.key()) { + break + } + fmt.Fprintf(os.Stderr, "Decoding revision failed, pageID: %d, index: %d, key: %x, error: %v\n", pageID, i, string(e.key()), err) + continue + } + + if startRev != 0 && rev.Main < startRev { + continue + } + + fmt.Printf("pageID=%d, index=%d/%d, ", pageID, i, p.count-1) + keyDecoder(e.key(), e.value()) + } + + pageID += uint64(p.overflow) + 1 + } + return nil +} + +func bytesToBucketKey(key []byte) (rev mvcc.BucketKey, err error) { + defer func() { + if r := recover(); r != nil { + err = fmt.Errorf("BytesToBucketKey failed: %v", r) + } + }() + rev = mvcc.BytesToBucketKey(key) + return rev, err +} + +func readPageAndHWMSize(dbPath string) (uint64, uint64, error) { + f, err := os.Open(dbPath) + if err != nil { + return 0, 0, err + } + defer f.Close() + + // read 4KB chunk + buf := make([]byte, 4096) + if _, err := io.ReadFull(f, buf); err != nil { + return 0, 0, err + } + + m := loadPageMeta(buf) + if m.magic != magic { + return 0, 0, fmt.Errorf("the Meta Page has wrong (unexpected) magic") + } + + return uint64(m.pageSize), m.pgid, nil +} + +func readPage(dbPath string, pageSize uint64, pageID uint64) (*page, []byte, error) { + f, err := os.Open(dbPath) + if err != nil { + return nil, nil, err + } + defer f.Close() + + buf := make([]byte, pageSize) + if _, err := f.ReadAt(buf, int64(pageID*pageSize)); err != nil { + return nil, nil, err + } + + p := loadPage(buf) + if p.id != pageID { + return nil, nil, fmt.Errorf("unexpected page id: %d, wanted: %d", p.id, pageID) + } + + if p.overflow == 0 { + return p, buf, nil + } + + buf = make([]byte, (uint64(p.overflow)+1)*pageSize) + if _, err := f.ReadAt(buf, int64(pageID*pageSize)); err != nil { + return nil, nil, err + } + + p = loadPage(buf) + if p.id != pageID { + return nil, nil, fmt.Errorf("unexpected page id: %d, wanted: %d", p.id, pageID) + } + + return p, buf, nil +} + +func exceptionCheck(key []byte) bool { + whiteKeyList := map[string]struct{}{ + "alarm": {}, + "auth": {}, + "authRoles": {}, + "authUsers": {}, + "cluster": {}, + "key": {}, + "lease": {}, + "members": {}, + "members_removed": {}, + "meta": {}, + } + + _, ok := whiteKeyList[string(key)] + return ok +} diff --git a/tools/etcd-dump-db/utils.go b/tools/etcd-dump-db/utils.go index 3af585a84..184cb5181 100644 --- a/tools/etcd-dump-db/utils.go +++ b/tools/etcd-dump-db/utils.go @@ -14,9 +14,34 @@ package main -import "os" +import ( + "os" + "unsafe" +) func existFileOrDir(name string) bool { _, err := os.Stat(name) return err == nil } + +func unsafeAdd(base unsafe.Pointer, offset uintptr) unsafe.Pointer { + return unsafe.Pointer(uintptr(base) + offset) +} + +func unsafeIndex(base unsafe.Pointer, offset uintptr, elemsz uintptr, n int) unsafe.Pointer { + return unsafe.Pointer(uintptr(base) + offset + uintptr(n)*elemsz) +} + +func unsafeByteSlice(base unsafe.Pointer, offset uintptr, i, j int) []byte { + // See: https://github.com/golang/go/wiki/cgo#turning-c-arrays-into-go-slices + // + // This memory is not allocated from C, but it is unmanaged by Go's + // garbage collector and should behave similarly, and the compiler + // should produce similar code. Note that this conversion allows a + // subslice to begin after the base address, with an optional offset, + // while the URL above does not cover this case and only slices from + // index 0. However, the wiki never says that the address must be to + // the beginning of a C allocation (or even that malloc was used at + // all), so this is believed to be correct. + return (*[pageMaxAllocSize]byte)(unsafeAdd(base, offset))[i:j:j] +} From e3bade81e762d71dd3e90392480e6a376d743871 Mon Sep 17 00:00:00 2001 From: Ivan Valdes Date: Thu, 5 Sep 2024 20:57:49 -0700 Subject: [PATCH 04/20] CHANGELOG: add note regarding updating to go 1.22.7 Signed-off-by: Ivan Valdes --- CHANGELOG/CHANGELOG-3.4.md | 3 +-- CHANGELOG/CHANGELOG-3.5.md | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/CHANGELOG/CHANGELOG-3.4.md b/CHANGELOG/CHANGELOG-3.4.md index 50a302715..8c2ba898b 100644 --- a/CHANGELOG/CHANGELOG-3.4.md +++ b/CHANGELOG/CHANGELOG-3.4.md @@ -14,10 +14,9 @@ Previous change logs can be found at [CHANGELOG-3.3](https://github.com/etcd-io/ - [Print gRPC metadata in guaranteed order using the official go fmt pkg](https://github.com/etcd-io/etcd/pull/18311). ### Dependencies -- Compile binaries using go [1.21.13](https://github.com/etcd-io/etcd/pull/18422). +- Compile binaries using [go 1.22.7](https://github.com/etcd-io/etcd/pull/18549). - Upgrade [bbolt to 1.3.11](https://github.com/etcd-io/etcd/pull/18488). -
## v3.4.33 (2024-06-13) diff --git a/CHANGELOG/CHANGELOG-3.5.md b/CHANGELOG/CHANGELOG-3.5.md index 53e1c0b3e..a60cf2720 100644 --- a/CHANGELOG/CHANGELOG-3.5.md +++ b/CHANGELOG/CHANGELOG-3.5.md @@ -11,10 +11,9 @@ Previous change logs can be found at [CHANGELOG-3.4](https://github.com/etcd-io/ - [Keep the tombstone during compaction if it happens to be the compaction revision](https://github.com/etcd-io/etcd/pull/18474) ### Dependencies -- Compile binaries using [go 1.21.13](https://github.com/etcd-io/etcd/pull/18421). +- Compile binaries using [go 1.22.7](https://github.com/etcd-io/etcd/pull/18550). - Upgrade [bbolt to v1.3.11](https://github.com/etcd-io/etcd/pull/18489). -
## v3.5.15 (2024-07-19) From 7fcc3cfefeead94b70a801d31ea1c3a512cb4979 Mon Sep 17 00:00:00 2001 From: Jalin Wang Date: Mon, 9 Sep 2024 10:17:31 +0800 Subject: [PATCH 05/20] CHANGELOG: add note for the CompactionSleepInterval flag(PR 18514) Signed-off-by: Jalin Wang --- CHANGELOG/CHANGELOG-3.5.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG/CHANGELOG-3.5.md b/CHANGELOG/CHANGELOG-3.5.md index a60cf2720..c227ef8a1 100644 --- a/CHANGELOG/CHANGELOG-3.5.md +++ b/CHANGELOG/CHANGELOG-3.5.md @@ -9,6 +9,8 @@ Previous change logs can be found at [CHANGELOG-3.4](https://github.com/etcd-io/ ### etcd server - Fix [performance regression issue caused by the `ensureLeadership` in lease renew](https://github.com/etcd-io/etcd/pull/18439). - [Keep the tombstone during compaction if it happens to be the compaction revision](https://github.com/etcd-io/etcd/pull/18474) +- Add [`etcd --experimental-compaction-sleep-interval`](https://github.com/etcd-io/etcd/pull/18514) flag to control the sleep interval between each compaction batch. + ### Dependencies - Compile binaries using [go 1.22.7](https://github.com/etcd-io/etcd/pull/18550). From a042354947cb69748cc9a2c7bb6ec2a6e1c695eb Mon Sep 17 00:00:00 2001 From: Lucas Rodriguez Date: Mon, 9 Sep 2024 10:39:44 -0500 Subject: [PATCH 06/20] Fix races in TestLessorRenewExtendPileup Signed-off-by: Lucas Rodriguez --- server/lease/lessor.go | 20 +++++++++++++++----- server/lease/lessor_test.go | 18 ++++++++---------- 2 files changed, 23 insertions(+), 15 deletions(-) diff --git a/server/lease/lessor.go b/server/lease/lessor.go index fc29b9b66..cf2028933 100644 --- a/server/lease/lessor.go +++ b/server/lease/lessor.go @@ -42,8 +42,8 @@ const MaxLeaseTTL = 9000000000 var ( forever = time.Time{} - // maximum number of leases to revoke per second; configurable for tests - leaseRevokeRate = 1000 + // default number of leases to revoke per second; configurable for tests + defaultLeaseRevokeRate = 1000 // maximum number of lease checkpoints recorded to the consensus log per second; configurable for tests leaseCheckpointRate = 1000 @@ -170,6 +170,9 @@ type lessor struct { // requests for shorter TTLs are extended to the minimum TTL. minLeaseTTL int64 + // maximum number of leases to revoke per second + leaseRevokeRate int + expiredC chan []*Lease // stopC is a channel whose closure indicates that the lessor should be stopped. stopC chan struct{} @@ -198,6 +201,8 @@ type LessorConfig struct { CheckpointInterval time.Duration ExpiredLeasesRetryInterval time.Duration CheckpointPersist bool + + leaseRevokeRate int } func NewLessor(lg *zap.Logger, b backend.Backend, cluster cluster, cfg LessorConfig) Lessor { @@ -207,12 +212,16 @@ func NewLessor(lg *zap.Logger, b backend.Backend, cluster cluster, cfg LessorCon func newLessor(lg *zap.Logger, b backend.Backend, cluster cluster, cfg LessorConfig) *lessor { checkpointInterval := cfg.CheckpointInterval expiredLeaseRetryInterval := cfg.ExpiredLeasesRetryInterval + leaseRevokeRate := cfg.leaseRevokeRate if checkpointInterval == 0 { checkpointInterval = defaultLeaseCheckpointInterval } if expiredLeaseRetryInterval == 0 { expiredLeaseRetryInterval = defaultExpiredleaseRetryInterval } + if leaseRevokeRate == 0 { + leaseRevokeRate = defaultLeaseRevokeRate + } l := &lessor{ leaseMap: make(map[LeaseID]*Lease), itemMap: make(map[LeaseItem]LeaseID), @@ -220,6 +229,7 @@ func newLessor(lg *zap.Logger, b backend.Backend, cluster cluster, cfg LessorCon leaseCheckpointHeap: make(LeaseQueue, 0), b: b, minLeaseTTL: cfg.MinLeaseTTL, + leaseRevokeRate: leaseRevokeRate, checkpointInterval: checkpointInterval, expiredLeaseRetryInterval: expiredLeaseRetryInterval, checkpointPersist: cfg.CheckpointPersist, @@ -473,7 +483,7 @@ func (le *lessor) Promote(extend time.Duration) { le.scheduleCheckpointIfNeeded(l) } - if len(le.leaseMap) < leaseRevokeRate { + if len(le.leaseMap) < le.leaseRevokeRate { // no possibility of lease pile-up return } @@ -487,7 +497,7 @@ func (le *lessor) Promote(extend time.Duration) { expires := 0 // have fewer expires than the total revoke rate so piled up leases // don't consume the entire revoke limit - targetExpiresPerSecond := (3 * leaseRevokeRate) / 4 + targetExpiresPerSecond := (3 * le.leaseRevokeRate) / 4 for _, l := range leases { remaining := l.Remaining() if remaining > nextWindow { @@ -623,7 +633,7 @@ func (le *lessor) revokeExpiredLeases() { var ls []*Lease // rate limit - revokeLimit := leaseRevokeRate / 2 + revokeLimit := le.leaseRevokeRate / 2 le.mu.RLock() if le.isPrimary() { diff --git a/server/lease/lessor_test.go b/server/lease/lessor_test.go index 736909169..0d32300ab 100644 --- a/server/lease/lessor_test.go +++ b/server/lease/lessor_test.go @@ -307,17 +307,15 @@ func TestLessorRenewWithCheckpointer(t *testing.T) { // TestLessorRenewExtendPileup ensures Lessor extends leases on promotion if too many // expire at the same time. func TestLessorRenewExtendPileup(t *testing.T) { - oldRevokeRate := leaseRevokeRate - defer func() { leaseRevokeRate = oldRevokeRate }() + leaseRevokeRate := 10 lg := zap.NewNop() - leaseRevokeRate = 10 dir, be := NewTestBackend(t) defer os.RemoveAll(dir) - le := newLessor(lg, be, clusterLatest(), LessorConfig{MinLeaseTTL: minLeaseTTL}) + le := newLessor(lg, be, clusterLatest(), LessorConfig{MinLeaseTTL: minLeaseTTL, leaseRevokeRate: leaseRevokeRate}) ttl := int64(10) - for i := 1; i <= leaseRevokeRate*10; i++ { + for i := 1; i <= le.leaseRevokeRate*10; i++ { if _, err := le.Grant(LeaseID(2*i), ttl); err != nil { t.Fatal(err) } @@ -334,7 +332,7 @@ func TestLessorRenewExtendPileup(t *testing.T) { bcfg.Path = filepath.Join(dir, "be") be = backend.New(bcfg) defer be.Close() - le = newLessor(lg, be, clusterLatest(), LessorConfig{MinLeaseTTL: minLeaseTTL}) + le = newLessor(lg, be, clusterLatest(), LessorConfig{MinLeaseTTL: minLeaseTTL, leaseRevokeRate: leaseRevokeRate}) defer le.Stop() // extend after recovery should extend expiration on lease pile-up @@ -349,11 +347,11 @@ func TestLessorRenewExtendPileup(t *testing.T) { for i := ttl; i < ttl+20; i++ { c := windowCounts[i] - if c > leaseRevokeRate { - t.Errorf("expected at most %d expiring at %ds, got %d", leaseRevokeRate, i, c) + if c > le.leaseRevokeRate { + t.Errorf("expected at most %d expiring at %ds, got %d", le.leaseRevokeRate, i, c) } - if c < leaseRevokeRate/2 { - t.Errorf("expected at least %d expiring at %ds, got %d", leaseRevokeRate/2, i, c) + if c < le.leaseRevokeRate/2 { + t.Errorf("expected at least %d expiring at %ds, got %d", le.leaseRevokeRate/2, i, c) } } } From eb100511a636588855fb1b1cdd455062778b98a9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Sep 2024 17:37:08 +0000 Subject: [PATCH 07/20] build(deps): bump distroless/static-debian12 from `ce46866` to `95eb83a` Bumps distroless/static-debian12 from `ce46866` to `95eb83a`. --- updated-dependencies: - dependency-name: distroless/static-debian12 dependency-type: direct:production ... Signed-off-by: dependabot[bot] --- Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Dockerfile b/Dockerfile index 14a489465..363c0cfe2 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,5 +1,5 @@ ARG ARCH=amd64 -FROM --platform=linux/${ARCH} gcr.io/distroless/static-debian12@sha256:ce46866b3a5170db3b49364900fb3168dc0833dfb46c26da5c77f22abb01d8c3 +FROM --platform=linux/${ARCH} gcr.io/distroless/static-debian12@sha256:95eb83a44a62c1c27e5f0b38d26085c486d71ece83dd64540b7209536bb13f6d ADD etcd /usr/local/bin/ ADD etcdctl /usr/local/bin/ From 00f8a47d223ebb8246dcce05c50ee7fe5d4230fe Mon Sep 17 00:00:00 2001 From: ArkaSaha30 Date: Wed, 11 Sep 2024 00:37:41 +0530 Subject: [PATCH 08/20] Improve release documentation This commit will improve the release documentation guide by incorporating the following points: 1. Step 5 can be simplified to do a single-branch checkout 2. Step 6 should clarify that the release directory is in /tmp/etcd-release-${VERSION}/etcd/release/ 3. Step 8 mention to add the "Release" label to the announcement message Signed-off-by: ArkaSaha30 --- Documentation/contributor-guide/release.md | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/Documentation/contributor-guide/release.md b/Documentation/contributor-guide/release.md index ca7dae3cc..5f045540e 100644 --- a/Documentation/contributor-guide/release.md +++ b/Documentation/contributor-guide/release.md @@ -72,12 +72,11 @@ which don't need to be executed before releasing each version. - `docker login gcr.io` - `docker login quay.io` 5. Clone the etcd repository and checkout the target branch, - - `git clone git@github.com:etcd-io/etcd.git` - - `git checkout release-3.X` + - `git clone --branch release-3.X git@github.com:etcd-io/etcd.git` 6. Run the release script under the repository's root directory, replacing `${VERSION}` with a value without the `v` prefix, i.e. `3.5.13`. - `DRY_RUN=false ./scripts/release.sh ${VERSION}` - It generates all release binaries under the directory `./release` and images. Binaries are pushed to the Google Cloud bucket + It generates all release binaries under the directory `/tmp/etcd-release-${VERSION}/etcd/release/` and images. Binaries are pushed to the Google Cloud bucket under project `etcd-development`, and images are pushed to `quay.io` and `gcr.io`. 7. Publish the release page on GitHub - Set the release title as the version name @@ -88,7 +87,7 @@ which don't need to be executed before releasing each version. - Publish the release 8. Announce to the etcd-dev googlegroup - Follow the format of previous release emails sent to etcd-dev@googlegroups.com, see an example below. After sending out the email, ask one of the mailing list maintainers to approve the email from the pending list. + Follow the format of previous release emails sent to etcd-dev@googlegroups.com, see an example below. After sending out the email, ask one of the mailing list maintainers to approve the email from the pending list. Additionally, label the release email as `Release`. ```text Hello, From 96e17d38e244ef18756a55d33066f2893402794b Mon Sep 17 00:00:00 2001 From: Ivan Valdes Date: Tue, 10 Sep 2024 13:43:19 -0700 Subject: [PATCH 09/20] Update release date for v3.5.16 Signed-off-by: Ivan Valdes --- CHANGELOG/CHANGELOG-3.5.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/CHANGELOG/CHANGELOG-3.5.md b/CHANGELOG/CHANGELOG-3.5.md index c227ef8a1..198245881 100644 --- a/CHANGELOG/CHANGELOG-3.5.md +++ b/CHANGELOG/CHANGELOG-3.5.md @@ -1,17 +1,19 @@ Previous change logs can be found at [CHANGELOG-3.4](https://github.com/etcd-io/etcd/blob/main/CHANGELOG/CHANGELOG-3.4.md). +
+ +## v3.5.17 (TBC)
-## v3.5.16 (TBC) +## v3.5.16 (2024-09-10) ### etcd server - Fix [performance regression issue caused by the `ensureLeadership` in lease renew](https://github.com/etcd-io/etcd/pull/18439). - [Keep the tombstone during compaction if it happens to be the compaction revision](https://github.com/etcd-io/etcd/pull/18474) - Add [`etcd --experimental-compaction-sleep-interval`](https://github.com/etcd-io/etcd/pull/18514) flag to control the sleep interval between each compaction batch. - ### Dependencies - Compile binaries using [go 1.22.7](https://github.com/etcd-io/etcd/pull/18550). - Upgrade [bbolt to v1.3.11](https://github.com/etcd-io/etcd/pull/18489). From 29f3bc2222b8f43172cb1dd8eaf65e59a9c029a8 Mon Sep 17 00:00:00 2001 From: Chun-Hung Tseng Date: Wed, 14 Aug 2024 23:48:01 +0200 Subject: [PATCH 10/20] Bump go toolchain to 1.22.7 Reference: - https://github.com/etcd-io/etcd/issues/18443 - https://github.com/etcd-io/etcd/issues/18548#issuecomment-2343061825 Signed-off-by: Chun-Hung Tseng --- .go-version | 2 +- api/go.mod | 2 +- client/internal/v2/go.mod | 2 +- client/pkg/go.mod | 2 +- client/v3/go.mod | 2 +- etcdctl/go.mod | 2 +- etcdutl/go.mod | 2 +- go.mod | 2 +- pkg/go.mod | 2 +- server/go.mod | 2 +- tests/go.mod | 2 +- tools/mod/go.mod | 2 +- tools/rw-heatmaps/go.mod | 2 +- tools/testgrid-analysis/go.mod | 2 +- 14 files changed, 14 insertions(+), 14 deletions(-) diff --git a/.go-version b/.go-version index 013173af5..87b26e8b1 100644 --- a/.go-version +++ b/.go-version @@ -1 +1 @@ -1.22.6 +1.22.7 diff --git a/api/go.mod b/api/go.mod index a84525d32..e88bdc8cf 100644 --- a/api/go.mod +++ b/api/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/api/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( github.com/coreos/go-semver v0.3.1 diff --git a/client/internal/v2/go.mod b/client/internal/v2/go.mod index 5a0c0029c..ce83c178a 100644 --- a/client/internal/v2/go.mod +++ b/client/internal/v2/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/client/v2 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( go.etcd.io/etcd/api/v3 v3.6.0-alpha.0 diff --git a/client/pkg/go.mod b/client/pkg/go.mod index f8345c3d6..aff71bf34 100644 --- a/client/pkg/go.mod +++ b/client/pkg/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/client/pkg/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( github.com/coreos/go-systemd/v22 v22.5.0 diff --git a/client/v3/go.mod b/client/v3/go.mod index 673f189ef..1fc510bf2 100644 --- a/client/v3/go.mod +++ b/client/v3/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/client/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( github.com/coreos/go-semver v0.3.1 diff --git a/etcdctl/go.mod b/etcdctl/go.mod index 18d89980c..0a0359079 100644 --- a/etcdctl/go.mod +++ b/etcdctl/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/etcdctl/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( github.com/bgentry/speakeasy v0.2.0 diff --git a/etcdutl/go.mod b/etcdutl/go.mod index 237757b11..6d715d937 100644 --- a/etcdutl/go.mod +++ b/etcdutl/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/etcdutl/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 replace ( go.etcd.io/etcd/api/v3 => ../api diff --git a/go.mod b/go.mod index 5117ed00b..615af9aff 100644 --- a/go.mod +++ b/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 replace ( go.etcd.io/etcd/api/v3 => ./api diff --git a/pkg/go.mod b/pkg/go.mod index 4e9d252cc..3f818c8fc 100644 --- a/pkg/go.mod +++ b/pkg/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/pkg/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( github.com/creack/pty v1.1.18 diff --git a/server/go.mod b/server/go.mod index 9f2129937..a5cd34093 100644 --- a/server/go.mod +++ b/server/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/server/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( github.com/coreos/go-semver v0.3.1 diff --git a/tests/go.mod b/tests/go.mod index 3612dba1d..0f028beec 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/tests/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 replace ( go.etcd.io/etcd/api/v3 => ../api diff --git a/tools/mod/go.mod b/tools/mod/go.mod index 1d7190c6f..aa43a5379 100644 --- a/tools/mod/go.mod +++ b/tools/mod/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/tools/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( github.com/alexfalkowski/gocovmerge v1.3.13 diff --git a/tools/rw-heatmaps/go.mod b/tools/rw-heatmaps/go.mod index 1c0a6f44f..ae6e0e9d8 100644 --- a/tools/rw-heatmaps/go.mod +++ b/tools/rw-heatmaps/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/tools/rw-heatmaps/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( github.com/spf13/cobra v1.8.1 diff --git a/tools/testgrid-analysis/go.mod b/tools/testgrid-analysis/go.mod index 6875e575e..489e9e87d 100644 --- a/tools/testgrid-analysis/go.mod +++ b/tools/testgrid-analysis/go.mod @@ -2,7 +2,7 @@ module go.etcd.io/etcd/tools/testgrid-analysis/v3 go 1.22 -toolchain go1.22.6 +toolchain go1.22.7 require ( github.com/GoogleCloudPlatform/testgrid v0.0.173 From 94e5502112a0ed3bb73cc4c48958c39cae9cf1ac Mon Sep 17 00:00:00 2001 From: Ivan Valdes Date: Wed, 11 Sep 2024 13:27:10 -0700 Subject: [PATCH 11/20] CHANGELOG: set release 3.4.34 date Signed-off-by: Ivan Valdes --- CHANGELOG/CHANGELOG-3.4.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/CHANGELOG/CHANGELOG-3.4.md b/CHANGELOG/CHANGELOG-3.4.md index 8c2ba898b..87dea8e7a 100644 --- a/CHANGELOG/CHANGELOG-3.4.md +++ b/CHANGELOG/CHANGELOG-3.4.md @@ -4,7 +4,11 @@ Previous change logs can be found at [CHANGELOG-3.3](https://github.com/etcd-io/
-## v3.4.34 (TBD) +## v3.4.35 (TBC) + +
+ +## v3.4.34 (2024-09-11) ### etcd server - Fix [performance regression issue caused by the `ensureLeadership` in lease renew](https://github.com/etcd-io/etcd/pull/18440). From 636f191368973b57713da8a13b8ce673de007d83 Mon Sep 17 00:00:00 2001 From: Chun-Hung Tseng Date: Wed, 11 Sep 2024 15:43:48 +0200 Subject: [PATCH 12/20] dependency: bump github.com/prometheus/common from 0.57.0 to 0.59.1 Reference: - https://github.com/etcd-io/etcd/pull/18562 - https://github.com/etcd-io/etcd/pull/18567 Signed-off-by: Chun-Hung Tseng --- client/v3/go.mod | 2 +- client/v3/go.sum | 4 ++-- etcdctl/go.sum | 4 ++-- etcdutl/go.mod | 2 +- etcdutl/go.sum | 4 ++-- go.mod | 2 +- go.sum | 4 ++-- server/go.mod | 2 +- server/go.sum | 4 ++-- tests/go.mod | 2 +- tests/go.sum | 4 ++-- tools/mod/go.mod | 2 +- tools/mod/go.sum | 2 ++ 13 files changed, 20 insertions(+), 18 deletions(-) diff --git a/client/v3/go.mod b/client/v3/go.mod index 1fc510bf2..18855cc9e 100644 --- a/client/v3/go.mod +++ b/client/v3/go.mod @@ -29,7 +29,7 @@ require ( github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.6.1 // indirect - github.com/prometheus/common v0.57.0 // indirect + github.com/prometheus/common v0.59.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/net v0.28.0 // indirect diff --git a/client/v3/go.sum b/client/v3/go.sum index 54837af81..fa5cab36a 100644 --- a/client/v3/go.sum +++ b/client/v3/go.sum @@ -40,8 +40,8 @@ github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjs github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= -github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= +github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= +github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= diff --git a/etcdctl/go.sum b/etcdctl/go.sum index b34421159..59e601736 100644 --- a/etcdctl/go.sum +++ b/etcdctl/go.sum @@ -54,8 +54,8 @@ github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjs github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= -github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= +github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= +github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= diff --git a/etcdutl/go.mod b/etcdutl/go.mod index 6d715d937..cb8b77d82 100644 --- a/etcdutl/go.mod +++ b/etcdutl/go.mod @@ -62,7 +62,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_golang v1.20.2 // indirect github.com/prometheus/client_model v0.6.1 // indirect - github.com/prometheus/common v0.57.0 // indirect + github.com/prometheus/common v0.59.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/rivo/uniseg v0.4.7 // indirect github.com/sirupsen/logrus v1.9.3 // indirect diff --git a/etcdutl/go.sum b/etcdutl/go.sum index 0fe311b68..95fd63033 100644 --- a/etcdutl/go.sum +++ b/etcdutl/go.sum @@ -68,8 +68,8 @@ github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjs github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= -github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= +github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= +github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= diff --git a/go.mod b/go.mod index 615af9aff..b5e07f710 100644 --- a/go.mod +++ b/go.mod @@ -72,7 +72,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_golang v1.20.2 // indirect github.com/prometheus/client_model v0.6.1 // indirect - github.com/prometheus/common v0.57.0 // indirect + github.com/prometheus/common v0.59.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/rivo/uniseg v0.4.7 // indirect github.com/sirupsen/logrus v1.9.3 // indirect diff --git a/go.sum b/go.sum index baae5e0eb..8bcf88bc5 100644 --- a/go.sum +++ b/go.sum @@ -108,8 +108,8 @@ github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/j github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= -github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= +github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= +github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= diff --git a/server/go.mod b/server/go.mod index a5cd34093..5dcc9ea14 100644 --- a/server/go.mod +++ b/server/go.mod @@ -61,7 +61,7 @@ require ( github.com/kylelemons/godebug v1.1.0 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/common v0.57.0 // indirect + github.com/prometheus/common v0.59.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/sirupsen/logrus v1.9.3 // indirect github.com/spf13/pflag v1.0.5 // indirect diff --git a/server/go.sum b/server/go.sum index 1c3443b83..0a28dd0ba 100644 --- a/server/go.sum +++ b/server/go.sum @@ -90,8 +90,8 @@ github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/j github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= -github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= +github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= +github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= diff --git a/tests/go.mod b/tests/go.mod index 0f028beec..1ce00042e 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -24,7 +24,7 @@ require ( github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 github.com/prometheus/client_golang v1.20.2 - github.com/prometheus/common v0.57.0 + github.com/prometheus/common v0.59.1 github.com/soheilhy/cmux v0.1.5 github.com/stretchr/testify v1.9.0 go.etcd.io/etcd/api/v3 v3.6.0-alpha.0 diff --git a/tests/go.sum b/tests/go.sum index 8bd5ab441..12b5d6412 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -112,8 +112,8 @@ github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/j github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= -github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= +github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= +github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= diff --git a/tools/mod/go.mod b/tools/mod/go.mod index aa43a5379..19181dccf 100644 --- a/tools/mod/go.mod +++ b/tools/mod/go.mod @@ -153,7 +153,7 @@ require ( github.com/polyfloyd/go-errorlint v1.6.0 // indirect github.com/prometheus/client_golang v1.20.2 // indirect github.com/prometheus/client_model v0.6.1 // indirect - github.com/prometheus/common v0.57.0 // indirect + github.com/prometheus/common v0.59.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/quasilyte/go-ruleguard v0.4.2 // indirect github.com/quasilyte/go-ruleguard/dsl v0.3.22 // indirect diff --git a/tools/mod/go.sum b/tools/mod/go.sum index 65ee3028b..08eb576f9 100644 --- a/tools/mod/go.sum +++ b/tools/mod/go.sum @@ -354,6 +354,8 @@ github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8= github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= +github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= +github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/quasilyte/go-ruleguard v0.4.2 h1:htXcXDK6/rO12kiTHKfHuqR4kr3Y4M0J0rOL6CH/BYs= From a976ba136d122a7b87bd01ad44eb8caa21e05250 Mon Sep 17 00:00:00 2001 From: Chun-Hung Tseng Date: Wed, 11 Sep 2024 15:46:52 +0200 Subject: [PATCH 13/20] dependency: bump github.com/prometheus/client_golang from 1.20.2 to 1.20.3 Reference: - https://github.com/etcd-io/etcd/pull/18564 - https://github.com/etcd-io/etcd/pull/18559 Signed-off-by: Chun-Hung Tseng --- client/v3/go.mod | 2 +- client/v3/go.sum | 4 ++-- etcdctl/go.sum | 4 ++-- etcdutl/go.mod | 2 +- etcdutl/go.sum | 4 ++-- go.mod | 2 +- go.sum | 4 ++-- server/go.mod | 2 +- server/go.sum | 4 ++-- tests/go.mod | 2 +- tests/go.sum | 4 ++-- tools/mod/go.mod | 2 +- tools/mod/go.sum | 2 ++ 13 files changed, 20 insertions(+), 18 deletions(-) diff --git a/client/v3/go.mod b/client/v3/go.mod index 18855cc9e..94cc16d92 100644 --- a/client/v3/go.mod +++ b/client/v3/go.mod @@ -8,7 +8,7 @@ require ( github.com/coreos/go-semver v0.3.1 github.com/dustin/go-humanize v1.0.1 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 - github.com/prometheus/client_golang v1.20.2 + github.com/prometheus/client_golang v1.20.3 github.com/stretchr/testify v1.9.0 go.etcd.io/etcd/api/v3 v3.6.0-alpha.0 go.etcd.io/etcd/client/pkg/v3 v3.6.0-alpha.0 diff --git a/client/v3/go.sum b/client/v3/go.sum index fa5cab36a..22997970c 100644 --- a/client/v3/go.sum +++ b/client/v3/go.sum @@ -36,8 +36,8 @@ github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= 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/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= -github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= +github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= diff --git a/etcdctl/go.sum b/etcdctl/go.sum index 59e601736..61e408839 100644 --- a/etcdctl/go.sum +++ b/etcdctl/go.sum @@ -50,8 +50,8 @@ github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= 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/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= -github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= +github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= diff --git a/etcdutl/go.mod b/etcdutl/go.mod index cb8b77d82..909da3f5a 100644 --- a/etcdutl/go.mod +++ b/etcdutl/go.mod @@ -60,7 +60,7 @@ require ( github.com/mattn/go-runewidth v0.0.15 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_golang v1.20.2 // indirect + github.com/prometheus/client_golang v1.20.3 // indirect github.com/prometheus/client_model v0.6.1 // indirect github.com/prometheus/common v0.59.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect diff --git a/etcdutl/go.sum b/etcdutl/go.sum index 95fd63033..927c93fd2 100644 --- a/etcdutl/go.sum +++ b/etcdutl/go.sum @@ -64,8 +64,8 @@ github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= 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/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= -github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= +github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= diff --git a/go.mod b/go.mod index b5e07f710..adae6d22e 100644 --- a/go.mod +++ b/go.mod @@ -70,7 +70,7 @@ require ( github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/olekukonko/tablewriter v0.0.5 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_golang v1.20.2 // indirect + github.com/prometheus/client_golang v1.20.3 // indirect github.com/prometheus/client_model v0.6.1 // indirect github.com/prometheus/common v0.59.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect diff --git a/go.sum b/go.sum index 8bcf88bc5..9be14c170 100644 --- a/go.sum +++ b/go.sum @@ -103,8 +103,8 @@ github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFSt github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 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/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= -github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= +github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= diff --git a/server/go.mod b/server/go.mod index 5dcc9ea14..115239e4c 100644 --- a/server/go.mod +++ b/server/go.mod @@ -18,7 +18,7 @@ require ( github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 github.com/jonboulle/clockwork v0.4.0 - github.com/prometheus/client_golang v1.20.2 + github.com/prometheus/client_golang v1.20.3 github.com/prometheus/client_model v0.6.1 github.com/soheilhy/cmux v0.1.5 github.com/spf13/cobra v1.8.1 diff --git a/server/go.sum b/server/go.sum index 0a28dd0ba..a938130d4 100644 --- a/server/go.sum +++ b/server/go.sum @@ -85,8 +85,8 @@ github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFSt github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 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/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= -github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= +github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= diff --git a/tests/go.mod b/tests/go.mod index 1ce00042e..df89d9b5b 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -23,7 +23,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 - github.com/prometheus/client_golang v1.20.2 + github.com/prometheus/client_golang v1.20.3 github.com/prometheus/common v0.59.1 github.com/soheilhy/cmux v0.1.5 github.com/stretchr/testify v1.9.0 diff --git a/tests/go.sum b/tests/go.sum index 12b5d6412..7f9a84e77 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -107,8 +107,8 @@ github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFSt github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= 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/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= -github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= +github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= diff --git a/tools/mod/go.mod b/tools/mod/go.mod index 19181dccf..a7a70483b 100644 --- a/tools/mod/go.mod +++ b/tools/mod/go.mod @@ -151,7 +151,7 @@ require ( github.com/pelletier/go-toml/v2 v2.2.2 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/polyfloyd/go-errorlint v1.6.0 // indirect - github.com/prometheus/client_golang v1.20.2 // indirect + github.com/prometheus/client_golang v1.20.3 // indirect github.com/prometheus/client_model v0.6.1 // indirect github.com/prometheus/common v0.59.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect diff --git a/tools/mod/go.sum b/tools/mod/go.sum index 08eb576f9..b52c0fe8c 100644 --- a/tools/mod/go.sum +++ b/tools/mod/go.sum @@ -348,6 +348,8 @@ github.com/prometheus/client_golang v1.20.0 h1:jBzTZ7B099Rg24tny+qngoynol8LtVYlA github.com/prometheus/client_golang v1.20.0/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= +github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G1dc= From 14d54893e04421fcccf690e6470516b1cb673f7e Mon Sep 17 00:00:00 2001 From: Chun-Hung Tseng Date: Wed, 11 Sep 2024 15:50:29 +0200 Subject: [PATCH 14/20] dependency: bump golang.org/x/crypto from 0.26.0 to 0.27.0 golang.org/x/sys and golang.org/x/text version inconsistency are also addressed: - bump golang.org/x/sys from 0.24.0 to 0.25.0 - bump golang.org/x/text from 0.17.0 to 0.18.0 Reference: - https://github.com/etcd-io/etcd/pull/18563 - https://github.com/etcd-io/etcd/pull/18558 - https://github.com/etcd-io/etcd/pull/18566 - https://github.com/etcd-io/etcd/pull/18565 Signed-off-by: Chun-Hung Tseng --- api/go.mod | 4 ++-- api/go.sum | 8 ++++---- client/pkg/go.mod | 2 +- client/pkg/go.sum | 4 ++-- client/v3/go.mod | 4 ++-- client/v3/go.sum | 8 ++++---- etcdctl/go.mod | 4 ++-- etcdctl/go.sum | 8 ++++---- etcdutl/go.mod | 6 +++--- etcdutl/go.sum | 12 ++++++------ go.mod | 6 +++--- go.sum | 12 ++++++------ pkg/go.mod | 4 ++-- pkg/go.sum | 8 ++++---- server/go.mod | 6 +++--- server/go.sum | 12 ++++++------ tests/go.mod | 6 +++--- tests/go.sum | 12 ++++++------ tools/mod/go.mod | 10 +++++----- tools/mod/go.sum | 30 ++++++++---------------------- tools/rw-heatmaps/go.mod | 2 +- tools/rw-heatmaps/go.sum | 4 ++-- tools/testgrid-analysis/go.mod | 4 ++-- tools/testgrid-analysis/go.sum | 8 ++++---- 24 files changed, 85 insertions(+), 99 deletions(-) diff --git a/api/go.mod b/api/go.mod index e88bdc8cf..85317d8db 100644 --- a/api/go.mod +++ b/api/go.mod @@ -21,8 +21,8 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/rogpeppe/go-internal v1.12.0 // indirect golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/api/go.sum b/api/go.sum index d77b105d6..f8b2115ab 100644 --- a/api/go.sum +++ b/api/go.sum @@ -42,12 +42,12 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= 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.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= 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.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= diff --git a/client/pkg/go.mod b/client/pkg/go.mod index aff71bf34..305abde17 100644 --- a/client/pkg/go.mod +++ b/client/pkg/go.mod @@ -8,7 +8,7 @@ require ( github.com/coreos/go-systemd/v22 v22.5.0 github.com/stretchr/testify v1.9.0 go.uber.org/zap v1.27.0 - golang.org/x/sys v0.24.0 + golang.org/x/sys v0.25.0 ) require ( diff --git a/client/pkg/go.sum b/client/pkg/go.sum index 86956db45..d1f7434c3 100644 --- a/client/pkg/go.sum +++ b/client/pkg/go.sum @@ -25,8 +25,8 @@ go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= diff --git a/client/v3/go.mod b/client/v3/go.mod index 94cc16d92..6cbf043c5 100644 --- a/client/v3/go.mod +++ b/client/v3/go.mod @@ -33,8 +33,8 @@ require ( github.com/prometheus/procfs v0.15.1 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/protobuf v1.34.2 // indirect diff --git a/client/v3/go.sum b/client/v3/go.sum index 22997970c..ea2d7c6f5 100644 --- a/client/v3/go.sum +++ b/client/v3/go.sum @@ -73,12 +73,12 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= 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.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= 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.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= diff --git a/etcdctl/go.mod b/etcdctl/go.mod index 0a0359079..f6be65c6a 100644 --- a/etcdctl/go.mod +++ b/etcdctl/go.mod @@ -35,8 +35,8 @@ require ( github.com/rivo/uniseg v0.4.7 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/protobuf v1.34.2 // indirect diff --git a/etcdctl/go.sum b/etcdctl/go.sum index 61e408839..489394413 100644 --- a/etcdctl/go.sum +++ b/etcdctl/go.sum @@ -95,12 +95,12 @@ golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= 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.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.6.0 h1:eTDhh4ZXt5Qf0augr54TN6suAUudPcawVZeIAPU7D4U= golang.org/x/time v0.6.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/etcdutl/go.mod b/etcdutl/go.mod index 909da3f5a..014d43613 100644 --- a/etcdutl/go.mod +++ b/etcdutl/go.mod @@ -80,10 +80,10 @@ require ( go.opentelemetry.io/otel/trace v1.29.0 // indirect go.opentelemetry.io/proto/otlp v1.3.1 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/crypto v0.26.0 // indirect + golang.org/x/crypto v0.27.0 // indirect golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect golang.org/x/time v0.6.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect diff --git a/etcdutl/go.sum b/etcdutl/go.sum index 927c93fd2..dea3a2e22 100644 --- a/etcdutl/go.sum +++ b/etcdutl/go.sum @@ -125,8 +125,8 @@ go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= -golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= +golang.org/x/crypto v0.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= +golang.org/x/crypto v0.27.0/go.mod h1:1Xngt8kV6Dvbssa53Ziq6Eqn0HqbZi5Z6R0ZpwQzt70= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -145,12 +145,12 @@ golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= 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.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.6.0 h1:eTDhh4ZXt5Qf0augr54TN6suAUudPcawVZeIAPU7D4U= golang.org/x/time v0.6.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/go.mod b/go.mod index adae6d22e..c8ebd008e 100644 --- a/go.mod +++ b/go.mod @@ -90,10 +90,10 @@ require ( go.opentelemetry.io/otel/trace v1.29.0 // indirect go.opentelemetry.io/proto/otlp v1.3.1 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/crypto v0.26.0 // indirect + golang.org/x/crypto v0.27.0 // indirect golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect diff --git a/go.sum b/go.sum index 9be14c170..6c61f14c5 100644 --- a/go.sum +++ b/go.sum @@ -174,8 +174,8 @@ go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= -golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= +golang.org/x/crypto v0.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= +golang.org/x/crypto v0.27.0/go.mod h1:1Xngt8kV6Dvbssa53Ziq6Eqn0HqbZi5Z6R0ZpwQzt70= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -209,12 +209,12 @@ golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= 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.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.6.0 h1:eTDhh4ZXt5Qf0augr54TN6suAUudPcawVZeIAPU7D4U= golang.org/x/time v0.6.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/pkg/go.mod b/pkg/go.mod index 3f818c8fc..f265d510e 100644 --- a/pkg/go.mod +++ b/pkg/go.mod @@ -22,8 +22,8 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/pkg/go.sum b/pkg/go.sum index 6c87c013a..765ab943b 100644 --- a/pkg/go.sum +++ b/pkg/go.sum @@ -35,10 +35,10 @@ go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= diff --git a/server/go.mod b/server/go.mod index 115239e4c..8e3dea92f 100644 --- a/server/go.mod +++ b/server/go.mod @@ -37,7 +37,7 @@ require ( go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.29.0 go.opentelemetry.io/otel/sdk v1.29.0 go.uber.org/zap v1.27.0 - golang.org/x/crypto v0.26.0 + golang.org/x/crypto v0.27.0 golang.org/x/net v0.28.0 golang.org/x/time v0.6.0 google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd @@ -70,8 +70,8 @@ require ( go.opentelemetry.io/otel/trace v1.29.0 // indirect go.opentelemetry.io/proto/otlp v1.3.1 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect gopkg.in/yaml.v3 v3.0.1 // indirect sigs.k8s.io/json v0.0.0-20211020170558-c049b76a60c6 // indirect diff --git a/server/go.sum b/server/go.sum index a938130d4..ddd0a084c 100644 --- a/server/go.sum +++ b/server/go.sum @@ -151,8 +151,8 @@ go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= -golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= +golang.org/x/crypto v0.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= +golang.org/x/crypto v0.27.0/go.mod h1:1Xngt8kV6Dvbssa53Ziq6Eqn0HqbZi5Z6R0ZpwQzt70= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -184,12 +184,12 @@ golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= 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.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.6.0 h1:eTDhh4ZXt5Qf0augr54TN6suAUudPcawVZeIAPU7D4U= golang.org/x/time v0.6.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/tests/go.mod b/tests/go.mod index df89d9b5b..508d9fa38 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -43,7 +43,7 @@ require ( go.opentelemetry.io/otel/trace v1.29.0 go.opentelemetry.io/proto/otlp v1.3.1 go.uber.org/zap v1.27.0 - golang.org/x/crypto v0.26.0 + golang.org/x/crypto v0.27.0 golang.org/x/sync v0.8.0 golang.org/x/time v0.6.0 google.golang.org/grpc v1.66.0 @@ -93,8 +93,8 @@ require ( go.opentelemetry.io/otel/metric v1.29.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect diff --git a/tests/go.sum b/tests/go.sum index 7f9a84e77..aa2955fda 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -178,8 +178,8 @@ go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= -golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= +golang.org/x/crypto v0.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= +golang.org/x/crypto v0.27.0/go.mod h1:1Xngt8kV6Dvbssa53Ziq6Eqn0HqbZi5Z6R0ZpwQzt70= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -213,12 +213,12 @@ golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= 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.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.6.0 h1:eTDhh4ZXt5Qf0augr54TN6suAUudPcawVZeIAPU7D4U= golang.org/x/time v0.6.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/tools/mod/go.mod b/tools/mod/go.mod index a7a70483b..951b5fc94 100644 --- a/tools/mod/go.mod +++ b/tools/mod/go.mod @@ -1,6 +1,6 @@ module go.etcd.io/etcd/tools/v3 -go 1.22 +go 1.22.1 toolchain go1.22.7 @@ -209,15 +209,15 @@ require ( go.uber.org/automaxprocs v1.5.3 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect - golang.org/x/crypto v0.26.0 // indirect + golang.org/x/crypto v0.27.0 // indirect golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56 // indirect golang.org/x/exp/typeparams v0.0.0-20240314144324-c7f7c6466f7f // indirect golang.org/x/mod v0.20.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/term v0.23.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/term v0.24.0 // indirect + golang.org/x/text v0.18.0 // indirect golang.org/x/tools v0.24.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect diff --git a/tools/mod/go.sum b/tools/mod/go.sum index b52c0fe8c..f0ec1d5a1 100644 --- a/tools/mod/go.sum +++ b/tools/mod/go.sum @@ -344,18 +344,10 @@ github.com/polyfloyd/go-errorlint v1.6.0 h1:tftWV9DE7txiFzPpztTAwyoRLKNj9gpVm2cg github.com/polyfloyd/go-errorlint v1.6.0/go.mod h1:HR7u8wuP1kb1NeN1zqTd1ZMlqUKPPHF+Id4vIPvDqVw= github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= github.com/prashantv/gostub v1.1.0/go.mod h1:A5zLQHz7ieHGG7is6LLXLz7I8+3LZzsrV0P1IAHhP5U= -github.com/prometheus/client_golang v1.20.0 h1:jBzTZ7B099Rg24tny+qngoynol8LtVYlA2bqx3vEloI= -github.com/prometheus/client_golang v1.20.0/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= -github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= -github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G1dc= -github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8= -github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= -github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= @@ -535,8 +527,8 @@ golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIi golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= -golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= -golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= +golang.org/x/crypto v0.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= +golang.org/x/crypto v0.27.0/go.mod h1:1Xngt8kV6Dvbssa53Ziq6Eqn0HqbZi5Z6R0ZpwQzt70= golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56 h1:2dVuKD2vS7b0QIHQbpyTISPd0LeHDbnYEryqj5Q1ug8= golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56/go.mod h1:M4RDyNAINzryxdtnbRXRL/OHtkFuWGRjvuhBJpk2IlY= golang.org/x/exp/typeparams v0.0.0-20220428152302-39d4317da171/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= @@ -627,8 +619,8 @@ golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -643,8 +635,8 @@ golang.org/x/term v0.10.0/go.mod h1:lpqdcUyK/oCiQxvxVrppt5ggO2KCZ5QblwqPnfZ6d5o= golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= -golang.org/x/term v0.23.0 h1:F6D4vR+EHoL9/sWAWgAR1H2DcHr4PareCbAaCo1RpuU= -golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= +golang.org/x/term v0.24.0 h1:Mh5cbb+Zk2hqqXNO7S1iTjEphVL+jb8ZWaqh/g+JWkM= +golang.org/x/term v0.24.0/go.mod h1:lOBK/LVxemqiMij05LGJ0tzNr8xlmwBRJ81PX6wVLH8= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= @@ -659,8 +651,8 @@ golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190321232350-e250d351ecad/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190910044552-dd2b5c81c578/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -694,16 +686,10 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/genproto/googleapis/api v0.0.0-20240814211410-ddb44dafa142 h1:wKguEg1hsxI2/L3hUYrpo1RVi48K+uTyzKqprwLXsb8= -google.golang.org/genproto/googleapis/api v0.0.0-20240814211410-ddb44dafa142/go.mod h1:d6be+8HhtEtucleCbxpPW9PA9XwISACu8nvpPqF0BVo= google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240814211410-ddb44dafa142 h1:e7S5W7MGGLaSu8j3YjdezkZ+m1/Nm0uRVRMEMGk26Xs= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240814211410-ddb44dafa142/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= diff --git a/tools/rw-heatmaps/go.mod b/tools/rw-heatmaps/go.mod index ae6e0e9d8..43ffbaa09 100644 --- a/tools/rw-heatmaps/go.mod +++ b/tools/rw-heatmaps/go.mod @@ -21,5 +21,5 @@ require ( github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect golang.org/x/image v0.18.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/text v0.18.0 // indirect ) diff --git a/tools/rw-heatmaps/go.sum b/tools/rw-heatmaps/go.sum index 49deecd68..a1b419439 100644 --- a/tools/rw-heatmaps/go.sum +++ b/tools/rw-heatmaps/go.sum @@ -52,8 +52,8 @@ golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= 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.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= 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.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= diff --git a/tools/testgrid-analysis/go.mod b/tools/testgrid-analysis/go.mod index 489e9e87d..d73b42411 100644 --- a/tools/testgrid-analysis/go.mod +++ b/tools/testgrid-analysis/go.mod @@ -16,8 +16,8 @@ require ( github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/spf13/pflag v1.0.5 // indirect golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/grpc v1.66.0 // indirect ) diff --git a/tools/testgrid-analysis/go.sum b/tools/testgrid-analysis/go.sum index 9527c3869..75c50bfe7 100644 --- a/tools/testgrid-analysis/go.sum +++ b/tools/testgrid-analysis/go.sum @@ -1463,8 +1463,8 @@ golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.9.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= 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.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1495,8 +1495,8 @@ golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.10.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= From 7e8e0bf1f907c63b62b1b98cfadf07859d616034 Mon Sep 17 00:00:00 2001 From: James Blair Date: Mon, 16 Sep 2024 11:31:30 +1200 Subject: [PATCH 15/20] Ensure commits are signed in scripts/release.sh. Signed-off-by: James Blair --- scripts/release.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/release.sh b/scripts/release.sh index 7c457e41d..c2940a1fc 100755 --- a/scripts/release.sh +++ b/scripts/release.sh @@ -154,7 +154,7 @@ main() { # shellcheck disable=SC2038,SC2046 run git add $(find . -name go.mod ! -path './release/*'| xargs) run git diff --staged | cat - run git commit -m "version: bump up to ${VERSION}" + run git commit --signoff --message "version: bump up to ${VERSION}" run git diff --staged | cat fi From 2d5e1858cd09712ba8fd03bf2f2779aad0ce20d4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 16 Sep 2024 17:47:08 +0000 Subject: [PATCH 16/20] build(deps): bump github/codeql-action from 3.26.6 to 3.26.7 Bumps [github/codeql-action](https://github.com/github/codeql-action) from 3.26.6 to 3.26.7. - [Release notes](https://github.com/github/codeql-action/releases) - [Changelog](https://github.com/github/codeql-action/blob/main/CHANGELOG.md) - [Commits](https://github.com/github/codeql-action/compare/4dd16135b69a43b6c8efb853346f8437d92d3c93...8214744c546c1e5c8f03dde8fab3a7353211988d) --- updated-dependencies: - dependency-name: github/codeql-action dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- .github/workflows/codeql-analysis.yml | 6 +++--- .github/workflows/release.yaml | 2 +- .github/workflows/scorecards.yml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index fc0928808..621b572c4 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -40,7 +40,7 @@ jobs: uses: actions/checkout@692973e3d937129bcbf40652eb9f2f61becf3332 # v4.1.7 # Initializes the CodeQL tools for scanning. - name: Initialize CodeQL - uses: github/codeql-action/init@4dd16135b69a43b6c8efb853346f8437d92d3c93 # v3.26.6 + uses: github/codeql-action/init@8214744c546c1e5c8f03dde8fab3a7353211988d # v3.26.7 with: # If you wish to specify custom queries, you can do so here or in a config file. # By default, queries listed here will override any specified in a config file. @@ -50,6 +50,6 @@ jobs: # Autobuild attempts to build any compiled languages (C/C++, C#, or Java). # If this step fails, then you should remove it and run the build manually (see below) - name: Autobuild - uses: github/codeql-action/autobuild@4dd16135b69a43b6c8efb853346f8437d92d3c93 # v3.26.6 + uses: github/codeql-action/autobuild@8214744c546c1e5c8f03dde8fab3a7353211988d # v3.26.7 - name: Perform CodeQL Analysis - uses: github/codeql-action/analyze@4dd16135b69a43b6c8efb853346f8437d92d3c93 # v3.26.6 + uses: github/codeql-action/analyze@8214744c546c1e5c8f03dde8fab3a7353211988d # v3.26.7 diff --git a/.github/workflows/release.yaml b/.github/workflows/release.yaml index 1358f9fa1..dcda1645d 100644 --- a/.github/workflows/release.yaml +++ b/.github/workflows/release.yaml @@ -67,6 +67,6 @@ jobs: format: 'sarif' output: 'trivy-results-${{ matrix.platforms }}.sarif' - name: upload scan results - uses: github/codeql-action/upload-sarif@4dd16135b69a43b6c8efb853346f8437d92d3c93 # v3.26.6 + uses: github/codeql-action/upload-sarif@8214744c546c1e5c8f03dde8fab3a7353211988d # v3.26.7 with: sarif_file: 'trivy-results-${{ matrix.platforms }}.sarif' diff --git a/.github/workflows/scorecards.yml b/.github/workflows/scorecards.yml index 6faa515a0..fb6c6e33f 100644 --- a/.github/workflows/scorecards.yml +++ b/.github/workflows/scorecards.yml @@ -50,6 +50,6 @@ jobs: # Upload the results to GitHub's code scanning dashboard. - name: "Upload to code-scanning" - uses: github/codeql-action/upload-sarif@4dd16135b69a43b6c8efb853346f8437d92d3c93 # v3.26.6 + uses: github/codeql-action/upload-sarif@8214744c546c1e5c8f03dde8fab3a7353211988d # v3.26.7 with: sarif_file: results.sarif From d12bb7e0fba2e4f9bca55942dcd96870c95bf06b Mon Sep 17 00:00:00 2001 From: Lucas Rodriguez Date: Mon, 16 Sep 2024 14:07:16 -0500 Subject: [PATCH 17/20] Ensure consistent file permissions on broken WAL Signed-off-by: Lucas Rodriguez --- server/storage/wal/repair.go | 2 +- server/storage/wal/repair_test.go | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/server/storage/wal/repair.go b/server/storage/wal/repair.go index 537340451..d1a887835 100644 --- a/server/storage/wal/repair.go +++ b/server/storage/wal/repair.go @@ -67,7 +67,7 @@ func Repair(lg *zap.Logger, dirpath string) bool { case errors.Is(err, io.ErrUnexpectedEOF): brokenName := f.Name() + ".broken" - bf, bferr := os.Create(brokenName) + bf, bferr := os.OpenFile(brokenName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, fileutil.PrivateFileMode) if bferr != nil { lg.Warn("failed to create backup file", zap.String("path", brokenName), zap.Error(bferr)) return false diff --git a/server/storage/wal/repair_test.go b/server/storage/wal/repair_test.go index b1fd9d25d..38e3641bd 100644 --- a/server/storage/wal/repair_test.go +++ b/server/storage/wal/repair_test.go @@ -18,12 +18,14 @@ import ( "fmt" "io" "os" + "path/filepath" "testing" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.uber.org/zap/zaptest" + "go.etcd.io/etcd/client/pkg/v3/fileutil" "go.etcd.io/etcd/server/v3/storage/wal/walpb" "go.etcd.io/raft/v3/raftpb" ) @@ -77,6 +79,14 @@ func testRepair(t *testing.T, ents [][]raftpb.Entry, corrupt corruptFunc, expect // repair the wal require.True(t, Repair(lg, p), "'Repair' returned 'false', want 'true'") + // verify the broken wal has correct permissions + bf := filepath.Join(p, filepath.Base(w.tail().Name())+".broken") + fi, err := os.Stat(bf) + require.NoError(t, err) + expectedPerms := fmt.Sprintf("%o", os.FileMode(fileutil.PrivateFileMode)) + actualPerms := fmt.Sprintf("%o", fi.Mode().Perm()) + require.Equal(t, expectedPerms, actualPerms, "unexpected file permissions on .broken wal") + // read it back w, err = Open(lg, p, walpb.Snapshot{}) require.NoError(t, err) From 7e3901fb04d5023c16a01ab1b9fed74e51a2c8e2 Mon Sep 17 00:00:00 2001 From: Ivan Valdes Date: Tue, 17 Sep 2024 13:57:19 -0700 Subject: [PATCH 18/20] docs: update release.md with improvements from v3.4.34 release Signed-off-by: Ivan Valdes --- Documentation/contributor-guide/release.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/Documentation/contributor-guide/release.md b/Documentation/contributor-guide/release.md index 5f045540e..e5726d00b 100644 --- a/Documentation/contributor-guide/release.md +++ b/Documentation/contributor-guide/release.md @@ -78,11 +78,14 @@ which don't need to be executed before releasing each version. It generates all release binaries under the directory `/tmp/etcd-release-${VERSION}/etcd/release/` and images. Binaries are pushed to the Google Cloud bucket under project `etcd-development`, and images are pushed to `quay.io` and `gcr.io`. + + **Remove the `quay.io` login entry from `~/.docker/config.json` after pushing Docker images.** 7. Publish the release page on GitHub - Set the release title as the version name - Choose the correct release tag (generated from step #4) - Follow the format of previous release pages - Attach the generated binaries and signature file + - Verify the historical binary size for each architecture. If there's a big difference, verify that it works for that architecture - Select whether it's a pre-release - Publish the release 8. Announce to the etcd-dev googlegroup From fc901bd01901391939072a6d84945e6ff55348c2 Mon Sep 17 00:00:00 2001 From: Chun-Hung Tseng Date: Sat, 14 Sep 2024 22:48:08 +0200 Subject: [PATCH 19/20] Add tools/mod to module_dirs As `tools/mod` also contains the `go.mod` file. We should add it to the `module_dirs` variable, so that when executing `./scripts/fix.sh`, the proper checks and fixes can be applied. To address the issue of broken unit tests and code coverage due to the directory's lack of Go code, we've introduced a new doc.go file. This file acts as a placeholder, enabling tools like golangci-lint and go test to function correctly. --- Discovered when working on https://github.com/etcd-io/etcd/pull/18575 The directories are checked against the following: - Command: `find . -type f -name go.mod -exec dirname {} \;` - Output: ``` ./etcdutl . ./tools/testgrid-analysis ./tools/rw-heatmaps ./tools/mod ./etcdctl ./tests ./server ./api ./client/internal/v2 ./client/v3 ./client/pkg ./pkg ``` Signed-off-by: Chun-Hung Tseng --- scripts/test_lib.sh | 2 +- tools/mod/doc.go | 22 ++++++++++++++++++++++ tools/mod/libs.go | 2 +- tools/mod/tools.go | 2 +- 4 files changed, 25 insertions(+), 3 deletions(-) create mode 100644 tools/mod/doc.go diff --git a/scripts/test_lib.sh b/scripts/test_lib.sh index 41e9a5e15..8303d1643 100644 --- a/scripts/test_lib.sh +++ b/scripts/test_lib.sh @@ -166,7 +166,7 @@ function run_for_module { } function module_dirs() { - echo "api pkg client/pkg client/internal/v2 client/v3 server etcdutl etcdctl tests tools/rw-heatmaps tools/testgrid-analysis ." + echo "api pkg client/pkg client/internal/v2 client/v3 server etcdutl etcdctl tests tools/mod tools/rw-heatmaps tools/testgrid-analysis ." } # maybe_run [cmd...] runs given command depending on the DRY_RUN flag. diff --git a/tools/mod/doc.go b/tools/mod/doc.go new file mode 100644 index 000000000..e9cd697f8 --- /dev/null +++ b/tools/mod/doc.go @@ -0,0 +1,22 @@ +// Copyright 2024 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// As this directory implements the pattern for tracking tool dependencies as documented here: +// https://go.dev/wiki/Modules#how-can-i-track-tool-dependencies-for-a-module, it doesn't +// contain any valid go source code in the directory directly. This would break scripts for +// unit testing, golangci-lint, and coverage calculation. +// +// Thus, to ensure tools to run normally, we've added this empty file. + +package mod diff --git a/tools/mod/libs.go b/tools/mod/libs.go index 48d537e2d..fd392d55a 100644 --- a/tools/mod/libs.go +++ b/tools/mod/libs.go @@ -15,7 +15,7 @@ //go:build libs // This file implements that pattern: -// https://github.com/golang/go/wiki/Modules#how-can-i-track-tool-dependencies-for-a-module +// https://go.dev/wiki/Modules#how-can-i-track-tool-dependencies-for-a-module // for etcd. Thanks to this file 'go mod tidy' does not removes dependencies. package libs diff --git a/tools/mod/tools.go b/tools/mod/tools.go index 72afb8045..ce9af7751 100644 --- a/tools/mod/tools.go +++ b/tools/mod/tools.go @@ -15,7 +15,7 @@ //go:build tools // This file implements that pattern: -// https://github.com/golang/go/wiki/Modules#how-can-i-track-tool-dependencies-for-a-module +// https://go.dev/wiki/Modules#how-can-i-track-tool-dependencies-for-a-module // for etcd. Thanks to this file 'go mod tidy' does not removes dependencies. package tools From 5b2faec9837ec647203a7db953d147bcdd231fb2 Mon Sep 17 00:00:00 2001 From: mevrin Date: Wed, 18 Sep 2024 21:40:57 -0400 Subject: [PATCH 20/20] fea: Remane modules_exp to modules_for_bom to improve code and clarity Signed-off-by: mevrin --- scripts/test.sh | 2 +- scripts/test_lib.sh | 2 +- scripts/updatebom.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scripts/test.sh b/scripts/test.sh index cb35af673..3b15638cf 100755 --- a/scripts/test.sh +++ b/scripts/test.sh @@ -509,7 +509,7 @@ function bom_pass { log_callout "Checking bill of materials..." # https://github.com/golang/go/commit/7c388cc89c76bc7167287fb488afcaf5a4aa12bf # shellcheck disable=SC2207 - modules=($(modules_exp)) + modules=($(modules_for_bom)) # Internally license-bill-of-materials tends to modify go.sum run cp go.sum go.sum.tmp || return 2 diff --git a/scripts/test_lib.sh b/scripts/test_lib.sh index 41e9a5e15..ebd739156 100644 --- a/scripts/test_lib.sh +++ b/scripts/test_lib.sh @@ -196,7 +196,7 @@ function modules() { echo "${modules[@]}" } -function modules_exp() { +function modules_for_bom() { for m in $(modules); do echo -n "${m}/... " done diff --git a/scripts/updatebom.sh b/scripts/updatebom.sh index dd482a325..2c6bf8720 100755 --- a/scripts/updatebom.sh +++ b/scripts/updatebom.sh @@ -12,7 +12,7 @@ function bom_fixlet { local modules # shellcheck disable=SC2207 - modules=($(modules_exp)) + modules=($(modules_for_bom)) if GOFLAGS=-mod=mod run_go_tool "github.com/appscodelabs/license-bill-of-materials" \ --override-file ./bill-of-materials.override.json \