Skip to content

Commit 45f2866

Browse files
craig[bot]golgeekspilchenmgartner
committed
148514: roachprod: gc ibm cloud r=DarrylWong,herkolategan a=golgeek The roachprod GC job had not been updated since we started running automated tests on IBM Cloud. This patch updates the job to the latest release and makes it aware of IBM Cloud credentials. Epic: CRDB-21133 Fixes: #143789 Release note: None 148914: sql/ttl: centralize row-level TTL progress updates at coordinator r=spilchen a=spilchen Previously, each processor in a row-level TTL job directly updated job progress, leading to fan-out issues in large clusters and performance hits during restarts when many spans had no expired rows. Gating logic was previously added to mitigate this. This change delegates all progress updates to the coordinator. Each processor now sends its progress to the coordinator, which performs the job progress update. The original gating logic remains but is now enforced by the coordinator. Closes: #135227 Epic: none Release note: none 149784: cli/debug: strip comments in statement bundle env.sql file r=mgartner a=mgartner SQL comments in statement bundle `env.sql` files are now ignored with the `cockroach debug sb recreate` command. Release note: None 149820: opt: fix sysbench-update-non-index benchmark r=mgartner a=mgartner The `sysbench-update-non-index` optimizer benchmark would previously fail with the error `variable sub-expressions are not allowed in...` because the placeholder string value was not correctly quoted. This has been fixed. Release note: None Co-authored-by: Ludovic Leroux <[email protected]> Co-authored-by: Matt Spilchen <[email protected]> Co-authored-by: Marcus Gartner <[email protected]>
5 parents d210117 + fcd9d58 + c0a4b84 + 89eff0e + 9e64fa3 commit 45f2866

File tree

16 files changed

+1062
-329
lines changed

16 files changed

+1062
-329
lines changed

pkg/cli/statement_bundle.go

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@ import (
1616
"os"
1717
"path/filepath"
1818
"regexp"
19+
"slices"
1920
"sort"
2021
"strconv"
2122
"strings"
@@ -54,8 +55,11 @@ and stats in an unzipped statement bundle directory.
5455
Args: cobra.ExactArgs(1),
5556
}
5657

57-
var placeholderPairs []string
58-
var explainPrefix string
58+
var (
59+
placeholderPairs []string
60+
explainPrefix string
61+
commentPattern = regexp.MustCompile(`^\s*--`)
62+
)
5963

