From b4ed9cb0f214ed9f5048999d57366fea1409a50e Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Tue, 14 Jun 2022 14:26:47 +0100 Subject: [PATCH 01/15] streamingccl: minor error style cleanups Capitalized error messages are rare in the code base, so I've made these more consistent with the rest of the code base. Release note: None --- .../streamclient/partitioned_stream_client.go | 8 ++++---- pkg/ccl/streamingccl/streamproducer/stream_lifetime.go | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/ccl/streamingccl/streamclient/partitioned_stream_client.go b/pkg/ccl/streamingccl/streamclient/partitioned_stream_client.go index e139a4b26c61..7009349ea279 100644 --- a/pkg/ccl/streamingccl/streamclient/partitioned_stream_client.go +++ b/pkg/ccl/streamingccl/streamclient/partitioned_stream_client.go @@ -65,7 +65,7 @@ func (p *partitionedStreamClient) Create( row := conn.QueryRowContext(ctx, `SELECT crdb_internal.start_replication_stream($1)`, tenantID.ToUint64()) if row.Err() != nil { - return streamID, errors.Wrapf(row.Err(), "Error in creating replication stream for tenant %s", tenantID.String()) + return streamID, errors.Wrapf(row.Err(), "error creating replication stream for tenant %s", tenantID.String()) } err = row.Scan(&streamID) @@ -85,7 +85,7 @@ func (p *partitionedStreamClient) Heartbeat( `SELECT crdb_internal.replication_stream_progress($1, $2)`, streamID, consumed.String()) if row.Err() != nil { return streampb.StreamReplicationStatus{}, - errors.Wrapf(row.Err(), "Error in sending heartbeats to replication stream %d", streamID) + errors.Wrapf(row.Err(), "error sending heartbeat to replication stream %d", streamID) } var rawStatus []byte @@ -121,7 +121,7 @@ func (p *partitionedStreamClient) Plan( row := conn.QueryRowContext(ctx, `SELECT crdb_internal.replication_stream_spec($1)`, streamID) if row.Err() != nil { - return nil, errors.Wrap(row.Err(), "Error in planning a replication stream") + return nil, errors.Wrapf(row.Err(), "error planning replication stream %d", streamID) } var rawSpec []byte @@ -207,7 +207,7 @@ func (p *partitionedStreamClient) Complete(ctx context.Context, streamID streami } row := conn.QueryRowContext(ctx, `SELECT crdb_internal.complete_replication_stream($1)`, streamID) if row.Err() != nil { - return errors.Wrap(row.Err(), "Error in completing a replication stream") + return errors.Wrapf(row.Err(), "error completing replication stream %d", streamID) } return nil } diff --git a/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go b/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go index 45a365fc7595..ae004d0a81a8 100644 --- a/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go +++ b/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go @@ -153,10 +153,10 @@ func getReplicationStreamSpec( // Returns error if the replication stream is not active j, err := jobExecCtx.ExecCfg().JobRegistry.LoadJob(evalCtx.Ctx(), jobspb.JobID(streamID)) if err != nil { - return nil, errors.Wrapf(err, "Replication stream %d has error", streamID) + return nil, errors.Wrapf(err, "replication stream %d has error", streamID) } if j.Status() != jobs.StatusRunning { - return nil, errors.Errorf("Replication stream %d is not running", streamID) + return nil, errors.Errorf("replication stream %d is not running", streamID) } // Partition the spans with SQLPlanner From ea559dfe0ba57259ca71d3c8ca1de6388954ea73 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Fri, 24 Jun 2022 11:00:18 -0700 Subject: [PATCH 02/15] execinfra: remove MetadataTest* processors This commit removes `MetadataTestSender` and `MetadataTestReceiver` processors since they no longer provide much value. I believe they were introduced when we added a `ProducerMetadata` as a return parameter to `Next` method in order to ensure that at least some artificial metadata is propagated correctly throughout the whole flow. The main goal of this commit is the removal of `fakedist-metadata` and `5node-metadata` logic test configs in order to speed up the CI time. The justification for removal of these processors without putting in their place is that these processors are not that useful - the only thing they can test is that *some* metadata is propagated through the row-based flows. Note that they don't test whether all necessary metadata is emitted (for example, whether `LeafTxnFinalState`). We've using the vectorized engine as the default for a while now, and these processors don't get planned with the vectorized flows. Thus, it seems silly to have a logic test config like `fakedist-metadata` that is part of the default configs. Release note: None --- pkg/sql/colexec/colbuilder/execplan.go | 12 - pkg/sql/colflow/BUILD.bazel | 1 - .../vectorized_meta_propagation_test.go | 127 +++++----- pkg/sql/distsql_physical_planner.go | 62 +---- pkg/sql/execinfra/BUILD.bazel | 2 - pkg/sql/execinfra/metadata_test_receiver.go | 238 ------------------ pkg/sql/execinfra/metadata_test_sender.go | 116 --------- pkg/sql/execinfra/processorsbase.go | 13 +- pkg/sql/execinfra/server_config.go | 19 -- pkg/sql/execinfrapb/flow_diagram.go | 10 - pkg/sql/execinfrapb/processors.proto | 16 +- pkg/sql/logictest/logic.go | 30 +-- .../testdata/logic_test/distsql_enum | 2 +- .../testdata/logic_test/distsql_stats | 2 +- .../logic_test/inverted_join_geospatial_dist | 2 +- pkg/sql/logictest/testdata/logic_test/orms | 2 +- .../testdata/logic_test/rename_column | 2 - .../testdata/logic_test/rename_database | 2 - .../testdata/logic_test/rename_index | 2 - .../testdata/logic_test/rename_table | 2 - .../logictest/testdata/logic_test/truncate | 2 - pkg/sql/rowexec/processors.go | 14 -- 22 files changed, 75 insertions(+), 603 deletions(-) delete mode 100644 pkg/sql/execinfra/metadata_test_receiver.go delete mode 100644 pkg/sql/execinfra/metadata_test_sender.go diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index ef15183726fc..7a9d30ba4233 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -243,8 +243,6 @@ func supportedNatively(spec *execinfrapb.ProcessorSpec) error { var ( errCoreUnsupportedNatively = errors.New("unsupported processor core") errLocalPlanNodeWrap = errors.New("LocalPlanNode core needs to be wrapped") - errMetadataTestSenderWrap = errors.New("core.MetadataTestSender is not supported") - errMetadataTestReceiverWrap = errors.New("core.MetadataTestReceiver is not supported") errChangeAggregatorWrap = errors.New("core.ChangeAggregator is not supported") errChangeFrontierWrap = errors.New("core.ChangeFrontier is not supported") errReadImportWrap = errors.New("core.ReadImport is not supported") @@ -282,16 +280,6 @@ func canWrap(mode sessiondatapb.VectorizeExecMode, spec *execinfrapb.ProcessorSp return errSamplerWrap case spec.Core.SampleAggregator != nil: return errSampleAggregatorWrap - case spec.Core.MetadataTestSender != nil: - // We do not wrap MetadataTestSender because of the way metadata is - // propagated through the vectorized flow - it is drained at the flow - // shutdown unlike these test processors expect. - return errMetadataTestSenderWrap - case spec.Core.MetadataTestReceiver != nil: - // We do not wrap MetadataTestReceiver because of the way metadata is - // propagated through the vectorized flow - it is drained at the flow - // shutdown unlike these test processors expect. - return errMetadataTestReceiverWrap case spec.Core.ZigzagJoiner != nil: case spec.Core.ProjectSet != nil: case spec.Core.Windower != nil: diff --git a/pkg/sql/colflow/BUILD.bazel b/pkg/sql/colflow/BUILD.bazel index b3f514c8483b..fc7abb1c484a 100644 --- a/pkg/sql/colflow/BUILD.bazel +++ b/pkg/sql/colflow/BUILD.bazel @@ -114,7 +114,6 @@ go_test( "//pkg/storage", "//pkg/testutils", "//pkg/testutils/colcontainerutils", - "//pkg/testutils/distsqlutils", "//pkg/testutils/serverutils", "//pkg/testutils/skip", "//pkg/testutils/sqlutils", diff --git a/pkg/sql/colflow/vectorized_meta_propagation_test.go b/pkg/sql/colflow/vectorized_meta_propagation_test.go index a821f6385c65..e1e1f0442fa1 100644 --- a/pkg/sql/colflow/vectorized_meta_propagation_test.go +++ b/pkg/sql/colflow/vectorized_meta_propagation_test.go @@ -14,26 +14,47 @@ import ( "context" "testing" + "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexectestutils" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/randgen" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/testutils/distsqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/stretchr/testify/require" ) +type testBatchReceiver struct { + batches []coldata.Batch + metadata []*execinfrapb.ProducerMetadata +} + +var _ execinfra.BatchReceiver = &testBatchReceiver{} + +func (t *testBatchReceiver) ProducerDone() {} + +func (t *testBatchReceiver) PushBatch( + batch coldata.Batch, meta *execinfrapb.ProducerMetadata, +) execinfra.ConsumerStatus { + status := execinfra.NeedMoreRows + if batch != nil { + t.batches = append(t.batches, batch) + } else if meta != nil { + t.metadata = append(t.metadata, meta) + } else { + status = execinfra.ConsumerClosed + } + return status +} + // TestVectorizedMetaPropagation tests whether metadata is correctly propagated -// alongside columnar operators. It sets up the following "flow": -// RowSource -> metadataTestSender -> columnarizer -> noopOperator -> -// -> materializer -> metadataTestReceiver. Metadata propagation is hooked up -// manually from the columnarizer into the materializer similar to how it is -// done in setupVectorizedFlow. +// in the vectorized flows. It creates a colexecop.Operator as well as a +// colexecop.MetadataSource which are hooked up into the BatchFlowCoordinator in +// the same way as in vectorizedFlowCreator.setupFlow. func TestVectorizedMetaPropagation(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() @@ -46,73 +67,43 @@ func TestVectorizedMetaPropagation(t *testing.T) { Cfg: &execinfra.ServerConfig{Settings: cluster.MakeTestingClusterSettings()}, } - nRows := 10 - nCols := 1 + nBatches := 10 typs := types.OneIntCol - input := distsqlutils.NewRowBuffer(typs, randgen.MakeIntRows(nRows, nCols), distsqlutils.RowBufferArgs{}) - mtsSpec := execinfrapb.ProcessorCoreUnion{ - MetadataTestSender: &execinfrapb.MetadataTestSenderSpec{ - ID: uuid.MakeV4().String(), + // Prepare the input operator. + batch := testAllocator.NewMemBatchWithFixedCapacity(typs, 1 /* capacity */) + batch.SetLength(1) + source := colexecop.NewRepeatableBatchSource(testAllocator, batch, typs) + source.ResetBatchesToReturn(nBatches) + + // Setup the metadata source. + expectedMetadata := []execinfrapb.ProducerMetadata{{RowNum: &execinfrapb.RemoteProducerMetadata_RowNum{LastMsg: true}}} + drainMetaCbCalled := false + metadataSource := colexectestutils.CallbackMetadataSource{ + DrainMetaCb: func() []execinfrapb.ProducerMetadata { + if drainMetaCbCalled { + return nil + } + drainMetaCbCalled = true + return expectedMetadata }, } - mts, err := execinfra.NewMetadataTestSender( - &flowCtx, - 0, - input, - &execinfrapb.PostProcessSpec{}, - nil, - uuid.MakeV4().String(), - ) - if err != nil { - t.Fatal(err) - } - col := colexec.NewBufferingColumnarizer(testAllocator, &flowCtx, 1, mts) - noop := colexecop.NewNoop(col) - mat := colexec.NewMaterializer( + output := &testBatchReceiver{} + f := colflow.NewBatchFlowCoordinator( &flowCtx, - 2, /* processorID */ + 0, /* processorID */ colexecargs.OpWithMetaInfo{ - Root: noop, - MetadataSources: colexecop.MetadataSources{col}, + Root: source, + MetadataSources: colexecop.MetadataSources{metadataSource}, }, - typs, + output, + func() {}, /* cancelFlow */ ) + f.Run(context.Background()) - mtr, err := execinfra.NewMetadataTestReceiver( - &flowCtx, - 3, - mat, - &execinfrapb.PostProcessSpec{}, - nil, - []string{mtsSpec.MetadataTestSender.ID}, - ) - if err != nil { - t.Fatal(err) - } - mtr.Start(ctx) - - rowCount, metaCount := 0, 0 - for { - row, meta := mtr.Next() - if row == nil && meta == nil { - break - } - if row != nil { - rowCount++ - } else if meta.Err != nil { - t.Fatal(meta.Err) - } else { - metaCount++ - } - } - if rowCount != nRows { - t.Fatalf("expected %d rows but %d received", nRows, rowCount) - } - if metaCount != nRows+1 { - // metadataTestSender sends a meta after each row plus an additional one to - // indicate the last meta. - t.Fatalf("expected %d meta but %d received", nRows+1, metaCount) - } + // Ensure that the expected number of batches and metadata objects have been + // pushed into the output. + require.Equal(t, nBatches, len(output.batches)) + require.Equal(t, len(expectedMetadata), len(output.metadata)) } diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index e7439e237626..ac22ff36a426 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -34,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/colflow" "github.com/cockroachdb/cockroach/pkg/sql/distsql" - "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfra/execagg" "github.com/cockroachdb/cockroach/pkg/sql/execinfra/execopnode" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -93,10 +92,6 @@ type DistSQLPlanner struct { distSQLSrv *distsql.ServerImpl spanResolver physicalplan.SpanResolver - // metadataTestTolerance is the minimum level required to plan metadata test - // processors. - metadataTestTolerance execinfra.MetadataTestLevel - // runnerChan is used to send out requests (for running SetupFlow RPCs) to a // pool of workers. runnerChan chan runnerRequest @@ -203,13 +198,12 @@ func NewDistSQLPlanner( connHealth: nodeDialer.ConnHealthTryDial, isAvailable: isAvailable, }, - distSender: distSender, - nodeDescs: nodeDescs, - rpcCtx: rpcCtx, - metadataTestTolerance: execinfra.NoExplain, - sqlInstanceProvider: sqlInstanceProvider, - codec: codec, - clock: clock, + distSender: distSender, + nodeDescs: nodeDescs, + rpcCtx: rpcCtx, + sqlInstanceProvider: sqlInstanceProvider, + codec: codec, + clock: clock, } dsp.parallelLocalScansSem = quotapool.NewIntPool("parallel local scans concurrency", @@ -227,10 +221,6 @@ func NewDistSQLPlanner( return dsp } -func (dsp *DistSQLPlanner) shouldPlanTestMetadata() bool { - return dsp.distSQLSrv.TestingKnobs.MetadataTestLevel >= dsp.metadataTestTolerance -} - // GetSQLInstanceInfo gets a node descriptor by node ID. func (dsp *DistSQLPlanner) GetSQLInstanceInfo( sqlInstanceID base.SQLInstanceID, @@ -3082,24 +3072,6 @@ func (dsp *DistSQLPlanner) createPhysPlanForPlanNode( planCtx.traceMetadata.associateNodeWithComponents(node, processors) } - if dsp.shouldPlanTestMetadata() { - if err := plan.CheckLastStagePost(); err != nil { - log.Fatalf(ctx, "%v", err) - } - plan.AddNoGroupingStageWithCoreFunc( - func(_ int, _ *physicalplan.Processor) execinfrapb.ProcessorCoreUnion { - return execinfrapb.ProcessorCoreUnion{ - MetadataTestSender: &execinfrapb.MetadataTestSenderSpec{ - ID: uuid.MakeV4().String(), - }, - } - }, - execinfrapb.PostProcessSpec{}, - plan.GetResultTypes(), - plan.MergeOrdering, - ) - } - return plan, err } @@ -4147,15 +4119,6 @@ func (dsp *DistSQLPlanner) FinalizePlan(planCtx *PlanningCtx, plan *PhysicalPlan func (dsp *DistSQLPlanner) finalizePlanWithRowCount( planCtx *PlanningCtx, plan *PhysicalPlan, rowCount int64, ) { - // Find all MetadataTestSenders in the plan, so that the MetadataTestReceiver - // knows how many sender IDs it should expect. - var metadataSenders []string - for _, proc := range plan.Processors { - if proc.Spec.Core.MetadataTestSender != nil { - metadataSenders = append(metadataSenders, proc.Spec.Core.MetadataTestSender.ID) - } - } - maybeMoveSingleFlowToGateway(planCtx, plan, rowCount) // Add a final "result" stage if necessary. @@ -4173,19 +4136,6 @@ func (dsp *DistSQLPlanner) finalizePlanWithRowCount( // PlanToStreamColMap is no longer necessary. plan.PlanToStreamColMap = nil - if len(metadataSenders) > 0 { - plan.AddSingleGroupStage( - dsp.gatewaySQLInstanceID, - execinfrapb.ProcessorCoreUnion{ - MetadataTestReceiver: &execinfrapb.MetadataTestReceiverSpec{ - SenderIDs: metadataSenders, - }, - }, - execinfrapb.PostProcessSpec{}, - plan.GetResultTypes(), - ) - } - // Set up the endpoints for plan.Streams. plan.PopulateEndpoints() diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index 45085797cc41..157574130912 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -7,8 +7,6 @@ go_library( srcs = [ "base.go", "flow_context.go", - "metadata_test_receiver.go", - "metadata_test_sender.go", "metrics.go", "outboxbase.go", "processorsbase.go", diff --git a/pkg/sql/execinfra/metadata_test_receiver.go b/pkg/sql/execinfra/metadata_test_receiver.go deleted file mode 100644 index b29afb6da476..000000000000 --- a/pkg/sql/execinfra/metadata_test_receiver.go +++ /dev/null @@ -1,238 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package execinfra - -import ( - "context" - "fmt" - - "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/rowenc" - "github.com/cockroachdb/cockroach/pkg/util" -) - -// MetadataTestReceiver is a Processors that is complimentary to -// MetadataTestSender which checks that all metadata emitted by latter is -// received. -type MetadataTestReceiver struct { - ProcessorBase - input RowSource - - // trailingErrMeta stores the error metadata received from the input. We - // do not return this metadata immediately because metadata propagation errors - // are prioritized over query errors, which ensures that tests which expect a - // query error can still fail if they do not properly propagate metadata. - trailingErrMeta []execinfrapb.ProducerMetadata - - senders []string - rowCounts map[string]rowNumCounter -} - -type rowNumCounter struct { - expected, actual int32 - seen util.FastIntSet - err error -} - -var _ Processor = &MetadataTestReceiver{} -var _ RowSource = &MetadataTestReceiver{} - -const metadataTestReceiverProcName = "meta receiver" - -// NewMetadataTestReceiver creates a new MetadataTestReceiver. -func NewMetadataTestReceiver( - flowCtx *FlowCtx, - processorID int32, - input RowSource, - post *execinfrapb.PostProcessSpec, - output RowReceiver, - senders []string, -) (*MetadataTestReceiver, error) { - mtr := &MetadataTestReceiver{ - input: input, - senders: senders, - rowCounts: make(map[string]rowNumCounter), - } - if err := mtr.Init( - mtr, - post, - input.OutputTypes(), - flowCtx, - processorID, - output, - nil, /* memMonitor */ - ProcStateOpts{ - InputsToDrain: []RowSource{input}, - TrailingMetaCallback: func() []execinfrapb.ProducerMetadata { - var trailingMeta []execinfrapb.ProducerMetadata - if mtr.rowCounts != nil { - if meta := mtr.checkRowNumMetadata(); meta != nil { - trailingMeta = append(trailingMeta, *meta) - } - } - mtr.InternalClose() - return trailingMeta - }, - }, - ); err != nil { - return nil, err - } - return mtr, nil -} - -// checkRowNumMetadata examines all of the received RowNum metadata to ensure -// that it has received exactly one of each expected RowNum. If the check -// detects dropped or repeated metadata, it returns error metadata. Otherwise, -// it returns nil. -func (mtr *MetadataTestReceiver) checkRowNumMetadata() *execinfrapb.ProducerMetadata { - defer func() { mtr.rowCounts = nil }() - - if len(mtr.rowCounts) != len(mtr.senders) { - var missingSenders string - for _, sender := range mtr.senders { - if _, exists := mtr.rowCounts[sender]; !exists { - if missingSenders == "" { - missingSenders = sender - } else { - missingSenders += fmt.Sprintf(", %s", sender) - } - } - } - return &execinfrapb.ProducerMetadata{ - Err: fmt.Errorf( - "expected %d metadata senders but found %d; missing %s", - len(mtr.senders), len(mtr.rowCounts), missingSenders, - ), - } - } - for id, cnt := range mtr.rowCounts { - if cnt.err != nil { - return &execinfrapb.ProducerMetadata{Err: cnt.err} - } - if cnt.expected != cnt.actual { - return &execinfrapb.ProducerMetadata{ - Err: fmt.Errorf( - "dropped metadata from sender %s: expected %d RowNum messages but got %d", - id, cnt.expected, cnt.actual), - } - } - for i := 0; i < int(cnt.expected); i++ { - if !cnt.seen.Contains(i) { - return &execinfrapb.ProducerMetadata{ - Err: fmt.Errorf( - "dropped and repeated metadata from sender %s: have %d messages but missing RowNum #%d", - id, cnt.expected, i+1), - } - } - } - } - - return nil -} - -// Start is part of the RowSource interface. -func (mtr *MetadataTestReceiver) Start(ctx context.Context) { - ctx = mtr.StartInternal(ctx, metadataTestReceiverProcName) - mtr.input.Start(ctx) -} - -// Next is part of the RowSource interface. -// -// This implementation doesn't follow the usual patterns of other processors; it -// makes more limited use of the ProcessorBase's facilities because it needs to -// inspect metadata while draining. -func (mtr *MetadataTestReceiver) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) { - for { - if mtr.State == StateTrailingMeta { - if meta := mtr.popTrailingMeta(); meta != nil { - return nil, meta - } - // If there's no more trailingMeta, we've moved to stateExhausted, and we - // might return some trailingErrMeta below. - } - if mtr.State == StateExhausted { - if len(mtr.trailingErrMeta) > 0 { - meta := mtr.trailingErrMeta[0] - mtr.trailingErrMeta = mtr.trailingErrMeta[1:] - return nil, &meta - } - return nil, nil - } - - row, meta := mtr.input.Next() - - if meta != nil { - if meta.RowNum != nil { - rowNum := meta.RowNum - rcnt, exists := mtr.rowCounts[rowNum.SenderID] - if !exists { - rcnt.expected = -1 - } - if rcnt.err != nil { - return nil, meta - } - if rowNum.LastMsg { - if rcnt.expected != -1 { - rcnt.err = fmt.Errorf( - "repeated metadata from reader %s: received more than one RowNum with LastMsg set", - rowNum.SenderID) - mtr.rowCounts[rowNum.SenderID] = rcnt - return nil, meta - } - rcnt.expected = rowNum.RowNum - } else { - rcnt.actual++ - rcnt.seen.Add(int(rowNum.RowNum - 1)) - } - mtr.rowCounts[rowNum.SenderID] = rcnt - } - if meta.Err != nil { - // Keep track of the err in trailingErrMeta, which will be returned - // after everything else (including ProcessorBase.trailingMeta). - mtr.trailingErrMeta = append(mtr.trailingErrMeta, *meta) - continue - } - - return nil, meta - } - - if row == nil { - mtr.moveToTrailingMeta() - continue - } - - // We don't use ProcessorBase.ProcessRowHelper() here because we need - // special handling for errors: this proc never starts draining in order for - // it to be as unintrusive as possible. - outRow, ok, err := mtr.OutputHelper.ProcessRow(mtr.Ctx, row) - if err != nil { - mtr.trailingMeta = append(mtr.trailingMeta, execinfrapb.ProducerMetadata{Err: err}) - continue - } - if outRow == nil { - if !ok { - mtr.MoveToDraining(nil /* err */) - } - continue - } - - // Swallow rows if we're draining. - if mtr.State == StateDraining { - continue - } - - if !ok { - mtr.MoveToDraining(nil /* err */) - } - - return outRow, nil - } -} diff --git a/pkg/sql/execinfra/metadata_test_sender.go b/pkg/sql/execinfra/metadata_test_sender.go deleted file mode 100644 index b9b4e234eecf..000000000000 --- a/pkg/sql/execinfra/metadata_test_sender.go +++ /dev/null @@ -1,116 +0,0 @@ -// Copyright 2018 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package execinfra - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/rowenc" -) - -// MetadataTestSender intersperses a metadata record after every row. -type MetadataTestSender struct { - ProcessorBase - input RowSource - id string - - // sendRowNumMeta is set to true when the next call to Next() must return - // RowNum metadata, and false otherwise. - sendRowNumMeta bool - rowNumCnt int32 -} - -var _ Processor = &MetadataTestSender{} -var _ RowSource = &MetadataTestSender{} - -const metadataTestSenderProcName = "meta sender" - -// NewMetadataTestSender creates a new MetadataTestSender. -func NewMetadataTestSender( - flowCtx *FlowCtx, - processorID int32, - input RowSource, - post *execinfrapb.PostProcessSpec, - output RowReceiver, - id string, -) (*MetadataTestSender, error) { - mts := &MetadataTestSender{input: input, id: id} - if err := mts.Init( - mts, - post, - input.OutputTypes(), - flowCtx, - processorID, - output, - nil, /* memMonitor */ - ProcStateOpts{ - InputsToDrain: []RowSource{mts.input}, - TrailingMetaCallback: func() []execinfrapb.ProducerMetadata { - mts.InternalClose() - // Send a final record with LastMsg set. - meta := execinfrapb.ProducerMetadata{ - RowNum: &execinfrapb.RemoteProducerMetadata_RowNum{ - RowNum: mts.rowNumCnt, - SenderID: mts.id, - LastMsg: true, - }, - } - return []execinfrapb.ProducerMetadata{meta} - }, - }, - ); err != nil { - return nil, err - } - return mts, nil -} - -// Start is part of the RowSource interface. -func (mts *MetadataTestSender) Start(ctx context.Context) { - ctx = mts.StartInternal(ctx, metadataTestSenderProcName) - mts.input.Start(ctx) -} - -// Next is part of the RowSource interface. -func (mts *MetadataTestSender) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) { - // Every call after a row has been returned returns a metadata record. - if mts.sendRowNumMeta { - mts.sendRowNumMeta = false - mts.rowNumCnt++ - return nil, &execinfrapb.ProducerMetadata{ - RowNum: &execinfrapb.RemoteProducerMetadata_RowNum{ - RowNum: mts.rowNumCnt, - SenderID: mts.id, - LastMsg: false, - }, - } - } - - for mts.State == StateRunning { - row, meta := mts.input.Next() - if meta != nil { - // Other processors will start draining when they get an error meta from - // their input. We don't do that here, as the mts should be an unintrusive - // as possible. - return nil, meta - } - if row == nil { - mts.MoveToDraining(nil /* err */) - break - } - - if outRow := mts.ProcessRowHelper(row); outRow != nil { - mts.sendRowNumMeta = true - return outRow, nil - } - } - return nil, mts.DrainHelper() -} diff --git a/pkg/sql/execinfra/processorsbase.go b/pkg/sql/execinfra/processorsbase.go index a771d6c12911..ba33a4254dec 100644 --- a/pkg/sql/execinfra/processorsbase.go +++ b/pkg/sql/execinfra/processorsbase.go @@ -702,16 +702,9 @@ func (pb *ProcessorBase) ProcessRowHelper(row rowenc.EncDatumRow) rowenc.EncDatu if !ok { pb.MoveToDraining(nil /* err */) } - // Note that outRow might be nil here. - // TODO(yuzefovich): there is a problem with this logging when MetadataTest* - // processors are planned - there is a mismatch between the row and the - // output types (rendering is added to the stage of test processors and the - // actual processors that are inputs to the test ones have an unset post - // processing; I think that we need to set the post processing on the stages - // of processors below the test ones). - //if outRow != nil && log.V(3) && pb.Ctx != nil { - // log.InfofDepth(pb.Ctx, 1, "pushing row %s", outRow.String(pb.Out.OutputTypes)) - //} + if outRow != nil && log.V(3) && pb.Ctx != nil { + log.InfofDepth(pb.Ctx, 1, "pushing row %s", outRow.String(pb.OutputHelper.OutputTypes)) + } return outRow } diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index b4b49a3feb68..85bc8384f937 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -258,11 +258,6 @@ type TestingKnobs struct { // to incoming flows to register. DrainFast bool - // MetadataTestLevel controls whether or not additional metadata test - // processors are planned, which send additional "RowNum" metadata that is - // checked by a test receiver on the gateway. - MetadataTestLevel MetadataTestLevel - // Changefeed contains testing knobs specific to the changefeed system. Changefeed base.ModuleTestingKnobs @@ -286,20 +281,6 @@ type TestingKnobs struct { IndexBackfillMergerTestingKnobs base.ModuleTestingKnobs } -// MetadataTestLevel represents the types of queries where metadata test -// processors are planned. -type MetadataTestLevel int - -const ( - // Off represents that no metadata test processors are planned. - Off MetadataTestLevel = iota - // NoExplain represents that metadata test processors are planned for all - // queries except EXPLAIN (DISTSQL) statements. - NoExplain - // On represents that metadata test processors are planned for all queries. - On -) - // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. func (*TestingKnobs) ModuleTestingKnobs() {} diff --git a/pkg/sql/execinfrapb/flow_diagram.go b/pkg/sql/execinfrapb/flow_diagram.go index 2678e3564f22..7c20e1b4bf92 100644 --- a/pkg/sql/execinfrapb/flow_diagram.go +++ b/pkg/sql/execinfrapb/flow_diagram.go @@ -84,16 +84,6 @@ func (f *FiltererSpec) summary() (string, []string) { } } -// summary implements the diagramCellType interface. -func (mts *MetadataTestSenderSpec) summary() (string, []string) { - return "MetadataTestSender", []string{mts.ID} -} - -// summary implements the diagramCellType interface. -func (*MetadataTestReceiverSpec) summary() (string, []string) { - return "MetadataTestReceiver", []string{} -} - // summary implements the diagramCellType interface. func (v *ValuesCoreSpec) summary() (string, []string) { var bytes uint64 diff --git a/pkg/sql/execinfrapb/processors.proto b/pkg/sql/execinfrapb/processors.proto index 099b4de3eb8f..91c5bce4850b 100644 --- a/pkg/sql/execinfrapb/processors.proto +++ b/pkg/sql/execinfrapb/processors.proto @@ -100,13 +100,8 @@ message ProcessorCoreUnion { optional ValuesCoreSpec values = 10; optional BackfillerSpec backfiller = 11; optional ReadImportDataSpec readImport = 13; - reserved 14; - reserved 20; optional SamplerSpec Sampler = 15; optional SampleAggregatorSpec SampleAggregator = 16; - reserved 17; - optional MetadataTestSenderSpec metadataTestSender = 18; - optional MetadataTestReceiverSpec metadataTestReceiver = 19; optional ZigzagJoinerSpec zigzagJoiner = 21; optional ProjectSetSpec projectSet = 22; optional WindowerSpec windower = 23; @@ -126,7 +121,7 @@ message ProcessorCoreUnion { optional ExportSpec exporter = 37; optional IndexBackfillMergerSpec indexBackfillMerger = 38; - reserved 6, 12; + reserved 6, 12, 14, 17, 18, 19, 20; } // NoopCoreSpec indicates a "no-op" processor core. This is used when we just @@ -147,12 +142,3 @@ message LocalPlanNodeSpec { optional uint32 NumInputs = 2 [(gogoproto.nullable) = false]; optional string Name = 3 [(gogoproto.nullable) = false]; } - -message MetadataTestSenderSpec { - optional string id = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "ID"]; -} - -message MetadataTestReceiverSpec { - repeated string sender_ids = 1 [(gogoproto.customname) = "SenderIDs"]; -} diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 020990c620b7..7870af1c838d 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -544,8 +544,6 @@ type testClusterConfig struct { // fall back to disk do so immediately, using only their disk-based // implementation. sqlExecUseDisk bool - // if set, enables DistSQL metadata propagation tests. - distSQLMetadataTestEnabled bool // if set and the -test.short flag is passed, skip this config. skipShort bool // If not empty, bootstrapVersion controls what version the cluster will be @@ -785,14 +783,6 @@ var logicTestConfigs = []testClusterConfig{ overrideDistSQLMode: "on", overrideVectorize: "off", }, - { - name: "fakedist-metadata", - numNodes: 3, - useFakeSpanResolver: true, - overrideDistSQLMode: "on", - distSQLMetadataTestEnabled: true, - skipShort: true, - }, { name: "fakedist-disk", numNodes: 3, @@ -818,13 +808,6 @@ var logicTestConfigs = []testClusterConfig{ // secondary tenants. See #75569 for more info. disableDefaultTestTenant: true, }, - { - name: "5node-metadata", - numNodes: 5, - overrideDistSQLMode: "on", - distSQLMetadataTestEnabled: true, - skipShort: true, - }, { name: "5node-disk", numNodes: 5, @@ -1020,7 +1003,6 @@ var ( "local-spec-planning", "fakedist", "fakedist-vec-off", - "fakedist-metadata", "fakedist-disk", "fakedist-spec-planning", } @@ -1028,7 +1010,6 @@ var ( fiveNodeDefaultConfigName = "5node-default-configs" fiveNodeDefaultConfigNames = []string{ "5node", - "5node-metadata", "5node-disk", "5node-spec-planning", } @@ -1772,16 +1753,9 @@ func (t *logicTest) newCluster( // relocate ranges correctly. params.ReplicationMode = base.ReplicationAuto } - distSQLKnobs := &execinfra.TestingKnobs{ - MetadataTestLevel: execinfra.Off, - } - if cfg.sqlExecUseDisk { - distSQLKnobs.ForceDiskSpill = true - } - if cfg.distSQLMetadataTestEnabled { - distSQLKnobs.MetadataTestLevel = execinfra.On + params.ServerArgs.Knobs.DistSQL = &execinfra.TestingKnobs{ + ForceDiskSpill: cfg.sqlExecUseDisk, } - params.ServerArgs.Knobs.DistSQL = distSQLKnobs if cfg.bootstrapVersion != (roachpb.Version{}) { if params.ServerArgs.Knobs.Server == nil { params.ServerArgs.Knobs.Server = &server.TestingKnobs{} diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_enum b/pkg/sql/logictest/testdata/logic_test/distsql_enum index 3dbdc7de6f5a..c4d278450502 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_enum +++ b/pkg/sql/logictest/testdata/logic_test/distsql_enum @@ -1,4 +1,4 @@ -# LogicTest: 5node-default-configs !5node-metadata +# LogicTest: 5node-default-configs # Regression test for nested tuple enum hydration (#74189) statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_stats b/pkg/sql/logictest/testdata/logic_test/distsql_stats index a3bb955e5880..1dd2f67b2fd9 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_stats +++ b/pkg/sql/logictest/testdata/logic_test/distsql_stats @@ -1,4 +1,4 @@ -# LogicTest: 5node 5node-metadata +# LogicTest: 5node # Disable histogram collection. statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist index b0b5734977ff..58df7c274998 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist +++ b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist @@ -1,4 +1,4 @@ -# LogicTest: 5node-default-configs !5node-metadata +# LogicTest: 5node-default-configs statement ok CREATE TABLE ltable( diff --git a/pkg/sql/logictest/testdata/logic_test/orms b/pkg/sql/logictest/testdata/logic_test/orms index cdcccdcdbff5..fc0b221a4a29 100644 --- a/pkg/sql/logictest/testdata/logic_test/orms +++ b/pkg/sql/logictest/testdata/logic_test/orms @@ -1,6 +1,6 @@ # TODO(mjibson): The fakedist-disk config produces an error. When fixed, # remove this config line. See #38985. -# LogicTest: local fakedist fakedist-metadata 3node-tenant +# LogicTest: local fakedist 3node-tenant ## This test file contains various complex queries that ORMs issue during ## startup or general use. diff --git a/pkg/sql/logictest/testdata/logic_test/rename_column b/pkg/sql/logictest/testdata/logic_test/rename_column index 992601351f2d..71566faed42b 100644 --- a/pkg/sql/logictest/testdata/logic_test/rename_column +++ b/pkg/sql/logictest/testdata/logic_test/rename_column @@ -1,5 +1,3 @@ -# LogicTest: !fakedist-metadata - statement ok CREATE TABLE users ( uid INT PRIMARY KEY, diff --git a/pkg/sql/logictest/testdata/logic_test/rename_database b/pkg/sql/logictest/testdata/logic_test/rename_database index b626c0cf4d1d..314143e99066 100644 --- a/pkg/sql/logictest/testdata/logic_test/rename_database +++ b/pkg/sql/logictest/testdata/logic_test/rename_database @@ -1,5 +1,3 @@ -# LogicTest: !fakedist-metadata - statement ok SET CLUSTER SETTING sql.cross_db_views.enabled = TRUE diff --git a/pkg/sql/logictest/testdata/logic_test/rename_index b/pkg/sql/logictest/testdata/logic_test/rename_index index 83c4c44b393c..9c78ec52f925 100644 --- a/pkg/sql/logictest/testdata/logic_test/rename_index +++ b/pkg/sql/logictest/testdata/logic_test/rename_index @@ -1,5 +1,3 @@ -# LogicTest: !fakedist-metadata - statement ok CREATE TABLE users ( id INT PRIMARY KEY, diff --git a/pkg/sql/logictest/testdata/logic_test/rename_table b/pkg/sql/logictest/testdata/logic_test/rename_table index 61216d381403..788a71db9fd8 100644 --- a/pkg/sql/logictest/testdata/logic_test/rename_table +++ b/pkg/sql/logictest/testdata/logic_test/rename_table @@ -1,5 +1,3 @@ -# LogicTest: !fakedist-metadata - statement error pgcode 42P01 relation "foo" does not exist ALTER TABLE foo RENAME TO bar diff --git a/pkg/sql/logictest/testdata/logic_test/truncate b/pkg/sql/logictest/testdata/logic_test/truncate index 3cfe8343cf35..7943a8e4f090 100644 --- a/pkg/sql/logictest/testdata/logic_test/truncate +++ b/pkg/sql/logictest/testdata/logic_test/truncate @@ -1,5 +1,3 @@ -# LogicTest: !fakedist-metadata - statement ok CREATE TABLE kv ( k INT PRIMARY KEY, diff --git a/pkg/sql/rowexec/processors.go b/pkg/sql/rowexec/processors.go index 0302ce5a847c..5d744a7fb31c 100644 --- a/pkg/sql/rowexec/processors.go +++ b/pkg/sql/rowexec/processors.go @@ -293,20 +293,6 @@ func NewProcessor( } return newBulkRowWriterProcessor(flowCtx, processorID, *core.BulkRowWriter, inputs[0], outputs[0]) } - if core.MetadataTestSender != nil { - if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { - return nil, err - } - return execinfra.NewMetadataTestSender(flowCtx, processorID, inputs[0], post, outputs[0], core.MetadataTestSender.ID) - } - if core.MetadataTestReceiver != nil { - if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { - return nil, err - } - return execinfra.NewMetadataTestReceiver( - flowCtx, processorID, inputs[0], post, outputs[0], core.MetadataTestReceiver.SenderIDs, - ) - } if core.ProjectSet != nil { if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { return nil, err From 8c9f9c885775f082c44404f808326b238a6d6b8c Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 27 Jun 2022 10:11:55 -0400 Subject: [PATCH 03/15] roachtest: skip decommissionBench/nodes=8/cpu=16/warehouses=3000 https://github.com/cockroachdb/cockroach/issues/82870 Release note: None --- pkg/cmd/roachtest/tests/decommissionbench.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/cmd/roachtest/tests/decommissionbench.go b/pkg/cmd/roachtest/tests/decommissionbench.go index 375d87c7ddb5..8d0bdf6d646c 100644 --- a/pkg/cmd/roachtest/tests/decommissionbench.go +++ b/pkg/cmd/roachtest/tests/decommissionbench.go @@ -43,6 +43,8 @@ type decommissionBenchSpec struct { // When true, the test will attempt to stop the node prior to decommission. whileDown bool + + skip string } // registerDecommissionBench defines all decommission benchmark configurations @@ -84,6 +86,7 @@ func registerDecommissionBench(r registry.Registry) { warehouses: 3000, load: true, admissionControl: true, + skip: "https://github.com/cockroachdb/cockroach/issues/82870", }, } { registerDecommissionBenchSpec(r, benchSpec) @@ -125,6 +128,7 @@ func registerDecommissionBenchSpec(r registry.Registry, benchSpec decommissionBe Cluster: r.MakeClusterSpec(benchSpec.nodes+1, spec.CPU(benchSpec.cpus)), Timeout: timeout, NonReleaseBlocker: true, + Skip: benchSpec.skip, Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { runDecommissionBench(ctx, t, c, benchSpec, timeout) }, From ee57d72f56e9f0c5240618a9b20dde9a71b18731 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Mon, 21 Feb 2022 18:27:47 +0000 Subject: [PATCH 04/15] kvserver: include MVCC range keys in replica consistency checks This patch adds handling of MVCC range keys in replica consistency checks. These are iterated over as part of `MVCCStats` calculations and hashed similarly to point keys. Range keys will only exist after the version gate `ExperimentalMVCCRangeTombstones` has been enabled, so a separate version gate is not necessary. Release note: None --- pkg/kv/kvserver/replica_consistency.go | 59 ++++++- pkg/kv/kvserver/replica_consistency_diff.go | 150 +++++++++++++----- pkg/kv/kvserver/replica_consistency_test.go | 95 +++++++++++ pkg/kv/kvserver/replica_test.go | 76 ++++----- .../testdata/replica_consistency_diff | 88 ++++++++++ .../testdata/replica_consistency_sha512 | 28 ++++ pkg/roachpb/internal_raft.proto | 15 +- 7 files changed, 421 insertions(+), 90 deletions(-) create mode 100644 pkg/kv/kvserver/testdata/replica_consistency_diff create mode 100644 pkg/kv/kvserver/testdata/replica_consistency_sha512 diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index 861696ab2edb..ce07f0b122dd 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -598,9 +598,8 @@ func (*Replica) sha512( var timestampBuf []byte hasher := sha512.New() - // TODO(erikgrinaker): add a range key visitor to hash range keys. pointKeyVisitor := func(unsafeKey storage.MVCCKey, unsafeValue []byte) error { - // Rate Limit the scan through the range + // Rate limit the scan through the range. if err := limiter.WaitN(ctx, int64(len(unsafeKey.Key)+len(unsafeValue))); err != nil { return err } @@ -643,6 +642,60 @@ func (*Replica) sha512( return err } + rangeKeyVisitor := func(rangeKV storage.MVCCRangeKeyValue) error { + // Rate limit the scan through the range. + err := limiter.WaitN(ctx, + int64(len(rangeKV.RangeKey.StartKey)+len(rangeKV.RangeKey.EndKey)+len(rangeKV.Value))) + if err != nil { + return err + } + + if snapshot != nil { + // Add (a copy of) the range key into the debug message. + rkv := roachpb.RaftSnapshotData_RangeKeyValue{ + Timestamp: rangeKV.RangeKey.Timestamp, + } + alloc, rkv.StartKey = alloc.Copy(rangeKV.RangeKey.StartKey, 0) + alloc, rkv.EndKey = alloc.Copy(rangeKV.RangeKey.EndKey, 0) + alloc, rkv.Value = alloc.Copy(rangeKV.Value, 0) + snapshot.RangeKV = append(snapshot.RangeKV, rkv) + } + + // Encode the length of the start key and end key. + binary.LittleEndian.PutUint64(intBuf[:], uint64(len(rangeKV.RangeKey.StartKey))) + if _, err := hasher.Write(intBuf[:]); err != nil { + return err + } + binary.LittleEndian.PutUint64(intBuf[:], uint64(len(rangeKV.RangeKey.EndKey))) + if _, err := hasher.Write(intBuf[:]); err != nil { + return err + } + binary.LittleEndian.PutUint64(intBuf[:], uint64(len(rangeKV.Value))) + if _, err := hasher.Write(intBuf[:]); err != nil { + return err + } + if _, err := hasher.Write(rangeKV.RangeKey.StartKey); err != nil { + return err + } + if _, err := hasher.Write(rangeKV.RangeKey.EndKey); err != nil { + return err + } + legacyTimestamp = rangeKV.RangeKey.Timestamp.ToLegacyTimestamp() + if size := legacyTimestamp.Size(); size > cap(timestampBuf) { + timestampBuf = make([]byte, size) + } else { + timestampBuf = timestampBuf[:size] + } + if _, err := protoutil.MarshalTo(&legacyTimestamp, timestampBuf); err != nil { + return err + } + if _, err := hasher.Write(timestampBuf); err != nil { + return err + } + _, err = hasher.Write(rangeKV.Value) + return err + } + var ms enginepb.MVCCStats // In statsOnly mode, we hash only the RangeAppliedState. In regular mode, hash // all of the replicated key space. @@ -660,7 +713,7 @@ func (*Replica) sha512( UpperBound: span.End, }) spanMS, err := storage.ComputeStatsForRangeWithVisitors( - iter, span.Start, span.End, 0 /* nowNanos */, pointKeyVisitor, nil /* rangeKeyVisitor */) + iter, span.Start, span.End, 0 /* nowNanos */, pointKeyVisitor, rangeKeyVisitor) iter.Close() if err != nil { return nil, err diff --git a/pkg/kv/kvserver/replica_consistency_diff.go b/pkg/kv/kvserver/replica_consistency_diff.go index 8d132fa31e37..92d9e8989934 100644 --- a/pkg/kv/kvserver/replica_consistency_diff.go +++ b/pkg/kv/kvserver/replica_consistency_diff.go @@ -12,6 +12,7 @@ package kvserver import ( "bytes" + fmt "fmt" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -26,6 +27,7 @@ type ReplicaSnapshotDiff struct { // holder. LeaseHolder bool Key roachpb.Key + EndKey roachpb.Key // only set for range keys (MVCCRangeKey) Timestamp hlc.Timestamp Value []byte } @@ -43,17 +45,33 @@ func (rsds ReplicaSnapshotDiffSlice) SafeFormat(buf redact.SafePrinter, _ rune) // Lease holder (RHS) has something follower (LHS) does not have. prefix = redact.SafeString("-") } - const format = `%s%s %s + if len(d.EndKey) > 0 { + const rangeKVFormat = `%s%s %s +%s ts:%s +%s value:%s +%s raw from/to/ts/value: %x %x %x %x +` + buf.Printf(rangeKVFormat, + prefix, d.Timestamp, roachpb.Span{Key: d.Key, EndKey: d.EndKey}, + prefix, d.Timestamp.GoTime(), + prefix, fmt.Sprintf("%q", d.Value), // just print the raw value for now + prefix, storage.EncodeMVCCKeyPrefix(d.Key), storage.EncodeMVCCKeyPrefix(d.EndKey), + storage.EncodeMVCCTimestampSuffix(d.Timestamp), d.Value) + + } else { + const kvFormat = `%s%s %s %s ts:%s %s value:%s %s raw mvcc_key/value: %x %x ` - mvccKey := storage.MVCCKey{Key: d.Key, Timestamp: d.Timestamp} - buf.Printf(format, - prefix, d.Timestamp, d.Key, - prefix, d.Timestamp.GoTime(), - prefix, SprintMVCCKeyValue(storage.MVCCKeyValue{Key: mvccKey, Value: d.Value}, false /* printKey */), - prefix, storage.EncodeMVCCKey(mvccKey), d.Value) + mvccKey := storage.MVCCKey{Key: d.Key, Timestamp: d.Timestamp} + buf.Printf(kvFormat, + prefix, d.Timestamp, d.Key, + prefix, d.Timestamp.GoTime(), + prefix, SprintMVCCKeyValue( + storage.MVCCKeyValue{Key: mvccKey, Value: d.Value}, false /* printKey */), + prefix, storage.EncodeMVCCKey(mvccKey), d.Value) + } } } @@ -66,15 +84,22 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice { if l == nil || r == nil { return nil } + var diff ReplicaSnapshotDiffSlice + diff = append(diff, diffKVs(l.KV, r.KV)...) + diff = append(diff, diffRangeKeys(l.RangeKV, r.RangeKV)...) + return diff +} + +func diffKVs(l, r []roachpb.RaftSnapshotData_KeyValue) ReplicaSnapshotDiffSlice { var diff []ReplicaSnapshotDiff i, j := 0, 0 for { var e, v roachpb.RaftSnapshotData_KeyValue - if i < len(l.KV) { - e = l.KV[i] + if i < len(l) { + e = l[i] } - if j < len(r.KV) { - v = r.KV[j] + if j < len(r) { + v = r[j] } addLeaseHolder := func() { @@ -88,41 +113,28 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice { // Compare keys. var comp int - // Check if it has finished traversing over all the lease holder keys. - if e.Key == nil { - if v.Key == nil { - // Done traversing over all the replica keys. Done! - break - } else { - comp = 1 - } + if e.Key == nil && v.Key == nil { + // Done! + break + } else if e.Key == nil { + // Finished traversing over all the lease holder keys. + comp = 1 + } else if v.Key == nil { + // Finished traversing over all the replica keys. + comp = -1 } else { - // Check if it has finished traversing over all the replica keys. - if v.Key == nil { - comp = -1 - } else { - // Both lease holder and replica keys exist. Compare them. - comp = bytes.Compare(e.Key, v.Key) - } + // Both lease holder and replica keys exist. Compare them. + comp = storage.MVCCKey{Key: e.Key, Timestamp: e.Timestamp}.Compare( + storage.MVCCKey{Key: v.Key, Timestamp: v.Timestamp}) } + switch comp { case -1: addLeaseHolder() - + case 1: + addReplica() case 0: - // Timestamp sorting is weird. Timestamp{} sorts first, the - // remainder sort in descending order. See storage/engine/doc.go. - if !e.Timestamp.EqOrdering(v.Timestamp) { - if e.Timestamp.IsEmpty() { - addLeaseHolder() - } else if v.Timestamp.IsEmpty() { - addReplica() - } else if v.Timestamp.Less(e.Timestamp) { - addLeaseHolder() - } else { - addReplica() - } - } else if !bytes.Equal(e.Value, v.Value) { + if !bytes.Equal(e.Value, v.Value) { addLeaseHolder() addReplica() } else { @@ -130,10 +142,66 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice { i++ j++ } + } + } + return diff +} +func diffRangeKeys(l, r []roachpb.RaftSnapshotData_RangeKeyValue) ReplicaSnapshotDiffSlice { + var diff []ReplicaSnapshotDiff + i, j := 0, 0 + for { + var e, v roachpb.RaftSnapshotData_RangeKeyValue + if i < len(l) { + e = l[i] + } + if j < len(r) { + v = r[j] + } + + addLeaseHolder := func() { + diff = append(diff, ReplicaSnapshotDiff{LeaseHolder: true, Key: e.StartKey, EndKey: e.EndKey, + Timestamp: e.Timestamp, Value: e.Value}) + i++ + } + addReplica := func() { + diff = append(diff, ReplicaSnapshotDiff{LeaseHolder: false, Key: v.StartKey, EndKey: v.EndKey, + Timestamp: v.Timestamp, Value: v.Value}) + j++ + } + + // Compare keys. + var comp int + if e.StartKey == nil && e.EndKey == nil && v.StartKey == nil && v.EndKey == nil { + // Done! + break + } else if e.StartKey == nil && e.EndKey == nil { + // Finished traversing over all the lease holder keys. + comp = 1 + } else if v.StartKey == nil && v.EndKey == nil { + // Finished traversing over all the replica keys. + comp = -1 + } else { + // Both lease holder and replica keys exist. Compare them. + eMVCC := storage.MVCCRangeKey{StartKey: e.StartKey, EndKey: e.EndKey, Timestamp: e.Timestamp} + vMVCC := storage.MVCCRangeKey{StartKey: v.StartKey, EndKey: v.EndKey, Timestamp: v.Timestamp} + comp = eMVCC.Compare(vMVCC) + } + + switch comp { + case -1: + addLeaseHolder() case 1: addReplica() - + case 0: + if !bytes.Equal(e.Value, v.Value) { + addLeaseHolder() + addReplica() + } else { + // No diff; skip. + i++ + j++ + } } } return diff diff --git a/pkg/kv/kvserver/replica_consistency_test.go b/pkg/kv/kvserver/replica_consistency_test.go index 66afdee534ff..9c5846463698 100644 --- a/pkg/kv/kvserver/replica_consistency_test.go +++ b/pkg/kv/kvserver/replica_consistency_test.go @@ -12,15 +12,22 @@ package kvserver import ( "context" + "fmt" + "strings" "testing" "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/stretchr/testify/require" @@ -118,3 +125,91 @@ func TestGetChecksumNotSuccessfulExitConditions(t *testing.T) { } require.Nil(t, rc.Checksum) } + +// TestReplicaChecksumSHA512 checks that a given dataset produces the expected +// checksum. +func TestReplicaChecksumSHA512(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + storage.DisableMetamorphicSimpleValueEncoding(t) + + ctx := context.Background() + sb := &strings.Builder{} + lim := quotapool.NewRateLimiter("rate", 1e9, 0) + eng := storage.NewDefaultInMemForTesting() + defer eng.Close() + + repl := &Replica{} // We don't actually need the replica at all, just the method. + desc := roachpb.RangeDescriptor{ + RangeID: 1, + StartKey: roachpb.RKey("a"), + EndKey: roachpb.RKey("z"), + } + + // Hash the empty state. + rh, err := repl.sha512(ctx, desc, eng, nil, roachpb.ChecksumMode_CHECK_FULL, lim) + require.NoError(t, err) + fmt.Fprintf(sb, "checksum0: %x\n", rh.SHA512[:]) + + // We incrementally add writes, and check the checksums after each write to + // make sure they differ such that each write affects the checksum. + kvs := []struct { + key string + endKey string + ts int64 + localTS int64 + value string + }{ + {"a", "", 1, 0, "a1"}, + {"a", "", 2, 0, ""}, + {"b", "", 3, 2, "b3"}, + {"i", "", 0, 0, "i0"}, + // Range keys can currently only be tombstones. + {"p", "q", 1, 0, ""}, + {"x", "z", 9, 8, ""}, + } + + for i, kv := range kvs { + key, endKey := roachpb.Key(kv.key), roachpb.Key(kv.endKey) + ts := hlc.Timestamp{WallTime: kv.ts} + localTS := hlc.ClockTimestamp{WallTime: kv.localTS} + var value roachpb.Value + if kv.value != "" { + value = roachpb.MakeValueFromString(kv.value) + } + + if len(endKey) > 0 { + require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone( + ctx, eng, nil, key, endKey, ts, localTS, nil, nil, 0)) + } else { + require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, ts, localTS, value, nil)) + } + + rh, err = repl.sha512(ctx, desc, eng, nil, roachpb.ChecksumMode_CHECK_FULL, lim) + require.NoError(t, err) + fmt.Fprintf(sb, "checksum%d: %x\n", i+1, rh.SHA512[:]) + } + + // Run another check to obtain a snapshot and stats for the final state. + kvSnapshot := roachpb.RaftSnapshotData{} + rh, err = repl.sha512(ctx, desc, eng, &kvSnapshot, roachpb.ChecksumMode_CHECK_FULL, lim) + require.NoError(t, err) + + jsonpb := protoutil.JSONPb{Indent: " "} + json, err := jsonpb.Marshal(&rh.RecomputedMS) + require.NoError(t, err) + fmt.Fprintf(sb, "stats: %s\n", string(json)) + + fmt.Fprint(sb, "snapshot:\n") + for _, kv := range kvSnapshot.KV { + fmt.Fprintf(sb, " %s=%q\n", storage.MVCCKey{Key: kv.Key, Timestamp: kv.Timestamp}, kv.Value) + } + for _, rkv := range kvSnapshot.RangeKV { + fmt.Fprintf(sb, " %s=%q\n", + storage.MVCCRangeKey{StartKey: rkv.StartKey, EndKey: rkv.EndKey, Timestamp: rkv.Timestamp}, + rkv.Value) + } + + echotest.Require(t, sb.String(), testutils.TestDataPath(t, "replica_consistency_sha512")) +} diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index a6a0063110b4..16f996ec0ba3 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -53,6 +53,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -7520,9 +7521,7 @@ func TestDiffRange(t *testing.T) { // TODO(tschottdorf): this test should really pass the data through a // RocksDB engine to verify that the original snapshots sort correctly. - if diff := diffRange(nil, nil); diff != nil { - t.Fatalf("diff of nils = %v", diff) - } + require.Empty(t, diffRange(nil, nil)) timestamp := hlc.Timestamp{WallTime: 1729, Logical: 1} value := []byte("foo") @@ -7549,12 +7548,19 @@ func TestDiffRange(t *testing.T) { {Key: []byte("zeroleft"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, Value: value}, {Key: []byte("zeroright"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, Value: value}, }, + // Fragmented range keys. + RangeKV: []roachpb.RaftSnapshotData_RangeKeyValue{ + {StartKey: []byte("A"), EndKey: []byte("C"), Timestamp: timestamp}, + {StartKey: []byte("A"), EndKey: []byte("C"), Timestamp: timestamp.Add(0, -1)}, + {StartKey: []byte("C"), EndKey: []byte("F"), Timestamp: timestamp}, + {StartKey: []byte("P"), EndKey: []byte("R"), Timestamp: timestamp}, + {StartKey: []byte("S"), EndKey: []byte("T"), Timestamp: timestamp, Value: []byte{1}}, + {StartKey: []byte("X"), EndKey: []byte("Z"), Timestamp: timestamp}, + }, } // No diff works. - if diff := diffRange(leaderSnapshot, leaderSnapshot); diff != nil { - t.Fatalf("diff of equal snapshots = %v", diff) - } + require.Empty(t, diffRange(leaderSnapshot, leaderSnapshot)) replicaSnapshot := &roachpb.RaftSnapshotData{ KV: []roachpb.RaftSnapshotData_KeyValue{ @@ -7572,6 +7578,13 @@ func TestDiffRange(t *testing.T) { {Key: []byte("zeroright"), Timestamp: hlc.Timestamp{}, Value: value}, {Key: []byte("zeroright"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, Value: value}, }, + RangeKV: []roachpb.RaftSnapshotData_RangeKeyValue{ + {StartKey: []byte("A"), EndKey: []byte("C"), Timestamp: timestamp}, + {StartKey: []byte("E"), EndKey: []byte("G"), Timestamp: timestamp}, + {StartKey: []byte("Q"), EndKey: []byte("R"), Timestamp: timestamp}, + {StartKey: []byte("S"), EndKey: []byte("T"), Timestamp: timestamp, Value: []byte{2}}, + {StartKey: []byte("X"), EndKey: []byte("Z"), Timestamp: timestamp.Add(0, 1)}, + }, } // The expected diff. @@ -7588,48 +7601,23 @@ func TestDiffRange(t *testing.T) { {LeaseHolder: false, Key: []byte("z"), Timestamp: timestamp, Value: value}, {LeaseHolder: true, Key: []byte("zeroleft"), Timestamp: hlc.Timestamp{}, Value: value}, {LeaseHolder: false, Key: []byte("zeroright"), Timestamp: hlc.Timestamp{}, Value: value}, - } - diff := diffRange(leaderSnapshot, replicaSnapshot) - - for i, e := range eDiff { - v := diff[i] - if e.LeaseHolder != v.LeaseHolder || !bytes.Equal(e.Key, v.Key) || e.Timestamp != v.Timestamp || !bytes.Equal(e.Value, v.Value) { - t.Fatalf("diff varies at row %d, want %v and got %v\n\ngot:\n%s\nexpected:\n%s", i, e, v, diff, eDiff) - } + {LeaseHolder: true, Key: []byte("A"), EndKey: []byte("C"), Timestamp: timestamp.Add(0, -1)}, + {LeaseHolder: true, Key: []byte("C"), EndKey: []byte("F"), Timestamp: timestamp}, + {LeaseHolder: false, Key: []byte("E"), EndKey: []byte("G"), Timestamp: timestamp}, + {LeaseHolder: true, Key: []byte("P"), EndKey: []byte("R"), Timestamp: timestamp}, + {LeaseHolder: false, Key: []byte("Q"), EndKey: []byte("R"), Timestamp: timestamp}, + {LeaseHolder: true, Key: []byte("S"), EndKey: []byte("T"), Timestamp: timestamp, Value: []byte{1}}, + {LeaseHolder: false, Key: []byte("S"), EndKey: []byte("T"), Timestamp: timestamp, Value: []byte{2}}, + {LeaseHolder: false, Key: []byte("X"), EndKey: []byte("Z"), Timestamp: timestamp.Add(0, 1)}, + {LeaseHolder: true, Key: []byte("X"), EndKey: []byte("Z"), Timestamp: timestamp}, } + diff := diffRange(leaderSnapshot, replicaSnapshot) + require.Equal(t, eDiff, diff) - // Document the stringifed output. This is what the consistency checker + // Assert the stringifed output. This is what the consistency checker // will actually print. - stringDiff := append(eDiff[:4], - ReplicaSnapshotDiff{Key: []byte("foo"), Value: value}, - ) - - const expDiff = `--- leaseholder -+++ follower --0.000001729,1 "a" -- ts:1970-01-01 00:00:00.000001729 +0000 UTC -- value:"foo" -- raw mvcc_key/value: 610000000000000006c1000000010d 666f6f -+0.000001729,1 "ab" -+ ts:1970-01-01 00:00:00.000001729 +0000 UTC -+ value:"foo" -+ raw mvcc_key/value: 61620000000000000006c1000000010d 666f6f --0.000001729,1 "abcd" -- ts:1970-01-01 00:00:00.000001729 +0000 UTC -- value:"foo" -- raw mvcc_key/value: 616263640000000000000006c1000000010d 666f6f -+0.000001729,1 "abcdef" -+ ts:1970-01-01 00:00:00.000001729 +0000 UTC -+ value:"foo" -+ raw mvcc_key/value: 6162636465660000000000000006c1000000010d 666f6f -+0,0 "foo" -+ ts:1970-01-01 00:00:00 +0000 UTC -+ value:"foo" -+ raw mvcc_key/value: 666f6f00 666f6f -` - - require.Equal(t, expDiff, stringDiff.String()) + echotest.Require(t, diff.String(), testutils.TestDataPath(t, "replica_consistency_diff")) } func TestSyncSnapshot(t *testing.T) { diff --git a/pkg/kv/kvserver/testdata/replica_consistency_diff b/pkg/kv/kvserver/testdata/replica_consistency_diff new file mode 100644 index 000000000000..fc447766d56f --- /dev/null +++ b/pkg/kv/kvserver/testdata/replica_consistency_diff @@ -0,0 +1,88 @@ +echo +---- +--- leaseholder ++++ follower +-0.000001729,1 "a" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 610000000000000006c1000000010d 666f6f ++0.000001729,1 "ab" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 61620000000000000006c1000000010d 666f6f +-0.000001729,1 "abcd" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 616263640000000000000006c1000000010d 666f6f ++0.000001729,1 "abcdef" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 6162636465660000000000000006c1000000010d 666f6f ++0.000001729,2 "abcdefg" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 616263646566670000000000000006c1000000020d 666f6f +-0.000001729,0 "abcdefg" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 616263646566670000000000000006c109 666f6f +-0.000001729,1 "x" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 780000000000000006c1000000010d 666f6f ++0.000001729,1 "x" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"bar" ++ raw mvcc_key/value: 780000000000000006c1000000010d 626172 +-0.000001729,1 "y" +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 790000000000000006c1000000010d 666f6f ++0.000001729,1 "z" ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 7a0000000000000006c1000000010d 666f6f +-0,0 "zeroleft" +- ts:1970-01-01 00:00:00 +0000 UTC +- value:"foo" +- raw mvcc_key/value: 7a65726f6c65667400 666f6f ++0,0 "zeroright" ++ ts:1970-01-01 00:00:00 +0000 UTC ++ value:"foo" ++ raw mvcc_key/value: 7a65726f726967687400 666f6f +-0.000001729,0 {A-C} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"" +- raw from/to/ts/value: 4100 4300 00000000000006c109 +-0.000001729,1 {C-F} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"" +- raw from/to/ts/value: 4300 4600 00000000000006c1000000010d ++0.000001729,1 {E-G} ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"" ++ raw from/to/ts/value: 4500 4700 00000000000006c1000000010d +-0.000001729,1 {P-R} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"" +- raw from/to/ts/value: 5000 5200 00000000000006c1000000010d ++0.000001729,1 {Q-R} ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"" ++ raw from/to/ts/value: 5100 5200 00000000000006c1000000010d +-0.000001729,1 {S-T} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"\x01" +- raw from/to/ts/value: 5300 5400 00000000000006c1000000010d 01 ++0.000001729,1 {S-T} ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"\x02" ++ raw from/to/ts/value: 5300 5400 00000000000006c1000000010d 02 ++0.000001729,2 {X-Z} ++ ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ value:"" ++ raw from/to/ts/value: 5800 5a00 00000000000006c1000000020d +-0.000001729,1 {X-Z} +- ts:1970-01-01 00:00:00.000001729 +0000 UTC +- value:"" +- raw from/to/ts/value: 5800 5a00 00000000000006c1000000010d diff --git a/pkg/kv/kvserver/testdata/replica_consistency_sha512 b/pkg/kv/kvserver/testdata/replica_consistency_sha512 new file mode 100644 index 000000000000..c82d11f7634d --- /dev/null +++ b/pkg/kv/kvserver/testdata/replica_consistency_sha512 @@ -0,0 +1,28 @@ +echo +---- +checksum0: cf83e1357eefb8bdf1542850d66d8007d620e4050b5715dc83f4a921d36ce9ce47d0d13c5d85f2b0ff8318d2877eec2f63b931bd47417a81a538327af927da3e +checksum1: 67ba8f144d3c44d3ed82a5ec3758750642315af4de193c08a52a385e1027945cdd1c4c6a8f54112be2dd71d476924505fbe23fad8d40061df74c596dc583fd5f +checksum2: fc2a7a023637caa80bb923075f7d39451a328a3fe1357c08353e594810a0c22c56cad1b494662d02dcf3187e0ce44780e5b4b039ddcf5dee6ba974a542a4ffb3 +checksum3: 05186bceae59a178713407959a26110715a1e299e6a9f1b37fc3e0f8d5a0c66bedbff8378a5f7b2e3d31c929a3b985088c7b714019b9eeacc50b17924925320a +checksum4: 4f5cc8176d559bfab8e52b74851b103fd73b9e713ce12aa380a16fe177ca6e21db75e3e85a58341ab437a5a766a071a2fe6e1f03841d334da7be2295794eb813 +checksum5: 3c5d5856a626aa29913e9790033b9c23b6dc5e42bdf2e665f7b60f58bec495adc246bf4e5f5bf1acbfc78c713f2ec7820b4ba7202897bb9f824a0b7b9e9cc98d +checksum6: ebe7fd3f41a68c2608a8b10dcc9db3b39bdb6c097d3fd99411e89d75419bb58dd80faf9846aa5e47d8cabc9dcfc894c6ea58f7e035eaaa3ee55c31faed2c8000 +stats: { + "liveBytes": "53", + "liveCount": "2", + "keyBytes": "42", + "keyCount": "3", + "valBytes": "44", + "valCount": "4", + "rangeKeyCount": "2", + "rangeKeyBytes": "26", + "rangeValCount": "2", + "rangeValBytes": "9" +} +snapshot: + "a"/0.000000002,0="" + "a"/0.000000001,0="\x00\x00\x00\x00\x03a1" + "b"/0.000000003,0="\x00\x00\x00\x04e\n\x02\b\x02\x00\x00\x00\x00\x03b3" + "i"/0,0="\x12\x04\b\x00\x10\x00\x18\x00 \x00(\x002\a\x00\x00\x00\x00\x03i0" + {p-q}/0.000000001,0="" + {x-z}/0.000000009,0="\x00\x00\x00\x04e\n\x02\b\b" diff --git a/pkg/roachpb/internal_raft.proto b/pkg/roachpb/internal_raft.proto index 5837411d43ff..7460b4bcc089 100644 --- a/pkg/roachpb/internal_raft.proto +++ b/pkg/roachpb/internal_raft.proto @@ -34,8 +34,9 @@ message RangeTombstone { (gogoproto.customname) = "NextReplicaID", (gogoproto.casttype) = "ReplicaID"]; } -// RaftSnapshotData is the payload of a raftpb.Snapshot. It contains a raw copy of -// all of the range's data and metadata, including the raft log, abort span, etc. +// RaftSnapshotData is a historical vestige that used to carry snapshot data, +// but is now only used in CollectChecksumResponse to carry range KV data to +// generate diffs for checksum mismatches. message RaftSnapshotData { message KeyValue { optional bytes key = 1; @@ -44,6 +45,16 @@ message RaftSnapshotData { } repeated KeyValue KV = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "KV"]; + + message RangeKeyValue { + optional bytes start_key = 1; + optional bytes end_key = 2; + optional util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; + optional bytes value = 4; + } + repeated RangeKeyValue range_kv = 4 [(gogoproto.nullable) = false, + (gogoproto.customname) = "RangeKV"]; + // These are really raftpb.Entry, but we model them as raw bytes to avoid // roundtripping through memory. repeated bytes log_entries = 3; From cd5530826c8097dbe0657b45c7c40a172ea6d02e Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Tue, 28 Jun 2022 10:28:23 +0100 Subject: [PATCH 05/15] streamingccl: deflake TestPartitionedStreamReplicationClient Previously, this test would fail occasionally with: ``` partitioned_stream_client_test.go:192: Error Trace: partitioned_stream_client_test.go:192 Error: Target error should be in err chain: expected: "context canceled" in chain: "pq: query execution canceled" Test: TestPartitionedStreamReplicationClient ``` This is a result of the lib/pq library asyncronously sending a CancelRequest when it notices the context cancellation. The cancel request may result in the query ending before the local context cancellation produces an error. We now count this query cancellation error as an acceptable response since the goal of the test is to assert that we respond to context cancellation. Fixes #76919 Release note: None --- pkg/ccl/streamingccl/streamclient/BUILD.bazel | 4 ++++ .../partitioned_stream_client_test.go | 21 ++++++++++++++++++- 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/pkg/ccl/streamingccl/streamclient/BUILD.bazel b/pkg/ccl/streamingccl/streamclient/BUILD.bazel index b4d3bee6c303..af2bd76f5944 100644 --- a/pkg/ccl/streamingccl/streamclient/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamclient/BUILD.bazel @@ -59,11 +59,13 @@ go_test( "//pkg/security/securitytest", "//pkg/server", "//pkg/sql/catalog/desctestutils", + "//pkg/sql/pgwire/pgcode", "//pkg/streaming", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/skip", "//pkg/testutils/testcluster", + "//pkg/util/cancelchecker", "//pkg/util/ctxgroup", "//pkg/util/hlc", "//pkg/util/leaktest", @@ -72,6 +74,8 @@ go_test( "//pkg/util/randutil", "//pkg/util/syncutil", "//pkg/util/timeutil", + "@com_github_cockroachdb_errors//:errors", + "@com_github_lib_pq//:pq", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/ccl/streamingccl/streamclient/partitioned_stream_client_test.go b/pkg/ccl/streamingccl/streamclient/partitioned_stream_client_test.go index 46059527f042..15a59e398422 100644 --- a/pkg/ccl/streamingccl/streamclient/partitioned_stream_client_test.go +++ b/pkg/ccl/streamingccl/streamclient/partitioned_stream_client_test.go @@ -11,6 +11,7 @@ package streamclient import ( "context" "fmt" + "strings" "testing" "time" @@ -23,16 +24,20 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/streaming" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" + "github.com/lib/pq" "github.com/stretchr/testify/require" ) @@ -189,7 +194,12 @@ INSERT INTO d.t2 VALUES (2); // Test if Subscribe can react to cancellation signal. cancelFn() - require.ErrorIs(t, cg.Wait(), context.Canceled) + + // When the context is cancelled, lib/pq sends a query cancellation message to + // the server. Occasionally, we see the error from this cancellation before + // the subscribe function sees our local context cancellation. + err = cg.Wait() + require.True(t, errors.Is(err, context.Canceled) || isQueryCanceledError(err)) // Testing client.Complete() err = client.Complete(ctx, streaming.StreamID(999)) @@ -205,3 +215,12 @@ SET CLUSTER SETTING stream_replication.stream_liveness_track_frequency = '200ms' h.SysDB.CheckQueryResultsRetry(t, fmt.Sprintf("SELECT status FROM [SHOW JOBS] WHERE job_id = %d", streamID), [][]string{{"succeeded"}}) } + +// isQueryCanceledError returns true if the error appears to be a query cancelled error. +func isQueryCanceledError(err error) bool { + var pqErr pq.Error + if ok := errors.As(err, &pqErr); ok { + return pqErr.Code == pq.ErrorCode(pgcode.QueryCanceled.String()) + } + return strings.Contains(err.Error(), cancelchecker.QueryCanceledError.Error()) +} From 39a562c5340adc407c85adcd1b2bec0f5be76344 Mon Sep 17 00:00:00 2001 From: Thomas Hardy Date: Thu, 16 Jun 2022 21:11:30 -0400 Subject: [PATCH 06/15] sql: add plan gist to sampled query telemetry log Partially resolves: #71328 This change adds a plan gist field to the sampled query telemetry log. The plan gist is written as a base64 encoded string. Release note (sql change): The sampled query telemetry log now includes a plan gist field. The plan gist field provides a compact representation of a logical plan for the sampled query, the field is written as a base64 encoded string. --- docs/generated/eventlog.md | 1 + pkg/sql/exec_log.go | 1 + pkg/sql/telemetry_logging_test.go | 5 +++++ pkg/util/log/eventpb/json_encode_generated.go | 10 ++++++++++ pkg/util/log/eventpb/telemetry.proto | 3 +++ 5 files changed, 20 insertions(+) diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index 1525b438f0d7..7ce22dbc7c8c 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -2458,6 +2458,7 @@ contains common SQL event/execution details. | `SkippedQueries` | skipped_queries indicate how many SQL statements were not considered for sampling prior to this one. If the field is omitted, or its value is zero, this indicates that no statement was omitted since the last event. | no | | `CostEstimate` | Cost of the query as estimated by the optimizer. | no | | `Distribution` | The distribution of the DistSQL query plan (local, full, or partial). | no | +| `PlanGist` | The query's plan gist bytes as a base64 encoded string. | no | #### Common fields diff --git a/pkg/sql/exec_log.go b/pkg/sql/exec_log.go index b44143c233e9..c28cc67ee2b2 100644 --- a/pkg/sql/exec_log.go +++ b/pkg/sql/exec_log.go @@ -389,6 +389,7 @@ func (p *planner) maybeLogStatementInternal( SkippedQueries: skippedQueries, CostEstimate: p.curPlan.instrumentation.costEstimate, Distribution: p.curPlan.instrumentation.distribution.String(), + PlanGist: p.curPlan.instrumentation.planGist.String(), }}) } else { telemetryMetrics.incSkippedQueryCount() diff --git a/pkg/sql/telemetry_logging_test.go b/pkg/sql/telemetry_logging_test.go index 39d8b5d3c560..2bae4296dd28 100644 --- a/pkg/sql/telemetry_logging_test.go +++ b/pkg/sql/telemetry_logging_test.go @@ -231,6 +231,11 @@ func TestTelemetryLogging(t *testing.T) { if !distRe.MatchString(e.Message) { t.Errorf("expected to find Distribution but none was found") } + // Match plan gist on any non-empty string value. + planGist := regexp.MustCompile("\"PlanGist\":(\"\\S+\")") + if !planGist.MatchString(e.Message) { + t.Errorf("expected to find PlanGist but none was found in: %s", e.Message) + } for _, eTag := range tc.expectedUnredactedTags { for _, tag := range strings.Split(e.Tags, ",") { kv := strings.Split(tag, "=") diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index 1a87e36e7a1e..f11fb41af51a 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -3285,6 +3285,16 @@ func (m *SampledQuery) AppendJSONFields(printComma bool, b redact.RedactableByte b = append(b, '"') } + if m.PlanGist != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"PlanGist\":\""...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(m.PlanGist))) + b = append(b, '"') + } + return printComma, b } diff --git a/pkg/util/log/eventpb/telemetry.proto b/pkg/util/log/eventpb/telemetry.proto index c8256a73c9d0..186dd8e0de7f 100644 --- a/pkg/util/log/eventpb/telemetry.proto +++ b/pkg/util/log/eventpb/telemetry.proto @@ -43,6 +43,9 @@ message SampledQuery { // The distribution of the DistSQL query plan (local, full, or partial). string distribution = 6 [(gogoproto.jsontag) = ",omitempty", (gogoproto.moretags) = "redact:\"nonsensitive\""]; + + // The query's plan gist bytes as a base64 encoded string. + string plan_gist = 7 [(gogoproto.jsontag) = ',omitempty', (gogoproto.moretags) = "redact:\"nonsensitive\""]; } // CapturedIndexUsageStats From d1d3c428069da68a64f86f9591f10227bdcc82f2 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Tue, 14 Jun 2022 14:27:55 -0400 Subject: [PATCH 07/15] roachprod: add promethius/grafana monitoring Previously, only roachtests could spin up prom/grafana servers that lasted the lifetime of the roachtest. This PR introduces new roachprod cmds that allow a roachprod user to easily spin up/down their own prom/grafana instances. The PR also hooks up roachtests that rely on prom/grafana into this new infrastructure. Release note: none --- pkg/cmd/roachprod/flags.go | 16 + pkg/cmd/roachprod/main.go | 43 ++ pkg/cmd/roachtest/BUILD.bazel | 1 + pkg/cmd/roachtest/cluster.go | 11 + pkg/cmd/roachtest/cluster/BUILD.bazel | 1 + .../roachtest/cluster/cluster_interface.go | 4 + pkg/cmd/roachtest/option/node_list_option.go | 11 + pkg/cmd/roachtest/tests/canary.go | 11 - pkg/cmd/roachtest/tests/tpcc.go | 45 +- pkg/gen/gomock.bzl | 1 - pkg/roachprod/BUILD.bazel | 1 + pkg/roachprod/install/BUILD.bazel | 1 + pkg/roachprod/install/cluster_synced.go | 109 +++-- pkg/roachprod/prometheus/BUILD.bazel | 25 +- .../prometheus/mocks_generated_test.go | 110 ----- pkg/roachprod/prometheus/prometheus.go | 383 +++++++++++------- pkg/roachprod/prometheus/prometheus_test.go | 209 ++++------ .../testdata/multipleScrapeNodes.txt | 19 + .../prometheus/testdata/usingMakeCommands.txt | 28 ++ pkg/roachprod/roachprod.go | 166 +++++++- 20 files changed, 707 insertions(+), 488 deletions(-) delete mode 100644 pkg/roachprod/prometheus/mocks_generated_test.go create mode 100644 pkg/roachprod/prometheus/testdata/multipleScrapeNodes.txt create mode 100644 pkg/roachprod/prometheus/testdata/usingMakeCommands.txt diff --git a/pkg/cmd/roachprod/flags.go b/pkg/cmd/roachprod/flags.go index 5341fafe53d6..248e6681a926 100644 --- a/pkg/cmd/roachprod/flags.go +++ b/pkg/cmd/roachprod/flags.go @@ -39,6 +39,9 @@ var ( destroyAllLocal bool extendLifetime time.Duration wipePreserveCerts bool + grafanaConfig string + grafanaurlOpen bool + grafanaDumpDir string listDetails bool listJSON bool listMine bool @@ -232,6 +235,19 @@ func initFlags() { cachedHostsCmd.Flags().StringVar(&cachedHostsCluster, "cluster", "", "print hosts matching cluster") + // TODO (msbutler): this flag should instead point to a relative file path that's check into + // the repo, not some random URL. + grafanaStartCmd.Flags().StringVar(&grafanaConfig, + "grafana-config", "", "URL to grafana json config") + + grafanaURLCmd.Flags().BoolVar(&grafanaurlOpen, + "open", false, "open the grafana dashboard url on the browser") + + grafanaStopCmd.Flags().StringVar(&grafanaDumpDir, "dump-dir", "", + "the absolute path, on the machine running roachprod, to dump prometheus data to.\n"+ + "In the dump-dir, the 'prometheus-docker-run.sh' script spins up a prometheus UI accessible on \n"+ + " 0.0.0.0:9090. If dump-dir is empty, no data will get dumped.") + for _, cmd := range []*cobra.Command{createCmd, destroyCmd, extendCmd, logsCmd} { cmd.Flags().StringVarP(&username, "username", "u", os.Getenv("ROACHPROD_USER"), "Username to run under, detect if blank") diff --git a/pkg/cmd/roachprod/main.go b/pkg/cmd/roachprod/main.go index ecfb6973379a..bfa17c6744a9 100644 --- a/pkg/cmd/roachprod/main.go +++ b/pkg/cmd/roachprod/main.go @@ -885,6 +885,46 @@ var getProvidersCmd = &cobra.Command{ }, } +var grafanaStartCmd = &cobra.Command{ + Use: `grafana-start `, + Short: `spins up a prometheus and grafana instances on the last node in the cluster`, + Long: `spins up a prometheus and grafana instances on the highest numbered node in the cluster +and will scrape from all nodes in the cluster`, + Args: cobra.ExactArgs(1), + Run: wrap(func(cmd *cobra.Command, args []string) error { + return roachprod.StartGrafana(context.Background(), roachprodLibraryLogger, args[0], + grafanaConfig, nil) + }), +} + +var grafanaStopCmd = &cobra.Command{ + Use: `grafana-stop `, + Short: `spins down prometheus and grafana instances on the last node in the cluster`, + Long: `spins down the prometheus and grafana instances on the last node in the cluster`, + Args: cobra.ExactArgs(1), + Run: wrap(func(cmd *cobra.Command, args []string) error { + return roachprod.StopGrafana(context.Background(), roachprodLibraryLogger, args[0], grafanaDumpDir) + }), +} + +var grafanaURLCmd = &cobra.Command{ + Use: `grafanaurl `, + Short: `returns a url to the grafana dashboard`, + Args: cobra.ExactArgs(1), + Run: wrap(func(cmd *cobra.Command, args []string) error { + urls, err := roachprod.GrafanaURL(context.Background(), roachprodLibraryLogger, args[0], + grafanaurlOpen) + if err != nil { + return err + } + for _, url := range urls { + fmt.Println(url) + } + fmt.Println("username: admin; pwd: admin") + return nil + }), +} + func main() { loggerCfg := logger.Config{Stdout: os.Stdout, Stderr: os.Stderr} var loggerError error @@ -935,6 +975,9 @@ func main() { cachedHostsCmd, versionCmd, getProvidersCmd, + grafanaStartCmd, + grafanaStopCmd, + grafanaURLCmd, ) setBashCompletionFunction() diff --git a/pkg/cmd/roachtest/BUILD.bazel b/pkg/cmd/roachtest/BUILD.bazel index 29d88e8a0bf0..ee3b91a299d6 100644 --- a/pkg/cmd/roachtest/BUILD.bazel +++ b/pkg/cmd/roachtest/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/roachprod/config", "//pkg/roachprod/install", "//pkg/roachprod/logger", + "//pkg/roachprod/prometheus", "//pkg/roachprod/vm", "//pkg/testutils/skip", "//pkg/util/contextutil", diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 7110744b5711..4e34fa6283d9 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -41,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachprod" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/prometheus" "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -2384,3 +2385,13 @@ func (c *clusterImpl) Extend(ctx context.Context, d time.Duration, l *logger.Log func (c *clusterImpl) NewMonitor(ctx context.Context, opts ...option.Option) cluster.Monitor { return newMonitor(ctx, c.t, c, opts...) } + +func (c *clusterImpl) StartGrafana( + ctx context.Context, l *logger.Logger, promCfg *prometheus.Config, +) error { + return roachprod.StartGrafana(ctx, l, c.name, "", promCfg) +} + +func (c *clusterImpl) StopGrafana(ctx context.Context, l *logger.Logger, dumpDir string) error { + return roachprod.StopGrafana(ctx, l, c.name, dumpDir) +} diff --git a/pkg/cmd/roachtest/cluster/BUILD.bazel b/pkg/cmd/roachtest/cluster/BUILD.bazel index b74f36a4689f..d77f58502166 100644 --- a/pkg/cmd/roachtest/cluster/BUILD.bazel +++ b/pkg/cmd/roachtest/cluster/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/cmd/roachtest/test", "//pkg/roachprod/install", "//pkg/roachprod/logger", + "//pkg/roachprod/prometheus", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/cmd/roachtest/cluster/cluster_interface.go b/pkg/cmd/roachtest/cluster/cluster_interface.go index 245ddf07840b..a1d42c28fe32 100644 --- a/pkg/cmd/roachtest/cluster/cluster_interface.go +++ b/pkg/cmd/roachtest/cluster/cluster_interface.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/prometheus" ) // Cluster is the interface through which a given roachtest interacts with the @@ -131,4 +132,7 @@ type Cluster interface { FetchTimeseriesData(ctx context.Context, t test.Test) error RefetchCertsFromNode(ctx context.Context, node int) error + + StartGrafana(ctx context.Context, l *logger.Logger, promCfg *prometheus.Config) error + StopGrafana(ctx context.Context, l *logger.Logger, dumpDir string) error } diff --git a/pkg/cmd/roachtest/option/node_list_option.go b/pkg/cmd/roachtest/option/node_list_option.go index cea08a3095fb..475d78d2c956 100644 --- a/pkg/cmd/roachtest/option/node_list_option.go +++ b/pkg/cmd/roachtest/option/node_list_option.go @@ -16,6 +16,8 @@ import ( "math/rand" "sort" "strconv" + + "github.com/cockroachdb/cockroach/pkg/roachprod/install" ) // A NodeListOption is a slice of roachprod node identifiers. The first node is @@ -116,3 +118,12 @@ func (n NodeListOption) String() string { } return buf.String() } + +// InstallNodes converts the NodeListOption to install.Nodes +func (n NodeListOption) InstallNodes() install.Nodes { + installNodes := make(install.Nodes, 0, len(n)) + for _, i := range n { + installNodes = append(installNodes, install.Node(i)) + } + return installNodes +} diff --git a/pkg/cmd/roachtest/tests/canary.go b/pkg/cmd/roachtest/tests/canary.go index 488ab408ab41..79ffc2709e43 100644 --- a/pkg/cmd/roachtest/tests/canary.go +++ b/pkg/cmd/roachtest/tests/canary.go @@ -109,17 +109,6 @@ var canaryRetryOptions = retry.Options{ MaxRetries: 10, } -type repeatRunner struct { - T test.Test - C cluster.Cluster -} - -func (rr repeatRunner) repeatRunE( - ctx context.Context, node option.NodeListOption, operation string, args ...string, -) error { - return repeatRunE(ctx, rr.T, rr.C, node, operation, args...) -} - // repeatRunE is the same function as c.RunE but with an automatic retry loop. func repeatRunE( ctx context.Context, diff --git a/pkg/cmd/roachtest/tests/tpcc.go b/pkg/cmd/roachtest/tests/tpcc.go index 3a86519f4453..a7aa3f24d588 100644 --- a/pkg/cmd/roachtest/tests/tpcc.go +++ b/pkg/cmd/roachtest/tests/tpcc.go @@ -99,7 +99,6 @@ type workloadInstance struct { } const workloadPProfStartPort = 33333 -const workloadPrometheusPort = 2112 // tpccImportCmd generates the command string to load tpcc data for the // specified warehouse count into a cluster. @@ -183,8 +182,7 @@ func runTPCC(ctx context.Context, t test.Test, c cluster.Cluster, opts tpccOptio workloadInstances = append( workloadInstances, workloadInstance{ - nodes: c.Range(1, c.Spec().NodeCount-1), - prometheusPort: workloadPrometheusPort, + nodes: c.Range(1, c.Spec().NodeCount-1), }, ) } @@ -202,7 +200,7 @@ func runTPCC(ctx context.Context, t test.Test, c cluster.Cluster, opts tpccOptio return } cep, err := opts.ChaosEventsProcessor( - promCfg.PrometheusNode, + c.Nodes(int(promCfg.PrometheusNode[0])), workloadInstances, ) if err != nil { @@ -1401,13 +1399,13 @@ func registerTPCCBench(r registry.Registry) { // makeWorkloadScrapeNodes creates a ScrapeNode for every workloadInstance. func makeWorkloadScrapeNodes( - workloadNode option.NodeListOption, workloadInstances []workloadInstance, + workloadNode install.Node, workloadInstances []workloadInstance, ) []prometheus.ScrapeNode { workloadScrapeNodes := make([]prometheus.ScrapeNode, len(workloadInstances)) for i, workloadInstance := range workloadInstances { workloadScrapeNodes[i] = prometheus.ScrapeNode{ - Nodes: workloadNode, - Port: workloadInstance.prometheusPort, + Node: workloadNode, + Port: workloadInstance.prometheusPort, } } return workloadScrapeNodes @@ -1435,14 +1433,14 @@ func setupPrometheusForTPCC( if opts.DisablePrometheus { return nil, func() {} } - workloadNode := c.Node(c.Spec().NodeCount) - cfg = &prometheus.Config{ - PrometheusNode: workloadNode, - // Scrape each CockroachDB node and the workload node. - ScrapeConfigs: append(prometheus.MakeInsecureCockroachScrapeConfig(c.Range(1, c.Spec().NodeCount-1)), - prometheus.MakeWorkloadScrapeConfig("workload", makeWorkloadScrapeNodes(workloadNode, workloadInstances)), - ), - } + cfg = &prometheus.Config{} + workloadNode := c.Node(c.Spec().NodeCount).InstallNodes()[0] + cfg.WithPrometheusNode(workloadNode) + cfg.WithNodeExporter(c.Range(1, c.Spec().NodeCount-1).InstallNodes()) + cfg.WithCluster(c.Range(1, c.Spec().NodeCount-1).InstallNodes()) + cfg.ScrapeConfigs = append(cfg.ScrapeConfigs, prometheus.MakeWorkloadScrapeConfig("workload", + "/", makeWorkloadScrapeNodes(workloadNode, workloadInstances))) + } if opts.DisablePrometheus { t.Fatal("test has PrometheusConfig but DisablePrometheus was on") @@ -1451,15 +1449,14 @@ func setupPrometheusForTPCC( t.Skip("skipping test as prometheus is needed, but prometheus does not yet work locally") return nil, func() {} } - _, saveSnap, err := prometheus.Init( - ctx, - *cfg, - c, - t.L(), - repeatRunner{C: c, T: t}.repeatRunE, - ) - if err != nil { + + if err := c.StartGrafana(ctx, t.L(), cfg); err != nil { t.Fatal(err) } - return cfg, func() { saveSnap(t.ArtifactsDir()) } + cleanupFunc := func() { + if err := c.StopGrafana(ctx, t.L(), t.ArtifactsDir()); err != nil { + t.L().ErrorfCtx(ctx, "Error(s) shutting down prom/grafana %s", err) + } + } + return cfg, cleanupFunc } diff --git a/pkg/gen/gomock.bzl b/pkg/gen/gomock.bzl index d122f64dc461..ccba0dacb12e 100644 --- a/pkg/gen/gomock.bzl +++ b/pkg/gen/gomock.bzl @@ -6,7 +6,6 @@ GOMOCK_SRCS = [ "//pkg/kv/kvclient/rangecache/rangecachemock:mocks_generated.go", "//pkg/kv/kvclient/rangefeed:mocks_generated_test.go", "//pkg/roachpb/roachpbmock:mocks_generated.go", - "//pkg/roachprod/prometheus:mocks_generated_test.go", "//pkg/security/certmgr:mocks_generated_test.go", "//pkg/sql/schemachanger/scexec:mocks_generated_test.go", "//pkg/util/log:mocks_generated_test.go", diff --git a/pkg/roachprod/BUILD.bazel b/pkg/roachprod/BUILD.bazel index 853ac9f5377e..b8df788e7ea1 100644 --- a/pkg/roachprod/BUILD.bazel +++ b/pkg/roachprod/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/roachprod/config", "//pkg/roachprod/install", "//pkg/roachprod/logger", + "//pkg/roachprod/prometheus", "//pkg/roachprod/vm", "//pkg/roachprod/vm/aws", "//pkg/roachprod/vm/azure", diff --git a/pkg/roachprod/install/BUILD.bazel b/pkg/roachprod/install/BUILD.bazel index 1a1becdce9b6..ad25e3ca9d5c 100644 --- a/pkg/roachprod/install/BUILD.bazel +++ b/pkg/roachprod/install/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "//pkg/util", "//pkg/util/httputil", "//pkg/util/log", + "//pkg/util/retry", "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/version", diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index ae6493795cc9..ae75398735ab 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -41,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachprod/vm/aws" "github.com/cockroachdb/cockroach/pkg/roachprod/vm/local" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/version" @@ -764,6 +765,35 @@ func (c *SyncedCluster) RunWithDetails( return results, nil } +var roachprodRetryOptions = retry.Options{ + InitialBackoff: 10 * time.Second, + Multiplier: 2, + MaxBackoff: 5 * time.Minute, + MaxRetries: 10, +} + +// RepeatRun is the same function as c.Run, but with an automatic retry loop. +func (c *SyncedCluster) RepeatRun( + ctx context.Context, l *logger.Logger, stdout, stderr io.Writer, nodes Nodes, title, + cmd string, +) error { + var lastError error + for attempt, r := 0, retry.StartWithCtx(ctx, roachprodRetryOptions); r.Next(); { + if ctx.Err() != nil { + return ctx.Err() + } + attempt++ + l.Printf("attempt %d - %s", attempt, title) + lastError = c.Run(ctx, l, stdout, stderr, nodes, title, cmd) + if lastError != nil { + l.Printf("error - retrying: %s", lastError) + continue + } + return nil + } + return errors.Wrapf(lastError, "all attempts failed for %s", title) +} + // Wait TODO(peter): document func (c *SyncedCluster) Wait(ctx context.Context, l *logger.Logger) error { display := fmt.Sprintf("%s: waiting for nodes to start", c.Name) @@ -1378,8 +1408,37 @@ func formatProgress(p float64) string { return fmt.Sprintf("[%s%s] %.0f%%", progressDone[i:], progressTodo[:i], 100*p) } +// PutString into the specified file on the specified remote node(s). +func (c *SyncedCluster) PutString( + ctx context.Context, l *logger.Logger, nodes Nodes, content string, dest string, mode os.FileMode, +) error { + if ctx.Err() != nil { + return errors.Wrap(ctx.Err(), "syncedCluster.PutString") + } + + temp, err := ioutil.TempFile("", filepath.Base(dest)) + if err != nil { + return errors.Wrap(err, "cluster.PutString") + } + if _, err := temp.WriteString(content); err != nil { + return errors.Wrap(err, "cluster.PutString") + } + temp.Close() + src := temp.Name() + + if err := os.Chmod(src, mode); err != nil { + return errors.Wrap(err, "cluster.PutString") + } + // NB: we intentionally don't remove the temp files. This is because roachprod + // will symlink them when running locally. + + return errors.Wrap(c.Put(ctx, l, nodes, src, dest), "syncedCluster.PutString") +} + // Put TODO(peter): document -func (c *SyncedCluster) Put(ctx context.Context, l *logger.Logger, src, dest string) error { +func (c *SyncedCluster) Put( + ctx context.Context, l *logger.Logger, nodes Nodes, src string, dest string, +) error { // Check if source file exists and if it's a symlink. var potentialSymlinkPath string var err error @@ -1410,24 +1469,24 @@ func (c *SyncedCluster) Put(ctx context.Context, l *logger.Logger, src, dest str detail = " (scp)" } } - l.Printf("%s: putting%s %s %s\n", c.Name, detail, src, dest) + l.Printf("%s: putting%s %s %s on nodes %v\n", c.Name, detail, src, dest, nodes) type result struct { index int err error } - results := make(chan result, len(c.Nodes)) - lines := make([]string, len(c.Nodes)) + results := make(chan result, len(nodes)) + lines := make([]string, len(nodes)) var linesMu syncutil.Mutex var wg sync.WaitGroup - wg.Add(len(c.Nodes)) + wg.Add(len(nodes)) // Each destination for the copy needs a source to copy from. We create a // channel that has capacity for each destination. If we try to add a source // and the channel is full we can simply drop that source as we know we won't // need to use it. - sources := make(chan int, len(c.Nodes)) + sources := make(chan int, len(nodes)) pushSource := func(i int) { select { case sources <- i: @@ -1441,7 +1500,7 @@ func (c *SyncedCluster) Put(ctx context.Context, l *logger.Logger, src, dest str } else { // In non-treedist mode, add the local source N times (once for each // destination). - for range c.Nodes { + for range nodes { pushSource(-1) } } @@ -1453,16 +1512,16 @@ func (c *SyncedCluster) Put(ctx context.Context, l *logger.Logger, src, dest str // Expand the destination to allow, for example, putting directly // into {store-dir}. e := expander{ - node: c.Nodes[i], + node: nodes[i], } dest, err := e.expand(ctx, l, c, dest) if err != nil { return "", err } - return fmt.Sprintf("%s@%s:%s", c.user(c.Nodes[i]), c.Host(c.Nodes[i]), dest), nil + return fmt.Sprintf("%s@%s:%s", c.user(nodes[i]), c.Host(nodes[i]), dest), nil } - for i := range c.Nodes { + for i := range nodes { go func(i int, dest string) { defer wg.Done() @@ -1470,7 +1529,7 @@ func (c *SyncedCluster) Put(ctx context.Context, l *logger.Logger, src, dest str // Expand the destination to allow, for example, putting directly // into {store-dir}. e := expander{ - node: c.Nodes[i], + node: nodes[i], } var err error dest, err = e.expand(ctx, l, c, dest) @@ -1495,7 +1554,7 @@ func (c *SyncedCluster) Put(ctx context.Context, l *logger.Logger, src, dest str if filepath.IsAbs(dest) { to = dest } else { - to = filepath.Join(c.localVMDir(c.Nodes[i]), dest) + to = filepath.Join(c.localVMDir(nodes[i]), dest) } // Remove the destination if it exists, ignoring errors which we'll // handle via the os.Symlink() call. @@ -1593,7 +1652,7 @@ func (c *SyncedCluster) Put(ctx context.Context, l *logger.Logger, src, dest str if !config.Quiet { linesMu.Lock() for i := range lines { - fmt.Fprintf(&writer, " %2d: ", c.Nodes[i]) + fmt.Fprintf(&writer, " %2d: ", nodes[i]) if lines[i] != "" { fmt.Fprintf(&writer, "%s", lines[i]) } else { @@ -1611,7 +1670,7 @@ func (c *SyncedCluster) Put(ctx context.Context, l *logger.Logger, src, dest str l.Printf("\n") linesMu.Lock() for i := range lines { - l.Printf(" %2d: %s", c.Nodes[i], lines[i]) + l.Printf(" %2d: %s", nodes[i], lines[i]) } linesMu.Unlock() } @@ -1770,14 +1829,14 @@ func (c *SyncedCluster) Logs( } // Get TODO(peter): document -func (c *SyncedCluster) Get(l *logger.Logger, src, dest string) error { +func (c *SyncedCluster) Get(l *logger.Logger, nodes Nodes, src, dest string) error { // TODO(peter): Only get 10 nodes at a time. When a node completes, output a // line indicating that. var detail string if !c.IsLocal() { detail = " (scp)" } - l.Printf("%s: getting%s %s %s\n", c.Name, detail, src, dest) + l.Printf("%s: getting%s %s %s on nodes %v\n", c.Name, detail, src, dest, nodes) type result struct { index int @@ -1785,20 +1844,20 @@ func (c *SyncedCluster) Get(l *logger.Logger, src, dest string) error { } var writer ui.Writer - results := make(chan result, len(c.Nodes)) - lines := make([]string, len(c.Nodes)) + results := make(chan result, len(nodes)) + lines := make([]string, len(nodes)) var linesMu syncutil.Mutex var wg sync.WaitGroup - for i := range c.Nodes { + for i := range nodes { wg.Add(1) go func(i int) { defer wg.Done() src := src dest := dest - if len(c.Nodes) > 1 { - base := fmt.Sprintf("%d.%s", c.Nodes[i], filepath.Base(dest)) + if len(nodes) > 1 { + base := fmt.Sprintf("%d.%s", nodes, filepath.Base(dest)) dest = filepath.Join(filepath.Dir(dest), base) } @@ -1810,7 +1869,7 @@ func (c *SyncedCluster) Get(l *logger.Logger, src, dest string) error { if c.IsLocal() { if !filepath.IsAbs(src) { - src = filepath.Join(c.localVMDir(c.Nodes[i]), src) + src = filepath.Join(c.localVMDir(nodes[i]), src) } var copy func(src, dest string, info os.FileInfo) error @@ -1881,7 +1940,7 @@ func (c *SyncedCluster) Get(l *logger.Logger, src, dest string) error { return } - err := c.scp(fmt.Sprintf("%s@%s:%s", c.user(c.Nodes[0]), c.Host(c.Nodes[i]), src), dest) + err := c.scp(fmt.Sprintf("%s@%s:%s", c.user(nodes[0]), c.Host(nodes[i]), src), dest) if err == nil { // Make sure all created files and directories are world readable. // The CRDB process intentionally sets a 0007 umask (resulting in @@ -1948,7 +2007,7 @@ func (c *SyncedCluster) Get(l *logger.Logger, src, dest string) error { if !config.Quiet && l.File == nil { linesMu.Lock() for i := range lines { - fmt.Fprintf(&writer, " %2d: ", c.Nodes[i]) + fmt.Fprintf(&writer, " %2d: ", nodes[i]) if lines[i] != "" { fmt.Fprintf(&writer, "%s", lines[i]) } else { @@ -1966,7 +2025,7 @@ func (c *SyncedCluster) Get(l *logger.Logger, src, dest string) error { l.Printf("\n") linesMu.Lock() for i := range lines { - l.Printf(" %2d: %s", c.Nodes[i], lines[i]) + l.Printf(" %2d: %s", nodes[i], lines[i]) } linesMu.Unlock() } diff --git a/pkg/roachprod/prometheus/BUILD.bazel b/pkg/roachprod/prometheus/BUILD.bazel index a5e0894fba8f..eeb2aa37a3b9 100644 --- a/pkg/roachprod/prometheus/BUILD.bazel +++ b/pkg/roachprod/prometheus/BUILD.bazel @@ -1,4 +1,3 @@ -load("@bazel_gomock//:gomock.bzl", "gomock") load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( @@ -7,8 +6,9 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/roachprod/prometheus", visibility = ["//visibility:public"], deps = [ - "//pkg/cmd/roachtest/option", + "//pkg/roachprod/install", "//pkg/roachprod/logger", + "@com_github_cockroachdb_errors//:errors", "@com_github_prometheus_client_golang//api/prometheus/v1:prometheus", "@com_github_prometheus_common//model", "@in_gopkg_yaml_v2//:yaml_v2", @@ -17,24 +17,13 @@ go_library( go_test( name = "prometheus_test", - srcs = [ - "prometheus_test.go", - ":prometheus_mocks", # keep - ], + srcs = ["prometheus_test.go"], + data = glob(["testdata/**"]), embed = [":prometheus"], deps = [ - "//pkg/cmd/roachtest/option", - "//pkg/roachprod/logger", - "@com_github_golang_mock//gomock", + "//pkg/roachprod/install", + "//pkg/testutils", + "//pkg/testutils/echotest", "@com_github_stretchr_testify//require", ], ) - -gomock( - name = "prometheus_mocks", - out = "mocks_generated_test.go", - interfaces = ["Cluster"], - library = ":prometheus", - package = "prometheus", - visibility = ["//visibility:public"], -) diff --git a/pkg/roachprod/prometheus/mocks_generated_test.go b/pkg/roachprod/prometheus/mocks_generated_test.go deleted file mode 100644 index d09a26102009..000000000000 --- a/pkg/roachprod/prometheus/mocks_generated_test.go +++ /dev/null @@ -1,110 +0,0 @@ -// Code generated by MockGen. DO NOT EDIT. -// Source: github.com/cockroachdb/cockroach/pkg/roachprod/prometheus (interfaces: Cluster) - -// Package prometheus is a generated GoMock package. -package prometheus - -import ( - context "context" - fs "io/fs" - reflect "reflect" - - option "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" - logger "github.com/cockroachdb/cockroach/pkg/roachprod/logger" - gomock "github.com/golang/mock/gomock" -) - -// MockCluster is a mock of Cluster interface. -type MockCluster struct { - ctrl *gomock.Controller - recorder *MockClusterMockRecorder -} - -// MockClusterMockRecorder is the mock recorder for MockCluster. -type MockClusterMockRecorder struct { - mock *MockCluster -} - -// NewMockCluster creates a new mock instance. -func NewMockCluster(ctrl *gomock.Controller) *MockCluster { - mock := &MockCluster{ctrl: ctrl} - mock.recorder = &MockClusterMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use. -func (m *MockCluster) EXPECT() *MockClusterMockRecorder { - return m.recorder -} - -// ExternalIP mocks base method. -func (m *MockCluster) ExternalIP(arg0 context.Context, arg1 *logger.Logger, arg2 option.NodeListOption) ([]string, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ExternalIP", arg0, arg1, arg2) - ret0, _ := ret[0].([]string) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ExternalIP indicates an expected call of ExternalIP. -func (mr *MockClusterMockRecorder) ExternalIP(arg0, arg1, arg2 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ExternalIP", reflect.TypeOf((*MockCluster)(nil).ExternalIP), arg0, arg1, arg2) -} - -// Get mocks base method. -func (m *MockCluster) Get(arg0 context.Context, arg1 *logger.Logger, arg2, arg3 string, arg4 ...option.Option) error { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1, arg2, arg3} - for _, a := range arg4 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "Get", varargs...) - ret0, _ := ret[0].(error) - return ret0 -} - -// Get indicates an expected call of Get. -func (mr *MockClusterMockRecorder) Get(arg0, arg1, arg2, arg3 interface{}, arg4 ...interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - varargs := append([]interface{}{arg0, arg1, arg2, arg3}, arg4...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Get", reflect.TypeOf((*MockCluster)(nil).Get), varargs...) -} - -// PutString mocks base method. -func (m *MockCluster) PutString(arg0 context.Context, arg1, arg2 string, arg3 fs.FileMode, arg4 ...option.Option) error { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1, arg2, arg3} - for _, a := range arg4 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "PutString", varargs...) - ret0, _ := ret[0].(error) - return ret0 -} - -// PutString indicates an expected call of PutString. -func (mr *MockClusterMockRecorder) PutString(arg0, arg1, arg2, arg3 interface{}, arg4 ...interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - varargs := append([]interface{}{arg0, arg1, arg2, arg3}, arg4...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PutString", reflect.TypeOf((*MockCluster)(nil).PutString), varargs...) -} - -// RunE mocks base method. -func (m *MockCluster) RunE(arg0 context.Context, arg1 option.NodeListOption, arg2 ...string) error { - m.ctrl.T.Helper() - varargs := []interface{}{arg0, arg1} - for _, a := range arg2 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "RunE", varargs...) - ret0, _ := ret[0].(error) - return ret0 -} - -// RunE indicates an expected call of RunE. -func (mr *MockClusterMockRecorder) RunE(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - varargs := append([]interface{}{arg0, arg1}, arg2...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RunE", reflect.TypeOf((*MockCluster)(nil).RunE), varargs...) -} diff --git a/pkg/roachprod/prometheus/prometheus.go b/pkg/roachprod/prometheus/prometheus.go index b74f7635585c..fc040e88835a 100644 --- a/pkg/roachprod/prometheus/prometheus.go +++ b/pkg/roachprod/prometheus/prometheus.go @@ -8,8 +8,6 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -//go:generate mockgen -package=prometheus -destination=mocks_generated_test.go . Cluster - package prometheus import ( @@ -20,25 +18,29 @@ import ( "strconv" "time" - "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" + "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/errors" promv1 "github.com/prometheus/client_golang/api/prometheus/v1" "github.com/prometheus/common/model" "gopkg.in/yaml.v2" ) +const defaultWorkloadPort = 2112 + // Client is an interface allowing queries against Prometheus. type Client interface { Query(ctx context.Context, query string, ts time.Time) (model.Value, promv1.Warnings, error) } -// ScrapeNode are nodes to scrape from. +// ScrapeNode is a node to scrape from. type ScrapeNode struct { - Nodes option.NodeListOption - Port int + Node install.Node + Port int } -// ScrapeConfig represents a single instance of scraping. +// ScrapeConfig represents a single instance of scraping, identified by the jobName +// Note how workload scrapes are set up differently than CRDB binary scrapes. type ScrapeConfig struct { JobName string MetricsPath string @@ -48,10 +50,18 @@ type ScrapeConfig struct { // Config is a monitor that watches over the running of prometheus. type Config struct { - PrometheusNode option.NodeListOption - ScrapeConfigs []ScrapeConfig - NodeExporter option.NodeListOption - Grafana GrafanaConfig + // PrometheusNode identifies a single node in the cluster to run the prometheus instance on. + // The type is install.Nodes merely for ease of use. + PrometheusNode install.Nodes + + // ScrapeConfigs provides the configurations for each scraping instance + ScrapeConfigs []ScrapeConfig + + // NodeExporter identifies each node in the cluster to scrape with the node exporter process + NodeExporter install.Nodes + + // Grafana provides the info to set up grafana + Grafana GrafanaConfig } // GrafanaConfig are options related to setting up a Grafana instance. @@ -67,30 +77,54 @@ type GrafanaConfig struct { DashboardURLs []string } -// WithWorkload sets up scraping for `workload` processes running on the given -// node(s) and port. Chains for convenience. -func (cfg *Config) WithWorkload(nodes option.NodeListOption, port int) *Config { - sn := ScrapeNode{Nodes: nodes, Port: port} +// WithWorkload sets up a scraping config for a single `workload` running on the +// given node and port. If the workload is in the config, the node and port will be +// added to the workload's scrape config (i.e. allows for chaining). If port == 0, +//defaultWorkloadPort is used. +func (cfg *Config) WithWorkload(workloadName string, nodes install.Node, port int) *Config { + + // Find the workload's scrapeConfig, if it exists. + var sc *ScrapeConfig for i := range cfg.ScrapeConfigs { - sc := &cfg.ScrapeConfigs[i] - if sc.JobName == "workload" { - sc.ScrapeNodes = append(sc.ScrapeNodes, sn) - return cfg + existing := &cfg.ScrapeConfigs[i] + // A workload scrape config name is unique. + if existing.JobName == workloadName { + sc = existing + break } } - cfg.ScrapeConfigs = append(cfg.ScrapeConfigs, MakeWorkloadScrapeConfig("workload", []ScrapeNode{sn})) - return cfg + if port == 0 { + port = defaultWorkloadPort + } + sn := ScrapeNode{Node: nodes, Port: port} + if sc == nil { + cfg.ScrapeConfigs = append(cfg.ScrapeConfigs, MakeWorkloadScrapeConfig(workloadName, "/", []ScrapeNode{sn})) + } else { + sc.ScrapeNodes = append(sc.ScrapeNodes, sn) + } + return nil +} + +// MakeWorkloadScrapeConfig creates a scrape config for a workload. +func MakeWorkloadScrapeConfig( + jobName string, metricsPath string, scrapeNodes []ScrapeNode, +) ScrapeConfig { + return ScrapeConfig{ + JobName: jobName, + MetricsPath: metricsPath, + ScrapeNodes: scrapeNodes, + } } // WithPrometheusNode specifies the node to set up prometheus on. -func (cfg *Config) WithPrometheusNode(node option.NodeListOption) *Config { - cfg.PrometheusNode = node +func (cfg *Config) WithPrometheusNode(node install.Node) *Config { + cfg.PrometheusNode = install.Nodes{node} return cfg } // WithCluster adds scraping for a CockroachDB cluster running on the given nodes. // Chains for convenience. -func (cfg *Config) WithCluster(nodes option.NodeListOption) *Config { +func (cfg *Config) WithCluster(nodes install.Nodes) *Config { cfg.ScrapeConfigs = append(cfg.ScrapeConfigs, MakeInsecureCockroachScrapeConfig(nodes)...) return cfg } @@ -105,23 +139,46 @@ func (cfg *Config) WithGrafanaDashboard(url string) *Config { return cfg } -// WithNodeExporter causes node_exporter to be set up on the specified machines. -// Chains for convenience. -func (cfg *Config) WithNodeExporter(nodes option.NodeListOption) *Config { - cfg.NodeExporter = cfg.NodeExporter.Merge(nodes) +// WithNodeExporter causes node_exporter to be set up on the specified machines, +// a separate process that sends hardware metrics to prometheus. +// For more on the node exporter process, see https://prometheus.io/docs/guides/node-exporter/ +func (cfg *Config) WithNodeExporter(nodes install.Nodes) *Config { + cfg.NodeExporter = nodes + // Add a scrape config for each node running node_exporter + for _, node := range cfg.NodeExporter { + s := strconv.Itoa(int(node)) + cfg.ScrapeConfigs = append(cfg.ScrapeConfigs, ScrapeConfig{ + JobName: "node_exporter-" + s, + MetricsPath: "/metrics", + Labels: map[string]string{"node": s}, + ScrapeNodes: []ScrapeNode{{ + Node: node, + Port: 9100}}, + }) + } return cfg } -// Cluster is a subset of roachtest.Cluster. -// It is abstracted to prevent a circular dependency on roachtest, as Cluster -// requires the test interface. -type Cluster interface { - ExternalIP(context.Context, *logger.Logger, option.NodeListOption) ([]string, error) - Get(ctx context.Context, l *logger.Logger, src, dest string, opts ...option.Option) error - RunE(ctx context.Context, node option.NodeListOption, args ...string) error - PutString( - ctx context.Context, content, dest string, mode os.FileMode, opts ...option.Option, - ) error +// MakeInsecureCockroachScrapeConfig creates a scrape config for each +// cockroach node. All nodes are assumed to be insecure and running on +// port 26258. +func MakeInsecureCockroachScrapeConfig(nodes install.Nodes) []ScrapeConfig { + var sl []ScrapeConfig + for _, node := range nodes { + s := strconv.Itoa(int(node)) + sl = append(sl, ScrapeConfig{ + JobName: "cockroach-n" + s, + MetricsPath: "/_status/vars", + Labels: map[string]string{"node": s}, + ScrapeNodes: []ScrapeNode{ + { + Node: node, + Port: 26258, + }, + }, + }) + } + return sl } // Prometheus contains metadata of a running instance of prometheus. @@ -131,120 +188,126 @@ type Prometheus struct { // Init creates a prometheus instance on the given cluster. func Init( - ctx context.Context, - cfg Config, - c Cluster, - l *logger.Logger, - repeatFunc func(context.Context, option.NodeListOption, string, ...string) error, -) (_ *Prometheus, saveSnap func(artifactsDir string), _ error) { + ctx context.Context, l *logger.Logger, c *install.SyncedCluster, cfg Config, +) (_ *Prometheus, _ error) { if len(cfg.NodeExporter) > 0 { - if err := repeatFunc(ctx, cfg.NodeExporter, "download node exporter", + if err := c.RepeatRun(ctx, l, os.Stdout, os.Stderr, cfg.NodeExporter, + "download node exporter", ` (sudo systemctl stop node_exporter || true) && rm -rf node_exporter && mkdir -p node_exporter && curl -fsSL \ https://github.com/prometheus/node_exporter/releases/download/v1.3.1/node_exporter-1.3.1.linux-amd64.tar.gz | tar zxv --strip-components 1 -C node_exporter `); err != nil { - return nil, nil, err + return nil, err } // Start node_exporter. - if err := c.RunE(ctx, cfg.NodeExporter, `cd node_exporter && + if err := c.Run(ctx, l, os.Stdout, os.Stderr, cfg.NodeExporter, "init node exporter", + `cd node_exporter && sudo systemd-run --unit node_exporter --same-dir ./node_exporter`, ); err != nil { - return nil, nil, err - } - for _, node := range cfg.NodeExporter { - s := strconv.Itoa(node) - cfg.ScrapeConfigs = append(cfg.ScrapeConfigs, ScrapeConfig{ - JobName: "node_exporter-" + s, - MetricsPath: "/metrics", - Labels: map[string]string{"node": s}, - ScrapeNodes: []ScrapeNode{{Nodes: []int{node}, Port: 9100}}, - }) + // TODO(msbutler): download binary for target platform. currently we + // hardcode downloading the linux binary. + return nil, errors.Wrap(err, "grafana-start currently cannot run on darwin") } } - - if err := repeatFunc( + if err := c.RepeatRun( ctx, + l, + os.Stdout, + os.Stderr, cfg.PrometheusNode, "reset prometheus", "sudo systemctl stop prometheus || echo 'no prometheus is running'", ); err != nil { - return nil, nil, err + return nil, err } - if err := repeatFunc( + if err := c.RepeatRun( ctx, + l, + os.Stdout, + os.Stderr, cfg.PrometheusNode, "download prometheus", `sudo rm -rf /tmp/prometheus && mkdir /tmp/prometheus && cd /tmp/prometheus && curl -fsSL https://storage.googleapis.com/cockroach-fixtures/prometheus/prometheus-2.27.1.linux-amd64.tar.gz | tar zxv --strip-components=1`, ); err != nil { - return nil, nil, err + return nil, err } - - yamlCfg, err := makeYAMLConfig( - ctx, - l, - c, - cfg.ScrapeConfigs, - ) + // create and upload prom config + nodeIPs, err := makeNodeIPMap(c) + if err != nil { + return nil, err + } + yamlCfg, err := makeYAMLConfig(cfg.ScrapeConfigs, nodeIPs) if err != nil { - return nil, nil, err + return nil, err } if err := c.PutString( ctx, + l, + cfg.PrometheusNode, yamlCfg, "/tmp/prometheus/prometheus.yml", 0644, - cfg.PrometheusNode, ); err != nil { - return nil, nil, err + return nil, err } // Start prometheus as systemd. - if err := c.RunE( + if err := c.Run( ctx, + l, + os.Stdout, + os.Stderr, cfg.PrometheusNode, + "start-prometheus", `cd /tmp/prometheus && sudo systemd-run --unit prometheus --same-dir \ ./prometheus --config.file=prometheus.yml --storage.tsdb.path=data/ --web.enable-admin-api`, ); err != nil { - return nil, nil, err + return nil, err } if cfg.Grafana.Enabled { // Install Grafana. - if err := repeatFunc(ctx, cfg.PrometheusNode, "install grafana", + if err := c.RepeatRun(ctx, l, + os.Stdout, + os.Stderr, cfg.PrometheusNode, "install grafana", `sudo apt-get install -qqy apt-transport-https && sudo apt-get install -qqy software-properties-common wget && wget -q -O - https://packages.grafana.com/gpg.key | sudo apt-key add - && echo "deb https://packages.grafana.com/enterprise/deb stable main" | sudo tee -a /etc/apt/sources.list.d/grafana.list && sudo apt-get update -qqy && sudo apt-get install -qqy grafana-enterprise && sudo mkdir -p /var/lib/grafana/dashboards`, ); err != nil { - return nil, nil, err + return nil, err } // Provision local prometheus instance as data source. - if err := repeatFunc(ctx, cfg.PrometheusNode, "permissions", + if err := c.RepeatRun(ctx, l, + os.Stdout, + os.Stderr, cfg.PrometheusNode, "permissions", `sudo chmod 777 /etc/grafana/provisioning/datasources /etc/grafana/provisioning/dashboards /var/lib/grafana/dashboards`, ); err != nil { - return nil, nil, err + return nil, err } - if err := c.PutString(ctx, `apiVersion: 1 + + // Set up grafana config + if err := c.PutString(ctx, l, cfg.PrometheusNode, `apiVersion: 1 datasources: - name: prometheusdata type: prometheus uid: localprom url: http://localhost:9090 -`, "/etc/grafana/provisioning/datasources/prometheus.yaml", 0644, cfg.PrometheusNode); err != nil { - return nil, nil, err +`, "/etc/grafana/provisioning/datasources/prometheus.yaml", 0644); err != nil { + return nil, err } - if err := c.PutString(ctx, `apiVersion: 1 + if err := c.PutString(ctx, l, cfg.PrometheusNode, `apiVersion: 1 providers: - name: 'default' @@ -254,40 +317,44 @@ providers: type: file options: path: /var/lib/grafana/dashboards -`, "/etc/grafana/provisioning/dashboards/cockroach.yaml", 0644, cfg.PrometheusNode); err != nil { - return nil, nil, err +`, "/etc/grafana/provisioning/dashboards/cockroach.yaml", 0644); err != nil { + return nil, err } - for idx, u := range cfg.Grafana.DashboardURLs { - if err := c.RunE(ctx, cfg.PrometheusNode, - "curl", "-fsSL", u, "-o", fmt.Sprintf("/var/lib/grafana/dashboards/%d.json", idx), - ); err != nil { + cmd := fmt.Sprintf("curl -fsSL %s -o /var/lib/grafana/dashboards/%d.json", u, idx) + if err := c.Run(ctx, l, os.Stdout, os.Stderr, cfg.PrometheusNode, "download dashboard", + cmd); err != nil { l.PrintfCtx(ctx, "failed to download dashboard from %s: %s", u, err) } } // Start Grafana. Default port is 3000. - if err := c.RunE(ctx, cfg.PrometheusNode, `sudo systemctl restart grafana-server`); err != nil { - return nil, nil, err + if err := c.Run(ctx, l, os.Stdout, os.Stderr, cfg.PrometheusNode, "start grafana", + `sudo systemctl restart grafana-server`); err != nil { + return nil, err } } p := &Prometheus{Config: cfg} - return p, func(destDir string) { - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) - defer cancel() - if err := p.Snapshot(ctx, c, l, destDir); err != nil { - l.Printf("failed to get prometheus snapshot: %v", err) - } - }, nil + return p, nil } // Snapshot takes a snapshot of prometheus and stores the snapshot and a script to spin up // a docker instance for it to the given directory. -func (pm *Prometheus) Snapshot(ctx context.Context, c Cluster, l *logger.Logger, dir string) error { - if err := c.RunE( +func Snapshot( + ctx context.Context, + c *install.SyncedCluster, + l *logger.Logger, + promNode install.Nodes, + dir string, +) error { + if err := c.Run( ctx, - pm.PrometheusNode, + l, + os.Stdout, + os.Stderr, + promNode, + "prometheus snapshot", `curl -XPOST http://localhost:9090/api/v1/admin/tsdb/snapshot && cd /tmp/prometheus && tar cvf prometheus-snapshot.tar.gz data/snapshots`, ); err != nil { @@ -316,18 +383,71 @@ docker run --privileged -p 9090:9090 \ --storage.tsdb.path=/prometheus \ --web.enable-admin-api `), 0755); err != nil { - return err + return errors.Wrap(err, "failed to write docker script") } return c.Get( - ctx, l, + promNode, "/tmp/prometheus/prometheus-snapshot.tar.gz", dir, - pm.PrometheusNode, ) } +// Shutdown stops all prom and grafana processes and, if dumpDir is passed, +// will download dump of prometheus data to the machine executing the roachprod binary. +func Shutdown( + ctx context.Context, + c *install.SyncedCluster, + l *logger.Logger, + nodes install.Nodes, + dumpDir string, +) error { + // We currently assume the last node contains the server. + promNode := install.Nodes{nodes[len(nodes)-1]} + + dumpSnapshot := func(dumpDir string) error { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + if err := Snapshot(ctx, c, l, promNode, dumpDir); err != nil { + l.Printf("failed to get prometheus snapshot: %v", err) + return err + } + return nil + } + var shutdownErr error + if dumpDir != "" { + if err := dumpSnapshot(dumpDir); err != nil { + shutdownErr = errors.CombineErrors(shutdownErr, err) + } + } + if err := c.Run(ctx, l, os.Stdout, os.Stderr, nodes, "stop node exporter", + `sudo systemctl stop node_exporter || echo 'Stopped node exporter'`); err != nil { + l.Printf("Failed to stop node exporter: %v", err) + shutdownErr = errors.CombineErrors(shutdownErr, err) + } + + if err := c.Run(ctx, l, os.Stdout, os.Stderr, promNode, "stop grafana", + `sudo systemctl stop grafana-server || echo 'Stopped grafana'`); err != nil { + l.Printf("Failed to stop grafana server: %v", err) + shutdownErr = errors.CombineErrors(shutdownErr, err) + } + + if err := c.RepeatRun( + ctx, + l, + os.Stdout, + os.Stderr, + promNode, + "stop prometheus", + "sudo systemctl stop prometheus || echo 'Stopped prometheus'", + ); err != nil { + l.Printf("Failed to stop prometheus server: %v", err) + shutdownErr = errors.CombineErrors(shutdownErr, err) + } + return shutdownErr +} + const ( // DefaultScrapeInterval is the default interval between prometheus // scrapes. @@ -337,10 +457,20 @@ const ( DefaultScrapeTimeout = 5 * time.Second ) +func makeNodeIPMap(c *install.SyncedCluster) (map[install.Node]string, error) { + nodes, err := install.ListNodes("all", len(c.VMs)) + if err != nil { + return nil, err + } + nodeIP := make(map[install.Node]string) + for i, n := range nodes { + nodeIP[n] = c.VMs[nodes[i]-1].PublicIP + } + return nodeIP, nil +} + // makeYAMLConfig creates a prometheus YAML config for the server to use. -func makeYAMLConfig( - ctx context.Context, l *logger.Logger, c Cluster, scrapeConfigs []ScrapeConfig, -) (string, error) { +func makeYAMLConfig(scrapeConfigs []ScrapeConfig, nodeIPs map[install.Node]string) (string, error) { type yamlStaticConfig struct { Labels map[string]string `yaml:",omitempty"` Targets []string @@ -367,13 +497,7 @@ func makeYAMLConfig( for _, scrapeConfig := range scrapeConfigs { var targets []string for _, scrapeNode := range scrapeConfig.ScrapeNodes { - ips, err := c.ExternalIP(ctx, l, scrapeNode.Nodes) - if err != nil { - return "", err - } - for _, ip := range ips { - targets = append(targets, fmt.Sprintf("%s:%d", ip, scrapeNode.Port)) - } + targets = append(targets, fmt.Sprintf("%s:%d", nodeIPs[scrapeNode.Node], scrapeNode.Port)) } cfg.ScrapeConfigs = append( @@ -390,39 +514,6 @@ func makeYAMLConfig( }, ) } - ret, err := yaml.Marshal(&cfg) return string(ret), err } - -// MakeWorkloadScrapeConfig creates a scrape config for a workload. -func MakeWorkloadScrapeConfig(jobName string, scrapeNodes []ScrapeNode) ScrapeConfig { - return ScrapeConfig{ - JobName: jobName, - MetricsPath: "/", - ScrapeNodes: scrapeNodes, - } -} - -// MakeInsecureCockroachScrapeConfig creates scrape configs for the given -// cockroach nodes. All nodes are assumed to be insecure and running on -// port 26258. -func MakeInsecureCockroachScrapeConfig(nodes option.NodeListOption) []ScrapeConfig { - var sl []ScrapeConfig - for _, node := range nodes { - s := strconv.Itoa(node) - sl = append(sl, ScrapeConfig{ - JobName: "cockroach-n" + s, - MetricsPath: "/_status/vars", - Labels: map[string]string{"node": s}, - ScrapeNodes: []ScrapeNode{ - { - Nodes: []int{node}, - Port: 26258, - }, - }, - }) - } - - return sl -} diff --git a/pkg/roachprod/prometheus/prometheus_test.go b/pkg/roachprod/prometheus/prometheus_test.go index 8beff15a9b04..67d49e1dc6dc 100644 --- a/pkg/roachprod/prometheus/prometheus_test.go +++ b/pkg/roachprod/prometheus/prometheus_test.go @@ -11,178 +11,125 @@ package prometheus import ( - "context" - "io/ioutil" + "fmt" "testing" - "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" - logger "github.com/cockroachdb/cockroach/pkg/roachprod/logger" - "github.com/golang/mock/gomock" + "github.com/cockroachdb/cockroach/pkg/roachprod/install" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/stretchr/testify/require" ) -func nilLogger() *logger.Logger { - lcfg := logger.Config{ - Stdout: ioutil.Discard, - Stderr: ioutil.Discard, - } - l, err := lcfg.NewLogger("" /* path */) - if err != nil { - panic(err) - } - return l +var nodeIPMap = map[install.Node]string{ + install.Node(1): "127.0.0.1", + install.Node(2): "127.0.0.2", + install.Node(3): "127.0.0.3", + install.Node(4): "127.0.0.4", + install.Node(5): "127.0.0.5", + install.Node(6): "127.0.0.6", + install.Node(7): "127.0.0.7", + install.Node(8): "127.0.0.8", + install.Node(9): "127.0.0.9", } func TestMakeYAMLConfig(t *testing.T) { - ctx := context.Background() testCases := []struct { - desc string - - mockCluster func(ctrl *gomock.Controller) Cluster - scrapeConfigs []ScrapeConfig - - expected string + testfile string + useWorkloadHelpers bool + cluster install.Nodes + workloadScrapeConfigs []ScrapeConfig }{ { - desc: "multiple scrape nodes", - mockCluster: func(ctrl *gomock.Controller) Cluster { - c := NewMockCluster(ctrl) - c.EXPECT(). - ExternalIP(ctx, nilLogger(), []int{1}). - Return([]string{"127.0.0.1"}, nil) - c.EXPECT(). - ExternalIP(ctx, nilLogger(), []int{3, 4, 5}). - Return([]string{"127.0.0.3", "127.0.0.4", "127.0.0.5"}, nil) - c.EXPECT(). - ExternalIP(ctx, nilLogger(), []int{6}). - Return([]string{"127.0.0.6"}, nil) - return c - }, - scrapeConfigs: []ScrapeConfig{ + testfile: "multipleScrapeNodes.txt", + useWorkloadHelpers: false, + workloadScrapeConfigs: []ScrapeConfig{ { - JobName: "workload1", + JobName: "workload0", MetricsPath: "/b", ScrapeNodes: []ScrapeNode{ { - Nodes: option.NodeListOption([]int{1}), - Port: 2002, + Node: install.Node(1), + Port: 2002, }, { - Nodes: option.NodeListOption([]int{3, 4, 5}), - Port: 2003, + Node: install.Node(3), + Port: 2003, + }, + { + Node: install.Node(4), + Port: 2003, + }, + { + Node: install.Node(5), + Port: 2003, }, }, }, { - JobName: "workload2", + JobName: "workload1", MetricsPath: "/c", ScrapeNodes: []ScrapeNode{ { - Nodes: option.NodeListOption([]int{6}), - Port: 2009, + Node: install.Node(6), + Port: 2009, }, }, }, }, - expected: `global: - scrape_interval: 10s - scrape_timeout: 5s -scrape_configs: -- job_name: workload1 - static_configs: - - targets: - - 127.0.0.1:2002 - - 127.0.0.3:2003 - - 127.0.0.4:2003 - - 127.0.0.5:2003 - metrics_path: /b -- job_name: workload2 - static_configs: - - targets: - - 127.0.0.6:2009 - metrics_path: /c -`, }, { - desc: "using make commands", - mockCluster: func(ctrl *gomock.Controller) Cluster { - c := NewMockCluster(ctrl) - c.EXPECT(). - ExternalIP(ctx, nilLogger(), []int{3, 4, 5}). - Return([]string{"127.0.0.3", "127.0.0.4", "127.0.0.5"}, nil) - c.EXPECT(). - ExternalIP(ctx, nilLogger(), []int{6}). - Return([]string{"127.0.0.6"}, nil) - c.EXPECT(). - ExternalIP(ctx, nilLogger(), []int{8}). - Return([]string{"127.0.0.8"}, nil) - c.EXPECT(). - ExternalIP(ctx, nilLogger(), []int{9}). - Return([]string{"127.0.0.9"}, nil) - return c - }, - scrapeConfigs: func() (sc []ScrapeConfig) { - sc = append(sc, MakeWorkloadScrapeConfig( - "workload", - []ScrapeNode{ + testfile: "usingMakeCommands.txt", + useWorkloadHelpers: true, + cluster: install.Nodes{8, 9}, + workloadScrapeConfigs: []ScrapeConfig{ + { + ScrapeNodes: []ScrapeNode{ + { + Node: install.Node(3), + Port: 2005, + }, { - Nodes: option.NodeListOption([]int{3, 4, 5}), - Port: 2005, + Node: install.Node(4), + Port: 2005, }, { - Nodes: option.NodeListOption([]int{6}), - Port: 2009, + Node: install.Node(5), + Port: 2005, + }, + { + Node: install.Node(6), + Port: 2009, }, }, - )) - sc = append(sc, MakeInsecureCockroachScrapeConfig( - option.NodeListOption([]int{8, 9}), - )...) - return sc - }(), - expected: `global: - scrape_interval: 10s - scrape_timeout: 5s -scrape_configs: -- job_name: workload - static_configs: - - targets: - - 127.0.0.3:2005 - - 127.0.0.4:2005 - - 127.0.0.5:2005 - - 127.0.0.6:2009 - metrics_path: / -- job_name: cockroach-n8 - static_configs: - - labels: - node: "8" - targets: - - 127.0.0.8:26258 - metrics_path: /_status/vars -- job_name: cockroach-n9 - static_configs: - - labels: - node: "9" - targets: - - 127.0.0.9:26258 - metrics_path: /_status/vars -`, + }, + }, }, } for _, tc := range testCases { - t.Run(tc.desc, func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() + t.Run(tc.testfile, func(t *testing.T) { + var promCfg Config + for i, workloadConfig := range tc.workloadScrapeConfigs { + if tc.useWorkloadHelpers { + for _, scrapeNode := range workloadConfig.ScrapeNodes { + // test appending to same workload + promCfg.WithWorkload( + "workload"+fmt.Sprint(i), + scrapeNode.Node, + scrapeNode.Port) + } + + } else { + promCfg.ScrapeConfigs = append(promCfg.ScrapeConfigs, workloadConfig) + } - cfg, err := makeYAMLConfig( - ctx, - nilLogger(), - tc.mockCluster(ctrl), - tc.scrapeConfigs, - ) + } + if tc.cluster != nil { + promCfg.WithCluster(tc.cluster) + } + cfg, err := makeYAMLConfig(promCfg.ScrapeConfigs, nodeIPMap) require.NoError(t, err) - require.Equal(t, tc.expected, cfg) + echotest.Require(t, cfg, testutils.TestDataPath(t, tc.testfile)) }) } } diff --git a/pkg/roachprod/prometheus/testdata/multipleScrapeNodes.txt b/pkg/roachprod/prometheus/testdata/multipleScrapeNodes.txt new file mode 100644 index 000000000000..85fa3430a5cf --- /dev/null +++ b/pkg/roachprod/prometheus/testdata/multipleScrapeNodes.txt @@ -0,0 +1,19 @@ +echo +---- +global: + scrape_interval: 10s + scrape_timeout: 5s +scrape_configs: +- job_name: workload0 + static_configs: + - targets: + - 127.0.0.1:2002 + - 127.0.0.3:2003 + - 127.0.0.4:2003 + - 127.0.0.5:2003 + metrics_path: /b +- job_name: workload1 + static_configs: + - targets: + - 127.0.0.6:2009 + metrics_path: /c diff --git a/pkg/roachprod/prometheus/testdata/usingMakeCommands.txt b/pkg/roachprod/prometheus/testdata/usingMakeCommands.txt new file mode 100644 index 000000000000..9e34cea68699 --- /dev/null +++ b/pkg/roachprod/prometheus/testdata/usingMakeCommands.txt @@ -0,0 +1,28 @@ +echo +---- +global: + scrape_interval: 10s + scrape_timeout: 5s +scrape_configs: +- job_name: workload0 + static_configs: + - targets: + - 127.0.0.3:2005 + - 127.0.0.4:2005 + - 127.0.0.5:2005 + - 127.0.0.6:2009 + metrics_path: / +- job_name: cockroach-n8 + static_configs: + - labels: + node: "8" + targets: + - 127.0.0.8:26258 + metrics_path: /_status/vars +- job_name: cockroach-n9 + static_configs: + - labels: + node: "9" + targets: + - 127.0.0.9:26258 + metrics_path: /_status/vars diff --git a/pkg/roachprod/roachprod.go b/pkg/roachprod/roachprod.go index 90d15999bfb2..9c16e9f9111b 100644 --- a/pkg/roachprod/roachprod.go +++ b/pkg/roachprod/roachprod.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachprod/config" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/prometheus" "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/roachprod/vm/aws" "github.com/cockroachdb/cockroach/pkg/roachprod/vm/azure" @@ -819,7 +820,7 @@ func Put( if err != nil { return err } - return c.Put(ctx, l, src, dest) + return c.Put(ctx, l, c.Nodes, src, dest) } // Get copies a remote file from the nodes in a cluster. @@ -833,7 +834,7 @@ func Get(l *logger.Logger, clusterName, src, dest string) error { if err != nil { return err } - return c.Get(l, src, dest) + return c.Get(l, c.Nodes, src, dest) } // PgURL generates pgurls for the nodes in a cluster. @@ -877,43 +878,44 @@ func PgURL( return urls, nil } -// AdminURL generates admin UI URLs for the nodes in a cluster. -func AdminURL( - l *logger.Logger, clusterName, path string, usePublicIPs, openInBrowser, secure bool, -) ([]string, error) { - if err := LoadClusters(); err != nil { - return nil, err - } - c, err := newCluster(l, clusterName, install.SecureOption(secure)) - if err != nil { - return nil, err - } +type urlConfig struct { + path string + usePublicIP bool + openInBrowser bool + secure bool + port int +} +func urlGenerator( + c *install.SyncedCluster, l *logger.Logger, nodes install.Nodes, config urlConfig, +) ([]string, error) { var urls []string - for i, node := range c.TargetNodes() { + for i, node := range nodes { host := vm.Name(c.Name, int(node)) + "." + gce.Subdomain // verify DNS is working / fallback to IPs if not. - if i == 0 && !usePublicIPs { + if i == 0 && !config.usePublicIP { if _, err := net.LookupHost(host); err != nil { fmt.Fprintf(l.Stderr, "no valid DNS (yet?). might need to re-run `sync`?\n") - usePublicIPs = true + config.usePublicIP = true } } - if usePublicIPs { + if config.usePublicIP { host = c.VMs[node-1].PublicIP } - port := c.NodeUIPort(node) + if config.port == 0 { + config.port = c.NodeUIPort(node) + } scheme := "http" if c.Secure { scheme = "https" } - if !strings.HasPrefix(path, "/") { - path = "/" + path + if !strings.HasPrefix(config.path, "/") { + config.path = "/" + config.path } - url := fmt.Sprintf("%s://%s:%d%s", scheme, host, port, path) - if openInBrowser { + url := fmt.Sprintf("%s://%s:%d%s", scheme, host, config.port, config.path) + if config.openInBrowser { if err := exec.Command("python", "-m", "webbrowser", url).Run(); err != nil { return nil, err } @@ -924,6 +926,26 @@ func AdminURL( return urls, nil } +// AdminURL generates admin UI URLs for the nodes in a cluster. +func AdminURL( + l *logger.Logger, clusterName, path string, usePublicIP, openInBrowser, secure bool, +) ([]string, error) { + if err := LoadClusters(); err != nil { + return nil, err + } + c, err := newCluster(l, clusterName, install.SecureOption(secure)) + if err != nil { + return nil, err + } + uConfig := urlConfig{ + path: path, + usePublicIP: usePublicIP, + openInBrowser: openInBrowser, + secure: secure, + } + return urlGenerator(c, l, c.TargetNodes(), uConfig) +} + // PprofOpts specifies the options needed by Pprof(). type PprofOpts struct { Heap bool @@ -1334,3 +1356,103 @@ func InitProviders() map[string]string { return providersState } + +// StartGrafana spins up a prometheus and grafana instance on the last node provided and scrapes +// from all other nodes. +func StartGrafana( + ctx context.Context, + l *logger.Logger, + clusterName string, + grafanaURL string, + promCfg *prometheus.Config, // passed iff grafanaURL is empty +) error { + if grafanaURL != "" && promCfg != nil { + return errors.New("cannot pass grafanaURL and a non empty promCfg") + } + if err := LoadClusters(); err != nil { + return err + } + c, err := newCluster(l, clusterName) + if err != nil { + return err + } + nodes, err := install.ListNodes("all", len(c.VMs)) + if err != nil { + return err + } + + if promCfg == nil { + promCfg = &prometheus.Config{} + // Configure the prometheus/grafana servers to run on the last node in the cluster + promCfg.WithPrometheusNode(nodes[len(nodes)-1]) + + // Configure scraping on all nodes in the cluster + promCfg.WithCluster(nodes) + promCfg.WithNodeExporter(nodes) + + // By default, spin up a grafana server + promCfg.Grafana.Enabled = true + if grafanaURL != "" { + promCfg.WithGrafanaDashboard(grafanaURL) + } + } + _, err = prometheus.Init(ctx, l, c, *promCfg) + if err != nil { + return err + } + urls, err := GrafanaURL(ctx, l, clusterName, false) + if err != nil { + return err + } + for i, url := range urls { + fmt.Printf("Grafana dashboard %d: %s\n", i, url) + } + return nil +} + +// StopGrafana shuts down prometheus and grafana servers on the last node in +// the cluster, if they exist. +func StopGrafana(ctx context.Context, l *logger.Logger, clusterName string, dumpDir string) error { + if err := LoadClusters(); err != nil { + return err + } + c, err := newCluster(l, clusterName) + if err != nil { + return err + } + nodes, err := install.ListNodes("all", len(c.VMs)) + if err != nil { + return err + } + if err := prometheus.Shutdown(ctx, c, l, nodes, dumpDir); err != nil { + return err + } + return nil +} + +// GrafanaURL returns a url to the grafana dashboard +func GrafanaURL( + ctx context.Context, l *logger.Logger, clusterName string, openInBrowser bool, +) ([]string, error) { + if err := LoadClusters(); err != nil { + return nil, err + } + c, err := newCluster(l, clusterName) + if err != nil { + return nil, err + } + nodes, err := install.ListNodes("all", len(c.VMs)) + if err != nil { + return nil, err + } + // grafana is assumed to be running on the last node in the target + grafanaNode := install.Nodes{nodes[len(nodes)-1]} + + uConfig := urlConfig{ + usePublicIP: true, + openInBrowser: openInBrowser, + secure: false, + port: 3000, + } + return urlGenerator(c, l, grafanaNode, uConfig) +} From e41f20b4fcab5de58187f7ba6ee6c8cdcbd0e07a Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Tue, 28 Jun 2022 15:55:30 -0400 Subject: [PATCH 08/15] amazon: add custom retryer to retry on `read: connection reset` This change implements a custom retryer that we use when initializing a new s3 client for interaction with the external storage sink. This change was motivated by the increased number of backup job failures we were observing with a `read: connection reset` error being thrown by s3. A read connection reset error is thrown when the SDK is unable to read the response of an underlying API request due to a connection reset. The DefaultRetryer in the AWS SDK does not treat this error as a retryable error since the SDK does not have knowledge about the idempotence of the request, and whether it is safe to retry - https://github.com/aws/aws-sdk-go/pull/2926#issuecomment-553637658. In CRDB all operations with s3 (read, write, list) are considered idempotent, and so we can treat the read connection reset error as retryable too. Release note (bug fix): Retry s3 operations when they error out with a read connection reset error instead of failing the top level job. --- pkg/cloud/amazon/BUILD.bazel | 2 ++ pkg/cloud/amazon/s3_storage.go | 42 ++++++++++++++++++++++++++++++++++ 2 files changed, 44 insertions(+) diff --git a/pkg/cloud/amazon/BUILD.bazel b/pkg/cloud/amazon/BUILD.bazel index 13240f9a98ab..3098b179ff5a 100644 --- a/pkg/cloud/amazon/BUILD.bazel +++ b/pkg/cloud/amazon/BUILD.bazel @@ -22,8 +22,10 @@ go_library( "//pkg/util/tracing", "@com_github_aws_aws_sdk_go//aws", "@com_github_aws_aws_sdk_go//aws/awserr", + "@com_github_aws_aws_sdk_go//aws/client", "@com_github_aws_aws_sdk_go//aws/credentials", "@com_github_aws_aws_sdk_go//aws/credentials/stscreds", + "@com_github_aws_aws_sdk_go//aws/request", "@com_github_aws_aws_sdk_go//aws/session", "@com_github_aws_aws_sdk_go//service/kms", "@com_github_aws_aws_sdk_go//service/s3", diff --git a/pkg/cloud/amazon/s3_storage.go b/pkg/cloud/amazon/s3_storage.go index 4c7e1be5afa9..b6b1cc81dc0e 100644 --- a/pkg/cloud/amazon/s3_storage.go +++ b/pkg/cloud/amazon/s3_storage.go @@ -21,8 +21,10 @@ import ( "github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/aws/awserr" + "github.com/aws/aws-sdk-go/aws/client" "github.com/aws/aws-sdk-go/aws/credentials" "github.com/aws/aws-sdk-go/aws/credentials/stscreds" + "github.com/aws/aws-sdk-go/aws/request" "github.com/aws/aws-sdk-go/aws/session" "github.com/aws/aws-sdk-go/service/s3" "github.com/aws/aws-sdk-go/service/s3/s3manager" @@ -86,6 +88,39 @@ type s3Storage struct { cached *s3Client } +var _ request.Retryer = &customRetryer{} + +// customRetryer implements the `request.Retryer` interface and allows for +// customization of the retry behaviour of an AWS client. +type customRetryer struct { + client.DefaultRetryer +} + +// isErrReadConnectionReset returns true if the underlying error is a read +// connection reset error. +// +// NB: A read connection reset error is thrown when the SDK is unable to read +// the response of an underlying API request due to a connection reset. The +// DefaultRetryer in the AWS SDK does not treat this error as a retryable error +// since the SDK does not have knowledge about the idempotence of the request, +// and whether it is safe to retry - +// https://github.com/aws/aws-sdk-go/pull/2926#issuecomment-553637658. +// +// In CRDB all operations with s3 (read, write, list) are considered idempotent, +// and so we can treat the read connection reset error as retryable too. +func isErrReadConnectionReset(err error) bool { + // The error string must match the one in + // github.com/aws/aws-sdk-go/aws/request/connection_reset_error.go. This is + // unfortunate but the only solution until the SDK exposes a specialized error + // code or type for this class of errors. + return err != nil && strings.Contains(err.Error(), "read: connection reset") +} + +// ShouldRetry implements the request.Retryer interface. +func (sr *customRetryer) ShouldRetry(r *request.Request) bool { + return sr.DefaultRetryer.ShouldRetry(r) || isErrReadConnectionReset(r.Error) +} + // s3Client wraps an SDK client and uploader for a given session. type s3Client struct { client *s3.S3 @@ -359,6 +394,13 @@ func newClient( opts.Config.LogLevel = aws.LogLevel(aws.LogDebugWithRequestRetries | aws.LogDebugWithRequestErrors) } + retryer := &customRetryer{ + DefaultRetryer: client.DefaultRetryer{ + NumMaxRetries: *opts.Config.MaxRetries, + }, + } + opts.Config.Retryer = retryer + var sess *session.Session var err error From f3faff959ce96900a6bae5c61c76ca082cdb7d2f Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 28 Jun 2022 17:23:14 -0700 Subject: [PATCH 09/15] sql: ensure that the plan is closed in apply joins in some error cases Previously, it was possible for the apply join's plan to be left unclosed when an error is encountered during the physical planning of the main query, and this has now been fixed. We do so by explicitly closing the plan in such a scenario. Release note: None --- pkg/sql/distsql_running.go | 8 +- pkg/sql/distsql_running_test.go | 5 +- .../logictest/testdata/logic_test/apply_join | 105 ++++++++++++++++++ 3 files changed, 114 insertions(+), 4 deletions(-) diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index cf9f1e1bfbc1..c86dd45565ef 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -1448,7 +1448,13 @@ func (dsp *DistSQLPlanner) PlanAndRun( physPlan, physPlanCleanup, err := dsp.createPhysPlan(ctx, planCtx, plan) if err != nil { recv.SetError(err) - return physPlanCleanup + return func() { + // Make sure to close the current plan in case of a physical + // planning error. Usually, this is done in runCleanup() below, but + // we won't get to that point, so we have to do so here. + planCtx.planner.curPlan.close(ctx) + physPlanCleanup() + } } dsp.finalizePlanWithRowCount(planCtx, physPlan, planCtx.planner.curPlan.mainRowCount) recv.expectedRowsRead = int64(physPlan.TotalEstimatedScannedRows) diff --git a/pkg/sql/distsql_running_test.go b/pkg/sql/distsql_running_test.go index f05d0c3aa811..a90bf3efff03 100644 --- a/pkg/sql/distsql_running_test.go +++ b/pkg/sql/distsql_running_test.go @@ -164,13 +164,12 @@ func TestDistSQLRunningInAbortedTxn(t *testing.T) { nil, /* testingPushCallback */ ) - // We need to re-plan every time, since close() below makes - // the plan unusable across retries. + // We need to re-plan every time, since the plan is closed automatically + // by PlanAndRun() below making it unusable across retries. p.stmt = makeStatement(stmt, clusterunique.ID{}) if err := p.makeOptimizerPlan(ctx); err != nil { t.Fatal(err) } - defer p.curPlan.close(ctx) evalCtx := p.ExtendedEvalContext() // We need distribute = true so that executing the plan involves marshaling diff --git a/pkg/sql/logictest/testdata/logic_test/apply_join b/pkg/sql/logictest/testdata/logic_test/apply_join index ca58728ad510..07ad165ff4cf 100644 --- a/pkg/sql/logictest/testdata/logic_test/apply_join +++ b/pkg/sql/logictest/testdata/logic_test/apply_join @@ -473,3 +473,108 @@ VALUES (VALUES (0:::OID), (3790322641:::OID)) AS tab_54747 (col_95055) ) ); + +# Regression tests for not closing the apply join's plan when the physical +# planning for the main query in that plan fails (#82705, #83368). +query error pq: parse_time\(\): could not parse \"\\\\E\" as type time +SELECT + '1 day':::INTERVAL AS col_218767, + ( + SELECT + jsonb_object_agg( + '{"baz": null, "foo": [[1.4294938406328335], {"baz": [[]]}, {"UkCCPdz_,`": true, "wT7Dq }": {"R\"3w": 0.11797676668867385}}], "vSedin$fKu\"D": {}}':::JSONB::STRING, + tab_130100.col_218774 + )::JSONB + AS col_218775 + FROM + ( + VALUES + (parse_time(e'\\E':::STRING::STRING)::TIME), + ( + CASE + WHEN (SELECT tab_130097.col_218770 AS col_218771 FROM (VALUES (NULL)) AS tab_130097 (col_218770) LIMIT 1:::INT8) THEN '13:42:12.743575':::TIME + ELSE '19:49:43.000308':::TIME + END + ), + ('14:36:56.737547':::TIME) + ) + AS tab_130098 (col_218772) + JOIN (VALUES ('14:54:42.42701':::TIME)) AS tab_130099 (col_218773) ON (tab_130098.col_218772) = (tab_130099.col_218773) + FULL JOIN (VALUES (tab_130094.col_218765)) AS tab_130100 (col_218774) ON NULL + WHERE + NULL + GROUP BY + tab_130098.col_218772, tab_130100.col_218774 + LIMIT + 1:::INT8 + ) + AS col_218776 +FROM + ( + VALUES + ( + '21 years 10 mons 899 days 17:53:39.838878':::INTERVAL, + ( + SELECT + '{";z>gyrXH`$": {}, "X,!6@?[,H": null, "b": "\"7i?^K[JB>o", "foobar": "b"}':::JSONB AS col_218764 + FROM + (VALUES (0:::INT8)) AS tab_130093 (col_218763) + LIMIT + 1:::INT8 + ) + ), + ('-60 years -6 mons -921 days -13:36:39.76583':::INTERVAL, '[{"OD}_yC": {}, "bar": {"Zkm3=(b~": {}, "a": {}}}, null, [], {}, [], [], []]':::JSONB) + ) + AS tab_130094 (col_218765, col_218766); + +query error pq: st_mpointfromwkb\(\): error parsing EWKB: wkb: unknown byte order: 11000000 +WITH + with_111870 (col_664924) AS (SELECT * FROM (VALUES (NULL)) AS tab_397795 (col_664924)) +SELECT + cte_ref_33032.col_664924 AS col_664951 +FROM + with_111870 AS cte_ref_33032 +WHERE + EXISTS( + SELECT + 1 AS col_664950 + FROM + (VALUES (NULL)) AS tab_397798 (col_664927) + JOIN ( + VALUES + ( + ( + SELECT + '0106000000080000000103000000010000000D000000F0505233A40CC9C11BB743615DDA01C2B88C1EFC973CE64166D33387A326FCC1AC21BE6137A8E7413BCC9C277AE3FBC1B48612908643E841DC9FDD40096DFDC1308D25E7556AE541C915C046D055F4C138B48164A87BF7414AEAE615FBCFF2C196A8DC43211F02421C0F170232E8EB4192294D0B1832F84130479BA27F0FF2411893A0BCE16BE1412066858D449DE04160672B715310CAC1849FFD3CD44CFD41CFC2ADCED035F5C18C7E25996257FB418B246ED77421F4C1D09F0FAA75D3D4C1F0505233A40CC9C11BB743615DDA01C20103000000010000000600000058468ACCBC64F8418786AC4B6294FCC1080375F090CAEF41F08A5E26CF9DD441BA8573D036C5F7414C86B6AFF353FB411C22C528802BE94164C2DBD3DB41FC41583B00514C79F0C1FE97F4B9D5B0F24158468ACCBC64F8418786AC4B6294FCC1010300000001000000070000006E657E9AF457FCC1621834F5D0ECFBC1C09A82DB0FF3A04102C5AF7FC470FCC1E8C86B17FB80FF41826A06E89AE2EEC102483755C6220242D026966E856BF74198B2A8B04BE0EF4108575F84B214FB410CEBE7176F0A01C2ACC9D3FCB23FF5416E657E9AF457FCC1621834F5D0ECFBC10103000000010000000700000005349AF6354B01C2F6FA430E22CCF9C168FE825E929602C2AF949EEAE3DA01C2ABA76CF17C1FF0C1CC8327CAA112FFC140817828CF20CFC160D5A39864E1DE417400B38D4B11F7C10EA5063A797AEBC1D69DB7923410F8C12C17009A5D1CE04105349AF6354B01C2F6FA430E22CCF9C101030000000100000008000000080745DE1832F341A4E452B9669BFAC1D0AAD87FEB24F241D8D3D9BDEFF1D94192AE3AE18C2700420265B06EDD33F041C89C281D0B2AE34118D587512EF3F9418038B4E65C9AC84138169B40A643EB417A1107E45270E4C1DC71F1EA3AC3F141C1AC97127529F2C14492972AAEC8F241080745DE1832F341A4E452B9669BFAC101030000000100000009000000675BC4C8FA9A00C240AFB3C37BB0B3C1BB07A39C2042F3C162AD6982C839EFC10ED9AF069F64E4C1FDF6D190F2AFF2C1D890039C94E6FF41DC85CD27FD7E00C2EEB2ABDFA4130242EA64F205DDB10142ECA5416845C1D5C130E460FD4C5BEC414C142C48A32DF6C10E424B732C530042C4EC62A5FF47E7C1C00FB9F4D691BF41675BC4C8FA9A00C240AFB3C37BB0B3C10103000000010000000B000000E0E65A62275FF3C1C04DD36847F0D7C14422412E8665FBC135032AE342CAF5C130CE2639FFB5E8415A87A92BF019E4C1B836C82B63FFF44100D66C891B5CD4C11A8144AFA273F541F8BE8E570F2CD3416CC0750B58DCE74100BD28C324C3DD417CBAB9E9FCD8FB410CBF47ADD13E004260B7BF4E0D61EE41E8125410DFDDF44125E767329DEFF2C1C8872D12881EE541D9C93E2A020102C23C505A59DC79E041E0E65A62275FF3C1C04DD36847F0D7C1010300000001000000050000008C59CC2680C3E341946C8F4E5C43DFC1B079180A387902424A8ABA12AB59EAC1A85532A10C64E6412C8C5B2BFF64024230B89A7910E2E1C1FE4D023E917CF3418C59CC2680C3E341946C8F4E5C43DFC1':::GEOMETRY + AS col_664939 + FROM + (VALUES (NULL)) AS tab_397805 (col_664938) + ) + ), + (st_mpointfromwkb(e'\\xc00a3213a312d8b9e4':::BYTES::BYTES, ('107.115.23.162/21':::INET::INET - '243.32.48.41/8':::INET::INET)::INT8::INT8)::GEOMETRY) + ) + AS tab_397806 (col_664940) ON cte_ref_33032.col_664924 + JOIN ( + VALUES + ( + '01030000400100000006000000604DBD5DDC12EE41E7F92AFA7236F5C1C806EE05E20FF8C1AE0EC58E089AF041037B428FCC74F4C1105BADD850C1FC41407E0CD5F406FC416C6E81576649F5C1A88849CFCCB2E7413DF8A105E4DBF9C1C05FB16532DADCC15BB2F12004BA00C2E0F5EDCFFDD8F5C114735FD70C27E2C188A7663531A5E0C1604DBD5DDC12EE41E7F92AFA7236F5C1C806EE05E20FF8C1':::GEOMETRY + ), + ( + st_translate( + '010700000000000000':::GEOMETRY::GEOMETRY, + atan2d(cosh((-1.7130878673648953):::FLOAT8::FLOAT8)::FLOAT8::FLOAT8, (-0.004513979274377716):::FLOAT8::FLOAT8)::FLOAT8::FLOAT8, + 0.4371603268852574:::FLOAT8::FLOAT8, + (-0.29366208391016146):::FLOAT8::FLOAT8 + )::GEOMETRY + ) + ) + AS tab_397811 (col_664945) ON (tab_397806.col_664940) = (tab_397811.col_664945) + JOIN ( + VALUES + ( + '0105000080030000000102000080020000005815AB9CA02EFD418A62B4D90D47FA411C81B0625EEDEA41B4A3F2F01CDEFAC1B28DADD0E1DF01421828B463FE52D6C101020000800300000090BA7156E31FCBC138ED53761B8EEB41B69D1E893034F341C887135538B9D0C11C215672AA65E941CC5E7459EE4FF7C1CE07DAA9DCD0ECC134C355C3C207ED4178A281C13FECD3410102000080030000001A4894B4A2AAEEC1BE1C45840D15FEC170113FBD1CB0E3414E7477CF58ADF841FA9631702A7BF7C1C06DD44EA605B1C154FAF1F87849FC41147370821DCCEA41942E895E2E45EAC1':::GEOMETRY, + ARRAY[2834432470:::OID, 1285215016:::OID, 2816797228:::OID] + ) + ) + AS tab_397813 (col_664948, col_664949) ON (tab_397806.col_664940) = (tab_397813.col_664948) + ); From 71d26e70afa82203609ff68600c5ceb02bba996a Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Wed, 29 Jun 2022 12:06:32 +0100 Subject: [PATCH 10/15] streamingccl: re-enabled TestRandomClientGeneration TestRandomClientGeneration was skipped in #61292 as a flake. However, in the time since then, other changes in this code broke this test more completely. Re-enabling the test requirea few unrelated changes: - The stream ingestion processor required a fully formed job to be able to poll the cutover time. Now, test code can set a cutoverProvider that doesn't depend on a full job record. - The request intercepting depended on an explicit client being set. This test was rather passing the processor a randgen URI. Now we pass the client explicitly and also update the test code to make it clear that the stream URI isn't actually used for anything. - The code was attempting to validate the number of rows using SQL. I haven't dug into how this was working in the past. But as we are connecting to the host tenant and the keys are being ingested to a guest tenant, we would need a connection to the guest tenant to validate the table data. I've simply removed this assertion since I don't think it was testing very much compared to the KV level assertions also used in the test. - The test code assumed that the partitions were keyed based on the subscription token rather than the subscription ID. It isn't clear what the original source of the flakiness was. However, the test has run a few hundred times under stress without issue. Alternatively, we could just delete this test. Fixes #61287 Release note: None --- .../streamclient/random_stream_client.go | 2 +- .../stream_ingestion_processor.go | 65 ++++++++++++++----- .../stream_ingestion_processor_test.go | 57 ++++++++-------- 3 files changed, 78 insertions(+), 46 deletions(-) diff --git a/pkg/ccl/streamingccl/streamclient/random_stream_client.go b/pkg/ccl/streamingccl/streamclient/random_stream_client.go index b37a1ff55519..9dd0794f93ab 100644 --- a/pkg/ccl/streamingccl/streamclient/random_stream_client.go +++ b/pkg/ccl/streamingccl/streamclient/random_stream_client.go @@ -284,7 +284,7 @@ func (m *randomStreamClient) getDescriptorAndNamespaceKVForTableID( // Generate namespace entry. codec := keys.MakeSQLCodec(config.tenantID) - key := catalogkeys.MakePublicObjectNameKey(codec, 50, testTable.Name) + key := catalogkeys.MakePublicObjectNameKey(codec, IngestionDatabaseID, testTable.Name) k := rekey(config.tenantID, key) var value roachpb.Value value.SetInt(int64(testTable.GetID())) diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go index b1382d3590d0..80c76c76966c 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl" "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl" "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" + "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv/bulk" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -97,6 +98,9 @@ type streamIngestionProcessor struct { // consuming multiple partitions from a stream. streamPartitionClients []streamclient.Client + // cutoverProvider indicates when the cutover time has been reached. + cutoverProvider cutoverProvider + // Checkpoint events may need to be buffered if they arrive within the same // minimumFlushInterval. bufferedCheckpoints map[string]hlc.Timestamp @@ -181,10 +185,14 @@ func newStreamIngestionDataProcessor( curBatch: make([]storage.MVCCKeyValue, 0), bufferedCheckpoints: make(map[string]hlc.Timestamp), maxFlushRateTimer: timeutil.NewTimer(), - cutoverCh: make(chan struct{}), - closePoller: make(chan struct{}), - rekeyer: rekeyer, - rewriteToDiffKey: spec.TenantRekey.NewID != spec.TenantRekey.OldID, + cutoverProvider: &cutoverFromJobProgress{ + jobID: jobspb.JobID(spec.JobID), + registry: flowCtx.Cfg.JobRegistry, + }, + cutoverCh: make(chan struct{}), + closePoller: make(chan struct{}), + rekeyer: rekeyer, + rewriteToDiffKey: spec.TenantRekey.NewID != spec.TenantRekey.OldID, } if err := sip.Init(sip, post, streamIngestionResultTypes, flowCtx, processorID, output, nil, /* memMonitor */ execinfra.ProcStateOpts{ @@ -352,9 +360,7 @@ func (sip *streamIngestionProcessor) checkForCutoverSignal( ctx context.Context, stopPoller chan struct{}, ) error { sv := &sip.flowCtx.Cfg.Settings.SV - registry := sip.flowCtx.Cfg.JobRegistry tick := time.NewTicker(cutoverSignalPollInterval.Get(sv)) - jobID := sip.spec.JobID defer tick.Stop() for { select { @@ -363,20 +369,11 @@ func (sip *streamIngestionProcessor) checkForCutoverSignal( case <-ctx.Done(): return ctx.Err() case <-tick.C: - j, err := registry.LoadJob(ctx, jobspb.JobID(jobID)) + cutoverReached, err := sip.cutoverProvider.cutoverReached(ctx) if err != nil { return err } - progress := j.Progress() - var sp *jobspb.Progress_StreamIngest - var ok bool - if sp, ok = progress.GetDetails().(*jobspb.Progress_StreamIngest); !ok { - return errors.Newf("unknown progress type %T in stream ingestion job %d", - j.Progress().Progress, jobID) - } - // Job has been signaled to complete. - if resolvedTimestamp := progress.GetHighWater(); !sp.StreamIngest.CutoverTime.IsEmpty() && - resolvedTimestamp != nil && sp.StreamIngest.CutoverTime.Less(*resolvedTimestamp) { + if cutoverReached { sip.cutoverCh <- struct{}{} return nil } @@ -664,6 +661,40 @@ func (sip *streamIngestionProcessor) flush() (*jobspb.ResolvedSpans, error) { return &flushedCheckpoints, sip.batcher.Reset(ctx) } +// cutoverProvider allows us to override how we decide when the job has reached +// the cutover places in tests. +type cutoverProvider interface { + cutoverReached(context.Context) (bool, error) +} + +// custoverFromJobProgress is a cutoverProvider that decides whether the cutover +// time has been reached based on the progress stored on the job record. +type cutoverFromJobProgress struct { + registry *jobs.Registry + jobID jobspb.JobID +} + +func (c *cutoverFromJobProgress) cutoverReached(ctx context.Context) (bool, error) { + j, err := c.registry.LoadJob(ctx, c.jobID) + if err != nil { + return false, err + } + progress := j.Progress() + var sp *jobspb.Progress_StreamIngest + var ok bool + if sp, ok = progress.GetDetails().(*jobspb.Progress_StreamIngest); !ok { + return false, errors.Newf("unknown progress type %T in stream ingestion job %d", + j.Progress().Progress, c.jobID) + } + // Job has been signaled to complete. + if resolvedTimestamp := progress.GetHighWater(); !sp.StreamIngest.CutoverTime.IsEmpty() && + resolvedTimestamp != nil && sp.StreamIngest.CutoverTime.Less(*resolvedTimestamp) { + return true, nil + } + + return false, nil +} + func init() { rowexec.NewStreamIngestionDataProcessor = newStreamIngestionDataProcessor } diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go index 644935edea86..cd23cdff0c14 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go @@ -38,8 +38,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/streaming" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/distsqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -202,8 +200,8 @@ func TestStreamIngestionProcessor(t *testing.T) { {ID: "1", SubscriptionToken: p1}, {ID: "2", SubscriptionToken: p2}, } - out, err := runStreamIngestionProcessor(ctx, t, registry, kvDB, "randomgen://test/", - partitions, startTime, nil /* interceptEvents */, tenantRekey, mockClient) + out, err := runStreamIngestionProcessor(ctx, t, registry, kvDB, + partitions, startTime, nil /* interceptEvents */, tenantRekey, mockClient, nil /* cutoverProvider */) require.NoError(t, err) actualRows := make(map[string]struct{}) @@ -238,8 +236,8 @@ func TestStreamIngestionProcessor(t *testing.T) { {SubscriptionToken: streamclient.SubscriptionToken("1")}, {SubscriptionToken: streamclient.SubscriptionToken("2")}, } - out, err := runStreamIngestionProcessor(ctx, t, registry, kvDB, "randomgen://test", - partitions, startTime, nil /* interceptEvents */, tenantRekey, &errorStreamClient{}) + out, err := runStreamIngestionProcessor(ctx, t, registry, kvDB, + partitions, startTime, nil /* interceptEvents */, tenantRekey, &errorStreamClient{}, nil /* cutoverProvider */) require.NoError(t, err) // Expect no rows, and just the error. @@ -262,8 +260,8 @@ func TestStreamIngestionProcessor(t *testing.T) { streamingTestingKnob := &sql.StreamingTestingKnobs{RunAfterReceivingEvent: func(ctx context.Context) { processEventCh <- struct{}{} }} - sip, out, err := getStreamIngestionProcessor(ctx, t, registry, kvDB, "randomgen://test/", - partitions, startTime, nil /* interceptEvents */, tenantRekey, mockClient, streamingTestingKnob) + sip, out, err := getStreamIngestionProcessor(ctx, t, registry, kvDB, + partitions, startTime, nil /* interceptEvents */, tenantRekey, mockClient, nil /* cutoverProvider */, streamingTestingKnob) defer func() { require.NoError(t, sip.forceClientForTests.Close()) }() @@ -310,7 +308,7 @@ func getPartitionSpanToTableID( // Aggregate the table IDs which should have been ingested. for _, pa := range partitions { - pKey := roachpb.Key(pa.SubscriptionToken) + pKey := roachpb.Key(pa.ID) pSpan := roachpb.Span{Key: pKey, EndKey: pKey.Next()} paURL, err := url.Parse(string(pa.SubscriptionToken)) require.NoError(t, err) @@ -401,11 +399,14 @@ func makeTestStreamURI( "&TENANT_ID=" + strconv.Itoa(tenantID) } +type noCutover struct{} + +func (n noCutover) cutoverReached(context.Context) (bool, error) { return false, nil } + // TestRandomClientGeneration tests the ingestion processor against a random // stream workload. func TestRandomClientGeneration(t *testing.T) { defer leaktest.AfterTest(t)() - skip.WithIssue(t, 61287, "flaky test") defer log.Scope(t).Close(t) ctx := context.Background() @@ -414,8 +415,6 @@ func TestRandomClientGeneration(t *testing.T) { defer tc.Stopper().Stop(ctx) registry := tc.Server(0).JobRegistry().(*jobs.Registry) kvDB := tc.Server(0).DB() - conn := tc.Conns[0] - sqlDB := sqlutils.MakeSQLRunner(conn) // TODO: Consider testing variations on these parameters. const tenantID = 20 @@ -447,8 +446,10 @@ func TestRandomClientGeneration(t *testing.T) { require.NoError(t, err) streamValidator := newStreamClientValidator(rekeyer) validator := registerValidatorWithClient(streamValidator) - out, err := runStreamIngestionProcessor(ctx, t, registry, kvDB, streamAddr, topo, - startTime, []streamclient.InterceptFn{cancelAfterCheckpoints, validator}, tenantRekey, nil /* mockClient */) + + out, err := runStreamIngestionProcessor(ctx, t, registry, kvDB, + topo, startTime, []streamclient.InterceptFn{cancelAfterCheckpoints, validator}, tenantRekey, + streamClient, noCutover{}) require.NoError(t, err) partitionSpanToTableID := getPartitionSpanToTableID(t, topo) @@ -466,6 +467,7 @@ func TestRandomClientGeneration(t *testing.T) { if row == nil { break } + datum := row[0].Datum protoBytes, ok := datum.(*tree.DBytes) require.True(t, ok) @@ -475,8 +477,8 @@ func TestRandomClientGeneration(t *testing.T) { for _, resolvedSpan := range resolvedSpans.ResolvedSpans { if _, ok := partitionSpanToTableID[resolvedSpan.Span.String()]; !ok { - t.Fatalf("expected resolved span %v to be either in one of the supplied partition"+ - " addresses %v", resolvedSpan.Span, topo) + t.Fatalf("expected resolved span %v to be in one of the supplied partition"+ + " addresses %v", resolvedSpan.Span, partitionSpanToTableID) } // All resolved timestamp events should be greater than the start time. @@ -497,11 +499,6 @@ func TestRandomClientGeneration(t *testing.T) { } for pSpan, id := range partitionSpanToTableID { - numRows, err := strconv.Atoi(sqlDB.QueryStr(t, fmt.Sprintf( - `SELECT count(*) FROM defaultdb.%s%d`, streamclient.IngestionTablePrefix, id))[0][0]) - require.NoError(t, err) - require.Greater(t, numRows, 0, "at least 1 row ingested expected") - // Scan the store for KVs ingested by this partition, and compare the MVCC // KVs against the KVEvents streamed up to the max ingested timestamp for // the partition. @@ -515,15 +512,15 @@ func runStreamIngestionProcessor( t *testing.T, registry *jobs.Registry, kvDB *kv.DB, - streamAddr string, partitions streamclient.Topology, startTime hlc.Timestamp, interceptEvents []streamclient.InterceptFn, tenantRekey execinfrapb.TenantRekey, mockClient streamclient.Client, + cutoverProvider cutoverProvider, ) (*distsqlutils.RowBuffer, error) { - sip, out, err := getStreamIngestionProcessor(ctx, t, registry, kvDB, streamAddr, - partitions, startTime, interceptEvents, tenantRekey, mockClient, nil /* streamingTestingKnobs */) + sip, out, err := getStreamIngestionProcessor(ctx, t, registry, kvDB, + partitions, startTime, interceptEvents, tenantRekey, mockClient, cutoverProvider, nil /* streamingTestingKnobs */) require.NoError(t, err) sip.Run(ctx) @@ -543,16 +540,19 @@ func getStreamIngestionProcessor( t *testing.T, registry *jobs.Registry, kvDB *kv.DB, - streamAddr string, partitions streamclient.Topology, startTime hlc.Timestamp, interceptEvents []streamclient.InterceptFn, tenantRekey execinfrapb.TenantRekey, mockClient streamclient.Client, + cutoverProvider cutoverProvider, streamingTestingKnobs *sql.StreamingTestingKnobs, ) (*streamIngestionProcessor, *distsqlutils.RowBuffer, error) { st := cluster.MakeTestingClusterSettings() evalCtx := eval.MakeTestingEvalContext(st) + if mockClient == nil { + return nil, nil, errors.AssertionFailedf("non-nil streamclient required") + } testDiskMonitor := execinfra.NewTestDiskMonitor(ctx, st) defer testDiskMonitor.Stop(ctx) @@ -573,7 +573,7 @@ func getStreamIngestionProcessor( post := execinfrapb.PostProcessSpec{} var spec execinfrapb.StreamIngestionDataSpec - spec.StreamAddress = streamAddr + spec.StreamAddress = "http://unused" spec.TenantRekey = tenantRekey spec.PartitionIds = make([]string, len(partitions)) spec.PartitionAddresses = make([]string, len(partitions)) @@ -592,8 +592,9 @@ func getStreamIngestionProcessor( t.Fatal("expected the processor that's created to be a split and scatter processor") } - if mockClient != nil { - sip.forceClientForTests = mockClient + sip.forceClientForTests = mockClient + if cutoverProvider != nil { + sip.cutoverProvider = cutoverProvider } if interceptable, ok := sip.forceClientForTests.(streamclient.InterceptableStreamClient); ok { From 155d4d031c2a498aed6a73dfbff595c69b9fda81 Mon Sep 17 00:00:00 2001 From: Shiranka Miskin Date: Thu, 26 May 2022 01:30:20 -0400 Subject: [PATCH 11/15] changefeedccl: fix changefeed telemetry resolved Resolves #81599 Our internal-use changefeed create / failed telemetry events would incorrectly record "yes" for any non-no value of resolved, rather than using the value that was passed in. This change resolves that, emitting yes for "resolved" and the value itself for "resolved=". Release note: None --- pkg/ccl/changefeedccl/changefeed_stmt.go | 2 +- pkg/ccl/changefeedccl/changefeed_test.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index c95c3c931817..d5d6d4d48b1b 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -1186,7 +1186,7 @@ func getCommonChangefeedEventDetails( resolvedValue, resolvedSet := opts[changefeedbase.OptResolvedTimestamps] if !resolvedSet { resolved = "no" - } else if resolved == `` { + } else if resolvedValue == `` { resolved = "yes" } else { resolved = resolvedValue diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 8b4dc0cf765f..f3f86a261de8 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -6663,14 +6663,14 @@ func TestChangefeedCreateTelemetryLogs(t *testing.T) { t.Run(`gcpubsub_sink_type with options`, func(t *testing.T) { pubsubFeedFactory := makePubsubFeedFactory(s.Server, s.DB) beforeCreatePubsub := timeutil.Now() - pubsubFeed := feed(t, pubsubFeedFactory, `CREATE CHANGEFEED FOR foo, bar WITH resolved, no_initial_scan`) + pubsubFeed := feed(t, pubsubFeedFactory, `CREATE CHANGEFEED FOR foo, bar WITH resolved="10s", no_initial_scan`) defer closeFeed(t, pubsubFeed) createLogs := checkCreateChangefeedLogs(t, beforeCreatePubsub.UnixNano()) require.Equal(t, 1, len(createLogs)) require.Equal(t, createLogs[0].SinkType, `gcpubsub`) require.Equal(t, createLogs[0].NumTables, int32(2)) - require.Equal(t, createLogs[0].Resolved, `yes`) + require.Equal(t, createLogs[0].Resolved, `10s`) require.Equal(t, createLogs[0].InitialScan, `no`) }) } From 6d03b870464f8c5e8676285db2042006495feb94 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Tue, 28 Jun 2022 17:49:09 -0400 Subject: [PATCH 12/15] changefeedccl: Support more stable functions. Add support for additional stable functions to CDC expressions. Fixes #83466 Release Notes: None --- pkg/ccl/changefeedccl/cdceval/BUILD.bazel | 2 +- pkg/ccl/changefeedccl/cdceval/constraint.go | 2 +- pkg/ccl/changefeedccl/cdceval/expr_eval.go | 69 +++-- pkg/ccl/changefeedccl/cdceval/functions.go | 28 +- .../changefeedccl/cdceval/functions_test.go | 248 ++++++++++++++++-- pkg/ccl/changefeedccl/cdcevent/event.go | 3 + 6 files changed, 290 insertions(+), 62 deletions(-) diff --git a/pkg/ccl/changefeedccl/cdceval/BUILD.bazel b/pkg/ccl/changefeedccl/cdceval/BUILD.bazel index b4aa76078235..be4a83528ec8 100644 --- a/pkg/ccl/changefeedccl/cdceval/BUILD.bazel +++ b/pkg/ccl/changefeedccl/cdceval/BUILD.bazel @@ -26,7 +26,6 @@ go_library( "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", - "//pkg/sql/sem/builtins", "//pkg/sql/sem/eval", "//pkg/sql/sem/normalize", "//pkg/sql/sem/tree", @@ -82,6 +81,7 @@ go_test( "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/randutil", + "//pkg/util/timeofday", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/ccl/changefeedccl/cdceval/constraint.go b/pkg/ccl/changefeedccl/cdceval/constraint.go index 09773d1f0348..404a993d35d0 100644 --- a/pkg/ccl/changefeedccl/cdceval/constraint.go +++ b/pkg/ccl/changefeedccl/cdceval/constraint.go @@ -94,7 +94,7 @@ func constrainSpansBySelectClause( } tableName := tableNameOrAlias(ed.TableName, selectClause.From.Tables[0]) - semaCtx := newSemaCtx(ed) + semaCtx := newSemaCtxWithTypeResolver(ed) return sc.ConstrainPrimaryIndexSpanByExpr( ctx, sql.BestEffortConstrain, tableName, ed.TableDescriptor(), evalCtx, semaCtx, selectClause.Where.Expr) diff --git a/pkg/ccl/changefeedccl/cdceval/expr_eval.go b/pkg/ccl/changefeedccl/cdceval/expr_eval.go index 3273eccbbd5d..08a295847733 100644 --- a/pkg/ccl/changefeedccl/cdceval/expr_eval.go +++ b/pkg/ccl/changefeedccl/cdceval/expr_eval.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/normalize" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -106,9 +105,10 @@ func (e *Evaluator) initSelectClause(sc *tree.SelectClause) error { "expected at least 1 projection") } + semaCtx := newSemaCtx() e.selectors = sc.Exprs for _, se := range e.selectors { - expr, err := validateExpressionForCDC(se.Expr) + expr, err := validateExpressionForCDC(se.Expr, semaCtx) if err != nil { return err } @@ -116,7 +116,7 @@ func (e *Evaluator) initSelectClause(sc *tree.SelectClause) error { } if sc.Where != nil { - expr, err := validateExpressionForCDC(sc.Where.Expr) + expr, err := validateExpressionForCDC(sc.Where.Expr, semaCtx) if err != nil { return err } @@ -207,7 +207,7 @@ func newExprEval( cols := ed.ResultColumns() e := &exprEval{ EventDescriptor: ed, - semaCtx: newSemaCtx(ed), + semaCtx: newSemaCtxWithTypeResolver(ed), evalCtx: evalCtx.Copy(), evalHelper: &rowContainer{cols: cols}, projection: cdcevent.MakeProjection(ed), @@ -268,6 +268,7 @@ func (e *exprEval) setupContext( e.rowEvalCtx.prevRow = prevRow e.rowEvalCtx.mvccTS = mvccTS e.evalCtx.TxnTimestamp = mvccTS.GoTime() + e.evalCtx.StmtTimestamp = mvccTS.GoTime() // Clear out all memo records e.rowEvalCtx.memo.prevJSON = nil @@ -482,7 +483,8 @@ func (e *exprEval) evalExpr( // if it consists of expressions supported by CDC. // This visitor is used early to sanity check expression. type cdcExprVisitor struct { - err error + semaCtx *tree.SemaContext + err error } var _ tree.Visitor = (*cdcExprVisitor)(nil) @@ -490,8 +492,8 @@ var _ tree.Visitor = (*cdcExprVisitor)(nil) // validateExpressionForCDC runs quick checks to make sure that expr is valid for // CDC use case. This doesn't catch all the invalid cases, but is a place to pick up // obviously wrong expressions. -func validateExpressionForCDC(expr tree.Expr) (tree.Expr, error) { - var v cdcExprVisitor +func validateExpressionForCDC(expr tree.Expr, semaCtx *tree.SemaContext) (tree.Expr, error) { + v := cdcExprVisitor{semaCtx: semaCtx} expr, _ = tree.WalkExpr(&v, expr) if v.err != nil { return nil, v.err @@ -508,7 +510,7 @@ func (v *cdcExprVisitor) VisitPre(expr tree.Expr) (bool, tree.Expr) { func (v *cdcExprVisitor) VisitPost(expr tree.Expr) tree.Expr { switch t := expr.(type) { case *tree.FuncExpr: - fn, err := checkFunctionSupported(t) + fn, err := checkFunctionSupported(t, v.semaCtx) if err != nil { v.err = err return expr @@ -582,19 +584,9 @@ func (v *cdcNameResolver) VisitPost(expr tree.Expr) tree.Expr { } } -func resolveCustomCDCFunction(name string, fnCall *tree.FuncExpr) *tree.FuncExpr { - fn, exists := cdcFunctions[name] - if !exists { - return nil - } - return &tree.FuncExpr{ - Func: tree.ResolvableFunctionReference{FunctionReference: fn}, - Type: fnCall.Type, - Exprs: fnCall.Exprs, - } -} - -func checkFunctionSupported(fnCall *tree.FuncExpr) (*tree.FuncExpr, error) { +func checkFunctionSupported( + fnCall *tree.FuncExpr, semaCtx *tree.SemaContext, +) (*tree.FuncExpr, error) { var fnName string var fnClass tree.FunctionClass var fnVolatility volatility.V @@ -610,23 +602,19 @@ func checkFunctionSupported(fnCall *tree.FuncExpr) (*tree.FuncExpr, error) { switch fn := fnCall.Func.FunctionReference.(type) { case *tree.UnresolvedName: - // We may not have function definition yet if function takes arguments, - // or it's one of the custom cdc functions. - fnName = fn.String() - props, overloads := builtins.GetBuiltinProperties(fn.String()) - if props == nil { - if custom := resolveCustomCDCFunction(fnName, fnCall); custom != nil { - return custom, nil - } + funDef, err := fn.ResolveFunction(semaCtx.SearchPath) + if err != nil { return nil, unsupportedFunctionErr() } - fnClass = props.Class - // Pick highest volatility overload. - for _, o := range overloads { - if o.Volatility > fnVolatility { - fnVolatility = o.Volatility - } + fnCall = &tree.FuncExpr{ + Func: tree.ResolvableFunctionReference{FunctionReference: funDef}, + Type: fnCall.Type, + Exprs: fnCall.Exprs, + } + if _, isCDCFn := cdcFunctions[funDef.Name]; isCDCFn { + return fnCall, nil } + return checkFunctionSupported(fnCall, semaCtx) case *tree.FunctionDefinition: fnName, fnClass = fn.Name, fn.Class if fnCall.ResolvedOverload() != nil { @@ -721,16 +709,21 @@ func rowEvalContextFromEvalContext(evalCtx *eval.Context) *rowEvalContext { const rejectInvalidCDCExprs = (tree.RejectAggregates | tree.RejectGenerators | tree.RejectWindowApplications | tree.RejectNestedGenerators) -// newSemaCtx returns new tree.SemaCtx configured for cdc. -func newSemaCtx(d *cdcevent.EventDescriptor) *tree.SemaContext { +// newSemaCtx returns new tree.SemaCtx configured for cdc without type resolver. +func newSemaCtx() *tree.SemaContext { sema := tree.MakeSemaContext() sema.SearchPath = &cdcCustomFunctionResolver{SearchPath: &sessiondata.DefaultSearchPath} sema.Properties.Require("cdc", rejectInvalidCDCExprs) + return &sema +} +// newSemaCtxWithTypeResolver returns new tree.SemaCtx configured for cdc. +func newSemaCtxWithTypeResolver(d *cdcevent.EventDescriptor) *tree.SemaContext { + sema := newSemaCtx() if d.HasUserDefinedTypes() { sema.TypeResolver = newTypeReferenceResolver(d) } - return &sema + return sema } // cdcTypeReferenceReesolver is responsible for resolving user defined types. diff --git a/pkg/ccl/changefeedccl/cdceval/functions.go b/pkg/ccl/changefeedccl/cdceval/functions.go index 91cb91c84263..4bcb7fb3be23 100644 --- a/pkg/ccl/changefeedccl/cdceval/functions.go +++ b/pkg/ccl/changefeedccl/cdceval/functions.go @@ -27,16 +27,36 @@ import ( // However, we can provide reasonable overrides to a small set of stable // functions that make sense in the context of CDC. var supportedVolatileBuiltinFunctions = makeStringSet( - // These functions can be supported given that we set the statement - // and transaction timestamp to be equal to MVCC timestamp of the event. - // TODO(yevgeniy): We also define cdc specific functions, s.a. cdc_mvcc_timestamp - // Maybe delete cdc_ overrides; or.... maybe disallow these builtins in favor of cdc_ specific overrides? + // These functions can be supported given that we set the statement and + // transaction timestamp to be equal to MVCC timestamp of the event. "current_date", "current_timestamp", "localtimestamp", + "localtime", "now", "statement_timestamp", "transaction_timestamp", + "timeofday", + "timezone", + + // jsonb functions are stable because they depend on eval + // context DataConversionConfig + "jsonb_build_array", + "jsonb_build_object", + "to_json", + "to_jsonb", + "row_to_json", + + // Misc functions that depend on eval context. + "overlaps", + "pg_collation_for", + "pg_typeof", + "quote_literal", + "quote_nullable", + + // TODO(yevgeniy): Support geometry. + //"st_asgeojson", + //"st_estimatedextent", ) // CDC Specific functions. diff --git a/pkg/ccl/changefeedccl/cdceval/functions_test.go b/pkg/ccl/changefeedccl/cdceval/functions_test.go index fabcf5e4095c..d698279688fb 100644 --- a/pkg/ccl/changefeedccl/cdceval/functions_test.go +++ b/pkg/ccl/changefeedccl/cdceval/functions_test.go @@ -11,7 +11,6 @@ package cdceval import ( "context" "fmt" - "strconv" "testing" "time" @@ -26,6 +25,8 @@ import ( jsonb "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/cockroachdb/cockroach/pkg/util/timeofday" "github.com/stretchr/testify/require" ) @@ -44,22 +45,87 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { ctx := context.Background() - t.Run("current_timestamp", func(t *testing.T) { - testRow := cdcevent.TestingMakeEventRow(desc, 0, randEncDatumRow(t, desc, 0), false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - "SELECT current_timestamp::int") - require.NoError(t, err) + t.Run("time", func(t *testing.T) { + // We'll run tests against some future time stamp to ensure + // that time functions use correct values. futureTS := s.Clock().Now().Add(int64(60*time.Minute), 0) - p, err := e.evalProjection(ctx, testRow, futureTS, testRow) - require.NoError(t, err) - require.Equal(t, - map[string]string{"current_timestamp": strconv.FormatInt(futureTS.GoTime().Unix(), 10)}, - slurpValues(t, p)) + expectTSTZ := func() string { + t.Helper() + d, err := tree.MakeDTimestampTZ(futureTS.GoTime(), time.Microsecond) + require.NoError(t, err) + return tree.AsStringWithFlags(d, tree.FmtExport) + }() + expectTS := func() string { + t.Helper() + d, err := tree.MakeDTimestamp(futureTS.GoTime(), time.Microsecond) + require.NoError(t, err) + return tree.AsStringWithFlags(d, tree.FmtExport) + }() + expectTime := func() string { + d := tree.MakeDTime(timeofday.FromTime(futureTS.GoTime().Round(time.Microsecond))) + return tree.AsStringWithFlags(d, tree.FmtExport) + }() + expectDate := func() string { + t.Helper() + d, err := tree.NewDDateFromTime(futureTS.GoTime()) + require.NoError(t, err) + return tree.AsStringWithFlags(d, tree.FmtExport) + }() + + for _, tc := range []struct { + fn string + expect string + }{ + {fn: "current_timestamp", expect: expectTSTZ}, + {fn: "current_date", expect: expectDate}, + {fn: "localtimestamp", expect: expectTS}, + {fn: "localtime", expect: expectTime}, + {fn: "now", expect: expectTSTZ}, + {fn: "statement_timestamp", expect: expectTSTZ}, + {fn: "transaction_timestamp", expect: expectTSTZ}, + { + fn: "timeofday", + expect: futureTS.GoTime().Format("Mon Jan 2 15:04:05.000000 2006 -0700"), + }, + } { + t.Run(tc.fn, func(t *testing.T) { + testRow := cdcevent.TestingMakeEventRow(desc, 0, + randEncDatumRow(t, desc, 0), false) + e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + fmt.Sprintf("SELECT %s()", tc.fn)) + require.NoError(t, err) + p, err := e.evalProjection(ctx, testRow, futureTS, testRow) + require.NoError(t, err) + require.Equal(t, map[string]string{tc.fn: tc.expect}, slurpValues(t, p)) + }) + } + + t.Run("timezone", func(t *testing.T) { + // Timezone has many overrides, some are immutable, and some are Stable. + // Call "stable" overload which relies on session data containing + // timezone. Since we don't do any special setup with session data, the + // default timezone is UTC. We'll use a "strange" timezone of -1h33m from + // UTC to test conversion. + testRow := cdcevent.TestingMakeEventRow(desc, 0, + randEncDatumRow(t, desc, 0), false) + + e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + fmt.Sprintf("SELECT timezone('+01:33:00', '%s'::time)", + futureTS.GoTime().Format("15:04:05"))) + require.NoError(t, err) + p, err := e.evalProjection(ctx, testRow, futureTS, testRow) + require.NoError(t, err) + + expectedTZ := fmt.Sprintf("%s-01:33:00", + futureTS.GoTime().Add(-93*time.Minute).Format("15:04:05")) + require.Equal(t, map[string]string{"timezone": expectedTZ}, slurpValues(t, p)) + }) }) t.Run("cdc_is_delete", func(t *testing.T) { for _, expectDelete := range []bool{true, false} { - testRow := cdcevent.TestingMakeEventRow(desc, 0, randEncDatumRow(t, desc, 0), expectDelete) + testRow := cdcevent.TestingMakeEventRow(desc, 0, + randEncDatumRow(t, desc, 0), expectDelete) e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, "SELECT cdc_is_delete()") require.NoError(t, err) @@ -72,6 +138,13 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { } }) + mustParseJSON := func(d tree.Datum) jsonb.JSON { + t.Helper() + j, err := tree.AsJSON(d, sessiondatapb.DataConversionConfig{}, time.UTC) + require.NoError(t, err) + return j + } + t.Run("cdc_prev", func(t *testing.T) { rowDatums := randEncDatumRow(t, desc, 0) testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) @@ -88,11 +161,7 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { // Otherwise, expect to get JSONB. b := jsonb.NewObjectBuilder(len(rowDatums)) for i, d := range rowDatums { - j, err := tree.AsJSON(d.Datum, sessiondatapb.DataConversionConfig{}, time.UTC) - if err != nil { - require.NoError(t, err) - } - b.Add(desc.PublicColumns()[i].GetName(), j) + b.Add(desc.PublicColumns()[i].GetName(), mustParseJSON(d.Datum)) } expectedJSON := b.Build() @@ -105,7 +174,8 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { t.Run(fmt.Sprintf("cdc_{mvcc,updated}_timestamp()%s", cast), func(t *testing.T) { schemaTS := s.Clock().Now().Add(int64(60*time.Minute), 0) mvccTS := schemaTS.Add(int64(30*time.Minute), 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, randEncDatumRow(t, desc, 0), false) + testRow := cdcevent.TestingMakeEventRow(desc, 0, + randEncDatumRow(t, desc, 0), false) testRow.EventDescriptor.SchemaTS = schemaTS e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, @@ -124,4 +194,146 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { slurpValues(t, p)) }) } + + t.Run("pg_collation_for", func(t *testing.T) { + testRow := cdcevent.TestingMakeEventRow(desc, 0, + randEncDatumRow(t, desc, 0), false) + e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + `SELECT pg_collation_for('hello' COLLATE de_DE) AS col`) + require.NoError(t, err) + + p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), testRow) + require.NoError(t, err) + require.Equal(t, map[string]string{"col": "\"de_de\""}, slurpValues(t, p)) + }) + + for _, fn := range []string{"to_json", "to_jsonb"} { + t.Run(fn, func(t *testing.T) { + rowDatums := randEncDatumRow(t, desc, 0) + testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) + e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + fmt.Sprintf("SELECT %s(a)", fn)) + require.NoError(t, err) + + p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + require.NoError(t, err) + require.Equal(t, + map[string]string{fn: mustParseJSON(rowDatums[0].Datum).String()}, + slurpValues(t, p)) + }) + } + + t.Run("row_to_json", func(t *testing.T) { + rowDatums := randEncDatumRow(t, desc, 0) + testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) + e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + "SELECT row_to_json(row(a, b, c))") + require.NoError(t, err) + + b := jsonb.NewObjectBuilder(len(rowDatums)) + for i, d := range rowDatums { + b.Add(fmt.Sprintf("f%d", i+1), mustParseJSON(d.Datum)) + } + expectedJSON := b.Build() + + p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + require.NoError(t, err) + require.Equal(t, map[string]string{"row_to_json": expectedJSON.String()}, slurpValues(t, p)) + }) + + t.Run("jsonb_build_array", func(t *testing.T) { + rowDatums := randEncDatumRow(t, desc, 0) + testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) + e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + "SELECT jsonb_build_array(a, a, 42) AS three_ints") + require.NoError(t, err) + + b := jsonb.NewArrayBuilder(3) + j := mustParseJSON(rowDatums[0].Datum) + b.Add(j) + b.Add(j) + b.Add(jsonb.FromInt(42)) + expectedJSON := b.Build() + + p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + require.NoError(t, err) + require.Equal(t, map[string]string{"three_ints": expectedJSON.String()}, slurpValues(t, p)) + }) + + t.Run("jsonb_build_object", func(t *testing.T) { + rowDatums := randEncDatumRow(t, desc, 0) + testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) + e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + "SELECT jsonb_build_object('a', a, 'b', b, 'c', c) AS obj") + require.NoError(t, err) + + b := jsonb.NewObjectBuilder(3) + b.Add("a", mustParseJSON(rowDatums[0].Datum)) + b.Add("b", mustParseJSON(rowDatums[1].Datum)) + b.Add("c", mustParseJSON(rowDatums[2].Datum)) + expectedJSON := b.Build() + + p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + require.NoError(t, err) + require.Equal(t, map[string]string{"obj": expectedJSON.String()}, slurpValues(t, p)) + }) + + for _, fn := range []string{"quote_literal", "quote_nullable"} { + // These functions have overloads; call the one that's stable overload + // (i.e. one that needs to convert types.Any to string. + t.Run(fn, func(t *testing.T) { + rowDatums := randEncDatumRow(t, desc, 0) + testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) + e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + fmt.Sprintf("SELECT %s(42)", fn)) + require.NoError(t, err) + + p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + require.NoError(t, err) + require.Equal(t, + map[string]string{fn: fmt.Sprintf("'%s'", jsonb.FromInt(42).String())}, + slurpValues(t, p)) + }) + } + + // overlaps has many overloads; most of them are immutable, but 1 is stable. + t.Run("overlaps", func(t *testing.T) { + rowDatums := randEncDatumRow(t, desc, 0) + testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) + e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + `SELECT overlaps(now(), interval '0', now(), interval '-1s')`) + require.NoError(t, err) + + p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + require.NoError(t, err) + require.Equal(t, map[string]string{"overlaps": "false"}, slurpValues(t, p)) + }) + + // Test that cdc specific functions correctly resolve overload, and that an + // error is returned when cdc function called with wrong arguments. + t.Run("cdc function errors", func(t *testing.T) { + // currently, all cdc functions take no args, so call these functions with + // some arguments. + rng, _ := randutil.NewTestRand() + fnArgs := func() string { + switch rng.Int31n(3) { + case 0: + return "a" + case 1: + return "a, b" + default: + return "a,b,c" + } + } + + for fn := range cdcFunctions { + t.Run(fn, func(t *testing.T) { + rowDatums := randEncDatumRow(t, desc, 0) + testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) + _, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, + fmt.Sprintf("SELECT %s(%s)", fn, fnArgs())) + require.Regexp(t, "unknown signature", err) + }) + } + }) } diff --git a/pkg/ccl/changefeedccl/cdcevent/event.go b/pkg/ccl/changefeedccl/cdcevent/event.go index cc2d0ef7db3e..9a485a4e74a3 100644 --- a/pkg/ccl/changefeedccl/cdcevent/event.go +++ b/pkg/ccl/changefeedccl/cdcevent/event.go @@ -98,6 +98,9 @@ func (r Row) DatumAt(at int) (tree.Datum, error) { return nil, errors.AssertionFailedf("column at %d out of bounds", at) } col := r.cols[at] + if col.ord >= len(r.datums) { + return nil, errors.AssertionFailedf("column ordinal at %d out of bounds", col.ord) + } encDatum := r.datums[col.ord] if err := encDatum.EnsureDecoded(col.Typ, r.alloc); err != nil { return nil, errors.Wrapf(err, "error decoding column %q as type %s", col.Name, col.Typ.String()) From 4548df3719ecd21b27188c2782f70fdbf0c97c76 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Tue, 28 Jun 2022 13:47:38 -0400 Subject: [PATCH 13/15] backupccl: create tree.SystemUsers, a new DescriptorCoverage enum Previously during planning and execution RESTORE SYSTEM USERS was identified by a `jobDetails` field. This refactor now identifies this flavor of restore with a new DescriptorCoverage enum value, `tree.SystemUsers. This refactor eases the logic around exposing extra processing steps for flavors of backup/restore that target different sets of descriptors. Release note: None --- pkg/ccl/backupccl/restore_job.go | 14 +++++++++++-- pkg/ccl/backupccl/restore_planning.go | 29 ++++++++++++++++----------- pkg/ccl/backupccl/targets.go | 3 +-- pkg/jobs/jobspb/jobs.proto | 3 +++ pkg/sql/parser/sql.y | 4 ++-- pkg/sql/sem/tree/backup.go | 14 +++++++------ 6 files changed, 43 insertions(+), 24 deletions(-) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index f76461985da1..05ea3cd9aa52 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -1509,7 +1509,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro if err := r.cleanupTempSystemTables(ctx, nil /* txn */); err != nil { return err } - } else if details.RestoreSystemUsers { + } else if isSystemUserRestore(details) { if err := r.restoreSystemUsers(ctx, p.ExecCfg().DB, mainData.systemTables); err != nil { return err } @@ -1560,6 +1560,16 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro return nil } +// isSystemUserRestore checks if the user called RESTORE SYSTEM USERS and guards +// against any mixed version issues. In 22.2, details.DescriptorCoverage +// identifies a system user restore, while in 22.1, details.RestoreSystemUsers +// identified this flavour of restore. +// +// TODO(msbutler): delete in 23.1 +func isSystemUserRestore(details jobspb.RestoreDetails) bool { + return details.DescriptorCoverage == tree.SystemUsers || details.RestoreSystemUsers +} + func revalidateIndexes( ctx context.Context, execCfg *sql.ExecutorConfig, @@ -1676,7 +1686,7 @@ func (r *restoreResumer) notifyStatsRefresherOfNewTables() { // This is the last of the IDs pre-allocated by the restore planner. // TODO(postamar): Store it directly in the details instead? This is brittle. func tempSystemDatabaseID(details jobspb.RestoreDetails) descpb.ID { - if details.DescriptorCoverage != tree.AllDescriptors && !details.RestoreSystemUsers { + if details.DescriptorCoverage != tree.AllDescriptors && !isSystemUserRestore(details) { return descpb.InvalidID } var maxPreAllocatedID descpb.ID diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 4a637e837e40..f35a3defa65a 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -175,7 +175,6 @@ func allocateDescriptorRewrites( opts tree.RestoreOptions, intoDB string, newDBName string, - restoreSystemUsers bool, ) (jobspb.DescRewriteMap, error) { descriptorRewrites := make(jobspb.DescRewriteMap) @@ -292,7 +291,7 @@ func allocateDescriptorRewrites( // in the backup and current max desc ID in the restoring cluster. This generator // keeps produced the next descriptor ID. var tempSysDBID descpb.ID - if descriptorCoverage == tree.AllDescriptors || restoreSystemUsers { + if descriptorCoverage == tree.AllDescriptors || descriptorCoverage == tree.SystemUsers { var err error if descriptorCoverage == tree.AllDescriptors { // Restore the key which generates descriptor IDs. @@ -323,7 +322,7 @@ func allocateDescriptorRewrites( if err != nil { return nil, err } - } else if restoreSystemUsers { + } else if descriptorCoverage == tree.SystemUsers { tempSysDBID, err = descidgen.GenerateUniqueDescID(ctx, p.ExecCfg().DB, p.ExecCfg().Codec) if err != nil { return nil, err @@ -706,7 +705,7 @@ func allocateDescriptorRewrites( // backup should have the same ID as they do in the backup. descriptorsToRemap := make([]catalog.Descriptor, 0, len(tablesByID)) for _, table := range tablesByID { - if descriptorCoverage == tree.AllDescriptors || restoreSystemUsers { + if descriptorCoverage == tree.AllDescriptors || descriptorCoverage == tree.SystemUsers { if table.ParentID == systemschema.SystemDB.GetID() { // This is a system table that should be marked for descriptor creation. descriptorsToRemap = append(descriptorsToRemap, table) @@ -988,7 +987,6 @@ func restoreJobDescription( kmsURIs []string, ) (string, error) { r := &tree.Restore{ - SystemUsers: restore.SystemUsers, DescriptorCoverage: restore.DescriptorCoverage, AsOf: restore.AsOf, Targets: restore.Targets, @@ -1065,7 +1063,7 @@ func restorePlanHook( var intoDBFn func() (string, error) if restoreStmt.Options.IntoDB != nil { - if restoreStmt.SystemUsers { + if restoreStmt.DescriptorCoverage == tree.SystemUsers { return nil, nil, nil, false, errors.New("cannot set into_db option when only restoring system users") } intoDBFn, err = p.TypeAsString(ctx, restoreStmt.Options.IntoDB, "RESTORE") @@ -1105,12 +1103,13 @@ func restorePlanHook( var newDBNameFn func() (string, error) if restoreStmt.Options.NewDBName != nil { - if restoreStmt.DescriptorCoverage == tree.AllDescriptors || len(restoreStmt.Targets.Databases) != 1 { + if restoreStmt.DescriptorCoverage == tree.AllDescriptors || + len(restoreStmt.Targets.Databases) != 1 { err = errors.New("new_db_name can only be used for RESTORE DATABASE with a single target" + " database") return nil, nil, nil, false, err } - if restoreStmt.SystemUsers { + if restoreStmt.DescriptorCoverage == tree.SystemUsers { return nil, nil, nil, false, errors.New("cannot set new_db_name option when only restoring system users") } newDBNameFn, err = p.TypeAsString(ctx, restoreStmt.Options.NewDBName, "RESTORE") @@ -1580,7 +1579,7 @@ func doRestorePlan( } sqlDescs, restoreDBs, tenants, err := selectTargets( - ctx, p, mainBackupManifests, restoreStmt.Targets, restoreStmt.DescriptorCoverage, endTime, restoreStmt.SystemUsers, + ctx, p, mainBackupManifests, restoreStmt.Targets, restoreStmt.DescriptorCoverage, endTime, ) if err != nil { return errors.Wrap(err, @@ -1733,8 +1732,7 @@ func doRestorePlan( restoreStmt.DescriptorCoverage, restoreStmt.Options, intoDB, - newDBName, - restoreStmt.SystemUsers) + newDBName) if err != nil { return err } @@ -1834,7 +1832,14 @@ func doRestorePlan( RevalidateIndexes: revalidateIndexes, DatabaseModifiers: databaseModifiers, DebugPauseOn: debugPauseOn, - RestoreSystemUsers: restoreStmt.SystemUsers, + + // A RESTORE SYSTEM USERS planned on a 22.1 node will use the + // RestoreSystemUsers field in the job details to identify this flavour of + // RESTORE. We must continue to check this field for mixed-version + // compatability. + // + // TODO(msbutler): Delete in 23.1 + RestoreSystemUsers: restoreStmt.DescriptorCoverage == tree.SystemUsers, PreRewriteTenantId: oldTenantID, Validation: jobspb.RestoreValidation_DefaultRestore, }, diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index c7bb1744dea6..a023e396f561 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -338,7 +338,6 @@ func selectTargets( targets tree.TargetList, descriptorCoverage tree.DescriptorCoverage, asOf hlc.Timestamp, - restoreSystemUsers bool, ) ([]catalog.Descriptor, []catalog.DatabaseDescriptor, []descpb.TenantInfoWithUsage, error) { allDescs, lastBackupManifest := loadSQLDescsFromBackupsAtTime(backupManifests, asOf) @@ -346,7 +345,7 @@ func selectTargets( return fullClusterTargetsRestore(allDescs, lastBackupManifest) } - if restoreSystemUsers { + if descriptorCoverage == tree.SystemUsers { systemTables := make([]catalog.Descriptor, 0) var users catalog.Descriptor for _, desc := range allDescs { diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 9732548c476f..8e0965539f20 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -329,6 +329,9 @@ message RestoreDetails { // DebugPauseOn describes the events that the job should pause itself on for debugging purposes. string debug_pause_on = 20; + + // RestoreSystemUsers is set to true if user runs RESTORE SYSTEM USERS. + // TODO(msbutler): delete in 23.1 bool restore_system_users = 22; // PreRewrittenTenantID is the ID of tenants[0] in the backup, aka its old ID; diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index fdda81278a7d..f9fb8da0a65a 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -3165,7 +3165,7 @@ restore_stmt: | RESTORE SYSTEM USERS FROM list_of_string_or_placeholder_opt_list opt_as_of_clause opt_with_restore_options { $$.val = &tree.Restore{ - SystemUsers: true, + DescriptorCoverage: tree.SystemUsers, From: $5.listOfStringOrPlaceholderOptList(), AsOf: $6.asOfClause(), Options: *($7.restoreOptions()), @@ -3174,7 +3174,7 @@ restore_stmt: | RESTORE SYSTEM USERS FROM string_or_placeholder IN list_of_string_or_placeholder_opt_list opt_as_of_clause opt_with_restore_options { $$.val = &tree.Restore{ - SystemUsers: true, + DescriptorCoverage: tree.SystemUsers, Subdir: $5.expr(), From: $7.listOfStringOrPlaceholderOptList(), AsOf: $8.asOfClause(), diff --git a/pkg/sql/sem/tree/backup.go b/pkg/sql/sem/tree/backup.go index e90bfa078080..79f1437c9973 100644 --- a/pkg/sql/sem/tree/backup.go +++ b/pkg/sql/sem/tree/backup.go @@ -15,9 +15,8 @@ import ( "github.com/google/go-cmp/cmp" ) -// DescriptorCoverage specifies whether or not a subset of descriptors were -// requested or if all the descriptors were requested, so all the descriptors -// are covered in a given backup. +// DescriptorCoverage specifies the subset of descriptors that are requested during a backup +// or a restore. type DescriptorCoverage int32 const ( @@ -28,10 +27,15 @@ const ( // backup is not said to have complete table coverage unless it was created // by a `BACKUP TO` command. RequestedDescriptors DescriptorCoverage = iota + // AllDescriptors table coverage means that backup is guaranteed to have all the // relevant data in the cluster. These can only be created by running a // full cluster backup with `BACKUP TO`. AllDescriptors + + // SystemUsers coverage indicates that only the system.users + // table will be restored from the backup. + SystemUsers ) // BackupOptions describes options for the BACKUP execution. @@ -139,9 +143,7 @@ var _ NodeFormatter = &RestoreOptions{} // Restore represents a RESTORE statement. type Restore struct { - Targets TargetList - // Whether this is the RESTORE SYSTEM USERS variant of RESTORE statement. - SystemUsers bool + Targets TargetList DescriptorCoverage DescriptorCoverage // From contains the URIs for the backup(s) we seek to restore. From 87e77619740aa6be30f434b2c852df9ea7d02133 Mon Sep 17 00:00:00 2001 From: Evan Wall Date: Thu, 9 Jun 2022 14:49:21 -0400 Subject: [PATCH 14/15] sql: support ttl_expiration_expression for row-level TTL refs https://github.com/cockroachdb/cockroach/issues/76916 Release note (sql change): Allow `CREATE TABLE ... WITH (ttl_expiration_expression='...')`. Allow `ALTER TABLE ... SET (ttl_expiration_expression='...')` and `ALTER TABLE ... RESET (ttl_expiration_expression)`. ttl_expiration_expression accepts an expression that returns a timestamp to support custom TTL calculation. --- pkg/sql/alter_table.go | 65 +++-- pkg/sql/catalog/catpb/catalog.go | 10 + pkg/sql/catalog/catpb/catalog.proto | 2 + pkg/sql/catalog/tabledesc/BUILD.bazel | 1 + pkg/sql/catalog/tabledesc/structured.go | 11 +- pkg/sql/catalog/tabledesc/ttl.go | 4 +- pkg/sql/catalog/tabledesc/validate.go | 50 ++-- pkg/sql/create_table.go | 4 +- .../testdata/logic_test/row_level_ttl | 273 +++++++++++++++++- .../tablestorageparam/BUILD.bazel | 1 + .../tablestorageparam/table_storage_param.go | 57 +++- pkg/sql/ttl/ttljob/ttljob.go | 17 +- pkg/sql/ttl/ttljob/ttljob_query_builder.go | 23 +- .../ttl/ttljob/ttljob_query_builder_test.go | 10 +- pkg/sql/ttl/ttljob/ttljob_test.go | 122 +++++--- 15 files changed, 522 insertions(+), 128 deletions(-) diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 4d3c7e36a875..787092680b01 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -1847,10 +1847,16 @@ func handleTTLStorageParamChange( tableDesc *tabledesc.Mutable, before, after *catpb.RowLevelTTL, ) error { - switch { - case before == nil && after == nil: - // Do not have to do anything here. - case before != nil && after != nil: + + if before == nil && after != nil { + if err := checkTTLEnabledForCluster(params.ctx, params.p.ExecCfg().Settings); err != nil { + return err + } + } + + // update existing config + if before != nil && after != nil { + // Update cron schedule if required. if before.DeletionCron != after.DeletionCron { env := JobSchedulerEnv(params.ExecCfg()) @@ -1871,8 +1877,9 @@ func handleTTLStorageParamChange( return err } } + // Update default expression on automated column if required. - if before.DurationExpr != after.DurationExpr { + if before.HasDurationExpr() && after.HasDurationExpr() && before.DurationExpr != after.DurationExpr { col, err := tableDesc.FindColumnWithName(colinfo.TTLDefaultExpirationColumnName) if err != nil { return err @@ -1905,11 +1912,10 @@ func handleTTLStorageParamChange( return err } } - case before == nil && after != nil: - if err := checkTTLEnabledForCluster(params.ctx, params.p.ExecCfg().Settings); err != nil { - return err - } + } + // create new column + if (before == nil || !before.HasDurationExpr()) && (after != nil && after.HasDurationExpr()) { // Adding a TTL requires adding the automatic column and deferring the TTL // addition to after the column is successfully added. tableDesc.RowLevelTTL = nil @@ -1949,28 +1955,33 @@ func handleTTLStorageParamChange( if err := tableDesc.AllocateIDs(params.ctx, version); err != nil { return err } - case before != nil && after == nil: + } + + // remove existing column + if (before != nil && before.HasDurationExpr()) && (after == nil || !after.HasDurationExpr()) { telemetry.Inc(sqltelemetry.RowLevelTTLDropped) - // Keep the TTL from beforehand, but create the DROP COLUMN job and the - // associated mutation. - tableDesc.RowLevelTTL = before + if before.HasDurationExpr() { + // Keep the TTL from beforehand, but create the DROP COLUMN job and the + // associated mutation. + tableDesc.RowLevelTTL = before - droppedViews, err := dropColumnImpl(params, tn, tableDesc, &tree.AlterTableDropColumn{ - Column: colinfo.TTLDefaultExpirationColumnName, - }) - if err != nil { - return err - } - // This should never happen as we do not CASCADE, but error again just in case. - if len(droppedViews) > 0 { - return pgerror.Newf(pgcode.InvalidParameterValue, "cannot drop TTL automatic column if it is depended on by a view") - } + droppedViews, err := dropColumnImpl(params, tn, tableDesc, &tree.AlterTableDropColumn{ + Column: colinfo.TTLDefaultExpirationColumnName, + }) + if err != nil { + return err + } + // This should never happen as we do not CASCADE, but error again just in case. + if len(droppedViews) > 0 { + return pgerror.Newf(pgcode.InvalidParameterValue, "cannot drop TTL automatic column if it is depended on by a view") + } - tableDesc.AddModifyRowLevelTTLMutation( - &descpb.ModifyRowLevelTTL{RowLevelTTL: before}, - descpb.DescriptorMutation_DROP, - ) + tableDesc.AddModifyRowLevelTTLMutation( + &descpb.ModifyRowLevelTTL{RowLevelTTL: before}, + descpb.DescriptorMutation_DROP, + ) + } } return nil diff --git a/pkg/sql/catalog/catpb/catalog.go b/pkg/sql/catalog/catpb/catalog.go index 07a3e71bba83..8e5c5be8857d 100644 --- a/pkg/sql/catalog/catpb/catalog.go +++ b/pkg/sql/catalog/catpb/catalog.go @@ -98,3 +98,13 @@ func (as *AutoStatsSettings) NoAutoStatsSettingsOverrides() bool { } return true } + +// HasDurationExpr is a utility method to determine if ttl_expires_after was set +func (rowLevelTTL *RowLevelTTL) HasDurationExpr() bool { + return rowLevelTTL.DurationExpr != "" +} + +// HasExpirationExpr is a utility method to determine if ttl_expiration_expression was set +func (rowLevelTTL *RowLevelTTL) HasExpirationExpr() bool { + return rowLevelTTL.ExpirationExpr != "" +} diff --git a/pkg/sql/catalog/catpb/catalog.proto b/pkg/sql/catalog/catpb/catalog.proto index 5954d1f7323d..405ec6e79435 100644 --- a/pkg/sql/catalog/catpb/catalog.proto +++ b/pkg/sql/catalog/catpb/catalog.proto @@ -216,6 +216,8 @@ message RowLevelTTL { // LabelMetrics is true if metrics for the TTL job should add a label containing // the relation name. optional bool label_metrics = 10 [(gogoproto.nullable) = false]; + // ExpirationExpr is the custom assigned expression for calculating when the TTL should apply to a row. + optional string expiration_expr = 11 [(gogoproto.nullable)=false, (gogoproto.casttype)="Expression"]; } // AutoStatsSettings represents settings related to automatic statistics diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 2a98395c2bea..e15e6c63297b 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "//pkg/sql/catalog/multiregion", "//pkg/sql/catalog/schemaexpr", "//pkg/sql/catalog/typedesc", + "//pkg/sql/lexbase", "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index a70ac7b6c380..7817a337263d 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/internal/validate" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" + "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -2577,8 +2578,14 @@ func (desc *wrapper) GetStorageParams(spaceBetweenEqual bool) []string { } if ttl := desc.GetRowLevelTTL(); ttl != nil { appendStorageParam(`ttl`, `'on'`) - appendStorageParam(`ttl_automatic_column`, `'on'`) - appendStorageParam(`ttl_expire_after`, string(ttl.DurationExpr)) + if ttl.HasDurationExpr() { + appendStorageParam(`ttl_automatic_column`, `'on'`) + appendStorageParam(`ttl_expire_after`, string(ttl.DurationExpr)) + } + if ttl.HasExpirationExpr() { + escapedTTLExpirationExpression := lexbase.EscapeSQLString(string(ttl.ExpirationExpr)) + appendStorageParam(`ttl_expiration_expression`, escapedTTLExpirationExpression) + } appendStorageParam(`ttl_job_cron`, fmt.Sprintf(`'%s'`, ttl.DeletionCronOrDefault())) if bs := ttl.SelectBatchSize; bs != 0 { appendStorageParam(`ttl_select_batch_size`, fmt.Sprintf(`%d`, bs)) diff --git a/pkg/sql/catalog/tabledesc/ttl.go b/pkg/sql/catalog/tabledesc/ttl.go index affced824e92..bc5b2b849059 100644 --- a/pkg/sql/catalog/tabledesc/ttl.go +++ b/pkg/sql/catalog/tabledesc/ttl.go @@ -24,10 +24,10 @@ func ValidateRowLevelTTL(ttl *catpb.RowLevelTTL) error { if ttl == nil { return nil } - if ttl.DurationExpr == "" { + if !ttl.HasDurationExpr() && !ttl.HasExpirationExpr() { return pgerror.Newf( pgcode.InvalidParameterValue, - `"ttl_expire_after" must be set`, + `"ttl_expire_after" and/or "ttl_expiration_expression" must be set`, ) } if ttl.DeleteBatchSize != 0 { diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 158cd49286d0..8e76250c3335 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -190,30 +190,40 @@ func (desc *wrapper) ValidateCrossReferences( // For row-level TTL, only ascending PKs are permitted. if desc.HasRowLevelTTL() { - pk := desc.GetPrimaryIndex() - if col, err := desc.FindColumnWithName(colinfo.TTLDefaultExpirationColumnName); err != nil { - vea.Report(errors.Wrapf(err, "expected column %s", colinfo.TTLDefaultExpirationColumnName)) - } else { - intervalExpr := desc.GetRowLevelTTL().DurationExpr - expectedStr := `current_timestamp():::TIMESTAMPTZ + ` + string(intervalExpr) - if col.GetDefaultExpr() != expectedStr { - vea.Report(pgerror.Newf( - pgcode.InvalidTableDefinition, - "expected DEFAULT expression of %s to be %s", - colinfo.TTLDefaultExpirationColumnName, - expectedStr, - )) + rowLevelTTL := desc.RowLevelTTL + if rowLevelTTL.HasDurationExpr() { + if col, err := desc.FindColumnWithName(colinfo.TTLDefaultExpirationColumnName); err != nil { + vea.Report(errors.Wrapf(err, "expected column %s", colinfo.TTLDefaultExpirationColumnName)) + } else { + intervalExpr := desc.GetRowLevelTTL().DurationExpr + expectedStr := `current_timestamp():::TIMESTAMPTZ + ` + string(intervalExpr) + if col.GetDefaultExpr() != expectedStr { + vea.Report(pgerror.Newf( + pgcode.InvalidTableDefinition, + "expected DEFAULT expression of %s to be %s", + colinfo.TTLDefaultExpirationColumnName, + expectedStr, + )) + } + if col.GetOnUpdateExpr() != expectedStr { + vea.Report(pgerror.Newf( + pgcode.InvalidTableDefinition, + "expected ON UPDATE expression of %s to be %s", + colinfo.TTLDefaultExpirationColumnName, + expectedStr, + )) + } } - if col.GetOnUpdateExpr() != expectedStr { - vea.Report(pgerror.Newf( - pgcode.InvalidTableDefinition, - "expected ON UPDATE expression of %s to be %s", - colinfo.TTLDefaultExpirationColumnName, - expectedStr, - )) + } + + if rowLevelTTL.HasExpirationExpr() { + _, err := parser.ParseExpr(string(rowLevelTTL.ExpirationExpr)) + if err != nil { + vea.Report(errors.Wrapf(err, "value of 'ttl_expiration_expression' must be a valid expression")) } } + pk := desc.GetPrimaryIndex() for i := 0; i < pk.NumKeyColumns(); i++ { dir := pk.GetKeyColumnDirection(i) if dir != catpb.IndexColumn_ASC { diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index c870461c6622..049b2865805d 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1472,8 +1472,8 @@ func NewTableDesc( } } - // Create the TTL column if one does not already exist. - if ttl := desc.GetRowLevelTTL(); ttl != nil { + // Create the TTL automatic column (crdb_internal_expiration) if one does not already exist. + if ttl := desc.GetRowLevelTTL(); ttl != nil && ttl.HasDurationExpr() { if err := checkTTLEnabledForCluster(ctx, st); err != nil { return nil, err } diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_ttl b/pkg/sql/logictest/testdata/logic_test/row_level_ttl index 21a505254d5a..6b1f1e95f6e9 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_ttl +++ b/pkg/sql/logictest/testdata/logic_test/row_level_ttl @@ -1,10 +1,18 @@ +subtest todo_add_subtests + statement error value of "ttl_expire_after" must be an interval CREATE TABLE tbl (id INT PRIMARY KEY, text TEXT) WITH (ttl_expire_after = ' xx invalid interval xx') statement error value of "ttl_expire_after" must be at least zero CREATE TABLE tbl (id INT PRIMARY KEY, text TEXT) WITH (ttl_expire_after = '-10 minutes') -statement error "ttl_expire_after" must be set +statement error parameter "ttl_expiration_expression" requires a string value +CREATE TABLE tbl (id INT PRIMARY KEY, text TEXT) WITH (ttl_expiration_expression = 0) + +statement error value of "ttl_expiration_expression" must be a valid expression: at or near "EOF": syntax error +CREATE TABLE tbl (id INT PRIMARY KEY, text TEXT) WITH (ttl_expiration_expression = '; DROP DATABASE defaultdb') + +statement error "ttl_expire_after" and/or "ttl_expiration_expression" must be set CREATE TABLE tbl (id INT PRIMARY KEY, text TEXT) WITH (ttl = 'on') statement error "ttl_expire_after" must be set if "ttl_automatic_column" is set @@ -53,7 +61,7 @@ SELECT crdb_internal.repair_ttl_table_scheduled_job('tbl'::regclass::oid) statement ok SELECT crdb_internal.validate_ttl_scheduled_jobs() -statement error resetting "ttl_expire_after" is not permitted\nHINT: use `RESET \(ttl\)` to remove TTL from the table +statement error "ttl_expire_after" and/or "ttl_expiration_expression" must be set ALTER TABLE tbl RESET (ttl_expire_after) statement error expected DEFAULT expression of crdb_internal_expiration to be current_timestamp\(\):::TIMESTAMPTZ \+ '00:10:00':::INTERVAL @@ -358,19 +366,19 @@ CREATE TABLE no_ttl_table (); statement error unsetting TTL automatic column not yet implemented ALTER TABLE no_ttl_table SET (ttl_automatic_column = 'off') -statement error "ttl_expire_after" must be set +statement error "ttl_expire_after" and/or "ttl_expiration_expression" must be set ALTER TABLE no_ttl_table SET (ttl_select_batch_size = 50) -statement error "ttl_expire_after" must be set +statement error "ttl_expire_after" and/or "ttl_expiration_expression" must be set ALTER TABLE no_ttl_table SET (ttl_delete_batch_size = 50) -statement error "ttl_expire_after" must be set +statement error "ttl_expire_after" and/or "ttl_expiration_expression" must be set ALTER TABLE no_ttl_table SET (ttl_job_cron = '@weekly') -statement error "ttl_expire_after" must be set +statement error "ttl_expire_after" and/or "ttl_expiration_expression" must be set ALTER TABLE no_ttl_table SET (ttl_pause = true) -statement error "ttl_expire_after" must be set +statement error "ttl_expire_after" and/or "ttl_expiration_expression" must be set ALTER TABLE no_ttl_table SET (ttl_label_metrics = true) statement ok @@ -442,6 +450,255 @@ CREATE TABLE public.tbl ( FAMILY fam_0_id_text_crdb_internal_expiration (id, text, crdb_internal_expiration) ) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '00:10:00':::INTERVAL, ttl_job_cron = '@hourly', ttl_label_metrics = true) +subtest end + +subtest create_table_ttl_expiration_expression + +statement ok +CREATE TABLE tbl_create_table_ttl_expiration_expression ( + id INT PRIMARY KEY, + expire_at TIMESTAMP, + FAMILY (id, expire_at) +) WITH (ttl_expiration_expression = 'expire_at') + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_create_table_ttl_expiration_expression] +---- +CREATE TABLE public.tbl_create_table_ttl_expiration_expression ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + CONSTRAINT tbl_create_table_ttl_expiration_expression_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at (id, expire_at) +) WITH (ttl = 'on', ttl_expiration_expression = 'expire_at', ttl_job_cron = '@hourly') + +statement ok +ALTER TABLE tbl_create_table_ttl_expiration_expression RESET (ttl) + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_create_table_ttl_expiration_expression] +---- +CREATE TABLE public.tbl_create_table_ttl_expiration_expression ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + CONSTRAINT tbl_create_table_ttl_expiration_expression_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at (id, expire_at) +) + +subtest end + +subtest create_table_ttl_expiration_expression_escape_sql + +statement ok +CREATE TABLE tbl_create_table_ttl_expiration_expression_escape_sql ( + id INT PRIMARY KEY, + expire_at TIMESTAMP, + FAMILY (id, expire_at) +) WITH (ttl_expiration_expression = 'IF(expire_at > ''2020-01-01 00:00:00'':::TIMESTAMP, expire_at, NULL)') + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_create_table_ttl_expiration_expression_escape_sql] +---- +CREATE TABLE public.tbl_create_table_ttl_expiration_expression_escape_sql ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + CONSTRAINT tbl_create_table_ttl_expiration_expression_escape_sql_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at (id, expire_at) +) WITH (ttl = 'on', ttl_expiration_expression = e'IF(expire_at > \'2020-01-01 00:00:00\':::TIMESTAMP, expire_at, NULL)', ttl_job_cron = '@hourly') + + +subtest end + +subtest alter_table_ttl_expiration_expression + +statement ok +CREATE TABLE tbl_alter_table_ttl_expiration_expression ( + id INT PRIMARY KEY, + expire_at TIMESTAMP, + FAMILY (id, expire_at) +) + +statement ok +ALTER TABLE tbl_alter_table_ttl_expiration_expression SET (ttl_expiration_expression = 'expire_at') + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_alter_table_ttl_expiration_expression] +---- +CREATE TABLE public.tbl_alter_table_ttl_expiration_expression ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + CONSTRAINT tbl_alter_table_ttl_expiration_expression_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at (id, expire_at) +) WITH (ttl = 'on', ttl_expiration_expression = 'expire_at', ttl_job_cron = '@hourly') + +# try setting it again +statement ok +ALTER TABLE tbl_alter_table_ttl_expiration_expression SET (ttl_expiration_expression = 'expire_at + ''5 minutes'':::INTERVAL') + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_alter_table_ttl_expiration_expression] +---- +CREATE TABLE public.tbl_alter_table_ttl_expiration_expression ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + CONSTRAINT tbl_alter_table_ttl_expiration_expression_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at (id, expire_at) +) WITH (ttl = 'on', ttl_expiration_expression = e'expire_at + \'5 minutes\':::INTERVAL', ttl_job_cron = '@hourly') + +statement ok +ALTER TABLE tbl_alter_table_ttl_expiration_expression RESET (ttl) + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_alter_table_ttl_expiration_expression] +---- +CREATE TABLE public.tbl_alter_table_ttl_expiration_expression ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + CONSTRAINT tbl_alter_table_ttl_expiration_expression_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at (id, expire_at) +) + +subtest end + +subtest add_ttl_expiration_expression_to_ttl_expire_after + +statement ok +CREATE TABLE tbl_add_ttl_expiration_expression_to_ttl_expire_after ( + id INT PRIMARY KEY, + expire_at TIMESTAMP, + FAMILY (id, expire_at) +) WITH (ttl_expire_after = '10 minutes') + +statement ok +ALTER TABLE tbl_add_ttl_expiration_expression_to_ttl_expire_after SET (ttl_expiration_expression = 'crdb_internal_expiration') + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_add_ttl_expiration_expression_to_ttl_expire_after] +---- +CREATE TABLE public.tbl_add_ttl_expiration_expression_to_ttl_expire_after ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + crdb_internal_expiration TIMESTAMPTZ NOT VISIBLE NOT NULL DEFAULT current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL, + CONSTRAINT tbl_add_ttl_expiration_expression_to_ttl_expire_after_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at_crdb_internal_expiration (id, expire_at, crdb_internal_expiration) +) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '00:10:00':::INTERVAL, ttl_expiration_expression = 'crdb_internal_expiration', ttl_job_cron = '@hourly') + +statement ok +ALTER TABLE tbl_add_ttl_expiration_expression_to_ttl_expire_after RESET (ttl_expiration_expression) + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_add_ttl_expiration_expression_to_ttl_expire_after] +---- +CREATE TABLE public.tbl_add_ttl_expiration_expression_to_ttl_expire_after ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + crdb_internal_expiration TIMESTAMPTZ NOT VISIBLE NOT NULL DEFAULT current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL, + CONSTRAINT tbl_add_ttl_expiration_expression_to_ttl_expire_after_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at_crdb_internal_expiration (id, expire_at, crdb_internal_expiration) +) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '00:10:00':::INTERVAL, ttl_job_cron = '@hourly') + +subtest end + +subtest add_ttl_expire_after_to_ttl_expiration_expression + +statement ok +CREATE TABLE tbl_add_ttl_expire_after_to_ttl_expiration_expression ( + id INT PRIMARY KEY, + expire_at TIMESTAMP, + FAMILY (id, expire_at) +) WITH (ttl_expiration_expression = 'expire_at') + +statement ok +ALTER TABLE tbl_add_ttl_expire_after_to_ttl_expiration_expression SET (ttl_expire_after = '10 minutes') + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_add_ttl_expire_after_to_ttl_expiration_expression] +---- +CREATE TABLE public.tbl_add_ttl_expire_after_to_ttl_expiration_expression ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + crdb_internal_expiration TIMESTAMPTZ NOT VISIBLE NOT NULL DEFAULT current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL, + CONSTRAINT tbl_add_ttl_expire_after_to_ttl_expiration_expression_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at (id, expire_at, crdb_internal_expiration) +) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '00:10:00':::INTERVAL, ttl_expiration_expression = 'expire_at', ttl_job_cron = '@hourly') + +statement ok +ALTER TABLE tbl_add_ttl_expire_after_to_ttl_expiration_expression RESET (ttl_expire_after) + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_add_ttl_expire_after_to_ttl_expiration_expression] +---- +CREATE TABLE public.tbl_add_ttl_expire_after_to_ttl_expiration_expression ( + id INT8 NOT NULL, + expire_at TIMESTAMP NULL, + CONSTRAINT tbl_add_ttl_expire_after_to_ttl_expiration_expression_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_expire_at (id, expire_at) +) + +subtest end + +subtest create_table_ttl_expire_after_and_ttl_expiration_expression + +statement ok +CREATE TABLE create_table_ttl_expire_after_and_ttl_expiration_expression ( + id INT PRIMARY KEY +) WITH (ttl_expire_after = '10 minutes', ttl_expiration_expression = 'crdb_internal_expiration') + +query T +SELECT create_statement FROM [SHOW CREATE TABLE create_table_ttl_expire_after_and_ttl_expiration_expression] +---- +CREATE TABLE public.create_table_ttl_expire_after_and_ttl_expiration_expression ( + id INT8 NOT NULL, + crdb_internal_expiration TIMESTAMPTZ NOT VISIBLE NOT NULL DEFAULT current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL, + CONSTRAINT create_table_ttl_expire_after_and_ttl_expiration_expression_pkey PRIMARY KEY (id ASC) +) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '00:10:00':::INTERVAL, ttl_expiration_expression = 'crdb_internal_expiration', ttl_job_cron = '@hourly') + +statement ok +ALTER TABLE create_table_ttl_expire_after_and_ttl_expiration_expression RESET (ttl) + +query T +SELECT create_statement FROM [SHOW CREATE TABLE create_table_ttl_expire_after_and_ttl_expiration_expression] +---- +CREATE TABLE public.create_table_ttl_expire_after_and_ttl_expiration_expression ( + id INT8 NOT NULL, + CONSTRAINT create_table_ttl_expire_after_and_ttl_expiration_expression_pkey PRIMARY KEY (id ASC) +) + +subtest end + +subtest alter_table_ttl_expire_after_and_ttl_expiration_expression + +statement ok +CREATE TABLE tbl_alter_table_ttl_expire_after_and_ttl_expiration_expression ( + id INT PRIMARY KEY +) + +statement ok +ALTER TABLE tbl_alter_table_ttl_expire_after_and_ttl_expiration_expression SET (ttl_expire_after = '10 minutes', ttl_expiration_expression = 'crdb_internal_expiration') + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_alter_table_ttl_expire_after_and_ttl_expiration_expression] +---- +CREATE TABLE public.tbl_alter_table_ttl_expire_after_and_ttl_expiration_expression ( + id INT8 NOT NULL, + crdb_internal_expiration TIMESTAMPTZ NOT VISIBLE NOT NULL DEFAULT current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL, + CONSTRAINT tbl_alter_table_ttl_expire_after_and_ttl_expiration_expression_pkey PRIMARY KEY (id ASC) +) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '00:10:00':::INTERVAL, ttl_expiration_expression = 'crdb_internal_expiration', ttl_job_cron = '@hourly') + +statement ok +ALTER TABLE tbl_alter_table_ttl_expire_after_and_ttl_expiration_expression RESET (ttl) + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl_alter_table_ttl_expire_after_and_ttl_expiration_expression] +---- +CREATE TABLE public.tbl_alter_table_ttl_expire_after_and_ttl_expiration_expression ( + id INT8 NOT NULL, + CONSTRAINT tbl_alter_table_ttl_expire_after_and_ttl_expiration_expression_pkey PRIMARY KEY (id ASC) +) + +subtest end + +subtest todo_add_subtests2 + # Test adding to TTL table with crdb_internal_expiration already defined. statement ok DROP TABLE tbl; @@ -588,3 +845,5 @@ ALTER TABLE test.public."Table-Name" WITH (ttl = 'on', ...) statement ok DROP TABLE "Table-Name" + +subtest end diff --git a/pkg/sql/storageparam/tablestorageparam/BUILD.bazel b/pkg/sql/storageparam/tablestorageparam/BUILD.bazel index 8970d6033e83..366235f8891c 100644 --- a/pkg/sql/storageparam/tablestorageparam/BUILD.bazel +++ b/pkg/sql/storageparam/tablestorageparam/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/tabledesc", "//pkg/sql/paramparse", + "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/pgwire/pgnotice", diff --git a/pkg/sql/storageparam/tablestorageparam/table_storage_param.go b/pkg/sql/storageparam/tablestorageparam/table_storage_param.go index 613e223d068f..d931cf37df5a 100644 --- a/pkg/sql/storageparam/tablestorageparam/table_storage_param.go +++ b/pkg/sql/storageparam/tablestorageparam/table_storage_param.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" + "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" @@ -47,7 +48,7 @@ func NewSetter(tableDesc *tabledesc.Mutable) *Setter { // RunPostChecks implements the Setter interface. func (po *Setter) RunPostChecks() error { ttl := po.tableDesc.GetRowLevelTTL() - if po.setAutomaticColumn && (ttl == nil || ttl.DurationExpr == "") { + if po.setAutomaticColumn && (ttl == nil || !ttl.HasDurationExpr()) { return pgerror.Newf( pgcode.InvalidParameterValue, `"ttl_expire_after" must be set if "ttl_automatic_column" is set`, @@ -197,10 +198,19 @@ var tableParams = map[string]tableParam{ var d *tree.DInterval if stringVal, err := paramparse.DatumAsString(evalCtx, key, datum); err == nil { d, err = tree.ParseDInterval(evalCtx.SessionData().GetIntervalStyle(), stringVal) - if err != nil || d == nil { + if err != nil { + return pgerror.Wrapf( + err, + pgcode.InvalidParameterValue, + `value of %q must be an interval`, + key, + ) + } + if d == nil { return pgerror.Newf( pgcode.InvalidParameterValue, - `value of "ttl_expire_after" must be an interval`, + `value of %q must be an interval`, + key, ) } } else { @@ -209,7 +219,7 @@ var tableParams = map[string]tableParam{ if !ok || d == nil { return pgerror.Newf( pgcode.InvalidParameterValue, - `value of "%s" must be an interval`, + `value of %q must be an interval`, key, ) } @@ -218,7 +228,7 @@ var tableParams = map[string]tableParam{ if d.Duration.Compare(duration.MakeDuration(0, 0, 0)) < 0 { return pgerror.Newf( pgcode.InvalidParameterValue, - `value of "%s" must be at least zero`, + `value of %q must be at least zero`, key, ) } @@ -227,13 +237,38 @@ var tableParams = map[string]tableParam{ return nil }, onReset: func(po *Setter, evalCtx *eval.Context, key string) error { - return errors.WithHintf( - pgerror.Newf( + if po.tableDesc.RowLevelTTL != nil { + po.tableDesc.RowLevelTTL.DurationExpr = "" + } + return nil + }, + }, + `ttl_expiration_expression`: { + onSet: func(ctx context.Context, po *Setter, semaCtx *tree.SemaContext, evalCtx *eval.Context, key string, datum tree.Datum) error { + stringVal, err := paramparse.DatumAsString(evalCtx, key, datum) + if err != nil { + return err + } + stringVal = strings.TrimSpace(stringVal) + // todo(wall): add type checking https://github.com/cockroachdb/cockroach/issues/76916 + _, err = parser.ParseExpr(stringVal) + if err != nil { + return pgerror.Wrapf( + err, pgcode.InvalidParameterValue, - `resetting "ttl_expire_after" is not permitted`, - ), - "use `RESET (ttl)` to remove TTL from the table", - ) + `value of %q must be a valid expression`, + key, + ) + } + rowLevelTTL := po.getOrCreateRowLevelTTL() + rowLevelTTL.ExpirationExpr = catpb.Expression(stringVal) + return nil + }, + onReset: func(po *Setter, evalCtx *eval.Context, key string) error { + if po.tableDesc.RowLevelTTL != nil { + po.tableDesc.RowLevelTTL.ExpirationExpr = "" + } + return nil }, }, `ttl_select_batch_size`: { diff --git a/pkg/sql/ttl/ttljob/ttljob.go b/pkg/sql/ttl/ttljob/ttljob.go index cc3f7e62ae26..b3c268a55d59 100644 --- a/pkg/sql/ttl/ttljob/ttljob.go +++ b/pkg/sql/ttl/ttljob/ttljob.go @@ -356,6 +356,11 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err deleteRateLimit, ) + ttlExpression := colinfo.TTLDefaultExpirationColumnName + if ttlSettings.HasExpirationExpr() { + ttlExpression = "(" + string(ttlSettings.ExpirationExpr) + ")" + } + statsCloseCh := make(chan struct{}) ch := make(chan rangeToProcess, rangeConcurrency) rowCount := int64(0) @@ -379,6 +384,7 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err deleteBatchSize, deleteRateLimiter, *aost, + ttlExpression, ) // add before returning err in case of partial success atomic.AddInt64(&rowCount, rangeRowCount) @@ -398,7 +404,7 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err if ttlSettings.RowStatsPollInterval != 0 { g.GoCtx(func(ctx context.Context) error { // Do once initially to ensure we have some base statistics. - fetchStatistics(ctx, p.ExecCfg(), knobs, relationName, details, metrics, aostDuration) + fetchStatistics(ctx, p.ExecCfg(), knobs, relationName, details, metrics, aostDuration, ttlExpression) // Wait until poll interval is reached, or early exit when we are done // with the TTL job. for { @@ -406,7 +412,7 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err case <-statsCloseCh: return nil case <-time.After(ttlSettings.RowStatsPollInterval): - fetchStatistics(ctx, p.ExecCfg(), knobs, relationName, details, metrics, aostDuration) + fetchStatistics(ctx, p.ExecCfg(), knobs, relationName, details, metrics, aostDuration, ttlExpression) } } }) @@ -521,12 +527,14 @@ func fetchStatistics( details jobspb.RowLevelTTLDetails, metrics rowLevelTTLMetrics, aostDuration time.Duration, + ttlExpression string, ) { if err := func() error { aost, err := tree.MakeDTimestampTZ(timeutil.Now().Add(aostDuration), time.Microsecond) if err != nil { return err } + for _, c := range []struct { opName string query string @@ -540,7 +548,7 @@ func fetchStatistics( }, { opName: fmt.Sprintf("ttl num expired rows stats %s", relationName), - query: `SELECT count(1) FROM [%d AS t] AS OF SYSTEM TIME %s WHERE ` + colinfo.TTLDefaultExpirationColumnName + ` < $1`, + query: `SELECT count(1) FROM [%d AS t] AS OF SYSTEM TIME %s WHERE ` + ttlExpression + ` < $1`, args: []interface{}{details.Cutoff}, gauge: metrics.TotalExpiredRows, }, @@ -590,6 +598,7 @@ func runTTLOnRange( selectBatchSize, deleteBatchSize int, deleteRateLimiter *quotapool.RateLimiter, aost tree.DTimestampTZ, + ttlExpression string, ) (rangeRowCount int64, err error) { metrics.NumActiveRanges.Inc(1) defer metrics.NumActiveRanges.Dec(1) @@ -609,6 +618,7 @@ func runTTLOnRange( endPK, aost, selectBatchSize, + ttlExpression, ) deleteBuilder := makeDeleteQueryBuilder( details.TableID, @@ -616,6 +626,7 @@ func runTTLOnRange( pkColumns, relationName, deleteBatchSize, + ttlExpression, ) for { diff --git a/pkg/sql/ttl/ttljob/ttljob_query_builder.go b/pkg/sql/ttl/ttljob/ttljob_query_builder.go index eb8e32a5eb94..c4b1e518c5b2 100644 --- a/pkg/sql/ttl/ttljob/ttljob_query_builder.go +++ b/pkg/sql/ttl/ttljob/ttljob_query_builder.go @@ -36,6 +36,7 @@ type selectQueryBuilder struct { startPK, endPK tree.Datums selectBatchSize int aost tree.DTimestampTZ + ttlExpression string // isFirst is true if we have not invoked a query using the builder yet. isFirst bool @@ -59,6 +60,7 @@ func makeSelectQueryBuilder( startPK, endPK tree.Datums, aost tree.DTimestampTZ, selectBatchSize int, + ttlExpression string, ) selectQueryBuilder { // We will have a maximum of 1 + len(pkColumns)*2 columns, where one // is reserved for AOST, and len(pkColumns) for both start and end key. @@ -79,6 +81,7 @@ func makeSelectQueryBuilder( endPK: endPK, aost: aost, selectBatchSize: selectBatchSize, + ttlExpression: ttlExpression, cachedArgs: cachedArgs, isFirst: true, @@ -133,12 +136,13 @@ func (b *selectQueryBuilder) buildQuery() string { return fmt.Sprintf( `SELECT %[1]s FROM [%[2]d AS tbl_name] AS OF SYSTEM TIME %[3]s -WHERE crdb_internal_expiration <= $1%[4]s%[5]s +WHERE %[4]s <= $1%[5]s%[6]s ORDER BY %[1]s -LIMIT %[6]d`, +LIMIT %[7]d`, b.pkColumnNamesSQL, b.tableID, b.aost.String(), + b.ttlExpression, filterClause, endFilterClause, b.selectBatchSize, @@ -210,6 +214,7 @@ type deleteQueryBuilder struct { pkColumns []string deleteBatchSize int deleteOpName string + ttlExpression string // cachedQuery is the cached query, which stays the same as long as we are // deleting up to deleteBatchSize elements. @@ -220,7 +225,12 @@ type deleteQueryBuilder struct { } func makeDeleteQueryBuilder( - tableID descpb.ID, cutoff time.Time, pkColumns []string, relationName string, deleteBatchSize int, + tableID descpb.ID, + cutoff time.Time, + pkColumns []string, + relationName string, + deleteBatchSize int, + ttlExpression string, ) deleteQueryBuilder { cachedArgs := make([]interface{}, 0, 1+len(pkColumns)*deleteBatchSize) cachedArgs = append(cachedArgs, cutoff) @@ -230,8 +240,8 @@ func makeDeleteQueryBuilder( pkColumns: pkColumns, deleteBatchSize: deleteBatchSize, deleteOpName: fmt.Sprintf("ttl delete %s", relationName), - - cachedArgs: cachedArgs, + ttlExpression: ttlExpression, + cachedArgs: cachedArgs, } } @@ -253,8 +263,9 @@ func (b *deleteQueryBuilder) buildQuery(numRows int) string { } return fmt.Sprintf( - `DELETE FROM [%d AS tbl_name] WHERE crdb_internal_expiration <= $1 AND (%s) IN (%s)`, + `DELETE FROM [%d AS tbl_name] WHERE %s <= $1 AND (%s) IN (%s)`, b.tableID, + b.ttlExpression, columnNamesSQL, placeholderStr, ) diff --git a/pkg/sql/ttl/ttljob/ttljob_query_builder_test.go b/pkg/sql/ttl/ttljob/ttljob_query_builder_test.go index 2a6e1e07d707..b2b586462b62 100644 --- a/pkg/sql/ttl/ttljob/ttljob_query_builder_test.go +++ b/pkg/sql/ttl/ttljob/ttljob_query_builder_test.go @@ -14,6 +14,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -49,6 +50,7 @@ func TestSelectQueryBuilder(t *testing.T) { tree.Datums{tree.NewDInt(200), tree.NewDInt(15)}, *mockTimestampTZ, 2, + colinfo.TTLDefaultExpirationColumnName, ), iterations: []iteration{ { @@ -109,6 +111,7 @@ LIMIT 2`, nil, *mockTimestampTZ, 2, + colinfo.TTLDefaultExpirationColumnName, ), iterations: []iteration{ { @@ -165,6 +168,7 @@ LIMIT 2`, tree.Datums{tree.NewDInt(181)}, *mockTimestampTZ, 2, + colinfo.TTLDefaultExpirationColumnName, ), iterations: []iteration{ { @@ -225,6 +229,7 @@ LIMIT 2`, tree.Datums{tree.NewDInt(200), tree.NewDInt(15)}, *mockTimestampTZ, 2, + colinfo.TTLDefaultExpirationColumnName, ), iterations: []iteration{ { @@ -284,6 +289,7 @@ LIMIT 2`, nil, *mockTimestampTZ, 2, + colinfo.TTLDefaultExpirationColumnName, ), iterations: []iteration{ { @@ -366,7 +372,7 @@ func TestDeleteQueryBuilder(t *testing.T) { }{ { desc: "single delete less than batch size", - b: makeDeleteQueryBuilder(1, mockTime, []string{"col1", "col2"}, "table_name", 3), + b: makeDeleteQueryBuilder(1, mockTime, []string{"col1", "col2"}, "table_name", 3, colinfo.TTLDefaultExpirationColumnName), iterations: []iteration{ { rows: []tree.Datums{ @@ -384,7 +390,7 @@ func TestDeleteQueryBuilder(t *testing.T) { }, { desc: "multiple deletes", - b: makeDeleteQueryBuilder(1, mockTime, []string{"col1", "col2"}, "table_name", 3), + b: makeDeleteQueryBuilder(1, mockTime, []string{"col1", "col2"}, "table_name", 3, colinfo.TTLDefaultExpirationColumnName), iterations: []iteration{ { rows: []tree.Datums{ diff --git a/pkg/sql/ttl/ttljob/ttljob_test.go b/pkg/sql/ttl/ttljob/ttljob_test.go index b389e149814d..9a28b5710d51 100644 --- a/pkg/sql/ttl/ttljob/ttljob_test.go +++ b/pkg/sql/ttl/ttljob/ttljob_test.go @@ -323,14 +323,16 @@ func TestRowLevelTTLJobRandomEntries(t *testing.T) { } type testCase struct { - desc string - createTable string - preSetup []string - postSetup []string - numExpiredRows int - numNonExpiredRows int - numSplits int - forceNonMultiTenant bool + desc string + createTable string + preSetup []string + postSetup []string + numExpiredRows int + numNonExpiredRows int + numSplits int + forceNonMultiTenant bool + expirationExpression string + addRow func(th *rowLevelTTLTestJobTestHelper, createTableStmt *tree.CreateTable, ts time.Time) } // Add some basic one and three column row-level TTL tests. testCases := []testCase{ @@ -443,6 +445,23 @@ func TestRowLevelTTLJobRandomEntries(t *testing.T) { numNonExpiredRows: 5, numSplits: 10, }, + { + desc: "ttl expiration expression", + createTable: `CREATE TABLE tbl ( + id UUID PRIMARY KEY DEFAULT gen_random_uuid(), + expire_at TIMESTAMP +) WITH (ttl_expiration_expression = 'expire_at')`, + numExpiredRows: 1001, + numNonExpiredRows: 5, + expirationExpression: "expire_at", + addRow: func(th *rowLevelTTLTestJobTestHelper, createTableStmt *tree.CreateTable, ts time.Time) { + th.sqlDB.Exec( + t, + "INSERT INTO tbl (expire_at) VALUES ($1)", + ts, + ) + }, + }, } // Also randomly generate random PKs. for i := 0; i < 5; i++ { @@ -471,6 +490,40 @@ func TestRowLevelTTLJobRandomEntries(t *testing.T) { ) } + defaultAddRow := func(th *rowLevelTTLTestJobTestHelper, createTableStmt *tree.CreateTable, ts time.Time) { + insertColumns := []string{"crdb_internal_expiration"} + placeholders := []string{"$1"} + values := []interface{}{ts} + + for _, def := range createTableStmt.Defs { + if def, ok := def.(*tree.ColumnTableDef); ok { + if def.HasDefaultExpr() { + continue + } + placeholders = append(placeholders, fmt.Sprintf("$%d", len(placeholders)+1)) + var b bytes.Buffer + lexbase.EncodeRestrictedSQLIdent(&b, string(def.Name), lexbase.EncNoFlags) + insertColumns = append(insertColumns, b.String()) + + d := randgen.RandDatum(rng, def.Type.(*types.T), false /* nullOk */) + f := tree.NewFmtCtx(tree.FmtBareStrings) + d.Format(f) + values = append(values, f.CloseAndGetString()) + } + } + + th.sqlDB.Exec( + t, + fmt.Sprintf( + "INSERT INTO %s (%s) VALUES (%s)", + createTableStmt.Table.Table(), + strings.Join(insertColumns, ","), + strings.Join(placeholders, ","), + ), + values..., + ) + } + for _, tc := range testCases { t.Run(tc.desc, func(t *testing.T) { // Log to make it slightly easier to reproduce a random config. @@ -502,40 +555,6 @@ func TestRowLevelTTLJobRandomEntries(t *testing.T) { createTableStmt, ok := stmt.AST.(*tree.CreateTable) require.True(t, ok) - addRow := func(ts time.Time) { - insertColumns := []string{"crdb_internal_expiration"} - placeholders := []string{"$1"} - values := []interface{}{ts} - - for _, def := range createTableStmt.Defs { - if def, ok := def.(*tree.ColumnTableDef); ok { - if def.HasDefaultExpr() { - continue - } - placeholders = append(placeholders, fmt.Sprintf("$%d", len(placeholders)+1)) - var b bytes.Buffer - lexbase.EncodeRestrictedSQLIdent(&b, string(def.Name), lexbase.EncNoFlags) - insertColumns = append(insertColumns, b.String()) - - d := randgen.RandDatum(rng, def.Type.(*types.T), false /* nullOk */) - f := tree.NewFmtCtx(tree.FmtBareStrings) - d.Format(f) - values = append(values, f.CloseAndGetString()) - } - } - - th.sqlDB.Exec( - t, - fmt.Sprintf( - "INSERT INTO %s (%s) VALUES (%s)", - createTableStmt.Table.Table(), - strings.Join(insertColumns, ","), - strings.Join(placeholders, ","), - ), - values..., - ) - } - // Split the ranges by a random PK value. if tc.numSplits > 0 { tbDesc := desctestutils.TestingGetPublicTableDescriptor( @@ -574,12 +593,17 @@ func TestRowLevelTTLJobRandomEntries(t *testing.T) { } } + addRow := defaultAddRow + if tc.addRow != nil { + addRow = tc.addRow + } + // Add expired and non-expired rows. for i := 0; i < tc.numExpiredRows; i++ { - addRow(timeutil.Now().Add(-time.Hour)) + addRow(th, createTableStmt, timeutil.Now().Add(-time.Hour)) } for i := 0; i < tc.numNonExpiredRows; i++ { - addRow(timeutil.Now().Add(time.Hour * 24 * 30)) + addRow(th, createTableStmt, timeutil.Now().Add(time.Hour*24*30)) } for _, stmt := range tc.postSetup { @@ -593,18 +617,24 @@ func TestRowLevelTTLJobRandomEntries(t *testing.T) { th.waitForSuccessfulScheduledJob(t) + table := createTableStmt.Table.Table() + // Check we have the number of expected rows. var numRows int th.sqlDB.QueryRow( t, - fmt.Sprintf(`SELECT count(1) FROM %s`, createTableStmt.Table.Table()), + fmt.Sprintf(`SELECT count(1) FROM %s`, table), ).Scan(&numRows) require.Equal(t, tc.numNonExpiredRows, numRows) // Also check all the rows expire way into the future. + expirationExpression := "crdb_internal_expiration" + if tc.expirationExpression != "" { + expirationExpression = tc.expirationExpression + } th.sqlDB.QueryRow( t, - fmt.Sprintf(`SELECT count(1) FROM %s WHERE crdb_internal_expiration >= now()`, createTableStmt.Table.Table()), + fmt.Sprintf(`SELECT count(1) FROM %s WHERE %s >= now()`, table, expirationExpression), ).Scan(&numRows) require.Equal(t, tc.numNonExpiredRows, numRows) From 05d697aa2210d4af8a3481a0089d7c86fe90bda7 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Wed, 29 Jun 2022 11:13:40 -0400 Subject: [PATCH 15/15] tree: DROP ROLE should not implement CCLOnlyStatement This was moved out of CCL licensing a few releases ago. Release note: None --- pkg/sql/sem/tree/stmt.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 567c56af126f..9c703d9bd973 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -842,8 +842,6 @@ func (*DropRole) StatementType() StatementType { return TypeDDL } // StatementTag returns a short string identifying the type of statement. func (*DropRole) StatementTag() string { return "DROP ROLE" } -func (*DropRole) cclOnlyStatement() {} - func (*DropRole) hiddenFromShowQueries() {} // StatementReturnType implements the Statement interface.