-
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.
Browse files
Browse the repository at this point in the history
116013: sql,opt: add hint for STRAIGHT join r=mgartner a=kevinmingtarja Our current join hints (a INNER HASH JOIN b, a LEFT LOOKUP JOIN b, etc) fixes both the join order and the join algorithm. This commit adds the syntax and support for hinting the join order without hinting the join algorithm. This will be useful for the (few) cases where the optimizer processes the tables in a suboptimal order. Resolves: #115308 Release note (sql change): It is now possible to hint to the optimizer that it should plan a straight join by using the syntax `... INNER STRAIGHT JOIN ...`. If the hint is provided, the optimizer will now fix the join order as given in the query, even if it estimates that a different plan using join reordering would have a lower cost. 117743: schemaworkload: Add metrics that keep track of workload operations r=fqazi a=rimadeodhar This PR adds metrics that indicate operations performed by the schema change workload. These metrics are output in logs at the end of a workload run. These metrics will function as a sanity check to ensure that the workload is doing something meaningful and will need to be checked manually. The current set of metrics are fairly rudimentary but will help provide some initial insight into the efficacy of this workload. Fixes: #81928 Epic: none Release note: None 118001: spanconfig: correctly order updates when applying in the KVSubscriber r=arulajmani a=arulajmani Previously, there was no ordering guarantee between KVSubscriber events at the same timestamp. As a result, if a batch of events included updates to overlapping spans at the same timestamp, we could apply additions before deletions -- this would cause the additions to get clobbered, which was not the intention. This could lead to missing span configurations, resulting in bugs such as the linked issue. This patch fixes the issue by sorting deletions before additions if two span configuration events have the same timestamp. Closes #110908 Release note (bug fix): Previously, altering from a Regional By Row table to a Regional By Table table could cause leaseholders to never move to the databse's primary region. This is now fixed. 118687: storage: check for replicated locks in MVCCClearTimeRange r=nvanbenschoten a=nvanbenschoten Fixes #113116. Previously, intents would be detected and returned in a `LockConflictError` from `MVCCClearTimeRange`. However, replicated locks with other strengths (shared, exclusive) would not. This commit fixes this, by calling ScanLocks at the beginning of the function. In doing so, it also adjusts how `MVCCClearTimeRange` handles intents. Previously, the function would only return intents whose current timestamp was in the clearing time range. This was probably fine for intents at higher timestamps than the clearing time range. However, it is error-prone for intents at lower timestamps, as those could have later been resolved to a higher timestamp, into the cleared time range. In practice, this probably did not matter, because we only issue MVCCClearTimeRanges on offline keyspaces. For simplicity, we now return conflicts and kick off resolution for locks of any strength at any timestamp. Release note: None 118746: cli: fix TestUnavailableZip failed r=koorosh a=koorosh This patch fixes issue introduced in da0e542 where code (that handles unparsed entries due to errors) was added in "if error" block instead of below it. It was wrong because in case of timeout error, `entries` variable is always nil. Now logic that handles partially failed entries parsing moved outside and invoked only if no timeout error happens. Release note: None Fixes: #117411 118758: build: ignore nogo rule `loopvarcapture` in external deps r=jayshrivastava a=jayshrivastava This change updates the `loopvarcapture` nogo check to only check first-party code. Release note: None Epic: None 118770: bincheck: verbose mode r=celiala a=rail This PR adds the -x flag to make the preparation steps visible in the logs. Epic: none Release note: None Co-authored-by: Kevin Mingtarja <[email protected]> Co-authored-by: rimadeodhar <[email protected]> Co-authored-by: Arul Ajmani <[email protected]> Co-authored-by: Nathan VanBenschoten <[email protected]> Co-authored-by: Andrii Vorobiov <[email protected]> Co-authored-by: Jayant Shrivastava <[email protected]> Co-authored-by: Rail Aliiev <[email protected]>
- Loading branch information
Showing
36 changed files
with
1,010 additions
and
96 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
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,6 +1,6 @@ | ||
#!/usr/bin/env bash | ||
|
||
set -euo pipefail | ||
set -exuo pipefail | ||
|
||
download_and_extract() { | ||
cockroach_version=$1 | ||
|
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 |
---|---|---|
@@ -1,6 +1,6 @@ | ||
#!/usr/bin/env bash | ||
|
||
set -euo pipefail | ||
set -exuo pipefail | ||
source ./download_binary.sh | ||
|
||
if [[ $# -ne 2 ]] | ||
|
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 |
---|---|---|
@@ -1,6 +1,6 @@ | ||
#!/usr/bin/env bash | ||
|
||
set -euo pipefail | ||
set -exuo pipefail | ||
source ./download_binary.sh | ||
|
||
# Verify arguments. | ||
|
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 |
---|---|---|
@@ -1,6 +1,6 @@ | ||
#!/usr/bin/env bash | ||
|
||
set -euo pipefail | ||
set -exuo pipefail | ||
source ./download_binary.sh | ||
|
||
# Verify arguments. | ||
|
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 |
---|---|---|
@@ -1,6 +1,6 @@ | ||
#!/usr/bin/env bash | ||
|
||
set -euo pipefail | ||
set -exuo pipefail | ||
source ./download_binary.sh | ||
|
||
if [[ $# -ne 2 ]] | ||
|
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
26 changes: 26 additions & 0 deletions
26
pkg/ccl/spanconfigccl/spanconfigkvsubscriberccl/BUILD.bazel
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,26 @@ | ||
load("@io_bazel_rules_go//go:def.bzl", "go_test") | ||
|
||
go_test( | ||
name = "spanconfigkvsubscriberccl_test", | ||
srcs = [ | ||
"kvsubscriber_test.go", | ||
"main_test.go", | ||
], | ||
deps = [ | ||
"//pkg/base", | ||
"//pkg/ccl", | ||
"//pkg/ccl/multiregionccl/multiregionccltestutils", | ||
"//pkg/jobs", | ||
"//pkg/kv/kvserver", | ||
"//pkg/security/securityassets", | ||
"//pkg/security/securitytest", | ||
"//pkg/server", | ||
"//pkg/testutils", | ||
"//pkg/testutils/serverutils", | ||
"//pkg/testutils/sqlutils", | ||
"//pkg/testutils/testcluster", | ||
"//pkg/util/leaktest", | ||
"//pkg/util/log", | ||
"//pkg/util/randutil", | ||
], | ||
) |
79 changes: 79 additions & 0 deletions
79
pkg/ccl/spanconfigccl/spanconfigkvsubscriberccl/kvsubscriber_test.go
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,79 @@ | ||
// Copyright 2024 The Cockroach Authors. | ||
// | ||
// Licensed as a CockroachDB Enterprise file under the Cockroach Community | ||
// License (the "License"); you may not use this file except in compliance with | ||
// the License. You may obtain a copy of the License at | ||
// | ||
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt | ||
|
||
package spanconfigkvsubscriberccl | ||
|
||
import ( | ||
"fmt" | ||
"testing" | ||
"time" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/base" | ||
"github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl/multiregionccltestutils" | ||
"github.com/cockroachdb/cockroach/pkg/jobs" | ||
"github.com/cockroachdb/cockroach/pkg/kv/kvserver" | ||
"github.com/cockroachdb/cockroach/pkg/testutils" | ||
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" | ||
"github.com/cockroachdb/cockroach/pkg/util/leaktest" | ||
"github.com/cockroachdb/cockroach/pkg/util/log" | ||
) | ||
|
||
// TestSpanConfigUpdatesApplyInCorrectOrder ensures the KVSubscriber applies | ||
// updates to its in-memory store in correct order. In particular, when there | ||
// are both deletions and additions at the same timestamp, the deletions should | ||
// be applied before the additions. This scenario is created by altering from a | ||
// regional by row table to a regional table -- doing so creates overlapping | ||
// updates with both deletions and additions with the same timestamo. | ||
// | ||
// Regression test for https://github.com/cockroachdb/cockroach/issues/110908. | ||
func TestSpanConfigUpdatesApplyInCorrectOrder(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
defer log.Scope(t).Close(t) | ||
|
||
tc, _, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster(t, 3, base.TestingKnobs{ | ||
Store: &kvserver.StoreTestingKnobs{ | ||
DisableMergeQueue: true, // we don't want the partitions to merge away | ||
}, | ||
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speed up schema changes. | ||
}, multiregionccltestutils.WithScanInterval(50*time.Millisecond)) | ||
defer cleanup() | ||
sqlDB := sqlutils.MakeSQLRunner(tc.ServerConn(0 /* idx */)) | ||
|
||
// Speed up the test. | ||
sqlDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'`) | ||
sqlDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '50ms'`) | ||
sqlDB.Exec(t, `SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '50ms'`) | ||
|
||
sqlDB.Exec(t, `CREATE DATABASE mr PRIMARY REGION "us-east1" REGIONS "us-east2", "us-east3"; USE mr`) | ||
sqlDB.Exec(t, `CREATE TABLE t() LOCALITY REGIONAL BY ROW`) | ||
|
||
testutils.SucceedsSoon(t, func() error { | ||
var count int | ||
sqlDB.QueryRow(t, | ||
"SELECT count(distinct lease_holder) from [show ranges from table t with details]", | ||
).Scan(&count) | ||
if count == 3 { | ||
return nil | ||
} | ||
return fmt.Errorf("waiting for each region to pick up leaseholders; count %d", count) | ||
}) | ||
|
||
sqlDB.Exec(t, `ALTER TABLE t SET LOCALITY REGIONAL BY TABLE`) | ||
testutils.SucceedsSoon(t, func() error { | ||
var count int | ||
sqlDB.QueryRow(t, | ||
"SELECT count(distinct lease_holder) from [show ranges from table t with details]", | ||
).Scan(&count) | ||
if count == 1 { | ||
return nil | ||
} | ||
return fmt.Errorf( | ||
"waiting for all partition leases to move to the primary region; number of regions %d", count, | ||
) | ||
}) | ||
} |
33 changes: 33 additions & 0 deletions
33
pkg/ccl/spanconfigccl/spanconfigkvsubscriberccl/main_test.go
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,33 @@ | ||
// Copyright 2024 The Cockroach Authors. | ||
// | ||
// Licensed as a CockroachDB Enterprise file under the Cockroach Community | ||
// License (the "License"); you may not use this file except in compliance with | ||
// the License. You may obtain a copy of the License at | ||
// | ||
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt | ||
|
||
package spanconfigkvsubscriberccl | ||
|
||
import ( | ||
"os" | ||
"testing" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/ccl" | ||
"github.com/cockroachdb/cockroach/pkg/security/securityassets" | ||
"github.com/cockroachdb/cockroach/pkg/security/securitytest" | ||
"github.com/cockroachdb/cockroach/pkg/server" | ||
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils" | ||
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster" | ||
"github.com/cockroachdb/cockroach/pkg/util/randutil" | ||
) | ||
|
||
//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go | ||
|
||
func TestMain(m *testing.M) { | ||
defer ccl.TestingEnableEnterprise()() | ||
securityassets.SetLoader(securitytest.EmbeddedAssets) | ||
randutil.SeedForTests() | ||
serverutils.InitTestServerFactory(server.TestServerFactory) | ||
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) | ||
os.Exit(m.Run()) | ||
} |
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
Oops, something went wrong.