-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
server: delete node status entries during decommissioning
Decommissioning a node did not remove its node status entry in the `status-node-` keyspace. This caused the decommissioned node to remain visible via e.g. `Status/Nodes` gRPC calls, affecting other systems that did not check against the node's liveness entry. This commit deletes a node's status entry when it is being decommissioned. Unfortunately, since the status entry is inline this cannot be transactional with the liveness entry, which can cause a stale status entry to be left behind if the operation should fail or the node should crash. In these cases, the decommission operation (which is idempotent) must be run again to remove it. To avoid a race condition where the decommissioned node's status recorder loop resurrects the status entry after it has been removed, this also changes the recorder to only update the status entry if it already exists. The initial status entry is written during node startup, and the node will now fail to start if the entry cannot be written. Release note (bug fix): remove a node's status entry when the node is decommissioned, to prevent it from appearing in API calls and UIs, and avoid it affecting node constraints such as localities and attributes for various operations.
- Loading branch information
1 parent
94465c4
commit 2ec4751
Showing
10 changed files
with
246 additions
and
116 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,71 @@ | ||
// Copyright 2016 The Cockroach Authors. | ||
// | ||
// Use of this software is governed by the Business Source License | ||
// included in the file licenses/BSL.txt. | ||
// | ||
// As of the Change Date specified in that file, in accordance with | ||
// the Business Source License, use of this software will be governed | ||
// by the Apache License, Version 2.0, included in the file | ||
// licenses/APL.txt. | ||
|
||
package server_test | ||
|
||
import ( | ||
"context" | ||
"testing" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/base" | ||
"github.com/cockroachdb/cockroach/pkg/keys" | ||
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" | ||
"github.com/cockroachdb/cockroach/pkg/roachpb" | ||
"github.com/cockroachdb/cockroach/pkg/testutils" | ||
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster" | ||
"github.com/cockroachdb/cockroach/pkg/util/leaktest" | ||
"github.com/cockroachdb/cockroach/pkg/util/log" | ||
"github.com/cockroachdb/errors" | ||
"github.com/stretchr/testify/require" | ||
) | ||
|
||
// This test should really be in server_test.go but can't because it uses the | ||
// server package, which is also imported by testutils/testcluster, causing | ||
// an import cycles. | ||
func TestDecommissionNodeStatus(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
defer log.Scope(t).Close(t) | ||
|
||
ctx := context.Background() | ||
tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ | ||
ReplicationMode: base.ReplicationManual, // saves time | ||
}) | ||
defer tc.Stopper().Stop(ctx) | ||
decomNodeID := tc.Server(2).NodeID() | ||
|
||
// Wait for node status entries to have been created. | ||
testutils.SucceedsSoon(t, func() error { | ||
for _, srv := range tc.Servers { | ||
entry, err := srv.DB().Get(ctx, keys.NodeStatusKey(srv.NodeID())) | ||
if err != nil { | ||
return err | ||
} else if entry.Value == nil { | ||
return errors.Errorf("expected to find node status entry for %d", srv.NodeID()) | ||
} | ||
} | ||
return nil | ||
}) | ||
|
||
require.NoError(t, tc.Servers[0].Decommission( | ||
ctx, livenesspb.MembershipStatus_DECOMMISSIONING, []roachpb.NodeID{decomNodeID})) | ||
require.NoError(t, tc.Servers[0].Decommission( | ||
ctx, livenesspb.MembershipStatus_DECOMMISSIONED, []roachpb.NodeID{decomNodeID})) | ||
|
||
// The node status entries should shortly be cleaned up. | ||
testutils.SucceedsSoon(t, func() error { | ||
entry, err := tc.Server(0).DB().Get(ctx, keys.NodeStatusKey(decomNodeID)) | ||
if err != nil { | ||
return err | ||
} else if entry.Value != nil { | ||
return errors.Errorf("found stale node status entry for %d", decomNodeID) | ||
} | ||
return nil | ||
}) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.