Skip to content

Commit 2d73bfc

Browse files
craig[bot]yuzefovichkyle-a-wongnameisbhaskar
committed
151774: kv,sql: explicitly set IsReverse on the Header r=yuzefovich a=yuzefovich When processing batches that touch multiple ranges, the DistSender needs to know whether to iterate across those ranges in the forward or reverse manner (i.e. ASC or DESC for range keys). Currently, it only uses the reverse direction if it finds at least one ReverseScan request in the batch, and it uses the forward direction otherwise. This goes against the needs of SQL which might issue point Gets when performing a SQL revscan operation, and currently in such a scenario we would hit an error (instead of returning the results in incorrect order). This commit fixes the issue by introducing `IsReverse` boolean on the batch header to explicitly indicate the direction for range iteration. It seems reasonable that the caller should be explicit about this, and we also add a validation that the boolean is set correctly (meaning that it should be `false` when only forward range requests are present and `true` when only reverse range requests are present). In order to simplify the story a bit, the DistSender will no longer allow batches that have both forward and reverse range requests. Previously, this limitation only applied to the batches with limits, but now it's extended to be unconditional. SQL never issues such batches, so the limitation seems acceptable. This limitation required some updates to the existing tests, including KVNemesis to not generate batches that are now disallowed. See also 619f395 for some related context. Given that the new header field is only examined on the KV client, the change can be backported with no mixed-version concerns. Fixes: #146637. Release note (bug fix): Previously, CockroachDB could hit an error `ERROR: span with results after resume span...` when evaluating some queries with ORDER BY ... DESC in an edge case. The bug has been present since about v22.1 and is now fixed. 151949: sql: prep migration for sampled_query and sampled_transaction events r=kyle-a-wong a=kyle-a-wong In v26.1, sampled_query and sampled_transaction events will be moved from the TELEMETRY logging channel to the SQL_EXEC logging channel. This commit gates this migration under the cluster setting: `log.channel_compatibility_mode.enabled` and will log these events to the SQL_EXEC channel if this setting is set to false. Users can set this setting to false in their clusters to validate, test, and identify potential downstream impacts to their logging setups and pipelines. Epic: [CRDB-53410](https://cockroachlabs.atlassian.net/browse/CRDB-53410) Part of: [CRDB-53412](https://cockroachlabs.atlassian.net/browse/CRDB-53412) Release note (ops change): sampled_query and sampled_transaction events will be moved to the SQL_EXEC channel in 26.1. In order to test the impact of these changes, users can set the setting: `log.channel_compatibility_mode.enabled` to false. Note that this will cause these logs to start logging in the SQL_EXEC channel so this shouldn't be tested in a production environment. ---- This is a stacked PR, only the last commit in this PR should be reviewed 152080: clusterversion: bump PreviousVersion r=RaduBerinde a=RaduBerinde Bump PreviousVersion to 25.3, add cockroach-go-testserver-25.3 tests. This is part of the [M.3 checklist](https://github.com/cockroachdb/cockroach/blob/master/pkg/clusterversion/README.md#m3-enable-upgrade-tests) (the roachtest fixtures were already done in #150712). - [x] Update `PreviousRelease` constant to 25.3. - [x] Verify the logic in `supportsSkipUpgradeTo` (`pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go`) is correct for the new release. - [x] Add `cockroach-go-testserver-25.3` logictest config and add it to `cockroach-go-testserver-configs`. Update the visibility for `cockroach_predecessor_version` in `pkg/sql/logictest/BUILD.bazel`. Run `./dev gen` and fix up any tests that fail (using a draft PR helps). - [x] Check that all gates for the previous release are identical on the `master` and release branch. Epic: REL-2736 Release note: None Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com> Co-authored-by: Kyle Wong <37189875+kyle-a-wong@users.noreply.github.com> Co-authored-by: Bhaskarjyoti Bora <bhaskar.bora@cockroachlabs.com>
4 parents 693c395 + 34ec3cb + 6e52e0c + b4a7d05 commit 2d73bfc

File tree

36 files changed

+503
-242
lines changed

36 files changed

+503
-242
lines changed

docs/generated/eventlog.md

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3374,6 +3374,11 @@ Fields in this struct should be updated in sync with apps_stats.proto.
33743374
An event of type `sampled_query` is the SQL query event logged to the telemetry channel. It
33753375
contains common SQL event/execution details.
33763376

3377+
Note: in version 26.1, these events will be moved to the `SQL_EXEC` channel.
3378+
To test compatability before this, set the cluster setting
3379+
`log.channel_compatibility_mode.enabled` to false. This will send the
3380+
events to `SQL_EXEC` instead of `TELEMETRY`.
3381+
33773382

33783383
| Field | Description | Sensitive |
33793384
|--|--|--|
@@ -3482,6 +3487,11 @@ contains common SQL event/execution details.
34823487

34833488
An event of type `sampled_transaction` is the event logged to telemetry at the end of transaction execution.
34843489

3490+
Note: in version 26.1, these events will be moved to the `SQL_EXEC` channel.
3491+
To test compatability before this, set the cluster setting
3492+
`log.channel_compatibility_mode.enabled` to false. This will send the
3493+
events to `SQL_EXEC` instead of `TELEMETRY`.
3494+
34853495

34863496
| Field | Description | Sensitive |
34873497
|--|--|--|

pkg/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -508,6 +508,7 @@ ALL_TESTS = [
508508
"//pkg/sql/logictest/tests/5node-disk:5node-disk_test",
509509
"//pkg/sql/logictest/tests/5node:5node_test",
510510
"//pkg/sql/logictest/tests/cockroach-go-testserver-25.2:cockroach-go-testserver-25_2_test",
511+
"//pkg/sql/logictest/tests/cockroach-go-testserver-25.3:cockroach-go-testserver-25_3_test",
511512
"//pkg/sql/logictest/tests/fakedist-disk:fakedist-disk_test",
512513
"//pkg/sql/logictest/tests/fakedist-vec-off:fakedist-vec-off_test",
513514
"//pkg/sql/logictest/tests/fakedist:fakedist_test",
@@ -2054,6 +2055,7 @@ GO_TARGETS = [
20542055
"//pkg/sql/logictest/tests/5node-disk:5node-disk_test",
20552056
"//pkg/sql/logictest/tests/5node:5node_test",
20562057
"//pkg/sql/logictest/tests/cockroach-go-testserver-25.2:cockroach-go-testserver-25_2_test",
2058+
"//pkg/sql/logictest/tests/cockroach-go-testserver-25.3:cockroach-go-testserver-25_3_test",
20572059
"//pkg/sql/logictest/tests/fakedist-disk:fakedist-disk_test",
20582060
"//pkg/sql/logictest/tests/fakedist-vec-off:fakedist-vec-off_test",
20592061
"//pkg/sql/logictest/tests/fakedist:fakedist_test",

pkg/ccl/telemetryccl/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ go_test(
2020
"//pkg/security/securityassets",
2121
"//pkg/security/securitytest",
2222
"//pkg/server",
23+
"//pkg/settings",
2324
"//pkg/sql",
2425
"//pkg/sql/sem/tree",
2526
"//pkg/sql/sqltestutils",

pkg/ccl/telemetryccl/telemetry_logging_test.go

Lines changed: 22 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@ import (
1313
"net/http"
1414
"net/http/httptest"
1515
"regexp"
16+
"slices"
1617
"strings"
1718
"testing"
1819

@@ -21,6 +22,7 @@ import (
2122
"github.com/cockroachdb/cockroach/pkg/cloud/nodelocal"
2223
"github.com/cockroachdb/cockroach/pkg/jobs"
2324
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
25+
"github.com/cockroachdb/cockroach/pkg/settings"
2426
"github.com/cockroachdb/cockroach/pkg/sql"
2527
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
2628
"github.com/cockroachdb/cockroach/pkg/testutils"
@@ -155,20 +157,29 @@ type expectedSampleQueryEvent struct {
155157
}
156158

157159
type telemetrySpy struct {
158-
t *testing.T
160+
t *testing.T
161+
sv *settings.Values
159162

160163
sampledQueries []eventpb.SampledQuery
161164
sampledQueriesRaw []logpb.Entry
162165
recoveryEvents []eventpb.RecoveryEvent
163166
}
164167

168+
func (l *telemetrySpy) channelsToIntercept() []log.Channel {
169+
if log.ShouldMigrateEvent(l.sv) {
170+
return []log.Channel{logpb.Channel_TELEMETRY, logpb.Channel_SQL_EXEC}
171+
}
172+
173+
return []log.Channel{logpb.Channel_TELEMETRY}
174+
}
175+
165176
func (l *telemetrySpy) Intercept(entry []byte) {
166177
var rawLog logpb.Entry
167178
if err := json.Unmarshal(entry, &rawLog); err != nil {
168179
l.t.Errorf("failed unmarshaling %s: %s", entry, err)
169180
}
170181

171-
if rawLog.Channel != logpb.Channel_TELEMETRY {
182+
if !slices.Contains(l.channelsToIntercept(), rawLog.Channel) {
172183
return
173184
}
174185

@@ -204,12 +215,6 @@ func TestBulkJobTelemetryLogging(t *testing.T) {
204215

205216
ctx := context.Background()
206217

207-
spy := &telemetrySpy{
208-
t: t,
209-
}
210-
cleanup := log.InterceptWith(ctx, spy)
211-
defer cleanup()
212-
213218
st := logtestutils.StubTime{}
214219
sqm := logtestutils.StubQueryStats{}
215220
sts := logtestutils.StubTracingStatus{}
@@ -229,6 +234,15 @@ func TestBulkJobTelemetryLogging(t *testing.T) {
229234
ExternalIODir: dir,
230235
},
231236
})
237+
238+
spy := &telemetrySpy{
239+
t: t,
240+
sv: &testCluster.Server(0).ClusterSettings().SV,
241+
}
242+
243+
cleanup := log.InterceptWith(ctx, spy)
244+
defer cleanup()
245+
232246
sqlDB := testCluster.ServerConn(0)
233247
defer func() {
234248
testCluster.Stopper().Stop(context.Background())

pkg/cli/testdata/declarative-rules/deprules

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,6 @@
11
dep
22
----
3-
debug declarative-print-rules 1000025.2 dep
3+
debug declarative-print-rules 1000025.3 dep
44
deprules
55
----
66
- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED'

pkg/clusterversion/cockroach_versions.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -283,7 +283,7 @@ const MinSupported Key = V25_2
283283

284284
// PreviousRelease is the logical cluster version of the previous release (which must
285285
// have at least an RC build published).
286-
const PreviousRelease Key = V25_2
286+
const PreviousRelease Key = V25_3
287287

288288
// V25_4 is a placeholder that will eventually be replaced by the actual 25.4
289289
// version Key, but in the meantime it points to the latest Key. The placeholder

pkg/crosscluster/logical/logical_replication_job_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2889,7 +2889,7 @@ func TestGetWriterType(t *testing.T) {
28892889

28902890
t.Run("immediate-mode", func(t *testing.T) {
28912891
st := cluster.MakeTestingClusterSettingsWithVersions(
2892-
clusterversion.V25_2.Version(),
2892+
clusterversion.V25_3.Version(),
28932893
clusterversion.PreviousRelease.Version(),
28942894
true, /* initializeVersion */
28952895
)

pkg/kv/batch.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -365,6 +365,8 @@ func (b *Batch) AddRawRequest(reqs ...kvpb.Request) {
365365
*kvpb.IncrementRequest,
366366
*kvpb.DeleteRequest:
367367
numRows = 1
368+
case *kvpb.ReverseScanRequest:
369+
b.Header.IsReverse = true
368370
}
369371
b.appendReqs(args)
370372
b.initResult(1 /* calls */, numRows, raw, nil)
@@ -735,6 +737,7 @@ func (b *Batch) scan(
735737
str kvpb.KeyLockingStrengthType,
736738
dur kvpb.KeyLockingDurabilityType,
737739
) {
740+
b.Header.IsReverse = isReverse
738741
begin, err := marshalKey(s)
739742
if err != nil {
740743
b.initResult(0, 0, notRaw, err)

pkg/kv/db.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -499,6 +499,7 @@ func (db *DB) scan(
499499
if maxRows > 0 {
500500
b.Header.MaxSpanRequestKeys = maxRows
501501
}
502+
b.Header.IsReverse = isReverse
502503
b.scan(begin, end, isReverse, str, dur)
503504
r, err := getOneResult(db.Run(ctx, b), b)
504505
return r.Rows, err

0 commit comments

Comments
 (0)