Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
83616: sql, sqlstats: create temporary stats container for all txns r=xinhaoz a=xinhaoz

# Commit 1
### sql: add txn_fingerprint_id to node_statement_statistics

This commit adds the `txn_fingerprint_id` column to
`crdb_internal.node_statement_statistics`.

Release note (sql): `txn_fingerprin_id` has been added to
`crdb_internal.node_statement_statistics`. The type of the
column is NULL or STRING.

------------------------------
# Commit 2
### sql, sqlstats: create temporary stats container for all txns

Fixes: #81470

Previously, the stats collector followed different procedures for stats
collection depending on whether or not the txn was explicit.

For explicit transactions, all the stmts in the txn must be recorded with
the same `transactionFingerprintID`, which is  only known after all stmts
in the txn have been executed. In order to record the correct
txnFingerprintID, a temporary stats container was created for stmts in the
current transaction. The `transactionFingerprintID` was then populated for
all stmts in the temp container, and the temp container was merged with
the parent.

For implict transactions, the assumption was there would only be a single
stmt in the txn, and so no temporary container was created, with stmts
being written directly to the application stats.

This assumption was incorrect, as it is possible for implicit txns to have
multiple stmts, such as stmts sent in a batch.

This commit ensures that stats are properly collected for implicit txns
with multiple stmts. The stats collector now follows the same procedure
for both explicit and implicit txns, creating a temporary container for
local txn stmts and merging on txn finish.

Release note (bug fix): Statement and transaction stats are now properly
recorded for implicit txns with multiple stmts.

83762: docs: add MVCC range tombstones tech note r=sumeerbhola,jbowens,nicktrav a=erikgrinaker

