Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
59196: ui: show correct zone config r=dhartunian a=nkodali

Previously on the table detail page in DB Console,
the default zone config for the db was shown, rather
than any table level settings. Additionally,
constraints and lease preferences were not being
serialized properly, displaying as
"...Object object..." This fixes both of those bugs.

The display for the zone configuration statement is
additionally updated to show the actual SQL statement
to replicate the zone config in the SQL shell.
Previously an invalid statement "CONFIGURE ZONE USING..."
was displayed.

Resolves #57896.
See also: https://github.com/cockroachlabs/support/issues/737.
See also: https://github.com/cockroachlabs/support/issues/727.

Release note (bug fix): Fixed a bug introduced in v20.1 in
DB Console where incorrect zone configuration values were
shown on the table details page and constraints and lease
preferences were not displayed.

Release note (ui change): Updates the table details page
to show table specific zone configuration values when set,
show constraints and lease preferences, and display
a valid statement to re-configure zone configuration
for that table.

59265: kvserver: cleanup store.go to remove functions exposed because cfg was private r=tbg a=lunevalex

A lot of tests required things from StoreConfig, which was previously
private on the Store. Now that #59059 exposed the StoreConfig,
we can clean up some of this and remove the extra getters added
to extract things from the config for example NodeLiveness, Raft settings etc.

Release note: None

59280: kvserver: replace multiTestContext with TestCluster/TestServer in node_liveness_test r=tbg a=lunevalex

Makes progress on #8299

multiTestContext is a legacy construct that is deprecated in favor of running
tests via TestCluster. This is one PR out of many to remove the usage of
multiTestContext in the node_liveness_test test cases.

Release note: None

Co-authored-by: Namrata Kodali <[email protected]>
Co-authored-by: Alex Lunev <[email protected]>
  • Loading branch information
