mirror of https://github.com/grpc/grpc-go.git
xds: cache locality (sub-balancer) (#3067)
When a locality is removed from EDS response, it's corresponding sub-balancer will be removed from balancer group. With this change, the sub-balancer won't be removed immediately. It will be kept in a cache (for 15 minutes by default). If the locality is re-added within the timeout, the sub-balancer in cache will be picked and re-used.
This commit is contained in:
parent
b53233ce4c
commit
041ead0785
|
@ -0,0 +1,131 @@
|
|||
/*
|
||||
* Copyright 2019 gRPC authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
// Package cache implements caches to be used in gRPC.
|
||||
package cache
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
type cacheEntry struct {
|
||||
item interface{}
|
||||
callback func()
|
||||
timer *time.Timer
|
||||
// deleted is set to true in Remove() when the call to timer.Stop() fails.
|
||||
// This can happen when the timer in the cache entry fires around the same
|
||||
// time that timer.stop() is called in Remove().
|
||||
deleted bool
|
||||
}
|
||||
|
||||
// TimeoutCache is a cache with items to be deleted after a timeout.
|
||||
type TimeoutCache struct {
|
||||
mu sync.Mutex
|
||||
timeout time.Duration
|
||||
cache map[interface{}]*cacheEntry
|
||||
}
|
||||
|
||||
// NewTimeoutCache creates a TimeoutCache with the given timeout.
|
||||
func NewTimeoutCache(timeout time.Duration) *TimeoutCache {
|
||||
return &TimeoutCache{
|
||||
timeout: timeout,
|
||||
cache: make(map[interface{}]*cacheEntry),
|
||||
}
|
||||
}
|
||||
|
||||
// Add adds an item to the cache, with the specified callback to be called when
|
||||
// the item is removed from the cache upon timeout. If the item is removed from
|
||||
// the cache using a call to Remove before the timeout expires, the callback
|
||||
// will not be called.
|
||||
//
|
||||
// If the Add was successful, it returns (newly added item, true). If there is
|
||||
// an existing entry for the specified key, the cache entry is not be updated
|
||||
// with the specified item and it returns (existing item, false).
|
||||
func (c *TimeoutCache) Add(key, item interface{}, callback func()) (interface{}, bool) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
if e, ok := c.cache[key]; ok {
|
||||
return e.item, false
|
||||
}
|
||||
|
||||
entry := &cacheEntry{
|
||||
item: item,
|
||||
callback: callback,
|
||||
}
|
||||
entry.timer = time.AfterFunc(c.timeout, func() {
|
||||
c.mu.Lock()
|
||||
if entry.deleted {
|
||||
c.mu.Unlock()
|
||||
// Abort the delete since this has been taken care of in Remove().
|
||||
return
|
||||
}
|
||||
delete(c.cache, key)
|
||||
c.mu.Unlock()
|
||||
entry.callback()
|
||||
})
|
||||
c.cache[key] = entry
|
||||
return item, true
|
||||
}
|
||||
|
||||
// Remove the item with the key from the cache.
|
||||
//
|
||||
// If the specified key exists in the cache, it returns (item associated with
|
||||
// key, true) and the callback associated with the item is guaranteed to be not
|
||||
// called. If the given key is not found in the cache, it returns (nil, false)
|
||||
func (c *TimeoutCache) Remove(key interface{}) (item interface{}, ok bool) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
entry, ok := c.removeInternal(key, false)
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
return entry.item, true
|
||||
}
|
||||
|
||||
// removeInternal removes and returns the item with key.
|
||||
//
|
||||
// caller must hold c.mu.
|
||||
func (c *TimeoutCache) removeInternal(key interface{}, runCallback bool) (*cacheEntry, bool) {
|
||||
entry, ok := c.cache[key]
|
||||
if !ok {
|
||||
return nil, false
|
||||
}
|
||||
delete(c.cache, key)
|
||||
if !entry.timer.Stop() {
|
||||
// If stop was not successful, the timer has fired (this can only happen
|
||||
// in a race). But the deleting function is blocked on c.mu because the
|
||||
// mutex was held by the caller of this function.
|
||||
//
|
||||
// Set deleted to true to abort the deleting function. When the lock is
|
||||
// released, the delete function will acquire the lock, check the value
|
||||
// of deleted and return.
|
||||
entry.deleted = true
|
||||
}
|
||||
if runCallback {
|
||||
entry.callback()
|
||||
}
|
||||
return entry, true
|
||||
}
|
||||
|
||||
// Clear removes all entries, and runs the callbacks if runCallback is true.
|
||||
func (c *TimeoutCache) Clear(runCallback bool) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
for key := range c.cache {
|
||||
c.removeInternal(key, runCallback)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,220 @@
|
|||
/*
|
||||
*
|
||||
* Copyright 2019 gRPC authors.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package cache
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
testCacheTimeout = 100 * time.Millisecond
|
||||
)
|
||||
|
||||
func (c *TimeoutCache) getForTesting(key interface{}) (*cacheEntry, bool) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
r, ok := c.cache[key]
|
||||
return r, ok
|
||||
}
|
||||
|
||||
// TestCacheExpire attempts to add an entry to the cache and verifies that it
|
||||
// was added successfully. It then makes sure that on timeout, it's removed and
|
||||
// the associated callback is called.
|
||||
func TestCacheExpire(t *testing.T) {
|
||||
const k, v = 1, "1"
|
||||
c := NewTimeoutCache(testCacheTimeout)
|
||||
|
||||
callbackChan := make(chan struct{})
|
||||
c.Add(k, v, func() { close(callbackChan) })
|
||||
|
||||
if gotV, ok := c.getForTesting(k); !ok || gotV.item != v {
|
||||
t.Fatalf("After Add(), before timeout, from cache got: %v, %v, want %v, %v", gotV.item, ok, v, true)
|
||||
}
|
||||
|
||||
select {
|
||||
case <-callbackChan:
|
||||
case <-time.After(testCacheTimeout * 2):
|
||||
t.Fatalf("timeout waiting for callback")
|
||||
}
|
||||
|
||||
if _, ok := c.getForTesting(k); ok {
|
||||
t.Fatalf("After Add(), after timeout, from cache got: _, %v, want _, %v", ok, false)
|
||||
}
|
||||
}
|
||||
|
||||
// TestCacheRemove attempts to remove an existing entry from the cache and
|
||||
// verifies that the entry is removed and the associated callback is not
|
||||
// invoked.
|
||||
func TestCacheRemove(t *testing.T) {
|
||||
const k, v = 1, "1"
|
||||
c := NewTimeoutCache(testCacheTimeout)
|
||||
|
||||
callbackChan := make(chan struct{})
|
||||
c.Add(k, v, func() { close(callbackChan) })
|
||||
|
||||
if got, ok := c.getForTesting(k); !ok || got.item != v {
|
||||
t.Fatalf("After Add(), before timeout, from cache got: %v, %v, want %v, %v", got.item, ok, v, true)
|
||||
}
|
||||
|
||||
time.Sleep(testCacheTimeout / 2)
|
||||
|
||||
gotV, gotOK := c.Remove(k)
|
||||
if !gotOK || gotV != v {
|
||||
t.Fatalf("After Add(), before timeout, Remove() got: %v, %v, want %v, %v", gotV, gotOK, v, true)
|
||||
}
|
||||
|
||||
if _, ok := c.getForTesting(k); ok {
|
||||
t.Fatalf("After Add(), before timeout, after Remove(), from cache got: _, %v, want _, %v", ok, false)
|
||||
}
|
||||
|
||||
select {
|
||||
case <-callbackChan:
|
||||
t.Fatalf("unexpected callback after retrieve")
|
||||
case <-time.After(testCacheTimeout * 2):
|
||||
}
|
||||
}
|
||||
|
||||
// TestCacheClearWithoutCallback attempts to clear all entries from the cache
|
||||
// and verifies that the associated callbacks are not invoked.
|
||||
func TestCacheClearWithoutCallback(t *testing.T) {
|
||||
var values []string
|
||||
const itemCount = 3
|
||||
for i := 0; i < itemCount; i++ {
|
||||
values = append(values, strconv.Itoa(i))
|
||||
}
|
||||
c := NewTimeoutCache(testCacheTimeout)
|
||||
|
||||
done := make(chan struct{})
|
||||
defer close(done)
|
||||
callbackChan := make(chan struct{}, itemCount)
|
||||
|
||||
for i, v := range values {
|
||||
callbackChanTemp := make(chan struct{})
|
||||
c.Add(i, v, func() { close(callbackChanTemp) })
|
||||
go func() {
|
||||
select {
|
||||
case <-callbackChanTemp:
|
||||
callbackChan <- struct{}{}
|
||||
case <-done:
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
for i, v := range values {
|
||||
if got, ok := c.getForTesting(i); !ok || got.item != v {
|
||||
t.Fatalf("After Add(), before timeout, from cache got: %v, %v, want %v, %v", got.item, ok, v, true)
|
||||
}
|
||||
}
|
||||
|
||||
time.Sleep(testCacheTimeout / 2)
|
||||
c.Clear(false)
|
||||
|
||||
for i := range values {
|
||||
if _, ok := c.getForTesting(i); ok {
|
||||
t.Fatalf("After Add(), before timeout, after Remove(), from cache got: _, %v, want _, %v", ok, false)
|
||||
}
|
||||
}
|
||||
|
||||
select {
|
||||
case <-callbackChan:
|
||||
t.Fatalf("unexpected callback after Clear")
|
||||
case <-time.After(testCacheTimeout * 2):
|
||||
}
|
||||
}
|
||||
|
||||
// TestCacheClearWithCallback attempts to clear all entries from the cache and
|
||||
// verifies that the associated callbacks are invoked.
|
||||
func TestCacheClearWithCallback(t *testing.T) {
|
||||
var values []string
|
||||
const itemCount = 3
|
||||
for i := 0; i < itemCount; i++ {
|
||||
values = append(values, strconv.Itoa(i))
|
||||
}
|
||||
c := NewTimeoutCache(time.Hour)
|
||||
|
||||
testDone := make(chan struct{})
|
||||
defer close(testDone)
|
||||
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(itemCount)
|
||||
for i, v := range values {
|
||||
callbackChanTemp := make(chan struct{})
|
||||
c.Add(i, v, func() { close(callbackChanTemp) })
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
select {
|
||||
case <-callbackChanTemp:
|
||||
case <-testDone:
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
allGoroutineDone := make(chan struct{}, itemCount)
|
||||
go func() {
|
||||
wg.Wait()
|
||||
close(allGoroutineDone)
|
||||
}()
|
||||
|
||||
for i, v := range values {
|
||||
if got, ok := c.getForTesting(i); !ok || got.item != v {
|
||||
t.Fatalf("After Add(), before timeout, from cache got: %v, %v, want %v, %v", got.item, ok, v, true)
|
||||
}
|
||||
}
|
||||
|
||||
time.Sleep(testCacheTimeout / 2)
|
||||
c.Clear(true)
|
||||
|
||||
for i := range values {
|
||||
if _, ok := c.getForTesting(i); ok {
|
||||
t.Fatalf("After Add(), before timeout, after Remove(), from cache got: _, %v, want _, %v", ok, false)
|
||||
}
|
||||
}
|
||||
|
||||
select {
|
||||
case <-allGoroutineDone:
|
||||
case <-time.After(testCacheTimeout * 2):
|
||||
t.Fatalf("timeout waiting for all callbacks")
|
||||
}
|
||||
}
|
||||
|
||||
// TestCacheRetrieveTimeoutRace simulates the case where an entry's timer fires
|
||||
// around the same time that Remove() is called for it. It verifies that there
|
||||
// is no deadlock.
|
||||
func TestCacheRetrieveTimeoutRace(t *testing.T) {
|
||||
c := NewTimeoutCache(time.Nanosecond)
|
||||
|
||||
done := make(chan struct{})
|
||||
go func() {
|
||||
for i := 0; i < 1000; i++ {
|
||||
// Add starts a timer with 1 ns timeout, then remove will race
|
||||
// with the timer.
|
||||
c.Add(i, strconv.Itoa(i), func() {})
|
||||
c.Remove(i)
|
||||
}
|
||||
close(done)
|
||||
}()
|
||||
|
||||
select {
|
||||
case <-time.After(time.Second):
|
||||
t.Fatalf("Test didn't finish within 1 second. Deadlock")
|
||||
case <-done:
|
||||
}
|
||||
}
|
|
@ -20,11 +20,13 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/balancer/base"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/internal/cache"
|
||||
"google.golang.org/grpc/internal/wrr"
|
||||
"google.golang.org/grpc/resolver"
|
||||
"google.golang.org/grpc/xds/internal"
|
||||
|
@ -38,7 +40,25 @@ import (
|
|||
//
|
||||
// When the config changes, it will pass the update to the underlying balancer
|
||||
// if it exists.
|
||||
//
|
||||
// TODO: rename to subBalanceWrapper (and move to a separate file?)
|
||||
type subBalancerWithConfig struct {
|
||||
// subBalancerWithConfig is passed to the sub-balancer as a ClientConn
|
||||
// wrapper, only to keep the state and picker. When sub-balancer is
|
||||
// restarted while in cache, the picker needs to be resent.
|
||||
//
|
||||
// It also contains the sub-balancer ID, so the parent balancer group can
|
||||
// keep track of SubConn/pickers and the sub-balancers they belong to. Some
|
||||
// of the actions are forwarded to the parent ClientConn with no change.
|
||||
// Some are forward to balancer group with the sub-balancer ID.
|
||||
balancer.ClientConn
|
||||
id internal.Locality
|
||||
group *balancerGroup
|
||||
|
||||
mu sync.Mutex
|
||||
state connectivity.State
|
||||
picker balancer.Picker
|
||||
|
||||
// The static part of sub-balancer. Keeps balancerBuilders and addresses.
|
||||
// To be used when restarting sub-balancer.
|
||||
builder balancer.Builder
|
||||
|
@ -48,8 +68,30 @@ type subBalancerWithConfig struct {
|
|||
balancer balancer.Balancer
|
||||
}
|
||||
|
||||
func (sbc *subBalancerWithConfig) startBalancer(bgcc *balancerGroupCC) {
|
||||
b := sbc.builder.Build(bgcc, balancer.BuildOptions{})
|
||||
// UpdateBalancerState overrides balancer.ClientConn, to keep state and picker.
|
||||
func (sbc *subBalancerWithConfig) UpdateBalancerState(state connectivity.State, picker balancer.Picker) {
|
||||
sbc.mu.Lock()
|
||||
sbc.state, sbc.picker = state, picker
|
||||
sbc.group.updateBalancerState(sbc.id, state, picker)
|
||||
sbc.mu.Unlock()
|
||||
}
|
||||
|
||||
// NewSubConn overrides balancer.ClientConn, so balancer group can keep track of
|
||||
// the relation between subconns and sub-balancers.
|
||||
func (sbc *subBalancerWithConfig) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) {
|
||||
return sbc.group.newSubConn(sbc, addrs, opts)
|
||||
}
|
||||
|
||||
func (sbc *subBalancerWithConfig) updateBalancerStateWithCachedPicker() {
|
||||
sbc.mu.Lock()
|
||||
if sbc.picker != nil {
|
||||
sbc.group.updateBalancerState(sbc.id, sbc.state, sbc.picker)
|
||||
}
|
||||
sbc.mu.Unlock()
|
||||
}
|
||||
|
||||
func (sbc *subBalancerWithConfig) startBalancer() {
|
||||
b := sbc.builder.Build(sbc, balancer.BuildOptions{})
|
||||
sbc.balancer = b
|
||||
if ub, ok := b.(balancer.V2Balancer); ok {
|
||||
ub.UpdateClientConnState(balancer.ClientConnState{ResolverState: resolver.State{Addresses: sbc.addrs}})
|
||||
|
@ -145,6 +187,8 @@ type balancerGroup struct {
|
|||
outgoingMu sync.Mutex
|
||||
outgoingStarted bool
|
||||
idToBalancerConfig map[internal.Locality]*subBalancerWithConfig
|
||||
// Cache for sub-balancers when they are removed.
|
||||
balancerCache *cache.TimeoutCache
|
||||
|
||||
// incomingMu and pickerMu are to make sure this balancer group doesn't send
|
||||
// updates to cc after it's closed.
|
||||
|
@ -169,7 +213,7 @@ type balancerGroup struct {
|
|||
// from sub-balancers after they are closed.
|
||||
incomingMu sync.Mutex
|
||||
incomingStarted bool // This boolean only guards calls back to ClientConn.
|
||||
scToID map[balancer.SubConn]internal.Locality
|
||||
scToSubBalancer map[balancer.SubConn]*subBalancerWithConfig
|
||||
// All balancer IDs exist as keys in this map, even if balancer group is not
|
||||
// started.
|
||||
//
|
||||
|
@ -177,13 +221,20 @@ type balancerGroup struct {
|
|||
idToPickerState map[internal.Locality]*pickerState
|
||||
}
|
||||
|
||||
// defaultSubBalancerCloseTimeout is defined as a variable instead of const for
|
||||
// testing.
|
||||
//
|
||||
// TODO: make it a parameter for newBalancerGroup().
|
||||
var defaultSubBalancerCloseTimeout = 15 * time.Minute
|
||||
|
||||
func newBalancerGroup(cc balancer.ClientConn, loadStore lrs.Store) *balancerGroup {
|
||||
return &balancerGroup{
|
||||
cc: cc,
|
||||
loadStore: loadStore,
|
||||
|
||||
idToBalancerConfig: make(map[internal.Locality]*subBalancerWithConfig),
|
||||
scToID: make(map[balancer.SubConn]internal.Locality),
|
||||
balancerCache: cache.NewTimeoutCache(defaultSubBalancerCloseTimeout),
|
||||
scToSubBalancer: make(map[balancer.SubConn]*subBalancerWithConfig),
|
||||
idToPickerState: make(map[internal.Locality]*pickerState),
|
||||
}
|
||||
}
|
||||
|
@ -199,12 +250,8 @@ func (bg *balancerGroup) start() {
|
|||
return
|
||||
}
|
||||
|
||||
for id, config := range bg.idToBalancerConfig {
|
||||
config.startBalancer(&balancerGroupCC{
|
||||
ClientConn: bg.cc,
|
||||
id: id,
|
||||
group: bg,
|
||||
})
|
||||
for _, config := range bg.idToBalancerConfig {
|
||||
config.startBalancer()
|
||||
}
|
||||
bg.outgoingStarted = true
|
||||
bg.outgoingMu.Unlock()
|
||||
|
@ -233,57 +280,80 @@ func (bg *balancerGroup) add(id internal.Locality, weight uint32, builder balanc
|
|||
|
||||
// Store data in static map, and then check to see if bg is started.
|
||||
bg.outgoingMu.Lock()
|
||||
if _, ok := bg.idToBalancerConfig[id]; ok {
|
||||
bg.outgoingMu.Unlock()
|
||||
grpclog.Warningf("balancer group: adding a balancer with existing ID: %s", id)
|
||||
return
|
||||
}
|
||||
sbc := &subBalancerWithConfig{
|
||||
builder: builder,
|
||||
}
|
||||
bg.idToBalancerConfig[id] = sbc
|
||||
|
||||
var sbc *subBalancerWithConfig
|
||||
// If outgoingStarted is true, search in the cache. Otherwise, cache is
|
||||
// guaranteed to be empty, searching is unnecessary.
|
||||
if bg.outgoingStarted {
|
||||
// Only start the balancer if bg is started. Otherwise, we only keep the
|
||||
// static data.
|
||||
sbc.startBalancer(&balancerGroupCC{
|
||||
if old, ok := bg.balancerCache.Remove(id); ok {
|
||||
sbc, _ = old.(*subBalancerWithConfig)
|
||||
if sbc != nil && sbc.builder != builder {
|
||||
// If the sub-balancer in cache was built with a different
|
||||
// balancer builder, don't use it, cleanup this old-balancer,
|
||||
// and behave as sub-balancer is not found in cache.
|
||||
//
|
||||
// NOTE that this will also drop the cached addresses for this
|
||||
// sub-balancer, which seems to be reasonable.
|
||||
sbc.stopBalancer()
|
||||
// cleanupSubConns must be done before the new balancer starts,
|
||||
// otherwise new SubConns created by the new balancer might be
|
||||
// removed by mistake.
|
||||
bg.cleanupSubConns(sbc)
|
||||
sbc = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
if sbc == nil {
|
||||
sbc = &subBalancerWithConfig{
|
||||
ClientConn: bg.cc,
|
||||
id: id,
|
||||
group: bg,
|
||||
})
|
||||
builder: builder,
|
||||
}
|
||||
if bg.outgoingStarted {
|
||||
// Only start the balancer if bg is started. Otherwise, we only keep the
|
||||
// static data.
|
||||
sbc.startBalancer()
|
||||
}
|
||||
} else {
|
||||
// When brining back a sub-balancer from cache, re-send the cached
|
||||
// picker and state.
|
||||
sbc.updateBalancerStateWithCachedPicker()
|
||||
}
|
||||
bg.idToBalancerConfig[id] = sbc
|
||||
bg.outgoingMu.Unlock()
|
||||
}
|
||||
|
||||
// remove removes the balancer with id from the group, and closes the balancer.
|
||||
// remove removes the balancer with id from the group.
|
||||
//
|
||||
// But doesn't close the balancer. The balancer is kept in a cache, and will be
|
||||
// closed after timeout. Cleanup work (closing sub-balancer and removing
|
||||
// subconns) will be done after timeout.
|
||||
//
|
||||
// It also removes the picker generated from this balancer from the picker
|
||||
// group. It always results in a picker update.
|
||||
func (bg *balancerGroup) remove(id internal.Locality) {
|
||||
bg.outgoingMu.Lock()
|
||||
if bg.outgoingStarted {
|
||||
if config, ok := bg.idToBalancerConfig[id]; ok {
|
||||
config.stopBalancer()
|
||||
if sbToRemove, ok := bg.idToBalancerConfig[id]; ok {
|
||||
if bg.outgoingStarted {
|
||||
bg.balancerCache.Add(id, sbToRemove, func() {
|
||||
// After timeout, when sub-balancer is removed from cache, need
|
||||
// to close the underlying sub-balancer, and remove all its
|
||||
// subconns.
|
||||
bg.outgoingMu.Lock()
|
||||
if bg.outgoingStarted {
|
||||
sbToRemove.stopBalancer()
|
||||
}
|
||||
bg.outgoingMu.Unlock()
|
||||
bg.cleanupSubConns(sbToRemove)
|
||||
})
|
||||
}
|
||||
delete(bg.idToBalancerConfig, id)
|
||||
} else {
|
||||
grpclog.Infof("balancer group: trying to remove a non-existing locality from balancer group: %v", id)
|
||||
}
|
||||
delete(bg.idToBalancerConfig, id)
|
||||
bg.outgoingMu.Unlock()
|
||||
|
||||
bg.incomingMu.Lock()
|
||||
// Remove SubConns.
|
||||
//
|
||||
// NOTE: if NewSubConn is called by this (closed) balancer later, the
|
||||
// SubConn will be leaked. This shouldn't happen if the balancer
|
||||
// implementation is correct. To make sure this never happens, we need to
|
||||
// add another layer (balancer manager) between balancer group and the
|
||||
// sub-balancers.
|
||||
for sc, bid := range bg.scToID {
|
||||
if bid == id {
|
||||
bg.cc.RemoveSubConn(sc)
|
||||
delete(bg.scToID, sc)
|
||||
}
|
||||
}
|
||||
|
||||
// Remove id and picker from picker map. This also results in future updates
|
||||
// for this ID to be ignored.
|
||||
delete(bg.idToPickerState, id)
|
||||
|
@ -296,6 +366,27 @@ func (bg *balancerGroup) remove(id internal.Locality) {
|
|||
bg.incomingMu.Unlock()
|
||||
}
|
||||
|
||||
// bg.remove(id) doesn't do cleanup for the sub-balancer. This function does
|
||||
// cleanup after the timeout.
|
||||
func (bg *balancerGroup) cleanupSubConns(config *subBalancerWithConfig) {
|
||||
bg.incomingMu.Lock()
|
||||
// Remove SubConns. This is only done after the balancer is
|
||||
// actually closed.
|
||||
//
|
||||
// NOTE: if NewSubConn is called by this (closed) balancer later, the
|
||||
// SubConn will be leaked. This shouldn't happen if the balancer
|
||||
// implementation is correct. To make sure this never happens, we need to
|
||||
// add another layer (balancer manager) between balancer group and the
|
||||
// sub-balancers.
|
||||
for sc, b := range bg.scToSubBalancer {
|
||||
if b == config {
|
||||
bg.cc.RemoveSubConn(sc)
|
||||
delete(bg.scToSubBalancer, sc)
|
||||
}
|
||||
}
|
||||
bg.incomingMu.Unlock()
|
||||
}
|
||||
|
||||
// changeWeight changes the weight of the balancer.
|
||||
//
|
||||
// newWeight should never be zero.
|
||||
|
@ -332,21 +423,19 @@ func (bg *balancerGroup) changeWeight(id internal.Locality, newWeight uint32) {
|
|||
func (bg *balancerGroup) handleSubConnStateChange(sc balancer.SubConn, state connectivity.State) {
|
||||
grpclog.Infof("balancer group: handle subconn state change: %p, %v", sc, state)
|
||||
bg.incomingMu.Lock()
|
||||
id, ok := bg.scToID[sc]
|
||||
config, ok := bg.scToSubBalancer[sc]
|
||||
if !ok {
|
||||
bg.incomingMu.Unlock()
|
||||
return
|
||||
}
|
||||
if state == connectivity.Shutdown {
|
||||
// Only delete sc from the map when state changed to Shutdown.
|
||||
delete(bg.scToID, sc)
|
||||
delete(bg.scToSubBalancer, sc)
|
||||
}
|
||||
bg.incomingMu.Unlock()
|
||||
|
||||
bg.outgoingMu.Lock()
|
||||
if config, ok := bg.idToBalancerConfig[id]; ok {
|
||||
config.handleSubConnStateChange(sc, state)
|
||||
}
|
||||
config.handleSubConnStateChange(sc, state)
|
||||
bg.outgoingMu.Unlock()
|
||||
}
|
||||
|
||||
|
@ -373,20 +462,22 @@ func (bg *balancerGroup) handleResolvedAddrs(id internal.Locality, addrs []resol
|
|||
// from map. Delete sc from the map only when state changes to Shutdown. Since
|
||||
// it's just forwarding the action, there's no need for a removeSubConn()
|
||||
// wrapper function.
|
||||
func (bg *balancerGroup) newSubConn(id internal.Locality, addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) {
|
||||
bg.incomingMu.Lock()
|
||||
defer bg.incomingMu.Unlock()
|
||||
if !bg.incomingStarted {
|
||||
return nil, fmt.Errorf("NewSubConn is called after balancer is closed")
|
||||
}
|
||||
func (bg *balancerGroup) newSubConn(config *subBalancerWithConfig, addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) {
|
||||
// NOTE: if balancer with id was already removed, this should also return
|
||||
// error. But since we call balancer.stopBalancer when removing the balancer, this
|
||||
// shouldn't happen.
|
||||
bg.incomingMu.Lock()
|
||||
if !bg.incomingStarted {
|
||||
bg.incomingMu.Unlock()
|
||||
return nil, fmt.Errorf("NewSubConn is called after balancer group is closed")
|
||||
}
|
||||
sc, err := bg.cc.NewSubConn(addrs, opts)
|
||||
if err != nil {
|
||||
bg.incomingMu.Unlock()
|
||||
return nil, err
|
||||
}
|
||||
bg.scToID[sc] = id
|
||||
bg.scToSubBalancer[sc] = config
|
||||
bg.incomingMu.Unlock()
|
||||
return sc, nil
|
||||
}
|
||||
|
||||
|
@ -401,7 +492,7 @@ func (bg *balancerGroup) updateBalancerState(id internal.Locality, state connect
|
|||
if !ok {
|
||||
// All state starts in IDLE. If ID is not in map, it's either removed,
|
||||
// or never existed.
|
||||
grpclog.Infof("balancer group: pickerState not found when update picker/state")
|
||||
grpclog.Warningf("balancer group: pickerState for %v not found when update picker/state", id)
|
||||
return
|
||||
}
|
||||
pickerSt.picker = newLoadReportPicker(picker, id, bg.loadStore)
|
||||
|
@ -424,9 +515,9 @@ func (bg *balancerGroup) close() {
|
|||
}
|
||||
|
||||
// Also remove all SubConns.
|
||||
for sc := range bg.scToID {
|
||||
for sc := range bg.scToSubBalancer {
|
||||
bg.cc.RemoveSubConn(sc)
|
||||
delete(bg.scToID, sc)
|
||||
delete(bg.scToSubBalancer, sc)
|
||||
}
|
||||
}
|
||||
bg.incomingMu.Unlock()
|
||||
|
@ -439,6 +530,9 @@ func (bg *balancerGroup) close() {
|
|||
}
|
||||
}
|
||||
bg.outgoingMu.Unlock()
|
||||
// Clear(true) runs clear function to close sub-balancers in cache. It
|
||||
// must be called out of outgoing mutex.
|
||||
bg.balancerCache.Clear(true)
|
||||
}
|
||||
|
||||
func buildPickerAndState(m map[internal.Locality]*pickerState) (connectivity.State, balancer.Picker) {
|
||||
|
@ -547,22 +641,3 @@ func (lrp *loadReportPicker) Pick(ctx context.Context, opts balancer.PickOptions
|
|||
}
|
||||
return
|
||||
}
|
||||
|
||||
// balancerGroupCC implements the balancer.ClientConn API and get passed to each
|
||||
// sub-balancer. It contains the sub-balancer ID, so the parent balancer can
|
||||
// keep track of SubConn/pickers and the sub-balancers they belong to.
|
||||
//
|
||||
// Some of the actions are forwarded to the parent ClientConn with no change.
|
||||
// Some are forward to balancer group with the sub-balancer ID.
|
||||
type balancerGroupCC struct {
|
||||
balancer.ClientConn
|
||||
id internal.Locality
|
||||
group *balancerGroup
|
||||
}
|
||||
|
||||
func (bgcc *balancerGroupCC) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) {
|
||||
return bgcc.group.newSubConn(bgcc.id, addrs, opts)
|
||||
}
|
||||
func (bgcc *balancerGroupCC) UpdateBalancerState(state connectivity.State, picker balancer.Picker) {
|
||||
bgcc.group.updateBalancerState(bgcc.id, state, picker)
|
||||
}
|
||||
|
|
|
@ -18,8 +18,10 @@ package edsbalancer
|
|||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/balancer"
|
||||
"google.golang.org/grpc/balancer/roundrobin"
|
||||
|
@ -32,9 +34,21 @@ import (
|
|||
var (
|
||||
rrBuilder = balancer.Get(roundrobin.Name)
|
||||
testBalancerIDs = []internal.Locality{{Region: "b1"}, {Region: "b2"}, {Region: "b3"}}
|
||||
testBackendAddrs = []resolver.Address{{Addr: "1.1.1.1:1"}, {Addr: "2.2.2.2:2"}, {Addr: "3.3.3.3:3"}, {Addr: "4.4.4.4:4"}}
|
||||
testBackendAddrs []resolver.Address
|
||||
)
|
||||
|
||||
const testBackendAddrsCount = 8
|
||||
|
||||
func init() {
|
||||
for i := 0; i < testBackendAddrsCount; i++ {
|
||||
testBackendAddrs = append(testBackendAddrs, resolver.Address{Addr: fmt.Sprintf("%d.%d.%d.%d:%d", i, i, i, i, i)})
|
||||
}
|
||||
|
||||
// Disable caching for all tests. It will be re-enabled in caching specific
|
||||
// tests.
|
||||
defaultSubBalancerCloseTimeout = time.Millisecond
|
||||
}
|
||||
|
||||
// 1 balancer, 1 backend -> 2 backends -> 1 backend.
|
||||
func TestBalancerGroup_OneRR_AddRemoveBackend(t *testing.T) {
|
||||
cc := newTestClientConn(t)
|
||||
|
@ -568,3 +582,260 @@ func TestBalancerGroup_start_close_deadlock(t *testing.T) {
|
|||
|
||||
bg.start()
|
||||
}
|
||||
|
||||
func replaceDefaultSubBalancerCloseTimeout(n time.Duration) func() {
|
||||
old := defaultSubBalancerCloseTimeout
|
||||
defaultSubBalancerCloseTimeout = n
|
||||
return func() { defaultSubBalancerCloseTimeout = old }
|
||||
}
|
||||
|
||||
// initBalancerGroupForCachingTest creates a balancer group, and initialize it
|
||||
// to be ready for caching tests.
|
||||
//
|
||||
// Two rr balancers are added to bg, each with 2 ready subConns. A sub-balancer
|
||||
// is removed later, so the balancer group returned has one sub-balancer in its
|
||||
// own map, and one sub-balancer in cache.
|
||||
func initBalancerGroupForCachingTest(t *testing.T) (*balancerGroup, *testClientConn, map[resolver.Address]balancer.SubConn) {
|
||||
cc := newTestClientConn(t)
|
||||
bg := newBalancerGroup(cc, nil)
|
||||
|
||||
// Add two balancers to group and send two resolved addresses to both
|
||||
// balancers.
|
||||
bg.add(testBalancerIDs[0], 2, rrBuilder)
|
||||
bg.handleResolvedAddrs(testBalancerIDs[0], testBackendAddrs[0:2])
|
||||
bg.add(testBalancerIDs[1], 1, rrBuilder)
|
||||
bg.handleResolvedAddrs(testBalancerIDs[1], testBackendAddrs[2:4])
|
||||
|
||||
bg.start()
|
||||
|
||||
m1 := make(map[resolver.Address]balancer.SubConn)
|
||||
for i := 0; i < 4; i++ {
|
||||
addrs := <-cc.newSubConnAddrsCh
|
||||
sc := <-cc.newSubConnCh
|
||||
m1[addrs[0]] = sc
|
||||
bg.handleSubConnStateChange(sc, connectivity.Connecting)
|
||||
bg.handleSubConnStateChange(sc, connectivity.Ready)
|
||||
}
|
||||
|
||||
// Test roundrobin on the last picker.
|
||||
p1 := <-cc.newPickerCh
|
||||
want := []balancer.SubConn{
|
||||
m1[testBackendAddrs[0]], m1[testBackendAddrs[0]],
|
||||
m1[testBackendAddrs[1]], m1[testBackendAddrs[1]],
|
||||
m1[testBackendAddrs[2]], m1[testBackendAddrs[3]],
|
||||
}
|
||||
if err := isRoundRobin(want, func() balancer.SubConn {
|
||||
sc, _, _ := p1.Pick(context.Background(), balancer.PickOptions{})
|
||||
return sc
|
||||
}); err != nil {
|
||||
t.Fatalf("want %v, got %v", want, err)
|
||||
}
|
||||
|
||||
bg.remove(testBalancerIDs[1])
|
||||
// Don't wait for SubConns to be removed after close, because they are only
|
||||
// removed after close timeout.
|
||||
for i := 0; i < 10; i++ {
|
||||
select {
|
||||
case <-cc.removeSubConnCh:
|
||||
t.Fatalf("Got request to remove subconn, want no remove subconn (because subconns were still in cache)")
|
||||
default:
|
||||
}
|
||||
time.Sleep(time.Millisecond)
|
||||
}
|
||||
// Test roundrobin on the with only sub-balancer0.
|
||||
p2 := <-cc.newPickerCh
|
||||
want = []balancer.SubConn{
|
||||
m1[testBackendAddrs[0]], m1[testBackendAddrs[1]],
|
||||
}
|
||||
if err := isRoundRobin(want, func() balancer.SubConn {
|
||||
sc, _, _ := p2.Pick(context.Background(), balancer.PickOptions{})
|
||||
return sc
|
||||
}); err != nil {
|
||||
t.Fatalf("want %v, got %v", want, err)
|
||||
}
|
||||
|
||||
return bg, cc, m1
|
||||
}
|
||||
|
||||
// Test that if a sub-balancer is removed, and re-added within close timeout,
|
||||
// the subConns won't be re-created.
|
||||
func TestBalancerGroup_locality_caching(t *testing.T) {
|
||||
defer replaceDefaultSubBalancerCloseTimeout(10 * time.Second)()
|
||||
bg, cc, addrToSC := initBalancerGroupForCachingTest(t)
|
||||
|
||||
// Turn down subconn for addr2, shouldn't get picker update because
|
||||
// sub-balancer1 was removed.
|
||||
bg.handleSubConnStateChange(addrToSC[testBackendAddrs[2]], connectivity.TransientFailure)
|
||||
for i := 0; i < 10; i++ {
|
||||
select {
|
||||
case <-cc.newPickerCh:
|
||||
t.Fatalf("Got new picker, want no new picker (because the sub-balancer was removed)")
|
||||
default:
|
||||
}
|
||||
time.Sleep(time.Millisecond)
|
||||
}
|
||||
|
||||
// Sleep, but sleep less then close timeout.
|
||||
time.Sleep(time.Millisecond * 100)
|
||||
|
||||
// Re-add sub-balancer-1, because subconns were in cache, no new subconns
|
||||
// should be created. But a new picker will still be generated, with subconn
|
||||
// states update to date.
|
||||
bg.add(testBalancerIDs[1], 1, rrBuilder)
|
||||
|
||||
p3 := <-cc.newPickerCh
|
||||
want := []balancer.SubConn{
|
||||
addrToSC[testBackendAddrs[0]], addrToSC[testBackendAddrs[0]],
|
||||
addrToSC[testBackendAddrs[1]], addrToSC[testBackendAddrs[1]],
|
||||
// addr2 is down, b2 only has addr3 in READY state.
|
||||
addrToSC[testBackendAddrs[3]], addrToSC[testBackendAddrs[3]],
|
||||
}
|
||||
if err := isRoundRobin(want, func() balancer.SubConn {
|
||||
sc, _, _ := p3.Pick(context.Background(), balancer.PickOptions{})
|
||||
return sc
|
||||
}); err != nil {
|
||||
t.Fatalf("want %v, got %v", want, err)
|
||||
}
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
select {
|
||||
case <-cc.newSubConnAddrsCh:
|
||||
t.Fatalf("Got new subconn, want no new subconn (because subconns were still in cache)")
|
||||
default:
|
||||
}
|
||||
time.Sleep(time.Millisecond * 10)
|
||||
}
|
||||
}
|
||||
|
||||
// Sub-balancers are put in cache when they are removed. If balancer group is
|
||||
// closed within close timeout, all subconns should still be rmeoved
|
||||
// immediately.
|
||||
func TestBalancerGroup_locality_caching_close_group(t *testing.T) {
|
||||
defer replaceDefaultSubBalancerCloseTimeout(10 * time.Second)()
|
||||
bg, cc, addrToSC := initBalancerGroupForCachingTest(t)
|
||||
|
||||
bg.close()
|
||||
// The balancer group is closed. The subconns should be removed immediately.
|
||||
removeTimeout := time.After(time.Millisecond * 500)
|
||||
scToRemove := map[balancer.SubConn]int{
|
||||
addrToSC[testBackendAddrs[0]]: 1,
|
||||
addrToSC[testBackendAddrs[1]]: 1,
|
||||
addrToSC[testBackendAddrs[2]]: 1,
|
||||
addrToSC[testBackendAddrs[3]]: 1,
|
||||
}
|
||||
for i := 0; i < len(scToRemove); i++ {
|
||||
select {
|
||||
case sc := <-cc.removeSubConnCh:
|
||||
c := scToRemove[sc]
|
||||
if c == 0 {
|
||||
t.Fatalf("Got removeSubConn for %v when there's %d remove expected", sc, c)
|
||||
}
|
||||
scToRemove[sc] = c - 1
|
||||
case <-removeTimeout:
|
||||
t.Fatalf("timeout waiting for subConns (from balancer in cache) to be removed")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Sub-balancers in cache will be closed if not re-added within timeout, and
|
||||
// subConns will be removed.
|
||||
func TestBalancerGroup_locality_caching_not_readd_within_timeout(t *testing.T) {
|
||||
defer replaceDefaultSubBalancerCloseTimeout(time.Second)()
|
||||
_, cc, addrToSC := initBalancerGroupForCachingTest(t)
|
||||
|
||||
// The sub-balancer is not re-added withtin timeout. The subconns should be
|
||||
// removed.
|
||||
removeTimeout := time.After(defaultSubBalancerCloseTimeout)
|
||||
scToRemove := map[balancer.SubConn]int{
|
||||
addrToSC[testBackendAddrs[2]]: 1,
|
||||
addrToSC[testBackendAddrs[3]]: 1,
|
||||
}
|
||||
for i := 0; i < len(scToRemove); i++ {
|
||||
select {
|
||||
case sc := <-cc.removeSubConnCh:
|
||||
c := scToRemove[sc]
|
||||
if c == 0 {
|
||||
t.Fatalf("Got removeSubConn for %v when there's %d remove expected", sc, c)
|
||||
}
|
||||
scToRemove[sc] = c - 1
|
||||
case <-removeTimeout:
|
||||
t.Fatalf("timeout waiting for subConns (from balancer in cache) to be removed")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Wrap the rr builder, so it behaves the same, but has a different pointer.
|
||||
type noopBalancerBuilderWrapper struct {
|
||||
balancer.Builder
|
||||
}
|
||||
|
||||
// After removing a sub-balancer, re-add with same ID, but different balancer
|
||||
// builder. Old subconns should be removed, and new subconns should be created.
|
||||
func TestBalancerGroup_locality_caching_readd_with_different_builder(t *testing.T) {
|
||||
defer replaceDefaultSubBalancerCloseTimeout(10 * time.Second)()
|
||||
bg, cc, addrToSC := initBalancerGroupForCachingTest(t)
|
||||
|
||||
// Re-add sub-balancer-1, but with a different balancer builder. The
|
||||
// sub-balancer was still in cache, but cann't be reused. This should cause
|
||||
// old sub-balancer's subconns to be removed immediately, and new subconns
|
||||
// to be created.
|
||||
bg.add(testBalancerIDs[1], 1, &noopBalancerBuilderWrapper{rrBuilder})
|
||||
|
||||
// The cached sub-balancer should be closed, and the subconns should be
|
||||
// removed immediately.
|
||||
removeTimeout := time.After(time.Millisecond * 500)
|
||||
scToRemove := map[balancer.SubConn]int{
|
||||
addrToSC[testBackendAddrs[2]]: 1,
|
||||
addrToSC[testBackendAddrs[3]]: 1,
|
||||
}
|
||||
for i := 0; i < len(scToRemove); i++ {
|
||||
select {
|
||||
case sc := <-cc.removeSubConnCh:
|
||||
c := scToRemove[sc]
|
||||
if c == 0 {
|
||||
t.Fatalf("Got removeSubConn for %v when there's %d remove expected", sc, c)
|
||||
}
|
||||
scToRemove[sc] = c - 1
|
||||
case <-removeTimeout:
|
||||
t.Fatalf("timeout waiting for subConns (from balancer in cache) to be removed")
|
||||
}
|
||||
}
|
||||
|
||||
bg.handleResolvedAddrs(testBalancerIDs[1], testBackendAddrs[4:6])
|
||||
|
||||
newSCTimeout := time.After(time.Millisecond * 500)
|
||||
scToAdd := map[resolver.Address]int{
|
||||
testBackendAddrs[4]: 1,
|
||||
testBackendAddrs[5]: 1,
|
||||
}
|
||||
for i := 0; i < len(scToAdd); i++ {
|
||||
select {
|
||||
case addr := <-cc.newSubConnAddrsCh:
|
||||
c := scToAdd[addr[0]]
|
||||
if c == 0 {
|
||||
t.Fatalf("Got newSubConn for %v when there's %d new expected", addr, c)
|
||||
}
|
||||
scToAdd[addr[0]] = c - 1
|
||||
sc := <-cc.newSubConnCh
|
||||
addrToSC[addr[0]] = sc
|
||||
bg.handleSubConnStateChange(sc, connectivity.Connecting)
|
||||
bg.handleSubConnStateChange(sc, connectivity.Ready)
|
||||
case <-newSCTimeout:
|
||||
t.Fatalf("timeout waiting for subConns (from new sub-balancer) to be newed")
|
||||
}
|
||||
}
|
||||
|
||||
// Test roundrobin on the new picker.
|
||||
p3 := <-cc.newPickerCh
|
||||
want := []balancer.SubConn{
|
||||
addrToSC[testBackendAddrs[0]], addrToSC[testBackendAddrs[0]],
|
||||
addrToSC[testBackendAddrs[1]], addrToSC[testBackendAddrs[1]],
|
||||
addrToSC[testBackendAddrs[4]], addrToSC[testBackendAddrs[5]],
|
||||
}
|
||||
if err := isRoundRobin(want, func() balancer.SubConn {
|
||||
sc, _, _ := p3.Pick(context.Background(), balancer.PickOptions{})
|
||||
return sc
|
||||
}); err != nil {
|
||||
t.Fatalf("want %v, got %v", want, err)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -341,6 +341,17 @@ func TestEDS_TwoLocalities(t *testing.T) {
|
|||
clab6.addLocality(testSubZones[2], 1, testEndpointAddrs[2:4])
|
||||
edsb.HandleEDSResponse(clab6.build())
|
||||
|
||||
// Changing weight of locality[1] to 0 caused it to be removed. It's subconn
|
||||
// should also be removed.
|
||||
//
|
||||
// NOTE: this is because we handle locality with weight 0 same as the
|
||||
// locality doesn't exist. If this changes in the future, this removeSubConn
|
||||
// behavior will also change.
|
||||
scToRemove2 := <-cc.removeSubConnCh
|
||||
if !reflect.DeepEqual(scToRemove2, sc2) {
|
||||
t.Fatalf("RemoveSubConn, want %v, got %v", sc2, scToRemove2)
|
||||
}
|
||||
|
||||
// Test pick with two subconns different locality weight.
|
||||
p6 := <-cc.newPickerCh
|
||||
// Locality-1 will be not be picked, and locality-2 will be picked.
|
||||
|
|
Loading…
Reference in New Issue