[ARVADOS] updated: 1.3.0-642-g633543684

Git user git at public.curoverse.com
Thu Apr 11 19:09:18 UTC 2019


Summary of changes:
 lib/controller/federation/conn.go       | 112 ++++++++++++++++++++++++--------
 lib/controller/federation_test.go       |  15 ++++-
 lib/controller/railsproxy/railsproxy.go |  13 +++-
 lib/controller/router/router.go         |   9 +--
 lib/controller/rpc/conn.go              |  80 +++++++++++++----------
 sdk/go/arvados/api.go                   |  35 +++++-----
 sdk/go/arvados/client.go                |   4 +-
 sdk/go/auth/auth.go                     |   2 +-
 sdk/go/auth/handlers.go                 |   6 +-
 9 files changed, 187 insertions(+), 89 deletions(-)

  discards  c1f8c92f0bc1b97e4dde0dc66c746b89490ac16d (commit)
       via  633543684bec7bcd34f9236759a983018770b8f3 (commit)

This update added new revisions after undoing existing revisions.  That is
to say, the old revision is not a strict subset of the new revision.  This
situation occurs when you --force push a change and generate a repository
containing something like this:

 * -- * -- B -- O -- O -- O (c1f8c92f0bc1b97e4dde0dc66c746b89490ac16d)
            \
             N -- N -- N (633543684bec7bcd34f9236759a983018770b8f3)

When this happens we assume that you've already had alert emails for all
of the O revisions, and so we here report only the revisions in the N
branch from the common base, B.

Those revisions listed above that are new to this repository have
not appeared on any other notification email; so we list those
revisions in full, below.


commit 633543684bec7bcd34f9236759a983018770b8f3
Author: Tom Clegg <tclegg at veritasgenetics.com>
Date:   Thu Apr 11 15:08:54 2019 -0400

    14287: Refactor controller to use strong types in API handlers.
    
    Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tclegg at veritasgenetics.com>

diff --git a/build/run-tests.sh b/build/run-tests.sh
index a37a0f731..981b69528 100755
--- a/build/run-tests.sh
+++ b/build/run-tests.sh
@@ -77,6 +77,10 @@ doc
 lib/cli
 lib/cmd
 lib/controller
+lib/controller/federation
+lib/controller/railsproxy
+lib/controller/router
+lib/controller/rpc
 lib/crunchstat
 lib/cloud
 lib/cloud/azure
@@ -394,7 +398,7 @@ start_services() {
         return 0
     fi
     . "$VENVDIR/bin/activate"
-    echo 'Starting API, keepproxy, keep-web, ws, arv-git-httpd, and nginx ssl proxy...'
+    echo 'Starting API, controller, keepproxy, keep-web, arv-git-httpd, ws, and nginx ssl proxy...'
     if [[ ! -d "$WORKSPACE/services/api/log" ]]; then
 	mkdir -p "$WORKSPACE/services/api/log"
     fi
@@ -821,6 +825,7 @@ do_install_once() {
     title "install $1"
     timer_reset
 
+    result=
     if which deactivate >/dev/null; then deactivate; fi
     if [[ "$1" != "env" ]] && ! . "$VENVDIR/bin/activate"; then
         result=1
@@ -974,50 +979,7 @@ pythonstuff=(
 )
 
 declare -a gostuff
-gostuff=(
-    cmd/arvados-client
-    cmd/arvados-server
-    lib/cli
-    lib/cmd
-    lib/controller
-    lib/crunchstat
-    lib/cloud
-    lib/cloud/azure
-    lib/cloud/ec2
-    lib/dispatchcloud
-    lib/dispatchcloud/container
-    lib/dispatchcloud/scheduler
-    lib/dispatchcloud/ssh_executor
-    lib/dispatchcloud/worker
-    lib/service
-    sdk/go/arvados
-    sdk/go/arvadosclient
-    sdk/go/auth
-    sdk/go/blockdigest
-    sdk/go/dispatch
-    sdk/go/health
-    sdk/go/httpserver
-    sdk/go/manifest
-    sdk/go/asyncbuf
-    sdk/go/crunchrunner
-    sdk/go/stats
-    services/arv-git-httpd
-    services/crunchstat
-    services/health
-    services/keep-web
-    services/keepstore
-    sdk/go/keepclient
-    services/keep-balance
-    services/keepproxy
-    services/crunch-dispatch-local
-    services/crunch-dispatch-slurm
-    services/crunch-run
-    services/ws
-    tools/keep-block-check
-    tools/keep-exercise
-    tools/keep-rsync
-    tools/sync-groups
-)
+gostuff=($(git grep -lw func | grep \\.go | sed -e 's/\/[^\/]*$//' | sort -u))
 
 install_apps/workbench() {
     cd "$WORKSPACE/apps/workbench" \
diff --git a/lib/controller/federation/conn.go b/lib/controller/federation/conn.go
new file mode 100644
index 000000000..a08ec48f4
--- /dev/null
+++ b/lib/controller/federation/conn.go
@@ -0,0 +1,309 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package federation
+
+import (
+	"context"
+	"crypto/md5"
+	"errors"
+	"fmt"
+	"net/http"
+	"net/url"
+	"regexp"
+	"strings"
+
+	"git.curoverse.com/arvados.git/lib/controller/railsproxy"
+	"git.curoverse.com/arvados.git/lib/controller/rpc"
+	"git.curoverse.com/arvados.git/sdk/go/arvados"
+	"git.curoverse.com/arvados.git/sdk/go/auth"
+	"git.curoverse.com/arvados.git/sdk/go/ctxlog"
+)
+
+type Interface interface {
+	CollectionCreate(ctx context.Context, options arvados.CreateOptions) (arvados.Collection, error)
+	CollectionUpdate(ctx context.Context, options arvados.UpdateOptions) (arvados.Collection, error)
+	CollectionGet(ctx context.Context, options arvados.GetOptions) (arvados.Collection, error)
+	CollectionList(ctx context.Context, options arvados.ListOptions) (arvados.CollectionList, error)
+	CollectionDelete(ctx context.Context, options arvados.DeleteOptions) (arvados.Collection, error)
+	ContainerCreate(ctx context.Context, options arvados.CreateOptions) (arvados.Container, error)
+	ContainerUpdate(ctx context.Context, options arvados.UpdateOptions) (arvados.Container, error)
+	ContainerGet(ctx context.Context, options arvados.GetOptions) (arvados.Container, error)
+	ContainerList(ctx context.Context, options arvados.ListOptions) (arvados.ContainerList, error)
+	ContainerDelete(ctx context.Context, options arvados.DeleteOptions) (arvados.Container, error)
+	ContainerLock(ctx context.Context, options arvados.GetOptions) (arvados.Container, error)
+	ContainerUnlock(ctx context.Context, options arvados.GetOptions) (arvados.Container, error)
+	SpecimenCreate(ctx context.Context, options arvados.CreateOptions) (arvados.Specimen, error)
+	SpecimenUpdate(ctx context.Context, options arvados.UpdateOptions) (arvados.Specimen, error)
+	SpecimenGet(ctx context.Context, options arvados.GetOptions) (arvados.Specimen, error)
+	SpecimenList(ctx context.Context, options arvados.ListOptions) (arvados.SpecimenList, error)
+	SpecimenDelete(ctx context.Context, options arvados.DeleteOptions) (arvados.Specimen, error)
+	APIClientAuthorizationCurrent(ctx context.Context, options arvados.GetOptions) (arvados.APIClientAuthorization, error)
+}
+
+type Conn struct {
+	cluster *arvados.Cluster
+	local   backend
+	remotes map[string]backend
+}
+
+func New(cluster *arvados.Cluster, np *arvados.NodeProfile) Interface {
+	local := railsproxy.NewConn(cluster, np)
+	remotes := map[string]backend{}
+	for id, remote := range cluster.RemoteClusters {
+		if !remote.Proxy {
+			continue
+		}
+		remotes[id] = rpc.NewConn(id, &url.URL{Scheme: remote.Scheme, Host: remote.Host}, remote.Insecure, saltedTokenProvider(local, id))
+	}
+
+	return &Conn{
+		cluster: cluster,
+		local:   local,
+		remotes: remotes,
+	}
+}
+
+// Return a new rpc.TokenProvider that takes the client-provided
+// tokens from an incoming request context, determines whether they
+// should (and can) be salted for the given remoteID, and returns the
+// resulting tokens.
+func saltedTokenProvider(local backend, remoteID string) rpc.TokenProvider {
+	return func(ctx context.Context) ([]string, error) {
+		var tokens []string
+		incoming, ok := ctx.Value(auth.ContextKeyCredentials).(*auth.Credentials)
+		if !ok {
+			return nil, errors.New("no token provided")
+		}
+		for _, token := range incoming.Tokens {
+			salted, err := auth.SaltToken(token, remoteID)
+			switch err {
+			case nil:
+				tokens = append(tokens, salted)
+			case auth.ErrSalted:
+				tokens = append(tokens, token)
+			case auth.ErrObsoleteToken:
+				ctx := context.WithValue(ctx, auth.ContextKeyCredentials, &auth.Credentials{Tokens: []string{token}})
+				aca, err := local.APIClientAuthorizationCurrent(ctx, arvados.GetOptions{})
+				if errStatus(err) == http.StatusUnauthorized {
+					// pass through unmodified
+					tokens = append(tokens, token)
+					continue
+				} else if err != nil {
+					return nil, err
+				}
+				salted, err := auth.SaltToken(aca.TokenV2(), remoteID)
+				if err != nil {
+					return nil, err
+				}
+				tokens = append(tokens, salted)
+			default:
+				return nil, err
+			}
+		}
+		return tokens, nil
+	}
+}
+
+// Return suitable backend for a query about the given cluster ID
+// ("aaaaa") or object UUID ("aaaaa-dz642-abcdefghijklmno").
+func (conn *Conn) chooseBackend(id string) backend {
+	if len(id) > 5 {
+		id = id[:5]
+	}
+	if id == conn.cluster.ClusterID {
+		return conn.local
+	} else if be, ok := conn.remotes[id]; ok {
+		return be
+	} else {
+		// TODO: return an "always error" backend?
+		return conn.local
+	}
+}
+
+// Call fn with the local backend; then, if fn returned 404, call fn
+// on the available remote backends (possibly concurrently) until one
+// succeeds.
+//
+// The second argument to fn is the cluster ID of the remote backend,
+// or "" for the local backend.
+//
+// A non-nil error means all backends failed.
+func (conn *Conn) tryLocalThenRemotes(ctx context.Context, fn func(context.Context, string, backend) error) error {
+	if err := fn(ctx, "", conn.local); err == nil || errStatus(err) != http.StatusNotFound {
+		return err
+	}
+
+	ctx, cancel := context.WithCancel(ctx)
+	defer cancel()
+	errchan := make(chan error, len(conn.remotes))
+	for remoteID, be := range conn.remotes {
+		remoteID, be := remoteID, be
+		go func() {
+			errchan <- fn(ctx, remoteID, be)
+		}()
+	}
+	all404 := true
+	var errs []error
+	for i := 0; i < cap(errchan); i++ {
+		err := <-errchan
+		if err == nil {
+			return nil
+		}
+		all404 = all404 && errStatus(err) == http.StatusNotFound
+		errs = append(errs, err)
+	}
+	if all404 {
+		return notFoundError{}
+	}
+	// FIXME: choose appropriate HTTP status
+	return fmt.Errorf("errors: %v", errs)
+}
+
+func (conn *Conn) CollectionCreate(ctx context.Context, options arvados.CreateOptions) (arvados.Collection, error) {
+	return conn.chooseBackend(options.ClusterID).CollectionCreate(ctx, options)
+}
+
+func (conn *Conn) CollectionUpdate(ctx context.Context, options arvados.UpdateOptions) (arvados.Collection, error) {
+	return conn.chooseBackend(options.UUID).CollectionUpdate(ctx, options)
+}
+
+func rewriteManifest(mt, remoteID string) string {
+	return regexp.MustCompile(` [0-9a-f]{32}\+[^ ]*`).ReplaceAllStringFunc(mt, func(tok string) string {
+		return strings.Replace(tok, "+A", "+R"+remoteID+"-", -1)
+	})
+}
+
+// this could be in sdk/go/arvados
+func portableDataHash(mt string) string {
+	h := md5.New()
+	blkRe := regexp.MustCompile(`^ [0-9a-f]{32}\+\d+`)
+	size := 0
+	_ = regexp.MustCompile(` ?[^ ]*`).ReplaceAllFunc([]byte(mt), func(tok []byte) []byte {
+		if m := blkRe.Find(tok); m != nil {
+			// write hash+size, ignore remaining block hints
+			tok = m
+		}
+		n, err := h.Write(tok)
+		if err != nil {
+			panic(err)
+		}
+		size += n
+		return nil
+	})
+	return fmt.Sprintf("%x+%d", h.Sum(nil), size)
+}
+
+func (conn *Conn) CollectionGet(ctx context.Context, options arvados.GetOptions) (arvados.Collection, error) {
+	if len(options.UUID) == 27 {
+		// UUID is really a UUID
+		c, err := conn.chooseBackend(options.UUID).CollectionGet(ctx, options)
+		if err == nil && options.UUID[:5] != conn.cluster.ClusterID {
+			c.ManifestText = rewriteManifest(c.ManifestText, options.UUID[:5])
+		}
+		return c, err
+	} else {
+		// UUID is a PDH
+		first := make(chan arvados.Collection, 1)
+		err := conn.tryLocalThenRemotes(ctx, func(ctx context.Context, remoteID string, be backend) error {
+			c, err := be.CollectionGet(ctx, options)
+			if err != nil {
+				return err
+			}
+			if pdh := portableDataHash(c.ManifestText); pdh != options.UUID {
+				ctxlog.FromContext(ctx).Warnf("bad portable data hash %q received from remote %q (expected %q)", pdh, remoteID, options.UUID)
+				return notFoundError{}
+			}
+			if remoteID != "" {
+				c.ManifestText = rewriteManifest(c.ManifestText, remoteID)
+			}
+			select {
+			case first <- c:
+				return nil
+			default:
+				// lost race, return value doesn't matter
+				return nil
+			}
+		})
+		if err != nil {
+			return arvados.Collection{}, err
+		}
+		return <-first, nil
+	}
+}
+
+func (conn *Conn) CollectionList(ctx context.Context, options arvados.ListOptions) (arvados.CollectionList, error) {
+	return conn.local.CollectionList(ctx, options)
+}
+
+func (conn *Conn) CollectionDelete(ctx context.Context, options arvados.DeleteOptions) (arvados.Collection, error) {
+	return conn.chooseBackend(options.UUID).CollectionDelete(ctx, options)
+}
+
+func (conn *Conn) ContainerCreate(ctx context.Context, options arvados.CreateOptions) (arvados.Container, error) {
+	return conn.chooseBackend(options.ClusterID).ContainerCreate(ctx, options)
+}
+
+func (conn *Conn) ContainerUpdate(ctx context.Context, options arvados.UpdateOptions) (arvados.Container, error) {
+	return conn.chooseBackend(options.UUID).ContainerUpdate(ctx, options)
+}
+
+func (conn *Conn) ContainerGet(ctx context.Context, options arvados.GetOptions) (arvados.Container, error) {
+	return conn.chooseBackend(options.UUID).ContainerGet(ctx, options)
+}
+
+func (conn *Conn) ContainerList(ctx context.Context, options arvados.ListOptions) (arvados.ContainerList, error) {
+	return conn.local.ContainerList(ctx, options)
+}
+
+func (conn *Conn) ContainerDelete(ctx context.Context, options arvados.DeleteOptions) (arvados.Container, error) {
+	return conn.chooseBackend(options.UUID).ContainerDelete(ctx, options)
+}
+
+func (conn *Conn) ContainerLock(ctx context.Context, options arvados.GetOptions) (arvados.Container, error) {
+	return conn.chooseBackend(options.UUID).ContainerLock(ctx, options)
+}
+
+func (conn *Conn) ContainerUnlock(ctx context.Context, options arvados.GetOptions) (arvados.Container, error) {
+	return conn.chooseBackend(options.UUID).ContainerUnlock(ctx, options)
+}
+
+func (conn *Conn) SpecimenCreate(ctx context.Context, options arvados.CreateOptions) (arvados.Specimen, error) {
+	return conn.chooseBackend(options.ClusterID).SpecimenCreate(ctx, options)
+}
+
+func (conn *Conn) SpecimenUpdate(ctx context.Context, options arvados.UpdateOptions) (arvados.Specimen, error) {
+	return conn.chooseBackend(options.UUID).SpecimenUpdate(ctx, options)
+}
+
+func (conn *Conn) SpecimenGet(ctx context.Context, options arvados.GetOptions) (arvados.Specimen, error) {
+	return conn.chooseBackend(options.UUID).SpecimenGet(ctx, options)
+}
+
+func (conn *Conn) SpecimenList(ctx context.Context, options arvados.ListOptions) (arvados.SpecimenList, error) {
+	return conn.local.SpecimenList(ctx, options)
+}
+
+func (conn *Conn) SpecimenDelete(ctx context.Context, options arvados.DeleteOptions) (arvados.Specimen, error) {
+	return conn.chooseBackend(options.UUID).SpecimenDelete(ctx, options)
+}
+
+func (conn *Conn) APIClientAuthorizationCurrent(ctx context.Context, options arvados.GetOptions) (arvados.APIClientAuthorization, error) {
+	return conn.chooseBackend(options.UUID).APIClientAuthorizationCurrent(ctx, options)
+}
+
+type backend interface{ Interface }
+
+type notFoundError struct{}
+
+func (notFoundError) HTTPStatus() int { return http.StatusNotFound }
+func (notFoundError) Error() string   { return "not found" }
+
+func errStatus(err error) int {
+	if httpErr, ok := err.(interface{ HTTPStatus() int }); ok {
+		return httpErr.HTTPStatus()
+	} else {
+		return http.StatusInternalServerError
+	}
+}
diff --git a/lib/controller/federation_test.go b/lib/controller/federation_test.go
index 62916acd2..06c8f0086 100644
--- a/lib/controller/federation_test.go
+++ b/lib/controller/federation_test.go
@@ -39,7 +39,8 @@ type FederationSuite struct {
 	// provided by the integration test environment.
 	remoteServer *httpserver.Server
 	// remoteMock ("zmock") appends each incoming request to
-	// remoteMockRequests, and returns an empty 200 response.
+	// remoteMockRequests, and returns 200 with an empty JSON
+	// object.
 	remoteMock         *httpserver.Server
 	remoteMockRequests []http.Request
 }
@@ -68,6 +69,7 @@ func (s *FederationSuite) SetUpTest(c *check.C) {
 			MaxItemsPerResponse:            1000,
 			MultiClusterRequestConcurrency: 4,
 		},
+		EnableBetaController14287: enableBetaController14287,
 	}, NodeProfile: &nodeProfile}
 	s.testServer = newServerFromIntegrationTestEnv(c)
 	s.testServer.Server.Handler = httpserver.AddRequestIDs(httpserver.LogRequests(s.log, s.testHandler))
@@ -96,6 +98,8 @@ func (s *FederationSuite) remoteMockHandler(w http.ResponseWriter, req *http.Req
 	req.Body.Close()
 	req.Body = ioutil.NopCloser(b)
 	s.remoteMockRequests = append(s.remoteMockRequests, *req)
+	// Repond 200 with a valid JSON object
+	fmt.Fprint(w, "{}")
 }
 
 func (s *FederationSuite) TearDownTest(c *check.C) {
@@ -107,15 +111,15 @@ func (s *FederationSuite) TearDownTest(c *check.C) {
 	}
 }
 
-func (s *FederationSuite) testRequest(req *http.Request) *http.Response {
+func (s *FederationSuite) testRequest(req *http.Request) *httptest.ResponseRecorder {
 	resp := httptest.NewRecorder()
 	s.testServer.Server.Handler.ServeHTTP(resp, req)
-	return resp.Result()
+	return resp
 }
 
 func (s *FederationSuite) TestLocalRequest(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+strings.Replace(arvadostest.WorkflowWithDefinitionYAMLUUID, "zzzzz-", "zhome-", 1), nil)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	s.checkHandledLocally(c, resp)
 }
 
@@ -130,7 +134,7 @@ func (s *FederationSuite) checkHandledLocally(c *check.C, resp *http.Response) {
 
 func (s *FederationSuite) TestNoAuth(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+arvadostest.WorkflowWithDefinitionYAMLUUID, nil)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusUnauthorized)
 	s.checkJSONErrorMatches(c, resp, `Not logged in`)
 }
