runtimes/google/ipc: switch to using verror2.

- use verror2 in ipc.Client, but don't yet change
the return types for StartCall, Finish etc.
- this makes extensive use of local verror2 definitions
in an attempt to support internationalization of 'detailed'
error messages without bloating the number of public ones.

Change-Id: I998db3c72001fb1683751ec55f941411197a46cb
diff --git a/runtimes/google/ipc/client.go b/runtimes/google/ipc/client.go
index 2121314..fe3f167 100644
--- a/runtimes/google/ipc/client.go
+++ b/runtimes/google/ipc/client.go
@@ -16,23 +16,65 @@
 	"veyron.io/veyron/veyron/runtimes/google/vtrace"
 
 	"veyron.io/veyron/veyron2/context"
+	"veyron.io/veyron/veyron2/i18n"
 	"veyron.io/veyron/veyron2/ipc"
 	"veyron.io/veyron/veyron2/ipc/stream"
 	"veyron.io/veyron/veyron2/naming"
 	"veyron.io/veyron/veyron2/options"
 	"veyron.io/veyron/veyron2/security"
 	"veyron.io/veyron/veyron2/vdl/vdlutil"
-	"veyron.io/veyron/veyron2/verror"
+	old_verror "veyron.io/veyron/veyron2/verror"
+	verror "veyron.io/veyron/veyron2/verror2"
 	"veyron.io/veyron/veyron2/vlog"
 	"veyron.io/veyron/veyron2/vom"
 )
 
+const pkgPath = "veyron.io/veyron/veyron/runtimes/google/ipc"
+
 var (
-	errNoServers              = verror.NoExistf("ipc: no servers")
-	errNoAccess               = verror.NoAccessf("ipc: client unwilling to access to server")
-	errFlowClosed             = verror.Abortedf("ipc: flow closed")
-	errRemainingStreamResults = verror.BadProtocolf("ipc: Finish called with remaining streaming results")
-	errNonRootedName          = verror.BadArgf("ipc: cannot connect to a non-rooted name")
+	// Local errs that are used to provide details to the public ones.
+	errClientCloseAlreadyCalled = verror.Register(pkgPath+".closeAlreadyCalled", verror.NoRetry,
+		"ipc.Client.Close has already been called")
+
+	errClientFinishAlreadyCalled = verror.Register(pkgPath+".finishAlreadyCalled", verror.NoRetry, "ipc.Call.Finish has already been called")
+
+	errNonRootedName = verror.Register(pkgPath+".nonRootedName", verror.NoRetry, "{3} does not appear to contain an address")
+
+	errInvalidEndpoint = verror.Register(pkgPath+".invalidEndpoint", verror.RetryRefetch, "{3} is an invalid endpoint")
+
+	errIncompatibleEndpoint = verror.Register(pkgPath+".invalidEndpoint", verror.RetryRefetch, "{3} is an incompatible endpoint")
+
+	errNotTrusted = verror.Register(pkgPath+".notTrusted", verror.RetryConnection, "name {3} not trusted using blessings {4}{:5}")
+
+	errAuthError = verror.Register(pkgPath+".authError", verror.RetryRefetch, "authentication error from server {3}{:4}")
+
+	errSystemRetry   = verror.Register(pkgPath+".sysErrorRetryConnection", verror.RetryConnection, "{:3:}")
+	errSystemNoRetry = verror.Register(pkgPath+".sysErrorNoRetry", verror.NoRetry, "{:3:}")
+
+	errRequestEncoding = verror.Register(pkgPath+".requestEncoding", verror.NoRetry, "failed to encode request {3}{:4}")
+
+	errDischargeEncoding = verror.Register(pkgPath+".dischargeEncoding", verror.NoRetry, "failed to encode discharge {3}{:4}")
+
+	errArgEncoding = verror.Register(pkgPath+".argEncoding", verror.NoRetry, "failed to encode arg #{3}{:4:}")
+
+	errMismatchedResults = verror.Register(pkgPath+".mismatchedResults", verror.NoRetry, "expected {3} results, but got {4}")
+
+	errResultDecoding = verror.Register(pkgPath+".resultDecoding", verror.NoRetry, "failed to decode result #{3}{:4}")
+
+	errResponseDecoding = verror.Register(pkgPath+".responseDecoding", verror.NoRetry, "failed to decode response{:3}")
+
+	errRemainingStreamResults = verror.Register(pkgPath+".remaingStreamResults", verror.NoRetry, "stream closed with remaining stream results")
+
+	errNoBlessings = verror.Register(pkgPath+".noBlessings", verror.NoRetry, "server has not presented any blessings")
+
+	errAuthNoPatternMatch = verror.Register(pkgPath+".authNoPatternMatch",
+		verror.NoRetry, "server blessings {3} do not match pattern {4}")
+
+	errDefaultAuthDenied = verror.Register(pkgPath+".defaultAuthDenied", verror.NoRetry, "default authorization precludes talking to server with blessings{:3}")
+
+	errBlessingGrant = verror.Register(pkgPath+".blessingGrantFailed", verror.NoRetry, "failed to grant blessing to server with blessings {3}{:4}")
+
+	errBlessingAdd = verror.Register(pkgPath+".blessingAddFailed", verror.NoRetry, "failed to add blessing granted to server {3}{:4}")
 )
 
 var serverPatternRegexp = regexp.MustCompile("^\\[([^\\]]+)\\](.*)")
@@ -87,11 +129,11 @@
 	return c, nil
 }
 
