ref: Move context.T out of rpc.ServerCall.

The purpose of this change is to make our usage of context.T more
consistent; it was a bit lame that we gave guidance to never wrap
context.T in another type, but were doing it ourselves.  The JS
code is also being changed to follow this convention (in separate
CLs), so we'll be consistent between Go and JS as well.

The server implementation used to look like this:
func (*impl) Foo(call rpc.ServerCall, ...)

Now it looks like this:
func (*impl) Foo(ctx *context.T, call rpc.ServerCall, ...)

Also added a ServerCall.Security() function, which returns the
security.Call.  The security.Call is still embedded inside
context.T for now; a subsequent change will remove it from
context.T and add an explicit security.Call argument where
necessary.  That's a separate CL since some of the choices may be
more controversial, and it's a smaller set of changes.

MultiPart: 2/8
Change-Id: If1ea84b4263836f7ddd82b965c35178a73d314cf
diff --git a/profiles/internal/rpc/benchmark/benchmark.vdl.go b/profiles/internal/rpc/benchmark/benchmark.vdl.go
index 9f9d9de..c67fec4 100644
--- a/profiles/internal/rpc/benchmark/benchmark.vdl.go
+++ b/profiles/internal/rpc/benchmark/benchmark.vdl.go
@@ -165,9 +165,9 @@
 // implements for Benchmark.
 type BenchmarkServerMethods interface {
 	// Echo returns the payload that it receives.
-	Echo(call rpc.ServerCall, Payload []byte) ([]byte, error)
+	Echo(ctx *context.T, call rpc.ServerCall, Payload []byte) ([]byte, error)
 	// EchoStream returns the payload that it receives via the stream.
-	EchoStream(BenchmarkEchoStreamServerCall) error
+	EchoStream(*context.T, BenchmarkEchoStreamServerCall) error
 }
 
 // BenchmarkServerStubMethods is the server interface containing
@@ -176,9 +176,9 @@
 // is the streaming methods.
 type BenchmarkServerStubMethods interface {
 	// Echo returns the payload that it receives.
-	Echo(call rpc.ServerCall, Payload []byte) ([]byte, error)
+	Echo(ctx *context.T, call rpc.ServerCall, Payload []byte) ([]byte, error)
 	// EchoStream returns the payload that it receives via the stream.
-	EchoStream(*BenchmarkEchoStreamServerCallStub) error
+	EchoStream(*context.T, *BenchmarkEchoStreamServerCallStub) error
 }
 
 // BenchmarkServerStub adds universal methods to BenchmarkServerStubMethods.
@@ -210,12 +210,12 @@
 	gs   *rpc.GlobState
 }
 