@@ -138,7 +142,7 @@ func (s *FederationSuite) TestNoAuth(c *check.C) {
 func (s *FederationSuite) TestBadAuth(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+arvadostest.WorkflowWithDefinitionYAMLUUID, nil)
 	req.Header.Set("Authorization", "Bearer aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa")
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusUnauthorized)
 	s.checkJSONErrorMatches(c, resp, `Not logged in`)
 }
@@ -146,7 +150,7 @@ func (s *FederationSuite) TestBadAuth(c *check.C) {
 func (s *FederationSuite) TestNoAccess(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+arvadostest.WorkflowWithDefinitionYAMLUUID, nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.SpectatorToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusNotFound)
 	s.checkJSONErrorMatches(c, resp, `.*not found`)
 }
@@ -154,7 +158,7 @@ func (s *FederationSuite) TestNoAccess(c *check.C) {
 func (s *FederationSuite) TestGetUnknownRemote(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+strings.Replace(arvadostest.WorkflowWithDefinitionYAMLUUID, "zzzzz-", "zz404-", 1), nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusNotFound)
 	s.checkJSONErrorMatches(c, resp, `.*no proxy available for cluster zz404`)
 }
@@ -166,7 +170,7 @@ func (s *FederationSuite) TestRemoteError(c *check.C) {
 
 	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+arvadostest.WorkflowWithDefinitionYAMLUUID, nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusBadGateway)
 	s.checkJSONErrorMatches(c, resp, `.*HTTP response to HTTPS client`)
 }
