Skip to content

Commit

Permalink
Merge #56954
Browse files Browse the repository at this point in the history
56954: sql: implement sequence caching and cached sequence serial normalization r=ajwerner a=jayshrivastava

### sql: implement sequence caching

Previously, incrementing sequences at a high throughput
would result in many distributed writes to the KV layer
due to MVCC. This has caused garbage collection problems
in the past. This would occur in situations such as
bulk importing data while using the sequence number as an
id for each new row being added.

This change allows clients to cache sequence numbers in their local
session data. When the cache is empty, the sequence will be
incremented once by the cache size * increment amount, which are
both sequence options. Then, all the intermediate values will be
cached locally on a node to be given out whenever the sequence is
incremented.

To accommodate schema changes, cached sequences values will be
invalidated when new descriptor versions are seen by the cache.
This invalidation can occur when old versions are seen as well
to accommodate schema change rollbacks.

Release note (sql change): Using the CACHE sequence option no longer
results in an "unimplemented" error. The CACHE option is now fully
implemented and will allow nodes to cache sequence numbers. A cache
size of 1 means that there is no cache, and cache sizes of less than 1
are not valid.

### sql: create benchmark for concurrent sequence increments

Previously, there was no benchmark that tested the performance
of concurrent increments to sequences. There was also no
benchmark which compared sequence performance based on
different cache sizes. This change updates a benchmark to measure
performance based on the above criteria.

Release note: None

### sql: add serial normalization setting for cached sequences

Closes: #51259

Release note (sql change): The `serial_normalization` session
variable can now be set to the value `sql_sequence_cached`.
Cached sequences will allow nodes to cache 256 sequence numbers
locally. The underlying sequence will only be incremened once (by
256 increments) when the cache is empty. Using `sql_sequence_cached`
will result in better performance than `sql_sequence` because the
former will perform fewer distributed calls to increment sequences.
However, cached seqences may contribute to large gaps between
sequence numbers if cached values are lost due to errors or
node outages.

###  sql: make cached sequences bounds-related semantics match pg semantics

Previously, cached sequences did not obey pg semantics
with respect to exceeding bounds. For example, if there
were a sequence with a cache size of 5 and max value of 2,
calling nextval(...) would immediately cause an error due
to exceeded bounds. According to postgres, nextval(...)
should return 1, then 2, then finally return an error due
to the max value of 2 being reached. This change alters
sequence caching behavior to match the above semantics.

Additionally, this change now makes it possible for sequences
to exceed the bounds set by MAXVALUE and MINVALUE. This is
because calling nextval(...) should be as fast as possible,
and the fastest way to do this is to let nextval(...) always
succeed on incrementing a sequence. Despite this, a user
will never see any values that exceed a sequence's bounds.
To make a sequence incrementable again after exceeding its
bounds, there are two options:
1. The user changes the sequence's value by calling setval(...).
2. The user performs a schema change to alter the sequences MinValue
   or MaxValue. If the value of a sequence exceeds its bounds,
   it must be restored to the original MinValue or MaxValue in
   the same transaction as the schema change.
This change adds code to handle case 2 above.

Release note: None

### Benchmark
Using `make bench PKG='./pkg/sql' BENCHES='BenchmarkSequenceIncrement' TESTFLAGS="--count=5 --benchmem" |& tee c{size} .txt`:

