Skip to content

Commit 6b4e13f

Browse files
craig[bot]wenyihu6shreyaskm623alyshanjahani-crlkyle-a-wong
committed
148476: kvserver: update store pool for leasequeue.process r=sumeerbhola,tbg a=wenyihu6 Previously, the lease queue didn’t update the local store pool correctly because `AllocationTransferLeaseOp.ApplyImpact` was unimplemented. This wasn’t an issue for the replicate queue, as it updates the local store pool directly through `rq.TransferLease` by calling `UpdateLocalStoresAfterLeaseTransfer`. This patch addresses the issue by calling `UpdateLocalStoresAfterLeaseTransfer` after `replica.AdminTransferLease` inside the lease queue. This fixes a bug where the allocator may make rebalancing decisions based on stale store pool data, failing to account for recent lease transfers not yet reflected in gossip. This state should eventually be corrected when real store capacity is measured and gossiped, and store pools are refreshed. This bug was found when reading the code. Epic: none Release note (bug fixes): Fixed a bug present since v24.1 where the allocator could make rebalancing decisions based on stale data, failing to account for recent local lease transfers not yet reflected in store capacity or gossip. 151110: k8s: fix python3 compatibility issue for multi-region setup r=NishanthNalluri a=shreyaskm623 Previously , the setup.py file was used for the kubernetes cluster setup across the multiple regions , which worked only for the python2 versions , this fix will make it compatible for the python3 versions as well Epic: none Fixes: https://cockroachlabs.atlassian.net/browse/CRDB-48938 Release note : None 151163: roachtest: bump min version for db-console/mixed-version-endpoints roachtest r=alyshanjahani-crl a=alyshanjahani-crl Previously the mixed version roachtest would fail whenever test plans included versions of crdb < 24.3. This was because a handful of /api/v2 endpoints were added in 24.3 (and not backported), and so they would return 404s in the test on versions prior. This commit bumps the min version to 24.3.0. Fixes: #150601, #148875, #149866 Release note: None 151218: integration: fix flaky TestInsightsIntegrationForContention r=kyle-a-wong a=kyle-a-wong This test has been seen to flake recently, due to the query on crdb_internal.node_transactions not returning any rows. To fix, it is now run within a testutils.SucceedSoon and will retry until it exists. Fixes: #151125, #150555, #150139 Release note: None Co-authored-by: wenyihu6 <[email protected]> Co-authored-by: Shreyas KM <[email protected]> Co-authored-by: Alyshan Jahani <[email protected]> Co-authored-by: Kyle Wong <[email protected]>
5 parents f448767 + a1eab56 + 5d43361 + 2bce287 + d07fe80 commit 6b4e13f

File tree

5 files changed

+30
-14
lines changed

5 files changed

+30
-14
lines changed

cloud/kubernetes/multiregion/setup.py

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,6 @@
66
# included in the /LICENSE file.
77

88

9-
import distutils.spawn
109
import json
1110
import os
1211
from subprocess import check_call,check_output
@@ -116,12 +115,12 @@
116115
for zone, context in contexts.items():
117116
external_ip = ''
118117
while True:
119-
external_ip = check_output(['kubectl', 'get', 'svc', 'kube-dns-lb', '--namespace', 'kube-system', '--context', context, '--template', '{{range .status.loadBalancer.ingress}}{{.ip}}{{end}}'])
118+
external_ip = check_output(['kubectl', 'get', 'svc', 'kube-dns-lb', '--namespace', 'kube-system', '--context', context, '--template', '{{range .status.loadBalancer.ingress}}{{.ip}}{{end}}']).decode('utf-8')
120119
if external_ip:
121120
break
122-
print 'Waiting for DNS load balancer IP in %s...' % (zone)
121+
print('Waiting for DNS load balancer IP in %s...' % (zone))
123122
sleep(10)
124-
print 'DNS endpoint for zone %s: %s' % (zone, external_ip)
123+
print('DNS endpoint for zone %s: %s' % (zone, external_ip))
125124
dns_ips[zone] = external_ip
126125

127126
# Update each cluster's DNS configuration with an appropriate configmap. Note
@@ -175,7 +174,7 @@
175174
check_call(['kubectl', 'apply', '-f', yaml_file, '--namespace', zone, '--context', context])
176175

177176
# Finally, initialize the cluster.
178-
print 'Sleeping 30 seconds before attempting to initialize cluster to give time for volumes to be created and pods started.'
177+
print('Sleeping 30 seconds before attempting to initialize cluster to give time for volumes to be created and pods started.')
179178
sleep(30)
180179
for zone, context in contexts.items():
181180
check_call(['kubectl', 'create', '-f', 'cluster-init-secure.yaml', '--namespace', zone, '--context', context])

