Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
87261: storage: add range keys and mvccstats to gc bench r=aliher1911 a=aliher1911

Previously gc benchmarking code didn't include neither range key
tombstones, nor mvcc stats calculations.
This is not realistic as production always uses stats and it is
a non-negligible part of the operation time.
This commit adds option to enable stats calculation during
benchmarking as well as adding underlying range key tombstone
to include everything.

Release justification: increasing bench coverage, non production.
Release note: None

87310: sql: parse sequence name instead of using name identifier directly r=chengxiong-ruan a=chengxiong-ruan

Previously, we use the sequence name identifier directly when building
a function in optbuilder if a function expression use any sequences.
This does work when sequence name string is qualifed because sequence
name cannot be found. We need to parse the name string into a table
name instead.

Release note: None.
Release justification: low risk bug fix.

87423: dev: if specified, dev generate bazel before other targets r=healthy-pod a=rickystewart

The results of this generation can impact most of the other generation
targets, so it should be done first.

Release justification: Non-production code changes
Release note: None

Co-authored-by: Oleg Afanasyev <[email protected]>
Co-authored-by: Chengxiong Ruan <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
4 people committed Sep 6, 2022
4 parents 47069e9 + ab4b121 + 2fbca6a + 45d8472 commit 0197ada
Show file tree
Hide file tree
Showing 6 changed files with 106 additions and 19 deletions.
2 changes: 1 addition & 1 deletion dev
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ fi
set -euo pipefail

# Bump this counter to force rebuilding `dev` on all machines.
DEV_VERSION=54
DEV_VERSION=55

