[ARVADOS] updated: 1.1.4-538-g8d3ecc67a

Git user git at public.curoverse.com
Tue Jul 10 15:12:22 EDT 2018


Summary of changes:
 lib/controller/federation.go               |  28 ++++++--
 lib/controller/federation_test.go          | 108 +++++++++++++++++++++++------
 lib/controller/handler.go                  |  30 ++++++++
 lib/controller/handler_test.go             |   5 +-
 lib/controller/proxy.go                    |   2 +-
 lib/controller/server_test.go              |  19 ++++-
 sdk/go/arvados/api_client_authorization.go |   4 ++
 sdk/go/arvados/config.go                   |   8 +++
 sdk/go/arvados/postgresql.go               |  20 ++++++
 sdk/go/auth/salt.go                        |   5 +-
 sdk/python/tests/run_test_server.py        |  16 ++++-
 services/ws/config.go                      |   4 +-
 services/ws/event_source.go                |  16 -----
 services/ws/event_source_test.go           |  30 +++-----
 services/ws/server.go                      |   2 +-
 15 files changed, 226 insertions(+), 71 deletions(-)
 create mode 100644 sdk/go/arvados/postgresql.go

       via  8d3ecc67ad5d208e5327b81dffda0c10ef0a8378 (commit)
       via  3934348cf5166ed53248bed7f2aa3b98d067cbc2 (commit)
       via  acf545571425b2a2d8a26cb703f75926b6b2a987 (commit)
       via  b29c13b372bd39075eabda49ea57b0cb6d5a8452 (commit)
       via  81d5bd428962c1819c0c60de5bc5d93710798424 (commit)
       via  d22f92638664d9744a38e60dce9f0d95a2d393f0 (commit)
       via  6b8a3d3d912573ab432c319121eaffd0bbe2216b (commit)
      from  424298694945c9f66676bd401753be44f6fc5fa8 (commit)

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 8d3ecc67ad5d208e5327b81dffda0c10ef0a8378
Author: Tom Clegg <tclegg at veritasgenetics.com>
Date:   Tue Jul 10 14:26:28 2018 -0400

    13493: Upgrade locally-issued v1 tokens to v2 when proxying.
    
    Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tclegg at veritasgenetics.com>

diff --git a/lib/controller/federation.go b/lib/controller/federation.go
index c1d34be0d..24b925053 100644
--- a/lib/controller/federation.go
+++ b/lib/controller/federation.go
@@ -6,11 +6,13 @@ package controller
 
 import (
 	"bytes"
+	"database/sql"
 	"io/ioutil"
 	"net/http"
 	"net/url"
 	"regexp"
 
+	"git.curoverse.com/arvados.git/sdk/go/arvados"
 	"git.curoverse.com/arvados.git/sdk/go/auth"
 	"git.curoverse.com/arvados.git/sdk/go/httpserver"
 )
@@ -77,11 +79,27 @@ func (h *Handler) saltAuthToken(req *http.Request, remote string) error {
 	}
 	token, err := auth.SaltToken(creds.Tokens[0], remote)
 	if err == auth.ErrObsoleteToken {
-		// FIXME: If the token exists in our own database,
-		// salt it for the remote. Otherwise, assume it was
-		// issued by the remote, and pass it through
-		// unmodified.
-		token = creds.Tokens[0]
+		// If the token exists in our own database, salt it
+		// for the remote. Otherwise, assume it was issued by
+		// the remote, and pass it through unmodified.
+		db, err := h.db(req)
+		if err != nil {
+			return err
+		}
+		aca := arvados.APIClientAuthorization{APIToken: creds.Tokens[0]}
+		err = db.QueryRowContext(req.Context(), `SELECT uuid FROM api_client_authorizations WHERE api_token=$1 AND (expires_at IS NULL OR expires_at > current_timestamp) LIMIT 1`, aca.APIToken).Scan(&aca.UUID)
+		if err == sql.ErrNoRows {
+			// Not ours; pass through unmodified.
+			token = aca.APIToken
+		} else if err != nil {
+			return err
+		} else {
+			// Found; make V2 version and salt it.
+			token, err = auth.SaltToken(aca.TokenV2(), remote)
+			if err != nil {
+				return err
+			}
+		}
 	} else if err != nil {
 		return err
 	}
