-
Notifications
You must be signed in to change notification settings - Fork 3.9k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
99325: sql: disable the streamer for queries which might use internal executor r=yuzefovich a=yuzefovich **row: allow for overlapping spans for range lookups** This commit fixes a bug with range lookup joins when the streamer is disabled. Previously, we could hit an error about "unordered spans" when initializing the fetch for range lookups if batch limits are used. Overlapping and unordered spans are actually expected for range lookups, so this commit simply disables the error check. It also adds an extensive comment for how exactly the fetcher will behave in such scenario. There is no regression test because it'll be introduced in the following commit (implicitly) and no release note since for the bug to occur the streamer must be disabled (and on 22.2 it would mean non-default config). Fixes: #99330. Release note: None **sql: disable the streamer for queries which might use internal executor** This commit fixes a possible violation of `kv.Txn` API that was introduced when we enabled the usage of the streamer by default in 22.2.0. Namely, the problem is as follows: the streamer requires the LeafTxn to be used since it can perform reads concurrently with other parts of the execution flow; however, if the flow contains a wrapped `planNode` which is using the internal executor, the query issued by the IE might use the RootTxn. As a result, we might have concurrency between the LeafTxn of the "outer" query and the RootTxn of the "inner" query which is not allowed. The fix in this commit is "quick" and is disallowing the usage of the streamer in more cases than strictly necessary. In particular: 1) it makes it so that the streamer is not used by the flow that has any `planNode`s (even if they don't use the IE at all and don't interact with the `kv.Txn` otherwise either). Auditing each `planNode` implementation is error-prone, and this "quick" fix should be more reliable. 2) it makes it so that the streamer is disabled for all queries issued by the IE. The thinking behind the second change is as follows: if the query issued by the IE uses the streamer, then it'll use the LeafTxn. The IE exposes the iterator API, so it might be possible for the user of the IE to keep the iterator "open" while returning the control flow back to the "outer" flow. If that "outer" flow is using the RootTxn, we can have the same concurrency violation with the "paused" IE iterator performing some reads in the streamer. Overall, this is "not terrible, not great" - we effectively fallback to the pre-22.2 behavior for some types of queries. For the queries that do process a lot of data, the streamer is likely to still be enabled. Fixes: #99093. Informs: #99209. Release note (bug fix): Since 22.2.0 CockroachDB could crash with "attempting to append refresh spans after the tracked timestamp has moved forward" error in some rare cases (most likely when querying `pg_catalog` and `crdb_internal` virtual tables), and this has now been fixed. The workaround before upgrading would be to run `SET CLUSTER SETTING sql.distsql.use_streamer.enabled = false;`. 99375: cli: don't fail drain cmd if cluster settings aren't available r=rafiss a=rafiss This makes the command more robust, since it should still work even if the settings cannot be fetched. If the cluster is not fully available, then this step may fail, but it should not prevent a drain command on a specific node. informs #98742 Release note: None 99392: sql: fix internal error when calling ts_rank with array longer than 4 r=rytaft a=rytaft Prior to this commit, an internal error could occur when an array longer than length 4 was passed to `ts_rank`. This commit fixes the error by truncating the array to length 4. Fixes #99334 Release note: None 99413: sql/tests: give more memory to TestSchemaChangesInParallel r=rafiss a=rafiss The test has previously failed due to running out of memory. Since it's an expensive test, this is somewhat expected. fixes #98850 Release note: None Co-authored-by: Yahor Yuzefovich <[email protected]> Co-authored-by: Rafi Shamim <[email protected]> Co-authored-by: Rebecca Taft <[email protected]>
- Loading branch information
Showing
23 changed files
with
305 additions
and
87 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,60 @@ | ||
// Copyright 2023 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 kvstreamer_test | ||
|
||
import ( | ||
"context" | ||
"testing" | ||
|
||
"github.com/cockroachdb/cockroach/pkg/base" | ||
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils" | ||
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" | ||
"github.com/cockroachdb/cockroach/pkg/util/leaktest" | ||
"github.com/cockroachdb/cockroach/pkg/util/log" | ||
"github.com/stretchr/testify/require" | ||
) | ||
|
||
// TestStreamerDisabledWithInternalExecutorQuery verifies that the streamer is | ||
// not used when the plan has a planNode that will use the internal executor. It | ||
// also confirms that the streamer is not used for queries issued by that | ||
// planNode. | ||
func TestStreamerDisabledWithInternalExecutorQuery(t *testing.T) { | ||
defer leaktest.AfterTest(t)() | ||
defer log.Scope(t).Close(t) | ||
|
||
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) | ||
ctx := context.Background() | ||
defer s.Stopper().Stop(ctx) | ||
|
||
// Trace a query which has a lookup join on top of a scan of a virtual | ||
// table, with that virtual table being populated by a query issued via the | ||
// internal executor. | ||
runner := sqlutils.MakeSQLRunner(db) | ||
runner.Exec(t, "COMMENT ON DATABASE defaultdb IS 'foo'") | ||
runner.Exec(t, "SET tracing = on") | ||
runner.Exec(t, ` | ||
SELECT | ||
c.* | ||
FROM | ||
crdb_internal.jobs AS j | ||
INNER LOOKUP JOIN system.comments AS c ON c.type = (j.num_runs - 1)::INT8 | ||
WHERE | ||
j.num_runs = 1; | ||
`) | ||
runner.Exec(t, "SET tracing = off") | ||
|
||
// Ensure that no streamer spans were created (meaning that the streamer | ||
// wasn't used, neither for the "outer" query nor for any "internal" ones). | ||
r := runner.QueryRow(t, "SELECT count(*) FROM [SHOW TRACE FOR SESSION] WHERE operation ILIKE '%streamer%'") | ||
var numStreamerSpans int | ||
r.Scan(&numStreamerSpans) | ||
require.Zero(t, numStreamerSpans) | ||
} |
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
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
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.