github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/integration/cluster_add_one_node_test.go (about)

     1  // +build integration
     2  
     3  // Copyright (c) 2016 Uber Technologies, Inc.
     4  //
     5  // Permission is hereby granted, free of charge, to any person obtaining a copy
     6  // of this software and associated documentation files (the "Software"), to deal
     7  // in the Software without restriction, including without limitation the rights
     8  // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
     9  // copies of the Software, and to permit persons to whom the Software is
    10  // furnished to do so, subject to the following conditions:
    11  //
    12  // The above copyright notice and this permission notice shall be included in
    13  // all copies or substantial portions of the Software.
    14  //
    15  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
    16  // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
    17  // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
    18  // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
    19  // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
    20  // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
    21  // THE SOFTWARE.
    22  
    23  package integration
    24  
    25  import (
    26  	"fmt"
    27  	"strconv"
    28  	"testing"
    29  	"time"
    30  
    31  	"github.com/m3db/m3/src/cluster/services"
    32  	"github.com/m3db/m3/src/cluster/shard"
    33  	"github.com/m3db/m3/src/dbnode/client"
    34  	"github.com/m3db/m3/src/dbnode/integration/fake"
    35  	"github.com/m3db/m3/src/dbnode/integration/generate"
    36  	"github.com/m3db/m3/src/dbnode/namespace"
    37  	"github.com/m3db/m3/src/dbnode/retention"
    38  	"github.com/m3db/m3/src/dbnode/topology"
    39  	"github.com/m3db/m3/src/dbnode/topology/testutil"
    40  	"github.com/m3db/m3/src/x/ident"
    41  	xtest "github.com/m3db/m3/src/x/test"
    42  	xtime "github.com/m3db/m3/src/x/time"
    43  
    44  	"github.com/stretchr/testify/require"
    45  )
    46  
    47  type idShard struct {
    48  	str   string
    49  	shard uint32
    50  }
    51  
    52  func TestClusterAddOneNode(t *testing.T) {
    53  	testClusterAddOneNode(t, false)
    54  }
    55  
    56  func testClusterAddOneNode(t *testing.T, verifyCommitlogCanBootstrapAfterNodeJoin bool) {
    57  	if testing.Short() {
    58  		t.SkipNow()
    59  	}
    60  
    61  	// Test setups.
    62  	log := xtest.NewLogger(t)
    63  	namesp, err := namespace.NewMetadata(testNamespaces[0],
    64  		namespace.NewOptions().SetRetentionOptions(
    65  			retention.NewOptions().
    66  				SetRetentionPeriod(6*time.Hour).
    67  				SetBlockSize(2*time.Hour).
    68  				SetBufferPast(10*time.Minute).
    69  				SetBufferFuture(2*time.Minute)))
    70  	require.NoError(t, err)
    71  	opts := NewTestOptions(t).
    72  		SetNamespaces([]namespace.Metadata{namesp}).
    73  		// Prevent snapshotting from happening too frequently to allow for the
    74  		// possibility of a snapshot occurring after the shard set is assigned,
    75  		// but not after the node finishes bootstrapping.
    76  		SetTickMinimumInterval(5 * time.Second)
    77  
    78  	minShard := uint32(0)
    79  	maxShard := uint32(opts.NumShards()) - uint32(1)
    80  	midShard := uint32((maxShard - minShard) / 2)
    81  
    82  	instances := struct {
    83  		start []services.ServiceInstance
    84  		add   []services.ServiceInstance
    85  		added []services.ServiceInstance
    86  	}{
    87  		start: []services.ServiceInstance{
    88  			node(t, 0, newClusterShardsRange(minShard, maxShard, shard.Available)),
    89  			node(t, 1, newClusterEmptyShardsRange()),
    90  		},
    91  
    92  		add: []services.ServiceInstance{
    93  			node(t, 0, concatShards(
    94  				newClusterShardsRange(minShard, midShard, shard.Available),
    95  				newClusterShardsRange(midShard+1, maxShard, shard.Leaving))),
    96  			node(t, 1, newClusterShardsRange(midShard+1, maxShard, shard.Initializing)),
    97  		},
    98  		added: []services.ServiceInstance{
    99  			node(t, 0, newClusterShardsRange(minShard, midShard, shard.Available)),
   100  			node(t, 1, newClusterShardsRange(midShard+1, maxShard, shard.Available)),
   101  		},
   102  	}
   103  
   104  	svc := fake.NewM3ClusterService().
   105  		SetInstances(instances.start).
   106  		SetReplication(services.NewServiceReplication().SetReplicas(1)).
   107  		SetSharding(services.NewServiceSharding().SetNumShards(opts.NumShards()))
   108  
   109  	svcs := fake.NewM3ClusterServices()
   110  	svcs.RegisterService("m3db", svc)
   111  
   112  	topoOpts := topology.NewDynamicOptions().
   113  		SetConfigServiceClient(fake.NewM3ClusterClient(svcs, nil))
   114  	topoInit := topology.NewDynamicInitializer(topoOpts)
   115  	setupOpts := []BootstrappableTestSetupOptions{
   116  		{
   117  			DisablePeersBootstrapper: true,
   118  			TopologyInitializer:      topoInit,
   119  		},
   120  		{
   121  			DisablePeersBootstrapper: false,
   122  			TopologyInitializer:      topoInit,
   123  		},
   124  	}
   125  	setups, closeFn := NewDefaultBootstrappableTestSetups(t, opts, setupOpts)
   126  	defer closeFn()
   127  
   128  	// Write test data for first node.
   129  	topo, err := topoInit.Init()
   130  	require.NoError(t, err)
   131  	ids := []idShard{}
   132  
   133  	// Boilerplate code to find two ID's that hash to the first half of the
   134  	// shards, and one ID that hashes to the second half of the shards.
   135  	shardSet := topo.Get().ShardSet()
   136  	i := 0
   137  	numFirstHalf := 0
   138  	numSecondHalf := 0
   139  	for {
   140  		if numFirstHalf == 2 && numSecondHalf == 1 {
   141  			break
   142  		}
   143  		idStr := strconv.Itoa(i)
   144  		shard := shardSet.Lookup(ident.StringID(idStr))
   145  		if shard < midShard && numFirstHalf < 2 {
   146  			ids = append(ids, idShard{str: idStr, shard: shard})
   147  			numFirstHalf++
   148  		}
   149  		if shard > midShard && numSecondHalf < 1 {
   150  			ids = append(ids, idShard{idStr, shard})
   151  			numSecondHalf++
   152  		}
   153  		i++
   154  	}
   155  
   156  	for _, id := range ids {
   157  		// Verify IDs will map to halves of the shard space.
   158  		require.Equal(t, id.shard, shardSet.Lookup(ident.StringID(id.str)))
   159  	}
   160  
   161  	var (
   162  		now        = setups[0].NowFn()()
   163  		blockStart = now
   164  		blockSize  = namesp.Options().RetentionOptions().BlockSize()
   165  		seriesMaps = generate.BlocksByStart([]generate.BlockConfig{
   166  			{IDs: []string{ids[0].str, ids[1].str}, NumPoints: 180, Start: blockStart.Add(-blockSize)},
   167  			{IDs: []string{ids[0].str, ids[2].str}, NumPoints: 90, Start: blockStart},
   168  		})
   169  	)
   170  	err = writeTestDataToDisk(namesp, setups[0], seriesMaps, 0)
   171  	require.NoError(t, err)
   172  
   173  	// Prepare verification of data on nodes.
   174  	expectedSeriesMaps := make([]map[xtime.UnixNano]generate.SeriesBlock, 2)
   175  	expectedSeriesIDs := make([]map[string]struct{}, 2)
   176  	for i := range expectedSeriesMaps {
   177  		expectedSeriesMaps[i] = make(map[xtime.UnixNano]generate.SeriesBlock)
   178  		expectedSeriesIDs[i] = make(map[string]struct{})
   179  	}
   180  	for start, series := range seriesMaps {
   181  		list := make([]generate.SeriesBlock, 2)
   182  		for j := range series {
   183  			if shardSet.Lookup(series[j].ID) < midShard+1 {
   184  				list[0] = append(list[0], series[j])
   185  			} else {
   186  				list[1] = append(list[1], series[j])
   187  			}
   188  		}
   189  		for i := range expectedSeriesMaps {
   190  			if len(list[i]) > 0 {
   191  				expectedSeriesMaps[i][start] = list[i]
   192  			}
   193  		}
   194  	}
   195  	for i := range expectedSeriesMaps {
   196  		for _, series := range expectedSeriesMaps[i] {
   197  			for _, elem := range series {
   198  				expectedSeriesIDs[i][elem.ID.String()] = struct{}{}
   199  			}
   200  		}
   201  	}
   202  	require.Equal(t, 2, len(expectedSeriesIDs[0]))
   203  	require.Equal(t, 1, len(expectedSeriesIDs[1]))
   204  
   205  	// Start the first server with filesystem bootstrapper.
   206  	require.NoError(t, setups[0].StartServer())
   207  
   208  	// Start the last server with peers and filesystem bootstrappers, no shards
   209  	// are assigned at first.
   210  	require.NoError(t, setups[1].StartServer())
   211  	log.Debug("servers are now up")
   212  
   213  	// Stop the servers on test completion.
   214  	defer func() {
   215  		setups.parallel(func(s TestSetup) {
   216  			require.NoError(t, s.StopServer())
   217  		})
   218  		log.Debug("servers are now down")
   219  	}()
   220  
   221  	// Bootstrap the new shards.
   222  	log.Debug("resharding to initialize shards on second node")
   223  	svc.SetInstances(instances.add)
   224  	svcs.NotifyServiceUpdate("m3db")
   225  	go func() {
   226  		for {
   227  			time.Sleep(time.Second)
   228  			for _, setup := range setups {
   229  				now = now.Add(time.Second)
   230  				setup.SetNowFn(now)
   231  			}
   232  		}
   233  	}()
   234  
   235  	// Generate some new data that will be written to the node while peer streaming is taking place
   236  	// to make sure that the data that is streamed in and the data that is received while streaming
   237  	// is going on are both handled correctly. In addition, this will ensure that we hold onto both
   238  	// sets of data durably after topology changes and that the node can be properly bootstrapped
   239  	// from just the filesystem and commitlog in a later portion of the test.
   240  	seriesToWriteDuringPeerStreaming := []string{
   241  		"series_after_bootstrap1",
   242  		"series_after_bootstrap2",
   243  	}
   244  	// Ensure that the new series belong that we're going to write belong to the host that is peer
   245  	// streaming data.
   246  	for _, seriesName := range seriesToWriteDuringPeerStreaming {
   247  		shard := shardSet.Lookup(ident.StringID(seriesName))
   248  		require.True(t, shard > midShard,
   249  			fmt.Sprintf("series: %s does not shard to second host", seriesName))
   250  	}
   251  	seriesReceivedDuringPeerStreaming := generate.BlocksByStart([]generate.BlockConfig{
   252  		{IDs: seriesToWriteDuringPeerStreaming, NumPoints: 90, Start: blockStart},
   253  	})
   254  	// Merge the newly generated series into the expected series map.
   255  	for blockStart, series := range seriesReceivedDuringPeerStreaming {
   256  		expectedSeriesMaps[1][blockStart] = append(expectedSeriesMaps[1][blockStart], series...)
   257  	}
   258  
   259  	// Spin up a background goroutine to issue the writes to the node while its streaming data
   260  	// from its peer.
   261  	doneWritingWhilePeerStreaming := make(chan struct{})
   262  	go func() {
   263  		for _, testData := range seriesReceivedDuringPeerStreaming {
   264  			err := setups[1].WriteBatch(namesp.ID(), testData)
   265  			// We expect consistency errors because we're only running with
   266  			// R.F = 2 and one node is leaving and one node is joining for
   267  			// each of the shards that is changing hands.
   268  			if !client.IsConsistencyResultError(err) {
   269  				panic(err)
   270  			}
   271  		}
   272  		doneWritingWhilePeerStreaming <- struct{}{}
   273  	}()
   274  
   275  	log.Debug("waiting for shards to be bootstrapped")
   276  	waitUntilHasBootstrappedShardsExactly(setups[1].DB(), testutil.Uint32Range(midShard+1, maxShard))
   277  
   278  	log.Debug("waiting for background writes to complete")
   279  	<-doneWritingWhilePeerStreaming
   280  
   281  	log.Debug("waiting for shards to be marked initialized")
   282  
   283  	allMarkedAvailable := func(
   284  		fakePlacementService fake.M3ClusterPlacementService,
   285  		instanceID string,
   286  		shards []shard.Shard,
   287  	) bool {
   288  		markedAvailable := fakePlacementService.InstanceShardsMarkedAvailable()
   289  		if len(markedAvailable) != 1 {
   290  			return false
   291  		}
   292  		if len(markedAvailable[instanceID]) != len(shards) {
   293  			return false
   294  		}
   295  		marked := shard.NewShards(nil)
   296  		for _, id := range markedAvailable[instanceID] {
   297  			marked.Add(shard.NewShard(id).SetState(shard.Available))
   298  		}
   299  		for _, shard := range shards {
   300  			if !marked.Contains(shard.ID()) {
   301  				return false
   302  			}
   303  		}
   304  		return true
   305  	}
   306  
   307  	fps := svcs.FakePlacementService()
   308  	shouldMark := instances.add[1].Shards().All()
   309  	for !allMarkedAvailable(fps, "testhost1", shouldMark) {
   310  		time.Sleep(100 * time.Millisecond)
   311  	}
   312  	log.Debug("all shards marked as initialized")
   313  
   314  	// Shed the old shards from the first node
   315  	log.Debug("resharding to shed shards from first node")
   316  	svc.SetInstances(instances.added)
   317  	svcs.NotifyServiceUpdate("m3db")
   318  	waitUntilHasBootstrappedShardsExactly(setups[0].DB(), testutil.Uint32Range(minShard, midShard))
   319  	waitUntilHasBootstrappedShardsExactly(setups[1].DB(), testutil.Uint32Range(midShard+1, maxShard))
   320  
   321  	log.Debug("verifying data in servers matches expected data set")
   322  
   323  	// Verify in-memory data match what we expect
   324  	for i := range setups {
   325  		verifySeriesMaps(t, setups[i], namesp.ID(), expectedSeriesMaps[i])
   326  	}
   327  
   328  	if verifyCommitlogCanBootstrapAfterNodeJoin {
   329  		// Verify that the node that joined the cluster can immediately bootstrap
   330  		// the data it streamed from its peers from the commitlog / snapshots as
   331  		// soon as all the shards have been marked as available (I.E as soon as
   332  		// when the placement change is considered "complete".)
   333  		//
   334  		// In addition, verify that any data that was received during the same block
   335  		// as the streamed data (I.E while peer streaming) is also present and
   336  		// bootstrappable from the commitlog bootstrapper.
   337  
   338  		// Reset the topology initializer as the M3DB session will have closed it.
   339  		require.NoError(t, setups[1].StopServer())
   340  		topoOpts := topology.NewDynamicOptions().
   341  			SetConfigServiceClient(fake.NewM3ClusterClient(svcs, nil))
   342  		topoInit := topology.NewDynamicInitializer(topoOpts)
   343  		setups[1].SetTopologyInitializer(topoInit)
   344  
   345  		// Start the server that performed peer streaming with only the filesystem and
   346  		// commitlog bootstrapper and make sure it has all the expected data.
   347  		startServerWithNewInspection(t, opts, setups[1])
   348  		verifySeriesMaps(t, setups[1], namesp.ID(), expectedSeriesMaps[1])
   349  	}
   350  }