Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
83265: kvserver/gc: remove range tombstones during GC r=erikgrinaker a=aliher1911


First commit adds support for range tombstones when removing point keys. Range tombstone will have the same effect as a point tombstone within the mvcc key history.

Second commit adds support for removal of range tombstones below GC threshold.


84493: sql: add parsing support for SHOW CREATE FUNCTION statement r=mgartner a=mgartner

This commit adds a `SHOW CREATE FUNCTION` statement to the SQL grammar.
This statement is not yet implemented and executing it results in an
error.

Release note: None

84504: execinfra: allow tenants to disable the streamer r=yuzefovich a=yuzefovich

Previously, we marked the setting that controls whether the streamer is
used as `TenantReadOnly` since we were not sure whether the streamer fit
well into the tenant cost model. Recently we revamped the cost model so
that it can now correctly predict the usage of the hardware resources by
the streamer, so at this point it seems safe to mark the setting
`TenantWritable`.

Informs: #82167

Release note: None

84515: ui: fix sorting of explain plans r=maryliag a=maryliag

Previously, the sorting on the plans on the
Explain Plans tab on Statement Details wasn't working.
This commit adds the missing code required to sort
that table.

Fixes #84079

https://www.loom.com/share/0f0ed0e1a8d04fc88def3b2460d617e6

Release note (bug fix): Sorting on the plans table inside the
Statement Details page is now properly working.

84519: clusterversion: remove some older versions r=yuzefovich a=yuzefovich

Release note: None

84601: rowexec: use OutOfOrder mode of streamer for lookup joins with ordering r=yuzefovich a=yuzefovich

Currently, the join reader always restores the required order for lookup
joins on its own since all looked up rows are buffered before any output
row is emitted. This observation allows us to use the OutOfOrder mode of
the streamer in such scenarios, so this commit makes such a change.
Previously, we would effectively maintain the order twice - both in the
streamer and in the join reader, and the former is redundant. This will
change in the future, but for now we can use the more-efficient mode.

```
name                                                  old time/op    new time/op    delta
LookupJoinEqColsAreKeyOrdering/Cockroach-24             6.64ms ± 1%    6.48ms ± 1%  -2.34%  (p=0.000 n=10+10)
LookupJoinEqColsAreKeyOrdering/MultinodeCockroach-24    7.89ms ± 1%    7.75ms ± 1%  -1.80%  (p=0.000 n=10+10)
LookupJoinOrdering/Cockroach-24                         9.01ms ± 3%    8.88ms ± 4%    ~     (p=0.218 n=10+10)
LookupJoinOrdering/MultinodeCockroach-24                12.1ms ± 4%    12.0ms ± 3%    ~     (p=0.393 n=10+10)

name                                                  old alloc/op   new alloc/op   delta
LookupJoinEqColsAreKeyOrdering/Cockroach-24             1.68MB ± 1%    1.60MB ± 1%  -4.93%  (p=0.000 n=10+10)
LookupJoinEqColsAreKeyOrdering/MultinodeCockroach-24    2.37MB ± 2%    2.29MB ± 2%  -3.11%  (p=0.000 n=10+10)
LookupJoinOrdering/Cockroach-24                         1.75MB ± 1%    1.66MB ± 1%  -5.01%  (p=0.000 n=10+9)
LookupJoinOrdering/MultinodeCockroach-24                2.36MB ± 1%    2.25MB ± 1%  -4.68%  (p=0.000 n=8+10)

name                                                  old allocs/op  new allocs/op  delta
LookupJoinEqColsAreKeyOrdering/Cockroach-24              10.0k ± 1%     10.0k ± 1%    ~     (p=0.278 n=10+9)
LookupJoinEqColsAreKeyOrdering/MultinodeCockroach-24     14.3k ± 1%     14.3k ± 1%    ~     (p=0.470 n=10+10)
LookupJoinOrdering/Cockroach-24                          12.4k ± 1%     12.5k ± 1%    ~     (p=0.780 n=10+10)
LookupJoinOrdering/MultinodeCockroach-24                 17.1k ± 1%     17.0k ± 1%    ~     (p=0.494 n=10+10)
```

Addresses: #82159.

Release note: None

84607: bench: add a benchmark of index join with ordering r=yuzefovich a=yuzefovich

Release note: None

Co-authored-by: Oleg Afanasyev <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
  • Loading branch information
