github.com/m3db/m3@v1.5.0/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  	replicas                 int
    38  	majority                 int
    39  }
    40  
    41  // NewStaticMap creates a new static topology map
    42  func NewStaticMap(opts StaticOptions) Map {
    43  	totalShards := len(opts.ShardSet().AllIDs())
    44  	hostShardSets := opts.HostShardSets()
    45  	topoMap := staticMap{
    46  		shardSet:                 opts.ShardSet(),
    47  		hostShardSets:            hostShardSets,
    48  		hostShardSetsByID:        make(map[string]HostShardSet),
    49  		orderedHosts:             make([]Host, 0, len(hostShardSets)),
    50  		hostsByShard:             make([][]Host, totalShards),
    51  		orderedShardHostsByShard: make([][]orderedShardHost, totalShards),
    52  		replicas:                 opts.Replicas(),
    53  		majority:                 Majority(opts.Replicas()),
    54  	}
    55  
    56  	for idx, hostShardSet := range hostShardSets {
    57  		host := hostShardSet.Host()
    58  		topoMap.hostShardSetsByID[host.ID()] = hostShardSet
    59  		topoMap.orderedHosts = append(topoMap.orderedHosts, host)
    60  		for _, shard := range hostShardSet.ShardSet().All() {
    61  			id := shard.ID()
    62  			topoMap.hostsByShard[id] = append(topoMap.hostsByShard[id], host)
    63  			elem := orderedShardHost{
    64  				idx:   idx,
    65  				shard: shard,
    66  				host:  host,
    67  			}
    68  			topoMap.orderedShardHostsByShard[id] =
    69  				append(topoMap.orderedShardHostsByShard[id], elem)
    70  		}
    71  	}
    72  
    73  	return &topoMap
    74  }
    75  
    76  type orderedShardHost struct {
    77  	idx   int
    78  	shard shard.Shard
    79  	host  Host
    80  }
    81  
    82  func (t *staticMap) Hosts() []Host {
    83  	return t.orderedHosts
    84  }
    85  
    86  func (t *staticMap) HostShardSets() []HostShardSet {
    87  	return t.hostShardSets
    88  }
    89  
    90  func (t *staticMap) LookupHostShardSet(id string) (HostShardSet, bool) {
    91  	value, ok := t.hostShardSetsByID[id]
    92  	return value, ok
    93  }
    94  
    95  func (t *staticMap) HostsLen() int {
    96  	return len(t.orderedHosts)
    97  }
    98  
    99  func (t *staticMap) ShardSet() sharding.ShardSet {
   100  	return t.shardSet
   101  }
   102  
   103  func (t *staticMap) Route(id ident.ID) (uint32, []Host, error) {
   104  	shard := t.shardSet.Lookup(id)
   105  	if int(shard) >= len(t.hostsByShard) {
   106  		return shard, nil, errUnownedShard
   107  	}
   108  	return shard, t.hostsByShard[shard], nil
   109  }
   110  
   111  func (t *staticMap) RouteForEach(id ident.ID, forEachFn RouteForEachFn) error {
   112  	return t.RouteShardForEach(t.shardSet.Lookup(id), forEachFn)
   113  }
   114  
   115  func (t *staticMap) RouteShard(shard uint32) ([]Host, error) {
   116  	if int(shard) >= len(t.hostsByShard) {
   117  		return nil, errUnownedShard
   118  	}
   119  	return t.hostsByShard[shard], nil
   120  }
   121  
   122  func (t *staticMap) RouteShardForEach(shard uint32, forEachFn RouteForEachFn) error {
   123  	if int(shard) >= len(t.orderedShardHostsByShard) {
   124  		return errUnownedShard
   125  	}
   126  	orderedShardHosts := t.orderedShardHostsByShard[shard]
   127  	for _, elem := range orderedShardHosts {
   128  		forEachFn(elem.idx, elem.shard, elem.host)
   129  	}
   130  	return nil
   131  }
   132  
   133  func (t *staticMap) Replicas() int {
   134  	return t.replicas
   135  }
   136  
   137  func (t *staticMap) MajorityReplicas() int {
   138  	return t.majority
   139  }
   140  
   141  type mapWatch struct {
   142  	xwatch.Watch
   143  }
   144  
   145  // NewMapWatch creates a new watch on a topology map
   146  // from a generic watch that watches a Map
   147  func NewMapWatch(w xwatch.Watch) MapWatch {
   148  	return &mapWatch{w}
   149  }
   150  
   151  func (w *mapWatch) C() <-chan struct{} {
   152  	return w.Watch.C()
   153  }
   154  
   155  func (w *mapWatch) Get() Map {
   156  	value := w.Watch.Get()
   157  	if value == nil {
   158  		return nil
   159  	}
   160  	return value.(Map)
   161  }
   162  
   163  func (w *mapWatch) Close() {
   164  	w.Watch.Close()
   165  }