Previous change logs can be found at CHANGELOG-3.3.
The minimum recommended etcd versions to run in production are 3.1.11+, 3.2.26+, and 3.3.11+.
See code changes and v3.4 upgrade guide for any breaking changes.
Again, before running upgrades from any previous release, please make sure to read change logs below and v3.4 upgrade guide.
See List of metrics for all metrics per release.
Note that any etcd_debugging_* metrics are experimental and subject to change.
etcd_debugging_mvcc_current_revision Prometheus metric.etcd_debugging_mvcc_compact_revision Prometheus metric.% characters in file descriptor warning message.embedembed.Config.ZapLoggerBuilder to allow creating a custom zap logger.google.golang.org/grpc from v1.23.0 to v1.23.1.See code changes and v3.4 upgrade guide for any breaking changes.
Again, before running upgrades from any previous release, please make sure to read change logs below and v3.4 upgrade guide.
grpc.WithBlock() to clientv3.Config.DialOptions.etcd_network_client_grpc_sent_bytes_total.read-only range request "key:\"/a\" range_end:\"/b\" " with result "range_response_count:3 size:96" took too long (97.966µs) to execute.etcd --auto-compaction-mode=revision --auto-compaction-retention=1000 automatically Compact on "latest revision" - 1000 every 5-minute (when latest revision is 30000, compact on revision 29000).etcd --auto-compaction-mode=periodic --auto-compaction-retention=24h automatically Compact with 24-hour retention windown for every 2.4-hour. Now, Compact happens for every 1-hour.etcd --auto-compaction-mode=periodic --auto-compaction-retention=30m automatically Compact with 30-minute retention windown for every 3-minute. Now, Compact happens for every 30-minute.etcd --auto-compaction-mode=periodic --auto-compaction-retention=24h).etcd --auto-compaction-mode=periodic --auto-compaction-retention=24h, v3.2.x, v3.3.0, v3.3.1, and v3.3.2 compact revision 2400, 2640, and 2880 for every 2.4-hour, while v3.3.3 or later compacts revision 2400, 2500, 2600 for every 1-hour.etcd --auto-compaction-mode=periodic --auto-compaction-retention=30m and writes per minute are about 1000, v3.3.0, v3.3.1, and v3.3.2 compact revision 30000, 33000, and 36000, for every 3-minute, while v3.3.3 or later compacts revision 30000, 60000, and 90000, for every 30-minute.Lookup non-blocking with concurrent Grant/Revoke.raft.ErrProposalDropped on internal Raft proposal drop in v3 applier and v2 applier.
raftpb.MsgProp arrives at current leader while there is an ongoing leadership transfer.snapshot package for easier snapshot workflow (see godoc.org/github.com/etcd/clientv3/snapshot for more).etcd --debug flag to see per-request debug information.snapshot status returns "snapshot file integrity check failed..." error.Verify function to perform corruption check on WAL contents.CommonName based auth.etcd --experimental-peer-skip-client-san-verification to skip verification of peer client address.etcd --experimental-compaction-batch-limit to sets the maximum revisions deleted in each compaction batch.grpc.WithBlock() to clientv3.Config.DialOptions.glide to Go module.
vendor directory under cmd/vendor directory to prevent conflicting transitive dependencies.cmd/vendor directory to vendor at repository root.cmd directory.go get/install/build on etcd packages (e.g. clientv3, tools/benchmark) enforce builds with etcd vendor directory.latest release container tag.
docker pull gcr.io/etcd-development/etcd:latest would not be up-to-date.docker pull gcr.io/etcd-development/etcd:v3.3 would still work.docker pull gcr.io/etcd-development/etcd:v3.4 would not work.docker pull gcr.io/etcd-development/etcd:v3.4.x instead, with the exact patch version.acbuild is not maintained anymore.*.aci files are not available from v3.4 release."github.com/coreos/etcd" to "github.com/etcd-io/etcd".
"go.etcd.io/etcd".import "go.etcd.io/etcd/raft".ETCDCTL_API=3 etcdctl default.
etcdctl set foo bar must be ETCDCTL_API=2 etcdctl set foo bar.ETCDCTL_API=3 etcdctl put foo bar could be just etcdctl put foo bar.etcd --enable-v2=false default.embed.DefaultEnableV2 false default.etcd --ca-file flag. Use etcd --trusted-ca-file instead (etcd --ca-file flag has been marked deprecated since v2.1).etcd --peer-ca-file flag. Use etcd --peer-trusted-ca-file instead (etcd --peer-ca-file flag has been marked deprecated since v2.1).pkg/transport.TLSInfo.CAFile field. Use pkg/transport.TLSInfo.TrustedCAFile instead (CAFile field has been marked deprecated since v2.1).--advertise-client-urls=http://:2379.--initial-advertise-peer-urls=http://:2380.ETCD_NAME=abc etcd --name=def.ETCD_INITIAL_CLUSTER_TOKEN=abc etcd --initial-cluster-token=def.ETCDCTL_ENDPOINTS=abc.com ETCDCTL_API=3 etcdctl endpoint health --endpoints=def.com.etcdserverpb.AuthRoleRevokePermissionRequest/key,range_end fields type from string to bytes.etcd_debugging_mvcc_db_total_size_in_bytes Prometheus metric (to be removed in v3.5). Use etcd_mvcc_db_total_size_in_bytes instead.etcd_debugging_mvcc_put_total Prometheus metric (to be removed in v3.5). Use etcd_mvcc_put_total instead.etcd_debugging_mvcc_delete_total Prometheus metric (to be removed in v3.5). Use etcd_mvcc_delete_total instead.etcd_debugging_mvcc_range_total Prometheus metric (to be removed in v3.5). Use etcd_mvcc_range_total instead.etcd_debugging_mvcc_txn_totalPrometheus metric (to be removed in v3.5). Use etcd_mvcc_txn_total instead.etcdserver.ServerConfig.SnapCount field to etcdserver.ServerConfig.SnapshotCount, to be consistent with the flag name etcd --snapshot-count.embed.Config.SnapCount field to embed.Config.SnapshotCount, to be consistent with the flag name etcd --snapshot-count.embed.Config.CorsInfo in *cors.CORSInfo type to embed.Config.CORS in map[string]struct{} type.embed.Config.SetupLogging.
embed.Config.Logger, embed.Config.LogOutputs, embed.Config.Debug fields.etcd --log-output to etcd --log-outputs to support multiple log outputs.
etcd --log-output will be deprecated in v3.5.embed.Config.LogOutput to embed.Config.LogOutputs to support multiple log outputs.embed.Config.LogOutputs type from string to []string to support multiple log outputs.
etcd --log-outputs accepts multiple writers, etcd configuration YAML file log-outputs field must be changed to []string type.etcd --config-file etcd.config.yaml can have log-outputs: default field, now must be log-outputs: [default].etcd --debug flag. Use etcd --log-level=debug flag instead.
etcd --debug flag in favor of etcd --log-level=debug.etcdctl snapshot exit codes with snapshot package.
snapshot save/restore commands).grpc.ErrClientConnClosing.
clientv3 and proxy/grpcproxy now does not return grpc.ErrClientConnClosing.grpc.ErrClientConnClosing has been deprecated in gRPC >= 1.10.clientv3.IsConnCanceled(error) or google.golang.org/grpc/status.FromError(error) instead./v3beta with /v3.
/v3alpha./v3beta in v3.5.curl -L http://localhost:2379/v3beta/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}' still works as a fallback to curl -L http://localhost:2379/v3/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}', but curl -L http://localhost:2379/v3beta/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}' won't work in v3.5. Use curl -L http://localhost:2379/v3/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}' instead.wal package function signatures to support structured logger and logging to file in server-side.
Open(dirpath string, snap walpb.Snapshot) (*WAL, error), now Open(lg *zap.Logger, dirpath string, snap walpb.Snapshot) (*WAL, error).OpenForRead(dirpath string, snap walpb.Snapshot) (*WAL, error), now OpenForRead(lg *zap.Logger, dirpath string, snap walpb.Snapshot) (*WAL, error).Repair(dirpath string) bool, now Repair(lg *zap.Logger, dirpath string) bool.Create(dirpath string, metadata []byte) (*WAL, error), now Create(lg *zap.Logger, dirpath string, metadata []byte) (*WAL, error).pkg/cors package.etcdserver.
"github.com/coreos/etcd/alarm" to "go.etcd.io/etcd/etcdserver/api/v3alarm"."github.com/coreos/etcd/compactor" to "go.etcd.io/etcd/etcdserver/api/v3compactor"."github.com/coreos/etcd/discovery" to "go.etcd.io/etcd/etcdserver/api/v2discovery"."github.com/coreos/etcd/etcdserver/auth" to "go.etcd.io/etcd/etcdserver/api/v2auth"."github.com/coreos/etcd/etcdserver/membership" to "go.etcd.io/etcd/etcdserver/api/membership"."github.com/coreos/etcd/etcdserver/stats" to "go.etcd.io/etcd/etcdserver/api/v2stats"."github.com/coreos/etcd/error" to "go.etcd.io/etcd/etcdserver/api/v2error"."github.com/coreos/etcd/rafthttp" to "go.etcd.io/etcd/etcdserver/api/rafthttp"."github.com/coreos/etcd/snap" to "go.etcd.io/etcd/etcdserver/api/snap"."github.com/coreos/etcd/store" to "go.etcd.io/etcd/etcdserver/api/v2store".pkg/adt.IntervalTree from struct to interface.
pkg/adt README and pkg/adt godoc./version defines version 3.4.x-pre, instead of 3.4.y+git.
3.4.5-pre, instead of 3.4.4+git.github.com/coreos/bbolt from v1.3.1-coreos.6 to go.etcd.io/bbolt v1.3.3.google.golang.org/grpc from v1.7.5 to v1.23.0.github.com/ugorji/go/codec to github.com/json-iterator/go, to regenerate v2 client (See #10667 for more).github.com/ghodss/yaml to sigs.k8s.io/yaml (See #10687 for more).golang.org/x/crypto from crypto@9419663f5 to crypto@0709b304e793.golang.org/x/net from net@66aacef3d to net@adae6a3d119a.golang.org/x/sys from sys@ebfc5b463 to sys@c7b8b68b1456.golang.org/x/text from text@b19bf474d to v0.3.0.golang.org/x/time from time@c06e80d93 to time@fbb02b229.github.com/golang/protobuf from golang/protobuf@1e59b77b5 to v1.3.2.gopkg.in/yaml.v2 from yaml@cd8b52f82 to yaml@5420a8b67.github.com/dgrijalva/jwt-go from v3.0.0 to v3.2.0.github.com/soheilhy/cmux from v0.1.3 to v0.1.4.github.com/google/btree from google/btree@925471ac9 to v1.0.0.github.com/spf13/cobra from spf13/cobra@1c44ec8d3 to v0.0.3.github.com/spf13/pflag from v1.0.0 to spf13/pflag@1ce0cc6db.github.com/coreos/go-systemd from v15 to v17.github.com/prometheus/client_golang from prometheus/client_golang@5cec1d042 to v1.0.0.github.com/grpc-ecosystem/go-grpc-prometheus from grpc-ecosystem/go-grpc-prometheus@0dafe0d49 to v1.2.0.github.com/grpc-ecosystem/grpc-gateway from v1.3.1 to v1.4.1.github.com/kr/pty to github.com/creack/pty, as the later has replaced the original module.github.com/gogo/protobuf from v1.0.0 to v1.2.1.See List of metrics for all metrics per release.
Note that any etcd_debugging_* metrics are experimental and subject to change.
etcd_snap_db_fsync_duration_seconds_count Prometheus metric.etcd_snap_db_save_total_duration_seconds_bucket Prometheus metric.etcd_network_snapshot_send_success Prometheus metric.etcd_network_snapshot_send_failures Prometheus metric.etcd_network_snapshot_send_total_duration_seconds Prometheus metric.etcd_network_snapshot_receive_success Prometheus metric.etcd_network_snapshot_receive_failures Prometheus metric.etcd_network_snapshot_receive_total_duration_seconds Prometheus metric.etcd_network_active_peers Prometheus metric.
"7339c4e5e833c029" server /metrics returns etcd_network_active_peers{Local="7339c4e5e833c029",Remote="729934363faa4a24"} 1 and etcd_network_active_peers{Local="7339c4e5e833c029",Remote="b548c2511513015"} 1. This indicates that the local node "7339c4e5e833c029" currently has two active remote peers "729934363faa4a24" and "b548c2511513015" in a 3-node cluster. If the node "b548c2511513015" is down, the local node "7339c4e5e833c029" will show etcd_network_active_peers{Local="7339c4e5e833c029",Remote="729934363faa4a24"} 1 and etcd_network_active_peers{Local="7339c4e5e833c029",Remote="b548c2511513015"} 0.etcd_network_disconnected_peers_total Prometheus metric.
"b548c2511513015" is down, the local node "7339c4e5e833c029" server /metrics would return etcd_network_disconnected_peers_total{Local="7339c4e5e833c029",Remote="b548c2511513015"} 1, while active peer metrics will show etcd_network_active_peers{Local="7339c4e5e833c029",Remote="729934363faa4a24"} 1 and etcd_network_active_peers{Local="7339c4e5e833c029",Remote="b548c2511513015"} 0.etcd_network_server_stream_failures_total Prometheus metric.
etcd_network_server_stream_failures_total{API="lease-keepalive",Type="receive"} 1etcd_network_server_stream_failures_total{API="watch",Type="receive"} 1etcd_network_peer_round_trip_time_seconds Prometheus metric to track leader heartbeats.
etcd_network_peer_round_trip_time_seconds Prometheus metric histogram upper-bound.
etcd_server_is_leader Prometheus metric.etcd_server_id Prometheus metric.etcd_cluster_version Prometheus metric.etcd_server_version Prometheus metric.
etcd-version-monitor.etcd_server_go_version Prometheus metric.etcd_server_health_success Prometheus metric.etcd_server_health_failures Prometheus metric.etcd_server_read_indexes_failed_total Prometheus metric.etcd_server_heartbeat_send_failures_total Prometheus metric.etcd_server_slow_apply_total Prometheus metric.etcd_server_slow_read_indexes_total Prometheus metric.etcd_server_quota_backend_bytes Prometheus metric.
etcd_mvcc_db_total_size_in_bytes and etcd_mvcc_db_total_size_in_use_in_bytes.etcd_server_quota_backend_bytes 2.147483648e+09 means current quota size is 2 GB.etcd_mvcc_db_total_size_in_bytes 20480 means current physically allocated DB size is 20 KB.etcd_mvcc_db_total_size_in_use_in_bytes 16384 means future DB size if defragment operation is complete.etcd_mvcc_db_total_size_in_bytes - etcd_mvcc_db_total_size_in_use_in_bytes is the number of bytes that can be saved on disk with defragment operation.etcd_mvcc_db_total_size_in_use_in_bytes Prometheus metric.
etcd_mvcc_db_total_size_in_bytes and etcd_mvcc_db_total_size_in_use_in_bytes.etcd_server_quota_backend_bytes 2.147483648e+09 means current quota size is 2 GB.etcd_mvcc_db_total_size_in_bytes 20480 means current physically allocated DB size is 20 KB.etcd_mvcc_db_total_size_in_use_in_bytes 16384 means future DB size if defragment operation is complete.etcd_mvcc_db_total_size_in_bytes - etcd_mvcc_db_total_size_in_use_in_bytes is the number of bytes that can be saved on disk with defragment operation.etcd_mvcc_db_open_read_transactions Prometheus metric.etcd_snap_fsync_duration_seconds Prometheus metric.etcd_disk_backend_defrag_duration_seconds Prometheus metric.etcd_mvcc_hash_duration_seconds Prometheus metric.etcd_mvcc_hash_rev_duration_seconds Prometheus metric.etcd_debugging_disk_backend_commit_rebalance_duration_seconds Prometheus metric.etcd_debugging_disk_backend_commit_spill_duration_seconds Prometheus metric.etcd_debugging_disk_backend_commit_write_duration_seconds Prometheus metric.etcd_debugging_lease_granted_total Prometheus metric.etcd_debugging_lease_revoked_total Prometheus metric.etcd_debugging_lease_renewed_total Prometheus metric.etcd_debugging_lease_ttl_total Prometheus metric.etcd_network_snapshot_send_inflights_total Prometheus metric.etcd_network_snapshot_receive_inflights_total Prometheus metric.etcd_server_snapshot_apply_in_progress_total Prometheus metric.etcd_server_is_learner Prometheus metric.etcd_server_learner_promote_failures Prometheus metric.etcd_server_learner_promote_successes Prometheus metric.etcd_debugging_mvcc_index_compaction_pause_duration_milliseconds Prometheus metric histogram upper-bound.
etcd_network_peer_sent_failures_total Prometheus metric count.etcd_debugging_server_lease_expired_total Prometheus metric.etcd_debugging_mvcc_db_total_size_in_bytes Prometheus metric (to be removed in v3.5). Use etcd_mvcc_db_total_size_in_bytes instead.etcd_debugging_mvcc_put_total Prometheus metric (to be removed in v3.5). Use etcd_mvcc_put_total instead.etcd_debugging_mvcc_delete_total Prometheus metric (to be removed in v3.5). Use etcd_mvcc_delete_total instead.etcd_debugging_mvcc_range_total Prometheus metric (to be removed in v3.5). Use etcd_mvcc_range_total instead.etcd_debugging_mvcc_txn_totalPrometheus metric (to be removed in v3.5). Use etcd_mvcc_txn_total instead.See security doc for more details.
etcd --cipher-suites flag.etcd --host-whitelist flag, etcdserver.Config.HostWhitelist, and embed.Config.HostWhitelist, to prevent "DNS Rebinding" attack.
"localhost" (or any other address). Then, all HTTP endpoints of etcd server listening on "localhost" becomes accessible, thus vulnerable to DNS rebinding attacks (CVE-2018-5702)."HostWhitelist" is not empty, only allow HTTP requests whose Host field is listed in whitelist."HostWhitelist" is "*", which means insecure server allows all client HTTP requests."localhost", "127.0.0.1", etc.).etcd --host-whitelist example.com, then the server will reject all HTTP requests whose Host field is not example.com (also rejects requests to "localhost").etcd --cors in v3 HTTP requests (gRPC gateway).ttl field for etcd Authentication JWT token.
etcd --auth-token jwt,pub-key=<pub key path>,priv-key=<priv key path>,sign-method=<sign method>,ttl=5m.etcdserver.ServerConfig.AuthToken.(*tls.Config).GetCertificate for TLS reload if and only if server's (*tls.Config).Certificates field is not empty, or (*tls.ClientHelloInfo).ServerName is not empty with a valid SNI from the client. Previously, etcd always populates (*tls.Config).Certificates on the initial client TLS handshake, as non-empty. Thus, client was always expected to supply a matching SNI in order to pass the TLS verification and to trigger (*tls.Config).GetCertificate to reload TLS assets.*tls.ClientHelloInfo with an empty ServerName field, thus failing to trigger the TLS reload on initial TLS handshake; this becomes a problem when expired certificates need to be replaced online.(*tls.Config).Certificates is created empty on initial TLS client handshake, first to trigger (*tls.Config).GetCertificate, and then to populate rest of the certificates on every new TLS connection, even when client SNI is empty (e.g. cert only includes IPs).rpctypes.ErrLeaderChanged.
etcd --initial-election-tick-advance flag to configure initial election tick fast-forward.
etcd --initial-election-tick-advance=true, then local member fast-forwards election ticks to speed up "initial" leader election trigger.etcd --initial-election-tick-advance=false.etcd --initial-election-tick-advance at the cost of slow initial bootstrap.etcd --pre-vote flag to enable to run an additional Raft election phase.
etcd --pre-vote=false by default.etcd --pre-vote=true by default.etcd --experimental-compaction-batch-limit to sets the maximum revisions deleted in each compaction batch.etcd --discovery-srv-name flag to support custom DNS SRV name with discovery.
_etcd-server-ssl._tcp.[YOUR_HOST] and _etcd-server._tcp.[YOUR_HOST].etcd --discovery-srv-name="foo", then query _etcd-server-ssl-foo._tcp.[YOUR_HOST] and _etcd-server-foo._tcp.[YOUR_HOST].etcd --cipher-suites flag.etcd --cors in v3 HTTP requests (gRPC gateway).etcd --log-output to etcd --log-outputs to support multiple log outputs.
etcd --log-output will be deprecated in v3.5.etcd --logger flag to support structured logger and multiple log outputs in server-side.
etcd --logger=capnslog will be deprecated in v3.5.etcd --logger=capnslog --log-outputs=default is the default setting and same as previous etcd server logging format.etcd --logger=zap --log-outputs=default is not supported when etcd --logger=zap.etcd --logger=zap --log-outputs=stderr instead.etcd --logger=zap --log-outputs=systemd/journal to send logs to the local systemd journal.etcd --logger=capnslog --log-outputs=default redirects server logs to local systemd journal. And if write to journald fails, it writes to os.Stderr as a fallback.os.Stderr, which is inefficient.etcd --logger=zap --log-outputs=stderr will log server operations in JSON-encoded format and writes logs to os.Stderr. Use this to override journald log redirects.etcd --logger=zap --log-outputs=stdout will log server operations in JSON-encoded format and writes logs to os.Stdout Use this to override journald log redirects.etcd --logger=zap --log-outputs=a.log will log server operations in JSON-encoded format and writes logs to the specified file a.log.etcd --logger=zap --log-outputs=a.log,b.log,c.log,stdout writes server logs to multiple files a.log, b.log and c.log at the same time and outputs to os.Stderr, in JSON-encoded format.etcd --logger=zap --log-outputs=/dev/null will discard all server logs.etcd --log-level flag to support log level.
etcd --debug flag in favor of etcd --log-level=debug.etcd --backend-batch-limit flag.etcd --backend-batch-interval flag.mvcc "unsynced" watcher restore operation.
mvcc server panic from restore operation.
curl -L http://localhost:2379/v3/election/proclaim -X POST -d '{"value":""}', curl -L http://localhost:2379/v3/election/resign -X POST -d '{"value":""}'.etcd --auto-compaction-mode revision --auto-compaction-retention 1 was translated to revision retention 3600000000000.etcd --auto-compaction-mode revision --auto-compaction-retention 1 is correctly parsed as revision retention 1.TTL values for Lease Grant.
TTL parameter to Grant request is unit of second.TTL values exceeding math.MaxInt64 expire in unexpected ways.rpctypes.ErrLeaseTTLTooLarge to client, when the requested TTL is larger than 9,000,000,000 seconds (which is >285 years).Lease is meant for short-periodic keepalives or sessions, in the range of seconds or minutes. Not for hours or days!raft.Config.CheckQuorum when starting with ForceNewCluster.etcd --wal-dir directory.
lost+found in WAL directory prevent etcd server boot.lost+found or a file that's not suffixed with .wal is considered non-initialized.ETCD_CONFIG_FILE env variable parsing in etcd.rafthttp transport pause/resume.Unavailable.InvalidArgument.isLearner field to etcdserverpb.Member, etcdserverpb.MemberAddRequest and etcdserverpb.StatusResponse as part of raft learner implementation.MemberPromote rpc to etcdserverpb.Cluster interface and the corresponding MemberPromoteRequest and MemberPromoteResponse as part of raft learner implementation.snapshot package for snapshot restore/save operations (see godoc.org/github.com/etcd/clientv3/snapshot for more).watch_id field to etcdserverpb.WatchCreateRequest to allow user-provided watch ID to mvcc.
watch_id is returned via etcdserverpb.WatchResponse, if any.fragment field to etcdserverpb.WatchCreateRequest to request etcd server to split watch events when the total size of events exceeds etcd --max-request-bytes flag value plus gRPC-overhead 512 bytes.
embed.DefaultMaxRequestBytes which is 1.5 MiB plus gRPC-overhead 512 bytes.fragment field true, the server will split watch events into a set of chunks, each of which is a subset of watch events below server-side request limit.etcd --max-request-bytes flag value is 1 MiB. Then, server will send 10 separate fragmented events to the client.etcd --max-request-bytes flag value is 1 MiB and clientv3.Config.MaxCallRecvMsgSize is 1 MiB. Then, server will try to send 5 separate fragmented events to the client, and the client will error with "code = ResourceExhausted desc = grpc: received message larger than max (...)".clientv3 does in etcd v3.4).raftAppliedIndex field to etcdserverpb.StatusResponse for current Raft applied index.errors field to etcdserverpb.StatusResponse for server-side error.
"etcdserver: no leader", "NOSPACE", "CORRUPT"dbSizeInUse field to etcdserverpb.StatusResponse for actual DB size after compaction.WatchRequest.WatchProgressRequest.
WithProgressNotify that can be triggered manually.Note: v3.5 will deprecate etcd --log-package-levels flag for capnslog; etcd --logger=zap --log-outputs=stderr will the default. v3.5 will deprecate [CLIENT-URL]/config/local/log endpoint.
embedembed.Config.CipherSuites to specify a list of supported cipher suites for TLS handshake between client/server and peers.
embed.Config.ClientTLSInfo.CipherSuites and embed.Config.CipherSuites cannot be non-empty at the same time.embed.Config.ClientTLSInfo.CipherSuites or embed.Config.CipherSuites.embed.Config.InitialElectionTickAdvance to enable/disable initial election tick fast-forward.
embed.NewConfig() would return *embed.Config with InitialElectionTickAdvance as true by default.embed.CompactorModePeriodic for compactor.ModePeriodic.embed.CompactorModeRevision for compactor.ModeRevision.embed.Config.CorsInfo in *cors.CORSInfo type to embed.Config.CORS in map[string]struct{} type.embed.Config.SetupLogging.
embed.Config.Logger, embed.Config.LogOutputs, embed.Config.Debug fields.embed.Config.Logger to support structured logger zap in server-side.embed.Config.LogLevel.embed.Config.SnapCount field to embed.Config.SnapshotCount, to be consistent with the flag name etcd --snapshot-count.embed.Config.LogOutput to embed.Config.LogOutputs to support multiple log outputs.embed.Config.LogOutputs type from string to []string to support multiple log outputs.embed.Config.BackendBatchLimit field.embed.Config.BackendBatchInterval field.embed.DefaultEnableV2 false default.pkg/adtpkg/adt.IntervalTree from struct to interface.
pkg/adt README and pkg/adt godoc.pkg/adt.IntervalTree test coverage.
pkg/adt README and pkg/adt godoc.integrationCLUSTER_DEBUG to enable test cluster logging.
capnslog in integration tests.MemberAddAsLearner to Clientv3.Cluster interface. This API is used to add a learner member to etcd cluster.MemberPromote to Clientv3.Cluster interface. This API is used to promote a learner member in etcd cluster.rpctypes.ErrLeaderChanged from server.
WithFragment OpOption to support watch events fragmentation when the total size of events exceeds etcd --max-request-bytes flag value plus gRPC-overhead 512 bytes.
embed.DefaultMaxRequestBytes which is 1.5 MiB plus gRPC-overhead 512 bytes.fragment field true, the server will split watch events into a set of chunks, each of which is a subset of watch events below server-side request limit.etcd --max-request-bytes flag value is 1 MiB. Then, server will send 10 separate fragmented events to the client.etcd --max-request-bytes flag value is 1 MiB and clientv3.Config.MaxCallRecvMsgSize is 1 MiB. Then, server will try to send 5 separate fragmented events to the client, and the client will error with "code = ResourceExhausted desc = grpc: received message larger than max (...)".Watcher.RequestProgress method.
WithProgressNotify that can be triggered manually.<-chan *clientv3LeaseKeepAliveResponse from clientv3.Lease.KeepAlive was never consumed or channel is full, client was sending keepalive request every 500ms instead of expected rate of every "TTL / 3" duration.PermitWithoutStream.
PermitWithoutStream to true, client can send keepalive pings to server without any active streams(RPCs). In other words, it allows sending keepalive pings with unary or simple RPC calls.PermitWithoutStream is set to false by default.clientv3/concurrency package.(*Client).Endpoints() method race condition.grpc.ErrClientConnClosing.
clientv3 and proxy/grpcproxy now does not return grpc.ErrClientConnClosing.grpc.ErrClientConnClosing has been deprecated in gRPC >= 1.10.clientv3.IsConnCanceled(error) or google.golang.org/grpc/status.FromError(error) instead.ETCDCTL_API=3 etcdctl default.
etcdctl set foo bar must be ETCDCTL_API=2 etcdctl set foo bar.ETCDCTL_API=3 etcdctl put foo bar could be just etcdctl put foo bar.etcdctl member add --learner and etcdctl member promote to add and promote raft learner member in etcd cluster.etcdctl --password flag.
: character in user name.etcdctl --user user --password password get fooetcdctl user add --new-user-password flag.etcdctl check datascale command.etcdctl check datascale --auto-compact, --auto-defrag flags.etcdctl check perf --auto-compact, --auto-defrag flags.etcdctl defrag --cluster flag.endpoint status.endpoint status.etcdctl endpoint health --write-out support.
etcdctl endpoint health.etcdctl watch [key] [range_end] -- [exec-command…] parsing.
ETCDCTL_API=3 etcdctl watch foo -- echo watch event received panicked.etcdctl move-leader command for TLS-enabled endpoints.progress command to etcdctl watch --interactive.
WithProgressNotify that can be triggered manually.etcdctl snapshot
save.
etcdctl snapshot save command using flag --command-timeout."proxy-namespace__lostleader" and a watch revision "int64(math.MaxInt64 - 2)"./v3beta with /v3.
/v3alpha./v3beta in v3.5.curl -L http://localhost:2379/v3beta/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}' still works as a fallback to curl -L http://localhost:2379/v3/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}', but curl -L http://localhost:2379/v3beta/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}' won't work in v3.5. Use curl -L http://localhost:2379/v3/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}' instead./{v3beta,v3}/lease/leases, /{v3beta,v3}/lease/revoke, /{v3beta,v3}/lease/timetolive.
etcd --cors in v3 HTTP requests (gRPC gateway).raftraft.ErrProposalDropped.
(r *raft) Step returns raft.ErrProposalDropped if a proposal has been ignored.raftpb.MsgProp arrives at current leader while there is an ongoing leadership transfer.becomeLeader and stepLeader by keeping track of latest pb.EntryConfChange index.
pendingConf boolean field scanning the entire tail of the log, which can delay hearbeat send.(n *node) ApplyConfChange.raft.Config.MaxUncommittedEntriesSize to limit the total size of the uncommitted entries in bytes.
raft.ErrProposalDropped error.raft.Ready.CommittedEntries pagination using raft.Config.MaxSizePerMsg.
maybeCommit.
String method.raft/tracker.Progress.node and RawNode explicit.raft/quorum to reason about committed indexes as well as vote outcomes for both majority and joint quorums.
raft/node.go to centralize configuration change application.raft/confchange to internally support joint consensus.RawNode for node's event loop.RawNode.Bootstrap method.raftpb.ConfChangeV2 to use joint quorums.
raftpb.ConfChange continues to work as today: it allows carrying out a single configuration change. A pb.ConfChange proposal gets added to the Raft log as such and is thus also observed by the app during Ready handling, and fed back to ApplyConfChange.raftpb.ConfChangeV2 allows joint configuration changes but will continue to carry out configuration changes in "one phase" (i.e. without ever entering a joint config) when this is possible.raftpb.ConfChangeV2 messages initiate configuration changes. They support both the simple "one at a time" membership change protocol and full Joint Consensus allowing for arbitrary changes in membership.raftpb.ConfState.Nodes to raftpb.ConfState.Voters.Progress in stable order.tracker.Progress.Next == c.LastIndex is that the follower has no log at all (and will thus likely need a snapshot), though the app may have applied a snapshot out of band before adding the replica (thus making the first index the better choice).walVerify function to perform corruption check on WAL contents.wal directory cleanup on creation failures.etcd-dump-logs --entry-type flag to support WAL log filtering by entry type.etcd-dump-logs --stream-decoder flag to support custom decoder.SHA256SUMS file to release assets.