5 people committed Jul 18, 2022
8 parents 09c8fa5 + 285aead + c8c5149 + 5a71d3e + 0d47e12 + a2ce495 + b9088e9 + 1f04334 commit eb28a3e
Show file tree
Hide file tree
Showing 80 changed files with 3,626 additions and 2,114 deletions.
44 changes: 31 additions & 13 deletions pkg/bench/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1136,13 +1136,10 @@ func BenchmarkPlanning(b *testing.B) {
})
}

// BenchmarkIndexJoin measure an index-join with 1000 rows.
func BenchmarkIndexJoin(b *testing.B) {
defer log.Scope(b).Close(b)
ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) {
// The table will have an extra column not contained in the index to force a
// join with the PK.
create := `
func setupIndexJoinBenchmark(b *testing.B, db *sqlutils.SQLRunner) {
// The table will have an extra column not contained in the index to force a
// join with the PK.
create := `
CREATE TABLE tidx (
k INT NOT NULL,
v INT NULL,
Expand All @@ -1152,13 +1149,20 @@ func BenchmarkIndexJoin(b *testing.B) {
FAMILY "primary" (k, v, extra)
)
`
// We'll insert 1000 rows with random values below 1000 in the index.
// We'll then force scanning of the secondary index which will require
// performing an index join to get 'extra' column.
insert := "insert into tidx(k,v) select generate_series(1,1000), (random()*1000)::int"
// We'll insert 1000 rows with random values below 1000 in the index.
// We'll then force scanning of the secondary index which will require
// performing an index join to get 'extra' column.
insert := "insert into tidx(k,v) select generate_series(1,1000), (random()*1000)::int"

db.Exec(b, create)
db.Exec(b, insert)
db.Exec(b, create)
db.Exec(b, insert)
}

// BenchmarkIndexJoin measure an index-join with 1000 rows.
func BenchmarkIndexJoin(b *testing.B) {
defer log.Scope(b).Close(b)
ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) {
setupIndexJoinBenchmark(b, db)
b.ResetTimer()

for i := 0; i < b.N; i++ {
Expand All @@ -1167,6 +1171,20 @@ func BenchmarkIndexJoin(b *testing.B) {
})
}

// BenchmarkIndexJoinOrdering is the same as BenchmarkIndexJoin when the
// ordering needs to be maintained.
func BenchmarkIndexJoinOrdering(b *testing.B) {
defer log.Scope(b).Close(b)
ForEachDB(b, func(b *testing.B, db *sqlutils.SQLRunner) {
setupIndexJoinBenchmark(b, db)
b.ResetTimer()

for i := 0; i < b.N; i++ {
db.Exec(b, "select * from bench.tidx@idx where v < 1000 order by v")
}
})
}

// BenchmarkIndexJoinColumnFamilies is the same as BenchmarkIndexJoin, only with
// the table having two column families.
func BenchmarkIndexJoinColumnFamilies(b *testing.B) {
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,6 @@ go_test(
"create_scheduled_backup_test.go",
"datadriven_test.go",
"full_cluster_backup_restore_test.go",
"insert_missing_public_schema_namespace_entry_restore_test.go",
"key_rewriter_test.go",
"main_test.go",
"partitioned_backup_test.go",
Expand Down

This file was deleted.

6 changes: 0 additions & 6 deletions pkg/ccl/backupccl/restore_data_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb"
"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/bulk"
Expand Down Expand Up @@ -409,11 +408,6 @@ func (rd *restoreDataProcessor) processRestoreSpanEntry(
defer sst.cleanup()

writeAtBatchTS := restoreAtNow.Get(&evalCtx.Settings.SV)
if writeAtBatchTS && !evalCtx.Settings.Version.IsActive(ctx, clusterversion.MVCCAddSSTable) {
return roachpb.BulkOpSummary{}, errors.Newf(
"cannot use %s until version %s", restoreAtNow.Key(), clusterversion.MVCCAddSSTable.String(),
)
}

// If the system tenant is restoring a guest tenant span, we don't want to
// forward all the restored data to now, as there may be importing tables in
Expand Down
2 changes: 0 additions & 2 deletions pkg/ccl/upgradeccl/upgradessccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,13 +6,11 @@ go_test(
srcs = [
"main_test.go",
"seed_span_counts_external_test.go",
"seed_tenant_span_configs_external_test.go",
],
deps = [
"//pkg/base",
"//pkg/ccl/kvccl/kvtenantccl",
"//pkg/clusterversion",
"//pkg/keys",
"//pkg/roachpb",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
_ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
Expand Down
Loading

0 comments on commit eb28a3e

Please sign in to comment.