Caching 256 values
```
name                     old time/op    new time/op    delta
SequenceIncrement/P-1-8     313µs ± 1%     107µs ± 1%  -65.85%  (p=0.008 n=5+5)
SequenceIncrement/P-2-8     327µs ± 2%      82µs ± 2%  -75.04%  (p=0.008 n=5+5)
SequenceIncrement/P-4-8     347µs ± 1%      70µs ± 2%  -79.89%  (p=0.008 n=5+5)
SequenceIncrement/P-8-8     389µs ± 2%      65µs ± 3%  -83.40%  (p=0.008 n=5+5)

name                     old alloc/op   new alloc/op   delta
SequenceIncrement/P-1-8     130kB ± 1%      57kB ± 1%  -55.76%  (p=0.008 n=5+5)
SequenceIncrement/P-2-8     131kB ± 0%      49kB ± 1%  -62.56%  (p=0.008 n=5+5)
SequenceIncrement/P-4-8     132kB ± 0%      46kB ± 0%  -65.26%  (p=0.008 n=5+5)
SequenceIncrement/P-8-8     134kB ± 0%      44kB ± 1%  -67.09%  (p=0.008 n=5+5)

name                     old allocs/op  new allocs/op  delta
SequenceIncrement/P-1-8       807 ± 0%       406 ± 0%  -49.75%  (p=0.000 n=5+4)
SequenceIncrement/P-2-8       812 ± 1%       363 ± 0%  -55.32%  (p=0.008 n=5+5)
SequenceIncrement/P-4-8       826 ± 1%       346 ± 0%  -58.10%  (p=0.008 n=5+5)
SequenceIncrement/P-8-8       863 ± 1%       341 ± 0%  -60.44%  (p=0.008 n=5+5)
```

