grpc_proxy.go 14 KB

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