-func (s implBenchmarkServerStub) Echo(call rpc.ServerCall, i0 []byte) ([]byte, error) {
-	return s.impl.Echo(call, i0)
+func (s implBenchmarkServerStub) Echo(ctx *context.T, call rpc.ServerCall, i0 []byte) ([]byte, error) {
+	return s.impl.Echo(ctx, call, i0)
 }
 
-func (s implBenchmarkServerStub) EchoStream(call *BenchmarkEchoStreamServerCallStub) error {
-	return s.impl.EchoStream(call)
+func (s implBenchmarkServerStub) EchoStream(ctx *context.T, call *BenchmarkEchoStreamServerCallStub) error {
+	return s.impl.EchoStream(ctx, call)
 }
 
 func (s implBenchmarkServerStub) Globber() *rpc.GlobState {
diff --git a/profiles/internal/rpc/benchmark/glob/glob_test.go b/profiles/internal/rpc/benchmark/glob/glob_test.go
index 85cddf1..c37e867 100644
--- a/profiles/internal/rpc/benchmark/glob/glob_test.go
+++ b/profiles/internal/rpc/benchmark/glob/glob_test.go
@@ -102,7 +102,7 @@
 	bufferSize int
 }
 
-func (o *globObject) Glob__(call rpc.ServerCall, pattern string) (<-chan naming.GlobReply, error) {
+func (o *globObject) Glob__(_ *context.T, _ rpc.ServerCall, pattern string) (<-chan naming.GlobReply, error) {
 	if pattern != "*" {
 		panic("this benchmark only works with pattern='*'")
 	}
@@ -122,7 +122,7 @@
 	bufferSize int
 }
 
-func (o *globChildrenObject) GlobChildren__(call rpc.ServerCall) (<-chan string, error) {
+func (o *globChildrenObject) GlobChildren__(_ *context.T, call rpc.ServerCall) (<-chan string, error) {
 	if call.Suffix() != "" {
 		return nil, nil
 	}
diff --git a/profiles/internal/rpc/benchmark/internal/server.go b/profiles/internal/rpc/benchmark/internal/server.go
index 0c94e23..6cce472 100644
--- a/profiles/internal/rpc/benchmark/internal/server.go
+++ b/profiles/internal/rpc/benchmark/internal/server.go
@@ -16,11 +16,11 @@
 type impl struct {
 }
 
-func (i *impl) Echo(call rpc.ServerCall, payload []byte) ([]byte, error) {
+func (i *impl) Echo(_ *context.T, _ rpc.ServerCall, payload []byte) ([]byte, error) {
 	return payload, nil
 }
 
-func (i *impl) EchoStream(call benchmark.BenchmarkEchoStreamServerCall) error {
+func (i *impl) EchoStream(_ *context.T, call benchmark.BenchmarkEchoStreamServerCall) error {
 	rStream := call.RecvStream()
 	sStream := call.SendStream()
 	for rStream.Advance() {
diff --git a/profiles/internal/rpc/cancel_test.go b/profiles/internal/rpc/cancel_test.go
index 97813d7..ee4fc99 100644
--- a/profiles/internal/rpc/cancel_test.go
+++ b/profiles/internal/rpc/cancel_test.go
@@ -7,16 +7,15 @@
 import (
 	"testing"
 
-	"v.io/x/ref/profiles/internal/rpc/stream"
-	"v.io/x/ref/profiles/internal/rpc/stream/manager"
-	tnaming "v.io/x/ref/profiles/internal/testing/mocks/naming"
-
 	"v.io/v23"
 	"v.io/v23/context"
 	"v.io/v23/namespace"
 	"v.io/v23/naming"
 	"v.io/v23/rpc"
 	"v.io/x/lib/vlog"
+	"v.io/x/ref/profiles/internal/rpc/stream"
+	"v.io/x/ref/profiles/internal/rpc/stream/manager"
+	tnaming "v.io/x/ref/profiles/internal/testing/mocks/naming"
 )
 
 type fakeAuthorizer int
@@ -35,7 +34,7 @@
 	stop     func() error
 }
 
-func (c *canceld) Run(call rpc.StreamServerCall) error {
+func (c *canceld) Run(ctx *context.T, _ rpc.StreamServerCall) error {
 	close(c.started)
 
 	client, err := InternalNewClient(c.sm, c.ns)
@@ -45,14 +44,14 @@
 	}
 
 	if c.child != "" {
-		if _, err = client.StartCall(call.Context(), c.child, "Run", []interface{}{}); err != nil {
+		if _, err = client.StartCall(ctx, c.child, "Run", []interface{}{}); err != nil {
 			vlog.Error(err)
 			return err
 		}
 	}
 
 	vlog.Info(c.name, " waiting for cancellation")
-	<-call.Context().Done()
+	<-ctx.Done()
 	vlog.Info(c.name, " canceled")
 	close(c.canceled)
 	return nil
diff --git a/profiles/internal/rpc/debug_test.go b/profiles/internal/rpc/debug_test.go
index 035aef8..b4081fe 100644
--- a/profiles/internal/rpc/debug_test.go
+++ b/profiles/internal/rpc/debug_test.go
@@ -11,11 +11,11 @@
 	"testing"
 
 	"v.io/v23"
+	"v.io/v23/context"
 	"v.io/v23/naming"
 	"v.io/v23/options"
 	"v.io/v23/rpc"
 	"v.io/x/lib/vlog"
-
 	"v.io/x/ref/lib/stats"
 	"v.io/x/ref/profiles/internal/rpc/stream/manager"
 	tnaming "v.io/x/ref/profiles/internal/testing/mocks/naming"
@@ -127,6 +127,6 @@
 type testObject struct {
 }
 
-func (o testObject) Foo(rpc.ServerCall) (string, error) {
+func (o testObject) Foo(*context.T, rpc.ServerCall) (string, error) {
 	return "BAR", nil
 }
diff --git a/profiles/internal/rpc/discharges_test.go b/profiles/internal/rpc/discharges_test.go
index 79a5966..7072a69 100644
--- a/profiles/internal/rpc/discharges_test.go
+++ b/profiles/internal/rpc/discharges_test.go
@@ -10,7 +10,6 @@
 
 	"v.io/v23/security"
 	"v.io/v23/vdl"
-
 	"v.io/x/ref/test/testutil"
 )
 
diff --git a/profiles/internal/rpc/full_test.go b/profiles/internal/rpc/full_test.go
index 8d1ee86..f74e4a8 100644
--- a/profiles/internal/rpc/full_test.go
+++ b/profiles/internal/rpc/full_test.go
@@ -31,10 +31,8 @@
 	"v.io/v23/vdl"
 	"v.io/v23/verror"
 	"v.io/v23/vtrace"
-	"v.io/x/lib/vlog"
-	"v.io/x/ref/profiles/internal/rpc/stream"
-
 	"v.io/x/lib/netstate"
+	"v.io/x/lib/vlog"
 	"v.io/x/ref/lib/stats"
 	"v.io/x/ref/profiles/internal/lib/publisher"
 	"v.io/x/ref/profiles/internal/lib/websocket"
@@ -42,6 +40,7 @@
 	_ "v.io/x/ref/profiles/internal/rpc/protocols/tcp"
 	_ "v.io/x/ref/profiles/internal/rpc/protocols/ws"
 	_ "v.io/x/ref/profiles/internal/rpc/protocols/wsh"
+	"v.io/x/ref/profiles/internal/rpc/stream"
 	imanager "v.io/x/ref/profiles/internal/rpc/stream/manager"
 	"v.io/x/ref/profiles/internal/rpc/stream/vc"
 	tnaming "v.io/x/ref/profiles/internal/testing/mocks/naming"
@@ -88,33 +87,33 @@
 
 type testServer struct{}
 
-func (*testServer) Closure(call rpc.ServerCall) error {
+func (*testServer) Closure(*context.T, rpc.ServerCall) error {
 	return nil
 }
 
-func (*testServer) Error(call rpc.ServerCall) error {
+func (*testServer) Error(*context.T, rpc.ServerCall) error {
 	return errMethod
 }
 
-func (*testServer) Echo(call rpc.ServerCall, arg string) (string, error) {
+func (*testServer) Echo(_ *context.T, call rpc.ServerCall, arg string) (string, error) {
 	return fmt.Sprintf("method:%q,suffix:%q,arg:%q", "Echo", call.Suffix(), arg), nil
 }
 
-func (*testServer) EchoUser(call rpc.ServerCall, arg string, u userType) (string, userType, error) {
+func (*testServer) EchoUser(_ *context.T, call rpc.ServerCall, arg string, u userType) (string, userType, error) {
 	return fmt.Sprintf("method:%q,suffix:%q,arg:%q", "EchoUser", call.Suffix(), arg), u, nil
 }
 
-func (*testServer) EchoBlessings(call rpc.ServerCall) (server, client string, _ error) {
-	local := security.LocalBlessingNames(call.Context())
-	remote, _ := security.RemoteBlessingNames(call.Context())
+func (*testServer) EchoBlessings(ctx *context.T, _ rpc.ServerCall) (server, client string, _ error) {
+	local := security.LocalBlessingNames(ctx)
+	remote, _ := security.RemoteBlessingNames(ctx)
 	return fmt.Sprintf("%v", local), fmt.Sprintf("%v", remote), nil
 }
 
-func (*testServer) EchoGrantedBlessings(call rpc.ServerCall, arg string) (result, blessing string, _ error) {
+func (*testServer) EchoGrantedBlessings(_ *context.T, call rpc.ServerCall, arg string) (result, blessing string, _ error) {
 	return arg, fmt.Sprintf("%v", call.GrantedBlessings()), nil
 }
 
-func (*testServer) EchoAndError(call rpc.ServerCall, arg string) (string, error) {
+func (*testServer) EchoAndError(_ *context.T, call rpc.ServerCall, arg string) (string, error) {
 	result := fmt.Sprintf("method:%q,suffix:%q,arg:%q", "EchoAndError", call.Suffix(), arg)
 	if arg == "error" {
 		return result, errMethod
@@ -122,7 +121,7 @@
 	return result, nil
 }
 
-func (*testServer) Stream(call rpc.StreamServerCall, arg string) (string, error) {
+func (*testServer) Stream(_ *context.T, call rpc.StreamServerCall, arg string) (string, error) {
 	result := fmt.Sprintf("method:%q,suffix:%q,arg:%q", "Stream", call.Suffix(), arg)
 	var u userType
 	var err error
@@ -138,7 +137,7 @@
 	return result, err
 }
 
-func (*testServer) Unauthorized(rpc.StreamServerCall) (string, error) {
+func (*testServer) Unauthorized(*context.T, rpc.StreamServerCall) (string, error) {
 	return "UnauthorizedResult", nil
 }
 
@@ -202,7 +201,7 @@
 	called bool
 }
 
-func (ds *dischargeServer) Discharge(call rpc.StreamServerCall, cav security.Caveat, _ security.DischargeImpetus) (security.Discharge, error) {
+func (ds *dischargeServer) Discharge(ctx *context.T, _ rpc.StreamServerCall, cav security.Caveat, _ security.DischargeImpetus) (security.Discharge, error) {
 	ds.mu.Lock()
 	ds.called = true
 	ds.mu.Unlock()
@@ -210,7 +209,7 @@
 	if tp == nil {
 		return security.Discharge{}, fmt.Errorf("discharger: %v does not represent a third-party caveat", cav)
 	}
-	if err := tp.Dischargeable(call.Context()); err != nil {
+	if err := tp.Dischargeable(ctx); err != nil {
 		return security.Discharge{}, fmt.Errorf("third-party caveat %v cannot be discharged for this context: %v", cav, err)
 	}
 	// Add a fakeTimeCaveat to be able to control discharge expiration via 'clock'.
@@ -218,7 +217,7 @@
 	if err != nil {
 		return security.Discharge{}, fmt.Errorf("failed to create an expiration on the discharge: %v", err)
 	}
-	return security.GetCall(call.Context()).LocalPrincipal().MintDischarge(cav, expiry)
+	return security.GetCall(ctx).LocalPrincipal().MintDischarge(cav, expiry)
 }
 
 func startServer(t *testing.T, ctx *context.T, principal security.Principal, sm stream.Manager, ns namespace.T, name string, disp rpc.Dispatcher, opts ...rpc.ServerOpt) (naming.Endpoint, rpc.Server) {
@@ -870,9 +869,9 @@
 	traceid []uniqueid.Id
 }
 
-func (s *dischargeTestServer) Discharge(call rpc.ServerCall, cav security.Caveat, impetus security.DischargeImpetus) (security.Discharge, error) {
+func (s *dischargeTestServer) Discharge(ctx *context.T, _ rpc.ServerCall, cav security.Caveat, impetus security.DischargeImpetus) (security.Discharge, error) {
 	s.impetus = append(s.impetus, impetus)
-	s.traceid = append(s.traceid, vtrace.GetSpan(call.Context()).Trace())
+	s.traceid = append(s.traceid, vtrace.GetSpan(ctx).Trace())
 	return security.Discharge{}, fmt.Errorf("discharges not issued")
 }
 
@@ -1345,13 +1344,13 @@
 	}
 }
 
-func (s *cancelTestServer) CancelStreamReader(call rpc.StreamServerCall) error {
+func (s *cancelTestServer) CancelStreamReader(ctx *context.T, call rpc.StreamServerCall) error {
 	close(s.started)
 	var b []byte
 	if err := call.Recv(&b); err != io.EOF {
 		s.t.Errorf("Got error %v, want io.EOF", err)
 	}
-	<-call.Context().Done()
+	<-ctx.Done()
 	close(s.cancelled)
 	return nil
 }
@@ -1359,9 +1358,9 @@
 // CancelStreamIgnorer doesn't read from it's input stream so all it's
 // buffers fill.  The intention is to show that call.Done() is closed
 // even when the stream is stalled.
-func (s *cancelTestServer) CancelStreamIgnorer(call rpc.StreamServerCall) error {
+func (s *cancelTestServer) CancelStreamIgnorer(ctx *context.T, _ rpc.StreamServerCall) error {
 	close(s.started)
-	<-call.Context().Done()
+	<-ctx.Done()
 	close(s.cancelled)
 	return nil
 }
@@ -1420,7 +1419,7 @@
 
 type streamRecvInGoroutineServer struct{ c chan error }
 
-func (s *streamRecvInGoroutineServer) RecvInGoroutine(call rpc.StreamServerCall) error {
+func (s *streamRecvInGoroutineServer) RecvInGoroutine(_ *context.T, call rpc.StreamServerCall) error {
 	// Spawn a goroutine to read streaming data from the client.
 	go func() {
 		var i interface{}
@@ -1981,12 +1980,12 @@
 	called bool
 }
 
-func (ed *expiryDischarger) Discharge(call rpc.StreamServerCall, cav security.Caveat, _ security.DischargeImpetus) (security.Discharge, error) {
+func (ed *expiryDischarger) Discharge(ctx *context.T, _ rpc.StreamServerCall, cav security.Caveat, _ security.DischargeImpetus) (security.Discharge, error) {
 	tp := cav.ThirdPartyDetails()
 	if tp == nil {
 		return security.Discharge{}, fmt.Errorf("discharger: %v does not represent a third-party caveat", cav)
 	}
-	if err := tp.Dischargeable(call.Context()); err != nil {
+	if err := tp.Dischargeable(ctx); err != nil {
 		return security.Discharge{}, fmt.Errorf("third-party caveat %v cannot be discharged for this context: %v", cav, err)
 	}
 	expDur := 10 * time.Millisecond
@@ -1997,7 +1996,7 @@
 	if err != nil {
 		return security.Discharge{}, fmt.Errorf("failed to create an expiration on the discharge: %v", err)
 	}
-	d, err := security.GetCall(call.Context()).LocalPrincipal().MintDischarge(cav, expiry)
+	d, err := security.GetCall(ctx).LocalPrincipal().MintDischarge(cav, expiry)
 	if err != nil {
 		return security.Discharge{}, err
 	}
diff --git a/profiles/internal/rpc/reserved.go b/profiles/internal/rpc/reserved.go
index c095378..1b1f1ff 100644
--- a/profiles/internal/rpc/reserved.go
+++ b/profiles/internal/rpc/reserved.go
@@ -76,8 +76,8 @@
 	}}
 }
 
-func (r *reservedMethods) Signature(call rpc.ServerCall) ([]signature.Interface, error) {
-	ctx, suffix := call.Context(), call.Suffix()
+func (r *reservedMethods) Signature(ctx *context.T, call rpc.ServerCall) ([]signature.Interface, error) {
+	suffix := call.Suffix()
 	disp := r.dispNormal
 	if naming.IsReserved(suffix) {
 		disp = r.dispReserved
@@ -96,13 +96,13 @@
 	if err != nil {
 		return nil, err
 	}
-	sig, err := invoker.Signature(call)
+	sig, err := invoker.Signature(ctx, call)
 	if err != nil {
 		return nil, err
 	}
 	// Append the reserved methods.  We wait until now to add the "__" prefix to
 	// each method, so that we can use the regular ReflectInvoker.Signature logic.
-	rsig, err := r.selfInvoker.Signature(call)
+	rsig, err := r.selfInvoker.Signature(ctx, call)
 	if err != nil {
 		return nil, err
 	}
@@ -114,13 +114,13 @@
 	return signature.CleanInterfaces(append(sig, rsig...)), nil
 }
 
-func (r *reservedMethods) MethodSignature(call rpc.ServerCall, method string) (signature.Method, error) {
+func (r *reservedMethods) MethodSignature(ctx *context.T, call rpc.ServerCall, method string) (signature.Method, error) {
 	// Reserved methods use our self invoker, to describe our own methods,
 	if naming.IsReserved(method) {
-		return r.selfInvoker.MethodSignature(call, naming.StripReserved(method))
+		return r.selfInvoker.MethodSignature(ctx, call, naming.StripReserved(method))
 	}
 
-	ctx, suffix := call.Context(), call.Suffix()
+	suffix := call.Suffix()
 	disp := r.dispNormal
 	if naming.IsReserved(suffix) {
 		disp = r.dispReserved
@@ -141,13 +141,13 @@
 	}
 	// TODO(toddw): Decide if we should hide the method signature if the
 	// caller doesn't have access to call it.
-	return invoker.MethodSignature(call, method)
+	return invoker.MethodSignature(ctx, call, method)
 }
 
-func (r *reservedMethods) Glob(call rpc.StreamServerCall, pattern string) error {
+func (r *reservedMethods) Glob(ctx *context.T, call rpc.StreamServerCall, pattern string) error {
 	// Copy the original call to shield ourselves from changes the flowServer makes.
 	glob := globInternal{r.dispNormal, r.dispReserved, call.Suffix()}
-	return glob.Glob(call, pattern)
+	return glob.Glob(ctx, call, pattern)
 }
 
 // globInternal handles ALL the Glob requests received by a server and
@@ -188,7 +188,7 @@
 // levels.
 const maxRecursiveGlobDepth = 10
 
-func (i *globInternal) Glob(call rpc.StreamServerCall, pattern string) error {
+func (i *globInternal) Glob(ctx *context.T, call rpc.StreamServerCall, pattern string) error {
 	vlog.VI(3).Infof("rpc Glob: Incoming request: %q.Glob(%q)", i.receiver, pattern)
 	g, err := glob.Parse(pattern)
 	if err != nil {
@@ -201,9 +201,9 @@
 		tags = []*vdl.Value{vdl.ValueOf(access.Debug)}
 	}
 	if disp == nil {
-		return reserved.NewErrGlobNotImplemented(call.Context())
+		return reserved.NewErrGlobNotImplemented(ctx)
 	}
-	call = callWithMethodTags(call, tags)
+	ctx, call = callWithMethodTags(ctx, call, tags)
 
 	type gState struct {
 		name  string
@@ -215,7 +215,7 @@
 	someMatchesOmitted := false
 	for len(queue) != 0 {
 		select {
-		case <-call.Context().Done():
+		case <-ctx.Done():
 			// RPC timed out or was canceled.
 			return nil
 		default:
@@ -223,8 +223,8 @@
 		state := queue[0]
 		queue = queue[1:]
 
-		subcall := callWithSuffix(call, naming.Join(i.receiver, state.name))
-		ctx, suffix := subcall.Context(), subcall.Suffix()
+		ctx, subcall := callWithSuffix(ctx, call, naming.Join(i.receiver, state.name))
+		suffix := subcall.Suffix()
 		if state.depth > maxRecursiveGlobDepth {
 			vlog.Errorf("rpc Glob: exceeded recursion limit (%d): %q", maxRecursiveGlobDepth, suffix)
 			call.Send(naming.GlobReplyError{
@@ -278,7 +278,7 @@
 		}
 		if gs.AllGlobber != nil {
 			vlog.VI(3).Infof("rpc Glob: %q implements AllGlobber", suffix)
-			ch, err := gs.AllGlobber.Glob__(subcall, state.glob.String())
+			ch, err := gs.AllGlobber.Glob__(ctx, subcall, state.glob.String())
 			if err != nil {
 				vlog.VI(3).Infof("rpc Glob: %q.Glob(%q) failed: %v", suffix, state.glob, err)
 				subcall.Send(naming.GlobReplyError{naming.GlobError{Name: state.name, Error: verror.Convert(verror.ErrInternal, ctx, err)}})
@@ -300,7 +300,7 @@
 			continue
 		}
 		vlog.VI(3).Infof("rpc Glob: %q implements ChildrenGlobber", suffix)
-		children, err := gs.ChildrenGlobber.GlobChildren__(subcall)
+		children, err := gs.ChildrenGlobber.GlobChildren__(ctx, subcall)
 		// The requested object doesn't exist.
 		if err != nil {
 			subcall.Send(naming.GlobReplyError{naming.GlobError{Name: state.name, Error: verror.Convert(verror.ErrInternal, ctx, err)}})
@@ -331,7 +331,7 @@
 		}
 	}
 	if someMatchesOmitted {
-		call.Send(naming.GlobReplyError{naming.GlobError{Error: reserved.NewErrGlobMatchesOmitted(call.Context())}})
+		call.Send(naming.GlobReplyError{naming.GlobError{Error: reserved.NewErrGlobMatchesOmitted(ctx)}})
 	}
 	return nil
 }
@@ -340,20 +340,22 @@
 // useful for our various special-cased reserved methods.
 type derivedServerCall struct {
 	rpc.StreamServerCall
-	ctx    *context.T
-	suffix string
+	suffix   string
+	security security.Call
 }
 
-func callWithSuffix(src rpc.StreamServerCall, suffix string) rpc.StreamServerCall {
-	return &derivedServerCall{src, securityCallWithSuffix(src.Context(), suffix), suffix}
+func callWithSuffix(ctx *context.T, src rpc.StreamServerCall, suffix string) (*context.T, rpc.StreamServerCall) {
+	sec := securityCallWithSuffix(src.Security(), suffix)
+	return security.SetCall(ctx, sec), &derivedServerCall{src, suffix, sec}
 }
 
-func callWithMethodTags(src rpc.StreamServerCall, tags []*vdl.Value) rpc.StreamServerCall {
-	return &derivedServerCall{src, securityCallWithMethodTags(src.Context(), tags), src.Suffix()}
+func callWithMethodTags(ctx *context.T, src rpc.StreamServerCall, tags []*vdl.Value) (*context.T, rpc.StreamServerCall) {
+	sec := securityCallWithMethodTags(src.Security(), tags)
+	return security.SetCall(ctx, sec), &derivedServerCall{src, src.Suffix(), sec}
 }
 
-func (c *derivedServerCall) Context() *context.T { return c.ctx }
-func (c *derivedServerCall) Suffix() string      { return c.suffix }
+func (c *derivedServerCall) Suffix() string          { return c.suffix }
+func (c *derivedServerCall) Security() security.Call { return c.security }
 
 type derivedSecurityCall struct {
 	security.Call
@@ -361,22 +363,12 @@
 	methodTags []*vdl.Value
 }
 
-func securityCallWithSuffix(ctx *context.T, suffix string) *context.T {
-	secCall := security.GetCall(ctx)
-	return security.SetCall(ctx, &derivedSecurityCall{
-		Call:       secCall,
-		suffix:     suffix,
-		methodTags: secCall.MethodTags(),
-	})
+func securityCallWithSuffix(src security.Call, suffix string) security.Call {
+	return &derivedSecurityCall{src, suffix, src.MethodTags()}
 }
 
-func securityCallWithMethodTags(ctx *context.T, tags []*vdl.Value) *context.T {
-	secCall := security.GetCall(ctx)
-	return security.SetCall(ctx, &derivedSecurityCall{
-		Call:       secCall,
-		suffix:     secCall.Suffix(),
-		methodTags: tags,
-	})
+func securityCallWithMethodTags(src security.Call, tags []*vdl.Value) security.Call {
+	return &derivedSecurityCall{src, src.Suffix(), tags}
 }
 
 func (c *derivedSecurityCall) Suffix() string           { return c.suffix }
diff --git a/profiles/internal/rpc/server.go b/profiles/internal/rpc/server.go
index 76260fa..ad19468 100644
--- a/profiles/internal/rpc/server.go
+++ b/profiles/internal/rpc/server.go
@@ -951,7 +951,7 @@
 // flowServer implements the RPC server-side protocol for a single RPC, over a
 // flow that's already connected to the client.
 type flowServer struct {
-	*context.T
+	ctx    *context.T     // context associated with the RPC
 	server *server        // rpc.Server that this flow server belongs to
 	disp   rpc.Dispatcher // rpc.Dispatcher that will serve RPCs on this flow
 	dec    *vom.Decoder   // to decode requests and args from the client
@@ -969,7 +969,10 @@
 	endStreamArgs    bool // are the stream args at EOF?
 }
 
-var _ rpc.Stream = (*flowServer)(nil)
+var (
+	_ rpc.StreamServerCall = (*flowServer)(nil)
+	_ security.Call        = (*flowServer)(nil)
+)
 
 func newFlowServer(flow stream.Flow, server *server) (*flowServer, error) {
 	server.Lock()
@@ -977,15 +980,14 @@
 	server.Unlock()
 
 	fs := &flowServer{
-		T:          server.ctx,
+		ctx:        server.ctx,
 		server:     server,
 		disp:       disp,
 		flow:       flow,
 		discharges: make(map[string]security.Discharge),
 	}
-	// Attach the flow server to fs.T (the embedded *context.T) to act
-	// as a security.Call.
-	fs.T = security.SetCall(fs.T, fs)
+	// Attach the flow server to fs.ctx to act as a security.Call.
+	fs.ctx = security.SetCall(fs.ctx, fs)
 	var err error
 	typedec := flow.VCDataCache().Get(vc.TypeDecoderKey{})
 	if typedec == nil {
@@ -1017,11 +1019,11 @@
 func (fs *flowServer) authorizeVtrace() error {
 	// Set up a context as though we were calling __debug/vtrace.
 	params := &security.CallParams{}
-	params.Copy(security.GetCall(fs.T))
+	params.Copy(security.GetCall(fs.ctx))
 	params.Method = "Trace"
 	params.MethodTags = []*vdl.Value{vdl.ValueOf(access.Debug)}
 	params.Suffix = "__debug/vtrace"
-	ctx := security.SetCall(fs.T, security.NewCall(params))
+	ctx := security.SetCall(fs.ctx, security.NewCall(params))
 
 	var auth security.Authorizer
 	if fs.server.dispReserved != nil {
@@ -1035,12 +1037,12 @@
 
 	results, err := fs.processRequest()
 
-	vtrace.GetSpan(fs.T).Finish()
+	vtrace.GetSpan(fs.ctx).Finish()
 
 	var traceResponse vtrace.Response
 	// Check if the caller is permitted to view vtrace data.
 	if fs.authorizeVtrace() == nil {
-		traceResponse = vtrace.GetResponse(fs.T)
+		traceResponse = vtrace.GetResponse(fs.ctx)
 	}
 
 	// Respond to the client with the response header and positional results.
@@ -1055,7 +1057,7 @@
 		if err == io.EOF {
 			return err
 		}
-		return verror.New(errResponseEncoding, fs.Context(), fs.LocalEndpoint().String(), fs.RemoteEndpoint().String(), err)
+		return verror.New(errResponseEncoding, fs.ctx, fs.LocalEndpoint().String(), fs.RemoteEndpoint().String(), err)
 	}
 	if response.Error != nil {
 		return response.Error
@@ -1065,7 +1067,7 @@
 			if err == io.EOF {
 				return err
 			}
-			return verror.New(errResultEncoding, fs.Context(), ix, fmt.Sprintf("%T=%v", res, res), err)
+			return verror.New(errResultEncoding, fs.ctx, ix, fmt.Sprintf("%T=%v", res, res), err)
 		}
 	}
 	// TODO(ashankar): Should unread data from the flow be drained?
@@ -1095,7 +1097,7 @@
 	// Decode the initial request.
 	var req rpc.Request
 	if err := fs.dec.Decode(&req); err != nil {
-		return nil, verror.New(verror.ErrBadProtocol, fs.T, newErrBadRequest(fs.T, err))
+		return nil, verror.New(verror.ErrBadProtocol, fs.ctx, newErrBadRequest(fs.ctx, err))
 	}
 	return &req, nil
 }
@@ -1106,7 +1108,7 @@
 	if err != nil {
 		// We don't know what the rpc call was supposed to be, but we'll create
 		// a placeholder span so we can capture annotations.
-		fs.T, _ = vtrace.SetNewSpan(fs.T, fmt.Sprintf("\"%s\".UNKNOWN", fs.Name()))
+		fs.ctx, _ = vtrace.SetNewSpan(fs.ctx, fmt.Sprintf("\"%s\".UNKNOWN", fs.suffix))
 		return nil, err
 	}
 	fs.method = req.Method
@@ -1115,16 +1117,16 @@
 	// TODO(mattr): Currently this allows users to trigger trace collection
 	// on the server even if they will not be allowed to collect the
 	// results later.  This might be considered a DOS vector.
-	spanName := fmt.Sprintf("\"%s\".%s", fs.Name(), fs.Method())
-	fs.T, _ = vtrace.SetContinuedTrace(fs.T, spanName, req.TraceRequest)
+	spanName := fmt.Sprintf("\"%s\".%s", fs.suffix, fs.method)
+	fs.ctx, _ = vtrace.SetContinuedTrace(fs.ctx, spanName, req.TraceRequest)
 
 	var cancel context.CancelFunc
 	if !req.Deadline.IsZero() {
-		fs.T, cancel = context.WithDeadline(fs.T, req.Deadline.Time)
+		fs.ctx, cancel = context.WithDeadline(fs.ctx, req.Deadline.Time)
 	} else {
-		fs.T, cancel = context.WithCancel(fs.T)
+		fs.ctx, cancel = context.WithCancel(fs.ctx)
 	}
-	fs.flow.SetDeadline(fs.Done())
+	fs.flow.SetDeadline(fs.ctx.Done())
 	go fs.cancelContextOnClose(cancel)
 
 	// Initialize security: blessings, discharges, etc.
@@ -1150,21 +1152,21 @@
 		return nil, err
 	}
 	if called, want := req.NumPosArgs, uint64(len(argptrs)); called != want {
-		return nil, newErrBadNumInputArgs(fs.T, fs.suffix, fs.method, called, want)
+		return nil, newErrBadNumInputArgs(fs.ctx, fs.suffix, fs.method, called, want)
 	}
 	for ix, argptr := range argptrs {
 		if err := fs.dec.Decode(argptr); err != nil {
-			return nil, newErrBadInputArg(fs.T, fs.suffix, fs.method, uint64(ix), err)
+			return nil, newErrBadInputArg(fs.ctx, fs.suffix, fs.method, uint64(ix), err)
 		}
 	}
 
 	// Check application's authorization policy.
-	if err := authorize(fs.T, auth); err != nil {
+	if err := authorize(fs.ctx, auth); err != nil {
 		return nil, err
 	}
 
 	// Invoke the method.
-	results, err := invoker.Invoke(strippedMethod, fs, argptrs)
+	results, err := invoker.Invoke(fs.ctx, fs, strippedMethod, argptrs)
 	fs.server.stats.record(fs.method, time.Since(fs.starttime))
 	return results, err
 }
@@ -1180,7 +1182,7 @@
 		// matter that the context is also cancelled.
 		fs.flow.SetDeadline(nil)
 		cancel()
-	case <-fs.Done():
+	case <-fs.ctx.Done():
 	}
 }
 
@@ -1205,12 +1207,12 @@
 		case obj != nil:
 			invoker, err := objectToInvoker(obj)
 			if err != nil {
-				return nil, nil, verror.New(verror.ErrInternal, fs.T, "invalid received object", err)
+				return nil, nil, verror.New(verror.ErrInternal, fs.ctx, "invalid received object", err)
 			}
 			return invoker, auth, nil
 		}
 	}
-	return nil, nil, verror.New(verror.ErrUnknownSuffix, fs.T, suffix)
+	return nil, nil, verror.New(verror.ErrUnknownSuffix, fs.ctx, suffix)
 }
 
 func objectToInvoker(obj interface{}) (rpc.Invoker, error) {
@@ -1226,7 +1228,7 @@
 func (fs *flowServer) initSecurity(req *rpc.Request) error {
 	// LocalPrincipal is nil which means we are operating under
 	// SecurityNone.
-	if fs.flow.LocalPrincipal() == nil {
+	if fs.LocalPrincipal() == nil {
 		return nil
 	}
 
@@ -1238,8 +1240,8 @@
 	// the server's identity as the blessing. Figure out what we want to do about
 	// this - should servers be able to assume that a blessing is something that
 	// does not have the authorizations that the server's own identity has?
-	if got, want := req.GrantedBlessings.PublicKey(), fs.flow.LocalPrincipal().PublicKey(); got != nil && !reflect.DeepEqual(got, want) {
-		return verror.New(verror.ErrNoAccess, fs.T, fmt.Sprintf("blessing granted not bound to this server(%v vs %v)", got, want))
+	if got, want := req.GrantedBlessings.PublicKey(), fs.LocalPrincipal().PublicKey(); got != nil && !reflect.DeepEqual(got, want) {
+		return verror.New(verror.ErrNoAccess, fs.ctx, fmt.Sprintf("blessing granted not bound to this server(%v vs %v)", got, want))
 	}
 	fs.grantedBlessings = req.GrantedBlessings
 
@@ -1250,12 +1252,12 @@
 		// TODO(suharshs,toddw): Figure out a way to only shutdown the current VC, instead
 		// of all VCs connected to the RemoteEndpoint.
 		fs.server.streamMgr.ShutdownEndpoint(fs.RemoteEndpoint())
-		return verror.New(verror.ErrBadProtocol, fs.T, newErrBadBlessingsCache(fs.T, err))
+		return verror.New(verror.ErrBadProtocol, fs.ctx, newErrBadBlessingsCache(fs.ctx, err))
 	}
 	// Verify that the blessings sent by the client in the request have the same public
 	// key as those sent by the client during VC establishment.
 	if got, want := fs.clientBlessings.PublicKey(), fs.flow.RemoteBlessings().PublicKey(); got != nil && !reflect.DeepEqual(got, want) {
-		return verror.New(verror.ErrNoAccess, fs.T, fmt.Sprintf("blessings sent with the request are bound to a different public key (%v) from the blessing used during VC establishment (%v)", got, want))
+		return verror.New(verror.ErrNoAccess, fs.ctx, fmt.Sprintf("blessings sent with the request are bound to a different public key (%v) from the blessing used during VC establishment (%v)", got, want))
 	}
 	fs.ackBlessings = true
 
@@ -1312,8 +1314,12 @@
 	return fs.dec.Decode(itemptr)
 }
 
-// Implementations of rpc.ServerCall methods.
+// Implementations of rpc.ServerCall and security.Call methods.
 
+func (fs *flowServer) Security() security.Call {
+	//nologcall
+	return fs
+}
 func (fs *flowServer) LocalDischarges() map[string]security.Discharge {
 	//nologcall
 	return fs.flow.LocalDischarges()
@@ -1338,20 +1344,6 @@
 	//nologcall
 	return fs.tags
 }
-func (fs *flowServer) Context() *context.T {
-	return fs.T
-}
-
-func (fs *flowServer) VanadiumContext() *context.T {
-	return fs.T
-}
-
-// TODO(cnicolaou): remove Name from rpc.ServerCall and all of
-// its implementations
-func (fs *flowServer) Name() string {
-	//nologcall
-	return fs.suffix
-}
 func (fs *flowServer) Suffix() string {
 	//nologcall
 	return fs.suffix
diff --git a/profiles/internal/rpc/server_test.go b/profiles/internal/rpc/server_test.go
index d2c72d7..095e902 100644
--- a/profiles/internal/rpc/server_test.go
+++ b/profiles/internal/rpc/server_test.go
@@ -34,7 +34,7 @@
 }
 type noExportedFieldsType struct{}
 
-func (noExportedFieldsType) F(_ rpc.ServerCall, f fieldType) error { return nil }
+func (noExportedFieldsType) F(_ *context.T, _ rpc.ServerCall, f fieldType) error { return nil }
 
 type badObjectDispatcher struct{}
 
@@ -129,7 +129,7 @@
 
 type statusServer struct{ ch chan struct{} }
 
-func (s *statusServer) Hang(call rpc.ServerCall) error {
+func (s *statusServer) Hang(*context.T, rpc.ServerCall) error {
 	<-s.ch
 	return nil
 }
diff --git a/profiles/internal/rpc/stress/internal/server.go b/profiles/internal/rpc/stress/internal/server.go
index 217995c..d6520e3 100644
--- a/profiles/internal/rpc/stress/internal/server.go
+++ b/profiles/internal/rpc/stress/internal/server.go
@@ -24,12 +24,12 @@
 	stop chan struct{}
 }
 
-func (s *impl) Sum(call rpc.ServerCall, arg stress.Arg) ([]byte, error) {
+func (s *impl) Sum(_ *context.T, _ rpc.ServerCall, arg stress.Arg) ([]byte, error) {
 	defer s.incSumCount()
 	return doSum(arg)
 }
 
-func (s *impl) SumStream(call stress.StressSumStreamServerCall) error {
+func (s *impl) SumStream(_ *context.T, call stress.StressSumStreamServerCall) error {
 	defer s.incSumStreamCount()
 	rStream := call.RecvStream()
 	sStream := call.SendStream()
@@ -46,13 +46,13 @@
 	return nil
 }
 
-func (s *impl) GetStats(call rpc.ServerCall) (stress.Stats, error) {
+func (s *impl) GetStats(*context.T, rpc.ServerCall) (stress.Stats, error) {
 	s.statsMu.Lock()
 	defer s.statsMu.Unlock()
 	return stress.Stats{s.sumCount, s.sumStreamCount}, nil
 }
 
-func (s *impl) Stop(call rpc.ServerCall) error {
+func (s *impl) Stop(*context.T, rpc.ServerCall) error {
 	s.stop <- struct{}{}
 	return nil
 }
diff --git a/profiles/internal/rpc/stress/stress.vdl.go b/profiles/internal/rpc/stress/stress.vdl.go
index 327a576..505c788 100644
--- a/profiles/internal/rpc/stress/stress.vdl.go
+++ b/profiles/internal/rpc/stress/stress.vdl.go
@@ -203,13 +203,13 @@
 // implements for Stress.
 type StressServerMethods interface {
 	// Do returns the checksum of the payload that it receives.
-	Sum(call rpc.ServerCall, arg Arg) ([]byte, error)
+	Sum(ctx *context.T, call rpc.ServerCall, arg Arg) ([]byte, error)
 	// DoStream returns the checksum of the payload that it receives via the stream.
-	SumStream(StressSumStreamServerCall) error
+	SumStream(*context.T, StressSumStreamServerCall) error
 	// GetStats returns the stats on the calls that the server received.
-	GetStats(rpc.ServerCall) (Stats, error)
+	GetStats(*context.T, rpc.ServerCall) (Stats, error)
 	// Stop stops the server.
-	Stop(rpc.ServerCall) error
+	Stop(*context.T, rpc.ServerCall) error
 }
 
 // StressServerStubMethods is the server interface containing
@@ -218,13 +218,13 @@
 // is the streaming methods.
 type StressServerStubMethods interface {
 	// Do returns the checksum of the payload that it receives.
-	Sum(call rpc.ServerCall, arg Arg) ([]byte, error)
+	Sum(ctx *context.T, call rpc.ServerCall, arg Arg) ([]byte, error)
 	// DoStream returns the checksum of the payload that it receives via the stream.
-	SumStream(*StressSumStreamServerCallStub) error
+	SumStream(*context.T, *StressSumStreamServerCallStub) error
 	// GetStats returns the stats on the calls that the server received.
-	GetStats(rpc.ServerCall) (Stats, error)
+	GetStats(*context.T, rpc.ServerCall) (Stats, error)
 	// Stop stops the server.
-	Stop(rpc.ServerCall) error
+	Stop(*context.T, rpc.ServerCall) error
 }
 
 // StressServerStub adds universal methods to StressServerStubMethods.
@@ -256,20 +256,20 @@
 	gs   *rpc.GlobState
 }
 
-func (s implStressServerStub) Sum(call rpc.ServerCall, i0 Arg) ([]byte, error) {
-	return s.impl.Sum(call, i0)
+func (s implStressServerStub) Sum(ctx *context.T, call rpc.ServerCall, i0 Arg) ([]byte, error) {
+	return s.impl.Sum(ctx, call, i0)
 }
 
-func (s implStressServerStub) SumStream(call *StressSumStreamServerCallStub) error {
-	return s.impl.SumStream(call)
+func (s implStressServerStub) SumStream(ctx *context.T, call *StressSumStreamServerCallStub) error {
+	return s.impl.SumStream(ctx, call)
 }
 
-func (s implStressServerStub) GetStats(call rpc.ServerCall) (Stats, error) {
-	return s.impl.GetStats(call)
+func (s implStressServerStub) GetStats(ctx *context.T, call rpc.ServerCall) (Stats, error) {
+	return s.impl.GetStats(ctx, call)
 }
 
-func (s implStressServerStub) Stop(call rpc.ServerCall) error {
-	return s.impl.Stop(call)
+func (s implStressServerStub) Stop(ctx *context.T, call rpc.ServerCall) error {
+	return s.impl.Stop(ctx, call)
 }
 
 func (s implStressServerStub) Globber() *rpc.GlobState {
diff --git a/profiles/internal/rpc/test/client_test.go b/profiles/internal/rpc/test/client_test.go
index 31de6ea..62a094b 100644
--- a/profiles/internal/rpc/test/client_test.go
+++ b/profiles/internal/rpc/test/client_test.go
@@ -23,7 +23,6 @@
 	"v.io/v23/security"
 	"v.io/v23/vdlroot/signature"
 	"v.io/v23/verror"
-
 	"v.io/x/ref/envvar"
 	_ "v.io/x/ref/profiles"
 	inaming "v.io/x/ref/profiles/internal/naming"
@@ -85,14 +84,14 @@
 	id, suffix string
 }
 
-func (es *echoServerObject) Echo(call rpc.ServerCall, m string) (string, error) {
+func (es *echoServerObject) Echo(_ *context.T, _ rpc.ServerCall, m string) (string, error) {
 	if len(es.suffix) > 0 {
 		return fmt.Sprintf("%s.%s: %s\n", es.id, es.suffix, m), nil
 	}
 	return fmt.Sprintf("%s: %s\n", es.id, m), nil
 }
 
-func (es *echoServerObject) Sleep(call rpc.ServerCall, d string) error {
+func (es *echoServerObject) Sleep(_ *context.T, _ rpc.ServerCall, d string) error {
 	duration, err := time.ParseDuration(d)
 	if err != nil {
 		return err
diff --git a/profiles/internal/rpc/test/glob_test.go b/profiles/internal/rpc/test/glob_test.go
index 579e0a3..792b0c2 100644
--- a/profiles/internal/rpc/test/glob_test.go
+++ b/profiles/internal/rpc/test/glob_test.go
@@ -19,7 +19,6 @@
 	"v.io/v23/rpc/reserved"
 	"v.io/v23/security"
 	"v.io/v23/verror"
-
 	"v.io/x/ref/lib/glob"
 	_ "v.io/x/ref/profiles"
 	"v.io/x/ref/test"
@@ -300,14 +299,14 @@
 	suffix []string
 }
 
-func (o *globObject) Glob__(call rpc.ServerCall, pattern string) (<-chan naming.GlobReply, error) {
+func (o *globObject) Glob__(ctx *context.T, _ rpc.ServerCall, pattern string) (<-chan naming.GlobReply, error) {
 	g, err := glob.Parse(pattern)
 	if err != nil {
 		return nil, err
 	}
 	n := o.n.find(o.suffix, false)
 	if n == nil {
-		return nil, verror.New(verror.ErrNoExist, call.Context(), o.suffix)
+		return nil, verror.New(verror.ErrNoExist, ctx, o.suffix)
 	}
 	ch := make(chan naming.GlobReply)
 	go func() {
@@ -336,10 +335,10 @@
 	suffix []string
 }
 
-func (o *vChildrenObject) GlobChildren__(call rpc.ServerCall) (<-chan string, error) {
+func (o *vChildrenObject) GlobChildren__(ctx *context.T, _ rpc.ServerCall) (<-chan string, error) {
 	n := o.n.find(o.suffix, false)
 	if n == nil {
-		return nil, verror.New(verror.ErrNoExist, call.Context(), o.suffix)
+		return nil, verror.New(verror.ErrNoExist, ctx, o.suffix)
 	}
 	ch := make(chan string, len(n.children))
 	for child, _ := range n.children {
@@ -383,6 +382,6 @@
 
 type leafObject struct{}
 
-func (l leafObject) Func(call rpc.ServerCall) error {
+func (l leafObject) Func(*context.T, rpc.ServerCall) error {
 	return nil
 }
diff --git a/profiles/internal/rpc/test/proxy_test.go b/profiles/internal/rpc/test/proxy_test.go
index ed47050..a162ce4 100644
--- a/profiles/internal/rpc/test/proxy_test.go
+++ b/profiles/internal/rpc/test/proxy_test.go
@@ -23,7 +23,6 @@
 	"v.io/v23/security"
 	"v.io/v23/verror"
 	"v.io/v23/vtrace"
-
 	"v.io/x/ref/lib/flags"
 	_ "v.io/x/ref/profiles"
 	"v.io/x/ref/profiles/internal/lib/publisher"
@@ -92,7 +91,7 @@
 
 type testServer struct{}
 
-func (*testServer) Echo(call rpc.ServerCall, arg string) (string, error) {
+func (*testServer) Echo(_ *context.T, call rpc.ServerCall, arg string) (string, error) {
 	return fmt.Sprintf("method:%q,suffix:%q,arg:%q", "Echo", call.Suffix(), arg), nil
 }
 
diff --git a/profiles/internal/rpc/test/retry_test.go b/profiles/internal/rpc/test/retry_test.go
index ad20630..d8d3ad4 100644
--- a/profiles/internal/rpc/test/retry_test.go
+++ b/profiles/internal/rpc/test/retry_test.go
@@ -20,7 +20,7 @@
 	called int // number of times TryAgain has been called
 }
 
-func (s *retryServer) TryAgain(call rpc.ServerCall) error {
+func (s *retryServer) TryAgain(ctx *context.T, _ rpc.ServerCall) error {
 	// If this is the second time this method is being called, return success.
 	if s.called > 0 {
 		s.called++
@@ -28,7 +28,7 @@
 	}
 	s.called++
 	// otherwise, return a verror with action code RetryBackoff.
-	return verror.New(errRetryThis, call.Context())
+	return verror.New(errRetryThis, ctx)
 }
 
 type allowEveryoneAuth struct{}
diff --git a/profiles/internal/rpc/test/signature_test.go b/profiles/internal/rpc/test/signature_test.go
index d653e01..2894418 100644
--- a/profiles/internal/rpc/test/signature_test.go
+++ b/profiles/internal/rpc/test/signature_test.go
@@ -16,7 +16,6 @@
 	"v.io/v23/rpc/reserved"
 	"v.io/v23/vdl"
 	"v.io/v23/vdlroot/signature"
-
 	_ "v.io/x/ref/profiles"
 	"v.io/x/ref/test"
 )
@@ -38,10 +37,10 @@
 
 type sigImpl struct{}
 
-func (sigImpl) NonStreaming0(rpc.ServerCall) error                       { panic("X") }
-func (sigImpl) NonStreaming1(_ rpc.ServerCall, _ string) (int64, error)  { panic("X") }
-func (sigImpl) Streaming0(_ *streamStringBool) error                     { panic("X") }
-func (sigImpl) Streaming1(_ *streamStringBool, _ int64) (float64, error) { panic("X") }
+func (sigImpl) NonStreaming0(*context.T, rpc.ServerCall) error                   { panic("X") }
+func (sigImpl) NonStreaming1(*context.T, rpc.ServerCall, string) (int64, error)  { panic("X") }
+func (sigImpl) Streaming0(*context.T, *streamStringBool) error                   { panic("X") }
+func (sigImpl) Streaming1(*context.T, *streamStringBool, int64) (float64, error) { panic("X") }
 
 type streamStringBool struct{ rpc.StreamServerCall }
 
diff --git a/profiles/internal/rpc/test/simple_test.go b/profiles/internal/rpc/test/simple_test.go
index c83ec25..22c3da4 100644
--- a/profiles/internal/rpc/test/simple_test.go
+++ b/profiles/internal/rpc/test/simple_test.go
@@ -10,6 +10,7 @@
 	"time"
 
 	"v.io/v23"
+	"v.io/v23/context"
 	"v.io/v23/rpc"
 )
 
@@ -17,7 +18,7 @@
 	done <-chan struct{}
 }
 
-func (s *simple) Sleep(call rpc.ServerCall) error {
+func (s *simple) Sleep(*context.T, rpc.ServerCall) error {
 	select {
 	case <-s.done:
 	case <-time.After(time.Hour):
@@ -25,15 +26,15 @@
 	return nil
 }
 
-func (s *simple) Ping(call rpc.ServerCall) (string, error) {
+func (s *simple) Ping(_ *context.T, _ rpc.ServerCall) (string, error) {
 	return "pong", nil
 }
 
-func (s *simple) Echo(call rpc.ServerCall, arg string) (string, error) {
+func (s *simple) Echo(_ *context.T, _ rpc.ServerCall, arg string) (string, error) {
 	return arg, nil
 }
 
-func (s *simple) Source(call rpc.StreamServerCall, start int) error {
+func (s *simple) Source(_ *context.T, call rpc.StreamServerCall, start int) error {
 	i := start
 	backoff := 25 * time.Millisecond
 	for {
@@ -48,7 +49,7 @@
 	}
 }
 
-func (s *simple) Sink(call rpc.StreamServerCall) (int, error) {
+func (s *simple) Sink(_ *context.T, call rpc.StreamServerCall) (int, error) {
 	i := 0
 	for {
 		if err := call.Recv(&i); err != nil {
@@ -60,7 +61,7 @@
 	}
 }
 
-func (s *simple) Inc(call rpc.StreamServerCall, inc int) (int, error) {
+func (s *simple) Inc(_ *context.T, call rpc.StreamServerCall, inc int) (int, error) {
 	i := 0
 	for {
 		if err := call.Recv(&i); err != nil {
diff --git a/profiles/internal/rpc/testutil_test.go b/profiles/internal/rpc/testutil_test.go
index b48f6c6..1e51fa4 100644
--- a/profiles/internal/rpc/testutil_test.go
+++ b/profiles/internal/rpc/testutil_test.go
@@ -103,9 +103,10 @@
 	m        string
 	ld, rd   security.Discharge
 	lep, rep naming.Endpoint
-	c        *context.T
 }
 
+var _ security.Call = (*mockCall)(nil)
+
 func (c *mockCall) Timestamp() (t time.Time) { return }
 func (c *mockCall) Method() string           { return c.m }
 func (c *mockCall) MethodTags() []*vdl.Value { return nil }
@@ -121,4 +122,3 @@
 func (c *mockCall) LocalPrincipal() security.Principal  { return c.p }
 func (c *mockCall) LocalBlessings() security.Blessings  { return c.l }
 func (c *mockCall) RemoteBlessings() security.Blessings { return c.r }
-func (c *mockCall) Context() *context.T                 { return c.c }