6064
func init() {
6165
statementBundleRecreateCmd.RunE = clierrorplus.MaybeDecorateError(runBundleRecreate)
@@ -169,8 +173,10 @@ func runBundleRecreate(cmd *cobra.Command, args []string) (resErr error) {
169173
return runDemoInternal(cmd, nil /* gen */, func(ctx context.Context, conn clisqlclient.Conn) error {
170174
// SET CLUSTER SETTING statements cannot be executed in multi-statement
171175
// implicit transaction, so we need to separate them out into their own
172-
// implicit transactions.
173-
initStmts := strings.Split(string(bundle.env), "SET CLUSTER SETTING")
176+
// implicit transactions. Comments are stripped from the env file first.
177+
lines := strings.Split(string(bundle.env), "\n")
178+
lines = slices.DeleteFunc(lines, commentPattern.MatchString)
179+
initStmts := strings.Split(strings.Join(lines, "\n"), "SET CLUSTER SETTING")
174180
for i := 1; i < len(initStmts); i++ {
175181
initStmts[i] = "SET CLUSTER SETTING " + initStmts[i]
176182
}

pkg/jobs/ingeststopped/ingesting_checker.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -92,7 +92,7 @@ func checkAllNodesForIngestingJob(
9292
)
9393
sql.FinalizePlan(ctx, planCtx, p)
9494

95-
res := sql.NewMetadataOnlyMetadataCallbackWriter()
95+
res := sql.NewMetadataOnlyMetadataCallbackWriter(func(context.Context, *execinfrapb.ProducerMetadata) error { return nil })
9696

9797
recv := sql.MakeDistSQLReceiver(
9898
ctx,

pkg/jobs/jobspb/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ go_library(
2323
"//pkg/util/iterutil",
2424
"//pkg/util/tracing/tracingpb",
2525
"@com_github_cockroachdb_errors//:errors",
26+
"@com_github_cockroachdb_redact//:redact",
2627
"@com_github_gogo_protobuf//jsonpb",
2728
],
2829
)

pkg/jobs/jobspb/jobs.go

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@ import (
1616
"github.com/cockroachdb/cockroach/pkg/util/hlc"
1717
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
1818
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
19+
"github.com/cockroachdb/redact"
1920
)
2021

2122
// JobID is the ID of a job.
@@ -164,3 +165,10 @@ func (b *BackupEncryptionOptions) IsEncrypted() bool {
164165
// For dumb reasons, there are two ways to represent no encryption.
165166
return !(b == nil || b.Mode == EncryptionMode_None)
166167
}
168+
169+
var _ redact.SafeFormatter = (*RowLevelTTLProcessorProgress)(nil)
170+
171+
// SafeFormat implements the redact.SafeFormatter interface.
172+
func (r *RowLevelTTLProcessorProgress) SafeFormat(p redact.SafePrinter, _ rune) {
173+
p.SafeString(redact.SafeString(r.String()))
174+
}

pkg/jobs/jobspb/jobs.proto

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1336,14 +1336,19 @@ message RowLevelTTLProcessorProgress {
13361336
(gogoproto.nullable) = false
13371337
];
13381338

1339-
// ProcessorRowCount is the row count of the DistSQL processor.
1340-
int64 processor_row_count = 3;
1339+
// DeletedRowCount is the number of rows deleted by this DistSQL processor.
1340+
int64 deleted_row_count = 3;
13411341

1342-
// ProcessorSpanCount is the number of spans of the DistSQL processor;
1343-
int64 processor_span_count = 4;
1342+
// TotalSpanCount is the total number of spans assigned to the DistSQL processor.
1343+
int64 total_span_count = 4;
1344+
1345+
// ProcessedSpanCount is the number of spans already processed.
1346+
int64 processed_span_count = 6;
13441347

13451348
// ProcessorConcurrency is the number parallel tasks the processor will do at once.
13461349
int64 processor_concurrency = 5;
1350+
1351+
// NEXT ID: 7
13471352
}
13481353

13491354
message SchemaTelemetryDetails {

pkg/roachprod/k8s/roachprod-gc.yaml

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ spec:
3131
spec:
3232
containers:
3333
- name: roachprod-gc-cronjob
34-
image: gcr.io/cockroach-dev-inf/cockroachlabs/roachprod:13afd784242
34+
image: gcr.io/cockroach-dev-inf/cockroachlabs/roachprod:fd18c1bd63d
3535
args:
3636
- gc
3737
- --gce-project=cockroach-ephemeral,cockroach-roachstress
@@ -40,6 +40,11 @@ spec:
4040
- --aws-account-ids=541263489771,337380398238
4141
- --azure-subscription-names=e2e-adhoc,e2e-infra,Microsoft Azure Sponsorship
4242
env:
43+
- name: IBM_APIKEY
44+
valueFrom:
45+
secretKeyRef:
46+
name: roachprod-gc-cronjob-creds
47+
key: ibm_apikey
4348
- name: SLACK_TOKEN
4449
valueFrom:
4550
secretKeyRef:

pkg/sql/distsql_running.go

Lines changed: 10 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@ import (
1818
"unsafe"
1919

2020
"github.com/cockroachdb/cockroach/pkg/base"
21+
"github.com/cockroachdb/cockroach/pkg/clusterversion"
2122
"github.com/cockroachdb/cockroach/pkg/col/coldata"
2223
"github.com/cockroachdb/cockroach/pkg/kv"
2324
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache"
@@ -456,9 +457,13 @@ func (dsp *DistSQLPlanner) setupFlows(
456457
if len(statementSQL) > setupFlowRequestStmtMaxLength {
457458
statementSQL = statementSQL[:setupFlowRequestStmtMaxLength]
458459
}
460+
v := execversion.V25_2
461+
if dsp.st.Version.IsActive(ctx, clusterversion.V25_4) {
462+
v = execversion.V25_4
463+
}
459464
setupReq := execinfrapb.SetupFlowRequest{
460465
LeafTxnInputState: leafInputState,
461-
Version: execversion.V25_2,
466+
Version: v,
462467
TraceKV: recv.tracing.KVTracingEnabled(),
463468
CollectStats: planCtx.collectExecStats,
464469
StatementSQL: statementSQL,
@@ -1163,12 +1168,12 @@ func NewMetadataCallbackWriter(
11631168
// NewMetadataOnlyMetadataCallbackWriter creates a new MetadataCallbackWriter
11641169
// that uses errOnlyResultWriter and only supports receiving
11651170
// execinfrapb.ProducerMetadata.
1166-
func NewMetadataOnlyMetadataCallbackWriter() *MetadataCallbackWriter {
1171+
func NewMetadataOnlyMetadataCallbackWriter(
1172+
metaFn func(ctx context.Context, meta *execinfrapb.ProducerMetadata) error,
1173+
) *MetadataCallbackWriter {
11671174
return NewMetadataCallbackWriter(
11681175
&errOnlyResultWriter{},
1169-
func(ctx context.Context, meta *execinfrapb.ProducerMetadata) error {
1170-
return nil
1171-
},
1176+
metaFn,
11721177
)
11731178
}
11741179

pkg/sql/execversion/version.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,12 +21,16 @@ type V uint32
2121
// only be used by the flows once the cluster has upgraded to 25.2.
2222
const V25_2 = V(73)
2323

24+
// V25_4 is the exec version of all binaries of 25.4 cockroach versions. It can
25+
// only be used by the flows once the cluster has upgraded to 25.4.
26+
const V25_4 = V(74)
27+
2428
// MinAccepted is the oldest version that the server is compatible with. A
2529
// server will not accept flows with older versions.
2630
const MinAccepted = V25_2
2731

2832
// Latest is the latest exec version supported by this binary.
29-
const Latest = V25_2
33+
const Latest = V25_4
3034

3135
var contextVersionKey = ctxutil.RegisterFastValueKey()
3236

pkg/sql/inspect/inspect_job.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -164,7 +164,8 @@ func (c *inspectResumer) runInspectPlan(
164164
) error {
165165
execCfg := jobExecCtx.ExecCfg()
166166

167-
metadataCallbackWriter := sql.NewMetadataOnlyMetadataCallbackWriter()
167+
metadataCallbackWriter := sql.NewMetadataOnlyMetadataCallbackWriter(
168+
func(context.Context, *execinfrapb.ProducerMetadata) error { return nil })
168169

169170
distSQLReceiver := sql.MakeDistSQLReceiver(
170171
ctx,

pkg/sql/opt/bench/bench_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -407,7 +407,7 @@ var queries = [...]benchQuery{
407407
{
408408
name: "sysbench-update-non-index",
409409
query: `UPDATE sbtest SET c=$2 WHERE id=$1`,
410-
args: []interface{}{10, "foo"},
410+
args: []interface{}{10, "'foo'"},
411411
},
412412

413413
// 1. Table with many columns.

0 commit comments

Comments
 (0)