Skip to content

Commit

Permalink
topo: fix obscure topo error messages
Browse files Browse the repository at this point in the history
Topo error messages were obscure because they were not
reporting the node they were failing. The fix is extensive
because many places check for error strings to identify
specific types of error.

To address this, the topo errors have been converted to
also contain a code, and convenience functions are used
to check for error types. This allows the error string
to contain more information.

Signed-off-by: Sugu Sougoumarane <ssougou@gmail.com>
  • Loading branch information
sougou committed Jun 9, 2018
1 parent 5f72be2 commit 411fba0
Show file tree
Hide file tree
Showing 80 changed files with 395 additions and 348 deletions.
10 changes: 5 additions & 5 deletions go/vt/discovery/tablet_stats_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -484,22 +484,22 @@ func (tc *TabletStatsCache) Unsubscribe(i int) error {
func (tc *TabletStatsCache) GetAggregateStats(target *querypb.Target) (*querypb.AggregateStats, error) {
e := tc.getEntry(target.Keyspace, target.Shard, target.TabletType)
if e == nil {
return nil, topo.ErrNoNode
return nil, topo.NewError(topo.NoNode, target.Keyspace+"/"+target.Shard+"@"+target.TabletType.String())
}

e.mu.RLock()
defer e.mu.RUnlock()
if target.TabletType == topodatapb.TabletType_MASTER {
if len(e.aggregates) == 0 {
return nil, topo.ErrNoNode
return nil, topo.NewError(topo.NoNode, target.Keyspace+"/"+target.Shard+"@"+target.TabletType.String())
}
for _, agg := range e.aggregates {
return agg, nil
}
}
agg, ok := e.aggregates[target.Cell]
if !ok {
return nil, topo.ErrNoNode
return nil, topo.NewError(topo.NoNode, target.Keyspace+"/"+target.Shard+"@"+target.TabletType.String())
}
return agg, nil
}
Expand All @@ -508,15 +508,15 @@ func (tc *TabletStatsCache) GetAggregateStats(target *querypb.Target) (*querypb.
func (tc *TabletStatsCache) GetMasterCell(keyspace, shard string) (cell string, err error) {
e := tc.getEntry(keyspace, shard, topodatapb.TabletType_MASTER)
if e == nil {
return "", topo.ErrNoNode
return "", topo.NewError(topo.NoNode, keyspace+"/"+shard+"@"+topodatapb.TabletType_MASTER.String())
}

e.mu.RLock()
defer e.mu.RUnlock()
for cell := range e.aggregates {
return cell, nil
}
return "", topo.ErrNoNode
return "", topo.NewError(topo.NoNode, keyspace+"/"+shard+"@"+topodatapb.TabletType_MASTER.String())
}

// Compile-time interface check.
Expand Down
6 changes: 3 additions & 3 deletions go/vt/discovery/topology_watcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,10 +79,10 @@ func NewCellTabletsWatcher(topoServer *topo.Server, tr TabletRecorder, cell stri
func NewShardReplicationWatcher(topoServer *topo.Server, tr TabletRecorder, cell, keyspace, shard string, refreshInterval time.Duration, topoReadConcurrency int) *TopologyWatcher {
return NewTopologyWatcher(topoServer, tr, cell, refreshInterval, true /* refreshKnownTablets */, topoReadConcurrency, func(tw *TopologyWatcher) ([]*topodatapb.TabletAlias, error) {
sri, err := tw.topoServer.GetShardReplication(tw.ctx, tw.cell, keyspace, shard)
switch err {
case nil:
switch {
case err == nil:
// we handle this case after this switch block
case topo.ErrNoNode:
case topo.IsErrType(err, topo.NoNode):
// this is not an error
return nil, nil
default:
Expand Down
2 changes: 1 addition & 1 deletion go/vt/srvtopo/discover.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ func FindAllTargets(ctx context.Context, ts Server, cell string, tabletTypes []t
// Get SrvKeyspace for cell/keyspace.
ks, err := ts.GetSrvKeyspace(ctx, cell, keyspace)
if err != nil {
if err == topo.ErrNoNode {
if topo.IsErrType(err, topo.NoNode) {
// Possibly a race condition, or leftover
// crud in the topology service. Just log it.
log.Warningf("GetSrvKeyspace(%v, %v) returned ErrNoNode, skipping that SrvKeyspace", cell, keyspace)
Expand Down
6 changes: 3 additions & 3 deletions go/vt/srvtopo/resilient_server.go
Original file line number Diff line number Diff line change
Expand Up @@ -439,7 +439,7 @@ func (server *ResilientServer) watchSrvKeyspace(callerCtx context.Context, entry
entry.lastErrorTime = time.Now()

// if the node disappears, delete the cached value
if current.Err == topo.ErrNoNode {
if topo.IsErrType(current.Err, topo.NoNode) {
entry.value = nil
}

Expand Down Expand Up @@ -483,7 +483,7 @@ func (server *ResilientServer) watchSrvKeyspace(callerCtx context.Context, entry
log.Errorf("%v", err)
server.counts.Add(errorCategory, 1)
entry.mutex.Lock()
if c.Err == topo.ErrNoNode {
if topo.IsErrType(c.Err, topo.NoNode) {
entry.value = nil
}
entry.watchState = watchStateIdle
Expand Down Expand Up @@ -529,7 +529,7 @@ func (server *ResilientServer) WatchSrvVSchema(ctx context.Context, cell string,
}
if current.Err != nil {
// Don't log if there is no VSchema to start with.
if current.Err != topo.ErrNoNode {
if !topo.IsErrType(current.Err, topo.NoNode) {
log.Warningf("Error watching vschema for cell %s (will wait 5s before retrying): %v", cell, current.Err)
}
} else {
Expand Down
14 changes: 7 additions & 7 deletions go/vt/srvtopo/resilient_server_flaky_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ func TestGetSrvKeyspace(t *testing.T) {

// Ask for a not-yet-created keyspace
_, err := rs.GetSrvKeyspace(context.Background(), "test_cell", "test_ks")
if err != topo.ErrNoNode {
if !topo.IsErrType(err, topo.NoNode) {
t.Fatalf("GetSrvKeyspace(not created) got unexpected error: %v", err)
}

Expand Down Expand Up @@ -106,7 +106,7 @@ func TestGetSrvKeyspace(t *testing.T) {
expiry = time.Now().Add(5 * time.Second)
for {
got, err = rs.GetSrvKeyspace(context.Background(), "test_cell", "test_ks")
if err == topo.ErrNoNode {
if topo.IsErrType(err, topo.NoNode) {
break
}
if time.Now().After(expiry) {
Expand Down Expand Up @@ -377,10 +377,10 @@ func TestGetSrvKeyspaceCreated(t *testing.T) {
expiry := time.Now().Add(5 * time.Second)
for {
got, err := rs.GetSrvKeyspace(context.Background(), "test_cell", "test_ks")
switch err {
case topo.ErrNoNode:
switch {
case topo.IsErrType(err, topo.NoNode):
// keep trying
case nil:
case err == nil:
// we got a value, see if it's good
if proto.Equal(want, got) {
return
Expand Down Expand Up @@ -419,7 +419,7 @@ func TestWatchSrvVSchema(t *testing.T) {

// WatchSrvVSchema won't return until it gets the initial value,
// which is not there, so we should get watchErr=topo.ErrNoNode.
if _, err := get(); err != topo.ErrNoNode {
if _, err := get(); !topo.IsErrType(err, topo.NoNode) {
t.Fatalf("WatchSrvVSchema didn't return topo.ErrNoNode at first, but got: %v", err)
}

Expand Down Expand Up @@ -469,7 +469,7 @@ func TestWatchSrvVSchema(t *testing.T) {
}
start = time.Now()
for {
if _, err := get(); err == topo.ErrNoNode {
if _, err := get(); topo.IsErrType(err, topo.NoNode) {
break
}
if time.Since(start) > 5*time.Second {
Expand Down
16 changes: 8 additions & 8 deletions go/vt/topo/cell_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,10 +45,10 @@ func pathForCellInfo(cell string) string {
// sorted by name.
func (ts *Server) GetCellInfoNames(ctx context.Context) ([]string, error) {
entries, err := ts.globalCell.ListDir(ctx, CellsPath, false /*full*/)
switch err {
case ErrNoNode:
switch {
case IsErrType(err, NoNode):
return nil, nil
case nil:
case err == nil:
return DirEntriesToStringArray(entries), nil
default:
return nil, err
Expand Down Expand Up @@ -102,20 +102,20 @@ func (ts *Server) UpdateCellInfoFields(ctx context.Context, cell string, update

// Read the file, unpack the contents.
contents, version, err := ts.globalCell.Get(ctx, filePath)
switch err {
case nil:
switch {
case err == nil:
if err := proto.Unmarshal(contents, ci); err != nil {
return err
}
case ErrNoNode:
case IsErrType(err, NoNode):
// Nothing to do.
default:
return err
}

// Call update method.
if err = update(ci); err != nil {
if err == ErrNoUpdateNeeded {
if IsErrType(err, NoUpdateNeeded) {
return nil
}
return err
Expand All @@ -126,7 +126,7 @@ func (ts *Server) UpdateCellInfoFields(ctx context.Context, cell string, update
if err != nil {
return err
}
if _, err = ts.globalCell.Update(ctx, filePath, contents, version); err != ErrBadVersion {
if _, err = ts.globalCell.Update(ctx, filePath, contents, version); !IsErrType(err, BadVersion) {
// This includes the 'err=nil' case.
return err
}
Expand Down
2 changes: 1 addition & 1 deletion go/vt/topo/consultopo/directory.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ func (s *Server) ListDir(ctx context.Context, dirPath string, full bool) ([]topo
if len(keys) == 0 {
// No key starts with this prefix, means the directory
// doesn't exist.
return nil, topo.ErrNoNode
return nil, topo.NewError(topo.NoNode, nodePath)
}

prefixLen := len(nodePath)
Expand Down
2 changes: 1 addition & 1 deletion go/vt/topo/consultopo/election.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ func (mp *consulMasterParticipation) WaitForMastership() (context.Context, error
// If Stop was already called, mp.done is closed, so we are interrupted.
select {
case <-mp.done:
return nil, topo.ErrInterrupted
return nil, topo.NewError(topo.Interrupted, "mastership")
default:
}

Expand Down
6 changes: 3 additions & 3 deletions go/vt/topo/consultopo/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,12 +37,12 @@ var (

// convertError converts a context error into a topo error. All errors
// are either application-level errors, or context errors.
func convertError(err error) error {
func convertError(err error, nodePath string) error {
switch err {
case context.Canceled:
return topo.ErrInterrupted
return topo.NewError(topo.Interrupted, nodePath)
case context.DeadlineExceeded:
return topo.ErrTimeout
return topo.NewError(topo.Timeout, nodePath)
}
return err
}
10 changes: 5 additions & 5 deletions go/vt/topo/consultopo/file.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ func (s *Server) Create(ctx context.Context, filePath string, contents []byte) (
}
if !ok {
// Transaction was rolled back, means the node exists.
return nil, topo.ErrNodeExists
return nil, topo.NewError(topo.NodeExists, nodePath)
}
return ConsulVersion(resp.Results[0].ModifyIndex), nil
}
Expand Down Expand Up @@ -77,7 +77,7 @@ func (s *Server) Update(ctx context.Context, filePath string, contents []byte, v
if !ok {
// Transaction was rolled back, means the node has a
// bad version.
return nil, topo.ErrBadVersion
return nil, topo.NewError(topo.BadVersion, nodePath)
}
return ConsulVersion(resp.Results[0].ModifyIndex), nil
}
Expand All @@ -91,7 +91,7 @@ func (s *Server) Get(ctx context.Context, filePath string) ([]byte, topo.Version
return nil, nil, err
}
if pair == nil {
return nil, nil, topo.ErrNoNode
return nil, nil, topo.NewError(topo.NoNode, nodePath)
}

return pair.Value, ConsulVersion(pair.ModifyIndex), nil
Expand Down Expand Up @@ -134,10 +134,10 @@ func (s *Server) Delete(ctx context.Context, filePath string, version topo.Versi
switch resp.Errors[0].OpIndex {
case 0:
// Get failed (operation 0), the node didn't exist.
return topo.ErrNoNode
return topo.NewError(topo.NoNode, nodePath)
case 1:
// DeleteCAS failed (operation 1), means bad version.
return topo.ErrBadVersion
return topo.NewError(topo.BadVersion, nodePath)
default:
// very unexpected.
return ErrBadResponse
Expand Down
4 changes: 2 additions & 2 deletions go/vt/topo/consultopo/lock.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ func (s *Server) Lock(ctx context.Context, dirPath, contents string) (topo.LockD
// easiest way to do this is to return convertError(err).
// It may lose some of the context, if this is an issue,
// maybe logging the error would work here.
return nil, convertError(err)
return nil, convertError(err, dirPath)
}

lockPath := path.Join(s.root, dirPath, locksFilename)
Expand All @@ -66,7 +66,7 @@ func (s *Server) Lock(ctx context.Context, dirPath, contents string) (topo.LockD
s.mu.Unlock()
select {
case <-ctx.Done():
return nil, convertError(ctx.Err())
return nil, convertError(ctx.Err(), dirPath)
case <-li.done:
}

Expand Down
6 changes: 3 additions & 3 deletions go/vt/topo/consultopo/watch.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func (s *Server) Watch(ctx context.Context, filePath string) (*topo.WatchData, <
}
if pair == nil {
// Node doesn't exist.
return &topo.WatchData{Err: topo.ErrNoNode}, nil, nil
return &topo.WatchData{Err: topo.NewError(topo.NoNode, nodePath)}, nil, nil
}

// Initial value to return.
Expand Down Expand Up @@ -80,7 +80,7 @@ func (s *Server) Watch(ctx context.Context, filePath string) (*topo.WatchData, <
// If the node disappeared, pair is nil.
if pair == nil {
notifications <- &topo.WatchData{
Err: topo.ErrNoNode,
Err: topo.NewError(topo.NoNode, nodePath),
}
return
}
Expand All @@ -97,7 +97,7 @@ func (s *Server) Watch(ctx context.Context, filePath string) (*topo.WatchData, <
select {
case <-watchCtx.Done():
notifications <- &topo.WatchData{
Err: convertError(watchCtx.Err()),
Err: convertError(watchCtx.Err(), nodePath),
}
return
default:
Expand Down
82 changes: 82 additions & 0 deletions go/vt/topo/errors.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/*
Copyright 2018 The Vitess Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/

package topo

import "fmt"

// ErrorCode is the error code for topo errors.
type ErrorCode int

// The following is the list of error codes.
const (
NodeExists = ErrorCode(iota)
NoNode
NodeNotEmpty
Timeout
Interrupted
BadVersion
PartialResult
NoUpdateNeeded
)

// Error represents a topo error.
type Error struct {
code ErrorCode
message string
}

// NewError creates a new topo error.
func NewError(code ErrorCode, node string) error {
var message string
switch code {
case NodeExists:
message = fmt.Sprintf("node already exists: %s", node)
case NoNode:
message = fmt.Sprintf("node doesn't exist: %s", node)
case NodeNotEmpty:
message = fmt.Sprintf("node not empty: %s", node)
case Timeout:
message = fmt.Sprintf("deadline exceeded: %s", node)
case Interrupted:
message = fmt.Sprintf("interrupted: %s", node)
case BadVersion:
message = fmt.Sprintf("bad node version: %s", node)
case PartialResult:
message = fmt.Sprintf("partial result: %s", node)
case NoUpdateNeeded:
message = fmt.Sprintf("no update needed: %s", node)
default:
message = fmt.Sprintf("unknonw code: %s", node)
}
return Error{
code: code,
message: message,
}
}

// Error satisfies error.
func (e Error) Error() string {
return e.message
}

// IsErrType returns true if the error has the specified ErrorCode.
func IsErrType(err error, code ErrorCode) bool {
if e, ok := err.(Error); ok {
return e.code == code
}
return false
}
Loading

0 comments on commit 411fba0

Please sign in to comment.