Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
83123: sql/stats: conversion of datums to and from quantile function values r=yuzefovich,rytaft,mgartner a=michae2

To predict histograms in statistics forecasts, we will use linear
regression over quantile functions. (Quantile functions are another
representation of histogram data, in a form more amenable to statistical
manipulation.)

The conversion of histograms to quantile functions will require
conversion of histogram bounds (datums) to quantile values (float64s).
And likewise, the inverse conversion from quantile functions back to
histograms will require the inverse conversion of float64 quantile
values back to datums. These conversions are a little different from our
usual SQL conversions in `eval.PerformCast`, so we add them to a new
quantile file in the `sql/stats` module.

This code was originally part of #77070 but has been pulled out to
simplify that PR. A few changes have been made:
- `histogramValue` has been renamed to `FromQuantileValue`.
- Support for `DECIMAL`, `TIME`, `TIMETZ`, and `INTERVAL` has been
  dropped. Clamping these types in `FromQuantileValue` was too complex
  for the first iteration of statistics forecasting. We expect the
  overwhelming majority of ascending keys to use `INT` or `TIMESTAMP`
  types.
- Bugs in `FLOAT4`, `TIMESTAMP` and `TIMESTAMPTZ` conversions have been
  fixed.
- We're now clamping timestamps to slightly tighter bounds to avoid the
  problems with infinite timestamps (see #41564).

Assists: #79872

Release note: None

83590: sql: fix and rename sql stats session transaction received time r=THardy98 a=THardy98

Resolves: #82894

Due to a change from #76792, implicit transactions can start before
`SessionQueryReceived` session phase time is set by the sqlstats system.
In turn, this caused the `SessionTransactionReceived` (now renamed as
`SessionTransactionStarted`) session phase time to be recorded
incorrectly, causing extremely large transactions times on the UI. This
change fixes this mistake by setting the actual transaction start time
as the `SessionTransactionStarted` session phase time, instead of
`SessionQueryReceived`.

Release note (bug fix): The `SessionTransactionReceived` session phase
time is no longer recorded incorrectly, fixing large transaction times
from appearing on the UI, also renamed to `SessionTransactionStarted`.

83943: outliers: protocol buffer adjustments r=matthewtodd a=matthewtodd

This handful of commits slightly re-shapes the protocol buffer messages we use for tracking outliers, making some of the work in #81021 a little easier to express.

83961: cmd/dev: add generate execgen subcommand r=ajwerner a=yuzefovich

Release note: None

83975: streamingccl: unskip TestTenantStreaming r=miretskiy a=stevendanna

This makes a couple of changes aimed at unskipping
TestTenantStreaming:

- Fix a nil pointer access in our stream status verification
  function. We changed the name of key that this function was
  accessing. This NPE was hidden by another panic along the unclean
  shutdown path in the server.

- Lower various intervals so that this test doesn't take 90 seconds.

I've run this under stress for a few hundred iterations without error.

Release note: None

83979: streamingccl: small logging and tracing cleanups r=miretskiy a=stevendanna

- Add a tracing span to ingestion job cutover.
- Move a particularly noisy log message to VInfo(3).
- Prefer log.VInfo to `if log.V(n) {}` in cases where we aren't doing
  expensive argument construction.

Release note: None

Co-authored-by: Michael Erickson <[email protected]>
Co-authored-by: Thomas Hardy <[email protected]>
Co-authored-by: Matthew Todd <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Steven Danna <[email protected]>
  • Loading branch information
6 people committed Jul 7, 2022
7 parents 4dfbe65 + fdc636b + f8f7459 + ccb1eac + faa81af + 5ba670c + 75971c3 commit bce2267
Show file tree
Hide file tree
Showing 27 changed files with 1,010 additions and 165 deletions.
2 changes: 1 addition & 1 deletion dev
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
set -euo pipefail

# Bump this counter to force rebuilding `dev` on all machines.
DEV_VERSION=39
DEV_VERSION=40

THIS_DIR=$(cd "$(dirname "$0")" && pwd)
BINARY_DIR=$THIS_DIR/bin/dev-versions
Expand Down
4 changes: 4 additions & 0 deletions pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/streaming"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -191,6 +192,9 @@ func (s *streamIngestionResumer) Resume(resumeCtx context.Context, execCtx inter
func revertToCutoverTimestamp(
ctx context.Context, execCtx interface{}, ingestionJobID jobspb.JobID,
) error {
ctx, span := tracing.ChildSpan(ctx, "streamingest.revertToCutoverTimestamp")
defer span.Finish()

p := execCtx.(sql.JobExecContext)
db := p.ExecCfg().DB
j, err := p.ExecCfg().JobRegistry.LoadJob(ctx, ingestionJobID)
Expand Down
24 changes: 15 additions & 9 deletions pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,9 +45,10 @@ import (
)

func verifyIngestionStats(t *testing.T, streamID int64, cutoverTime time.Time, stats string) {
fetchValueKey := func(j json2.JSON, key string) json2.JSON {
fetchRequiredValueKey := func(j json2.JSON, key string) json2.JSON {
val, err := j.FetchValKey(key)
require.NoError(t, err)
require.NotNilf(t, val, "expected key %q to in json %q", key, j)
return val
}

Expand All @@ -60,26 +61,25 @@ func verifyIngestionStats(t *testing.T, streamID int64, cutoverTime time.Time, s
statsJSON, err := json2.ParseJSON(stats)
require.NoError(t, err)

ingestionProgress := fetchValueKey(statsJSON, "ingestion_progress")
ingestionProgress := fetchRequiredValueKey(statsJSON, "ingestion_progress")
require.Equal(t, cutoverTime.UnixNano(),
parseInt64(fetchValueKey(fetchValueKey(ingestionProgress, "cutover_time"), "wall_time").String()))
parseInt64(fetchRequiredValueKey(fetchRequiredValueKey(ingestionProgress, "cutover_time"), "wall_time").String()))

partitionProgressIter, err := fetchValueKey(ingestionProgress, "partition_progress").ObjectIter()
partitionProgressIter, err := fetchRequiredValueKey(ingestionProgress, "partition_progress").ObjectIter()
require.NoError(t, err)
for partitionProgressIter.Next() {
require.Less(t, cutoverTime.UnixNano(), parseInt64(fetchValueKey(fetchValueKey(
require.Less(t, cutoverTime.UnixNano(), parseInt64(fetchRequiredValueKey(fetchRequiredValueKey(
partitionProgressIter.Value(), "ingested_timestamp"), "wall_time").String()))
}

require.Equal(t, strconv.Itoa(int(streampb.StreamReplicationStatus_STREAM_INACTIVE)),
fetchValueKey(fetchValueKey(statsJSON, "stream_replication_status"), "stream_status").String())
fetchRequiredValueKey(fetchRequiredValueKey(statsJSON, "producer_status"), "stream_status").String())
}

// TestTenantStreaming tests that tenants can stream changes end-to-end.
func TestTenantStreaming(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
skip.WithIssue(t, 83697)

skip.UnderRace(t, "slow under race")

Expand Down Expand Up @@ -120,6 +120,7 @@ SET CLUSTER SETTING kv.rangefeed.enabled = true;
SET CLUSTER SETTING kv.closed_timestamp.target_duration = '1s';
SET CLUSTER SETTING changefeed.experimental_poll_interval = '10ms';
SET CLUSTER SETTING stream_replication.min_checkpoint_frequency = '1s';
SET CLUSTER SETTING stream_replication.stream_liveness_track_frequency = '500ms';
`,
";")...)

Expand All @@ -129,14 +130,19 @@ SET CLUSTER SETTING stream_replication.min_checkpoint_frequency = '1s';
// is required. Tracked with #76378.
// TODO(ajstorm): This may be the right course of action here as the
// replication is now being run inside a tenant.
base.TestServerArgs{DisableDefaultTestTenant: true},
base.TestServerArgs{
DisableDefaultTestTenant: true,
Knobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
},
},
roachpb.MakeTenantID(20))
defer cleanupDest()
// destSQL refers to the system tenant as that's the one that's running the
// job.
destSQL := hDest.SysDB
destSQL.ExecMultiple(t, strings.Split(`
SET CLUSTER SETTING stream_replication.consumer_heartbeat_frequency = '2s';
SET CLUSTER SETTING stream_replication.consumer_heartbeat_frequency = '100ms';
SET CLUSTER SETTING bulkio.stream_ingestion.minimum_flush_interval = '500ms';
SET CLUSTER SETTING bulkio.stream_ingestion.cutover_signal_poll_interval = '100ms';
SET CLUSTER SETTING streaming.partition_progress_frequency = '100ms';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -591,7 +591,7 @@ func (sip *streamIngestionProcessor) bufferKV(kv *roachpb.KeyValue) error {
}

func (sip *streamIngestionProcessor) bufferCheckpoint(event partitionEvent) error {
log.Infof(sip.Ctx, "got checkpoint %v", event.GetResolved())
log.VInfof(sip.Ctx, 3, "got checkpoint %v", event.GetResolved())
resolvedTimePtr := event.GetResolved()
if resolvedTimePtr == nil {
return errors.New("checkpoint event expected to have a resolved timestamp")
Expand Down
16 changes: 4 additions & 12 deletions pkg/ccl/streamingccl/streamproducer/event_stream.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,19 +223,15 @@ func (s *eventStream) onValue(ctx context.Context, value *roachpb.RangeFeedValue
select {
case <-ctx.Done():
case s.eventsCh <- roachpb.RangeFeedEvent{Val: value}:
if log.V(1) {
log.Infof(ctx, "onValue: %s@%s", value.Key, value.Value.Timestamp)
}
log.VInfof(ctx, 1, "onValue: %s@%s", value.Key, value.Value.Timestamp)
}
}

func (s *eventStream) onCheckpoint(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) {
select {
case <-ctx.Done():
case s.eventsCh <- roachpb.RangeFeedEvent{Checkpoint: checkpoint}:
if log.V(1) {
log.Infof(ctx, "onCheckpoint: %s@%s", checkpoint.Span, checkpoint.ResolvedTS)
}
log.VInfof(ctx, 1, "onCheckpoint: %s@%s", checkpoint.Span, checkpoint.ResolvedTS)
}
}

Expand All @@ -248,9 +244,7 @@ func (s *eventStream) onSpanCompleted(ctx context.Context, sp roachpb.Span) erro
case <-ctx.Done():
return ctx.Err()
case s.eventsCh <- roachpb.RangeFeedEvent{Checkpoint: &checkpoint}:
if log.V(1) {
log.Infof(ctx, "onSpanCompleted: %s@%s", checkpoint.Span, checkpoint.ResolvedTS)
}
log.VInfof(ctx, 1, "onSpanCompleted: %s@%s", checkpoint.Span, checkpoint.ResolvedTS)
return nil
}
}
Expand All @@ -259,9 +253,7 @@ func (s *eventStream) onSSTable(ctx context.Context, sst *roachpb.RangeFeedSSTab
select {
case <-ctx.Done():
case s.eventsCh <- roachpb.RangeFeedEvent{SST: sst}:
if log.V(1) {
log.Infof(ctx, "onSSTable: %s@%s", sst.Span, sst.WriteTS)
}
log.VInfof(ctx, 1, "onSSTable: %s@%s", sst.Span, sst.WriteTS)
}
}

Expand Down
6 changes: 6 additions & 0 deletions pkg/cmd/dev/generate.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ func makeGenerateCmd(runE func(cmd *cobra.Command, args []string) error) *cobra.
dev generate bazel # DEPS.bzl and BUILD.bazel files
dev generate cgo # files that help non-Bazel systems (IDEs, go) link to our C dependencies
dev generate docs # generates documentation
dev generate execgen # generates execgen go code (subset of 'dev generate go')
dev generate go # generates go code (execgen, stringer, protobufs, etc.), plus everything 'cgo' generates
dev generate go_nocgo # generates go code (execgen, stringer, protobufs, etc.)
dev generate protobuf # *.pb.go files (subset of 'dev generate go')
Expand All @@ -63,6 +64,7 @@ func (d *dev) generate(cmd *cobra.Command, targets []string) error {
"bazel": d.generateBazel,
"cgo": d.generateCgo,
"docs": d.generateDocs,
"execgen": d.generateExecgen,
"go": d.generateGo,
"go_nocgo": d.generateGoNoCgo,
"protobuf": d.generateProtobuf,
Expand Down Expand Up @@ -154,6 +156,10 @@ func (d *dev) generateDocs(cmd *cobra.Command) error {
return d.generateRedactSafe(ctx)
}

func (d *dev) generateExecgen(cmd *cobra.Command) error {
return d.generateTarget(cmd.Context(), "//pkg/gen:execgen")
}

func (d *dev) generateGoAndDocs(cmd *cobra.Command) error {
ctx := cmd.Context()
if err := d.generateTarget(ctx, "//pkg/gen"); err != nil {
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/clusterunique/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,5 +9,6 @@ go_library(
"//pkg/base",
"//pkg/util/hlc",
"//pkg/util/uint128",
"@com_github_cockroachdb_errors//:errors",
],
)
20 changes: 20 additions & 0 deletions pkg/sql/clusterunique/id.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/uint128"
"github.com/cockroachdb/errors"
)

// ID represents an identifier that is guaranteed to be unique across
Expand Down Expand Up @@ -53,3 +54,22 @@ func IDFromBytes(b []byte) ID {
func (id ID) GetNodeID() int32 {
return int32(0xFFFFFFFF & id.Lo)
}

// Size returns the marshalled size of id in bytes.
func (id ID) Size() int {
return len(id.GetBytes())
}

// MarshalTo marshals id to data.
func (id ID) MarshalTo(data []byte) (int, error) {
return copy(data, id.GetBytes()), nil
}

// Unmarshal unmarshals data to id.
func (id *ID) Unmarshal(data []byte) error {
if len(data) != 16 {
return errors.Errorf("input data %s for uint128 must be 16 bytes", data)
}
id.Uint128 = uint128.FromBytes(data)
return nil
}
9 changes: 5 additions & 4 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -2180,10 +2180,7 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) {
txnStart := ex.state.mu.txnStart
ex.state.mu.RUnlock()

// Transaction received time is the time at which the statement that prompted
// the creation of this transaction was received.
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionTransactionReceived,
ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionQueryReceived))
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionTransactionStarted, txnStart)
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionFirstStartExecTransaction, timeutil.Now())
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionMostRecentStartExecTransaction,
ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionFirstStartExecTransaction))
Expand Down Expand Up @@ -2277,6 +2274,10 @@ func (ex *connExecutor) recordTransactionFinish(
BytesRead: ex.extraTxnState.bytesRead,
}

if ex.server.cfg.TestingKnobs.OnRecordTxnFinish != nil {
ex.server.cfg.TestingKnobs.OnRecordTxnFinish(ex.executorType == executorTypeInternal, ex.phaseTimes, ex.planner.stmt.SQL)
}

return ex.statsCollector.RecordTransaction(
ctx,
transactionFingerprintID,
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -6211,9 +6211,9 @@ CREATE TABLE crdb_internal.node_execution_outliers (
ctx context.Context, o *outliers.Outlier,
) {
err = errors.CombineErrors(err, addRow(
tree.NewDString(hex.EncodeToString(o.Session.ID)),
tree.NewDUuid(tree.DUuid{UUID: *o.Transaction.ID}),
tree.NewDString(hex.EncodeToString(o.Statement.ID)),
tree.NewDString(hex.EncodeToString(o.Session.ID.GetBytes())),
tree.NewDUuid(tree.DUuid{UUID: o.Transaction.ID}),
tree.NewDString(hex.EncodeToString(o.Statement.ID.GetBytes())),
tree.NewDBytes(tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(o.Statement.FingerprintID)))),
))
})
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/sessioninit"
"github.com/cockroachdb/cockroach/pkg/sql/sessionphase"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
Expand Down Expand Up @@ -1462,6 +1463,10 @@ type ExecutorTestingKnobs struct {
// AfterBackupCheckpoint if set will be called after a BACKUP-CHECKPOINT
// is written.
AfterBackupCheckpoint func()

// OnRecordTxnFinish, if set, will be called as we record a transaction
// finishing.
OnRecordTxnFinish func(isInternal bool, phaseTimes *sessionphase.Times, stmt string)
}

// PGWireTestingKnobs contains knobs for the pgwire module.
Expand Down
36 changes: 22 additions & 14 deletions pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -924,7 +924,7 @@ func (d *DFloat) Prev(ctx CompareContext) (Datum, bool) {
return nil, false
}
if f == math.Inf(-1) {
return dNaNFloat, true
return DNaNFloat, true
}
return NewDFloat(DFloat(math.Nextafter(f, math.Inf(-1)))), true
}
Expand All @@ -933,22 +933,28 @@ func (d *DFloat) Prev(ctx CompareContext) (Datum, bool) {
func (d *DFloat) Next(ctx CompareContext) (Datum, bool) {
f := float64(*d)
if math.IsNaN(f) {
return dNegInfFloat, true
return DNegInfFloat, true
}
if f == math.Inf(+1) {
return nil, false
}
return NewDFloat(DFloat(math.Nextafter(f, math.Inf(+1)))), true
}

var dZeroFloat = NewDFloat(0.0)
var dPosInfFloat = NewDFloat(DFloat(math.Inf(+1)))
var dNegInfFloat = NewDFloat(DFloat(math.Inf(-1)))
var dNaNFloat = NewDFloat(DFloat(math.NaN()))
var (
// DZeroFloat is the DFloat for zero.
DZeroFloat = NewDFloat(0)
// DPosInfFloat is the DFloat for positive infinity.
DPosInfFloat = NewDFloat(DFloat(math.Inf(+1)))
// DNegInfFloat is the DFloat for negative infinity.
DNegInfFloat = NewDFloat(DFloat(math.Inf(-1)))
// DNaNFloat is the DFloat for NaN.
DNaNFloat = NewDFloat(DFloat(math.NaN()))
)

// IsMax implements the Datum interface.
func (d *DFloat) IsMax(ctx CompareContext) bool {
return *d == *dPosInfFloat
return *d == *DPosInfFloat
}

// IsMin implements the Datum interface.
Expand All @@ -958,12 +964,12 @@ func (d *DFloat) IsMin(ctx CompareContext) bool {

// Max implements the Datum interface.
func (d *DFloat) Max(ctx CompareContext) (Datum, bool) {
return dPosInfFloat, true
return DPosInfFloat, true
}

// Min implements the Datum interface.
func (d *DFloat) Min(ctx CompareContext) (Datum, bool) {
return dNaNFloat, true
return DNaNFloat, true
}

// AmbiguousFormat implements the Datum interface.
Expand Down Expand Up @@ -2553,7 +2559,8 @@ func MustMakeDTimestamp(t time.Time, precision time.Duration) *DTimestamp {
return ret
}

var dZeroTimestamp = &DTimestamp{}
// DZeroTimestamp is the zero-valued DTimestamp.
var DZeroTimestamp = &DTimestamp{}

// time.Time formats.
const (
Expand Down Expand Up @@ -2868,7 +2875,8 @@ func ParseDTimestampTZ(
return d, dependsOnContext, err
}

var dZeroTimestampTZ = &DTimestampTZ{}
// DZeroTimestampTZ is the zero-valued DTimestampTZ.
var DZeroTimestampTZ = &DTimestampTZ{}

// AsDTimestampTZ attempts to retrieve a DTimestampTZ from an Expr, returning a
// DTimestampTZ and a flag signifying whether the assertion was successful. The
Expand Down Expand Up @@ -5303,21 +5311,21 @@ func NewDefaultDatum(collationEnv *CollationEnvironment, t *types.T) (d Datum, e
case types.IntFamily:
return DZero, nil
case types.FloatFamily:
return dZeroFloat, nil
return DZeroFloat, nil
case types.DecimalFamily:
return dZeroDecimal, nil
case types.DateFamily:
return dEpochDate, nil
case types.TimestampFamily:
return dZeroTimestamp, nil
return DZeroTimestamp, nil
case types.IntervalFamily:
return dZeroInterval, nil
case types.StringFamily:
return dEmptyString, nil
case types.BytesFamily:
return dEmptyBytes, nil
case types.TimestampTZFamily:
return dZeroTimestampTZ, nil
return DZeroTimestampTZ, nil
case types.CollatedStringFamily:
return NewDCollatedString("", t.Locale(), collationEnv)
case types.OidFamily:
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/sessionphase/session_phase.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,9 +59,9 @@ const (
// have no execution, like SHOW TRANSACTION STATUS.
SessionQueryServiced

// SessionTransactionReceived is the SessionPhase when a transaction is
// received.
SessionTransactionReceived
// SessionTransactionStarted is the SessionPhase when a transaction is
// started.
SessionTransactionStarted

// SessionFirstStartExecTransaction is the SessionPhase when a transaction
// is started for the first time.
Expand Down Expand Up @@ -197,7 +197,7 @@ func (t *Times) GetTransactionRetryLatency() time.Duration {
// GetTransactionServiceLatency returns the total time to service the
// transaction.
func (t *Times) GetTransactionServiceLatency() time.Duration {
return t.times[SessionEndExecTransaction].Sub(t.times[SessionTransactionReceived])
return t.times[SessionEndExecTransaction].Sub(t.times[SessionTransactionStarted])
}

// GetCommitLatency returns the total time spent for the transaction to
Expand Down
Loading

0 comments on commit bce2267

Please sign in to comment.