github.com/m3db/m3@v1.5.0/src/dbnode/client/write_state.go (about)

     1  // Copyright (c) 2016 Uber Technologies, Inc.
     2  //
     3  // Permission is hereby granted, free of charge, to any person obtaining a copy
     4  // of this software and associated documentation files (the "Software"), to deal
     5  // in the Software without restriction, including without limitation the rights
     6  // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
     7  // copies of the Software, and to permit persons to whom the Software is
     8  // furnished to do so, subject to the following conditions:
     9  //
    10  // The above copyright notice and this permission notice shall be included in
    11  // all copies or substantial portions of the Software.
    12  //
    13  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
    14  // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
    15  // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
    16  // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
    17  // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
    18  // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
    19  // THE SOFTWARE.
    20  
    21  package client
    22  
    23  import (
    24  	"fmt"
    25  	"sync"
    26  
    27  	"github.com/m3db/m3/src/cluster/shard"
    28  	"github.com/m3db/m3/src/dbnode/topology"
    29  	"github.com/m3db/m3/src/x/checked"
    30  	xerrors "github.com/m3db/m3/src/x/errors"
    31  	"github.com/m3db/m3/src/x/ident"
    32  	"github.com/m3db/m3/src/x/pool"
    33  	"github.com/m3db/m3/src/x/serialize"
    34  )
    35  
    36  // writeOp represents a generic write operation
    37  type writeOp interface {
    38  	op
    39  
    40  	ShardID() uint32
    41  
    42  	SetCompletionFn(fn completionFn)
    43  
    44  	Close()
    45  }
    46  
    47  type writeState struct {
    48  	sync.Cond
    49  	sync.Mutex
    50  	refCounter
    51  
    52  	consistencyLevel                     topology.ConsistencyLevel
    53  	shardsLeavingCountTowardsConsistency bool
    54  	topoMap                              topology.Map
    55  	op                                   writeOp
    56  	nsID                                 ident.ID
    57  	tsID                                 ident.ID
    58  	tagEncoder                           serialize.TagEncoder
    59  	annotation                           checked.Bytes
    60  	majority, pending                    int32
    61  	success                              int32
    62  	errors                               []error
    63  
    64  	queues         []hostQueue
    65  	tagEncoderPool serialize.TagEncoderPool
    66  	pool           *writeStatePool
    67  }
    68  
    69  func newWriteState(
    70  	encoderPool serialize.TagEncoderPool,
    71  	pool *writeStatePool,
    72  ) *writeState {
    73  	w := &writeState{
    74  		pool:           pool,
    75  		tagEncoderPool: encoderPool,
    76  	}
    77  	w.destructorFn = w.close
    78  	w.L = w
    79  	return w
    80  }
    81  
    82  func (w *writeState) close() {
    83  	w.op.Close()
    84  
    85  	w.nsID.Finalize()
    86  	w.tsID.Finalize()
    87  
    88  	if w.annotation != nil {
    89  		w.annotation.DecRef()
    90  		w.annotation.Finalize()
    91  	}
    92  
    93  	if enc := w.tagEncoder; enc != nil {
    94  		enc.Finalize()
    95  	}
    96  
    97  	w.op, w.majority, w.pending, w.success = nil, 0, 0, 0
    98  	w.nsID, w.tsID, w.tagEncoder, w.annotation = nil, nil, nil, nil
    99  
   100  	for i := range w.errors {
   101  		w.errors[i] = nil
   102  	}
   103  	w.errors = w.errors[:0]
   104  
   105  	for i := range w.queues {
   106  		w.queues[i] = nil
   107  	}
   108  	w.queues = w.queues[:0]
   109  
   110  	if w.pool == nil {
   111  		return
   112  	}
   113  	w.pool.Put(w)
   114  }
   115  
   116  func (w *writeState) completionFn(result interface{}, err error) {
   117  	hostID := result.(topology.Host).ID()
   118  	// NB(bl) panic on invalid result, it indicates a bug in the code
   119  
   120  	w.Lock()
   121  	w.pending--
   122  
   123  	var wErr error
   124  
   125  	if err != nil {
   126  		if IsBadRequestError(err) {
   127  			// Wrap with invalid params and non-retryable so it is
   128  			// not retried.
   129  			err = xerrors.NewInvalidParamsError(err)
   130  			err = xerrors.NewNonRetryableError(err)
   131  		}
   132  		wErr = xerrors.NewRenamedError(err, fmt.Errorf("error writing to host %s: %v", hostID, err))
   133  	} else if hostShardSet, ok := w.topoMap.LookupHostShardSet(hostID); !ok {
   134  		errStr := "missing host shard in writeState completionFn: %s"
   135  		wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, hostID))
   136  	} else if shardState, err := hostShardSet.ShardSet().LookupStateByID(w.op.ShardID()); err != nil {
   137  		errStr := "missing shard %d in host %s"
   138  		wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, w.op.ShardID(), hostID))
   139  	} else {
   140  		available := shardState == shard.Available
   141  		leaving := shardState == shard.Leaving
   142  		leavingAndShardsLeavingCountTowardsConsistency := leaving &&
   143  			w.shardsLeavingCountTowardsConsistency
   144  		// NB(bl): Only count writes to available shards towards success.
   145  		// NB(r): If shard is leaving and configured to allow writes to leaving
   146  		// shards to count towards consistency then allow that to count
   147  		// to success.
   148  		if !available && !leavingAndShardsLeavingCountTowardsConsistency {
   149  			var errStr string
   150  			switch shardState {
   151  			case shard.Initializing:
   152  				errStr = "shard %d in host %s is not available (initializing)"
   153  			case shard.Leaving:
   154  				errStr = "shard %d in host %s not available (leaving)"
   155  			default:
   156  				errStr = "shard %d in host %s not available (unknown state)"
   157  			}
   158  			wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, w.op.ShardID(), hostID))
   159  		} else {
   160  			w.success++
   161  		}
   162  	}
   163  
   164  	if wErr != nil {
   165  		w.errors = append(w.errors, wErr)
   166  	}
   167  
   168  	switch w.consistencyLevel {
   169  	case topology.ConsistencyLevelOne:
   170  		if w.success > 0 || w.pending == 0 {
   171  			w.Signal()
   172  		}
   173  	case topology.ConsistencyLevelMajority:
   174  		if w.success >= w.majority || w.pending == 0 {
   175  			w.Signal()
   176  		}
   177  	case topology.ConsistencyLevelAll:
   178  		if w.pending == 0 {
   179  			w.Signal()
   180  		}
   181  	}
   182  
   183  	w.Unlock()
   184  	w.decRef()
   185  }
   186  
   187  type writeStatePool struct {
   188  	pool           pool.ObjectPool
   189  	tagEncoderPool serialize.TagEncoderPool
   190  }
   191  
   192  func newWriteStatePool(
   193  	tagEncoderPool serialize.TagEncoderPool,
   194  	opts pool.ObjectPoolOptions,
   195  ) *writeStatePool {
   196  	p := pool.NewObjectPool(opts)
   197  	return &writeStatePool{
   198  		pool:           p,
   199  		tagEncoderPool: tagEncoderPool,
   200  	}
   201  }
   202  
   203  func (p *writeStatePool) Init() {
   204  	p.pool.Init(func() interface{} {
   205  		return newWriteState(p.tagEncoderPool, p)
   206  	})
   207  }
   208  
   209  func (p *writeStatePool) Get() *writeState {
   210  	return p.pool.Get().(*writeState)
   211  }
   212  
   213  func (p *writeStatePool) Put(w *writeState) {
   214  	p.pool.Put(w)
   215  }