serve.go 9.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332
  1. // Copyright 2015 The etcd Authors
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package embed
  15. import (
  16. "context"
  17. "io/ioutil"
  18. defaultLog "log"
  19. "net"
  20. "net/http"
  21. "strings"
  22. "time"
  23. "github.com/coreos/etcd/etcdserver"
  24. "github.com/coreos/etcd/etcdserver/api/v3client"
  25. "github.com/coreos/etcd/etcdserver/api/v3election"
  26. "github.com/coreos/etcd/etcdserver/api/v3election/v3electionpb"
  27. v3electiongw "github.com/coreos/etcd/etcdserver/api/v3election/v3electionpb/gw"
  28. "github.com/coreos/etcd/etcdserver/api/v3lock"
  29. "github.com/coreos/etcd/etcdserver/api/v3lock/v3lockpb"
  30. v3lockgw "github.com/coreos/etcd/etcdserver/api/v3lock/v3lockpb/gw"
  31. "github.com/coreos/etcd/etcdserver/api/v3rpc"
  32. etcdservergw "github.com/coreos/etcd/etcdserver/etcdserverpb/gw"
  33. "github.com/coreos/etcd/pkg/debugutil"
  34. "github.com/coreos/etcd/pkg/transport"
  35. gw "github.com/grpc-ecosystem/grpc-gateway/runtime"
  36. "github.com/soheilhy/cmux"
  37. "github.com/tmc/grpc-websocket-proxy/wsproxy"
  38. "golang.org/x/net/trace"
  39. "google.golang.org/grpc"
  40. "google.golang.org/grpc/credentials"
  41. )
  42. type serveCtx struct {
  43. l net.Listener
  44. addr string
  45. secure bool
  46. insecure bool
  47. ctx context.Context
  48. cancel context.CancelFunc
  49. userHandlers map[string]http.Handler
  50. serviceRegister func(*grpc.Server)
  51. secureHTTPServer *http.Server
  52. secureGrpcServerC chan *grpc.Server
  53. insecureGrpcServerC chan *grpc.Server
  54. }
  55. func newServeCtx() *serveCtx {
  56. ctx, cancel := context.WithCancel(context.Background())
  57. return &serveCtx{
  58. ctx: ctx,
  59. cancel: cancel,
  60. userHandlers: make(map[string]http.Handler),
  61. secureGrpcServerC: make(chan *grpc.Server, 1),
  62. insecureGrpcServerC: make(chan *grpc.Server, 1),
  63. }
  64. }
  65. // serve accepts incoming connections on the listener l,
  66. // creating a new service goroutine for each. The service goroutines
  67. // read requests and then call handler to reply to them.
  68. func (sctx *serveCtx) serve(
  69. s *etcdserver.EtcdServer,
  70. tlsinfo *transport.TLSInfo,
  71. handler http.Handler,
  72. errHandler func(error),
  73. gopts ...grpc.ServerOption) error {
  74. logger := defaultLog.New(ioutil.Discard, "etcdhttp", 0)
  75. <-s.ReadyNotify()
  76. plog.Info("ready to serve client requests")
  77. m := cmux.New(sctx.l)
  78. v3c := v3client.New(s)
  79. servElection := v3election.NewElectionServer(v3c)
  80. servLock := v3lock.NewLockServer(v3c)
  81. if sctx.insecure {
  82. gs := v3rpc.Server(s, nil, gopts...)
  83. sctx.insecureGrpcServerC <- gs
  84. v3electionpb.RegisterElectionServer(gs, servElection)
  85. v3lockpb.RegisterLockServer(gs, servLock)
  86. if sctx.serviceRegister != nil {
  87. sctx.serviceRegister(gs)
  88. }
  89. grpcl := m.Match(cmux.HTTP2())
  90. go func() { errHandler(gs.Serve(grpcl)) }()
  91. opts := []grpc.DialOption{
  92. grpc.WithInsecure(),
  93. }
  94. gwmux, err := sctx.registerGateway(opts)
  95. if err != nil {
  96. return err
  97. }
  98. httpmux := sctx.createMux(gwmux, handler)
  99. srvhttp := &http.Server{
  100. Handler: wrapMux(httpmux),
  101. ErrorLog: logger, // do not log user error
  102. }
  103. httpl := m.Match(cmux.HTTP1())
  104. go func() { errHandler(srvhttp.Serve(httpl)) }()
  105. plog.Noticef("serving insecure client requests on %s, this is strongly discouraged!", sctx.l.Addr().String())
  106. }
  107. if sctx.secure {
  108. tlscfg, tlsErr := tlsinfo.ServerConfig()
  109. if tlsErr != nil {
  110. return tlsErr
  111. }
  112. gs := v3rpc.Server(s, tlscfg, gopts...)
  113. sctx.secureGrpcServerC <- gs
  114. v3electionpb.RegisterElectionServer(gs, servElection)
  115. v3lockpb.RegisterLockServer(gs, servLock)
  116. if sctx.serviceRegister != nil {
  117. sctx.serviceRegister(gs)
  118. }
  119. handler = grpcHandlerFunc(gs, handler)
  120. dtls := tlscfg.Clone()
  121. // trust local server
  122. dtls.InsecureSkipVerify = true
  123. creds := credentials.NewTLS(dtls)
  124. opts := []grpc.DialOption{grpc.WithTransportCredentials(creds)}
  125. gwmux, err := sctx.registerGateway(opts)
  126. if err != nil {
  127. return err
  128. }
  129. tlsl, lerr := transport.NewTLSListener(m.Match(cmux.Any()), tlsinfo)
  130. if lerr != nil {
  131. return lerr
  132. }
  133. // TODO: add debug flag; enable logging when debug flag is set
  134. httpmux := sctx.createMux(gwmux, handler)
  135. srv := &http.Server{
  136. Handler: wrapMux(httpmux),
  137. TLSConfig: tlscfg,
  138. ErrorLog: logger, // do not log user error
  139. }
  140. go func() { errHandler(srv.Serve(tlsl)) }()
  141. sctx.secureHTTPServer = srv
  142. plog.Infof("serving client requests on %s", sctx.l.Addr().String())
  143. }
  144. close(sctx.secureGrpcServerC)
  145. close(sctx.insecureGrpcServerC)
  146. return m.Serve()
  147. }
  148. // grpcHandlerFunc returns an http.Handler that delegates to grpcServer on incoming gRPC
  149. // connections or otherHandler otherwise. Given in gRPC docs.
  150. func grpcHandlerFunc(grpcServer *grpc.Server, otherHandler http.Handler) http.Handler {
  151. if otherHandler == nil {
  152. return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
  153. grpcServer.ServeHTTP(w, r)
  154. })
  155. }
  156. return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
  157. if r.ProtoMajor == 2 && strings.Contains(r.Header.Get("Content-Type"), "application/grpc") {
  158. grpcServer.ServeHTTP(w, r)
  159. } else {
  160. otherHandler.ServeHTTP(w, r)
  161. }
  162. })
  163. }
  164. type registerHandlerFunc func(context.Context, *gw.ServeMux, *grpc.ClientConn) error
  165. func (sctx *serveCtx) registerGateway(opts []grpc.DialOption) (*gw.ServeMux, error) {
  166. ctx := sctx.ctx
  167. conn, err := grpc.DialContext(ctx, sctx.addr, opts...)
  168. if err != nil {
  169. return nil, err
  170. }
  171. gwmux := gw.NewServeMux()
  172. handlers := []registerHandlerFunc{
  173. etcdservergw.RegisterKVHandler,
  174. etcdservergw.RegisterWatchHandler,
  175. etcdservergw.RegisterLeaseHandler,
  176. etcdservergw.RegisterClusterHandler,
  177. etcdservergw.RegisterMaintenanceHandler,
  178. etcdservergw.RegisterAuthHandler,
  179. v3lockgw.RegisterLockHandler,
  180. v3electiongw.RegisterElectionHandler,
  181. }
  182. for _, h := range handlers {
  183. if err := h(ctx, gwmux, conn); err != nil {
  184. return nil, err
  185. }
  186. }
  187. go func() {
  188. <-ctx.Done()
  189. if cerr := conn.Close(); cerr != nil {
  190. plog.Warningf("failed to close conn to %s: %v", sctx.l.Addr().String(), cerr)
  191. }
  192. }()
  193. return gwmux, nil
  194. }
  195. func (sctx *serveCtx) createMux(gwmux *gw.ServeMux, handler http.Handler) *http.ServeMux {
  196. httpmux := http.NewServeMux()
  197. for path, h := range sctx.userHandlers {
  198. httpmux.Handle(path, h)
  199. }
  200. httpmux.Handle(
  201. "/v3beta/",
  202. wsproxy.WebsocketProxy(
  203. gwmux,
  204. wsproxy.WithRequestMutator(
  205. // Default to the POST method for streams
  206. func(incoming *http.Request, outgoing *http.Request) *http.Request {
  207. outgoing.Method = "POST"
  208. return outgoing
  209. },
  210. ),
  211. ),
  212. )
  213. if handler != nil {
  214. httpmux.Handle("/", handler)
  215. }
  216. return httpmux
  217. }
  218. // wraps HTTP multiplexer to mute requests to /v3alpha
  219. // TODO: deprecate this in 3.4 release
  220. func wrapMux(mux *http.ServeMux) http.Handler { return &v3alphaMutator{mux: mux} }
  221. type v3alphaMutator struct {
  222. mux *http.ServeMux
  223. }
  224. func (m *v3alphaMutator) ServeHTTP(rw http.ResponseWriter, req *http.Request) {
  225. if req != nil && req.URL != nil && strings.HasPrefix(req.URL.Path, "/v3alpha/") {
  226. req.URL.Path = strings.Replace(req.URL.Path, "/v3alpha/", "/v3beta/", 1)
  227. }
  228. m.mux.ServeHTTP(rw, req)
  229. }
  230. func (sctx *serveCtx) registerUserHandler(s string, h http.Handler) {
  231. if sctx.userHandlers[s] != nil {
  232. plog.Warningf("path %s already registered by user handler", s)
  233. return
  234. }
  235. sctx.userHandlers[s] = h
  236. }
  237. func (sctx *serveCtx) registerPprof() {
  238. for p, h := range debugutil.PProfHandlers() {
  239. sctx.registerUserHandler(p, h)
  240. }
  241. }
  242. func (sctx *serveCtx) registerTrace() {
  243. reqf := func(w http.ResponseWriter, r *http.Request) { trace.Render(w, r, true) }
  244. sctx.registerUserHandler("/debug/requests", http.HandlerFunc(reqf))
  245. evf := func(w http.ResponseWriter, r *http.Request) { trace.RenderEvents(w, r, true) }
  246. sctx.registerUserHandler("/debug/events", http.HandlerFunc(evf))
  247. }
  248. // Attempt to gracefully tear down gRPC server(s) and any associated mechanisms
  249. func teardownServeCtx(sctx *serveCtx, timeout time.Duration) {
  250. if sctx.secure && len(sctx.secureGrpcServerC) > 0 {
  251. gs := <-sctx.secureGrpcServerC
  252. stopSecureServer(gs, sctx.secureHTTPServer, timeout)
  253. }
  254. if sctx.insecure && len(sctx.insecureGrpcServerC) > 0 {
  255. gs := <-sctx.insecureGrpcServerC
  256. stopInsecureServer(gs, timeout)
  257. }
  258. // Close any open gRPC connections
  259. sctx.cancel()
  260. }
  261. // When using grpc's ServerHandlerTransport we are responsible for gracefully
  262. // stopping connections and shutting down.
  263. // https://github.com/grpc/grpc-go/issues/1384#issuecomment-317124531
  264. func stopSecureServer(gs *grpc.Server, httpSrv *http.Server, timeout time.Duration) {
  265. ctx, cancel := context.WithTimeout(context.Background(), timeout)
  266. defer cancel()
  267. // Stop accepting new connections await pending handlers
  268. httpSrv.Shutdown(ctx)
  269. // Teardown gRPC server
  270. gs.Stop()
  271. }
  272. // Gracefully shutdown gRPC server when using HTTP2 transport.
  273. func stopInsecureServer(gs *grpc.Server, timeout time.Duration) {
  274. ch := make(chan struct{})
  275. go func() {
  276. defer close(ch)
  277. // close listeners to stop accepting new connections,
  278. // will block on any existing transports
  279. gs.GracefulStop()
  280. }()
  281. // wait until all pending RPCs are finished
  282. select {
  283. case <-ch:
  284. case <-time.After(timeout):
  285. // took too long, manually close open transports
  286. // e.g. watch streams
  287. gs.Stop()
  288. // concurrent GracefulStop should be interrupted
  289. <-ch
  290. }
  291. }