github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/topology/map.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 topology
    22  
    23  import (
    24  	"github.com/m3db/m3/src/cluster/shard"
    25  	"github.com/m3db/m3/src/dbnode/sharding"
    26  	"github.com/m3db/m3/src/x/ident"
    27  	xwatch "github.com/m3db/m3/src/x/watch"
    28  )
    29  
    30  type staticMap struct {
    31  	shardSet                 sharding.ShardSet
    32  	hostShardSets            []HostShardSet
    33  	hostShardSetsByID        map[string]HostShardSet
    34  	orderedHosts             []Host
    35  	hostsByShard             [][]Host
    36  	orderedShardHostsByShard [][]orderedShardHost
    37  	initializingHostMap      map[string]map[uint32]string // it stores {leavingHostID : {shardID : initializingHostID}}
    38  	replicas                 int
    39  	majority                 int
    40  }
    41  
    42  // NewStaticMap creates a new static topology map
    43  func NewStaticMap(opts StaticOptions) Map {
    44  	totalShards := len(opts.ShardSet().AllIDs())
    45  	hostShardSets := opts.HostShardSets()
    46  	topoMap := staticMap{
    47  		shardSet:                 opts.ShardSet(),
    48  		hostShardSets:            hostShardSets,
    49  		hostShardSetsByID:        make(map[string]HostShardSet),
    50  		orderedHosts:             make([]Host, 0, len(hostShardSets)),
    51  		hostsByShard:             make([][]Host, totalShards),
    52  		orderedShardHostsByShard: make([][]orderedShardHost, totalShards),
    53  		replicas:                 opts.Replicas(),
    54  		majority:                 Majority(opts.Replicas()),
    55  	}
    56  
    57  	for idx, hostShardSet := range hostShardSets {
    58  		host := hostShardSet.Host()
    59  		topoMap.hostShardSetsByID[host.ID()] = hostShardSet
    60  		topoMap.orderedHosts = append(topoMap.orderedHosts, host)
    61  
    62  		for _, shard := range hostShardSet.ShardSet().All() {
    63  			id := shard.ID()
    64  			topoMap.hostsByShard[id] = append(topoMap.hostsByShard[id], host)
    65  			elem := orderedShardHost{
    66  				idx:   idx,
    67  				shard: shard,
    68  				host:  host,
    69  			}
    70  			topoMap.orderedShardHostsByShard[id] =
    71  				append(topoMap.orderedShardHostsByShard[id], elem)
    72  		}
    73  	}
    74  	for _, hostShardSet := range hostShardSets {
    75  		host := hostShardSet.Host()
    76  		var shardToInitializingHost map[uint32]string
    77  		for _, shard := range hostShardSet.ShardSet().All() {
    78  			if shard.SourceID() != "" {
    79  				if topoMap.initializingHostMap == nil {
    80  					topoMap.initializingHostMap = make(map[string]map[uint32]string)
    81  				}
    82  				if shardToInitializingHost == nil {
    83  					shardToInitializingHost = make(map[uint32]string)
    84  				}
    85  				shardToInitializingHost[shard.ID()] = host.ID()
    86  				topoMap.initializingHostMap[shard.SourceID()] = shardToInitializingHost
    87  			}
    88  		}
    89  	}
    90  	return &topoMap
    91  }
    92  
    93  type orderedShardHost struct {
    94  	idx   int
    95  	shard shard.Shard
    96  	host  Host
    97  }
    98  
    99  func (t *staticMap) Hosts() []Host {
   100  	return t.orderedHosts
   101  }
   102  
   103  func (t *staticMap) HostShardSets() []HostShardSet {
   104  	return t.hostShardSets
   105  }
   106  
   107  func (t *staticMap) LookupHostShardSet(id string) (HostShardSet, bool) {
   108  	value, ok := t.hostShardSetsByID[id]
   109  	return value, ok
   110  }
   111  
   112  func (t *staticMap) LookupInitializingHostPair(leavingHostID string, id uint32) (string, bool) {
   113  	value, ok := t.initializingHostMap[leavingHostID]
   114  	if !ok {
   115  		return "", false
   116  	}
   117  	initializingHost, ok := value[id]
   118  	return initializingHost, ok
   119  }
   120  
   121  func (t *staticMap) HostsLen() int {
   122  	return len(t.orderedHosts)
   123  }
   124  
   125  func (t *staticMap) ShardSet() sharding.ShardSet {
   126  	return t.shardSet
   127  }
   128  
   129  func (t *staticMap) Route(id ident.ID) (uint32, []Host, error) {
   130  	shard := t.shardSet.Lookup(id)
   131  	if int(shard) >= len(t.hostsByShard) {
   132  		return shard, nil, errUnownedShard
   133  	}
   134  	return shard, t.hostsByShard[shard], nil
   135  }
   136  
   137  func (t *staticMap) RouteForEach(id ident.ID, forEachFn RouteForEachFn) error {
   138  	return t.RouteShardForEach(t.shardSet.Lookup(id), forEachFn)
   139  }
   140  
   141  func (t *staticMap) RouteShard(shard uint32) ([]Host, error) {
   142  	if int(shard) >= len(t.hostsByShard) {
   143  		return nil, errUnownedShard
   144  	}
   145  	return t.hostsByShard[shard], nil
   146  }
   147  
   148  func (t *staticMap) RouteShardForEach(shard uint32, forEachFn RouteForEachFn) error {
   149  	if int(shard) >= len(t.orderedShardHostsByShard) {
   150  		return errUnownedShard
   151  	}
   152  	orderedShardHosts := t.orderedShardHostsByShard[shard]
   153  	for _, elem := range orderedShardHosts {
   154  		forEachFn(elem.idx, elem.shard, elem.host)
   155  	}
   156  	return nil
   157  }
   158  
   159  func (t *staticMap) Replicas() int {
   160  	return t.replicas
   161  }
   162  
   163  func (t *staticMap) MajorityReplicas() int {
   164  	return t.majority
   165  }
   166  
   167  type mapWatch struct {
   168  	xwatch.Watch
   169  }
   170  
   171  // NewMapWatch creates a new watch on a topology map
   172  // from a generic watch that watches a Map
   173  func NewMapWatch(w xwatch.Watch) MapWatch {
   174  	return &mapWatch{w}
   175  }
   176  
   177  func (w *mapWatch) C() <-chan struct{} {
   178  	return w.Watch.C()
   179  }
   180  
   181  func (w *mapWatch) Get() Map {
   182  	value := w.Watch.Get()
   183  	if value == nil {
   184  		return nil
   185  	}
   186  	return value.(Map)
   187  }
   188  
   189  func (w *mapWatch) Close() {
   190  	w.Watch.Close()
   191  }