Browse Source

Merge pull request #9372 from gyuho/origin

*: mitigate DNS rebinding attacks in insecure etcd server
Gyuho Lee 7 years ago
parent
commit
a7e5790c82

+ 9 - 0
CHANGELOG-3.4.md

@@ -52,6 +52,15 @@ See [code changes](https://github.com/coreos/etcd/compare/v3.3.0...v3.4.0) and [
   - If not given, etcd queries `_etcd-server-ssl._tcp.[YOUR_HOST]` and `_etcd-server._tcp.[YOUR_HOST]`.
   - 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]`.
   - 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.
   - Useful for operating multiple etcd clusters under the same domain.
+- Add [`--host-whitelist`](https://github.com/coreos/etcd/pull/9372) flag, [`etcdserver.Config.HostWhitelist`](https://github.com/coreos/etcd/pull/9372), and [`embed.Config.HostWhitelist`](https://github.com/coreos/etcd/pull/9372), to prevent ["DNS Rebinding"](https://en.wikipedia.org/wiki/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)](https://bugs.chromium.org/p/project-zero/issues/detail?id=1447#c2).
+  - 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 empty, 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 not `example.com` (also rejects requests to `"localhost"`).
 - Define `embed.CompactorModePeriodic` for `compactor.ModePeriodic`.
 - Define `embed.CompactorModePeriodic` for `compactor.ModePeriodic`.
 - Define `embed.CompactorModeRevision` for `compactor.ModeRevision`.
 - Define `embed.CompactorModeRevision` for `compactor.ModeRevision`.
 
 

+ 13 - 0
Documentation/op-guide/security.md

@@ -321,6 +321,19 @@ I | embed: serving client requests on 127.0.0.1:22379
 I | embed: serving client requests on 127.0.0.1:2379
 I | embed: serving client requests on 127.0.0.1:2379
 ```
 ```
 
 
+## Notes for Host Whitelist
+
+`etcd --host-whitelist` flag specifies acceptable hostnames from HTTP client requests. Client origin policy protects against ["DNS Rebinding"](https://en.wikipedia.org/wiki/DNS_rebinding) attacks to insecure etcd servers. That is, 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. See [CVE-2018-5702](https://bugs.chromium.org/p/project-zero/issues/detail?id=1447#c2) for more detail.
+
+Client origin policy works as follows:
+
+1. If client connection is secure via HTTPS, allow any hostnames.
+2. If client connection is not secure and `"HostWhitelist"` is not empty, only allow HTTP requests whose Host field is listed in whitelist.
+
+Note that the client origin policy is enforced whether authentication is enabled or not, for tighter controls.
+
+By default, `etcd --host-whitelist` and `embed.Config.HostWhitelist` are set *empty* to allow all hostnames. Note that 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.).
+
 ## Frequently asked questions
 ## Frequently asked questions
 
 
 ### I'm seeing a SSLv3 alert handshake failure when using TLS client authentication?
 ### I'm seeing a SSLv3 alert handshake failure when using TLS client authentication?

+ 30 - 2
embed/config.go

@@ -79,8 +79,9 @@ var (
 	DefaultInitialAdvertisePeerURLs = "http://localhost:2380"
 	DefaultInitialAdvertisePeerURLs = "http://localhost:2380"
 	DefaultAdvertiseClientURLs      = "http://localhost:2379"
 	DefaultAdvertiseClientURLs      = "http://localhost:2379"
 
 
-	defaultHostname   string
-	defaultHostStatus error
+	defaultHostname      string
+	defaultHostStatus    error
+	defaultHostWhitelist = []string{} // if empty, allow all
 )
 )
 
 
 var (
 var (
@@ -171,6 +172,32 @@ type Config struct {
 	PeerTLSInfo   transport.TLSInfo
 	PeerTLSInfo   transport.TLSInfo
 	PeerAutoTLS   bool
 	PeerAutoTLS   bool
 
 
+	// HostWhitelist lists acceptable hostnames from HTTP client requests.
+	// Client origin policy protects against "DNS Rebinding" attacks
+	// to insecure etcd servers. That is, 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. See "CVE-2018-5702" for more detail.
+	//
+	// 1. If client connection is secure via HTTPS, allow any hostnames.
+	// 2. If client connection is not secure and "HostWhitelist" is not empty,
+	//    only allow HTTP requests whose Host field is listed in whitelist.
+	//
+	// Note that the client origin policy is enforced whether authentication
+	// is enabled or not, for tighter controls.
+	//
+	// By default, "HostWhitelist" is empty, which allows any hostnames.
+	// Note that when specifying hostnames, loopback addresses are not added
+	// automatically. To allow loopback interfaces, leave it empty or add them
+	// to whitelist manually (e.g. "localhost", "127.0.0.1", etc.).
+	//
+	// CVE-2018-5702 reference:
+	// - https://bugs.chromium.org/p/project-zero/issues/detail?id=1447#c2
+	// - https://github.com/transmission/transmission/pull/468
+	// - https://github.com/coreos/etcd/issues/9353
+	HostWhitelist []string `json:"host-whitelist"`
+
 	// debug
 	// debug
 
 
 	Debug                 bool   `json:"debug"`
 	Debug                 bool   `json:"debug"`
@@ -264,6 +291,7 @@ func NewConfig() *Config {
 		LogOutput:             DefaultLogOutput,
 		LogOutput:             DefaultLogOutput,
 		Metrics:               "basic",
 		Metrics:               "basic",
 		EnableV2:              DefaultEnableV2,
 		EnableV2:              DefaultEnableV2,
+		HostWhitelist:         defaultHostWhitelist,
 		AuthToken:             "simple",
 		AuthToken:             "simple",
 	}
 	}
 	cfg.InitialCluster = cfg.InitialClusterFromName(cfg.Name)
 	cfg.InitialCluster = cfg.InitialClusterFromName(cfg.Name)

+ 8 - 0
embed/etcd.go

@@ -174,9 +174,17 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
 		Debug:                   cfg.Debug,
 		Debug:                   cfg.Debug,
 	}
 	}
 
 
+	srvcfg.HostWhitelist = make(map[string]struct{}, len(cfg.HostWhitelist))
+	for _, h := range cfg.HostWhitelist {
+		if h != "" {
+			srvcfg.HostWhitelist[h] = struct{}{}
+		}
+	}
+
 	if e.Server, err = etcdserver.NewServer(srvcfg); err != nil {
 	if e.Server, err = etcdserver.NewServer(srvcfg); err != nil {
 		return e, err
 		return e, err
 	}
 	}
+	plog.Infof("%s starting with host whitelist %q", e.Server.ID(), cfg.HostWhitelist)
 
 
 	// buffer channel so goroutines on closed connections won't wait forever
 	// buffer channel so goroutines on closed connections won't wait forever
 	e.errc = make(chan error, len(e.Peers)+len(e.Clients)+2*len(e.sctxs))
 	e.errc = make(chan error, len(e.Peers)+len(e.Clients)+2*len(e.sctxs))

+ 33 - 4
embed/serve.go

@@ -16,6 +16,7 @@ package embed
 
 
 import (
 import (
 	"context"
 	"context"
+	"fmt"
 	"io/ioutil"
 	"io/ioutil"
 	defaultLog "log"
 	defaultLog "log"
 	"net"
 	"net"
@@ -33,6 +34,7 @@ import (
 	"github.com/coreos/etcd/etcdserver/api/v3rpc"
 	"github.com/coreos/etcd/etcdserver/api/v3rpc"
 	etcdservergw "github.com/coreos/etcd/etcdserver/etcdserverpb/gw"
 	etcdservergw "github.com/coreos/etcd/etcdserver/etcdserverpb/gw"
 	"github.com/coreos/etcd/pkg/debugutil"
 	"github.com/coreos/etcd/pkg/debugutil"
+	"github.com/coreos/etcd/pkg/httputil"
 	"github.com/coreos/etcd/pkg/transport"
 	"github.com/coreos/etcd/pkg/transport"
 
 
 	gw "github.com/grpc-ecosystem/grpc-gateway/runtime"
 	gw "github.com/grpc-ecosystem/grpc-gateway/runtime"
@@ -114,7 +116,7 @@ func (sctx *serveCtx) serve(
 		httpmux := sctx.createMux(gwmux, handler)
 		httpmux := sctx.createMux(gwmux, handler)
 
 
 		srvhttp := &http.Server{
 		srvhttp := &http.Server{
-			Handler:  wrapMux(httpmux),
+			Handler:  wrapMux(s, httpmux),
 			ErrorLog: logger, // do not log user error
 			ErrorLog: logger, // do not log user error
 		}
 		}
 		httpl := m.Match(cmux.HTTP1())
 		httpl := m.Match(cmux.HTTP1())
@@ -157,7 +159,7 @@ func (sctx *serveCtx) serve(
 		httpmux := sctx.createMux(gwmux, handler)
 		httpmux := sctx.createMux(gwmux, handler)
 
 
 		srv := &http.Server{
 		srv := &http.Server{
-			Handler:   wrapMux(httpmux),
+			Handler:   wrapMux(s, httpmux),
 			TLSConfig: tlscfg,
 			TLSConfig: tlscfg,
 			ErrorLog:  logger, // do not log user error
 			ErrorLog:  logger, // do not log user error
 		}
 		}
@@ -252,11 +254,12 @@ func (sctx *serveCtx) createMux(gwmux *gw.ServeMux, handler http.Handler) *http.
 // - mutate gRPC gateway request paths
 // - mutate gRPC gateway request paths
 // - check hostname whitelist
 // - check hostname whitelist
 // client HTTP requests goes here first
 // client HTTP requests goes here first
-func wrapMux(mux *http.ServeMux) http.Handler {
-	return &httpWrapper{mux: mux}
+func wrapMux(s *etcdserver.EtcdServer, mux *http.ServeMux) http.Handler {
+	return &httpWrapper{s: s, mux: mux}
 }
 }
 
 
 type httpWrapper struct {
 type httpWrapper struct {
+	s   *etcdserver.EtcdServer
 	mux *http.ServeMux
 	mux *http.ServeMux
 }
 }
 
 
@@ -265,9 +268,35 @@ func (m *httpWrapper) ServeHTTP(rw http.ResponseWriter, req *http.Request) {
 	if req != nil && req.URL != nil && strings.HasPrefix(req.URL.Path, "/v3beta/") {
 	if req != nil && req.URL != nil && strings.HasPrefix(req.URL.Path, "/v3beta/") {
 		req.URL.Path = strings.Replace(req.URL.Path, "/v3beta/", "/v3/", 1)
 		req.URL.Path = strings.Replace(req.URL.Path, "/v3beta/", "/v3/", 1)
 	}
 	}
+
+	if req.TLS == nil { // check origin if client connection is not secure
+		host := httputil.GetHostname(req)
+		if !m.s.IsHostWhitelisted(host) {
+			plog.Warningf("rejecting HTTP request from %q to prevent DNS rebinding attacks", host)
+			// TODO: use Go's "http.StatusMisdirectedRequest" (421)
+			// https://github.com/golang/go/commit/4b8a7eafef039af1834ef9bfa879257c4a72b7b5
+			http.Error(rw, errCVE20185702(host), 421)
+			return
+		}
+	}
+
 	m.mux.ServeHTTP(rw, req)
 	m.mux.ServeHTTP(rw, req)
 }
 }
 
 
+// https://github.com/transmission/transmission/pull/468
+func errCVE20185702(host string) string {
+	return fmt.Sprintf(`
+etcd received your request, but the Host header was unrecognized.
+
+To fix this, choose one of the following options:
+- Enable TLS, then any HTTPS request will be allowed.
+- Add the hostname you want to use to the whitelist in settings.
+  - e.g. etcd --host-whitelist %q
+
+This requirement has been added to help prevent "DNS Rebinding" attacks (CVE-2018-5702).
+`, host)
+}
+
 func (sctx *serveCtx) registerUserHandler(s string, h http.Handler) {
 func (sctx *serveCtx) registerUserHandler(s string, h http.Handler) {
 	if sctx.userHandlers[s] != nil {
 	if sctx.userHandlers[s] != nil {
 		plog.Warningf("path %s already registered by user handler", s)
 		plog.Warningf("path %s already registered by user handler", s)

+ 15 - 4
etcdmain/config.go

@@ -85,10 +85,11 @@ type config struct {
 
 
 // configFlags has the set of flags used for command line parsing a Config
 // configFlags has the set of flags used for command line parsing a Config
 type configFlags struct {
 type configFlags struct {
-	flagSet      *flag.FlagSet
-	clusterState *flags.StringsFlag
-	fallback     *flags.StringsFlag
-	proxy        *flags.StringsFlag
+	flagSet       *flag.FlagSet
+	hostWhitelist string
+	clusterState  *flags.StringsFlag
+	fallback      *flags.StringsFlag
+	proxy         *flags.StringsFlag
 }
 }
 
 
 func newConfig() *config {
 func newConfig() *config {
@@ -189,6 +190,7 @@ func newConfig() *config {
 	fs.BoolVar(&cfg.ec.PeerAutoTLS, "peer-auto-tls", false, "Peer TLS using generated certificates")
 	fs.BoolVar(&cfg.ec.PeerAutoTLS, "peer-auto-tls", false, "Peer TLS using generated certificates")
 	fs.StringVar(&cfg.ec.PeerTLSInfo.CRLFile, "peer-crl-file", "", "Path to the peer certificate revocation list file.")
 	fs.StringVar(&cfg.ec.PeerTLSInfo.CRLFile, "peer-crl-file", "", "Path to the peer certificate revocation list file.")
 	fs.StringVar(&cfg.ec.PeerTLSInfo.AllowedCN, "peer-cert-allowed-cn", "", "Allowed CN for inter peer authentication.")
 	fs.StringVar(&cfg.ec.PeerTLSInfo.AllowedCN, "peer-cert-allowed-cn", "", "Allowed CN for inter peer authentication.")
+	fs.StringVar(&cfg.cf.hostWhitelist, "host-whitelist", "", "Comma-separated acceptable hostnames from HTTP client requests, if server is not secure (empty means allow all).")
 
 
 	// logging
 	// logging
 	fs.BoolVar(&cfg.ec.Debug, "debug", false, "Enable debug-level logging for etcd.")
 	fs.BoolVar(&cfg.ec.Debug, "debug", false, "Enable debug-level logging for etcd.")
@@ -275,6 +277,15 @@ func (cfg *config) configFromCmdLine() error {
 		cfg.ec.ListenMetricsUrls = []url.URL(u)
 		cfg.ec.ListenMetricsUrls = []url.URL(u)
 	}
 	}
 
 
+	hosts := []string{}
+	for _, h := range strings.Split(cfg.cf.hostWhitelist, ",") {
+		h = strings.TrimSpace(h)
+		if h != "" {
+			hosts = append(hosts, h)
+		}
+	}
+	cfg.ec.HostWhitelist = hosts
+
 	cfg.ec.ClusterState = cfg.cf.clusterState.String()
 	cfg.ec.ClusterState = cfg.cf.clusterState.String()
 	cfg.cp.Fallback = cfg.cf.fallback.String()
 	cfg.cp.Fallback = cfg.cf.fallback.String()
 	cfg.cp.Proxy = cfg.cf.proxy.String()
 	cfg.cp.Proxy = cfg.cf.proxy.String()

+ 2 - 0
etcdmain/help.go

@@ -158,6 +158,8 @@ security flags:
 		peer TLS using self-generated certificates if --peer-key-file and --peer-cert-file are not provided.
 		peer TLS using self-generated certificates if --peer-key-file and --peer-cert-file are not provided.
 	--peer-crl-file ''
 	--peer-crl-file ''
 		path to the peer certificate revocation list file.
 		path to the peer certificate revocation list file.
+	--host-whitelist ''
+		acceptable hostnames from HTTP client requests, if server is not secure (empty means allow all).
 
 
 logging flags
 logging flags
 
 

+ 5 - 0
etcdserver/config.go

@@ -47,6 +47,11 @@ type ServerConfig struct {
 	ForceNewCluster     bool
 	ForceNewCluster     bool
 	PeerTLSInfo         transport.TLSInfo
 	PeerTLSInfo         transport.TLSInfo
 
 
+	// HostWhitelist lists acceptable hostnames from client requests.
+	// If server is insecure (no TLS), server only accepts requests
+	// whose Host header value exists in this white list.
+	HostWhitelist map[string]struct{}
+
 	TickMs           uint
 	TickMs           uint
 	ElectionTicks    int
 	ElectionTicks    int
 	BootstrapTimeout time.Duration
 	BootstrapTimeout time.Duration

+ 13 - 0
etcdserver/server.go

@@ -251,6 +251,8 @@ type EtcdServer struct {
 
 
 	leadTimeMu      sync.RWMutex
 	leadTimeMu      sync.RWMutex
 	leadElectedTime time.Time
 	leadElectedTime time.Time
+
+	hostWhitelist map[string]struct{}
 }
 }
 
 
 // NewServer creates a new EtcdServer from the supplied configuration. The
 // NewServer creates a new EtcdServer from the supplied configuration. The
@@ -434,6 +436,7 @@ func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) {
 		peerRt:        prt,
 		peerRt:        prt,
 		reqIDGen:      idutil.NewGenerator(uint16(id), time.Now()),
 		reqIDGen:      idutil.NewGenerator(uint16(id), time.Now()),
 		forceVersionC: make(chan struct{}),
 		forceVersionC: make(chan struct{}),
+		hostWhitelist: cfg.HostWhitelist,
 	}
 	}
 
 
 	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
 	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
@@ -626,6 +629,16 @@ func (s *EtcdServer) ReportSnapshot(id uint64, status raft.SnapshotStatus) {
 	s.r.ReportSnapshot(id, status)
 	s.r.ReportSnapshot(id, status)
 }
 }
 
 
+// IsHostWhitelisted returns true if the host is whitelisted.
+// If whitelist is empty, allow all.
+func (s *EtcdServer) IsHostWhitelisted(host string) bool {
+	if len(s.hostWhitelist) == 0 { // allow all
+		return true
+	}
+	_, ok := s.hostWhitelist[host]
+	return ok
+}
+
 type etcdProgress struct {
 type etcdProgress struct {
 	confState raftpb.ConfState
 	confState raftpb.ConfState
 	snapi     uint64
 	snapi     uint64

+ 0 - 1
hack/scripts-dev/docker-dns/certs/run.sh

@@ -32,7 +32,6 @@ ETCDCTL_API=3 ./etcdctl \
   --endpoints=https://m1.etcd.local:2379,https://m2.etcd.local:22379,https://m3.etcd.local:32379 \
   --endpoints=https://m1.etcd.local:2379,https://m2.etcd.local:22379,https://m3.etcd.local:32379 \
   get abc
   get abc
 
 
-# TODO: add host header check to enforce same-origin-policy
 printf "\nWriting v2 key...\n"
 printf "\nWriting v2 key...\n"
 curl -L https://127.0.0.1:2379/v2/keys/queue \
 curl -L https://127.0.0.1:2379/v2/keys/queue \
   --cacert /certs/ca.crt \
   --cacert /certs/ca.crt \

+ 3 - 3
hack/scripts-dev/docker-dns/insecure/Procfile

@@ -1,6 +1,6 @@
 # Use goreman to run `go get github.com/mattn/goreman`
 # Use goreman to run `go get github.com/mattn/goreman`
-etcd1: ./etcd --name m1 --data-dir /tmp/m1.data --listen-client-urls http://127.0.0.1:2379 --advertise-client-urls http://m1.etcd.local:2379 --listen-peer-urls http://127.0.0.1:2380 --initial-advertise-peer-urls=http://m1.etcd.local:2380 --initial-cluster-token tkn --initial-cluster=m1=http://m1.etcd.local:2380,m2=http://m2.etcd.local:22380,m3=http://m3.etcd.local:32380
+etcd1: ./etcd --name m1 --data-dir /tmp/m1.data --listen-client-urls http://127.0.0.1:2379 --advertise-client-urls http://m1.etcd.local:2379 --listen-peer-urls http://127.0.0.1:2380 --initial-advertise-peer-urls=http://m1.etcd.local:2380 --initial-cluster-token tkn --initial-cluster=m1=http://m1.etcd.local:2380,m2=http://m2.etcd.local:22380,m3=http://m3.etcd.local:32380 --host-whitelist "localhost,127.0.0.1,m1.etcd.local"
 
 
-etcd2: ./etcd --name m2 --data-dir /tmp/m2.data --listen-client-urls http://127.0.0.1:22379 --advertise-client-urls http://m2.etcd.local:22379 --listen-peer-urls http://127.0.0.1:22380 --initial-advertise-peer-urls=http://m2.etcd.local:22380 --initial-cluster-token tkn --initial-cluster=m1=http://m1.etcd.local:2380,m2=http://m2.etcd.local:22380,m3=http://m3.etcd.local:32380
+etcd2: ./etcd --name m2 --data-dir /tmp/m2.data --listen-client-urls http://127.0.0.1:22379 --advertise-client-urls http://m2.etcd.local:22379 --listen-peer-urls http://127.0.0.1:22380 --initial-advertise-peer-urls=http://m2.etcd.local:22380 --initial-cluster-token tkn --initial-cluster=m1=http://m1.etcd.local:2380,m2=http://m2.etcd.local:22380,m3=http://m3.etcd.local:32380 --host-whitelist "localhost,127.0.0.1,m1.etcd.local"
 
 
-etcd3: ./etcd --name m3 --data-dir /tmp/m3.data --listen-client-urls http://127.0.0.1:32379 --advertise-client-urls http://m3.etcd.local:32379 --listen-peer-urls http://127.0.0.1:32380 --initial-advertise-peer-urls=http://m3.etcd.local:32380 --initial-cluster-token tkn --initial-cluster=m1=http://m1.etcd.local:2380,m2=http://m2.etcd.local:22380,m3=http://m3.etcd.local:32380
+etcd3: ./etcd --name m3 --data-dir /tmp/m3.data --listen-client-urls http://127.0.0.1:32379 --advertise-client-urls http://m3.etcd.local:32379 --listen-peer-urls http://127.0.0.1:32380 --initial-advertise-peer-urls=http://m3.etcd.local:32380 --initial-cluster-token tkn --initial-cluster=m1=http://m1.etcd.local:2380,m2=http://m2.etcd.local:22380,m3=http://m3.etcd.local:32380 --host-whitelist "localhost,127.0.0.1,m1.etcd.local"

+ 0 - 1
hack/scripts-dev/docker-dns/insecure/run.sh

@@ -23,7 +23,6 @@ ETCDCTL_API=3 ./etcdctl \
   --endpoints=http://m1.etcd.local:2379,http://m2.etcd.local:22379,http://m3.etcd.local:32379 \
   --endpoints=http://m1.etcd.local:2379,http://m2.etcd.local:22379,http://m3.etcd.local:32379 \
   get abc
   get abc
 
 
-# TODO: add host header check to enforce same-origin-policy
 printf "\nWriting v2 key...\n"
 printf "\nWriting v2 key...\n"
 curl \
 curl \
   -L http://127.0.0.1:2379/v2/keys/queue \
   -L http://127.0.0.1:2379/v2/keys/queue \

+ 30 - 2
pkg/httputil/httputil.go

@@ -1,22 +1,50 @@
+// Copyright 2018 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.
+
 // Copyright 2015 The Go Authors. All rights reserved.
 // Copyright 2015 The Go Authors. All rights reserved.
 // Use of this source code is governed by a BSD-style
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 // license that can be found in the LICENSE file.
 
 
-// borrowed from golang/net/context/ctxhttp/cancelreq.go
-
 // Package httputil provides HTTP utility functions.
 // Package httputil provides HTTP utility functions.
 package httputil
 package httputil
 
 
 import (
 import (
 	"io"
 	"io"
 	"io/ioutil"
 	"io/ioutil"
+	"net"
 	"net/http"
 	"net/http"
 )
 )
 
 
 // GracefulClose drains http.Response.Body until it hits EOF
 // GracefulClose drains http.Response.Body until it hits EOF
 // and closes it. This prevents TCP/TLS connections from closing,
 // and closes it. This prevents TCP/TLS connections from closing,
 // therefore available for reuse.
 // therefore available for reuse.
+// Borrowed from golang/net/context/ctxhttp/cancelreq.go.
 func GracefulClose(resp *http.Response) {
 func GracefulClose(resp *http.Response) {
 	io.Copy(ioutil.Discard, resp.Body)
 	io.Copy(ioutil.Discard, resp.Body)
 	resp.Body.Close()
 	resp.Body.Close()
 }
 }
+
+// GetHostname returns the hostname from request Host field.
+// It returns empty string, if Host field contains invalid
+// value (e.g. "localhost:::" with too many colons).
+func GetHostname(req *http.Request) string {
+	if req == nil {
+		return ""
+	}
+	h, _, err := net.SplitHostPort(req.Host)
+	if err != nil {
+		return req.Host
+	}
+	return h
+}

+ 50 - 0
pkg/httputil/httputil_test.go

@@ -0,0 +1,50 @@
+// Copyright 2018 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 httputil
+
+import (
+	"net/http"
+	"testing"
+)
+
+func TestGetHostname(t *testing.T) {
+	tt := []struct {
+		req  *http.Request
+		host string
+	}{
+		{&http.Request{Host: "localhost"}, "localhost"},
+		{&http.Request{Host: "localhost:2379"}, "localhost"},
+		{&http.Request{Host: "localhost."}, "localhost."},
+		{&http.Request{Host: "localhost.:2379"}, "localhost."},
+		{&http.Request{Host: "127.0.0.1"}, "127.0.0.1"},
+		{&http.Request{Host: "127.0.0.1:2379"}, "127.0.0.1"},
+
+		{&http.Request{Host: "localhos"}, "localhos"},
+		{&http.Request{Host: "localhos:2379"}, "localhos"},
+		{&http.Request{Host: "localhos."}, "localhos."},
+		{&http.Request{Host: "localhos.:2379"}, "localhos."},
+		{&http.Request{Host: "1.2.3.4"}, "1.2.3.4"},
+		{&http.Request{Host: "1.2.3.4:2379"}, "1.2.3.4"},
+
+		// too many colons in address
+		{&http.Request{Host: "localhost:::::"}, "localhost:::::"},
+	}
+	for i := range tt {
+		hv := GetHostname(tt[i].req)
+		if hv != tt[i].host {
+			t.Errorf("#%d: %q expected host %q, got '%v'", i, tt[i].req.Host, tt[i].host, hv)
+		}
+	}
+}