pkg/cmd/roachtest/tests/db_console_endpoints.go

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -110,10 +110,8 @@ func registerDBConsoleEndpointsMixedVersion(r registry.Registry) {
110110
func runDBConsoleMixedVersion(ctx context.Context, t test.Test, c cluster.Cluster) {
111111
mvt := mixedversion.NewTest(ctx, t, t.L(), c,
112112
c.CRDBNodes(),
113-
// We test only upgrades from 23.2 in this test because it uses
114-
// the `workload init` command, which is only supported
115-
// reliably multi-tenant mode starting from that version.
116-
mixedversion.MinimumSupportedVersion("v23.2.0"),
113+
// In 24.3 new endpoints were added to /api/v2 server.
114+
mixedversion.MinimumSupportedVersion("v24.3.0"),
117115
)
118116

119117
mvt.InMixedVersion("test db console endpoints", func(ctx context.Context, l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error {

pkg/kv/kvserver/lease_queue.go

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -139,7 +139,15 @@ func (lq *leaseQueue) process(
139139
if err := repl.AdminTransferLease(ctx, transferOp.Target.StoreID, false /* bypassSafetyChecks */); err != nil {
140140
return false, errors.Wrapf(err, "%s: unable to transfer lease to s%d", repl, transferOp.Target)
141141
}
142-
change.Op.ApplyImpact(lq.storePool)
142+
143+
// TODO(wenyihu6): Initially, change.Op.ApplyImpact was used here. This was
144+
// a problem since AllocationTransferLeaseOp.ApplyImpact was left
145+
// unimplemented. We should either implement
146+
// AllocationTransferLeaseOp.ApplyImpact correctly or remove the use of
147+
// ApplyImpact entirely. The replicate queue does not have this issue since
148+
// it uses rq.TransferLease, which updates the local store pool directly.
149+
lq.storePool.UpdateLocalStoresAfterLeaseTransfer(
150+
transferOp.Source.StoreID, transferOp.Target.StoreID, transferOp.Usage)
143151
}
144152

145153
return true, nil

pkg/sql/sqlstats/insights/integration/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ go_test(
2626
"//pkg/util/log",
2727
"//pkg/util/timeutil",
2828
"//pkg/util/uuid",
29+
"@com_github_cockroachdb_errors//:errors",
2930
"@com_github_stretchr_testify//require",
3031
],
3132
)

pkg/sql/sqlstats/insights/integration/insights_test.go

Lines changed: 14 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@ import (
3838
"github.com/cockroachdb/cockroach/pkg/util/log"
3939
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
4040
"github.com/cockroachdb/cockroach/pkg/util/uuid"
41+
"github.com/cockroachdb/errors"
4142
"github.com/stretchr/testify/require"
4243
)
4344

@@ -803,7 +804,6 @@ func TestInsightsIntegrationForContention(t *testing.T) {
803804
// Chan to wait for the txn to complete to avoid checking for insights before the txn is committed.
804805
txnDoneChan := make(chan struct{})
805806

806-
observerConn := sqlutils.MakeSQLRunner(tc.ApplicationLayer(0).SQLConn(t))
807807
txConn := sqlutils.MakeSQLRunner(tc.ApplicationLayer(0).SQLConn(t))
808808
tx := txConn.Begin(t)
809809

@@ -828,10 +828,20 @@ func TestInsightsIntegrationForContention(t *testing.T) {
828828
require.NoError(t, errTxn)
829829

830830
var waitingTxnID uuid.UUID
831-
observerConn.QueryRow(t,
832-
`SELECT id
831+
testutils.SucceedsSoon(t, func() error {
832+
r, err := tc.ApplicationLayer(0).SQLConn(t).Query(`SELECT id
833833
FROM crdb_internal.node_transactions
834-
WHERE application_name = 'waiting_txn'`).Scan(&waitingTxnID)
834+
WHERE application_name = 'waiting_txn'`)
835+
require.NoError(t, err)
836+
defer r.Close()
837+
838+
if !r.Next() {
839+
return errors.New("waiting_txn not found")
840+
}
841+
err = r.Scan(&waitingTxnID)
842+
require.NoError(t, err)
843+
return nil
844+
})
835845

836846
require.NoError(t, tx.Commit())
837847

0 commit comments

Comments
 (0)