Previous change logs can be found at CHANGELOG-3.3.
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.
- Rewrite client balancer with new gRPC balancer interface.
- Add jitter to watch progress notify to prevent spikes in
etcd_network_client_grpc_sent_bytes_total
. - Improve slow requests warning logging.
- e.g.
etcdserver: read-only range request "key:\"\\000\" range_end:\"\\000\" " took too long [3.389041388s] to execute
- e.g.
- Improve TLS setup error logging to help debug TLS-enabled cluster configuring issues.
- Improve long-running concurrent read transactions under light write workloads.
- Previously, periodic commit on pending writes blocks incoming read transactions, even if there is no pending write.
- Now, periodic commit operation does not block concurrent read transactions, thus improves long-running read transaction performance.
- Adjust election timeout on server restart to reduce disruptive rejoining servers.
- Previously, etcd fast-forwards election ticks on server start, with only one tick left for leader election. This is to speed up start phase, without having to wait until all election ticks elapse. Advancing election ticks is useful for cross datacenter deployments with larger election timeouts. However, it was affecting cluster availability if the last tick elapses before leader contacts the restarted node.
- Now, when etcd restarts, it adjusts election ticks with more than one tick left, thus more time for leader to prevent disruptive restart.
- Add Raft Pre-Vote feature to reduce disruptive rejoining servers.
- For instance, a flaky(or rejoining) member may drop in and out, and start campaign. This member will end up with a higher term, and ignore all incoming messages with lower term. In this case, a new leader eventually need to get elected, thus disruptive to cluster availability. Raft implements Pre-Vote phase to prevent this kind of disruptions. If enabled, Raft runs an additional phase of election to check if pre-candidate can get enough votes to win an election.
- Adjust periodic compaction retention window.
- e.g.
--auto-compaction-mode=revision --auto-compaction-retention=1000
automaticallyCompact
on"latest revision" - 1000
every 5-minute (when latest revision is 30000, compact on revision 29000). - e.g. Previously,
--auto-compaction-mode=periodic --auto-compaction-retention=24h
automaticallyCompact
with 24-hour retention windown for every 2.4-hour. Now,Compact
happens for every 1-hour. - e.g. Previously,
--auto-compaction-mode=periodic --auto-compaction-retention=30m
automaticallyCompact
with 30-minute retention windown for every 3-minute. Now,Compact
happens for every 30-minute. - Periodic compactor keeps recording latest revisions for every compaction period when given period is less than 1-hour, or for every 1-hour when given compaction period is greater than 1-hour (e.g. 1-hour when
--auto-compaction-mode=periodic --auto-compaction-retention=24h
). - For every compaction period or 1-hour, compactor uses the last revision that was fetched before compaction period, to discard historical data.
- The retention window of compaction period moves for every given compaction period or hour.
- For instance, when hourly writes are 100 and
--auto-compaction-mode=periodic --auto-compaction-retention=24h
,v3.2.x
,v3.3.0
,v3.3.1
, andv3.3.2
compact revision 2400, 2640, and 2880 for every 2.4-hour, whilev3.3.3
or later compacts revision 2400, 2500, 2600 for every 1-hour. - Futhermore, when
--auto-compaction-mode=periodic --auto-compaction-retention=30m
and writes per minute are about 1000,v3.3.0
,v3.3.1
, andv3.3.2
compact revision 30000, 33000, and 36000, for every 3-minute, whilev3.3.3
or later compacts revision 30000, 60000, and 90000, for every 30-minute.
- e.g.
- Improve lease expire/revoke operation performance, address lease scalability issue.
- Make Lease
Lookup
non-blocking with concurrentGrant
/Revoke
. - Make etcd server return
raft.ErrProposalDropped
on internal Raft proposal drop in v3 applier and v2 applier.- e.g. a node is removed from cluster, or
raftpb.MsgProp
arrives at current leader while there is an ongoing leadership transfer.
- e.g. a node is removed from cluster, or
- Add
snapshot
package for easier snapshot workflow (seegodoc.org/github.com/etcd/snapshot
for more). - Improve functional tester coverage: proxy layer to run network fault tests in CI, TLS is enabled both for server and client, liveness mode, shuffle test sequence, membership reconfiguration failure cases, disastrous quorum loss and snapshot recover from a seed member, embedded etcd.
- Improve index compaction blocking by using a copy on write clone to avoid holding the lock for the traversal of the entire index.
- Remove
etcd --ca-file
flag, instead use--trusted-ca-file
(--ca-file
has been deprecated since v2.1). - Remove
etcd --peer-ca-file
flag, instead use--peer-trusted-ca-file
(--peer-ca-file
has been deprecated since v2.1). - Remove
pkg/transport.TLSInfo.CAFile
field, instead usepkg/transport.TLSInfo.TrustedCAFile
(CAFile
has been deprecated since v2.1). - Drop ACIs from official release.
- AppC was officially suspended, as of late 2016.
acbuild
is not maintained anymore.*.aci
files are not available fromv3.4
release.
- Exit on empty hosts in advertise URLs.
- Address advertise client URLs accepts empty hosts.
- e.g. exit with error on
--advertise-client-urls=http://:2379
. - e.g. exit with error on
--initial-advertise-peer-urls=http://:2380
.
- Exit on shadowed environment variables.
- Address error on shadowed environment variables.
- e.g. exit with error on
ETCD_NAME=abc etcd --name=def
. - e.g. exit with error on
ETCD_INITIAL_CLUSTER_TOKEN=abc etcd --initial-cluster-token=def
. - e.g. exit with error on
ETCDCTL_ENDPOINTS=abc.com ETCDCTL_API=3 etcdctl endpoint health --endpoints=def.com
.
- Change
etcdserverpb.AuthRoleRevokePermissionRequest/key,range_end
fields type fromstring
tobytes
. - Change
embed.Config.CorsInfo
in*cors.CORSInfo
type toembed.Config.CORS
inmap[string]struct{}
type. - Remove
embed.Config.SetupLogging
.- Now logger is set up automatically based on
embed.Config.Logger
,embed.Config.LogOutputs
,embed.Config.Debug
fields.
- Now logger is set up automatically based on
- Rename
etcd --log-output
to--log-outputs
to support multiple log outputs.etcd --log-output
will be deprecated in v3.5.
- Rename
embed.Config.LogOutput
toembed.Config.LogOutputs
to support multiple log outputs. - Change
embed.Config.LogOutputs
type fromstring
to[]string
to support multiple log outputs.- Now that
--log-outputs
accepts multiple writers, etcd configuration YAML filelog-outputs
field must be changed to[]string
type. - Previously,
--config-file etcd.config.yaml
can havelog-outputs: default
field, now must belog-outputs: [default]
.
- Now that
- Change v3
etcdctl snapshot
exit codes withsnapshot
package.- Exit on error with exit code 1 (no more exit code 5 or 6 on
snapshot save/restore
commands).
- Exit on error with exit code 1 (no more exit code 5 or 6 on
- Migrate dependency management tool from
glide
togolang/dep
.- <= 3.3 puts
vendor
directory undercmd/vendor
directory to prevent conflicting transitive dependencies. - 3.4 moves
cmd/vendor
directory tovendor
at repository root. - Remove recursive symlinks in
cmd
directory. - Now
go get/install/build
onetcd
packages (e.g.clientv3
,tools/benchmark
) enforce builds with etcdvendor
directory.
- <= 3.3 puts
- Replace gRPC gateway endpoint
/v3beta
with/v3
.- Deprecated
/v3alpha
. - To deprecate
/v3beta
in v3.5. - In v3.4,
curl -L http://localhost:2379/v3beta/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}'
still works as a fallback tocurl -L http://localhost:2379/v3/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}'
, butcurl -L http://localhost:2379/v3beta/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}'
won't work in v3.5. Usecurl -L http://localhost:2379/v3/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}'
instead.
- Deprecated
- Change
wal
package function signatures to support structured logger and logging to file in server-side.- Previously,
Open(dirpath string, snap walpb.Snapshot) (*WAL, error)
, nowOpen(lg *zap.Logger, dirpath string, snap walpb.Snapshot) (*WAL, error)
. - Previously,
OpenForRead(dirpath string, snap walpb.Snapshot) (*WAL, error)
, nowOpenForRead(lg *zap.Logger, dirpath string, snap walpb.Snapshot) (*WAL, error)
. - Previously,
Repair(dirpath string) bool
, nowRepair(lg *zap.Logger, dirpath string) bool
. - Previously,
Create(dirpath string, metadata []byte) (*WAL, error)
, nowCreate(lg *zap.Logger, dirpath string, metadata []byte) (*WAL, error)
.
- Previously,
- Remove
pkg/cors
package. - Move internal package
"github.com/coreos/etcd/snap"
to"github.com/coreos/etcd/raftsnap"
. - Move internal package
"github.com/coreos/etcd/etcdserver/auth"
to"github.com/coreos/etcd/etcdserver/v2auth"
. - Move internal package
"github.com/coreos/etcd/error"
to"github.com/coreos/etcd/etcdserver/v2error"
. - Move internal package
"github.com/coreos/etcd/store"
to"github.com/coreos/etcd/etcdserver/v2store"
. --experimental-enable-v2v3
has been deprecated,--enable-v2v3
flag is now stable.
- Upgrade
google.golang.org/grpc
fromv1.7.5
tov1.11.1
. - Upgrade
github.com/ugorji/go/codec
tov1.1.1
, and regenerate v2client
. - Upgrade
github.com/soheilhy/cmux
fromv0.1.3
tov0.1.4
. - Upgrade
github.com/google/btree
fromgoogle/btree@925471ac9
togoogle/btree@e89373fe6
. - Upgrade
github.com/spf13/cobra
fromspf13/cobra@1c44ec8d3
tospf13/cobra@cd30c2a7e
. - Upgrade
github.com/spf13/pflag
fromv1.0.0
tospf13/pflag@1ce0cc6db
. - Upgrade
github.com/coreos/go-systemd
fromv15
tov16
.
- Add
etcd_server_is_leader
Prometheus metric. - Add
etcd_debugging_mvcc_db_total_size_in_use_in_bytes
Prometheus metric. - Add missing
etcd_network_peer_sent_failures_total
count. - Fix
etcd_debugging_server_lease_expired_total
Prometheus metric. - Fix race conditions in v2 server stat collecting.
See security doc for more details.
- Add
etcd --host-whitelist
flag,etcdserver.Config.HostWhitelist
, andembed.Config.HostWhitelist
, to prevent "DNS Rebinding" attack.- Any website can simply create an authorized DNS name, and direct DNS to
"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). - Client origin enforce policy works as follow:
- If client connection is secure via HTTPS, allow any hostnames..
- If client connection is not secure and
"HostWhitelist"
is not empty, only allow HTTP requests whose Host field is listed in whitelist.
- By default,
"HostWhitelist"
is"*"
, which means insecure server allows all client HTTP requests. - Note that the client origin policy is enforced whether authentication is enabled or not, for tighter controls.
- When specifying hostnames, loopback addresses are not added automatically. To allow loopback interfaces, add them to whitelist manually (e.g.
"localhost"
,"127.0.0.1"
, etc.). - e.g.
etcd --host-whitelist example.com
, then the server will reject all HTTP requests whose Host field is notexample.com
(also rejects requests to"localhost"
).
- Any website can simply create an authorized DNS name, and direct DNS to
- Support
etcd --cors
in v3 HTTP requests (gRPC gateway). - Support TLS cipher suite lists.
- Support
ttl
field foretcd
Authentication JWT token.- e.g.
etcd --auth-token jwt,pub-key=<pub key path>,priv-key=<priv key path>,sign-method=<sign method>,ttl=5m
.
- e.g.
- Allow empty token provider in
etcdserver.ServerConfig.AuthToken
. - Fix TLS reload when certificate SAN field only includes IP addresses but no domain names.
- In Go, server calls
(*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. - However, a certificate whose SAN field does not include any domain names but only IP addresses would request
*tls.ClientHelloInfo
with an emptyServerName
field, thus failing to trigger the TLS reload on initial TLS handshake; this becomes a problem when expired certificates need to be replaced online. - Now,
(*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).
- In Go, server calls
- Add
--initial-election-tick-advance
flag to configure initial election tick fast-forward.- By default,
--initial-election-tick-advance=true
, then local member fast-forwards election ticks to speed up "initial" leader election trigger. - This benefits the case of larger election ticks. For instance, cross datacenter deployment may require longer election timeout of 10-second. If true, local node does not need wait up to 10-second. Instead, forwards its election ticks to 8-second, and have only 2-second left before leader election.
- Major assumptions are that: cluster has no active leader thus advancing ticks enables faster leader election. Or cluster already has an established leader, and rejoining follower is likely to receive heartbeats from the leader after tick advance and before election timeout.
- However, when network from leader to rejoining follower is congested, and the follower does not receive leader heartbeat within left election ticks, disruptive election has to happen thus affecting cluster availabilities.
- Now, this can be disabled by setting
--initial-election-tick-advance=false
. - Disabling this would slow down initial bootstrap process for cross datacenter deployments. Make tradeoffs by configuring
--initial-election-tick-advance
at the cost of slow initial bootstrap. - If single-node, it advances ticks regardless.
- Address disruptive rejoining follower node.
- By default,
- Add
--pre-vote
flag to enable to run an additional Raft election phase.- For instance, a flaky(or rejoining) member may drop in and out, and start campaign. This member will end up with a higher term, and ignore all incoming messages with lower term. In this case, a new leader eventually need to get elected, thus disruptive to cluster availability. Raft implements Pre-Vote phase to prevent this kind of disruptions. If enabled, Raft runs an additional phase of election to check if pre-candidate can get enough votes to win an election.
--pre-vote=false
by default.- v3.5 will enable
--pre-vote=true
by default.
--initial-corrupt-check
flag is now stable (--experimental-initial-corrupt-check
haisbeen deprecated).--initial-corrupt-check=true
by default, to check cluster database hashes before serving client/peer traffic.
--corrupt-check-time
flag is now stable (--experimental-corrupt-check-time
haisbeen deprecated).--corrupt-check-time=12h
by default, to check cluster database hashes for every 12-hour.
--enable-v2v3
flag is now stable.--experimental-enable-v2v3
has been deprecated.- Added more v2v3 integration tests.
--enable-v2=true --enable-v2v3=''
by default, to enable v2 API server that is backed by v2 store.--enable-v2=true --enable-v2v3=/aaa
to enable v2 API server that is backed by v3 storage.--enable-v2=false --enable-v2v3=''
to disable v2 API server.--enable-v2=false --enable-v2v3=/aaa
to disable v2 API server. TODO: error?- Automatically create parent directory if it does not exist (fix issue#9609).
- v4.0 will configure
--enable-v2=true --enable-v2v3=/aaa
to enable v2 API server that is backed by v3 storage.
- Add
--discovery-srv-name
flag to support custom DNS SRV name with discovery.- If not given, etcd queries
_etcd-server-ssl._tcp.[YOUR_HOST]
and_etcd-server._tcp.[YOUR_HOST]
. - If
--discovery-srv-name="foo"
, then query_etcd-server-ssl-foo._tcp.[YOUR_HOST]
and_etcd-server-foo._tcp.[YOUR_HOST]
. - Useful for operating multiple etcd clusters under the same domain.
- If not given, etcd queries
- Support
etcd --cors
in v3 HTTP requests (gRPC gateway). - Rename
etcd --log-output
to--log-outputs
to support multiple log outputs.etcd --log-output
will be deprecated in v3.5.
- Add
--logger
flag to support structured logger and multiple log outputs in server-side.etcd --logger=capnslog
will be deprecated in v3.5.- Main motivation is to promote automated etcd monitoring, rather than looking back server logs when it starts breaking. Future development will make etcd log as few as possible, and make etcd easier to monitor with metrics and alerts.
- e.g.
--logger=capnslog --log-outputs=default
is the default setting and same as previous etcd server logging format. - e.g.
--logger=zap --log-outputs=default
will log server operations in JSON-encoded format and writes logs toos.Stderr
. - e.g. If etcd parent process ID (
ppid
) is 1 (e.g. run with systemd),--logger=zap --log-outputs=default
will redirect server logs to local systemd journal in JSON-encoded format. And if write to journald fails, it writes toos.Stderr
as a fallback. - e.g.
--logger=zap --log-outputs=stderr
will log server operations in JSON-encoded format and writes logs toos.Stderr
. Use this to override journald log redirects. - e.g.
--logger=zap --log-outputs=stdout
will log server operations in JSON-encoded format and writes logs toos.Stdout
Use this to override journald log redirects. - e.g.
--logger=zap --log-outputs=a.log
will log server operations in JSON-encoded format and writes logs to the specified filea.log
. - e.g.
--logger=zap --log-outputs=a.log,b.log,c.log,stdout
writes server logs to multiple filesa.log
,b.log
andc.log
at the same time and outputs tostdout
, in JSON-encoded format. - e.g.
--logger=zap --log-outputs=/dev/null
will discard all server logs.
- Add
embed.Config.InitialElectionTickAdvance
to enable/disable initial election tick fast-forward.embed.NewConfig()
would return*embed.Config
withInitialElectionTickAdvance
as true by default.
- Define
embed.CompactorModePeriodic
forcompactor.ModePeriodic
. - Define
embed.CompactorModeRevision
forcompactor.ModeRevision
. - Change
embed.Config.CorsInfo
in*cors.CORSInfo
type toembed.Config.CORS
inmap[string]struct{}
type. - Remove
embed.Config.SetupLogging
.- Now logger is set up automatically based on
embed.Config.Logger
,embed.Config.LogOutputs
,embed.Config.Debug
fields.
- Now logger is set up automatically based on
- Add
embed.Config.Logger
to support structured loggerzap
in server-side. - Rename
embed.Config.LogOutput
toembed.Config.LogOutputs
to support multiple log outputs. - Change
embed.Config.LogOutputs
type fromstring
to[]string
to support multiple log outputs.
- Add
snapshot
package for snapshot restore/save operations (seegodoc.org/github.com/etcd/snapshot
for more). - Add
watch_id
field toetcdserverpb.WatchCreateRequest
, allow user-provided watch ID tomvcc
.- Corresponding
watch_id
is returned viaetcdserverpb.WatchResponse
, if any.
- Corresponding
- Add
raftAppliedIndex
field toetcdserverpb.StatusResponse
for current Raft applied index. - Add
errors
field toetcdserverpb.StatusResponse
for server-side error.- e.g.
"etcdserver: no leader", "NOSPACE", "CORRUPT"
- e.g.
- Add
dbSizeInUse
field toetcdserverpb.StatusResponse
for actual DB size after compaction.
- Add
check datascale
command. - Add
check datascale --auto-compact, --auto-defrag
flags. - Add
check perf --auto-compact, --auto-defrag
flags. - Add
defrag --cluster
flag. - Add "raft applied index" field to
endpoint status
. - Add "errors" field to
endpoint status
. - Add
endpoint health --write-out
support.- Previously,
endpoint health --write-out json
did not work.
- Previously,
- Replace gRPC gateway endpoint
/v3beta
with/v3
.- Deprecated
/v3alpha
. - To deprecate
/v3beta
in v3.5. - In v3.4,
curl -L http://localhost:2379/v3beta/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}'
still works as a fallback tocurl -L http://localhost:2379/v3/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}'
, butcurl -L http://localhost:2379/v3beta/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}'
won't work in v3.5. Usecurl -L http://localhost:2379/v3/kv/put -X POST -d '{"key": "Zm9v", "value": "YmFy"}'
instead.
- Deprecated
- Add API endpoints
/{v3beta,v3}/lease/leases, /{v3beta,v3}/lease/revoke, /{v3beta,v3}/lease/timetolive
. - Support
etcd --cors
in v3 HTTP requests (gRPC gateway).
- Fix deadlock during PreVote migration process.
- Add
raft.ErrProposalDropped
.- Now
(r *raft) Step
returnsraft.ErrProposalDropped
if a proposal has been ignored. - e.g. a node is removed from cluster, or
raftpb.MsgProp
arrives at current leader while there is an ongoing leadership transfer.
- Now
- Improve Raft
becomeLeader
andstepLeader
by keeping track of latestpb.EntryConfChange
index.- Previously record
pendingConf
boolean field scanning the entire tail of the log, which can delay hearbeat send.
- Previously record
- Fix missing learner nodes on
(n *node) ApplyConfChange
.
- Fix
mvcc
"unsynced" watcher restore operation.- "unsynced" watcher is watcher that needs to be in sync with events that have happened.
- That is, "unsynced" watcher is the slow watcher that was requested on old revision.
- "unsynced" watcher restore operation was not correctly populating its underlying watcher group.
- Which possibly causes missing events from "unsynced" watchers.
- Fix server panic on invalid Election Proclaim/Resign HTTP(S) requests.
- Previously, wrong-formatted HTTP requests to Election API could trigger panic in etcd server.
- e.g.
curl -L http://localhost:2379/v3/election/proclaim -X POST -d '{"value":""}'
,curl -L http://localhost:2379/v3/election/resign -X POST -d '{"value":""}'
.
- Fix revision-based compaction retention parsing.
- Previously,
etcd --auto-compaction-mode revision --auto-compaction-retention 1
was translated to revision retention 3600000000000. - Now,
etcd --auto-compaction-mode revision --auto-compaction-retention 1
is correctly parsed as revision retention 1.
- Previously,
- Prevent overflow by large
TTL
values forLease
Grant
.TTL
parameter toGrant
request is unit of second.- Leases with too large
TTL
values exceedingmath.MaxInt64
expire in unexpected ways. - Server now returns
rpctypes.ErrLeaseTTLTooLarge
to client, when the requestedTTL
is larger than 9,000,000,000 seconds (which is >285 years). - Again, etcd
Lease
is meant for short-periodic keepalives or sessions, in the range of seconds or minutes. Not for hours or days!
- Enable etcd server
raft.Config.CheckQuorum
when starting withForceNewCluster
.
- Require Go 1.10+.
- Compile with Go 1.10.1.
- Add
etcd-dump-logs -entry-type
flag to support WAL log filtering by entry type.