@@ -174,7 +178,7 @@ func (s *FederationSuite) TestRemoteError(c *check.C) {
 func (s *FederationSuite) TestGetRemoteWorkflow(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+arvadostest.WorkflowWithDefinitionYAMLUUID, nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var wf arvados.Workflow
 	c.Check(json.NewDecoder(resp.Body).Decode(&wf), check.IsNil)
@@ -185,7 +189,7 @@ func (s *FederationSuite) TestGetRemoteWorkflow(c *check.C) {
 func (s *FederationSuite) TestOptionsMethod(c *check.C) {
 	req := httptest.NewRequest("OPTIONS", "/arvados/v1/workflows/"+arvadostest.WorkflowWithDefinitionYAMLUUID, nil)
 	req.Header.Set("Origin", "https://example.com")
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	body, err := ioutil.ReadAll(resp.Body)
 	c.Check(err, check.IsNil)
@@ -201,7 +205,7 @@ func (s *FederationSuite) TestOptionsMethod(c *check.C) {
 
 func (s *FederationSuite) TestRemoteWithTokenInQuery(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+strings.Replace(arvadostest.WorkflowWithDefinitionYAMLUUID, "zzzzz-", "zmock-", 1)+"?api_token="+arvadostest.ActiveToken, nil)
-	s.testRequest(req)
+	s.testRequest(req).Result()
 	c.Assert(s.remoteMockRequests, check.HasLen, 1)
 	pr := s.remoteMockRequests[0]
 	// Token is salted and moved from query to Authorization header.
@@ -210,28 +214,51 @@ func (s *FederationSuite) TestRemoteWithTokenInQuery(c *check.C) {
 }
 
 func (s *FederationSuite) TestLocalTokenSalted(c *check.C) {
-	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+strings.Replace(arvadostest.WorkflowWithDefinitionYAMLUUID, "zzzzz-", "zmock-", 1), nil)
-	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	s.testRequest(req)
-	c.Assert(s.remoteMockRequests, check.HasLen, 1)
-	pr := s.remoteMockRequests[0]
-	// The salted token here has a "zzzzz-" UUID instead of a
-	// "ztest-" UUID because ztest's local database has the
-	// "zzzzz-" test fixtures. The "secret" part is HMAC(sha1,
-	// arvadostest.ActiveToken, "zmock") = "7fd3...".
-	c.Check(pr.Header.Get("Authorization"), check.Equals, "Bearer v2/zzzzz-gj3su-077z32aux8dg2s1/7fd31b61f39c0e82a4155592163218272cedacdc")
+	defer s.localServiceReturns404(c).Close()
+	for _, path := range []string{
+		// During the transition to the strongly typed
+		// controller implementation (#14287), workflows and
+		// collections test different code paths.
+		"/arvados/v1/workflows/" + strings.Replace(arvadostest.WorkflowWithDefinitionYAMLUUID, "zzzzz-", "zmock-", 1),
+		"/arvados/v1/collections/" + strings.Replace(arvadostest.UserAgreementCollection, "zzzzz-", "zmock-", 1),
+	} {
+		c.Log("testing path ", path)
+		s.remoteMockRequests = nil
+		req := httptest.NewRequest("GET", path, nil)
+		req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
+		s.testRequest(req).Result()
+		c.Assert(s.remoteMockRequests, check.HasLen, 1)
+		pr := s.remoteMockRequests[0]
+		// The salted token here has a "zzzzz-" UUID instead of a
+		// "ztest-" UUID because ztest's local database has the
+		// "zzzzz-" test fixtures. The "secret" part is HMAC(sha1,
+		// arvadostest.ActiveToken, "zmock") = "7fd3...".
+		c.Check(pr.Header.Get("Authorization"), check.Equals, "Bearer v2/zzzzz-gj3su-077z32aux8dg2s1/7fd31b61f39c0e82a4155592163218272cedacdc")
+	}
 }
 
 func (s *FederationSuite) TestRemoteTokenNotSalted(c *check.C) {
+	defer s.localServiceReturns404(c).Close()
 	// remoteToken can be any v1 token that doesn't appear in
 	// ztest's local db.
 	remoteToken := "abcdef00000000000000000000000000000000000000000000"
-	req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+strings.Replace(arvadostest.WorkflowWithDefinitionYAMLUUID, "zzzzz-", "zmock-", 1), nil)
-	req.Header.Set("Authorization", "Bearer "+remoteToken)
-	s.testRequest(req)
-	c.Assert(s.remoteMockRequests, check.HasLen, 1)
-	pr := s.remoteMockRequests[0]
-	c.Check(pr.Header.Get("Authorization"), check.Equals, "Bearer "+remoteToken)
+
+	for _, path := range []string{
+		// During the transition to the strongly typed
+		// controller implementation (#14287), workflows and
+		// collections test different code paths.
+		"/arvados/v1/workflows/" + strings.Replace(arvadostest.WorkflowWithDefinitionYAMLUUID, "zzzzz-", "zmock-", 1),
+		"/arvados/v1/collections/" + strings.Replace(arvadostest.UserAgreementCollection, "zzzzz-", "zmock-", 1),
+	} {
+		c.Log("testing path ", path)
+		s.remoteMockRequests = nil
+		req := httptest.NewRequest("GET", path, nil)
+		req.Header.Set("Authorization", "Bearer "+remoteToken)
+		s.testRequest(req).Result()
+		c.Assert(s.remoteMockRequests, check.HasLen, 1)
+		pr := s.remoteMockRequests[0]
+		c.Check(pr.Header.Get("Authorization"), check.Equals, "Bearer "+remoteToken)
+	}
 }
 
 func (s *FederationSuite) TestWorkflowCRUD(c *check.C) {
@@ -273,7 +300,7 @@ func (s *FederationSuite) TestWorkflowCRUD(c *check.C) {
 		req := httptest.NewRequest(method, "/arvados/v1/workflows/"+wf.UUID, strings.NewReader(form.Encode()))
 		req.Header.Set("Content-type", "application/x-www-form-urlencoded")
 		req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-		resp := s.testRequest(req)
+		resp := s.testRequest(req).Result()
 		s.checkResponseOK(c, resp)
 		err := json.NewDecoder(resp.Body).Decode(&wf)
 		c.Check(err, check.IsNil)
@@ -283,7 +310,7 @@ func (s *FederationSuite) TestWorkflowCRUD(c *check.C) {
 	{
 		req := httptest.NewRequest("DELETE", "/arvados/v1/workflows/"+wf.UUID, nil)
 		req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-		resp := s.testRequest(req)
+		resp := s.testRequest(req).Result()
 		s.checkResponseOK(c, resp)
 		err := json.NewDecoder(resp.Body).Decode(&wf)
 		c.Check(err, check.IsNil)
@@ -291,7 +318,7 @@ func (s *FederationSuite) TestWorkflowCRUD(c *check.C) {
 	{
 		req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+wf.UUID, nil)
 		req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-		resp := s.testRequest(req)
+		resp := s.testRequest(req).Result()
 		c.Check(resp.StatusCode, check.Equals, http.StatusNotFound)
 	}
 }
@@ -333,7 +360,15 @@ func (s *FederationSuite) localServiceHandler(c *check.C, h http.Handler) *https
 
 func (s *FederationSuite) localServiceReturns404(c *check.C) *httpserver.Server {
 	return s.localServiceHandler(c, http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) {
-		w.WriteHeader(404)
+		if req.URL.Path == "/arvados/v1/api_client_authorizations/current" {
+			if req.Header.Get("Authorization") == "Bearer "+arvadostest.ActiveToken {
+				json.NewEncoder(w).Encode(arvados.APIClientAuthorization{UUID: arvadostest.ActiveTokenUUID, APIToken: arvadostest.ActiveToken})
+			} else {
+				w.WriteHeader(http.StatusUnauthorized)
+			}
+		} else {
+			w.WriteHeader(404)
+		}
 	}))
 }
 
@@ -350,7 +385,7 @@ func (s *FederationSuite) TestGetLocalCollection(c *check.C) {
 
 	req := httptest.NewRequest("GET", "/arvados/v1/collections/"+arvadostest.UserAgreementCollection, nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var col arvados.Collection
@@ -367,7 +402,7 @@ func (s *FederationSuite) TestGetLocalCollection(c *check.C) {
 	}).Encode()))
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
 	req.Header.Set("Content-Type", "application/x-www-form-urlencoded; charset=UTF-8")
-	resp = s.testRequest(req)
+	resp = s.testRequest(req).Result()
 
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	col = arvados.Collection{}
@@ -383,7 +418,7 @@ func (s *FederationSuite) TestGetRemoteCollection(c *check.C) {
 
 	req := httptest.NewRequest("GET", "/arvados/v1/collections/"+arvadostest.UserAgreementCollection, nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var col arvados.Collection
 	c.Check(json.NewDecoder(resp.Body).Decode(&col), check.IsNil)
@@ -398,7 +433,7 @@ func (s *FederationSuite) TestGetRemoteCollectionError(c *check.C) {
 
 	req := httptest.NewRequest("GET", "/arvados/v1/collections/zzzzz-4zz18-fakefakefakefak", nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusNotFound)
 }
 
@@ -425,7 +460,7 @@ func (s *FederationSuite) TestGetLocalCollectionByPDH(c *check.C) {
 
 	req := httptest.NewRequest("GET", "/arvados/v1/collections/"+arvadostest.UserAgreementPDH, nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var col arvados.Collection
@@ -441,7 +476,7 @@ func (s *FederationSuite) TestGetRemoteCollectionByPDH(c *check.C) {
 
 	req := httptest.NewRequest("GET", "/arvados/v1/collections/"+arvadostest.UserAgreementPDH, nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 
@@ -459,7 +494,7 @@ func (s *FederationSuite) TestGetCollectionByPDHError(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/collections/99999999999999999999999999999999+99", nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
 
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	defer resp.Body.Close()
 
 	c.Check(resp.StatusCode, check.Equals, http.StatusNotFound)
@@ -498,7 +533,7 @@ func (s *FederationSuite) TestGetCollectionByPDHErrorBadHash(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/collections/99999999999999999999999999999999+99", nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
 
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	defer resp.Body.Close()
 
 	c.Check(resp.StatusCode, check.Equals, http.StatusNotFound)
@@ -514,7 +549,7 @@ func (s *FederationSuite) TestSaltedTokenGetCollectionByPDH(c *check.C) {
 
 	req := httptest.NewRequest("GET", "/arvados/v1/collections/"+arvadostest.UserAgreementPDH, nil)
 	req.Header.Set("Authorization", "Bearer v2/zzzzz-gj3su-077z32aux8dg2s1/282d7d172b6cfdce364c5ed12ddf7417b2d00065")
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var col arvados.Collection
@@ -535,7 +570,7 @@ func (s *FederationSuite) TestSaltedTokenGetCollectionByPDHError(c *check.C) {
 
 	req := httptest.NewRequest("GET", "/arvados/v1/collections/99999999999999999999999999999999+99", nil)
 	req.Header.Set("Authorization", "Bearer v2/zzzzz-gj3su-077z32aux8dg2s1/282d7d172b6cfdce364c5ed12ddf7417b2d00065")
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 
 	c.Check(resp.StatusCode, check.Equals, http.StatusNotFound)
 }
@@ -544,7 +579,7 @@ func (s *FederationSuite) TestGetRemoteContainerRequest(c *check.C) {
 	defer s.localServiceReturns404(c).Close()
 	req := httptest.NewRequest("GET", "/arvados/v1/container_requests/"+arvadostest.QueuedContainerRequestUUID, nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var cr arvados.ContainerRequest
 	c.Check(json.NewDecoder(resp.Body).Decode(&cr), check.IsNil)
@@ -559,7 +594,7 @@ func (s *FederationSuite) TestUpdateRemoteContainerRequest(c *check.C) {
 			strings.NewReader(fmt.Sprintf(`{"container_request": {"priority": %d}}`, pri)))
 		req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
 		req.Header.Set("Content-type", "application/json")
-		resp := s.testRequest(req)
+		resp := s.testRequest(req).Result()
 		c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 		var cr arvados.ContainerRequest
 		c.Check(json.NewDecoder(resp.Body).Decode(&cr), check.IsNil)
@@ -587,7 +622,7 @@ func (s *FederationSuite) TestCreateRemoteContainerRequest(c *check.C) {
 `))
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
 	req.Header.Set("Content-type", "application/json")
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var cr arvados.ContainerRequest
 	c.Check(json.NewDecoder(resp.Body).Decode(&cr), check.IsNil)
@@ -624,7 +659,7 @@ func (s *FederationSuite) TestCreateRemoteContainerRequestCheckRuntimeToken(c *c
 	s.testHandler.Cluster.NodeProfiles["*"] = np
 	s.testHandler.NodeProfile = &np
 
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var cr struct {
 		arvados.ContainerRequest `json:"container_request"`
@@ -655,7 +690,7 @@ func (s *FederationSuite) TestCreateRemoteContainerRequestCheckSetRuntimeToken(c
 `))
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
 	req.Header.Set("Content-type", "application/json")
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var cr struct {
 		arvados.ContainerRequest `json:"container_request"`
@@ -684,7 +719,7 @@ func (s *FederationSuite) TestCreateRemoteContainerRequestRuntimeTokenFromAuth(c
 `))
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveTokenV2+"/zzzzz-dz642-parentcontainer")
 	req.Header.Set("Content-type", "application/json")
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var cr struct {
 		arvados.ContainerRequest `json:"container_request"`
@@ -710,7 +745,7 @@ func (s *FederationSuite) TestCreateRemoteContainerRequestError(c *check.C) {
 `))
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
 	req.Header.Set("Content-type", "application/json")
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusNotFound)
 }
 
@@ -719,7 +754,7 @@ func (s *FederationSuite) TestGetRemoteContainer(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/containers/"+arvadostest.QueuedContainerUUID, nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
 	resp := s.testRequest(req)
-	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
+	c.Check(resp.Code, check.Equals, http.StatusOK)
 	var cn arvados.Container
 	c.Check(json.NewDecoder(resp.Body).Decode(&cn), check.IsNil)
 	c.Check(cn.UUID, check.Equals, arvadostest.QueuedContainerUUID)
@@ -730,10 +765,11 @@ func (s *FederationSuite) TestListRemoteContainer(c *check.C) {
 	req := httptest.NewRequest("GET", "/arvados/v1/containers?count=none&filters="+
 		url.QueryEscape(fmt.Sprintf(`[["uuid", "in", ["%v"]]]`, arvadostest.QueuedContainerUUID)), nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var cn arvados.ContainerList
 	c.Check(json.NewDecoder(resp.Body).Decode(&cn), check.IsNil)
+	c.Assert(cn.Items, check.HasLen, 1)
 	c.Check(cn.Items[0].UUID, check.Equals, arvadostest.QueuedContainerUUID)
 }
 
@@ -750,7 +786,7 @@ func (s *FederationSuite) TestListMultiRemoteContainers(c *check.C) {
 		url.QueryEscape(`["uuid", "command"]`)),
 		nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	var cn arvados.ContainerList
 	c.Check(json.NewDecoder(resp.Body).Decode(&cn), check.IsNil)
@@ -773,7 +809,7 @@ func (s *FederationSuite) TestListMultiRemoteContainerError(c *check.C) {
 		url.QueryEscape(`["uuid", "command"]`)),
 		nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusBadGateway)
 	s.checkJSONErrorMatches(c, resp, `error fetching from zhome \(404 Not Found\): EOF`)
 }
@@ -799,7 +835,7 @@ func (s *FederationSuite) TestListMultiRemoteContainersPaged(c *check.C) {
 			arvadostest.QueuedContainerUUID))),
 		nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	c.Check(callCount, check.Equals, 2)
 	var cn arvados.ContainerList
@@ -835,7 +871,7 @@ func (s *FederationSuite) TestListMultiRemoteContainersMissing(c *check.C) {
 			arvadostest.QueuedContainerUUID))),
 		nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusOK)
 	c.Check(callCount, check.Equals, 2)
 	var cn arvados.ContainerList
@@ -856,7 +892,7 @@ func (s *FederationSuite) TestListMultiRemoteContainerPageSizeError(c *check.C)
 			arvadostest.QueuedContainerUUID))),
 		nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusBadRequest)
 	s.checkJSONErrorMatches(c, resp, `Federated multi-object request for 2 objects which is more than max page size 1.`)
 }
@@ -867,7 +903,7 @@ func (s *FederationSuite) TestListMultiRemoteContainerLimitError(c *check.C) {
 			arvadostest.QueuedContainerUUID))),
 		nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusBadRequest)
 	s.checkJSONErrorMatches(c, resp, `Federated multi-object may not provide 'limit', 'offset' or 'order'.`)
 }