diff --git a/lib/controller/federation_test.go b/lib/controller/federation_test.go
index 8ca1aa523..72f4259c9 100644
--- a/lib/controller/federation_test.go
+++ b/lib/controller/federation_test.go
@@ -55,7 +55,8 @@ func (s *FederationSuite) SetUpTest(c *check.C) {
 		RailsAPI:   arvados.SystemServiceInstance{Listen: ":1"}, // local reqs will error "connection refused"
 	}
 	s.testHandler = &Handler{Cluster: &arvados.Cluster{
-		ClusterID: "zhome",
+		ClusterID:  "zhome",
+		PostgreSQL: integrationTestCluster().PostgreSQL,
 		NodeProfiles: map[string]arvados.NodeProfile{
 			"*": nodeProfile,
 		},
@@ -77,6 +78,8 @@ func (s *FederationSuite) SetUpTest(c *check.C) {
 	}
 
 	c.Assert(s.testServer.Start(), check.IsNil)
+
+	s.remoteMockRequests = nil
 }
 
 func (s *FederationSuite) remoteMockHandler(w http.ResponseWriter, req *http.Request) {
@@ -172,8 +175,34 @@ func (s *FederationSuite) TestRemoteWithTokenInQuery(c *check.C) {
 	s.testRequest(req)
 	c.Assert(len(s.remoteMockRequests), check.Equals, 1)
 	pr := s.remoteMockRequests[0]
+	// Token is salted and moved from query to Authorization header.
 	c.Check(pr.URL.String(), check.Not(check.Matches), `.*api_token=.*`)
-	c.Check(pr.Header.Get("Authorization"), check.Equals, "Bearer "+arvadostest.ActiveToken)
+	c.Check(pr.Header.Get("Authorization"), check.Equals, "Bearer v2/zzzzz-gj3su-077z32aux8dg2s1/7fd31b61f39c0e82a4155592163218272cedacdc")
+}
+
+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(len(s.remoteMockRequests), check.Equals, 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) {
+	// 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(len(s.remoteMockRequests), check.Equals, 1)
+	pr := s.remoteMockRequests[0]
+	c.Check(pr.Header.Get("Authorization"), check.Equals, "Bearer "+remoteToken)
 }
 
 func (s *FederationSuite) TestWorkflowCRUD(c *check.C) {
diff --git a/sdk/go/arvados/api_client_authorization.go b/sdk/go/arvados/api_client_authorization.go
index 3343bdb9a..ec0239eb3 100644
--- a/sdk/go/arvados/api_client_authorization.go
+++ b/sdk/go/arvados/api_client_authorization.go
@@ -14,3 +14,7 @@ type APIClientAuthorization struct {
 type APIClientAuthorizationList struct {
 	Items []APIClientAuthorization `json:"items"`
 }
+
+func (aca APIClientAuthorization) TokenV2() string {
+	return "v2/" + aca.UUID + "/" + aca.APIToken
+}

commit 3934348cf5166ed53248bed7f2aa3b98d067cbc2
Author: Tom Clegg <tclegg at veritasgenetics.com>
Date:   Tue Jul 10 14:25:20 2018 -0400

    13493: Fix HMAC usage.
    
    Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tclegg at veritasgenetics.com>

diff --git a/sdk/go/auth/salt.go b/sdk/go/auth/salt.go
index f669eb276..667a30f5e 100644
--- a/sdk/go/auth/salt.go
+++ b/sdk/go/auth/salt.go
@@ -9,6 +9,7 @@ import (
 	"crypto/sha1"
 	"errors"
 	"fmt"
+	"io"
 	"regexp"
 	"strings"
 )
@@ -33,7 +34,9 @@ func SaltToken(token, remote string) (string, error) {
 	secret := parts[2]
 	if len(secret) != 40 {
 		// not already salted
-		secret = fmt.Sprintf("%x", hmac.New(sha1.New, []byte(secret)).Sum([]byte(remote)))
+		hmac := hmac.New(sha1.New, []byte(secret))
+		io.WriteString(hmac, remote)
+		secret = fmt.Sprintf("%x", hmac.Sum(nil))
 		return "v2/" + uuid + "/" + secret, nil
 	} else if strings.HasPrefix(uuid, remote) {
 		// already salted for the desired remote

commit acf545571425b2a2d8a26cb703f75926b6b2a987
Author: Tom Clegg <tclegg at veritasgenetics.com>
Date:   Tue Jul 10 14:25:10 2018 -0400

    13493: Connect to database.
    
    Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tclegg at veritasgenetics.com>

diff --git a/lib/controller/handler.go b/lib/controller/handler.go
index 78ad89e70..c50f98273 100644
--- a/lib/controller/handler.go
+++ b/lib/controller/handler.go
@@ -5,6 +5,8 @@
 package controller
 
 import (
+	"database/sql"
+	"errors"
 	"net"
 	"net/http"
 	"net/url"
@@ -15,6 +17,7 @@ import (
 	"git.curoverse.com/arvados.git/sdk/go/arvados"
 	"git.curoverse.com/arvados.git/sdk/go/health"
 	"git.curoverse.com/arvados.git/sdk/go/httpserver"
+	_ "github.com/lib/pq"
 )
 
 type Handler struct {
@@ -26,6 +29,8 @@ type Handler struct {
 	proxy          *proxy
 	secureClient   *http.Client
 	insecureClient *http.Client
+	pgdb           *sql.DB
+	pgdbMtx        sync.Mutex
 }
 
 func (h *Handler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
@@ -65,6 +70,31 @@ func (h *Handler) setup() {
 	}
 }
 
+var errDBConnection = errors.New("database connection error")
+
+func (h *Handler) db(req *http.Request) (*sql.DB, error) {
+	h.pgdbMtx.Lock()
+	defer h.pgdbMtx.Unlock()
+	if h.pgdb != nil {
+		return h.pgdb, nil
+	}
+
+	db, err := sql.Open("postgres", h.Cluster.PostgreSQL.Connection.String())
+	if err != nil {
+		httpserver.Logger(req).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 {
+		httpserver.Logger(req).WithError(err).Error("postgresql connect succeeded but ping failed")
+		return nil, errDBConnection
+	}
+	h.pgdb = db
+	return db, nil
+}
+
 type middlewareFunc func(http.ResponseWriter, *http.Request, http.Handler)
 
 func prepend(next http.Handler, middleware middlewareFunc) http.Handler {
diff --git a/lib/controller/handler_test.go b/lib/controller/handler_test.go
index 3aedfb889..2e833ed18 100644
--- a/lib/controller/handler_test.go
+++ b/lib/controller/handler_test.go
@@ -34,7 +34,8 @@ type HandlerSuite struct {
 
 func (s *HandlerSuite) SetUpTest(c *check.C) {
 	s.cluster = &arvados.Cluster{
-		ClusterID: "zzzzz",
+		ClusterID:  "zzzzz",
+		PostgreSQL: integrationTestCluster().PostgreSQL,
 		NodeProfiles: map[string]arvados.NodeProfile{
 			"*": {
 				Controller: arvados.SystemServiceInstance{Listen: ":"},
diff --git a/lib/controller/server_test.go b/lib/controller/server_test.go
index 381e08948..7742cf4ea 100644
--- a/lib/controller/server_test.go
+++ b/lib/controller/server_test.go
@@ -8,6 +8,7 @@ import (
 	"bytes"
 	"net/http"
 	"os"
+	"path/filepath"
 
 	"git.curoverse.com/arvados.git/sdk/go/arvados"
 	"git.curoverse.com/arvados.git/sdk/go/httpserver"
@@ -26,6 +27,18 @@ func (tl *logWriter) Write(buf []byte) (int, error) {
 	return len(buf), nil
 }
 
+func integrationTestCluster() *arvados.Cluster {
+	cfg, err := arvados.GetConfig(filepath.Join(os.Getenv("WORKSPACE"), "tmp", "arvados.yml"))
+	if err != nil {
+		panic(err)
+	}
+	cc, err := cfg.GetCluster("zzzzz")
+	if err != nil {
+		panic(err)
+	}
+	return cc
+}
+
 // Return a new unstarted controller server, using the Rails API
 // provided by the integration-testing environment.
 func newServerFromIntegrationTestEnv(c *check.C) *httpserver.Server {
@@ -38,7 +51,8 @@ func newServerFromIntegrationTestEnv(c *check.C) *httpserver.Server {
 		RailsAPI:   arvados.SystemServiceInstance{Listen: os.Getenv("ARVADOS_TEST_API_HOST"), TLS: true, Insecure: true},
 	}
 	handler := &Handler{Cluster: &arvados.Cluster{
-		ClusterID: "zzzzz",
+		ClusterID:  "zzzzz",
+		PostgreSQL: integrationTestCluster().PostgreSQL,
 		NodeProfiles: map[string]arvados.NodeProfile{
 			"*": nodeProfile,
 		},
diff --git a/sdk/go/arvados/config.go b/sdk/go/arvados/config.go
index 0225736c2..608bc223b 100644
--- a/sdk/go/arvados/config.go
+++ b/sdk/go/arvados/config.go
@@ -59,7 +59,8 @@ type Cluster struct {
 }
 
 type PostgreSQL struct {
-	Connection PostgreSQLConnection
+	Connection     PostgreSQLConnection
+	ConnectionPool int
 }
 
 type PostgreSQLConnection map[string]string
diff --git a/sdk/python/tests/run_test_server.py b/sdk/python/tests/run_test_server.py
index bad3738db..05cefbc9e 100644
--- a/sdk/python/tests/run_test_server.py
+++ b/sdk/python/tests/run_test_server.py
@@ -408,6 +408,13 @@ def run_controller():
         f.write("""
 Clusters:
   zzzzz:
+    PostgreSQL:
+      ConnectionPool: 32
+      Connection:
+        host: {}
+        dbname: {}
+        user: {}
+        password: {}
     NodeProfiles:
       "*":
         "arvados-controller":
@@ -416,7 +423,14 @@ Clusters:
           Listen: ":{}"
           TLS: true
           Insecure: true
-        """.format(port, rails_api_port))
+        """.format(
+            _dbconfig('host'),
+            _dbconfig('database'),
+            _dbconfig('username'),
+            _dbconfig('password'),
+            port,
+            rails_api_port,
+        ))
     logf = open(_logfilename('controller'), 'a')
     controller = subprocess.Popen(
         ["arvados-server", "controller", "-config", conf],
diff --git a/services/ws/event_source_test.go b/services/ws/event_source_test.go
index ea6063a0c..ac5d130d6 100644
--- a/services/ws/event_source_test.go
+++ b/services/ws/event_source_test.go
@@ -7,10 +7,12 @@ package main
 import (
 	"database/sql"
 	"fmt"
+	"os"
+	"path/filepath"
 	"sync"
 	"time"
 
-	"git.curoverse.com/arvados.git/sdk/go/config"
+	"git.curoverse.com/arvados.git/sdk/go/arvados"
 	check "gopkg.in/check.v1"
 )
 
@@ -18,30 +20,20 @@ var _ = check.Suite(&eventSourceSuite{})
 
 type eventSourceSuite struct{}
 
-func testDBConfig() pgConfig {
-	var railsDB struct {
-		Test struct {
-			Database string
-			Username string
-			Password string
-			Host     string
-		}
-	}
-	err := config.LoadFile(&railsDB, "../api/config/database.yml")
+func testDBConfig() arvados.PostgreSQLConnection {
+	cfg, err := arvados.GetConfig(filepath.Join(os.Getenv("WORKSPACE"), "tmp", "arvados.yml"))
 	if err != nil {
 		panic(err)
 	}
-	cfg := pgConfig{
-		"dbname":   railsDB.Test.Database,
-		"host":     railsDB.Test.Host,
-		"password": railsDB.Test.Password,
-		"user":     railsDB.Test.Username,
+	cc, err := cfg.GetCluster("zzzzz")
+	if err != nil {
+		panic(err)
 	}
-	return cfg
+	return cc.PostgreSQL.Connection
 }
 
 func testDB() *sql.DB {
-	db, err := sql.Open("postgres", testDBConfig().ConnectionString())
+	db, err := sql.Open("postgres", testDBConfig().String())
 	if err != nil {
 		panic(err)
 	}
@@ -52,7 +44,7 @@ func (*eventSourceSuite) TestEventSource(c *check.C) {
 	cfg := testDBConfig()
 	db := testDB()
 	pges := &pgEventSource{
-		DataSource: cfg.ConnectionString(),
+		DataSource: cfg.String(),
 		QueueSize:  4,
 	}
 	go pges.Run()

commit b29c13b372bd39075eabda49ea57b0cb6d5a8452
Author: Tom Clegg <tclegg at veritasgenetics.com>
Date:   Mon Jul 9 09:59:51 2018 -0400

    13493: Add PostgreSQL connection to cluster config.
    
    Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tclegg at veritasgenetics.com>

diff --git a/sdk/go/arvados/config.go b/sdk/go/arvados/config.go
index dfe2e7b31..0225736c2 100644
--- a/sdk/go/arvados/config.go
+++ b/sdk/go/arvados/config.go
@@ -55,8 +55,15 @@ type Cluster struct {
 	InstanceTypes      []InstanceType
 	HTTPRequestTimeout Duration
 	RemoteClusters     map[string]RemoteCluster
+	PostgreSQL         PostgreSQL
 }
 
+type PostgreSQL struct {
+	Connection PostgreSQLConnection
+}
+
+type PostgreSQLConnection map[string]string
+
 type RemoteCluster struct {
 	// API endpoint host or host:port; default is {id}.arvadosapi.com
 	Host string
diff --git a/sdk/go/arvados/postgresql.go b/sdk/go/arvados/postgresql.go
new file mode 100644
index 000000000..47953ce9d
--- /dev/null
+++ b/sdk/go/arvados/postgresql.go
@@ -0,0 +1,20 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: Apache-2.0
+
+package arvados
+
+import "strings"
+
+func (c PostgreSQLConnection) String() string {
+	s := ""
+	for k, v := range c {
+		s += strings.ToLower(k)
+		s += "='"
+		s += strings.Replace(
+			strings.Replace(v, `\`, `\\`, -1),
+			`'`, `\'`, -1)
+		s += "' "
+	}
+	return s
+}
diff --git a/services/ws/config.go b/services/ws/config.go
index d4cf51143..ead1ec20c 100644
--- a/services/ws/config.go
+++ b/services/ws/config.go
@@ -12,7 +12,7 @@ import (
 
 type wsConfig struct {
 	Client       arvados.Client
-	Postgres     pgConfig
+	Postgres     arvados.PostgreSQLConnection
 	PostgresPool int
 	Listen       string
 	LogLevel     string
@@ -30,7 +30,7 @@ func defaultConfig() wsConfig {
 		Client: arvados.Client{
 			APIHost: "localhost:443",
 		},
-		Postgres: pgConfig{
+		Postgres: arvados.PostgreSQLConnection{
 			"dbname":                    "arvados_production",
 			"user":                      "arvados",
 			"password":                  "xyzzy",
diff --git a/services/ws/event_source.go b/services/ws/event_source.go
index 9acfca50e..309dab7a4 100644
--- a/services/ws/event_source.go
+++ b/services/ws/event_source.go
@@ -8,7 +8,6 @@ import (
 	"context"
 	"database/sql"
 	"strconv"
-	"strings"
 	"sync"
 	"sync/atomic"
 	"time"
@@ -17,21 +16,6 @@ import (
 	"github.com/lib/pq"
 )
 
-type pgConfig map[string]string
-
-func (c pgConfig) ConnectionString() string {
-	s := ""
-	for k, v := range c {
-		s += k
-		s += "='"
-		s += strings.Replace(
-			strings.Replace(v, `\`, `\\`, -1),
-			`'`, `\'`, -1)
-		s += "' "
-	}
-	return s
-}
-
 type pgEventSource struct {
 	DataSource   string
 	MaxOpenConns int
diff --git a/services/ws/server.go b/services/ws/server.go
index 36ce7ae59..eda7ff2a4 100644
--- a/services/ws/server.go
+++ b/services/ws/server.go
@@ -48,7 +48,7 @@ func (srv *server) setup() {
 
 	srv.listener = ln
 	srv.eventSource = &pgEventSource{
-		DataSource:   srv.wsConfig.Postgres.ConnectionString(),
+		DataSource:   srv.wsConfig.Postgres.String(),
 		MaxOpenConns: srv.wsConfig.PostgresPool,
 		QueueSize:    srv.wsConfig.ServerEventQueue,
 	}

commit 81d5bd428962c1819c0c60de5bc5d93710798424
Author: Tom Clegg <tclegg at veritasgenetics.com>
Date:   Fri Jul 6 16:42:33 2018 -0400

    13493: Return 502 Bad Gateway on proxy connection/proto failure.
    
    Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tclegg at veritasgenetics.com>

diff --git a/lib/controller/federation_test.go b/lib/controller/federation_test.go
index 3848e0393..8ca1aa523 100644
--- a/lib/controller/federation_test.go
+++ b/lib/controller/federation_test.go
@@ -109,7 +109,7 @@ func (s *FederationSuite) checkHandledLocally(c *check.C, resp *http.Response) {
 	// it doesn't have its own stub/test Rails API, so we rely on
 	// "connection refused" to indicate the controller tried to
 	// proxy the request to its local Rails API.
-	c.Check(resp.StatusCode, check.Equals, http.StatusInternalServerError)
+	c.Check(resp.StatusCode, check.Equals, http.StatusBadGateway)
 	s.checkJSONErrorMatches(c, resp, `.*connection refused`)
 }
 
@@ -152,7 +152,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)
-	c.Check(resp.StatusCode, check.Equals, http.StatusInternalServerError)
+	c.Check(resp.StatusCode, check.Equals, http.StatusBadGateway)
 	s.checkJSONErrorMatches(c, resp, `.*HTTP response to HTTPS client`)
 }
 
diff --git a/lib/controller/handler_test.go b/lib/controller/handler_test.go
index c132b659d..3aedfb889 100644
--- a/lib/controller/handler_test.go
+++ b/lib/controller/handler_test.go
@@ -65,7 +65,7 @@ func (s *HandlerSuite) TestRequestTimeout(c *check.C) {
 	req := httptest.NewRequest("GET", "/discovery/v1/apis/arvados/v1/rest", nil)
 	resp := httptest.NewRecorder()
 	s.handler.ServeHTTP(resp, req)
-	c.Check(resp.Code, check.Equals, http.StatusInternalServerError)
+	c.Check(resp.Code, check.Equals, http.StatusBadGateway)
 	var jresp httpserver.ErrorResponse
 	err := json.Unmarshal(resp.Body.Bytes(), &jresp)
 	c.Check(err, check.IsNil)
diff --git a/lib/controller/proxy.go b/lib/controller/proxy.go
index 7905f91d0..1ff383d89 100644
--- a/lib/controller/proxy.go
+++ b/lib/controller/proxy.go
@@ -63,7 +63,7 @@ func (p *proxy) Do(w http.ResponseWriter, reqIn *http.Request, urlOut *url.URL,
 	}).WithContext(ctx)
 	resp, err := client.Do(reqOut)
 	if err != nil {
-		httpserver.Error(w, err.Error(), http.StatusInternalServerError)
+		httpserver.Error(w, err.Error(), http.StatusBadGateway)
 		return
 	}
 	for k, v := range resp.Header {

commit d22f92638664d9744a38e60dce9f0d95a2d393f0
Author: Tom Clegg <tclegg at veritasgenetics.com>
Date:   Fri Jul 6 15:57:20 2018 -0400

    13493: Fix excess newline.
    
    Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tclegg at veritasgenetics.com>

diff --git a/lib/controller/server_test.go b/lib/controller/server_test.go
index 3d06e7809..381e08948 100644
--- a/lib/controller/server_test.go
+++ b/lib/controller/server_test.go
@@ -22,7 +22,7 @@ type logWriter struct {
 }
 
 func (tl *logWriter) Write(buf []byte) (int, error) {
-	tl.logfunc(string(buf))
+	tl.logfunc(string(bytes.TrimRight(buf, "\n")))
 	return len(buf), nil
 }
 

commit 6b8a3d3d912573ab432c319121eaffd0bbe2216b
Author: Tom Clegg <tclegg at veritasgenetics.com>
Date:   Fri Jul 6 15:56:46 2018 -0400

    13493: Test DELETE and PUT methods.
    
    Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tclegg at veritasgenetics.com>

diff --git a/lib/controller/federation_test.go b/lib/controller/federation_test.go
index 08c8e8f53..3848e0393 100644
--- a/lib/controller/federation_test.go
+++ b/lib/controller/federation_test.go
@@ -176,29 +176,66 @@ func (s *FederationSuite) TestRemoteWithTokenInQuery(c *check.C) {
 	c.Check(pr.Header.Get("Authorization"), check.Equals, "Bearer "+arvadostest.ActiveToken)
 }
 
-func (s *FederationSuite) TestUpdateRemoteWorkflow(c *check.C) {
-	updateDescription := func(descr string) *http.Response {
-		req := httptest.NewRequest("PATCH", "/arvados/v1/workflows/"+arvadostest.WorkflowWithDefinitionYAMLUUID, strings.NewReader(url.Values{
-			"workflow": {`{"description":"` + descr + `"}`},
+func (s *FederationSuite) TestWorkflowCRUD(c *check.C) {
+	wf := arvados.Workflow{
+		Description: "TestCRUD",
+	}
+	{
+		body := &strings.Builder{}
+		json.NewEncoder(body).Encode(&wf)
+		req := httptest.NewRequest("POST", "/arvados/v1/workflows", strings.NewReader(url.Values{
+			"workflow": {body.String()},
 		}.Encode()))
 		req.Header.Set("Content-type", "application/x-www-form-urlencoded")
 		req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
-		resp := s.testRequest(req)
+		rec := httptest.NewRecorder()
+		s.remoteServer.Server.Handler.ServeHTTP(rec, req) // direct to remote -- can't proxy a create req because no uuid
+		resp := rec.Result()
 		s.checkResponseOK(c, resp)
-		return resp
+		json.NewDecoder(resp.Body).Decode(&wf)
+
+		defer func() {
+			req := httptest.NewRequest("DELETE", "/arvados/v1/workflows/"+wf.UUID, nil)
+			req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
+			s.remoteServer.Server.Handler.ServeHTTP(httptest.NewRecorder(), req)
+		}()
+		c.Check(wf.UUID, check.Not(check.Equals), "")
+
+		c.Assert(wf.ModifiedAt, check.NotNil)
+		c.Logf("wf.ModifiedAt: %v", wf.ModifiedAt)
+		c.Check(time.Since(*wf.ModifiedAt) < time.Minute, check.Equals, true)
 	}
+	for _, method := range []string{"PATCH", "PUT", "POST"} {
+		form := url.Values{
+			"workflow": {`{"description": "Updated with ` + method + `"}`},
+		}
+		if method == "POST" {
+			form["_method"] = []string{"PATCH"}
+		}
+		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)
+		s.checkResponseOK(c, resp)
+		err := json.NewDecoder(resp.Body).Decode(&wf)
+		c.Check(err, check.IsNil)
 
-	// Update description twice so running this test twice in a
-	// row still causes ModifiedAt to change
-	updateDescription("updated once by TestUpdateRemoteWorkflow")
-	resp := updateDescription("updated twice by TestUpdateRemoteWorkflow")
-
-	var wf arvados.Workflow
-	c.Check(json.NewDecoder(resp.Body).Decode(&wf), check.IsNil)
-	c.Check(wf.UUID, check.Equals, arvadostest.WorkflowWithDefinitionYAMLUUID)
-	c.Assert(wf.ModifiedAt, check.NotNil)
-	c.Logf("%s", *wf.ModifiedAt)
-	c.Check(time.Since(*wf.ModifiedAt) < time.Minute, check.Equals, true)
+		c.Check(wf.Description, check.Equals, "Updated with "+method)
+	}
+	{
+		req := httptest.NewRequest("DELETE", "/arvados/v1/workflows/"+wf.UUID, nil)
+		req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
+		resp := s.testRequest(req)
+		s.checkResponseOK(c, resp)
+		err := json.NewDecoder(resp.Body).Decode(&wf)
+		c.Check(err, check.IsNil)
+	}
+	{
+		req := httptest.NewRequest("GET", "/arvados/v1/workflows/"+wf.UUID, nil)
+		req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveToken)
+		resp := s.testRequest(req)
+		c.Check(resp.StatusCode, check.Equals, http.StatusNotFound)
+	}
 }
 
 func (s *FederationSuite) checkResponseOK(c *check.C, resp *http.Response) {
diff --git a/lib/controller/server_test.go b/lib/controller/server_test.go
index 2ad922244..3d06e7809 100644
--- a/lib/controller/server_test.go
+++ b/lib/controller/server_test.go
@@ -5,6 +5,7 @@
 package controller
 
 import (
+	"bytes"
 	"net/http"
 	"os"
 

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


hooks/post-receive
-- 




More information about the arvados-commits mailing list