3 people committed Jan 28, 2021
4 parents bfe532e + c40ec31 + e4c8933 + 0b01151 commit eddbc07
Show file tree
Hide file tree
Showing 15 changed files with 930 additions and 690 deletions.
1 change: 1 addition & 0 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -2893,6 +2893,7 @@ a table.
| zone_config | [cockroach.config.zonepb.ZoneConfig](#cockroach.server.serverpb.TableDetailsResponse-cockroach.config.zonepb.ZoneConfig) | | The zone configuration in effect for this table. | [reserved](#support-status) |
| zone_config_level | [ZoneConfigurationLevel](#cockroach.server.serverpb.TableDetailsResponse-cockroach.server.serverpb.ZoneConfigurationLevel) | | The level at which this object's zone configuration is set. | [reserved](#support-status) |
| descriptor_id | [int64](#cockroach.server.serverpb.TableDetailsResponse-int64) | | descriptor_id is an identifier used to uniquely identify this table. It can be used to find events pertaining to this table by filtering on the 'target_id' field of events. | [reserved](#support-status) |
| configure_zone_statement | [string](#cockroach.server.serverpb.TableDetailsResponse-string) | | configure_zone_statement is the output of "SHOW ZONE CONFIGURATION FOR TABLE" for this table. It is a SQL statement that would re-configure the table's current zone if executed. | [reserved](#support-status) |



Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ func TestGossipNodeLivenessOnLeaseChange(t *testing.T) {
// liveness are not triggering gossiping.
for _, s := range tc.Servers {
pErr := s.Stores().VisitStores(func(store *kvserver.Store) error {
store.NodeLiveness().PauseHeartbeatLoopForTest()
store.GetStoreConfig().NodeLiveness.PauseHeartbeatLoopForTest()
return nil
})
if pErr != nil {
Expand Down
12 changes: 6 additions & 6 deletions pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1152,7 +1152,7 @@ func TestRequestsOnLaggingReplica(t *testing.T) {

// Stop the heartbeats so that n1's lease can expire.
log.Infof(ctx, "test: suspending heartbeats for n1")
resumeN1Heartbeats := partitionStore.NodeLiveness().PauseAllHeartbeatsForTest()
resumeN1Heartbeats := partitionStore.GetStoreConfig().NodeLiveness.PauseAllHeartbeatsForTest()

// Wait until another replica campaigns and becomes leader, replacing the
// partitioned one.
Expand Down Expand Up @@ -2456,7 +2456,7 @@ func TestRaftHeartbeats(t *testing.T) {
store := tc.GetFirstStoreFromServer(t, int(leaderRepl.StoreID()-1))

// Wait for several ticks to elapse.
ticksToWait := 2 * store.RaftElectionTimeoutTicks()
ticksToWait := 2 * store.GetStoreConfig().RaftElectionTimeoutTicks
ticks := store.Metrics().RaftTicks.Count
for targetTicks := ticks() + int64(ticksToWait); ticks() < targetTicks; {
time.Sleep(time.Millisecond)
Expand Down Expand Up @@ -2509,7 +2509,7 @@ func TestReportUnreachableHeartbeats(t *testing.T) {
t.Fatal(err)
}

ticksToWait := 2 * leaderStore.RaftElectionTimeoutTicks()
ticksToWait := 2 * leaderStore.GetStoreConfig().RaftElectionTimeoutTicks
ticks := leaderStore.Metrics().RaftTicks.Count
for targetTicks := ticks() + int64(ticksToWait); ticks() < targetTicks; {
time.Sleep(time.Millisecond)
Expand Down Expand Up @@ -2564,7 +2564,7 @@ outer:
tc.RemoveVotersOrFatal(t, key, tc.Target(leaderIdx))
cb := tc.Servers[replicaIdx].RaftTransport().GetCircuitBreaker(tc.Target(replicaIdx).NodeID, rpc.DefaultClass)
cb.Break()
time.Sleep(tc.GetFirstStoreFromServer(t, replicaIdx).CoalescedHeartbeatsInterval())
time.Sleep(tc.GetFirstStoreFromServer(t, replicaIdx).GetStoreConfig().CoalescedHeartbeatsInterval)
cb.Reset()
tc.AddVotersOrFatal(t, key, tc.Target(leaderIdx))
continue outer
Expand Down Expand Up @@ -4034,7 +4034,7 @@ func TestRangeQuiescence(t *testing.T) {
})
defer tc.Stopper().Stop(ctx)

pauseNodeLivenessHeartbeatLoopsTC(tc)
pauseNodeLivenessHeartbeatLoops(tc)
key := tc.ScratchRange(t)
tc.AddVotersOrFatal(t, key, tc.Targets(1, 2)...)

Expand Down Expand Up @@ -4065,7 +4065,7 @@ func TestRangeQuiescence(t *testing.T) {
// Wait for a bunch of ticks to occur which will allow the follower time to
// campaign.
ticks := tc.GetFirstStoreFromServer(t, followerIdx).Metrics().RaftTicks.Count
for targetTicks := ticks() + int64(2*tc.GetFirstStoreFromServer(t, followerIdx).RaftElectionTimeoutTicks()); ticks() < targetTicks; {
for targetTicks := ticks() + int64(2*tc.GetFirstStoreFromServer(t, followerIdx).GetStoreConfig().RaftElectionTimeoutTicks); ticks() < targetTicks; {
time.Sleep(time.Millisecond)
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/closed_timestamp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -692,10 +692,10 @@ func forceLeaseTransferOnSubsumedRange(
}
return nil
})
restartHeartbeats := oldLeaseholderStore.NodeLiveness().PauseAllHeartbeatsForTest()
restartHeartbeats := oldLeaseholderStore.GetStoreConfig().NodeLiveness.PauseAllHeartbeatsForTest()
defer restartHeartbeats()
log.Infof(ctx, "test: paused RHS rightLeaseholder's liveness heartbeats")
time.Sleep(oldLeaseholderStore.NodeLiveness().GetLivenessThreshold())
time.Sleep(oldLeaseholderStore.GetStoreConfig().NodeLiveness.GetLivenessThreshold())

// Send a read request from one of the followers of RHS so that it notices
// that the current rightLeaseholder has stopped heartbeating. This will prompt
Expand Down Expand Up @@ -955,7 +955,7 @@ func getCurrentMaxClosed(
attempts := 0
for attempts == 0 || timeutil.Now().Before(deadline) {
attempts++
store.ClosedTimestamp().Storage.VisitDescending(target.NodeID, func(entry ctpb.Entry) (done bool) {
store.GetStoreConfig().ClosedTimestamp.Storage.VisitDescending(target.NodeID, func(entry ctpb.Entry) (done bool) {
if _, ok := entry.MLAI[desc.RangeID]; ok {
maxClosed = entry
return true
Expand Down
Loading

0 comments on commit eddbc07

Please sign in to comment.