-func (c *client) createFlow(ep naming.Endpoint) (stream.Flow, error) {
+func (c *client) createFlow(ctx context.T, ep naming.Endpoint) (stream.Flow, verror.E) {
 	c.vcMapMu.Lock()
 	defer c.vcMapMu.Unlock()
 	if c.vcMap == nil {
-		return nil, fmt.Errorf("client has been closed")
+		return nil, verror.Make(errClientCloseAlreadyCalled, ctx)
 	}
 	if vcinfo := c.vcMap[ep.String()]; vcinfo != nil {
 		if flow, err := vcinfo.vc.Connect(); err == nil {
@@ -112,11 +154,15 @@
 	vc, err := sm.Dial(ep, vcOpts...)
 	c.vcMapMu.Lock()
 	if err != nil {
-		return nil, err
+		if strings.Contains(err.Error(), "authentication failed") {
+			return nil, verror.Make(errAuthError, ctx, ep, err)
+		} else {
+			return nil, verror.Make(errSystemRetry, ctx, err)
+		}
 	}
 	if c.vcMap == nil {
 		sm.ShutdownEndpoint(ep)
-		return nil, fmt.Errorf("client has been closed")
+		return nil, verror.Make(errClientCloseAlreadyCalled, ctx)
 	}
 	if othervc, exists := c.vcMap[ep.String()]; exists {
 		vc = othervc.vc
@@ -125,33 +171,40 @@
 	} else {
 		c.vcMap[ep.String()] = &vcInfo{vc: vc, remoteEP: ep}
 	}
-	return vc.Connect()
+	flow, err := vc.Connect()
+	if err != nil {
+
+		return nil, verror.Make(errAuthError, ctx, ep, err)
+	}
+	return flow, nil
 }
 
 // connectFlow parses an endpoint and a suffix out of the server and establishes
 // a flow to the endpoint, returning the parsed suffix.
 // The server name passed in should be a rooted name, of the form "/ep/suffix" or
 // "/ep//suffix", or just "/ep".
-func (c *client) connectFlow(server string) (stream.Flow, string, error) {
+func (c *client) connectFlow(ctx context.T, server string) (stream.Flow, string, verror.E) {
 	address, suffix := naming.SplitAddressName(server)
 	if len(address) == 0 {
-		return nil, "", errNonRootedName
+		return nil, "", verror.Make(errNonRootedName, ctx, server)
 	}
 	ep, err := inaming.NewEndpoint(address)
 	if err != nil {
-		return nil, "", err
+		return nil, "", verror.Make(errInvalidEndpoint, ctx, address)
 	}
 	if err = version.CheckCompatibility(ep); err != nil {
-		return nil, "", err
+		return nil, "", verror.Make(errIncompatibleEndpoint, ctx, ep)
 	}
-	flow, err := c.createFlow(ep)
-	if err != nil {
-		return nil, "", err
+	flow, verr := c.createFlow(ctx, ep)
+	if verr != nil {
+		return nil, "", verr
 	}
 	return flow, suffix, nil
 }
 
 // A randomized exponential backoff.  The randomness deters error convoys from forming.
+// TODO(cnicolaou): rationalize this and the backoff in ipc.Server. Note
+// that rand is not thread safe and may crash.
 func backoff(n int, deadline time.Time) bool {
 	b := time.Duration(math.Pow(1.5+(rand.Float64()/2.0), float64(n)) * float64(time.Second))
 	if b > maxBackoff {
@@ -171,29 +224,6 @@
 	return true
 }
 
-// TODO(p): replace these checks with m3b's retry bit when it exists.  This is currently a colossal hack.
-func retriable(err error) bool {
-	e := err.Error()
-	// Authentication errors are permanent.
-	if strings.Contains(e, "authorized") {
-		return false
-	}
-	// Resolution errors are retriable.
-	if strings.Contains(e, "ipc: Resolve") {
-		return true
-	}
-	// Kernel level errors are retriable.
-	if strings.Contains(e, "errno") {
-		return true
-	}
-	// Connection refused is retriable.
-	if strings.Contains(e, "connection refused") {
-		return true
-	}
-
-	return false
-}
-
 func getRetryTimeoutOpt(opts []ipc.CallOpt) (time.Duration, bool) {
 	for _, o := range opts {
 		if r, ok := o.(options.RetryTimeout); ok {
@@ -238,8 +268,9 @@
 // startCall ensures StartCall always returns verror.E.
 func (c *client) startCall(ctx context.T, name, method string, args []interface{}, opts []ipc.CallOpt) (ipc.Call, verror.E) {
 	if ctx == nil {
-		return nil, verror.BadArgf("ipc: %s.%s called with nil context", name, method)
+		return nil, verror.ExplicitMake(verror.BadArg, i18n.NoLangID, "ipc.Client", "StartCall")
 	}
+	ctx = verror.ContextWithComponentName(ctx, "ipc.Client")
 
 	// Context specified deadline.
 	deadline, hasDeadline := ctx.Deadline()
@@ -263,7 +294,7 @@
 			return call, nil
 		}
 		lastErr = err
-		if time.Now().After(deadline) || !retriable(err) {
+		if time.Now().After(deadline) || err.Action() != verror.RetryConnection {
 			break
 		}
 	}
@@ -271,20 +302,19 @@
 }
 
 type serverStatus struct {
-	index     int
-	suffix    string
-	flow      stream.Flow
-	errConn   verror.E
-	errAccess verror.E
+	index  int
+	suffix string
+	flow   stream.Flow
+	err    verror.E
 }
 
 // TODO(cnicolaou): implement real, configurable load balancing.
-func (c *client) tryServer(index int, server string, ch chan<- *serverStatus) {
+func (c *client) tryServer(ctx context.T, index int, server string, ch chan<- *serverStatus) {
 	status := &serverStatus{index: index}
-	var err error
-	if status.flow, status.suffix, err = c.connectFlow(server); err != nil {
-		vlog.VI(2).Infof("ipc: couldn't connect to server %v: %v", server, err)
-		status.errConn = verror.NoExistf("ipc: %q: %s", server, err)
+	var err verror.E
+	if status.flow, status.suffix, err = c.connectFlow(ctx, server); err != nil {
+		vlog.VI(2).Infof("ipc: err: %s", err)
+		status.err = err
 		status.flow = nil
 	}
 	ch <- status
@@ -300,22 +330,31 @@
 		servers = []string{name}
 	} else {
 		if resolved, err := c.ns.Resolve(ctx, name); err != nil {
-			return nil, verror.NoExistf("ipc: Resolve(%q) failed: %v", name, err)
+			return nil, verror.Make(verror.NoExist, ctx, name, err)
 		} else {
+			if len(resolved) == 0 {
+				return nil, verror.Make(verror.NoServers, ctx, name)
+			}
 			// An empty set of protocols means all protocols...
 			ordered, err := filterAndOrderServers(resolved, c.preferredProtocols)
-			if len(ordered) == 0 {
-				return nil, verror.NoExistf("ipc: %q: %s", name, err)
+			if err != nil {
+				return nil, verror.Make(verror.NoServers, ctx, name, err)
+			} else if len(ordered) == 0 {
+				// sooo annoying....
+				r := []interface{}{err}
+				r = append(r, name)
+				for _, s := range resolved {
+					r = append(r, s)
+				}
+				return nil, verror.Make(verror.NoServers, ctx, r)
 			}
 			servers = ordered
 		}
 	}
+
 	// servers is now orderd by the priority heurestic implemented in
 	// filterAndOrderServers.
 	attempts := len(servers)
-	if attempts == 0 {
-		return nil, errNoServers
-	}
 
 	// Try to connect to all servers in parallel.  Provide sufficient buffering
 	// for all of the connections to finish instantaneously. This is important
@@ -325,7 +364,7 @@
 	responses := make([]*serverStatus, attempts)
 	ch := make(chan *serverStatus, attempts)
 	for i, server := range servers {
-		go c.tryServer(i, server, ch)
+		go c.tryServer(ctx, i, server, ch)
 	}
 
 	delay := time.Duration(ipc.NoTimeout)
@@ -351,7 +390,7 @@
 			}
 		case <-timeoutChan:
 			vlog.VI(2).Infof("ipc: timeout on connection to server %v ", name)
-			return c.failedTryCall(name, method, servers, responses, ch)
+			return c.failedTryCall(ctx, name, method, servers, responses, ch)
 		}
 
 		// Process new responses, in priority order.
@@ -375,9 +414,10 @@
 			if r.flow.LocalPrincipal() != nil {
 				// Validate caveats on the server's identity for the context associated with this call.
 				var err error
-				if serverB, grantedB, err = c.authorizeServer(r.flow, name, method, serverPattern, opts); err != nil {
-					vlog.VI(2).Infof("ipc: client unwilling to invoke %q.%q on server %v: %v", name, method, r.flow.RemoteBlessings(), err)
-					r.errAccess = verror.NoAccessf("ipc: unwilling to invoke %q.%q on server %v: %v", name, method, r.flow.RemoteBlessings(), err)
+				if serverB, grantedB, err = c.authorizeServer(ctx, r.flow, name, method, serverPattern, opts); err != nil {
+					r.err = verror.Make(errNotTrusted, ctx,
+						name, r.flow.RemoteBlessings(), err)
+					vlog.VI(2).Infof("ipc: err: %s", r.err)
 					r.flow.Close()
 					r.flow = nil
 					continue
@@ -415,7 +455,7 @@
 			return fc, nil
 		}
 		if numResponses == len(responses) {
-			return c.failedTryCall(name, method, servers, responses, ch)
+			return c.failedTryCall(ctx, name, method, servers, responses, ch)
 		}
 	}
 }
@@ -449,30 +489,38 @@
 // failedTryCall performs asynchronous cleanup for tryCall, and returns an
 // appropriate error from the responses we've already received.  All parallel
 // calls in tryCall failed or we timed out if we get here.
-func (c *client) failedTryCall(name, method string, servers []string, responses []*serverStatus, ch chan *serverStatus) (ipc.Call, verror.E) {
+func (c *client) failedTryCall(ctx context.T, name, method string, servers []string, responses []*serverStatus, ch chan *serverStatus) (ipc.Call, verror.E) {
 	go cleanupTryCall(nil, responses, ch)
 	c.ns.FlushCacheEntry(name)
-	// TODO(cnicolaou): introduce a third error code here for mixed
-	// conn/access errors.
-	var errs []verror.E
-	for _, r := range responses {
-		switch {
-		case r != nil && r.errConn != nil:
-			errs = append(errs, r.errConn)
-		case r != nil && r.errAccess != nil:
-			errs = append(errs, r.errAccess)
+	noconn, untrusted := []string{}, []string{}
+	for i, r := range responses {
+		if r != nil && r.err != nil {
+			vlog.VI(2).Infof("Server: %s: %s", servers[i], r.err)
+			switch {
+			case verror.Is(r.err, errNotTrusted.ID) || verror.Is(r.err, errAuthError.ID):
+				untrusted = append(untrusted, r.err.Error())
+			default:
+				noconn = append(noconn, r.err.Error())
+			}
 		}
 	}
-	return nil, verror.NoExistf("ipc: client failed to invoke %q.%q: on %v: %v", name, method, servers, errs)
+	switch {
+	case len(untrusted) > 0 && len(noconn) > 0:
+		return nil, verror.Make(verror.NoServersAndAuth, ctx, append(noconn, untrusted...))
+	case len(noconn) > 0:
+		return nil, verror.Make(verror.NoServers, ctx, noconn)
+	default:
+		return nil, verror.Make(verror.NotTrusted, ctx, untrusted)
+	}
 }
 
 // authorizeServer validates that the server (remote end of flow) has the credentials to serve
 // the RPC name.method for the client (local end of the flow). It returns the blessings at the
 // server that are authorized for this purpose and any blessings that are to be granted to
 // the server (via ipc.Granter implementations in opts.)
-func (c *client) authorizeServer(flow stream.Flow, name, method string, serverPattern security.BlessingPattern, opts []ipc.CallOpt) (serverBlessings []string, grantedBlessings security.Blessings, err error) {
+func (c *client) authorizeServer(ctx context.T, flow stream.Flow, name, method string, serverPattern security.BlessingPattern, opts []ipc.CallOpt) (serverBlessings []string, grantedBlessings security.Blessings, err verror.E) {
 	if flow.RemoteBlessings() == nil {
-		return nil, nil, fmt.Errorf("server has not presented any blessings")
+		return nil, nil, verror.Make(errNoBlessings, ctx)
 	}
 	ctxt := security.NewContext(&security.ContextParams{
 		LocalPrincipal:   flow.LocalPrincipal(),
@@ -486,20 +534,20 @@
 	serverBlessings = flow.RemoteBlessings().ForContext(ctxt)
 	if serverPattern != "" {
 		if !serverPattern.MatchedBy(serverBlessings...) {
-			return nil, nil, fmt.Errorf("server %v does not match the provided pattern %q", serverBlessings, serverPattern)
+			return nil, nil, verror.Make(errAuthNoPatternMatch, ctx, serverBlessings, serverPattern)
 		}
 	} else if enableSecureServerAuth {
 		if err := (defaultAuthorizer{}).Authorize(ctxt); err != nil {
-			return nil, nil, fmt.Errorf("default authorization precludes talking to server %v", serverBlessings)
+			return nil, nil, verror.Make(errDefaultAuthDenied, ctx, serverBlessings)
 		}
 	}
 	for _, o := range opts {
 		switch v := o.(type) {
 		case ipc.Granter:
 			if b, err := v.Grant(flow.RemoteBlessings()); err != nil {
-				return nil, nil, fmt.Errorf("failed to grant blessing to server %v: %v", serverBlessings, err)
+				return nil, nil, verror.Make(errBlessingGrant, ctx, serverBlessings, err)
 			} else if grantedBlessings, err = security.UnionOfBlessings(grantedBlessings, b); err != nil {
-				return nil, nil, fmt.Errorf("failed to add blessing granted to server %v: %v", serverBlessings, err)
+				return nil, nil, verror.Make(errBlessingAdd, ctx, serverBlessings, err)
 			}
 		}
 	}
@@ -556,7 +604,7 @@
 
 func (fc *flowClient) close(verr verror.E) verror.E {
 	if err := fc.flow.Close(); err != nil && verr == nil {
-		verr = verror.Internalf("ipc: flow close failed: %v", err)
+		verr = verror.Make(errSystemNoRetry, fc.ctx, err)
 	}
 	return verr
 }
@@ -577,16 +625,16 @@
 		TraceRequest:     vtrace.Request(fc.ctx),
 	}
 	if err := fc.enc.Encode(req); err != nil {
-		return fc.close(verror.BadProtocolf("ipc: request encoding failed: %v", err))
+		return fc.close(badProtocol(fc.ctx, verror.Make(errRequestEncoding, fc.ctx, req, err)))
 	}
 	for _, d := range fc.discharges {
 		if err := fc.enc.Encode(d); err != nil {
-			return fc.close(verror.BadProtocolf("ipc: failed to encode discharge for %x: %v", d.ID(), err))
+			return fc.close(badProtocol(fc.ctx, verror.Make(errDischargeEncoding, fc.ctx, d.ID(), err)))
 		}
 	}
 	for ix, arg := range args {
 		if err := fc.enc.Encode(arg); err != nil {
-			return fc.close(verror.BadProtocolf("ipc: arg %d encoding failed: %v", ix, err))
+			return fc.close(badProtocol(fc.ctx, verror.Make(errArgEncoding, fc.ctx, ix, err)))
 		}
 	}
 	return nil
@@ -595,15 +643,15 @@
 func (fc *flowClient) Send(item interface{}) error {
 	defer vlog.LogCall()()
 	if fc.sendClosed {
-		return errFlowClosed
+		return verror.Make(verror.Aborted, fc.ctx)
 	}
 
 	// The empty request header indicates what follows is a streaming arg.
 	if err := fc.enc.Encode(ipc.Request{}); err != nil {
-		return fc.close(verror.BadProtocolf("ipc: streaming request header encoding failed: %v", err))
+		return fc.close(badProtocol(fc.ctx, verror.Make(errRequestEncoding, fc.ctx, ipc.Request{}, err)))
 	}
 	if err := fc.enc.Encode(item); err != nil {
-		return fc.close(verror.BadProtocolf("ipc: streaming arg encoding failed: %v", err))
+		return fc.close(badProtocol(fc.ctx, verror.Make(errArgEncoding, fc.ctx, -1, err)))
 	}
 	return nil
 }
@@ -619,7 +667,7 @@
 
 	// Decode the response header and handle errors and EOF.
 	if err := fc.dec.Decode(&fc.response); err != nil {
-		return fc.close(verror.BadProtocolf("ipc: response header decoding failed: %v", err))
+		return fc.close(badProtocol(fc.ctx, verror.Make(errResponseDecoding, fc.ctx, err)))
 	}
 	if fc.response.Error != nil {
 		return fc.response.Error
@@ -632,7 +680,7 @@
 	}
 	// Decode the streaming result.
 	if err := fc.dec.Decode(itemptr); err != nil {
-		return fc.close(verror.BadProtocolf("ipc: streaming result decoding failed: %v", err))
+		return fc.close(badProtocol(fc.ctx, verror.Make(errResponseDecoding, fc.ctx, err)))
 	}
 	return nil
 }
@@ -676,10 +724,15 @@
 	return err
 }
 
+func badProtocol(ctx context.T, err verror.E) verror.E {
+	return verror.Make(verror.BadProtocol, ctx, err)
+}
+
 // finish ensures Finish always returns verror.E.
 func (fc *flowClient) finish(resultptrs ...interface{}) verror.E {
 	if fc.finished {
-		return fc.close(verror.BadProtocolf("ipc: multiple calls to Finish not allowed"))
+		err := verror.Make(errClientFinishAlreadyCalled, fc.ctx)
+		return fc.close(verror.Make(verror.BadState, fc.ctx, err))
 	}
 	fc.finished = true
 	// Call closeSend implicitly, if the user hasn't already called it.  There are
@@ -701,11 +754,11 @@
 	// Decode the response header, if it hasn't already been decoded by Recv.
 	if fc.response.Error == nil && !fc.response.EndStreamResults {
 		if err := fc.dec.Decode(&fc.response); err != nil {
-			return fc.close(verror.BadProtocolf("ipc: response header decoding failed: %v", err))
+			return fc.close(badProtocol(fc.ctx, verror.Make(errResponseDecoding, fc.ctx, err)))
 		}
 		// The response header must indicate the streaming results have ended.
 		if fc.response.Error == nil && !fc.response.EndStreamResults {
-			return fc.close(errRemainingStreamResults)
+			return fc.close(badProtocol(fc.ctx, verror.Make(errRemainingStreamResults, fc.ctx)))
 		}
 	}
 
@@ -713,7 +766,9 @@
 	vtrace.MergeResponse(fc.ctx, &fc.response.TraceResponse)
 
 	if fc.response.Error != nil {
-		if verror.Is(fc.response.Error, verror.NoAccess) && fc.dc != nil {
+		// TODO(cnicolaou): remove verror.NoAccess with verror version
+		// when ipc.Server is converted.
+		if verror.Is(fc.response.Error, old_verror.NoAccess) && fc.dc != nil {
 			// In case the error was caused by a bad discharge, we do not want to get stuck
 			// with retrying again and again with this discharge. As there is no direct way
 			// to detect it, we conservatively flush all discharges we used from the cache.
@@ -721,14 +776,16 @@
 			vlog.VI(3).Infof("Discarging %d discharges as RPC failed with %v", len(fc.discharges), fc.response.Error)
 			fc.dc.Invalidate(fc.discharges...)
 		}
-		return fc.close(verror.ConvertWithDefault(verror.Internal, fc.response.Error))
+		// TODO(cnicolaou): we turn this into a non-retryable error until
+		// we have verror on the server side.
+		return fc.close(verror.Convert(verror.Internal, fc.ctx, fc.response.Error))
 	}
 	if got, want := fc.response.NumPosResults, uint64(len(resultptrs)); got != want {
-		return fc.close(verror.BadProtocolf("ipc: server sent %d results, client expected %d (%#v)", got, want, resultptrs))
+		return fc.close(badProtocol(fc.ctx, verror.Make(errMismatchedResults, fc.ctx, got, want)))
 	}
 	for ix, r := range resultptrs {
 		if err := fc.dec.Decode(r); err != nil {
-			return fc.close(verror.BadProtocolf("ipc: result #%d decoding failed: %v", ix, err))
+			return fc.close(badProtocol(fc.ctx, verror.Make(errResultDecoding, fc.ctx, ix, err)))
 		}
 	}
 	return fc.close(nil)
diff --git a/runtimes/google/ipc/full_test.go b/runtimes/google/ipc/full_test.go
index 045bd84..d58264a 100644
--- a/runtimes/google/ipc/full_test.go
+++ b/runtimes/google/ipc/full_test.go
@@ -5,6 +5,7 @@
 	"fmt"
 	"io"
 	"net"
+	"os"
 	"path/filepath"
 	"reflect"
 	"runtime"
@@ -19,7 +20,7 @@
 	"veyron.io/veyron/veyron2/options"
 	"veyron.io/veyron/veyron2/security"
 	"veyron.io/veyron/veyron2/vdl/vdlutil"
-	"veyron.io/veyron/veyron2/verror"
+	verror "veyron.io/veyron/veyron2/verror2"
 	"veyron.io/veyron/veyron2/vlog"
 	"veyron.io/veyron/veyron2/vom"
 
@@ -41,7 +42,7 @@
 }
 
 var (
-	errMethod  = verror.Abortedf("server returned an error")
+	errMethod  = verror.Make(verror.Aborted, nil)
 	clock      = new(fakeClock)
 	listenSpec = ipc.ListenSpec{Protocol: "tcp", Address: "127.0.0.1:0"}
 )
@@ -290,11 +291,17 @@
 	return
 }
 
-func matchesErrorPattern(err error, pattern string) bool {
-	if (len(pattern) == 0) != (err == nil) {
-		return false
+func matchesErrorPattern(err error, id verror.IDAction, pattern string) bool {
+	if len(pattern) > 0 && err != nil {
+		if strings.Index(err.Error(), pattern) < 0 {
+			fmt.Fprintf(os.Stderr, "got error msg: %q, expected: %q\n", err, pattern)
+		}
 	}
-	return err == nil || strings.Index(err.Error(), pattern) >= 0
+	// TODO(cnicolaou): Move this special case into verror.Is.
+	if reflect.DeepEqual(id, verror.IDAction{}) {
+		return err == nil
+	}
+	return verror.Is(err, id.ID)
 }
 
 func TestMultipleCallsToServeAndName(t *testing.T) {
@@ -391,27 +398,28 @@
 		tests = []struct {
 			server  security.Blessings       // blessings presented by the server to the client.
 			pattern security.BlessingPattern // pattern on the server identity expected by the client.
+			errID   verror.IDAction
 			err     string
 		}{
 			// Client accepts talking to the server only if the server's blessings match the provided pattern
-			{bServer, security.AllPrincipals, ""},
-			{bServer, "root/server", ""},
-			{bServer, "root/otherserver", nameErr},
-			{bServer, "otherroot/server", nameErr},
+			{bServer, security.AllPrincipals, verror.Success, ""},
+			{bServer, "root/server", verror.Success, ""},
+			{bServer, "root/otherserver", verror.NotTrusted, nameErr},
+			{bServer, "otherroot/server", verror.NotTrusted, nameErr},
 
 			// and, if the server's blessing has third-party caveats then the server provides
 			// appropriate discharges.
-			{bServerTPValid, security.AllPrincipals, ""},
-			{bServerTPValid, "root/serverWithTPCaveats", ""},
-			{bServerTPValid, "root/otherserver", nameErr},
-			{bServerTPValid, "otherroot/server", nameErr},
+			{bServerTPValid, security.AllPrincipals, verror.Success, ""},
+			{bServerTPValid, "root/serverWithTPCaveats", verror.Success, ""},
+			{bServerTPValid, "root/otherserver", verror.NotTrusted, nameErr},
+			{bServerTPValid, "otherroot/server", verror.NotTrusted, nameErr},
 
 			// Client does not talk to a server that presents expired blessings.
-			{bServerExpired, security.AllPrincipals, vcErr},
+			{bServerExpired, security.AllPrincipals, verror.NotTrusted, vcErr},
 
 			// Client does not talk to a server that fails to provide discharges for
 			// third-party caveats on the blessings presented by it.
-			{bServerTPExpired, security.AllPrincipals, vcErr},
+			{bServerTPExpired, security.AllPrincipals, verror.NotTrusted, vcErr},
 		}
 	)
 
@@ -433,7 +441,7 @@
 	// Set a blessing that the client is willing to share with servers with blessings
 	// from pprovider.
 	pclient.BlessingStore().Set(bless(pprovider, pclient, "client"), "root/...")
-	for _, test := range tests {
+	for i, test := range tests {
 		name := fmt.Sprintf("(%q@%q)", test.pattern, test.server)
 		if err := pserver.BlessingStore().SetDefault(test.server); err != nil {
 			t.Fatalf("SetDefault failed on server's BlessingStore: %v", err)
@@ -447,8 +455,9 @@
 			t.Errorf("%s: failed to create client: %v", name, err)
 			continue
 		}
-		if call, err := client.StartCall(testContext(), fmt.Sprintf("[%s]%s/suffix", test.pattern, serverName), "Method", nil); !matchesErrorPattern(err, test.err) {
-			t.Errorf(`%s: client.StartCall: got error "%v", want to match "%v"`, name, err, test.err)
+		call, err := client.StartCall(testContext(), fmt.Sprintf("[%s]%s/suffix", test.pattern, serverName), "Method", nil)
+		if !matchesErrorPattern(err, test.errID, test.err) {
+			t.Errorf(`%d: %s: client.StartCall: got error "%v", want to match "%v"`, i, name, err, test.err)
 		} else if call != nil {
 			blessings, proof := call.RemoteBlessings()
 			if proof == nil {
@@ -458,6 +467,7 @@
 				t.Errorf("%s: %q.MatchedBy(%v) failed", name, test.pattern, blessings)
 			}
 		}
+		vlog.Infof("\nC")
 		client.Close()
 
 	}
@@ -600,9 +610,8 @@
 		t.Fatalf(`call.Finish got error "%v"`, err)
 	}
 	// Calling Finish a second time should result in a useful error.
-	err = call.Finish(&results)
-	if got, want := err, verror.BadProtocolf("ipc: multiple calls to Finish not allowed"); got != want {
-		t.Fatalf(`call.Finish got error "%v", want "%v"`, got, want)
+	if err = call.Finish(&results); !matchesErrorPattern(err, verror.BadState, "xxx") {
+		t.Fatalf(`got "%v", want "%v"`, err, verror.BadState)
 	}
 }
 
@@ -625,23 +634,24 @@
 
 	tests := []struct {
 		granter                       ipc.Granter
+		startErrID, finishErrID       verror.IDAction
 		blessing, starterr, finisherr string
 	}{
 		{blessing: "<nil>"},
 		{granter: granter{b: bless(pclient, pserver, "blessed")}, blessing: "client/blessed"},
-		{granter: granter{err: errors.New("hell no")}, starterr: "hell no"},
-		{granter: granter{b: pclient.BlessingStore().Default()}, finisherr: "blessing granted not bound to this server"},
+		{granter: granter{err: errors.New("hell no")}, startErrID: verror.NotTrusted, starterr: "hell no"},
+		{granter: granter{b: pclient.BlessingStore().Default()}, finishErrID: verror.NoAccess, finisherr: "blessing granted not bound to this server"},
 	}
-	for _, test := range tests {
+	for i, test := range tests {
 		call, err := b.client.StartCall(testContext(), "mountpoint/server/suffix", "EchoGrantedBlessings", []interface{}{"argument"}, test.granter)
-		if !matchesErrorPattern(err, test.starterr) {
-			t.Errorf("%+v: StartCall returned error %v", test, err)
+		if !matchesErrorPattern(err, test.startErrID, test.starterr) {
+			t.Errorf("%d: %+v: StartCall returned error %v", i, test, err)
 		}
 		if err != nil {
 			continue
 		}
 		var result, blessing string
-		if err = call.Finish(&result, &blessing); !matchesErrorPattern(err, test.finisherr) {
+		if err = call.Finish(&result, &blessing); !matchesErrorPattern(err, test.finishErrID, test.finisherr) {
 			t.Errorf("%+v: Finish returned error %v", test, err)
 		}
 		if err != nil {
@@ -911,8 +921,8 @@
 			t.Errorf(`%s call.Finish got error: "%v", wanted the RPC to succeed`, name, err)
 		} else if err == nil && !test.authorized {
 			t.Errorf("%s call.Finish succeeded, expected authorization failure", name)
-		} else if !test.authorized && !verror.Is(err, verror.NoAccess) {
-			t.Errorf("%s. call.Finish returned error %v(%v), wanted %v", name, verror.Convert(err).ErrorID(), err, verror.NoAccess)
+		} else if !test.authorized && !verror.Is(err, verror.NoAccess.ID) {
+			t.Errorf("%s. call.Finish returned error %v(%v), wanted %v", name, verror.Convert(verror.NoAccess, nil, err).ErrorID(), err, verror.NoAccess)
 		}
 	}
 }
@@ -940,17 +950,17 @@
 	if b.client, err = InternalNewClient(b.sm, b.ns, vc.LocalPrincipal{pclient}, dc); err != nil {
 		t.Fatalf("InternalNewClient failed: %v", err)
 	}
-	call := func() error {
+	call := func() verror.E {
 		call, err := b.client.StartCall(testContext(), "mountpoint/server/aclAuth", "Echo", []interface{}{"batman"})
 		if err != nil {
-			return fmt.Errorf("client.StartCall failed: %v", err)
+			return err.(verror.E) //fmt.Errorf("client.StartCall failed: %v", err)
 		}
 		var got string
 		if err := call.Finish(&got); err != nil {
-			return fmt.Errorf("client.Finish failed: %v", err)
+			return err.(verror.E) //fmt.Errorf("client.Finish failed: %v", err)
 		}
 		if want := `method:"Echo",suffix:"aclAuth",arg:"batman"`; got != want {
-			return fmt.Errorf("Got [%v] want [%v]", got, want)
+			return verror.Convert(verror.BadArg, nil, fmt.Errorf("Got [%v] want [%v]", got, want))
 		}
 		return nil
 	}
@@ -961,7 +971,7 @@
 	}
 	// Advance virtual clock, which will invalidate the discharge
 	clock.Advance(1)
-	if err, want := call(), "not authorized"; !matchesErrorPattern(err, want) {
+	if err, want := call(), "not authorized"; !matchesErrorPattern(err, verror.NoAccess, want) {
 		t.Errorf("Got error [%v] wanted to match pattern %q", err, want)
 	}
 	// But retrying will succeed since the discharge should be purged from cache and refreshed
@@ -1245,8 +1255,8 @@
 	if call != nil {
 		t.Errorf("Expected nil interface got: %#v", call)
 	}
-	if !verror.Is(err, verror.BadArg) {
-		t.Errorf("Expected a BadArg error, got: %s", err.Error())
+	if !verror.Is(err, verror.BadArg.ID) {
+		t.Errorf("Expected an BadArg error, got: %s", err.Error())
 	}
 }
 
diff --git a/runtimes/google/ipc/server.go b/runtimes/google/ipc/server.go
index abd5986..009e56c 100644
--- a/runtimes/google/ipc/server.go
+++ b/runtimes/google/ipc/server.go
@@ -873,7 +873,7 @@
 	// 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 blessings != nil && !reflect.DeepEqual(blessings.PublicKey(), fs.flow.LocalPrincipal().PublicKey()) {
-		return verror.BadProtocolf("ipc: blessing granted not bound to this server(%v vs %v)", blessings.PublicKey(), fs.flow.LocalPrincipal().PublicKey())
+		return verror.NoAccessf("ipc: blessing granted not bound to this server(%v vs %v)", blessings.PublicKey(), fs.flow.LocalPrincipal().PublicKey())
 	}
 	// Receive third party caveat discharges the client sent
 	for i := uint64(0); i < req.NumDischarges; i++ {
diff --git a/runtimes/google/ipc/testutil_test.go b/runtimes/google/ipc/testutil_test.go
index 4144d97..02f07c7 100644
--- a/runtimes/google/ipc/testutil_test.go
+++ b/runtimes/google/ipc/testutil_test.go
@@ -4,10 +4,10 @@
 	"reflect"
 	"testing"
 
-	"veyron.io/veyron/veyron/lib/testutil"
-
 	"veyron.io/veyron/veyron2/security"
-	"veyron.io/veyron/veyron2/verror"
+	"veyron.io/veyron/veyron2/verror2"
+
+	"veyron.io/veyron/veyron/lib/testutil"
 )
 
 func init() { testutil.Init() }
@@ -22,7 +22,7 @@
 			// reasons for this check and conditions for when it
 			// can be removed can be seen in the comments for
 			// result2vom.
-			var verr verror.E
+			var verr verror2.E
 			typ = reflect.ValueOf(&verr).Elem().Type()
 		}
 		outs[ix] = reflect.New(typ).Interface()
@@ -34,9 +34,23 @@
 	for ix, res := range gotptrs {
 		got := reflect.ValueOf(res).Elem().Interface()
 		want := want[ix]
-		if !reflect.DeepEqual(got, want) {
-			t.Errorf("%s result %d got %v, want %v", name, ix, got, want)
+		switch g := got.(type) {
+		case verror2.Standard:
+			w, ok := want.(verror2.Standard)
+			// don't use reflect deep equal on verror's since they contain
+			// a list of stack PCs which will be different.
+			if !ok {
+				t.Errorf("%s result %d got type %T, want %T", name, ix, g, w)
+			}
+			if !verror2.Is(g, w.IDAction.ID) {
+				t.Errorf("%s result %d got %v, want %v", name, ix, g, w)
+			}
+		default:
+			if !reflect.DeepEqual(got, want) {
+				t.Errorf("%s result %d got %v, want %v", name, ix, got, want)
+			}
 		}
+
 	}
 }
 
diff --git a/runtimes/google/testing/mocks/runtime/panic_runtime.go b/runtimes/google/testing/mocks/runtime/panic_runtime.go
index d70852b..1eb085b 100644
--- a/runtimes/google/testing/mocks/runtime/panic_runtime.go
+++ b/runtimes/google/testing/mocks/runtime/panic_runtime.go
@@ -8,6 +8,7 @@
 	"veyron.io/veyron/veyron2/ipc/stream"
 	"veyron.io/veyron/veyron2/naming"
 	"veyron.io/veyron/veyron2/security"
+	"veyron.io/veyron/veyron2/uniqueid"
 	"veyron.io/veyron/veyron2/vlog"
 	"veyron.io/veyron/veyron2/vtrace"
 )
@@ -17,20 +18,23 @@
 // implementation but you don't want it to be used.
 type PanicRuntime struct {
 	unique int // Make non-empty to ensure pointer instances are unique.
+
 }
 
 const badRuntime = "The runtime implmentation should not call methods on runtime intances."
 
-func (*PanicRuntime) Profile() veyron2.Profile                               { panic(badRuntime) }
-func (*PanicRuntime) AppCycle() veyron2.AppCycle                             { panic(badRuntime) }
-func (*PanicRuntime) Publisher() *config.Publisher                           { panic(badRuntime) }
-func (*PanicRuntime) Principal() security.Principal                          { panic(badRuntime) }
-func (*PanicRuntime) NewClient(opts ...ipc.ClientOpt) (ipc.Client, error)    { panic(badRuntime) }
-func (*PanicRuntime) NewServer(opts ...ipc.ServerOpt) (ipc.Server, error)    { panic(badRuntime) }
-func (*PanicRuntime) Client() ipc.Client                                     { panic(badRuntime) }
-func (*PanicRuntime) NewContext() context.T                                  { panic(badRuntime) }
-func (*PanicRuntime) WithNewSpan(context.T, string) (context.T, vtrace.Span) { panic(badRuntime) }
-func (*PanicRuntime) SpanFromContext(context.T) vtrace.Span                  { panic(badRuntime) }
+func (*PanicRuntime) Profile() veyron2.Profile                            { panic(badRuntime) }
+func (*PanicRuntime) AppCycle() veyron2.AppCycle                          { panic(badRuntime) }
+func (*PanicRuntime) Publisher() *config.Publisher                        { panic(badRuntime) }
+func (*PanicRuntime) Principal() security.Principal                       { panic(badRuntime) }
+func (*PanicRuntime) NewClient(opts ...ipc.ClientOpt) (ipc.Client, error) { panic(badRuntime) }
+func (*PanicRuntime) NewServer(opts ...ipc.ServerOpt) (ipc.Server, error) { panic(badRuntime) }
+func (*PanicRuntime) Client() ipc.Client                                  { panic(badRuntime) }
+func (*PanicRuntime) NewContext() context.T                               { panic(badRuntime) }
+
+func (PanicRuntime) WithNewSpan(c context.T, m string) (context.T, vtrace.Span) { return c, &span{m} }
+
+func (*PanicRuntime) SpanFromContext(context.T) vtrace.Span { return &span{} }
 func (*PanicRuntime) NewStreamManager(opts ...stream.ManagerOpt) (stream.Manager, error) {
 	panic(badRuntime)
 }
@@ -45,3 +49,12 @@
 }
 func (*PanicRuntime) VtraceStore() vtrace.Store { panic(badRuntime) }
 func (*PanicRuntime) Cleanup()                  { panic(badRuntime) }
+
+type span struct{ m string }
+
+func (s *span) Name() string        { return s.m + ".panic" }
+func (*span) ID() uniqueid.ID       { return uniqueid.ID{} }
+func (s *span) Parent() uniqueid.ID { return s.ID() }
+func (*span) Annotate(string)       {}
+func (*span) Finish()               {}
+func (*span) Trace() vtrace.Trace   { return nil }