For the other cache sizes I tested, the performance improvement is virtually the same as above. 
[c1.txt](https://github.com/cockroachdb/cockroach/files/5692305/c1.txt)
[c32.txt](https://github.com/cockroachdb/cockroach/files/5692306/c32.txt)
[c64.txt](https://github.com/cockroachdb/cockroach/files/5692307/c64.txt)
[c128.txt](https://github.com/cockroachdb/cockroach/files/5692308/c128.txt)
[c256.txt](https://github.com/cockroachdb/cockroach/files/5692310/c256.txt)
[c512.txt](https://github.com/cockroachdb/cockroach/files/5692326/c512.txt)
[c1024.txt](https://github.com/cockroachdb/cockroach/files/5692309/c1024.txt)


Co-authored-by: Jayant Shrivastava <jayants@cockroachlabs.com>
Co-authored-by: Jayant Shrivastava <jshrivastava03@gmail.com>
  • Loading branch information
3 people committed Feb 17, 2021
2 parents 304f082 + 91c162a commit bededc2
Show file tree
Hide file tree
Showing 16 changed files with 1,364 additions and 441 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@
<tr><td><code>sql.defaults.default_int_size</code></td><td>integer</td><td><code>8</code></td><td>the size, in bytes, of an INT type</td></tr>
<tr><td><code>sql.defaults.disallow_full_table_scans.enabled</code></td><td>boolean</td><td><code>false</code></td><td>setting to true rejects queries that have planned a full table scan</td></tr>
<tr><td><code>sql.defaults.results_buffer.size</code></td><td>byte size</td><td><code>16 KiB</code></td><td>default size of the buffer that accumulates results for a statement or a batch of statements before they are sent to the client. This can be overridden on an individual connection with the 'results_buffer_size' parameter. Note that auto-retries generally only happen while no results have been delivered to the client, so reducing this size can increase the number of retriable errors a client receives. On the other hand, increasing the buffer size can increase the delay until the client receives the first result row. Updating the setting only affects new connections. Setting to 0 disables any buffering.</td></tr>
<tr><td><code>sql.defaults.serial_normalization</code></td><td>enumeration</td><td><code>rowid</code></td><td>default handling of SERIAL in table definitions [rowid = 0, virtual_sequence = 1, sql_sequence = 2]</td></tr>
<tr><td><code>sql.defaults.serial_normalization</code></td><td>enumeration</td><td><code>rowid</code></td><td>default handling of SERIAL in table definitions [rowid = 0, virtual_sequence = 1, sql_sequence = 2, sql_sequence_cached = 3]</td></tr>
<tr><td><code>sql.distsql.max_running_flows</code></td><td>integer</td><td><code>500</code></td><td>maximum number of concurrent flows that can be run on a node</td></tr>
<tr><td><code>sql.log.slow_query.experimental_full_table_scans.enabled</code></td><td>boolean</td><td><code>false</code></td><td>when set to true, statements that perform a full table/index scan will be logged to the slow query log even if they do not meet the latency threshold. Must have the slow query log enabled for this setting to have any effect.</td></tr>
<tr><td><code>sql.log.slow_query.internal_queries.enabled</code></td><td>boolean</td><td><code>false</code></td><td>when set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect.</td></tr>
Expand Down
53 changes: 53 additions & 0 deletions pkg/sql/alter_sequence.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,12 +63,65 @@ func (n *alterSequenceNode) startExec(params runParams) error {
telemetry.Inc(sqltelemetry.SchemaChangeAlterCounter("sequence"))
desc := n.seqDesc

oldMinValue := desc.SequenceOpts.MinValue
oldMaxValue := desc.SequenceOpts.MaxValue

err := assignSequenceOptions(
desc.SequenceOpts, n.n.Options, false /* setDefaults */, &params, desc.GetID(), desc.ParentID,
)
if err != nil {
return err
}
opts := desc.SequenceOpts
seqValueKey := params.p.ExecCfg().Codec.SequenceKey(uint32(desc.ID))
if err != nil {
return err
}

getSequenceValue := func() (int64, error) {
kv, err := params.p.txn.Get(params.ctx, seqValueKey)
if err != nil {
return 0, err
}
return kv.ValueInt(), nil
}

// Due to the semantics of sequence caching (see sql.planner.incrementSequenceUsingCache()),
// it is possible for a sequence to have a value that exceeds its MinValue or MaxValue. Users
// do no see values extending the sequence's bounds, and instead see "bounds exceeded" errors.
// To make a usable again after exceeding its bounds, there are two options:
// 1. The user changes the sequence's value by calling setval(...)
// 2. The user performs a schema change to alter the sequences MinValue or MaxValue. In this case, the
// value of the sequence must be restored to the original MinValue or MaxValue transactionally.
// The code below handles the second case.

// The sequence is decreasing and the minvalue is being decreased.
if opts.Increment < 0 && desc.SequenceOpts.MinValue < oldMinValue {
sequenceVal, err := getSequenceValue()
if err != nil {
return err
}

// If the sequence exceeded the old MinValue, it must be changed to start at the old MinValue.
if sequenceVal < oldMinValue {
err := params.p.txn.Put(params.ctx, seqValueKey, oldMinValue)
if err != nil {
return err
}
}
} else if opts.Increment > 0 && desc.SequenceOpts.MaxValue > oldMaxValue {
sequenceVal, err := getSequenceValue()
if err != nil {
return err
}

if sequenceVal > oldMaxValue {
err := params.p.txn.Put(params.ctx, seqValueKey, oldMaxValue)
if err != nil {
return err
}
}
}

if err := params.p.writeSchemaChange(
params.ctx, n.seqDesc, descpb.InvalidMutationID, tree.AsStringWithFQNames(n.n, params.Ann()),
Expand Down
16 changes: 16 additions & 0 deletions pkg/sql/catalog/descpb/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -320,6 +320,22 @@ func (opts *TableDescriptor_SequenceOpts) HasOwner() bool {
return !opts.SequenceOwner.Equal(TableDescriptor_SequenceOpts_SequenceOwner{})
}

// EffectiveCacheSize returns the CacheSize field of a sequence option with
// the exception that it will return 1 if the CacheSize field is 0.
// A cache size of 1 indicates that there is no caching. The returned value
// will always be greater than or equal to 1.
//
// Prior to #51259, sequence caching was unimplemented and cache sizes were
// left uninitialized (ie. to have a value of 0). If a sequence has a cache
// size of 0, it should be treated in the same was as sequences with cache
// sizes of 1.
func (opts *TableDescriptor_SequenceOpts) EffectiveCacheSize() int64 {
if opts.CacheSize == 0 {
return 1
}
return opts.CacheSize
}

// SafeValue implements the redact.SafeValue interface.
func (ConstraintValidity) SafeValue() {}

Expand Down
Loading

0 comments on commit bededc2

Please sign in to comment.