grpc_proxy.go 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352
  1. // Copyright 2016 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 etcdmain
  15. import (
  16. "context"
  17. "fmt"
  18. "math"
  19. "net"
  20. "net/http"
  21. "net/url"
  22. "os"
  23. "path/filepath"
  24. "time"
  25. "github.com/coreos/etcd/clientv3"
  26. "github.com/coreos/etcd/clientv3/namespace"
  27. "github.com/coreos/etcd/clientv3/ordering"
  28. "github.com/coreos/etcd/etcdserver/api/etcdhttp"
  29. "github.com/coreos/etcd/etcdserver/api/v3election/v3electionpb"
  30. "github.com/coreos/etcd/etcdserver/api/v3lock/v3lockpb"
  31. pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
  32. "github.com/coreos/etcd/pkg/debugutil"
  33. "github.com/coreos/etcd/pkg/transport"
  34. "github.com/coreos/etcd/proxy/grpcproxy"
  35. "github.com/cockroachdb/cmux"
  36. grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
  37. "github.com/spf13/cobra"
  38. "google.golang.org/grpc"
  39. )
  40. var (
  41. grpcProxyListenAddr string
  42. grpcProxyMetricsListenAddr string
  43. grpcProxyEndpoints []string
  44. grpcProxyDNSCluster string
  45. grpcProxyInsecureDiscovery bool
  46. grpcProxyDataDir string
  47. // tls for connecting to etcd
  48. grpcProxyCA string
  49. grpcProxyCert string
  50. grpcProxyKey string
  51. grpcProxyInsecureSkipTLSVerify bool
  52. // tls for clients connecting to proxy
  53. grpcProxyListenCA string
  54. grpcProxyListenCert string
  55. grpcProxyListenKey string
  56. grpcProxyListenAutoTLS bool
  57. grpcProxyListenCRL string
  58. grpcProxyAdvertiseClientURL string
  59. grpcProxyResolverPrefix string
  60. grpcProxyResolverTTL int
  61. grpcProxyNamespace string
  62. grpcProxyEnablePprof bool
  63. grpcProxyEnableOrdering bool
  64. )
  65. func init() {
  66. rootCmd.AddCommand(newGRPCProxyCommand())
  67. }
  68. // newGRPCProxyCommand returns the cobra command for "grpc-proxy".
  69. func newGRPCProxyCommand() *cobra.Command {
  70. lpc := &cobra.Command{
  71. Use: "grpc-proxy <subcommand>",
  72. Short: "grpc-proxy related command",
  73. }
  74. lpc.AddCommand(newGRPCProxyStartCommand())
  75. return lpc
  76. }
  77. func newGRPCProxyStartCommand() *cobra.Command {
  78. cmd := cobra.Command{
  79. Use: "start",
  80. Short: "start the grpc proxy",
  81. Run: startGRPCProxy,
  82. }
  83. cmd.Flags().StringVar(&grpcProxyListenAddr, "listen-addr", "127.0.0.1:23790", "listen address")
  84. cmd.Flags().StringVar(&grpcProxyDNSCluster, "discovery-srv", "", "DNS domain used to bootstrap initial cluster")
  85. cmd.Flags().StringVar(&grpcProxyMetricsListenAddr, "metrics-addr", "", "listen for /metrics requests on an additional interface")
  86. cmd.Flags().BoolVar(&grpcProxyInsecureDiscovery, "insecure-discovery", false, "accept insecure SRV records")
  87. cmd.Flags().StringSliceVar(&grpcProxyEndpoints, "endpoints", []string{"127.0.0.1:2379"}, "comma separated etcd cluster endpoints")
  88. cmd.Flags().StringVar(&grpcProxyAdvertiseClientURL, "advertise-client-url", "127.0.0.1:23790", "advertise address to register (must be reachable by client)")
  89. cmd.Flags().StringVar(&grpcProxyResolverPrefix, "resolver-prefix", "", "prefix to use for registering proxy (must be shared with other grpc-proxy members)")
  90. cmd.Flags().IntVar(&grpcProxyResolverTTL, "resolver-ttl", 0, "specify TTL, in seconds, when registering proxy endpoints")
  91. cmd.Flags().StringVar(&grpcProxyNamespace, "namespace", "", "string to prefix to all keys for namespacing requests")
  92. cmd.Flags().BoolVar(&grpcProxyEnablePprof, "enable-pprof", false, `Enable runtime profiling data via HTTP server. Address is at client URL + "/debug/pprof/"`)
  93. cmd.Flags().StringVar(&grpcProxyDataDir, "data-dir", "default.proxy", "Data directory for persistent data")
  94. // client TLS for connecting to server
  95. cmd.Flags().StringVar(&grpcProxyCert, "cert", "", "identify secure connections with etcd servers using this TLS certificate file")
  96. cmd.Flags().StringVar(&grpcProxyKey, "key", "", "identify secure connections with etcd servers using this TLS key file")
  97. cmd.Flags().StringVar(&grpcProxyCA, "cacert", "", "verify certificates of TLS-enabled secure etcd servers using this CA bundle")
  98. cmd.Flags().BoolVar(&grpcProxyInsecureSkipTLSVerify, "insecure-skip-tls-verify", false, "skip authentication of etcd server TLS certificates")
  99. // client TLS for connecting to proxy
  100. cmd.Flags().StringVar(&grpcProxyListenCert, "cert-file", "", "identify secure connections to the proxy using this TLS certificate file")
  101. cmd.Flags().StringVar(&grpcProxyListenKey, "key-file", "", "identify secure connections to the proxy using this TLS key file")
  102. cmd.Flags().StringVar(&grpcProxyListenCA, "trusted-ca-file", "", "verify certificates of TLS-enabled secure proxy using this CA bundle")
  103. cmd.Flags().BoolVar(&grpcProxyListenAutoTLS, "auto-tls", false, "proxy TLS using generated certificates")
  104. cmd.Flags().StringVar(&grpcProxyListenCRL, "client-crl-file", "", "proxy client certificate revocation list file.")
  105. // experimental flags
  106. cmd.Flags().BoolVar(&grpcProxyEnableOrdering, "experimental-serializable-ordering", false, "Ensure serializable reads have monotonically increasing store revisions across endpoints.")
  107. return &cmd
  108. }
  109. func startGRPCProxy(cmd *cobra.Command, args []string) {
  110. checkArgs()
  111. tlsinfo := newTLS(grpcProxyListenCA, grpcProxyListenCert, grpcProxyListenKey)
  112. if tlsinfo == nil && grpcProxyListenAutoTLS {
  113. host := []string{"https://" + grpcProxyListenAddr}
  114. dir := filepath.Join(grpcProxyDataDir, "fixtures", "proxy")
  115. autoTLS, err := transport.SelfCert(dir, host)
  116. if err != nil {
  117. plog.Fatal(err)
  118. }
  119. tlsinfo = &autoTLS
  120. }
  121. if tlsinfo != nil {
  122. plog.Infof("ServerTLS: %s", tlsinfo)
  123. }
  124. m := mustListenCMux(tlsinfo)
  125. grpcl := m.Match(cmux.HTTP2())
  126. defer func() {
  127. grpcl.Close()
  128. plog.Infof("stopping listening for grpc-proxy client requests on %s", grpcProxyListenAddr)
  129. }()
  130. client := mustNewClient()
  131. srvhttp, httpl := mustHTTPListener(m, tlsinfo, client)
  132. errc := make(chan error)
  133. go func() { errc <- newGRPCProxyServer(client).Serve(grpcl) }()
  134. go func() { errc <- srvhttp.Serve(httpl) }()
  135. go func() { errc <- m.Serve() }()
  136. if len(grpcProxyMetricsListenAddr) > 0 {
  137. mhttpl := mustMetricsListener(tlsinfo)
  138. go func() {
  139. mux := http.NewServeMux()
  140. etcdhttp.HandlePrometheus(mux)
  141. grpcproxy.HandleHealth(mux, client)
  142. plog.Fatal(http.Serve(mhttpl, mux))
  143. }()
  144. }
  145. // grpc-proxy is initialized, ready to serve
  146. notifySystemd()
  147. fmt.Fprintln(os.Stderr, <-errc)
  148. os.Exit(1)
  149. }
  150. func checkArgs() {
  151. if grpcProxyResolverPrefix != "" && grpcProxyResolverTTL < 1 {
  152. fmt.Fprintln(os.Stderr, fmt.Errorf("invalid resolver-ttl %d", grpcProxyResolverTTL))
  153. os.Exit(1)
  154. }
  155. if grpcProxyResolverPrefix == "" && grpcProxyResolverTTL > 0 {
  156. fmt.Fprintln(os.Stderr, fmt.Errorf("invalid resolver-prefix %q", grpcProxyResolverPrefix))
  157. os.Exit(1)
  158. }
  159. if grpcProxyResolverPrefix != "" && grpcProxyResolverTTL > 0 && grpcProxyAdvertiseClientURL == "" {
  160. fmt.Fprintln(os.Stderr, fmt.Errorf("invalid advertise-client-url %q", grpcProxyAdvertiseClientURL))
  161. os.Exit(1)
  162. }
  163. }
  164. func mustNewClient() *clientv3.Client {
  165. srvs := discoverEndpoints(grpcProxyDNSCluster, grpcProxyCA, grpcProxyInsecureDiscovery)
  166. eps := srvs.Endpoints
  167. if len(eps) == 0 {
  168. eps = grpcProxyEndpoints
  169. }
  170. cfg, err := newClientCfg(eps)
  171. if err != nil {
  172. fmt.Fprintln(os.Stderr, err)
  173. os.Exit(1)
  174. }
  175. client, err := clientv3.New(*cfg)
  176. if err != nil {
  177. fmt.Fprintln(os.Stderr, err)
  178. os.Exit(1)
  179. }
  180. return client
  181. }
  182. func newClientCfg(eps []string) (*clientv3.Config, error) {
  183. // set tls if any one tls option set
  184. cfg := clientv3.Config{
  185. Endpoints: eps,
  186. DialTimeout: 5 * time.Second,
  187. }
  188. tls := newTLS(grpcProxyCA, grpcProxyCert, grpcProxyKey)
  189. if tls == nil && grpcProxyInsecureSkipTLSVerify {
  190. tls = &transport.TLSInfo{}
  191. }
  192. if tls != nil {
  193. clientTLS, err := tls.ClientConfig()
  194. if err != nil {
  195. return nil, err
  196. }
  197. clientTLS.InsecureSkipVerify = grpcProxyInsecureSkipTLSVerify
  198. cfg.TLS = clientTLS
  199. plog.Infof("ClientTLS: %s", tls)
  200. }
  201. return &cfg, nil
  202. }
  203. func newTLS(ca, cert, key string) *transport.TLSInfo {
  204. if ca == "" && cert == "" && key == "" {
  205. return nil
  206. }
  207. return &transport.TLSInfo{CAFile: ca, CertFile: cert, KeyFile: key}
  208. }
  209. func mustListenCMux(tlsinfo *transport.TLSInfo) cmux.CMux {
  210. l, err := net.Listen("tcp", grpcProxyListenAddr)
  211. if err != nil {
  212. fmt.Fprintln(os.Stderr, err)
  213. os.Exit(1)
  214. }
  215. if l, err = transport.NewKeepAliveListener(l, "tcp", nil); err != nil {
  216. fmt.Fprintln(os.Stderr, err)
  217. os.Exit(1)
  218. }
  219. if tlsinfo != nil {
  220. tlsinfo.CRLFile = grpcProxyListenCRL
  221. if l, err = transport.NewTLSListener(l, tlsinfo); err != nil {
  222. plog.Fatal(err)
  223. }
  224. }
  225. plog.Infof("listening for grpc-proxy client requests on %s", grpcProxyListenAddr)
  226. return cmux.New(l)
  227. }
  228. func newGRPCProxyServer(client *clientv3.Client) *grpc.Server {
  229. if grpcProxyEnableOrdering {
  230. vf := ordering.NewOrderViolationSwitchEndpointClosure(*client)
  231. client.KV = ordering.NewKV(client.KV, vf)
  232. plog.Infof("waiting for linearized read from cluster to recover ordering")
  233. for {
  234. _, err := client.KV.Get(context.TODO(), "_", clientv3.WithKeysOnly())
  235. if err == nil {
  236. break
  237. }
  238. plog.Warningf("ordering recovery failed, retrying in 1s (%v)", err)
  239. time.Sleep(time.Second)
  240. }
  241. }
  242. if len(grpcProxyNamespace) > 0 {
  243. client.KV = namespace.NewKV(client.KV, grpcProxyNamespace)
  244. client.Watcher = namespace.NewWatcher(client.Watcher, grpcProxyNamespace)
  245. client.Lease = namespace.NewLease(client.Lease, grpcProxyNamespace)
  246. }
  247. kvp, _ := grpcproxy.NewKvProxy(client)
  248. watchp, _ := grpcproxy.NewWatchProxy(client)
  249. if grpcProxyResolverPrefix != "" {
  250. grpcproxy.Register(client, grpcProxyResolverPrefix, grpcProxyAdvertiseClientURL, grpcProxyResolverTTL)
  251. }
  252. clusterp, _ := grpcproxy.NewClusterProxy(client, grpcProxyAdvertiseClientURL, grpcProxyResolverPrefix)
  253. leasep, _ := grpcproxy.NewLeaseProxy(client)
  254. mainp := grpcproxy.NewMaintenanceProxy(client)
  255. authp := grpcproxy.NewAuthProxy(client)
  256. electionp := grpcproxy.NewElectionProxy(client)
  257. lockp := grpcproxy.NewLockProxy(client)
  258. server := grpc.NewServer(
  259. grpc.StreamInterceptor(grpc_prometheus.StreamServerInterceptor),
  260. grpc.UnaryInterceptor(grpc_prometheus.UnaryServerInterceptor),
  261. grpc.MaxConcurrentStreams(math.MaxUint32),
  262. )
  263. pb.RegisterKVServer(server, kvp)
  264. pb.RegisterWatchServer(server, watchp)
  265. pb.RegisterClusterServer(server, clusterp)
  266. pb.RegisterLeaseServer(server, leasep)
  267. pb.RegisterMaintenanceServer(server, mainp)
  268. pb.RegisterAuthServer(server, authp)
  269. v3electionpb.RegisterElectionServer(server, electionp)
  270. v3lockpb.RegisterLockServer(server, lockp)
  271. return server
  272. }
  273. func mustHTTPListener(m cmux.CMux, tlsinfo *transport.TLSInfo, c *clientv3.Client) (*http.Server, net.Listener) {
  274. httpmux := http.NewServeMux()
  275. httpmux.HandleFunc("/", http.NotFound)
  276. etcdhttp.HandlePrometheus(httpmux)
  277. grpcproxy.HandleHealth(httpmux, c)
  278. if grpcProxyEnablePprof {
  279. for p, h := range debugutil.PProfHandlers() {
  280. httpmux.Handle(p, h)
  281. }
  282. plog.Infof("pprof is enabled under %s", debugutil.HTTPPrefixPProf)
  283. }
  284. srvhttp := &http.Server{Handler: httpmux}
  285. if tlsinfo == nil {
  286. return srvhttp, m.Match(cmux.HTTP1())
  287. }
  288. srvTLS, err := tlsinfo.ServerConfig()
  289. if err != nil {
  290. plog.Fatalf("could not setup TLS (%v)", err)
  291. }
  292. srvhttp.TLSConfig = srvTLS
  293. return srvhttp, m.Match(cmux.Any())
  294. }
  295. func mustMetricsListener(tlsinfo *transport.TLSInfo) net.Listener {
  296. murl, err := url.Parse(grpcProxyMetricsListenAddr)
  297. if err != nil {
  298. fmt.Fprintf(os.Stderr, "cannot parse %q", grpcProxyMetricsListenAddr)
  299. os.Exit(1)
  300. }
  301. ml, err := transport.NewListener(murl.Host, murl.Scheme, tlsinfo)
  302. if err != nil {
  303. fmt.Fprintln(os.Stderr, err)
  304. os.Exit(1)
  305. }
  306. plog.Info("grpc-proxy: listening for metrics on ", murl.String())
  307. return ml
  308. }