github.com/m3db/m3@v1.5.0/src/dbnode/storage/bootstrap/bootstrapper/uninitialized/source.go (about)

     1  // Copyright (c) 2018 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 uninitialized
    22  
    23  import (
    24  	"fmt"
    25  
    26  	"github.com/m3db/m3/src/cluster/shard"
    27  	"github.com/m3db/m3/src/dbnode/namespace"
    28  	"github.com/m3db/m3/src/dbnode/storage/bootstrap"
    29  	"github.com/m3db/m3/src/dbnode/storage/bootstrap/result"
    30  	"github.com/m3db/m3/src/dbnode/topology"
    31  	"github.com/m3db/m3/src/x/context"
    32  )
    33  
    34  // The purpose of the unitializedSource is to succeed bootstraps for any
    35  // shard/time-ranges if the cluster they're associated with has never
    36  // been completely initialized (is a new cluster). This is required for
    37  // allowing us to configure the bootstrappers such that the commitlog
    38  // bootstrapper can precede the peers bootstrapper and still succeed bootstraps
    39  // for brand new namespaces without permitting unintentional data loss by
    40  // putting the noop-all or noop-none bootstrappers at the end of the process.
    41  // Behavior is best understood by reading the test cases for the test:
    42  // TestUnitializedSourceAvailableDataAndAvailableIndex
    43  type uninitializedTopologySource struct {
    44  	opts            Options
    45  	instrumentation *instrumentation
    46  }
    47  
    48  // newTopologyUninitializedSource creates a new uninitialized source.
    49  func newTopologyUninitializedSource(opts Options) bootstrap.Source {
    50  	return &uninitializedTopologySource{
    51  		opts:            opts,
    52  		instrumentation: newInstrumentation(opts),
    53  	}
    54  }
    55  
    56  func (s *uninitializedTopologySource) AvailableData(
    57  	ns namespace.Metadata,
    58  	shardsTimeRanges result.ShardTimeRanges,
    59  	_ bootstrap.Cache,
    60  	runOpts bootstrap.RunOptions,
    61  ) (result.ShardTimeRanges, error) {
    62  	return s.availability(ns, shardsTimeRanges, runOpts)
    63  }
    64  
    65  func (s *uninitializedTopologySource) AvailableIndex(
    66  	ns namespace.Metadata,
    67  	shardsTimeRanges result.ShardTimeRanges,
    68  	_ bootstrap.Cache,
    69  	runOpts bootstrap.RunOptions,
    70  ) (result.ShardTimeRanges, error) {
    71  	return s.availability(ns, shardsTimeRanges, runOpts)
    72  }
    73  
    74  func (s *uninitializedTopologySource) availability(
    75  	_ namespace.Metadata,
    76  	shardsTimeRanges result.ShardTimeRanges,
    77  	runOpts bootstrap.RunOptions,
    78  ) (result.ShardTimeRanges, error) {
    79  	var (
    80  		topoState                = runOpts.InitialTopologyState()
    81  		availableShardTimeRanges = result.NewShardTimeRanges()
    82  	)
    83  
    84  	for shardIDUint := range shardsTimeRanges.Iter() {
    85  		shardID := topology.ShardID(shardIDUint)
    86  		hostShardStates, ok := topoState.ShardStates[shardID]
    87  		if !ok {
    88  			// This shard was not part of the topology when the bootstrapping
    89  			// process began.
    90  			continue
    91  		}
    92  
    93  		// The basic idea for the algorithm is that on a shard-by-shard basis we
    94  		// need to determine if the cluster is "new" in the sense that it has
    95  		// never been completely initialized (reached a state where all the hosts
    96  		// in the topology are "available" for that specific shard).
    97  		// In order to determine this, we simply count the number of hosts in the
    98  		// "initializing" state. If this number is larger than zero, than the
    99  		// cluster is "new".
   100  		// The one exception to this case is when we perform topology changes and
   101  		// we end up with one extra node that is initializing which should be offset
   102  		// by the corresponding node that is leaving. I.E if numInitializing > 0
   103  		// BUT numLeaving >= numInitializing then it is still not a new namespace.
   104  		// See the TestUnitializedSourceAvailableDataAndAvailableIndex test for more details.
   105  		var (
   106  			numAvailable    = 0
   107  			numInitializing = 0
   108  			numLeaving      = 0
   109  		)
   110  		for _, hostState := range hostShardStates {
   111  			shardState := hostState.ShardState
   112  			switch shardState {
   113  			case shard.Initializing:
   114  				numInitializing++
   115  			case shard.Leaving:
   116  				numLeaving++
   117  			case shard.Available:
   118  				numAvailable++
   119  			case shard.Unknown:
   120  				fallthrough
   121  			default:
   122  				return nil, fmt.Errorf("unknown shard state: %v", shardState)
   123  			}
   124  		}
   125  
   126  		// This heuristic works for all scenarios except for if we tried to change the replication
   127  		// factor of a cluster that was already initialized. In that case, we might have to come
   128  		// up with a new heuristic, or simply require that the peers bootstrapper be configured as
   129  		// a bootstrapper if users want to change the replication factor dynamically, which is fine
   130  		// because otherwise you'd have to wait for one entire retention period for the replicaiton
   131  		// factor to actually increase correctly.
   132  		shardHasNeverBeenCompletelyInitialized := numInitializing-numLeaving > 0
   133  		if shardHasNeverBeenCompletelyInitialized {
   134  			if tr, ok := shardsTimeRanges.Get(shardIDUint); ok {
   135  				availableShardTimeRanges.Set(shardIDUint, tr)
   136  			}
   137  		}
   138  	}
   139  
   140  	return availableShardTimeRanges, nil
   141  }
   142  
   143  func (s *uninitializedTopologySource) Read(
   144  	ctx context.Context,
   145  	namespaces bootstrap.Namespaces,
   146  	_ bootstrap.Cache,
   147  ) (bootstrap.NamespaceResults, error) {
   148  	instrCtx := s.instrumentation.uninitializedBootstrapperSourceReadStarted(ctx)
   149  	defer instrCtx.finish()
   150  
   151  	results := bootstrap.NamespaceResults{
   152  		Results: bootstrap.NewNamespaceResultsMap(bootstrap.NamespaceResultsMapOptions{}),
   153  	}
   154  	for _, elem := range namespaces.Namespaces.Iter() {
   155  		ns := elem.Value()
   156  
   157  		namespaceResult := bootstrap.NamespaceResult{
   158  			Metadata: ns.Metadata,
   159  			Shards:   ns.Shards,
   160  		}
   161  
   162  		availability, err := s.availability(ns.Metadata,
   163  			ns.DataRunOptions.ShardTimeRanges, ns.DataRunOptions.RunOptions)
   164  		if err != nil {
   165  			return bootstrap.NamespaceResults{}, err
   166  		}
   167  
   168  		missing := ns.DataRunOptions.ShardTimeRanges.Copy()
   169  		missing.Subtract(availability)
   170  
   171  		if missing.IsEmpty() {
   172  			namespaceResult.DataResult = result.NewDataBootstrapResult()
   173  		} else {
   174  			namespaceResult.DataResult = missing.ToUnfulfilledDataResult()
   175  		}
   176  
   177  		if ns.Metadata.Options().IndexOptions().Enabled() {
   178  			if missing.IsEmpty() {
   179  				namespaceResult.IndexResult = result.NewIndexBootstrapResult()
   180  			} else {
   181  				namespaceResult.IndexResult = missing.ToUnfulfilledIndexResult()
   182  			}
   183  		}
   184  
   185  		results.Results.Set(ns.Metadata.ID(), namespaceResult)
   186  	}
   187  
   188  	return results, nil
   189  }