Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
82352: server, sql: surface session txnCount, recent txn fingerprints, active time r=xinhaoz a=xinhaoz

Finishing up Gerardo's PR, original review here: #80717

--------------------------------------------------
Partially addresses #74257.

Previously, the status server did not provide session details such as
total number of transactions executed, transaction fingerprint
IDs, and total active time. This change adds the aforementioned session
details to the `serverpb.Session` struct.

To track recently executed transaction fingerprint IDs, a FIFO cache
`TxnFingerprintIDCache` is introduced with its corresponding cluster
setting `TxnFingerprintIDBufferCapacity` to control the capacity. The
default capacity is set at 100 fingerprints.

The total number of transactions executed is filled using the existing
`txnCounter` from the `extraTxnState` in `connExecutor`. The total active
time is calculated by introducing a `timeutil.StopWatch` to the connection
executor, which is started and stopped when a transaction is started and
finished respectively.

Release note (api change): the `serverpb.Session` struct now has three
new fields: number of transactions executed, transaction fingerprint
IDs, and total active time.

82623: backupinfo: introduce a backupinfo package r=stevendanna a=adityamaru

The backupinfo package contains logic related to interacting
with information and metadata describing the backup. After this
change we have `backupdest` depending on `backupinfo`.

Release note: None

82718: kvserver: emit MVCC range tombstones over rangefeeds r=aliher1911 a=erikgrinaker

This patch adds MVCC range tombstone support in rangefeeds. Whenever an
MVCC range tombstone is written, a new `MVCCDeleteRangeOp` logical op
is recorded and emitted across the rangefeed as a `RangeFeedDeleteRange`
event. MVCC range tombstones will only be written when the
`MVCCRangeTombstones` version gate has been enabled.

Changefeeds will emit an error for these events. We do not expect to see
these in online spans with changefeeds, since they are initially only
planned for use with schema GC and import rollbacks.

The rangefeed client library has been extended with support for these
events, but no existing callers handle them for the same reason as
changefeeds. Initial scans do not emit regular tombstones, and thus not
range tombstones either, but catchup scans will emit them if
encountered.

This patch has rudimentary testing of MVCC range tombstones in
rangefeeds. A later patch will add a data-driven test harness for
rangefeeds with more exhaustive tests.

Resolves #82449.
Touches #70433.

Release note: None

82936: sql/schemachanger: implement DROP OWNED BY r=jasonmchan a=jasonmchan