@@ -878,7 +914,7 @@ func (s *FederationSuite) TestListMultiRemoteContainerOffsetError(c *check.C) {
 			arvadostest.QueuedContainerUUID))),
 		nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusBadRequest)
 	s.checkJSONErrorMatches(c, resp, `Federated multi-object may not provide 'limit', 'offset' or 'order'.`)
 }
@@ -889,7 +925,7 @@ func (s *FederationSuite) TestListMultiRemoteContainerOrderError(c *check.C) {
 			arvadostest.QueuedContainerUUID))),
 		nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusBadRequest)
 	s.checkJSONErrorMatches(c, resp, `Federated multi-object may not provide 'limit', 'offset' or 'order'.`)
 }
@@ -901,7 +937,7 @@ func (s *FederationSuite) TestListMultiRemoteContainerSelectError(c *check.C) {
 		url.QueryEscape(`["command"]`)),
 		nil)
 	req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-	resp := s.testRequest(req)
+	resp := s.testRequest(req).Result()
 	c.Check(resp.StatusCode, check.Equals, http.StatusBadRequest)
 	s.checkJSONErrorMatches(c, resp, `Federated multi-object request must include 'uuid' in 'select'`)
 }
diff --git a/lib/controller/handler.go b/lib/controller/handler.go
index 53125ae55..c799b617f 100644
--- a/lib/controller/handler.go
+++ b/lib/controller/handler.go
@@ -8,13 +8,14 @@ import (
 	"context"
 	"database/sql"
 	"errors"
-	"net"
 	"net/http"
 	"net/url"
 	"strings"
 	"sync"
 	"time"
 
+	"git.curoverse.com/arvados.git/lib/controller/railsproxy"
+	"git.curoverse.com/arvados.git/lib/controller/router"
 	"git.curoverse.com/arvados.git/sdk/go/arvados"
 	"git.curoverse.com/arvados.git/sdk/go/health"
 	"git.curoverse.com/arvados.git/sdk/go/httpserver"
@@ -61,7 +62,7 @@ func (h *Handler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
 
 func (h *Handler) CheckHealth() error {
 	h.setupOnce.Do(h.setup)
-	_, _, err := findRailsAPI(h.Cluster, h.NodeProfile)
+	_, _, err := railsproxy.FindRailsAPI(h.Cluster, h.NodeProfile)
 	return err
 }
 
@@ -73,6 +74,13 @@ func (h *Handler) setup() {
 		Token:  h.Cluster.ManagementToken,
 		Prefix: "/_health/",
 	})
+
+	if h.Cluster.EnableBetaController14287 {
+		rtr := router.New(h.Cluster, h.NodeProfile)
+		mux.Handle("/arvados/v1/collections", rtr)
+		mux.Handle("/arvados/v1/collections/", rtr)
+	}
+
 	hs := http.NotFoundHandler()
 	hs = prepend(hs, h.proxyRailsAPI)
 	hs = h.setupProxyRemoteCluster(hs)
@@ -126,7 +134,7 @@ func prepend(next http.Handler, middleware middlewareFunc) http.Handler {
 }
 
 func (h *Handler) localClusterRequest(req *http.Request) (*http.Response, error) {
-	urlOut, insecure, err := findRailsAPI(h.Cluster, h.NodeProfile)
+	urlOut, insecure, err := railsproxy.FindRailsAPI(h.Cluster, h.NodeProfile)
 	if err != nil {
 		return nil, err
 	}
@@ -151,23 +159,3 @@ func (h *Handler) proxyRailsAPI(w http.ResponseWriter, req *http.Request, next h
 		httpserver.Logger(req).WithError(err).WithField("bytesCopied", n).Error("error copying response body")
 	}
 }
-
-// For now, findRailsAPI always uses the rails API running on this
-// node.
-func findRailsAPI(cluster *arvados.Cluster, np *arvados.NodeProfile) (*url.URL, bool, error) {
-	hostport := np.RailsAPI.Listen
-	if len(hostport) > 1 && hostport[0] == ':' && strings.TrimRight(hostport[1:], "0123456789") == "" {
-		// ":12345" => connect to indicated port on localhost
-		hostport = "localhost" + hostport
-	} else if _, _, err := net.SplitHostPort(hostport); err == nil {
-		// "[::1]:12345" => connect to indicated address & port
-	} else {
-		return nil, false, err
-	}
-	proto := "http"
-	if np.RailsAPI.TLS {
-		proto = "https"
-	}
-	url, err := url.Parse(proto + "://" + hostport)
-	return url, np.RailsAPI.Insecure, err
-}
diff --git a/lib/controller/handler_test.go b/lib/controller/handler_test.go
index 96110ea85..7041d3504 100644
--- a/lib/controller/handler_test.go
+++ b/lib/controller/handler_test.go
@@ -22,9 +22,13 @@ import (
 	check "gopkg.in/check.v1"
 )
 
+var enableBetaController14287 bool
+
 // Gocheck boilerplate
 func Test(t *testing.T) {
-	check.TestingT(t)
+	for _, enableBetaController14287 = range []bool{false, true} {
+		check.TestingT(t)
+	}
 }
 
 var _ = check.Suite(&HandlerSuite{})
@@ -48,6 +52,7 @@ func (s *HandlerSuite) SetUpTest(c *check.C) {
 				RailsAPI:   arvados.SystemServiceInstance{Listen: os.Getenv("ARVADOS_TEST_API_HOST"), TLS: true, Insecure: true},
 			},
 		},
+		EnableBetaController14287: enableBetaController14287,
 	}
 	node := s.cluster.NodeProfiles["*"]
 	s.handler = newHandler(s.ctx, s.cluster, &node, "")
diff --git a/lib/controller/proxy.go b/lib/controller/proxy.go
index c0b94c2b5..9eac9362c 100644
--- a/lib/controller/proxy.go
+++ b/lib/controller/proxy.go
@@ -25,20 +25,23 @@ func (h HTTPError) Error() string {
 	return h.Message
 }
 
-// headers that shouldn't be forwarded when proxying. See
-// https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers
 var dropHeaders = map[string]bool{
+	// Headers that shouldn't be forwarded when proxying. See
+	// https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers
 	"Connection":          true,
 	"Keep-Alive":          true,
 	"Proxy-Authenticate":  true,
 	"Proxy-Authorization": true,
-	// this line makes gofmt 1.10 and 1.11 agree
-	"TE":                true,
-	"Trailer":           true,
-	"Transfer-Encoding": true, // *-Encoding headers interfer with Go's automatic compression/decompression
-	"Content-Encoding":  true,
+	// (comment/space here makes gofmt1.10 agree with gofmt1.11)
+	"TE":      true,
+	"Trailer": true,
+	"Upgrade": true,
+
+	// Headers that would interfere with Go's automatic
+	// compression/decompression if we forwarded them.
 	"Accept-Encoding":   true,
-	"Upgrade":           true,
+	"Content-Encoding":  true,
+	"Transfer-Encoding": true,
 }
 
 type ResponseFilter func(*http.Response, error) (*http.Response, error)
