import (
"context"
+ "encoding/json"
"errors"
"fmt"
+ "io/ioutil"
+ "mime"
"net/http"
+ "net/http/httptest"
"net/url"
"strings"
"sync"
"time"
+ "git.arvados.org/arvados.git/lib/controller/api"
"git.arvados.org/arvados.git/lib/controller/federation"
"git.arvados.org/arvados.git/lib/controller/localdb"
"git.arvados.org/arvados.git/lib/controller/railsproxy"
"git.arvados.org/arvados.git/lib/controller/router"
+ "git.arvados.org/arvados.git/lib/ctrlctx"
"git.arvados.org/arvados.git/sdk/go/arvados"
"git.arvados.org/arvados.git/sdk/go/ctxlog"
"git.arvados.org/arvados.git/sdk/go/health"
"git.arvados.org/arvados.git/sdk/go/httpserver"
- "github.com/jmoiron/sqlx"
+
+ // sqlx needs lib/pq to talk to PostgreSQL
_ "github.com/lib/pq"
)
type Handler struct {
- Cluster *arvados.Cluster
+ Cluster *arvados.Cluster
+ BackgroundContext context.Context
setupOnce sync.Once
+ federation *federation.Conn
handlerStack http.Handler
proxy *proxy
secureClient *http.Client
insecureClient *http.Client
- pgdb *sqlx.DB
- pgdbMtx sync.Mutex
+ dbConnector ctrlctx.DBConnector
+ limitLogCreate chan struct{}
+
+ cache map[string]*cacheEnt
}
func (h *Handler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
req.URL.Path = strings.Replace(req.URL.Path, "//", "/", -1)
}
}
- if h.Cluster.API.RequestTimeout > 0 {
- ctx, cancel := context.WithDeadline(req.Context(), time.Now().Add(time.Duration(h.Cluster.API.RequestTimeout)))
- req = req.WithContext(ctx)
- defer cancel()
- }
-
h.handlerStack.ServeHTTP(w, req)
}
func (h *Handler) CheckHealth() error {
h.setupOnce.Do(h.setup)
- _, err := h.db(context.TODO())
+ _, err := h.dbConnector.GetDB(context.TODO())
if err != nil {
return err
}
_, _, err = railsproxy.FindRailsAPI(h.Cluster)
- return err
+ if err != nil {
+ return err
+ }
+ if h.Cluster.API.VocabularyPath != "" {
+ req, err := http.NewRequest("GET", "/arvados/v1/vocabulary", nil)
+ if err != nil {
+ return err
+ }
+ var resp httptest.ResponseRecorder
+ h.handlerStack.ServeHTTP(&resp, req)
+ if resp.Result().StatusCode != http.StatusOK {
+ return fmt.Errorf("%d %s", resp.Result().StatusCode, resp.Result().Status)
+ }
+ }
+ return nil
}
func (h *Handler) Done() <-chan struct{} {
func (h *Handler) setup() {
mux := http.NewServeMux()
+ healthFuncs := make(map[string]health.Func)
+
+ h.dbConnector = ctrlctx.DBConnector{PostgreSQL: h.Cluster.PostgreSQL}
+ go func() {
+ <-h.BackgroundContext.Done()
+ h.dbConnector.Close()
+ }()
+ oidcAuthorizer := localdb.OIDCAccessTokenAuthorizer(h.Cluster, h.dbConnector.GetDB)
+ h.federation = federation.New(h.BackgroundContext, h.Cluster, &healthFuncs, h.dbConnector.GetDB)
+ rtr := router.New(h.federation, router.Config{
+ MaxRequestSize: h.Cluster.API.MaxRequestSize,
+ WrapCalls: api.ComposeWrappers(
+ ctrlctx.WrapCallsInTransactions(h.dbConnector.GetDB),
+ oidcAuthorizer.WrapCalls,
+ ctrlctx.WrapCallsWithAuth(h.Cluster)),
+ })
+
+ healthRoutes := health.Routes{"ping": func() error { _, err := h.dbConnector.GetDB(context.TODO()); return err }}
+ for name, f := range healthFuncs {
+ healthRoutes[name] = f
+ }
mux.Handle("/_health/", &health.Handler{
Token: h.Cluster.ManagementToken,
Prefix: "/_health/",
- Routes: health.Routes{"ping": func() error { _, err := h.db(context.TODO()); return err }},
+ Routes: healthRoutes,
})
-
- rtr := router.New(federation.New(h.Cluster), localdb.WrapCallsInTransactions(h.db))
mux.Handle("/arvados/v1/config", rtr)
- mux.Handle("/"+arvados.EndpointUserAuthenticate.Path, rtr)
-
- if !h.Cluster.ForceLegacyAPI14 {
- mux.Handle("/arvados/v1/collections", rtr)
- mux.Handle("/arvados/v1/collections/", rtr)
- mux.Handle("/arvados/v1/users", rtr)
- mux.Handle("/arvados/v1/users/", rtr)
- mux.Handle("/login", rtr)
- mux.Handle("/logout", rtr)
- }
+ mux.Handle("/arvados/v1/vocabulary", rtr)
+ mux.Handle("/"+arvados.EndpointUserAuthenticate.Path, rtr) // must come before .../users/
+ mux.Handle("/arvados/v1/collections", rtr)
+ mux.Handle("/arvados/v1/collections/", rtr)
+ mux.Handle("/arvados/v1/users", rtr)
+ mux.Handle("/arvados/v1/users/", rtr)
+ mux.Handle("/arvados/v1/connect/", rtr)
+ mux.Handle("/arvados/v1/container_requests", rtr)
+ mux.Handle("/arvados/v1/container_requests/", rtr)
+ mux.Handle("/arvados/v1/groups", rtr)
+ mux.Handle("/arvados/v1/groups/", rtr)
+ mux.Handle("/arvados/v1/links", rtr)
+ mux.Handle("/arvados/v1/links/", rtr)
+ mux.Handle("/arvados/v1/authorized_keys", rtr)
+ mux.Handle("/arvados/v1/authorized_keys/", rtr)
+ mux.Handle("/login", rtr)
+ mux.Handle("/logout", rtr)
+ mux.Handle("/arvados/v1/api_client_authorizations", rtr)
+ mux.Handle("/arvados/v1/api_client_authorizations/", rtr)
hs := http.NotFoundHandler()
hs = prepend(hs, h.proxyRailsAPI)
+ hs = prepend(hs, h.routeContainerEndpoints(rtr))
+ hs = prepend(hs, h.limitLogCreateRequests)
hs = h.setupProxyRemoteCluster(hs)
+ hs = prepend(hs, oidcAuthorizer.Middleware)
mux.Handle("/", hs)
h.handlerStack = mux
ic.CheckRedirect = neverRedirect
h.insecureClient = &ic
+ logCreateLimit := int(float64(h.Cluster.API.MaxConcurrentRequests) * h.Cluster.API.LogCreateRequestFraction)
+ if logCreateLimit == 0 && h.Cluster.API.LogCreateRequestFraction > 0 {
+ logCreateLimit = 1
+ }
+ h.limitLogCreate = make(chan struct{}, logCreateLimit)
+
h.proxy = &proxy{
Name: "arvados-controller",
}
-}
-
-var errDBConnection = errors.New("database connection error")
-
-func (h *Handler) db(ctx context.Context) (*sqlx.DB, error) {
- h.pgdbMtx.Lock()
- defer h.pgdbMtx.Unlock()
- if h.pgdb != nil {
- return h.pgdb, nil
+ h.cache = map[string]*cacheEnt{
+ "/discovery/v1/apis/arvados/v1/rest": &cacheEnt{validate: validateDiscoveryDoc},
}
- db, err := sqlx.Open("postgres", h.Cluster.PostgreSQL.Connection.String())
- if err != nil {
- ctxlog.FromContext(ctx).WithError(err).Error("postgresql connect failed")
- return nil, errDBConnection
- }
- if p := h.Cluster.PostgreSQL.ConnectionPool; p > 0 {
- db.SetMaxOpenConns(p)
- }
- if err := db.Ping(); err != nil {
- ctxlog.FromContext(ctx).WithError(err).Error("postgresql connect scuceeded but ping failed")
- return nil, errDBConnection
- }
- h.pgdb = db
- return db, nil
+ go h.trashSweepWorker()
+ go h.containerLogSweepWorker()
}
type middlewareFunc func(http.ResponseWriter, *http.Request, http.Handler)
return h.proxy.Do(req, urlOut, client)
}
+// Route /arvados/v1/containers/{uuid}/log*, .../ssh, and
+// .../gateway_tunnel to rtr, pass everything else to next.
+//
+// (http.ServeMux doesn't let us route these without also routing
+// everything under /containers/, which we don't want yet.)
+func (h *Handler) routeContainerEndpoints(rtr http.Handler) middlewareFunc {
+ return func(w http.ResponseWriter, req *http.Request, next http.Handler) {
+ trim := strings.TrimPrefix(req.URL.Path, "/arvados/v1/containers/")
+ if trim != req.URL.Path && (strings.Index(trim, "/log") == 27 ||
+ strings.Index(trim, "/ssh") == 27 ||
+ strings.Index(trim, "/gateway_tunnel") == 27) {
+ rtr.ServeHTTP(w, req)
+ } else {
+ next.ServeHTTP(w, req)
+ }
+ }
+}
+
+func (h *Handler) limitLogCreateRequests(w http.ResponseWriter, req *http.Request, next http.Handler) {
+ if cap(h.limitLogCreate) > 0 && req.Method == http.MethodPost && strings.HasPrefix(req.URL.Path, "/arvados/v1/logs") {
+ select {
+ case h.limitLogCreate <- struct{}{}:
+ defer func() { <-h.limitLogCreate }()
+ next.ServeHTTP(w, req)
+ default:
+ http.Error(w, "Excess log messages", http.StatusServiceUnavailable)
+ }
+ return
+ }
+ next.ServeHTTP(w, req)
+}
+
+// cacheEnt implements a basic stale-while-revalidate cache, suitable
+// for the Arvados discovery document.
+type cacheEnt struct {
+ validate func(body []byte) error
+ mtx sync.Mutex
+ header http.Header
+ body []byte
+ expireAfter time.Time
+ refreshAfter time.Time
+ refreshLock sync.Mutex
+}
+
+const (
+ cacheTTL = 5 * time.Minute
+ cacheExpire = 24 * time.Hour
+)
+
+func (ent *cacheEnt) refresh(path string, do func(*http.Request) (*http.Response, error)) (http.Header, []byte, error) {
+ ent.refreshLock.Lock()
+ defer ent.refreshLock.Unlock()
+ if header, body, needRefresh := ent.response(); !needRefresh {
+ // another goroutine refreshed successfully while we
+ // were waiting for refreshLock
+ return header, body, nil
+ } else if body != nil {
+ // Cache is present, but expired. We'll try to refresh
+ // below. Meanwhile, other refresh() calls will queue
+ // up for refreshLock -- and we don't want them to
+ // turn into N upstream requests, even if upstream is
+ // failing. (If we succeed we'll update the expiry
+ // time again below with the real cacheTTL -- this
+ // just takes care of the error case.)
+ ent.mtx.Lock()
+ ent.refreshAfter = time.Now().Add(time.Second)
+ ent.mtx.Unlock()
+ }
+
+ ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(time.Minute))
+ defer cancel()
+ // 0.0.0.0:0 is just a placeholder here -- do(), which is
+ // localClusterRequest(), will replace the scheme and host
+ // parts with the real proxy destination.
+ req, err := http.NewRequestWithContext(ctx, http.MethodGet, "http://0.0.0.0:0/"+path, nil)
+ if err != nil {
+ return nil, nil, err
+ }
+ resp, err := do(req)
+ if err != nil {
+ return nil, nil, err
+ }
+ if resp.StatusCode != http.StatusOK {
+ return nil, nil, fmt.Errorf("HTTP status %d", resp.StatusCode)
+ }
+ body, err := ioutil.ReadAll(resp.Body)
+ if err != nil {
+ return nil, nil, fmt.Errorf("Read error: %w", err)
+ }
+ header := http.Header{}
+ for k, v := range resp.Header {
+ if !dropHeaders[k] && k != "X-Request-Id" {
+ header[k] = v
+ }
+ }
+ if ent.validate != nil {
+ if err := ent.validate(body); err != nil {
+ return nil, nil, err
+ }
+ } else if mediatype, _, err := mime.ParseMediaType(header.Get("Content-Type")); err == nil && mediatype == "application/json" {
+ if !json.Valid(body) {
+ return nil, nil, errors.New("invalid JSON encoding in response")
+ }
+ }
+ ent.mtx.Lock()
+ defer ent.mtx.Unlock()
+ ent.header = header
+ ent.body = body
+ ent.refreshAfter = time.Now().Add(cacheTTL)
+ ent.expireAfter = time.Now().Add(cacheExpire)
+ return ent.header, ent.body, nil
+}
+
+func (ent *cacheEnt) response() (http.Header, []byte, bool) {
+ ent.mtx.Lock()
+ defer ent.mtx.Unlock()
+ if ent.expireAfter.Before(time.Now()) {
+ ent.header, ent.body, ent.refreshAfter = nil, nil, time.Time{}
+ }
+ return ent.header, ent.body, ent.refreshAfter.Before(time.Now())
+}
+
+func (ent *cacheEnt) ServeHTTP(ctx context.Context, w http.ResponseWriter, path string, do func(*http.Request) (*http.Response, error)) {
+ header, body, needRefresh := ent.response()
+ if body == nil {
+ // need to fetch before we can return anything
+ var err error
+ header, body, err = ent.refresh(path, do)
+ if err != nil {
+ http.Error(w, err.Error(), http.StatusBadGateway)
+ return
+ }
+ } else if needRefresh {
+ // re-fetch in background
+ go func() {
+ _, _, err := ent.refresh(path, do)
+ if err != nil {
+ ctxlog.FromContext(ctx).WithError(err).WithField("path", path).Warn("error refreshing cache")
+ }
+ }()
+ }
+ for k, v := range header {
+ w.Header()[k] = v
+ }
+ w.WriteHeader(http.StatusOK)
+ w.Write(body)
+}
+
func (h *Handler) proxyRailsAPI(w http.ResponseWriter, req *http.Request, next http.Handler) {
+ if ent, ok := h.cache[req.URL.Path]; ok && req.Method == http.MethodGet {
+ ent.ServeHTTP(req.Context(), w, req.URL.Path, h.localClusterRequest)
+ return
+ }
resp, err := h.localClusterRequest(req)
n, err := h.proxy.ForwardResponse(w, resp, err)
if err != nil {
}
return best, cluster.TLS.Insecure, nil
}
+
+func validateDiscoveryDoc(body []byte) error {
+ var dd arvados.DiscoveryDocument
+ err := json.Unmarshal(body, &dd)
+ if err != nil {
+ return fmt.Errorf("error decoding JSON response: %w", err)
+ }
+ if dd.BasePath == "" {
+ return errors.New("error in discovery document: no value for basePath")
+ }
+ return nil
+}