THIS_DIR=$(cd "$(dirname "$0")" && pwd)
BINARY_DIR=$THIS_DIR/bin/dev-versions
Expand Down
7 changes: 7 additions & 0 deletions pkg/cmd/dev/generate.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,13 @@ func (d *dev) generate(cmd *cobra.Command, targets []string) error {
for _, target := range targets {
targetsMap[target] = struct{}{}
}
// NB: We have to run the bazel generator first if it's specified.
if _, ok := targetsMap["bazel"]; ok {
delete(targetsMap, "bazel")
if err := generatorTargetMapping["bazel"](cmd); err != nil {
return err
}
}
{
// In this case, generating both go and cgo would duplicate work.
// Generate go_nocgo instead.
Expand Down
25 changes: 25 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/udf
Original file line number Diff line number Diff line change
Expand Up @@ -1211,6 +1211,31 @@ CREATE FUNCTION public.f_test_cor(IN a INT8, IN b INT8)
SELECT 3;
$$

subtest seq_qualified_name

statement ok
CREATE SCHEMA sc_seq_qualified_name;
CREATE SEQUENCE sc_seq_qualified_name.sq;

statement error pq: relation "sc_seq_qualified_name.sq" does not exist
CREATE FUNCTION f_seq_qualified_name() RETURNS INT LANGUAGE SQL AS $$ SELECT * FROM nextval('"sc_seq_qualified_name.sq"') $$;

statement ok
CREATE FUNCTION f_seq_qualified_name() RETURNS INT LANGUAGE SQL AS $$ SELECT nextval('sc_seq_qualified_name.sq') $$;

query I
SELECT f_seq_qualified_name()
----
1

statement ok
CREATE FUNCTION f_seq_qualified_name_quoted() RETURNS INT LANGUAGE SQL AS $$ SELECT nextval('"sc_seq_qualified_name"."sq"') $$;

query I
SELECT f_seq_qualified_name_quoted()
----
2

subtest execution

statement ok
Expand Down
25 changes: 25 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/views
Original file line number Diff line number Diff line change
Expand Up @@ -1336,3 +1336,28 @@ a INT8 true NULL · {materialized_view_with_null_pkey} fa
b STRING true NULL · {materialized_view_with_null_pkey} false
c INT8 true NULL · {materialized_view_with_null_pkey} false
rowid INT8 false unique_rowid() · {materialized_view_with_null_pkey} true

subtest seq_qualified_name

statement ok
CREATE SCHEMA sc_seq_qualified_name;
CREATE SEQUENCE sc_seq_qualified_name.sq;

statement error pq: relation "sc_seq_qualified_name.sq" does not exist
CREATE VIEW v_seq_rewrite_quoted AS SELECT nextval('"sc_seq_qualified_name.sq"');

statement ok
CREATE VIEW v_seq_rewrite AS SELECT nextval('sc_seq_qualified_name.sq');

query I
SELECT * FROM v_seq_rewrite
----
1

statement ok
CREATE VIEW v_seq_rewrite_quoted AS SELECT nextval('"sc_seq_qualified_name"."sq"');

query I
SELECT * FROM v_seq_rewrite_quoted
----
2
7 changes: 5 additions & 2 deletions pkg/sql/opt/optbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -582,8 +582,11 @@ func (b *Builder) buildFunction(
panic(err)
}
} else {
tn := tree.MakeUnqualifiedTableName(tree.Name(seqIdentifier.SeqName))
ds, _, _ = b.resolveDataSource(&tn, privilege.SELECT)
tn, err := parser.ParseQualifiedTableName(seqIdentifier.SeqName)
if err != nil {
panic(err)
}
ds, _, _ = b.resolveDataSource(tn, privilege.SELECT)
}
b.schemaDeps = append(b.schemaDeps, opt.SchemaDep{
DataSource: ds,
Expand Down
59 changes: 43 additions & 16 deletions pkg/storage/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,8 @@ func BenchmarkMVCCGarbageCollect(b *testing.B) {
{2, []int{1}},
{1024, []int{1, 16, 32, 512, 1015, 1023}},
}
numRangeTombstones := []int{0, 1}
updateStats := []bool{false, true}
engineMakers := []struct {
name string
create engineMaker
Expand All @@ -83,16 +85,26 @@ func BenchmarkMVCCGarbageCollect(b *testing.B) {
b.Run(fmt.Sprintf("numVersions=%d", versions.total), func(b *testing.B) {
for _, toDelete := range versions.toDelete {
b.Run(fmt.Sprintf("deleteVersions=%d", toDelete), func(b *testing.B) {
runMVCCGarbageCollect(ctx, b, engineImpl.create,
benchGarbageCollectOptions{
benchDataOptions: benchDataOptions{
numKeys: numKeys,
numVersions: versions.total,
valueBytes: valSize,
},
keyBytes: keySize,
deleteVersions: toDelete,
for _, rangeTombstones := range numRangeTombstones {
b.Run(fmt.Sprintf("numRangeTs=%d", rangeTombstones), func(b *testing.B) {
for _, stats := range updateStats {
b.Run(fmt.Sprintf("updateStats=%t", stats), func(b *testing.B) {
runMVCCGarbageCollect(ctx, b, engineImpl.create,
benchGarbageCollectOptions{
benchDataOptions: benchDataOptions{
numKeys: numKeys,
numVersions: versions.total,
valueBytes: valSize,
numRangeKeys: rangeTombstones,
},
keyBytes: keySize,
deleteVersions: toDelete,
updateStats: stats,
})
})
}
})
}
})
}
})
Expand Down Expand Up @@ -1514,6 +1526,7 @@ type benchGarbageCollectOptions struct {
benchDataOptions
keyBytes int
deleteVersions int
updateStats bool
}

func runMVCCGarbageCollect(
Expand All @@ -1526,31 +1539,41 @@ func runMVCCGarbageCollect(
ts := hlc.Timestamp{}.Add(time.Date(2000, 1, 1, 0, 0, 0, 0, time.UTC).UnixNano(), 0)
val := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, opts.valueBytes))

// We write values at ts+(0,i), set now=ts+(1,0) so that we're ahead of all
// We write values at ts+(1,i), set now=ts+(2,0) so that we're ahead of all
// the writes. This value doesn't matter in practice, as it's used only for
// stats updates.
now := ts.Add(1, 0)
now := ts.Add(2, 0)

// Write 'numKeys' of the given 'keySize' and 'valSize' to the given engine.
// For each key, write 'numVersions' versions, and add a GCRequest_GCKey to
// the returned slice that affects the oldest 'deleteVersions' versions. The
// first write for each key will be at `ts`, the second one at `ts+(0,1)`,
// etc.
// first write for each key will be at `ts+(1,0)`, the second one
// at `ts+(1,1)`, etc.
// If numRangeKeys is set to 1 then range tombstone will be written at ts.
//
// NB: a real invocation of MVCCGarbageCollect typically has most of the keys
// in sorted order. Here they will be ordered randomly.
setup := func() (gcKeys []roachpb.GCRequest_GCKey) {
batch := eng.NewBatch()
if opts.numRangeKeys > 1 {
b.Fatal("Invalid bench data config. Number of range keys can be 0 or 1")
}
if opts.numRangeKeys == 1 {
if err := MVCCDeleteRangeUsingTombstone(ctx, batch, nil, keys.LocalMax, keys.MaxKey,
ts, hlc.ClockTimestamp{}, nil, nil, true, 0, nil); err != nil {
b.Fatal(err)
}
}
for i := 0; i < opts.numKeys; i++ {
key := randutil.RandBytes(rng, opts.keyBytes)
if opts.deleteVersions > 0 {
gcKeys = append(gcKeys, roachpb.GCRequest_GCKey{
Timestamp: ts.Add(0, int32(opts.deleteVersions-1)),
Timestamp: ts.Add(1, int32(opts.deleteVersions-1)),
Key: key,
})
}
for j := 0; j < opts.numVersions; j++ {
if err := MVCCPut(ctx, batch, nil, key, ts.Add(0, int32(j)), hlc.ClockTimestamp{}, val, nil); err != nil {
if err := MVCCPut(ctx, batch, nil, key, ts.Add(1, int32(j)), hlc.ClockTimestamp{}, val, nil); err != nil {
b.Fatal(err)
}
}
Expand All @@ -1564,10 +1587,14 @@ func runMVCCGarbageCollect(

gcKeys := setup()

var ms *enginepb.MVCCStats
if opts.updateStats {
ms = &enginepb.MVCCStats{}
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
batch := eng.NewBatch()
if err := MVCCGarbageCollect(ctx, batch, nil /* ms */, gcKeys, now); err != nil {
if err := MVCCGarbageCollect(ctx, batch, ms, gcKeys, now); err != nil {
b.Fatal(err)
}
batch.Close()
Expand Down

0 comments on commit 0197ada

Please sign in to comment.