diff --git a/lib/controller/railsproxy/railsproxy.go b/lib/controller/railsproxy/railsproxy.go
new file mode 100644
index 000000000..db1a3f5e6
--- /dev/null
+++ b/lib/controller/railsproxy/railsproxy.go
@@ -0,0 +1,56 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+// Package railsproxy implements Arvados APIs by proxying to the
+// RailsAPI server on the local machine.
+package railsproxy
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"net"
+	"net/url"
+	"strings"
+
+	"git.curoverse.com/arvados.git/lib/controller/rpc"
+	"git.curoverse.com/arvados.git/sdk/go/arvados"
+	"git.curoverse.com/arvados.git/sdk/go/auth"
+)
+
+// For now, FindRailsAPI always uses the rails API running on this
+// node.
+func FindRailsAPI(cluster *arvados.Cluster, np *arvados.NodeProfile) (*url.URL, bool, error) {
+	hostport := np.RailsAPI.Listen
+	if len(hostport) > 1 && hostport[0] == ':' && strings.TrimRight(hostport[1:], "0123456789") == "" {
+		// ":12345" => connect to indicated port on localhost
+		hostport = "localhost" + hostport
+	} else if _, _, err := net.SplitHostPort(hostport); err == nil {
+		// "[::1]:12345" => connect to indicated address & port
+	} else {
+		return nil, false, err
+	}
+	proto := "http"
+	if np.RailsAPI.TLS {
+		proto = "https"
+	}
+	url, err := url.Parse(proto + "://" + hostport)
+	return url, np.RailsAPI.Insecure, err
+}
+
+func NewConn(cluster *arvados.Cluster, np *arvados.NodeProfile) *rpc.Conn {
+	url, insecure, err := FindRailsAPI(cluster, np)
+	if err != nil {
+		panic(fmt.Sprintf("NodeProfile RailsAPI %#v: %s", np.RailsAPI, err))
+	}
+	return rpc.NewConn(cluster.ClusterID, url, insecure, provideIncomingToken)
+}
+
+func provideIncomingToken(ctx context.Context) ([]string, error) {
+	incoming, ok := ctx.Value(auth.ContextKeyCredentials).(*auth.Credentials)
+	if !ok {
+		return nil, errors.New("no token provided")
+	}
+	return incoming.Tokens, nil
+}
diff --git a/lib/controller/router/error.go b/lib/controller/router/error.go
new file mode 100644
index 000000000..6db5f3155
--- /dev/null
+++ b/lib/controller/router/error.go
@@ -0,0 +1,18 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package router
+
+type errorWithStatus struct {
+	code int
+	error
+}
+
+func (err errorWithStatus) HTTPStatus() int {
+	return err.code
+}
+
+func httpError(code int, err error) error {
+	return errorWithStatus{code: code, error: err}
+}
diff --git a/lib/controller/router/request.go b/lib/controller/router/request.go
new file mode 100644
index 000000000..67d4e0ffb
--- /dev/null
+++ b/lib/controller/router/request.go
@@ -0,0 +1,112 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package router
+
+import (
+	"encoding/json"
+	"io"
+	"mime"
+	"net/http"
+	"strconv"
+	"strings"
+
+	"github.com/julienschmidt/httprouter"
+)
+
+// Parse req as an Arvados V1 API request and return the request
+// parameters.
+//
+// If the request has a parameter whose name is attrsKey (e.g.,
+// "collection"), it is renamed to "attrs".
+func (rtr *router) loadRequestParams(req *http.Request, attrsKey string) (map[string]interface{}, error) {
+	err := req.ParseForm()
+	if err != nil {
+		return nil, httpError(http.StatusBadRequest, err)
+	}
+	params := map[string]interface{}{}
+	for k, values := range req.Form {
+		for _, v := range values {
+			switch {
+			case v == "null" || v == "":
+				params[k] = nil
+			case strings.HasPrefix(v, "["):
+				var j []interface{}
+				err := json.Unmarshal([]byte(v), &j)
+				if err != nil {
+					return nil, err
+				}
+				params[k] = j
+			case strings.HasPrefix(v, "{"):
+				var j map[string]interface{}
+				err := json.Unmarshal([]byte(v), &j)
+				if err != nil {
+					return nil, err
+				}
+				params[k] = j
+			case strings.HasPrefix(v, "\""):
+				var j string
+				err := json.Unmarshal([]byte(v), &j)
+				if err != nil {
+					return nil, err
+				}
+				params[k] = j
+			case k == "limit" || k == "offset":
+				params[k], err = strconv.ParseInt(v, 10, 64)
+				if err != nil {
+					return nil, err
+				}
+			default:
+				params[k] = v
+			}
+			// TODO: Need to accept "?foo[]=bar&foo[]=baz"
+			// as foo=["bar","baz"]?
+		}
+	}
+	if ct, _, err := mime.ParseMediaType(req.Header.Get("Content-Type")); err != nil && ct == "application/json" {
+		jsonParams := map[string]interface{}{}
+		err := json.NewDecoder(req.Body).Decode(jsonParams)
+		if err != nil {
+			return nil, httpError(http.StatusBadRequest, err)
+		}
+		for k, v := range jsonParams {
+			params[k] = v
+		}
+		if attrsKey != "" && params[attrsKey] == nil {
+			// Copy top-level parameters from JSON request
+			// body into params[attrsKey]. Some SDKs rely
+			// on this Rails API feature; see
+			// https://api.rubyonrails.org/v5.2.1/classes/ActionController/ParamsWrapper.html
+			params[attrsKey] = jsonParams
+		}
+	}
+
+	routeParams, _ := req.Context().Value(httprouter.ParamsKey).(httprouter.Params)
+	for _, p := range routeParams {
+		params[p.Key] = p.Value
+	}
+
+	if v, ok := params[attrsKey]; ok && attrsKey != "" {
+		params["attrs"] = v
+		delete(params, attrsKey)
+	}
+	return params, nil
+}
+
+// Copy src to dst, using json as an intermediate format in order to
+// invoke src's json-marshaling and dst's json-unmarshaling behaviors.
+func (rtr *router) transcode(src interface{}, dst interface{}) error {
+	var errw error
+	pr, pw := io.Pipe()
+	go func() {
+		defer pw.Close()
+		errw = json.NewEncoder(pw).Encode(src)
+	}()
+	defer pr.Close()
+	err := json.NewDecoder(pr).Decode(dst)
+	if errw != nil {
+		return errw
+	}
+	return err
+}
diff --git a/lib/controller/router/response.go b/lib/controller/router/response.go
new file mode 100644
index 000000000..65e0159fa
--- /dev/null
+++ b/lib/controller/router/response.go
@@ -0,0 +1,53 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package router
+
+import (
+	"encoding/json"
+	"net/http"
+
+	"git.curoverse.com/arvados.git/sdk/go/arvados"
+	"git.curoverse.com/arvados.git/sdk/go/httpserver"
+)
+
+type responseOptions struct {
+	Select []string
+}
+
+func (rtr *router) responseOptions(opts interface{}) (responseOptions, error) {
+	var rOpts responseOptions
+	switch opts := opts.(type) {
+	case *arvados.GetOptions:
+		rOpts.Select = opts.Select
+	}
+	return rOpts, nil
+}
+
+func (rtr *router) sendResponse(w http.ResponseWriter, resp interface{}, opts responseOptions) {
+	var tmp map[string]interface{}
+	err := rtr.transcode(resp, &tmp)
+	if err != nil {
+		rtr.sendError(w, err)
+		return
+	}
+	if len(opts.Select) > 0 {
+		selected := map[string]interface{}{}
+		for _, attr := range opts.Select {
+			if v, ok := tmp[attr]; ok {
+				selected[attr] = v
+			}
+		}
+		tmp = selected
+	}
+	json.NewEncoder(w).Encode(tmp)
+}
+
+func (rtr *router) sendError(w http.ResponseWriter, err error) {
+	code := http.StatusInternalServerError
+	if err, ok := err.(interface{ HTTPStatus() int }); ok {
+		code = err.HTTPStatus()
+	}
+	httpserver.Error(w, err.Error(), code)
+}
diff --git a/lib/controller/router/router.go b/lib/controller/router/router.go
new file mode 100644
index 000000000..4a6f9b5af
--- /dev/null
+++ b/lib/controller/router/router.go
@@ -0,0 +1,210 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package router
+
+import (
+	"context"
+	"net/http"
+
+	"git.curoverse.com/arvados.git/lib/controller/federation"
+	"git.curoverse.com/arvados.git/sdk/go/arvados"
+	"git.curoverse.com/arvados.git/sdk/go/auth"
+	"git.curoverse.com/arvados.git/sdk/go/ctxlog"
+	"github.com/julienschmidt/httprouter"
+)
+
+type router struct {
+	mux *httprouter.Router
+	fed federation.Interface
+}
+
+func New(cluster *arvados.Cluster, np *arvados.NodeProfile) *router {
+	rtr := &router{
+		mux: httprouter.New(),
+		fed: federation.New(cluster, np),
+	}
+	rtr.addRoutes(cluster)
+	return rtr
+}
+
+func (rtr *router) addRoutes(cluster *arvados.Cluster) {
+	for _, route := range []struct {
+		endpoint    arvados.APIEndpoint
+		defaultOpts func() interface{}
+		exec        func(ctx context.Context, opts interface{}) (interface{}, error)
+	}{
+		{
+			arvados.EndpointCollectionCreate,
+			func() interface{} { return &arvados.CreateOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.CollectionCreate(ctx, *opts.(*arvados.CreateOptions))
+			},
+		},
+		{
+			arvados.EndpointCollectionUpdate,
+			func() interface{} { return &arvados.UpdateOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.CollectionUpdate(ctx, *opts.(*arvados.UpdateOptions))
+			},
+		},
+		{
+			arvados.EndpointCollectionGet,
+			func() interface{} { return &arvados.GetOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.CollectionGet(ctx, *opts.(*arvados.GetOptions))
+			},
+		},
+		{
+			arvados.EndpointCollectionList,
+			func() interface{} { return &arvados.ListOptions{Limit: -1} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.CollectionList(ctx, *opts.(*arvados.ListOptions))
+			},
+		},
+		{
+			arvados.EndpointCollectionDelete,
+			func() interface{} { return &arvados.DeleteOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.CollectionDelete(ctx, *opts.(*arvados.DeleteOptions))
+			},
+		},
+		{
+			arvados.EndpointContainerCreate,
+			func() interface{} { return &arvados.CreateOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.ContainerCreate(ctx, *opts.(*arvados.CreateOptions))
+			},
+		},
+		{
+			arvados.EndpointContainerUpdate,
+			func() interface{} { return &arvados.UpdateOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.ContainerUpdate(ctx, *opts.(*arvados.UpdateOptions))
+			},
+		},
+		{
+			arvados.EndpointContainerGet,
+			func() interface{} { return &arvados.GetOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.ContainerGet(ctx, *opts.(*arvados.GetOptions))
+			},
+		},
+		{
+			arvados.EndpointContainerList,
+			func() interface{} { return &arvados.ListOptions{Limit: -1} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.ContainerList(ctx, *opts.(*arvados.ListOptions))
+			},
+		},
+		{
+			arvados.EndpointContainerDelete,
+			func() interface{} { return &arvados.DeleteOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.ContainerDelete(ctx, *opts.(*arvados.DeleteOptions))
+			},
+		},
+		{
+			arvados.EndpointContainerLock,
+			func() interface{} {
+				return &arvados.GetOptions{Select: []string{"uuid", "state", "priority", "auth_uuid", "locked_by_uuid"}}
+			},
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.ContainerLock(ctx, *opts.(*arvados.GetOptions))
+			},
+		},
+		{
+			arvados.EndpointContainerUnlock,
+			func() interface{} {
+				return &arvados.GetOptions{Select: []string{"uuid", "state", "priority", "auth_uuid", "locked_by_uuid"}}
+			},
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.ContainerUnlock(ctx, *opts.(*arvados.GetOptions))
+			},
+		},
+		{
+			arvados.EndpointSpecimenCreate,
+			func() interface{} { return &arvados.CreateOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.SpecimenCreate(ctx, *opts.(*arvados.CreateOptions))
+			},
+		},
+		{
+			arvados.EndpointSpecimenUpdate,
+			func() interface{} { return &arvados.UpdateOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.SpecimenUpdate(ctx, *opts.(*arvados.UpdateOptions))
+			},
+		},
+		{
+			arvados.EndpointSpecimenGet,
+			func() interface{} { return &arvados.GetOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.SpecimenGet(ctx, *opts.(*arvados.GetOptions))
+			},
+		},
+		{
+			arvados.EndpointSpecimenList,
+			func() interface{} { return &arvados.ListOptions{Limit: -1} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.SpecimenList(ctx, *opts.(*arvados.ListOptions))
+			},
+		},
+		{
+			arvados.EndpointSpecimenDelete,
+			func() interface{} { return &arvados.DeleteOptions{} },
+			func(ctx context.Context, opts interface{}) (interface{}, error) {
+				return rtr.fed.SpecimenDelete(ctx, *opts.(*arvados.DeleteOptions))
+			},
+		},
+	} {
+		route := route
+		methods := []string{route.endpoint.Method}
+		if route.endpoint.Method == "PATCH" {
+			methods = append(methods, "PUT")
+		}
+		for _, method := range methods {
+			rtr.mux.HandlerFunc(method, "/"+route.endpoint.Path, func(w http.ResponseWriter, req *http.Request) {
+				params, err := rtr.loadRequestParams(req, route.endpoint.AttrsKey)
+				if err != nil {
+					rtr.sendError(w, err)
+					return
+				}
+				opts := route.defaultOpts()
+				err = rtr.transcode(params, opts)
+				if err != nil {
+					rtr.sendError(w, err)
+					return
+				}
+				respOpts, err := rtr.responseOptions(opts)
+				if err != nil {
+					rtr.sendError(w, err)
+					return
+				}
+
+				creds := auth.CredentialsFromRequest(req)
+				ctx := req.Context()
+				ctx = context.WithValue(ctx, auth.ContextKeyCredentials, creds)
+				ctx = arvados.ContextWithRequestID(ctx, req.Header.Get("X-Request-Id"))
+				resp, err := route.exec(ctx, opts)
+				if err != nil {
+					ctxlog.FromContext(ctx).WithError(err).Infof("returning error response for %#v", err)
+					rtr.sendError(w, err)
+					return
+				}
+				rtr.sendResponse(w, resp, respOpts)
+			})
+		}
+	}
+}
+
+func (rtr *router) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	r.ParseForm()
+	if m := r.FormValue("_method"); m != "" {
+		r2 := *r
+		r = &r2
+		r.Method = m
+	}
+	rtr.mux.ServeHTTP(w, r)
+}
diff --git a/lib/controller/router/router_test.go b/lib/controller/router/router_test.go
new file mode 100644
index 000000000..97710d265
--- /dev/null
+++ b/lib/controller/router/router_test.go
@@ -0,0 +1,127 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package router
+
+import (
+	"encoding/json"
+	"io"
+	"net/http"
+	"net/http/httptest"
+	"os"
+	"testing"
+
+	"git.curoverse.com/arvados.git/sdk/go/arvados"
+	"git.curoverse.com/arvados.git/sdk/go/arvadostest"
+	check "gopkg.in/check.v1"
+)
+
+// Gocheck boilerplate
+func Test(t *testing.T) {
+	check.TestingT(t)
+}
+
+var _ = check.Suite(&RouterSuite{})
+
+type RouterSuite struct {
+	rtr *router
+}
+
+func (s *RouterSuite) SetUpTest(c *check.C) {
+	s.rtr = New(&arvados.Cluster{}, &arvados.NodeProfile{RailsAPI: arvados.SystemServiceInstance{Listen: os.Getenv("ARVADOS_TEST_API_HOST"), TLS: true, Insecure: true}})
+}
+
+func (s *RouterSuite) TearDownTest(c *check.C) {
+	err := arvados.NewClientFromEnv().RequestAndDecode(nil, "POST", "database/reset", nil, nil)
+	c.Check(err, check.IsNil)
+}
+
+func (s *RouterSuite) doRequest(c *check.C, token, method, path string, hdrs http.Header, body io.Reader) (*http.Request, *httptest.ResponseRecorder, map[string]interface{}) {
+	req := httptest.NewRequest(method, path, body)
+	for k, v := range hdrs {
+		req.Header[k] = v
+	}
+	req.Header.Set("Authorization", "Bearer "+token)
+	rw := httptest.NewRecorder()
+	s.rtr.ServeHTTP(rw, req)
+	c.Logf("response body: %s", rw.Body.String())
+	var jresp map[string]interface{}
+	err := json.Unmarshal(rw.Body.Bytes(), &jresp)
+	c.Check(err, check.IsNil)
+	return req, rw, jresp
+}
+
+func (s *RouterSuite) TestContainerList(c *check.C) {
+	token := arvadostest.ActiveTokenV2
+
+	_, rw, jresp := s.doRequest(c, token, "GET", `/arvados/v1/containers?limit=0`, nil, nil)
+	c.Check(rw.Code, check.Equals, http.StatusOK)
+	c.Check(jresp["items_available"], check.FitsTypeOf, float64(0))
+	c.Check(jresp["items_available"].(float64) > 2, check.Equals, true)
+	c.Check(jresp["items"], check.HasLen, 0)
+
+	_, rw, jresp = s.doRequest(c, token, "GET", `/arvados/v1/containers?limit=2&select=["uuid","command"]`, nil, nil)
+	c.Check(rw.Code, check.Equals, http.StatusOK)
+	c.Check(jresp["items_available"], check.FitsTypeOf, float64(0))
+	c.Check(jresp["items_available"].(float64) > 2, check.Equals, true)
+	c.Check(jresp["items"], check.HasLen, 2)
+	item0 := jresp["items"].([]interface{})[0].(map[string]interface{})
+	c.Check(item0["uuid"], check.HasLen, 27)
+	c.Check(item0["command"], check.FitsTypeOf, []interface{}{})
+	c.Check(item0["command"].([]interface{})[0], check.FitsTypeOf, "")
+	c.Check(item0["mounts"], check.IsNil)
+
+	_, rw, jresp = s.doRequest(c, token, "GET", `/arvados/v1/containers`, nil, nil)
+	c.Check(rw.Code, check.Equals, http.StatusOK)
+	c.Check(jresp["items_available"], check.FitsTypeOf, float64(0))
+	c.Check(jresp["items_available"].(float64) > 2, check.Equals, true)
+	avail := int(jresp["items_available"].(float64))
+	c.Check(jresp["items"], check.HasLen, avail)
+	item0 = jresp["items"].([]interface{})[0].(map[string]interface{})
+	c.Check(item0["uuid"], check.HasLen, 27)
+	c.Check(item0["command"], check.FitsTypeOf, []interface{}{})
+	c.Check(item0["command"].([]interface{})[0], check.FitsTypeOf, "")
+	c.Check(item0["mounts"], check.NotNil)
+}
+
+func (s *RouterSuite) TestContainerLock(c *check.C) {
+	uuid := arvadostest.QueuedContainerUUID
+	token := arvadostest.ActiveTokenV2
+	_, rw, jresp := s.doRequest(c, token, "POST", "/arvados/v1/containers/"+uuid+"/lock", nil, nil)
+	c.Check(rw.Code, check.Equals, http.StatusOK)
+	c.Check(jresp["uuid"], check.HasLen, 27)
+	c.Check(jresp["state"], check.Equals, "Locked")
+	_, rw, jresp = s.doRequest(c, token, "POST", "/arvados/v1/containers/"+uuid+"/lock", nil, nil)
+	c.Check(rw.Code, check.Equals, http.StatusUnprocessableEntity)
+	c.Check(rw.Body.String(), check.Not(check.Matches), `.*"uuid":.*`)
+	_, rw, jresp = s.doRequest(c, token, "POST", "/arvados/v1/containers/"+uuid+"/unlock", nil, nil)
+	c.Check(rw.Code, check.Equals, http.StatusOK)
+	c.Check(jresp["uuid"], check.HasLen, 27)
+	c.Check(jresp["state"], check.Equals, "Queued")
+	c.Check(jresp["environment"], check.IsNil)
+	_, rw, jresp = s.doRequest(c, token, "POST", "/arvados/v1/containers/"+uuid+"/unlock", nil, nil)
+	c.Check(rw.Code, check.Equals, http.StatusUnprocessableEntity)
+	c.Check(jresp["uuid"], check.IsNil)
+}
+
+func (s *RouterSuite) TestSelectParam(c *check.C) {
+	uuid := arvadostest.QueuedContainerUUID
+	token := arvadostest.ActiveTokenV2
+	for _, sel := range [][]string{
+		{"uuid", "command"},
+		{"uuid", "command", "uuid"},
+		{"", "command", "uuid"},
+	} {
+		j, err := json.Marshal(sel)
+		c.Assert(err, check.IsNil)
+		_, rw, resp := s.doRequest(c, token, "GET", "/arvados/v1/containers/"+uuid+"?select="+string(j), nil, nil)
+		c.Check(rw.Code, check.Equals, http.StatusOK)
+
+		c.Check(resp["uuid"], check.HasLen, 27)
+		c.Check(resp["command"], check.HasLen, 2)
+		c.Check(resp["mounts"], check.IsNil)
+		_, hasMounts := resp["mounts"]
+		c.Check(hasMounts, check.Equals, false)
+	}
+}
diff --git a/lib/controller/rpc/conn.go b/lib/controller/rpc/conn.go
new file mode 100644
index 000000000..7c23ed170
--- /dev/null
+++ b/lib/controller/rpc/conn.go
@@ -0,0 +1,234 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package rpc
+
+import (
+	"context"
+	"crypto/tls"
+	"encoding/json"
+	"fmt"
+	"io"
+	"net"
+	"net/http"
+	"net/url"
+	"strings"
+	"time"
+
+	"git.curoverse.com/arvados.git/sdk/go/arvados"
+)
+
+type contextKey string
+
+const ContextKeyCredentials contextKey = "credentials"
+
+type TokenProvider func(context.Context) ([]string, error)
+
+type Conn struct {
+	clusterID     string
+	httpClient    http.Client
+	baseURL       url.URL
+	tokenProvider TokenProvider
+}
+
+func NewConn(clusterID string, url *url.URL, insecure bool, tp TokenProvider) *Conn {
+	transport := http.DefaultTransport
+	if insecure {
+		// It's not safe to copy *http.DefaultTransport
+		// because it has a mutex (which might be locked)
+		// protecting a private map (which might not be nil).
+		// So we build our own, using the Go 1.12 default
+		// values, ignoring any changes the application has
+		// made to http.DefaultTransport.
+		transport = &http.Transport{
+			DialContext: (&net.Dialer{
+				Timeout:   30 * time.Second,
+				KeepAlive: 30 * time.Second,
+				DualStack: true,
+			}).DialContext,
+			MaxIdleConns:          100,
+			IdleConnTimeout:       90 * time.Second,
+			TLSHandshakeTimeout:   10 * time.Second,
+			ExpectContinueTimeout: 1 * time.Second,
+			TLSClientConfig:       &tls.Config{InsecureSkipVerify: true},
+		}
+	}
+	return &Conn{
+		clusterID:     clusterID,
+		httpClient:    http.Client{Transport: transport},
+		baseURL:       *url,
+		tokenProvider: tp,
+	}
+}
+
+func (conn *Conn) requestAndDecode(ctx context.Context, dst interface{}, ep arvados.APIEndpoint, body io.Reader, opts interface{}) error {
+	aClient := arvados.Client{
+		Client:  &conn.httpClient,
+		Scheme:  conn.baseURL.Scheme,
+		APIHost: conn.baseURL.Host,
+	}
+	tokens, err := conn.tokenProvider(ctx)
+	if err != nil {
+		return err
+	} else if len(tokens) == 0 {
+		return fmt.Errorf("bug: token provider returned no tokens and no error")
+	}
+	ctx = context.WithValue(ctx, "Authorization", "Bearer "+tokens[0])
+
+	// Encode opts to JSON and decode from there to a
+	// map[string]interface{}, so we can munge the query params
+	// using the JSON key names specified by opts' struct tags.
+	j, err := json.Marshal(opts)
+	if err != nil {
+		return fmt.Errorf("%T: requestAndDecode: Marshal opts: %s", conn, err)
+	}
+	var params map[string]interface{}
+	err = json.Unmarshal(j, &params)
+	if err != nil {
+		return fmt.Errorf("%T: requestAndDecode: Unmarshal opts: %s", conn, err)
+	}
+	if attrs, ok := params["attrs"]; ok && ep.AttrsKey != "" {
+		params[ep.AttrsKey] = attrs
+		delete(params, "attrs")
+	}
+	if limit, ok := params["limit"].(float64); ok && limit < 0 {
+		// Negative limit means "not specified" here, but some
+		// servers/versions do not accept that, so we need to
+		// remove it entirely.
+		delete(params, "limit")
+	}
+	path := ep.Path
+	if strings.Contains(ep.Path, "/:uuid") {
+		uuid, _ := params["uuid"].(string)
+		path = strings.Replace(path, "/:uuid", "/"+uuid, 1)
+		delete(params, "uuid")
+	}
+	return aClient.RequestAndDecodeContext(ctx, dst, ep.Method, path, body, params)
+}
+
+func (conn *Conn) CollectionCreate(ctx context.Context, options arvados.CreateOptions) (arvados.Collection, error) {
+	ep := arvados.EndpointCollectionCreate
+	var resp arvados.Collection
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) CollectionUpdate(ctx context.Context, options arvados.UpdateOptions) (arvados.Collection, error) {
+	ep := arvados.EndpointCollectionUpdate
+	var resp arvados.Collection
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) CollectionGet(ctx context.Context, options arvados.GetOptions) (arvados.Collection, error) {
+	ep := arvados.EndpointCollectionGet
+	var resp arvados.Collection
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) CollectionList(ctx context.Context, options arvados.ListOptions) (arvados.CollectionList, error) {
+	ep := arvados.EndpointCollectionList
+	var resp arvados.CollectionList
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) CollectionDelete(ctx context.Context, options arvados.DeleteOptions) (arvados.Collection, error) {
+	ep := arvados.EndpointCollectionDelete
+	var resp arvados.Collection
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) ContainerCreate(ctx context.Context, options arvados.CreateOptions) (arvados.Container, error) {
+	ep := arvados.EndpointContainerCreate
+	var resp arvados.Container
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) ContainerUpdate(ctx context.Context, options arvados.UpdateOptions) (arvados.Container, error) {
+	ep := arvados.EndpointContainerUpdate
+	var resp arvados.Container
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) ContainerGet(ctx context.Context, options arvados.GetOptions) (arvados.Container, error) {
+	ep := arvados.EndpointContainerGet
+	var resp arvados.Container
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) ContainerList(ctx context.Context, options arvados.ListOptions) (arvados.ContainerList, error) {
+	ep := arvados.EndpointContainerList
+	var resp arvados.ContainerList
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) ContainerDelete(ctx context.Context, options arvados.DeleteOptions) (arvados.Container, error) {
+	ep := arvados.EndpointContainerDelete
+	var resp arvados.Container
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) ContainerLock(ctx context.Context, options arvados.GetOptions) (arvados.Container, error) {
+	ep := arvados.EndpointContainerLock
+	var resp arvados.Container
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) ContainerUnlock(ctx context.Context, options arvados.GetOptions) (arvados.Container, error) {
+	ep := arvados.EndpointContainerUnlock
+	var resp arvados.Container
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) SpecimenCreate(ctx context.Context, options arvados.CreateOptions) (arvados.Specimen, error) {
+	ep := arvados.EndpointSpecimenCreate
+	var resp arvados.Specimen
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) SpecimenUpdate(ctx context.Context, options arvados.UpdateOptions) (arvados.Specimen, error) {
+	ep := arvados.EndpointSpecimenUpdate
+	var resp arvados.Specimen
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) SpecimenGet(ctx context.Context, options arvados.GetOptions) (arvados.Specimen, error) {
+	ep := arvados.EndpointSpecimenGet
+	var resp arvados.Specimen
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) SpecimenList(ctx context.Context, options arvados.ListOptions) (arvados.SpecimenList, error) {
+	ep := arvados.EndpointSpecimenList
+	var resp arvados.SpecimenList
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) SpecimenDelete(ctx context.Context, options arvados.DeleteOptions) (arvados.Specimen, error) {
+	ep := arvados.EndpointSpecimenDelete
+	var resp arvados.Specimen
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
+
+func (conn *Conn) APIClientAuthorizationCurrent(ctx context.Context, options arvados.GetOptions) (arvados.APIClientAuthorization, error) {
+	ep := arvados.EndpointAPIClientAuthorizationCurrent
+	var resp arvados.APIClientAuthorization
+	err := conn.requestAndDecode(ctx, &resp, ep, nil, options)
+	return resp, err
+}
diff --git a/lib/controller/rpc/conn_test.go b/lib/controller/rpc/conn_test.go
new file mode 100644
index 000000000..9c2b70dce
--- /dev/null
+++ b/lib/controller/rpc/conn_test.go
@@ -0,0 +1,75 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package rpc
+
+import (
+	"context"
+	"net/url"
+	"os"
+	"testing"
+
+	"git.curoverse.com/arvados.git/sdk/go/arvados"
+	"git.curoverse.com/arvados.git/sdk/go/arvadostest"
+	"git.curoverse.com/arvados.git/sdk/go/ctxlog"
+	"github.com/sirupsen/logrus"
+	check "gopkg.in/check.v1"
+)
+
+// Gocheck boilerplate
+func Test(t *testing.T) {
+	check.TestingT(t)
+}
+
+var _ = check.Suite(&RPCSuite{})
+
+type RPCSuite struct {
+	log  logrus.FieldLogger
+	ctx  context.Context
+	conn *Conn
+}
+
+func (s *RPCSuite) SetUpTest(c *check.C) {
+	ctx := ctxlog.Context(context.Background(), ctxlog.TestLogger(c))
+	s.ctx = ContextWithTokens(ctx, []string{arvadostest.ActiveToken})
+	s.conn = NewConn("zzzzz", &url.URL{Scheme: "https", Host: os.Getenv("ARVADOS_TEST_API_HOST")}, true)
+}
+
+func (s *RPCSuite) TestCollectionCreate(c *check.C) {
+	coll, err := s.conn.CollectionCreate(s.ctx, arvados.CreateOptions{Attrs: map[string]interface{}{
+		"owner_uuid":         arvadostest.ActiveUserUUID,
+		"portable_data_hash": "d41d8cd98f00b204e9800998ecf8427e+0",
+	}})
+	c.Check(err, check.IsNil)
+	c.Check(coll.UUID, check.HasLen, 27)
+}
+
+func (s *RPCSuite) TestSpecimenCRUD(c *check.C) {
+	sp, err := s.conn.SpecimenCreate(s.ctx, arvados.CreateOptions{Attrs: map[string]interface{}{
+		"owner_uuid": arvadostest.ActiveUserUUID,
+		"properties": map[string]string{"foo": "bar"},
+	}})
+	c.Check(err, check.IsNil)
+	c.Check(sp.UUID, check.HasLen, 27)
+	c.Check(sp.Properties, check.HasLen, 1)
+	c.Check(sp.Properties["foo"], check.Equals, "bar")
+
+	spGet, err := s.conn.SpecimenGet(s.ctx, arvados.GetOptions{UUID: sp.UUID})
+	c.Check(spGet.UUID, check.Equals, sp.UUID)
+	c.Check(spGet.Properties["foo"], check.Equals, "bar")
+
+	spList, err := s.conn.SpecimenList(s.ctx, arvados.ListOptions{Limit: -1, Filters: []arvados.Filter{{"uuid", "=", sp.UUID}}})
+	c.Check(spList.ItemsAvailable, check.Equals, 1)
+	c.Assert(spList.Items, check.HasLen, 1)
+	c.Check(spList.Items[0].UUID, check.Equals, sp.UUID)
+	c.Check(spList.Items[0].Properties["foo"], check.Equals, "bar")
+
+	anonCtx := ContextWithTokens(context.Background(), []string{arvadostest.AnonymousToken})
+	spList, err = s.conn.SpecimenList(anonCtx, arvados.ListOptions{Limit: -1, Filters: []arvados.Filter{{"uuid", "=", sp.UUID}}})
+	c.Check(spList.ItemsAvailable, check.Equals, 0)
+	c.Check(spList.Items, check.HasLen, 0)
+
+	spDel, err := s.conn.SpecimenDelete(s.ctx, arvados.DeleteOptions{UUID: sp.UUID})
+	c.Check(spDel.UUID, check.Equals, sp.UUID)
+}
diff --git a/lib/controller/server_test.go b/lib/controller/server_test.go
index ae89c3d7e..e5fd41712 100644
--- a/lib/controller/server_test.go
+++ b/lib/controller/server_test.go
@@ -42,6 +42,7 @@ func newServerFromIntegrationTestEnv(c *check.C) *httpserver.Server {
 		NodeProfiles: map[string]arvados.NodeProfile{
 			"*": nodeProfile,
 		},
+		EnableBetaController14287: enableBetaController14287,
 	}, NodeProfile: &nodeProfile}
 
 	srv := &httpserver.Server{
diff --git a/sdk/go/arvados/api.go b/sdk/go/arvados/api.go
new file mode 100644
index 000000000..4cdf7c0e1
--- /dev/null
+++ b/sdk/go/arvados/api.go
@@ -0,0 +1,61 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: Apache-2.0
+
+package arvados
+
+type APIEndpoint struct {
+	Method string
+	Path   string
+	// "new attributes" key for create/update requests
+	AttrsKey string
+}
+
+var (
+	EndpointCollectionCreate              = APIEndpoint{"POST", "arvados/v1/collections", "collection"}
+	EndpointCollectionUpdate              = APIEndpoint{"PATCH", "arvados/v1/collections/:uuid", "collection"}
+	EndpointCollectionGet                 = APIEndpoint{"GET", "arvados/v1/collections/:uuid", ""}
+	EndpointCollectionList                = APIEndpoint{"GET", "arvados/v1/collections", ""}
+	EndpointCollectionDelete              = APIEndpoint{"DELETE", "arvados/v1/collections/:uuid", ""}
+	EndpointSpecimenCreate                = APIEndpoint{"POST", "arvados/v1/specimens", "specimen"}
+	EndpointSpecimenUpdate                = APIEndpoint{"PATCH", "arvados/v1/specimens/:uuid", "specimen"}
+	EndpointSpecimenGet                   = APIEndpoint{"GET", "arvados/v1/specimens/:uuid", ""}
+	EndpointSpecimenList                  = APIEndpoint{"GET", "arvados/v1/specimens", ""}
+	EndpointSpecimenDelete                = APIEndpoint{"DELETE", "arvados/v1/specimens/:uuid", ""}
+	EndpointContainerCreate               = APIEndpoint{"POST", "arvados/v1/containers", "container"}
+	EndpointContainerUpdate               = APIEndpoint{"PATCH", "arvados/v1/containers/:uuid", "container"}
+	EndpointContainerGet                  = APIEndpoint{"GET", "arvados/v1/containers/:uuid", ""}
+	EndpointContainerList                 = APIEndpoint{"GET", "arvados/v1/containers", ""}
+	EndpointContainerDelete               = APIEndpoint{"DELETE", "arvados/v1/containers/:uuid", ""}
+	EndpointContainerLock                 = APIEndpoint{"POST", "arvados/v1/containers/:uuid/lock", ""}
+	EndpointContainerUnlock               = APIEndpoint{"POST", "arvados/v1/containers/:uuid/unlock", ""}
+	EndpointAPIClientAuthorizationCurrent = APIEndpoint{"GET", "arvados/v1/api_client_authorizations/current", ""}
+)
+
+type GetOptions struct {
+	UUID   string   `json:"uuid"`
+	Select []string `json:"select"`
+}
+
+type ListOptions struct {
+	Select  []string `json:"select"`
+	Filters []Filter `json:"filters"`
+	Limit   int      `json:"limit"`
+	Offset  int      `json:"offset"`
+}
+
+type CreateOptions struct {
+	ClusterID        string                 `json:"cluster_id"`
+	EnsureUniqueName bool                   `json:"ensure_unique_name"`
+	Select           []string               `json:"select"`
+	Attrs            map[string]interface{} `json:"attrs"`
+}
+
+type UpdateOptions struct {
+	UUID  string                 `json:"uuid"`
+	Attrs map[string]interface{} `json:"attrs"`
+}
+
+type DeleteOptions struct {
+	UUID string `json:"uuid"`
+}
diff --git a/sdk/go/arvados/client.go b/sdk/go/arvados/client.go
index cbc2ca72f..2ea6baf88 100644
--- a/sdk/go/arvados/client.go
+++ b/sdk/go/arvados/client.go
@@ -35,6 +35,9 @@ type Client struct {
 	// DefaultSecureClient or InsecureHTTPClient will be used.
 	Client *http.Client `json:"-"`
 
+	// Protocol scheme: "http", "https", or "" (https)
+	Scheme string
+
 	// Hostname (or host:port) of Arvados API server.
 	APIHost string
 
@@ -79,6 +82,7 @@ func NewClientFromConfig(cluster *Cluster) (*Client, error) {
 		return nil, fmt.Errorf("no host in config Services.Controller.ExternalURL: %v", ctrlURL)
 	}
 	return &Client{
+		Scheme:   ctrlURL.Scheme,
 		APIHost:  ctrlURL.Host,
 		Insecure: cluster.TLS.Insecure,
 	}, nil
@@ -105,6 +109,7 @@ func NewClientFromEnv() *Client {
 		insecure = true
 	}
 	return &Client{
+		Scheme:          "https",
 		APIHost:         os.Getenv("ARVADOS_API_HOST"),
 		AuthToken:       os.Getenv("ARVADOS_API_TOKEN"),
 		Insecure:        insecure,
@@ -117,7 +122,9 @@ var reqIDGen = httpserver.IDGenerator{Prefix: "req-"}
 // Do adds Authorization and X-Request-Id headers and then calls
 // (*http.Client)Do().
 func (c *Client) Do(req *http.Request) (*http.Response, error) {
-	if c.AuthToken != "" {
+	if auth, _ := req.Context().Value("Authorization").(string); auth != "" {
+		req.Header.Add("Authorization", auth)
+	} else if c.AuthToken != "" {
 		req.Header.Add("Authorization", "OAuth2 "+c.AuthToken)
 	}
 
@@ -203,6 +210,9 @@ func anythingToValues(params interface{}) (url.Values, error) {
 		if err != nil {
 			return nil, err
 		}
+		if string(j) == "null" {
+			continue
+		}
 		urlValues.Set(k, string(j))
 	}
 	return urlValues, nil
@@ -216,6 +226,10 @@ func anythingToValues(params interface{}) (url.Values, error) {
 //
 // path must not contain a query string.
 func (c *Client) RequestAndDecode(dst interface{}, method, path string, body io.Reader, params interface{}) error {
+	return c.RequestAndDecodeContext(context.Background(), dst, method, path, body, params)
+}
+
+func (c *Client) RequestAndDecodeContext(ctx context.Context, dst interface{}, method, path string, body io.Reader, params interface{}) error {
 	if body, ok := body.(io.Closer); ok {
 		// Ensure body is closed even if we error out early
 		defer body.Close()
@@ -243,6 +257,7 @@ func (c *Client) RequestAndDecode(dst interface{}, method, path string, body io.
 	if err != nil {
 		return err
 	}
+	req = req.WithContext(ctx)
 	req.Header.Set("Content-type", "application/x-www-form-urlencoded")
 	return c.DoAndDecode(dst, req)
 }
@@ -265,13 +280,9 @@ func (c *Client) UpdateBody(rsc resource) io.Reader {
 	return bytes.NewBufferString(v.Encode())
 }
 
-type contextKey string
-
-var contextKeyRequestID contextKey = "X-Request-Id"
-
 func (c *Client) WithRequestID(reqid string) *Client {
 	cc := *c
-	cc.ctx = context.WithValue(cc.context(), contextKeyRequestID, reqid)
+	cc.ctx = ContextWithRequestID(cc.context(), reqid)
 	return &cc
 }
 
@@ -294,7 +305,11 @@ func (c *Client) httpClient() *http.Client {
 }
 
 func (c *Client) apiURL(path string) string {
-	return "https://" + c.APIHost + "/" + path
+	scheme := c.Scheme
+	if scheme == "" {
+		scheme = "https"
+	}
+	return scheme + "://" + c.APIHost + "/" + path
 }
 
 // DiscoveryDocument is the Arvados server's description of itself.
diff --git a/sdk/go/arvados/collection.go b/sdk/go/arvados/collection.go
index 5b6130060..f374eea07 100644
--- a/sdk/go/arvados/collection.go
+++ b/sdk/go/arvados/collection.go
@@ -73,7 +73,6 @@ func (c *Collection) SizedDigests() ([]SizedDigest, error) {
 	return sds, scanner.Err()
 }
 
-// CollectionList is an arvados#collectionList resource.
 type CollectionList struct {
 	Items          []Collection `json:"items"`
 	ItemsAvailable int          `json:"items_available"`
diff --git a/sdk/go/arvados/config.go b/sdk/go/arvados/config.go
index 2965d5ecb..d309748f4 100644
--- a/sdk/go/arvados/config.go
+++ b/sdk/go/arvados/config.go
@@ -71,6 +71,8 @@ type Cluster struct {
 	RequestLimits      RequestLimits
 	Logging            Logging
 	TLS                TLS
+
+	EnableBetaController14287 bool
 }
 
 type Services struct {
diff --git a/sdk/go/arvados/context.go b/sdk/go/arvados/context.go
new file mode 100644
index 000000000..555cfc8e9
--- /dev/null
+++ b/sdk/go/arvados/context.go
@@ -0,0 +1,17 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: Apache-2.0
+
+package arvados
+
+import (
+	"context"
+)
+
+type contextKey string
+
+var contextKeyRequestID contextKey = "X-Request-Id"
+
+func ContextWithRequestID(ctx context.Context, reqid string) context.Context {
+	return context.WithValue(ctx, contextKeyRequestID, reqid)
+}
diff --git a/sdk/go/arvados/error.go b/sdk/go/arvados/error.go
index 9a0485578..5329a5146 100644
--- a/sdk/go/arvados/error.go
+++ b/sdk/go/arvados/error.go
@@ -31,6 +31,10 @@ func (e TransactionError) Error() (s string) {
 	return
 }
 
+func (e TransactionError) HTTPStatus() int {
+	return e.StatusCode
+}
+
 func newTransactionError(req *http.Request, resp *http.Response, buf []byte) *TransactionError {
 	var e TransactionError
 	if json.Unmarshal(buf, &e) != nil {
diff --git a/sdk/go/arvados/resource_list.go b/sdk/go/arvados/resource_list.go
index 14ce098cf..505ba51ec 100644
--- a/sdk/go/arvados/resource_list.go
+++ b/sdk/go/arvados/resource_list.go
@@ -4,7 +4,10 @@
 
 package arvados
 
-import "encoding/json"
+import (
+	"encoding/json"
+	"fmt"
+)
 
 // ResourceListParams expresses which results are requested in a
 // list/index API.
@@ -27,7 +30,35 @@ type Filter struct {
 	Operand  interface{}
 }
 
-// MarshalJSON encodes a Filter in the form expected by the API.
+// MarshalJSON encodes a Filter to a JSON array.
 func (f *Filter) MarshalJSON() ([]byte, error) {
 	return json.Marshal([]interface{}{f.Attr, f.Operator, f.Operand})
 }
+
+// UnmarshalJSON decodes a JSON array to a Filter.
+func (f *Filter) UnmarshalJSON(data []byte) error {
+	var elements []interface{}
+	err := json.Unmarshal(data, &elements)
+	if err != nil {
+		return err
+	}
+	if len(elements) != 3 {
+		return fmt.Errorf("invalid filter %q: must have 3 elements", data)
+	}
+	attr, ok := elements[0].(string)
+	if !ok {
+		return fmt.Errorf("invalid filter attr %q", elements[0])
+	}
+	op, ok := elements[1].(string)
+	if !ok {
+		return fmt.Errorf("invalid filter operator %q", elements[1])
+	}
+	operand := elements[2]
+	switch operand.(type) {
+	case string, float64, []interface{}:
+	default:
+		return fmt.Errorf("invalid filter operand %q", elements[2])
+	}
+	*f = Filter{attr, op, operand}
+	return nil
+}
diff --git a/sdk/go/arvados/specimen.go b/sdk/go/arvados/specimen.go
new file mode 100644
index 000000000..e320ca2c3
--- /dev/null
+++ b/sdk/go/arvados/specimen.go
@@ -0,0 +1,23 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: Apache-2.0
+
+package arvados
+
+import "time"
+
+type Specimen struct {
+	UUID       string                 `json:"uuid"`
+	OwnerUUID  string                 `json:"owner_uuid"`
+	CreatedAt  time.Time              `json:"created_at"`
+	ModifiedAt time.Time              `json:"modified_at"`
+	UpdatedAt  time.Time              `json:"updated_at"`
+	Properties map[string]interface{} `json:"properties"`
+}
+
+type SpecimenList struct {
+	Items          []Specimen `json:"items"`
+	ItemsAvailable int        `json:"items_available"`
+	Offset         int        `json:"offset"`
+	Limit          int        `json:"limit"`
+}
diff --git a/sdk/go/auth/auth.go b/sdk/go/auth/auth.go
index 3c266e0d3..de3b1e952 100644
--- a/sdk/go/auth/auth.go
+++ b/sdk/go/auth/auth.go
@@ -20,7 +20,7 @@ func NewCredentials() *Credentials {
 }
 
 func CredentialsFromRequest(r *http.Request) *Credentials {
-	if c, ok := r.Context().Value(contextKeyCredentials).(*Credentials); ok {
+	if c, ok := r.Context().Value(ContextKeyCredentials).(*Credentials); ok {
 		// preloaded by middleware
 		return c
 	}
diff --git a/sdk/go/auth/handlers.go b/sdk/go/auth/handlers.go
index ad1fa5141..9fa501ab7 100644
--- a/sdk/go/auth/handlers.go
+++ b/sdk/go/auth/handlers.go
@@ -11,15 +11,15 @@ import (
 
 type contextKey string
 
-var contextKeyCredentials contextKey = "credentials"
+var ContextKeyCredentials contextKey = "credentials"
 
 // LoadToken wraps the next handler, adding credentials to the request
 // context so subsequent handlers can access them efficiently via
 // CredentialsFromRequest.
 func LoadToken(next http.Handler) http.Handler {
 	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
-		if _, ok := r.Context().Value(contextKeyCredentials).(*Credentials); !ok {
-			r = r.WithContext(context.WithValue(r.Context(), contextKeyCredentials, CredentialsFromRequest(r)))
+		if _, ok := r.Context().Value(ContextKeyCredentials).(*Credentials); !ok {
+			r = r.WithContext(context.WithValue(r.Context(), ContextKeyCredentials, CredentialsFromRequest(r)))
 		}
 		next.ServeHTTP(w, r)
 	})
diff --git a/sdk/go/httpserver/error.go b/sdk/go/httpserver/error.go
index 1ccf8c047..b222e18ea 100644
--- a/sdk/go/httpserver/error.go
+++ b/sdk/go/httpserver/error.go
@@ -14,10 +14,7 @@ type ErrorResponse struct {
 }
 
 func Error(w http.ResponseWriter, error string, code int) {
-	w.Header().Set("Content-Type", "application/json")
-	w.Header().Set("X-Content-Type-Options", "nosniff")
-	w.WriteHeader(code)
-	json.NewEncoder(w).Encode(ErrorResponse{Errors: []string{error}})
+	Errors(w, []string{error}, code)
 }
 
 func Errors(w http.ResponseWriter, errors []string, code int) {
diff --git a/sdk/go/keepclient/keepclient.go b/sdk/go/keepclient/keepclient.go
index ab610d65e..c8dd09de8 100644
--- a/sdk/go/keepclient/keepclient.go
+++ b/sdk/go/keepclient/keepclient.go
@@ -551,7 +551,7 @@ func (kc *KeepClient) httpClient() HTTPClient {
 		// It's not safe to copy *http.DefaultTransport
 		// because it has a mutex (which might be locked)
 		// protecting a private map (which might not be nil).
-		// So we build our own, using the Go 1.10 default
+		// So we build our own, using the Go 1.12 default
 		// values, ignoring any changes the application has
 		// made to http.DefaultTransport.
 		Transport: &http.Transport{
@@ -563,7 +563,7 @@ func (kc *KeepClient) httpClient() HTTPClient {
 			MaxIdleConns:          100,
 			IdleConnTimeout:       90 * time.Second,
 			TLSHandshakeTimeout:   tlsTimeout,
-			ExpectContinueTimeout: time.Second,
+			ExpectContinueTimeout: 1 * time.Second,
 			TLSClientConfig:       arvadosclient.MakeTLSConfig(kc.Arvados.ApiInsecure),
 		},
 	}
diff --git a/services/crunch-run/crunchrun.go b/services/crunch-run/crunchrun.go
index 84b578a3e..3261291b5 100644
--- a/services/crunch-run/crunchrun.go
+++ b/services/crunch-run/crunchrun.go
@@ -987,7 +987,7 @@ func (runner *ContainerRunner) AttachStreams() (err error) {
 		go func() {
 			_, err := io.Copy(response.Conn, stdinRdr)
 			if err != nil {
-				runner.CrunchLog.Printf("While writing stdin collection to docker container %q", err)
+				runner.CrunchLog.Printf("While writing stdin collection to docker container: %v", err)
 				runner.stop(nil)
 			}
 			stdinRdr.Close()
@@ -997,7 +997,7 @@ func (runner *ContainerRunner) AttachStreams() (err error) {
 		go func() {
 			_, err := io.Copy(response.Conn, bytes.NewReader(stdinJson))
 			if err != nil {
-				runner.CrunchLog.Printf("While writing stdin json to docker container %q", err)
+				runner.CrunchLog.Printf("While writing stdin json to docker container: %v", err)
 				runner.stop(nil)
 			}
 			response.CloseWrite()

-----------------------------------------------------------------------


hooks/post-receive
-- 




More information about the arvados-commits mailing list