[Rendered version](https://github.com/erikgrinaker/cockroach/blob/mvcc-range-tombstones-tech-note/docs/tech-notes/mvcc-range-tombstones.md)

---

This describes the current state, but the details will likely change as
we iterate on the implementation.

Resolves #83406.

Release note: None

83873: tenantsettingswatcher: remove the version gate r=yuzefovich a=yuzefovich

This commit removes the version gate for the tenant settings.

Release note: None

83902: server: remove TLS cert data retrieval over HTTP r=catj-cockroach a=knz

Back in CockroachDB v1.1 (v17.2 in the new calver scheme), we
introduced a certificate rotation mechanism. To help
teach/troubleshoot that feature, we also provided a way for the
operator to view the certificate details in the DB Console (expiration
time, addresses, etc.)

This work was done in PR #16087, to solve issues #15027/#1674.

However, as part of that PR, the implementation of the back-end API
also included the *data* of the cert (including the cert signature
and the signature chain) in the response payload.

This additional payload was never used in a user-facing feature: the
DB Console does not display it nor does it contain a link to "download
the cert file". The back-end API is not public either, so we are not
expecting end-users to have legitimate uses for this feature.

Meanwhile, leaking cert data through an API runs dangerously close
to violating PCI guidelines (not quite, since keys are not exposed,
but still...).

So in order to avoid a remark on this during PCI review cycles, and to
remove the chance this will be misused, this patch removes the
data payload from the cert response.

The DB Console screen corresponding to the original work remains
unaffected.

For reference here's how the console screen looks: 
![image](https://user-images.githubusercontent.com/642886/177591040-f554fdf0-2b04-48f6-af36-0b94c0bcaf4c.png)


Co-authored-by: Xin Hao Zhang <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
5 people committed Jul 6, 2022
5 parents 0848d90 + 607026a + 5e1cfb5 + ad06d0c + 8b07764 commit ce1b42b
Show file tree
Hide file tree
Showing 26 changed files with 759 additions and 360 deletions.
1 change: 0 additions & 1 deletion docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@ Support status: [reserved](#support-status)
| ----- | ---- | ----- | ----------- | -------------- |
| type | [CertificateDetails.CertificateType](#cockroach.server.serverpb.CertificatesResponse-cockroach.server.serverpb.CertificateDetails.CertificateType) | | | [reserved](#support-status) |
| error_message | [string](#cockroach.server.serverpb.CertificatesResponse-string) | | "error_message" and "data" are mutually exclusive. | [reserved](#support-status) |
| data | [bytes](#cockroach.server.serverpb.CertificatesResponse-bytes) | | data is the raw file contents of the certificate. This means PEM-encoded DER data. | [reserved](#support-status) |
| fields | [CertificateDetails.Fields](#cockroach.server.serverpb.CertificatesResponse-cockroach.server.serverpb.CertificateDetails.Fields) | repeated | | [reserved](#support-status) |


Expand Down
605 changes: 605 additions & 0 deletions docs/tech-notes/mvcc-range-tombstones.md

Large diffs are not rendered by default.

4 changes: 2 additions & 2 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -195,10 +195,10 @@ SELECT * FROM crdb_internal.leases WHERE node_id < 0
----
node_id table_id name parent_id expiration deleted

query ITTTTTIIITRRRRRRRRRRRRRRRRRRRRRRRRRRBBTTT colnames
query ITTTTTIIITRRRRRRRRRRRRRRRRRRRRRRRRRRBBTTTT colnames
SELECT * FROM crdb_internal.node_statement_statistics WHERE node_id < 0
----
node_id application_name flags statement_id key anonymized count first_attempt_count max_retries last_error rows_avg rows_var parse_lat_avg parse_lat_var plan_lat_avg plan_lat_var run_lat_avg run_lat_var service_lat_avg service_lat_var overhead_lat_avg overhead_lat_var bytes_read_avg bytes_read_var rows_read_avg rows_read_var network_bytes_avg network_bytes_var network_msgs_avg network_msgs_var max_mem_usage_avg max_mem_usage_var max_disk_usage_avg max_disk_usage_var contention_time_avg contention_time_var implicit_txn full_scan sample_plan database_name exec_node_ids
node_id application_name flags statement_id key anonymized count first_attempt_count max_retries last_error rows_avg rows_var parse_lat_avg parse_lat_var plan_lat_avg plan_lat_var run_lat_avg run_lat_var service_lat_avg service_lat_var overhead_lat_avg overhead_lat_var bytes_read_avg bytes_read_var rows_read_avg rows_read_var network_bytes_avg network_bytes_var network_msgs_avg network_msgs_var max_mem_usage_avg max_mem_usage_var max_disk_usage_avg max_disk_usage_var contention_time_avg contention_time_var implicit_txn full_scan sample_plan database_name exec_node_ids txn_fingerprint_id

query ITTTIIRRRRRRRRRRRRRRRRRR colnames
SELECT * FROM crdb_internal.node_transaction_statistics WHERE node_id < 0
Expand Down
6 changes: 0 additions & 6 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -263,8 +263,6 @@ const (
// on lease transfer Raft proposals. New leaseholders now forward their clock
// directly to the new lease start time.
DontProposeWriteTimestampForLeaseTransfers
// TenantSettingsTable adds the system table for tracking tenant usage.
TenantSettingsTable
// EnablePebbleFormatVersionBlockProperties enables a new Pebble SSTable
// format version for block property collectors.
// NB: this cluster version is paired with PebbleFormatBlockPropertyCollector
Expand Down Expand Up @@ -540,10 +538,6 @@ var versionsSingleton = keyedVersions{
Key: DontProposeWriteTimestampForLeaseTransfers,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 60},
},
{
Key: TenantSettingsTable,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 62},
},
{
Key: EnablePebbleFormatVersionBlockProperties,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 64},
Expand Down
77 changes: 38 additions & 39 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

4 changes: 1 addition & 3 deletions pkg/server/serverpb/status.proto
Original file line number Diff line number Diff line change
Expand Up @@ -73,9 +73,7 @@ message CertificateDetails {
CertificateType type = 1;
// "error_message" and "data" are mutually exclusive.
string error_message = 2;
// data is the raw file contents of the certificate. This means PEM-encoded
// DER data.
bytes data = 3;
reserved 3;
repeated Fields fields = 4 [ (gogoproto.nullable) = false ];
}

Expand Down
3 changes: 1 addition & 2 deletions pkg/server/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -938,8 +938,7 @@ func (s *statusServer) Certificates(
}

if cert.Error == nil {
details.Data = cert.FileContents
if err := extractCertFields(details.Data, &details); err != nil {
if err := extractCertFields(cert.FileContents, &details); err != nil {
details.ErrorMessage = err.Error()
}
} else {
Expand Down
20 changes: 0 additions & 20 deletions pkg/server/status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,8 +38,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server/diagnostics/diagnosticspb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
Expand Down Expand Up @@ -1320,37 +1318,19 @@ func TestCertificatesResponse(t *testing.T) {
t.Errorf("expected %d certificates, found %d", e, a)
}

// Read the certificates from the embedded assets.
caPath := filepath.Join(security.EmbeddedCertsDir, security.EmbeddedCACert)
nodePath := filepath.Join(security.EmbeddedCertsDir, security.EmbeddedNodeCert)

caFile, err := securitytest.EmbeddedAssets.ReadFile(caPath)
if err != nil {
t.Fatal(err)
}

nodeFile, err := securitytest.EmbeddedAssets.ReadFile(nodePath)
if err != nil {
t.Fatal(err)
}

// The response is ordered: CA cert followed by node cert.
cert := response.Certificates[0]
if a, e := cert.Type, serverpb.CertificateDetails_CA; a != e {
t.Errorf("wrong type %s, expected %s", a, e)
} else if cert.ErrorMessage != "" {
t.Errorf("expected cert without error, got %v", cert.ErrorMessage)
} else if a, e := cert.Data, caFile; !bytes.Equal(a, e) {
t.Errorf("mismatched contents: %s vs %s", a, e)
}

cert = response.Certificates[1]
if a, e := cert.Type, serverpb.CertificateDetails_NODE; a != e {
t.Errorf("wrong type %s, expected %s", a, e)
} else if cert.ErrorMessage != "" {
t.Errorf("expected cert without error, got %v", cert.ErrorMessage)
} else if a, e := cert.Data, nodeFile; !bytes.Equal(a, e) {
t.Errorf("mismatched contents: %s vs %s", a, e)
}
}

Expand Down
4 changes: 0 additions & 4 deletions pkg/server/tenantsettingswatcher/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/server/tenantsettingswatcher",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/keys",
"//pkg/kv/kvclient/rangefeed",
"//pkg/kv/kvclient/rangefeed/rangefeedbuffer",
Expand Down Expand Up @@ -44,7 +43,6 @@ go_test(
embed = [":tenantsettingswatcher"],
deps = [
"//pkg/base",
"//pkg/clusterversion",
"//pkg/keys",
"//pkg/roachpb",
"//pkg/security/securityassets",
Expand All @@ -53,12 +51,10 @@ go_test(
"//pkg/settings",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
],
)
67 changes: 6 additions & 61 deletions pkg/server/tenantsettingswatcher/watcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,9 +12,7 @@ package tenantsettingswatcher

import (
"context"
"sync"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer"
Expand Down Expand Up @@ -78,56 +76,14 @@ func New(

// Start will start the Watcher.
//
// If the current cluster version indicates that we have a tenant settings
// table, this function sets up the rangefeed and waits for the initial scan. An
// error will be returned if the initial table scan hits an error, the context
// is canceled or the stopper is stopped prior to the initial data being
// retrieved.
//
// Otherwise, Start sets up a background task that waits for the right version
// and starts the rangefeed when appropriate. WaitUntilStarted can be used to
// wait for the rangefeed setup.
// This function sets up the rangefeed and waits for the initial scan. An error
// will be returned if the initial table scan hits an error, the context is
// canceled or the stopper is stopped prior to the initial data being retrieved.
func (w *Watcher) Start(ctx context.Context, sysTableResolver catalog.SystemTableIDResolver) error {
w.startCh = make(chan struct{})
if w.st.Version.IsActive(ctx, clusterversion.TenantSettingsTable) {
// We are not in a mixed-version scenario; start the rangefeed now.
w.startErr = w.startRangeFeed(ctx, sysTableResolver)
close(w.startCh)
return w.startErr
}
// Set up an on-change callback that closes this channel once the version
// supports tenant settings.
versionOkCh := make(chan struct{})
var once sync.Once
w.st.Version.SetOnChange(func(ctx context.Context, newVersion clusterversion.ClusterVersion) {
if newVersion.IsActive(clusterversion.TenantSettingsTable) {
once.Do(func() {
close(versionOkCh)
})
}
})
// Now check the version again, in case the version changed just before
// SetOnChange.
if w.st.Version.IsActive(ctx, clusterversion.TenantSettingsTable) {
w.startErr = w.startRangeFeed(ctx, sysTableResolver)
close(w.startCh)
return w.startErr
}
return w.stopper.RunAsyncTask(ctx, "tenantsettingswatcher-start", func(ctx context.Context) {
log.Infof(ctx, "tenantsettingswatcher waiting for the appropriate version")
select {
case <-versionOkCh:
case <-w.stopper.ShouldQuiesce():
return
}
log.Infof(ctx, "tenantsettingswatcher can now start")
w.startErr = w.startRangeFeed(ctx, sysTableResolver)
if w.startErr != nil {
// We are not equipped to handle this error asynchronously.
log.Warningf(ctx, "error starting tenantsettingswatcher rangefeed: %v", w.startErr)
}
close(w.startCh)
})
w.startErr = w.startRangeFeed(ctx, sysTableResolver)
close(w.startCh)
return w.startErr
}

// startRangeFeed starts the range feed and waits for the initial table scan. An
Expand Down Expand Up @@ -237,11 +193,6 @@ func (w *Watcher) startRangeFeed(

// WaitForStart waits until the rangefeed is set up. Returns an error if the
// rangefeed setup failed.
//
// If the cluster version does not support tenant settings, returns immediately
// with no error. Note that it is still legal to call GetTenantOverrides and
// GetAllTenantOverrides in this state. When the cluster version is upgraded,
// the settings will start being updated.
func (w *Watcher) WaitForStart(ctx context.Context) error {
// Fast path check.
select {
Expand All @@ -252,12 +203,6 @@ func (w *Watcher) WaitForStart(ctx context.Context) error {
if w.startCh == nil {
return errors.AssertionFailedf("Start() was not yet called")
}
if !w.st.Version.IsActive(ctx, clusterversion.TenantSettingsTable) {
// If this happens, then we are running new tenant code against a host
// cluster that was not fully upgraded.
log.Warningf(ctx, "tenant requested settings before host cluster version upgrade")
return nil
}
select {
case <-w.startCh:
return w.startErr
Expand Down
Loading

0 comments on commit ce1b42b

Please sign in to comment.