github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/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 "time" 27 28 "github.com/m3db/m3/src/cluster/shard" 29 "github.com/m3db/m3/src/dbnode/topology" 30 "github.com/m3db/m3/src/x/checked" 31 xerrors "github.com/m3db/m3/src/x/errors" 32 "github.com/m3db/m3/src/x/ident" 33 "github.com/m3db/m3/src/x/pool" 34 "github.com/m3db/m3/src/x/sampler" 35 "github.com/m3db/m3/src/x/serialize" 36 37 "go.uber.org/zap" 38 ) 39 40 type countTowardsConsistency int64 41 42 const ( 43 undefinedCountTowardsConsistency countTowardsConsistency = iota 44 availableCountTowardsConsistency 45 leavingCountTowardsConsistency 46 initializingCountTowardsConsistency 47 shardLeavingIndividuallyCountTowardsConsistency 48 shardLeavingAsPairCountTowardsConsistency 49 shardInitializingAsPairCountTowardsConsistency 50 ) 51 52 // writeOp represents a generic write operation 53 type writeOp interface { 54 op 55 56 ShardID() uint32 57 58 SetCompletionFn(fn completionFn) 59 60 Close() 61 } 62 63 type writeState struct { 64 sync.Cond 65 sync.Mutex 66 refCounter 67 68 consistencyLevel topology.ConsistencyLevel 69 shardsLeavingCountTowardsConsistency bool 70 shardsLeavingAndInitializingCountTowardsConsistency bool 71 leavingAndInitializingPairCounted bool 72 topoMap topology.Map 73 hostSuccessList []string 74 op writeOp 75 nsID ident.ID 76 tsID ident.ID 77 tagEncoder serialize.TagEncoder 78 annotation checked.Bytes 79 majority, pending int32 80 success int32 81 errors []error 82 lastResetTime time.Time 83 queues []hostQueue 84 tagEncoderPool serialize.TagEncoderPool 85 pool *writeStatePool 86 } 87 88 func newWriteState( 89 encoderPool serialize.TagEncoderPool, 90 pool *writeStatePool, 91 ) *writeState { 92 w := &writeState{ 93 pool: pool, 94 tagEncoderPool: encoderPool, 95 } 96 w.destructorFn = w.close 97 w.L = w 98 return w 99 } 100 101 func (w *writeState) close() { 102 w.op.Close() 103 104 w.nsID.Finalize() 105 w.tsID.Finalize() 106 var emptyString string 107 for i := range w.hostSuccessList { 108 w.hostSuccessList[i] = emptyString 109 } 110 w.hostSuccessList = w.hostSuccessList[:0] 111 if w.annotation != nil { 112 w.annotation.DecRef() 113 w.annotation.Finalize() 114 } 115 116 if enc := w.tagEncoder; enc != nil { 117 enc.Finalize() 118 } 119 120 w.op, w.majority, w.pending, w.success = nil, 0, 0, 0 121 w.nsID, w.tsID, w.tagEncoder, w.annotation = nil, nil, nil, nil 122 123 for i := range w.errors { 124 w.errors[i] = nil 125 } 126 w.errors = w.errors[:0] 127 128 w.lastResetTime = time.Time{} 129 130 for i := range w.queues { 131 w.queues[i] = nil 132 } 133 w.queues = w.queues[:0] 134 135 if w.pool == nil { 136 return 137 } 138 w.pool.Put(w) 139 } 140 141 func (w *writeState) completionFn(result interface{}, err error) { 142 host := result.(topology.Host) 143 hostID := host.ID() 144 // NB(bl) panic on invalid result, it indicates a bug in the code 145 146 w.Lock() 147 w.pending-- 148 149 var ( 150 took time.Duration 151 wErr error 152 ) 153 if !w.lastResetTime.IsZero() { 154 took = time.Since(w.lastResetTime) 155 } 156 if err != nil { 157 if IsBadRequestError(err) { 158 // Wrap with invalid params and non-retryable so it is 159 // not retried. 160 err = xerrors.NewInvalidParamsError(err) 161 err = xerrors.NewNonRetryableError(err) 162 } 163 164 w.pool.MaybeLogHostError(maybeHostWriteError{err: err, host: host, reqRespTime: took}) 165 wErr = xerrors.NewRenamedError(err, fmt.Errorf("error writing to host %s: %v", hostID, err)) 166 } else if hostShardSet, ok := w.topoMap.LookupHostShardSet(hostID); !ok { 167 errStr := "missing host shard in writeState completionFn: %s" 168 wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, hostID)) 169 } else if shardState, err := hostShardSet.ShardSet().LookupStateByID(w.op.ShardID()); err != nil { 170 errStr := "missing shard %d in host %s" 171 wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, w.op.ShardID(), hostID)) 172 } else { 173 // in below conditions we consider the write success 174 // 1. writes to available shards towards success. 175 // 2. If shard is leaving and configured to allow writes to leaving 176 // shards to count towards consistency then allow that to count 177 // to success 178 // 3. If shardsLeavingAndInitializingCountTowardsConsistency flag is true then count the success on writing to both 179 // leaving and initializing as pair. 180 switch newCountTowardsConsistency(shardState, 181 w.shardsLeavingCountTowardsConsistency, 182 w.shardsLeavingAndInitializingCountTowardsConsistency) { 183 case availableCountTowardsConsistency: 184 w.success++ 185 case shardLeavingIndividuallyCountTowardsConsistency: 186 w.success++ 187 case shardLeavingAsPairCountTowardsConsistency: 188 // get the initializing host corresponding to the leaving host. 189 initializingHostID, ok := w.topoMap.LookupInitializingHostPair(hostID, w.op.ShardID()) 190 if !ok || initializingHostID == "" { 191 errStr := "no initializing host for shard id %d in host %s" 192 wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, w.op.ShardID(), hostID)) 193 } else { 194 w.setHostSuccessListWithLock(hostID, initializingHostID) 195 } 196 case shardInitializingAsPairCountTowardsConsistency: 197 shard, err := hostShardSet.ShardSet().LookupShard(w.op.ShardID()) 198 if err != nil { 199 errStr := "no shard id %d in host %s" 200 wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, w.op.ShardID(), hostID)) 201 } else { 202 // get the leaving host corresponding to the initializing host. 203 leavingHostID := shard.SourceID() 204 if leavingHostID == "" { 205 errStr := "no leaving host for shard id %d in host %s" 206 wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, w.op.ShardID(), hostID)) 207 } else { 208 w.setHostSuccessListWithLock(hostID, leavingHostID) 209 } 210 } 211 case leavingCountTowardsConsistency: 212 errStr := "shard %d in host %s not available (leaving)" 213 wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, w.op.ShardID(), hostID)) 214 case initializingCountTowardsConsistency: 215 errStr := "shard %d in host %s is not available (initializing)" 216 wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, w.op.ShardID(), hostID)) 217 default: 218 errStr := "shard %d in host %s not available (unknown state)" 219 wErr = xerrors.NewRetryableError(fmt.Errorf(errStr, w.op.ShardID(), hostID)) 220 } 221 } 222 223 if wErr != nil { 224 w.errors = append(w.errors, wErr) 225 } 226 227 switch w.consistencyLevel { 228 case topology.ConsistencyLevelOne: 229 if w.success > 0 || w.pending == 0 { 230 w.Signal() 231 } 232 case topology.ConsistencyLevelMajority: 233 if w.success >= w.majority || w.pending == 0 { 234 w.Signal() 235 } 236 case topology.ConsistencyLevelAll: 237 if w.pending == 0 { 238 w.Signal() 239 } 240 } 241 242 w.Unlock() 243 w.decRef() 244 } 245 246 func (w *writeState) setHostSuccessListWithLock(hostID, pairedHostID string) { 247 if findHost(w.hostSuccessList, pairedHostID) { 248 w.success++ 249 w.leavingAndInitializingPairCounted = true 250 } 251 w.hostSuccessList = append(w.hostSuccessList, hostID) 252 } 253 254 type writeStatePool struct { 255 pool pool.ObjectPool 256 tagEncoderPool serialize.TagEncoderPool 257 logger *zap.Logger 258 logHostErrorSampler *sampler.Sampler 259 } 260 261 func newWriteStatePool( 262 tagEncoderPool serialize.TagEncoderPool, 263 opts pool.ObjectPoolOptions, 264 logger *zap.Logger, 265 logHostErrorSampler *sampler.Sampler, 266 ) *writeStatePool { 267 p := pool.NewObjectPool(opts) 268 return &writeStatePool{ 269 pool: p, 270 tagEncoderPool: tagEncoderPool, 271 logger: logger, 272 logHostErrorSampler: logHostErrorSampler, 273 } 274 } 275 276 func (p *writeStatePool) Init() { 277 p.pool.Init(func() interface{} { 278 return newWriteState(p.tagEncoderPool, p) 279 }) 280 } 281 282 func (p *writeStatePool) Get() *writeState { 283 return p.pool.Get().(*writeState) 284 } 285 286 func (p *writeStatePool) Put(w *writeState) { 287 p.pool.Put(w) 288 } 289 290 func (p *writeStatePool) MaybeLogHostError(hostErr maybeHostWriteError) { 291 if hostErr.err == nil { 292 // No error, this is an expected code path when host request doesn't 293 // encounter an error. 294 return 295 } 296 297 if !p.logHostErrorSampler.Sample() { 298 return 299 } 300 301 p.logger.Warn("sampled error writing to host (may not lead to consistency result error)", 302 zap.Stringer("host", hostErr.host), 303 zap.Duration("reqRespTime", hostErr.reqRespTime), 304 zap.Error(hostErr.err)) 305 } 306 307 type maybeHostWriteError struct { 308 // Note: both these fields should be set always. 309 host topology.Host 310 reqRespTime time.Duration 311 312 // Error field is optionally set when there is actually an error. 313 err error 314 } 315 316 func newCountTowardsConsistency( 317 shardState shard.State, 318 leavingCountsIndividually bool, 319 leavingAndInitializingCountsAsPair bool, 320 ) countTowardsConsistency { 321 isAvailable := shardState == shard.Available 322 isLeaving := shardState == shard.Leaving 323 isInitializing := shardState == shard.Initializing 324 325 if isAvailable { 326 return availableCountTowardsConsistency 327 } 328 if isLeaving && leavingCountsIndividually { 329 return shardLeavingIndividuallyCountTowardsConsistency 330 } 331 if isLeaving && leavingAndInitializingCountsAsPair { 332 return shardLeavingAsPairCountTowardsConsistency 333 } 334 if isInitializing && leavingAndInitializingCountsAsPair { 335 return shardInitializingAsPairCountTowardsConsistency 336 } 337 if isLeaving { 338 return leavingCountTowardsConsistency 339 } 340 if isInitializing { 341 return initializingCountTowardsConsistency 342 } 343 return undefinedCountTowardsConsistency 344 } 345 346 func findHost(hostSuccessList []string, hostID string) bool { 347 // The reason for iterating over list(hostSuccessList) instead of taking map here is the slice performs better over 348 // the map for less than 10 datasets. 349 for _, val := range hostSuccessList { 350 if val == hostID { 351 return true 352 } 353 } 354 return false 355 }