Previously, we did not support the DROP OWNED BY statement (#55381).
This commit adds partial support for DROP OWNED BY in the declarative
schema changer. Followup work is needed to support the CASCADE modifier.

Release note (sql change): Support `DROP OWNED BY`.

83229: ui: remove option 10/30 min from SQL Activity page r=maryliag a=maryliag

Note to reviewers: only 2nd commit is relevant to this PR

Previously we had the options for 10 and 30min on
SQL Activity pages, which created some confusion, since
we would always show the last 1h info.
This commit remove those 2 options.
If the user select any of those options on the Metrics
page, it will get updated to 1h on the SQL Activity
pages.

<img width="444" alt="Screen Shot 2022-06-22 at 5 43 53 PM" src="https://user-images.githubusercontent.com/1017486/175144243-2f084e0b-5e09-4874-9640-e7eea6179343.png">

https://www.loom.com/share/226e54322df6456aa2039b5c54f72eb1


Fixes #82914

Release note (ui change): Removal of the 10 and 30min options
on the SQL Activity page.

83420: ui: improve tooltip UX with text updates r=ericharmeling a=ericharmeling

Fixes #81374.
Fixes #83256.
Fixes #81248.
Fixes #79018.

Note the following:

- The updates resolving #79018 effectively revert the tooltip text for Rows Read to the original wording (which [was updated for accuracy](e379e9d#diff-492398441e971e355a687a4ce333a9766e2195287d0227682444d5dc0eb7ee1a)). I assume this is okay. `@kevin-v-ngo`
- The updates resolving #81248 do not in fact refer to the time intervals as date ranges, as this language is misleading (a 1h interval is an interval and not a date range). Instead, this update just removes the anchor and the link to the non-existent Interval Range section of https://www.cockroachlabs.com/docs/stable/ui-statements-page.html. We may want to consider updating the docs to call the "time picker" data type a time interval and not a date range. This appears to have been the case in previous releases (https://www.cockroachlabs.com/docs/v21.1/ui-statements-page#time-interval). `@stbof` 

Release note (ui change): Updated tooltips on the Statements and
Transactions pages in the DB Console for improved UX.

83428: sql: rename anonymizedStmt in sqlstats pkg to stmtNoConstants r=ericharmeling a=ericharmeling

Note that this commit does not change any files outside the sqlstats package.

Fixes #80725.

Release note: None

83468: ui: update all dates to use same format r=maryliag a=maryliag

Update all dates to use the same format.

Fixes #81159

Release note: None

83520: kv: don't try to reject lease transfer when flushing proposal buffer r=nvanbenschoten a=nvanbenschoten

Fixes #83498.
Fixes #83402.
Fixes #83308.

This was fallout from #82758.

This commit adds logic to `propBuf.maybeRejectUnsafeProposalLocked` to avoid
trying to reject proposals based on the state of the raft group when the group
is not provided (e.g. when flushing the buffer). We already had this logic for
`RequestLease` (indirectly), but did not for `TransferLease`.

Co-authored-by: Gerardo Torres <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Jason Chan <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
Co-authored-by: Eric Harmeling <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
8 people committed Jun 28, 2022
10 parents 969bcf7 + ceb5981 + a743d82 + 566dff2 + 421856b + e4d7f25 + b3571fe + c9267e2 + b44d6ec + 5d4c0d7 commit 5541cf8
Show file tree
Hide file tree
Showing 112 changed files with 4,273 additions and 695 deletions.
6 changes: 6 additions & 0 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -2111,6 +2111,9 @@ Session represents one SQL session.
| last_active_query_no_constants | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | The SQL statement fingerprint of the last query executed on this session, compatible with StatementStatisticsKey. | [reserved](#support-status) |
| status | [Session.Status](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.Session.Status) | | The session's status. | [reserved](#support-status) |
| end | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | Timestamp of session's end. | [reserved](#support-status) |
| num_txns_executed | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | Count of the number of transactions that have been opened on this session. This count includes transactions that are in progress. | [reserved](#support-status) |
| txn_fingerprint_ids | [uint64](#cockroach.server.serverpb.ListSessionsResponse-uint64) | repeated | List of transaction fingerprint IDs in this session. | [reserved](#support-status) |
| total_active_time | [google.protobuf.Duration](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Duration) | | The session's total active time. | [reserved](#support-status) |



Expand Down Expand Up @@ -2247,6 +2250,9 @@ Session represents one SQL session.
| last_active_query_no_constants | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | The SQL statement fingerprint of the last query executed on this session, compatible with StatementStatisticsKey. | [reserved](#support-status) |
| status | [Session.Status](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.Session.Status) | | The session's status. | [reserved](#support-status) |
| end | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | Timestamp of session's end. | [reserved](#support-status) |
| num_txns_executed | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | Count of the number of transactions that have been opened on this session. This count includes transactions that are in progress. | [reserved](#support-status) |
| txn_fingerprint_ids | [uint64](#cockroach.server.serverpb.ListSessionsResponse-uint64) | repeated | List of transaction fingerprint IDs in this session. | [reserved](#support-status) |
| total_active_time | [google.protobuf.Duration](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Duration) | | The session's total active time. | [reserved](#support-status) |



Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -273,6 +273,7 @@ sql.ttl.default_delete_rate_limit integer 0 default delete rate limit for all TT
sql.ttl.default_range_concurrency integer 1 default amount of ranges to process at once during a TTL delete
sql.ttl.default_select_batch_size integer 500 default amount of rows to select in a single query during a TTL job
sql.ttl.job.enabled boolean true whether the TTL job is enabled
sql.txn_fingerprint_id_cache.capacity integer 100 the maximum number of txn fingerprint IDs stored
timeseries.storage.enabled boolean true if set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhere
timeseries.storage.resolution_10s.ttl duration 240h0m0s the maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion.
timeseries.storage.resolution_30m.ttl duration 2160h0m0s the maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion.
Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,7 @@
<tr><td><code>sql.ttl.default_range_concurrency</code></td><td>integer</td><td><code>1</code></td><td>default amount of ranges to process at once during a TTL delete</td></tr>
<tr><td><code>sql.ttl.default_select_batch_size</code></td><td>integer</td><td><code>500</code></td><td>default amount of rows to select in a single query during a TTL job</td></tr>
<tr><td><code>sql.ttl.job.enabled</code></td><td>boolean</td><td><code>true</code></td><td>whether the TTL job is enabled</td></tr>
<tr><td><code>sql.txn_fingerprint_id_cache.capacity</code></td><td>integer</td><td><code>100</code></td><td>the maximum number of txn fingerprint IDs stored</td></tr>
<tr><td><code>timeseries.storage.enabled</code></td><td>boolean</td><td><code>true</code></td><td>if set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhere</td></tr>
<tr><td><code>timeseries.storage.resolution_10s.ttl</code></td><td>duration</td><td><code>240h0m0s</code></td><td>the maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion.</td></tr>
<tr><td><code>timeseries.storage.resolution_30m.ttl</code></td><td>duration</td><td><code>2160h0m0s</code></td><td>the maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion.</td></tr>
Expand Down
29 changes: 29 additions & 0 deletions docs/generated/swagger/spec.json
Original file line number Diff line number Diff line change
Expand Up @@ -773,6 +773,12 @@
},
"x-go-package": "github.com/cockroachdb/cockroach/pkg/server/serverpb"
},
"Duration": {
"description": "A Duration represents the elapsed time between two instants\nas an int64 nanosecond count. The representation limits the\nlargest representable duration to approximately 290 years.",
"type": "integer",
"format": "int64",
"x-go-package": "time"
},
"EventsResponse": {
"description": "EventsResponse contains a set of event log entries. This is always limited\nto the latest N entries (N is enforced in the associated endpoint).",
"type": "object",
Expand Down Expand Up @@ -1225,6 +1231,12 @@
"node_id": {
"$ref": "#/definitions/NodeID"
},
"num_txns_executed": {
"description": "Count of the number of transactions that have been opened on this session.\nThis count includes transactions that are in progress.",
"type": "integer",
"format": "int32",
"x-go-name": "NumTxnsExecuted"
},
"start": {
"description": "Timestamp of session's start.",
"type": "string",
Expand All @@ -1234,6 +1246,17 @@
"status": {
"$ref": "#/definitions/Session_Status"
},
"total_active_time": {
"$ref": "#/definitions/Duration"
},
"txn_fingerprint_ids": {
"description": "List of transaction fingerprint IDs in this session.",
"type": "array",
"items": {
"$ref": "#/definitions/TransactionFingerprintID"
},
"x-go-name": "TxnFingerprintIDs"
},
"username": {
"description": "Username of the user for this session.",
"type": "string",
Expand Down Expand Up @@ -1498,6 +1521,12 @@
},
"x-go-package": "github.com/cockroachdb/cockroach/pkg/util/hlc"
},
"TransactionFingerprintID": {
"description": "TransactionFingerprintID is the hashed string constructed using the\nindividual statement fingerprint IDs that comprise the transaction.",
"type": "integer",
"format": "uint64",
"x-go-package": "github.com/cockroachdb/cockroach/pkg/roachpb"
},
"TxnInfo": {
"type": "object",
"title": "TxnInfo represents an in flight user transaction on some Session.",
Expand Down
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@ ALL_TESTS = [
"//pkg/build/starlarkutil:starlarkutil_test",
"//pkg/build/util:util_test",
"//pkg/ccl/backupccl/backupdest:backupdest_test",
"//pkg/ccl/backupccl/backupinfo:backupinfo_disallowed_imports_test",
"//pkg/ccl/backupccl/backupinfo:backupinfo_test",
"//pkg/ccl/backupccl/backupresolver:backupresolver_test",
"//pkg/ccl/backupccl:backupccl_test",
"//pkg/ccl/baseccl:baseccl_test",
Expand Down
7 changes: 2 additions & 5 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,6 @@ go_library(
srcs = [
"alter_backup_planning.go",
"backup_job.go",
"backup_metadata.go",
"backup_planning.go",
"backup_planning_tenant.go",
"backup_processor.go",
Expand All @@ -14,7 +13,6 @@ go_library(
"create_scheduled_backup.go",
"file_sst_sink.go",
"key_rewriter.go",
"manifest_handling.go",
"restoration_data.go",
"restore_data_processor.go",
"restore_job.go",
Expand All @@ -37,6 +35,7 @@ go_library(
"//pkg/ccl/backupccl/backupbase",
"//pkg/ccl/backupccl/backupdest",
"//pkg/ccl/backupccl/backupencryption",
"//pkg/ccl/backupccl/backupinfo",
"//pkg/ccl/backupccl/backuppb",
"//pkg/ccl/backupccl/backupresolver",
"//pkg/ccl/backupccl/backuputils",
Expand Down Expand Up @@ -112,10 +111,8 @@ go_library(
"//pkg/util/admission/admissionpb",
"//pkg/util/contextutil",
"//pkg/util/ctxgroup",
"//pkg/util/encoding",
"//pkg/util/hlc",
"//pkg/util/interval",
"//pkg/util/ioctx",
"//pkg/util/json",
"//pkg/util/log",
"//pkg/util/log/eventpb",
Expand Down Expand Up @@ -144,7 +141,6 @@ go_test(
"alter_backup_test.go",
"backup_cloud_test.go",
"backup_intents_test.go",
"backup_metadata_test.go",
"backup_planning_test.go",
"backup_rand_test.go",
"backup_tenant_test.go",
Expand Down Expand Up @@ -178,6 +174,7 @@ go_test(
"//pkg/ccl/backupccl/backupbase",
"//pkg/ccl/backupccl/backupdest",
"//pkg/ccl/backupccl/backupencryption",
"//pkg/ccl/backupccl/backupinfo",
"//pkg/ccl/backupccl/backuppb",
"//pkg/ccl/kvccl",
"//pkg/ccl/multiregionccl",
Expand Down
43 changes: 23 additions & 20 deletions pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupdest"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
Expand Down Expand Up @@ -243,7 +244,7 @@ func backup(
RevisionStartTime: backupManifest.RevisionStartTime,
})

err := writeBackupManifestCheckpoint(
err := backupinfo.WriteBackupManifestCheckpoint(
ctx, defaultURI, encryption, backupManifest, execCtx.ExecCfg(), execCtx.User(),
)
if err != nil {
Expand Down Expand Up @@ -290,8 +291,8 @@ func backup(
// Set a unique filename for each partition backup descriptor. The ID
// ensures uniqueness, and the kv string appended to the end is for
// readability.
filename := fmt.Sprintf("%s_%d_%s",
backupPartitionDescriptorPrefix, nextPartitionedDescFilenameID, sanitizeLocalityKV(kv))
filename := fmt.Sprintf("%s_%d_%s", backupPartitionDescriptorPrefix,
nextPartitionedDescFilenameID, backupinfo.SanitizeLocalityKV(kv))
nextPartitionedDescFilenameID++
backupManifest.PartitionDescriptorFilenames = append(backupManifest.PartitionDescriptorFilenames, filename)
desc := backuppb.BackupPartitionDescriptor{
Expand All @@ -306,15 +307,16 @@ func backup(
return err
}
defer store.Close()
return writeBackupPartitionDescriptor(ctx, store, filename, encryption, &desc)
return backupinfo.WriteBackupPartitionDescriptor(ctx, store, filename, encryption, &desc)
}(); err != nil {
return roachpb.RowCount{}, err
}
}
}

resumerSpan.RecordStructured(&types.StringValue{Value: "writing backup manifest"})
if err := writeBackupManifest(ctx, settings, defaultStore, backupbase.BackupManifestName, encryption, backupManifest); err != nil {
if err := backupinfo.WriteBackupManifest(ctx, settings, defaultStore, backupbase.BackupManifestName,
encryption, backupManifest); err != nil {
return roachpb.RowCount{}, err
}
var tableStatistics []*stats.TableStatisticProto
Expand Down Expand Up @@ -344,12 +346,12 @@ func backup(
}

resumerSpan.RecordStructured(&types.StringValue{Value: "writing backup table statistics"})
if err := writeTableStatistics(ctx, defaultStore, backupStatisticsFileName, encryption, &statsTable); err != nil {
if err := backupinfo.WriteTableStatistics(ctx, defaultStore, encryption, &statsTable); err != nil {
return roachpb.RowCount{}, err
}

if writeMetadataSST.Get(&settings.SV) {
if err := writeBackupMetadataSST(ctx, defaultStore, encryption, backupManifest, tableStatistics); err != nil {
if backupinfo.WriteMetadataSST.Get(&settings.SV) {
if err := backupinfo.WriteBackupMetadataSST(ctx, defaultStore, encryption, backupManifest, tableStatistics); err != nil {
err = errors.Wrap(err, "writing forward-compat metadata sst")
if !build.IsRelease() {
return roachpb.RowCount{}, err
Expand Down Expand Up @@ -448,7 +450,7 @@ func (b *backupResumer) Resume(ctx context.Context, execCtx interface{}) error {
}
}

if err := writeBackupManifestCheckpoint(
if err := backupinfo.WriteBackupManifestCheckpoint(
ctx, details.URI, details.EncryptionOptions, backupManifest, p.ExecCfg(), p.User(),
); err != nil {
return err
Expand Down Expand Up @@ -730,20 +732,21 @@ func (b *backupResumer) readManifestOnResume(
// they could be using either the new or the old foreign key
// representations. We should just preserve whatever representation the
// table descriptors were using and leave them alone.
desc, memSize, err := readBackupCheckpointManifest(ctx, mem, defaultStore, backupManifestCheckpointName,
details.EncryptionOptions)
desc, memSize, err := backupinfo.ReadBackupCheckpointManifest(ctx, mem, defaultStore,
backupinfo.BackupManifestCheckpointName, details.EncryptionOptions)
if err != nil {
if !errors.Is(err, cloud.ErrFileDoesNotExist) {
return nil, 0, errors.Wrapf(err, "reading backup checkpoint")
}
// Try reading temp checkpoint.
tmpCheckpoint := tempCheckpointFileNameForJob(b.job.ID())
desc, memSize, err = readBackupCheckpointManifest(ctx, mem, defaultStore, tmpCheckpoint, details.EncryptionOptions)
tmpCheckpoint := backupinfo.TempCheckpointFileNameForJob(b.job.ID())
desc, memSize, err = backupinfo.ReadBackupCheckpointManifest(ctx, mem, defaultStore,
tmpCheckpoint, details.EncryptionOptions)
if err != nil {
return nil, 0, err
}
// "Rename" temp checkpoint.
if err := writeBackupManifestCheckpoint(
if err := backupinfo.WriteBackupManifestCheckpoint(
ctx, details.URI, details.EncryptionOptions, &desc, cfg, user,
); err != nil {
mem.Shrink(ctx, memSize)
Expand All @@ -753,8 +756,8 @@ func (b *backupResumer) readManifestOnResume(
if err := defaultStore.Delete(ctx, tmpCheckpoint); err != nil {
log.Errorf(ctx, "error removing temporary checkpoint %s", tmpCheckpoint)
}
if err := defaultStore.Delete(ctx, backupProgressDirectory+"/"+tmpCheckpoint); err != nil {
log.Errorf(ctx, "error removing temporary checkpoint %s", backupProgressDirectory+"/"+tmpCheckpoint)
if err := defaultStore.Delete(ctx, backupinfo.BackupProgressDirectory+"/"+tmpCheckpoint); err != nil {
log.Errorf(ctx, "error removing temporary checkpoint %s", backupinfo.BackupProgressDirectory+"/"+tmpCheckpoint)
}
}

Expand Down Expand Up @@ -850,18 +853,18 @@ func (b *backupResumer) deleteCheckpoint(
defer exportStore.Close()
// We first attempt to delete from base directory to account for older
// backups, and then from the progress directory.
err = exportStore.Delete(ctx, backupManifestCheckpointName)
err = exportStore.Delete(ctx, backupinfo.BackupManifestCheckpointName)
if err != nil {
log.Warningf(ctx, "unable to delete checkpointed backup descriptor file in base directory: %+v", err)
}
err = exportStore.Delete(ctx, backupManifestCheckpointName+backupManifestChecksumSuffix)
err = exportStore.Delete(ctx, backupinfo.BackupManifestCheckpointName+backupinfo.BackupManifestChecksumSuffix)
if err != nil {
log.Warningf(ctx, "unable to delete checkpoint checksum file in base directory: %+v", err)
}
// Delete will not delete a nonempty directory, so we have to go through
// all files and delete each file one by one.
return exportStore.List(ctx, backupProgressDirectory, "", func(p string) error {
return exportStore.Delete(ctx, backupProgressDirectory+p)
return exportStore.List(ctx, backupinfo.BackupProgressDirectory, "", func(p string) error {
return exportStore.Delete(ctx, backupinfo.BackupProgressDirectory+p)
})
}(); err != nil {
log.Warningf(ctx, "unable to delete checkpointed backup descriptor file in progress directory: %+v", err)
Expand Down
Loading

0 comments on commit 5541cf8

Please sign in to comment.