Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
119481: telemetry: don't resample transaction for telemetry on restart r=xinhaoz a=xinhaoz

### telemetry: don't resample transaction for telemetry on restart

We will only attempt to sample the transaction once at the
start of its execution.

Additional changes:
- Move the sampling decision for transactions to be closer to section
updating `extraTxnState`
- Use the concrete statement types in `shouldForceLogStatement` to
prevent force logging on BEGIN and COMMIT statements.

Epic: none
Fixes: #119284

Release note: None

### eventpbgen: rename Nullable property to NotNullable

Let's rename the `Nullable` field to `NotNullable` for the
`fieldInfo` struct used in generating json encoding functions
for protobuf definitons. Note this field currently only applies to
`nestedMessage` types.

Epic: none

Release note: None

119893: lint: various fixes and improvements r=rail a=rickystewart

1. Use code generation to remove the `lint` build tag for running lints.
   On the Bazel side, you're not likely to accidentally run these tests
   and build tags just slow things down.
2. Fix the script for the nightly lint to exit with the correct status
   code and generate code before running.
3. Fix some lint errors in `lint_test.go` itself.

Epic: none
Release note: None

119951: sessiondatapb: fix build failures under `race` r=rail a=rickystewart

Regressed in #118546

Closes #119950

Epic: none
Release note: None

Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com>
Co-authored-by: Ricky Stewart <ricky@cockroachlabs.com>
  • Loading branch information
3 people committed Mar 5, 2024
4 parents 7345fae + e6f7cee + feba0c7 + ce18529 commit 3e61bd3
Show file tree
Hide file tree
Showing 10 changed files with 89 additions and 42 deletions.
2 changes: 0 additions & 2 deletions build/teamcity/cockroach/nightlies/lint_urls.sh
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,5 @@ dir="$(dirname $(dirname $(dirname $(dirname "${0}"))))"
source "$dir/teamcity-support.sh" # For $root
source "$dir/teamcity-bazel-support.sh" # For run_bazel

tc_start_block "lint urls"
BAZEL_SUPPORT_EXTRA_DOCKER_ARGS="-e BUILD_VCS_NUMBER -e GITHUB_API_TOKEN -e GITHUB_REPO -e TC_BUILDTYPE_ID -e TC_BUILD_BRANCH -e TC_BUILD_ID -e TC_SERVER_URL" \
run_bazel build/teamcity/cockroach/nightlies/lint_urls_impl.sh
tc_end_block "lint urls"
9 changes: 8 additions & 1 deletion build/teamcity/cockroach/nightlies/lint_urls_impl.sh
Original file line number Diff line number Diff line change
Expand Up @@ -4,12 +4,19 @@ set -xeuo pipefail

dir="$(dirname $(dirname $(dirname $(dirname "${0}"))))"

# GCAssert and unused need generated files in the workspace to work properly.
# generated files requirements -- begin
bazel run //pkg/gen:code
bazel run //pkg/cmd/generate-cgo:generate-cgo --run_under="cd $(bazel info workspace) && "
# generated files requirements -- end

bazel build //pkg/cmd/bazci --config=ci
BAZEL_BIN=$(bazel info bazel-bin --config=ci)
exit_status=0
XML_OUTPUT_FILE=/artifacts/test.xml GO_TEST_WRAP_TESTV=1 GO_TEST_WRAP=1 bazel \
run --config=ci --config=test --define gotags=bazel,gss,nightly \
run --config=ci --config=test --define gotags=bazel,gss,nightly,lint \
//build/bazelutil:lint || exit_status=$?
# The schema of the output test.xml will be slightly wrong -- ask `bazci` to fix
# it up.
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci munge-test-xml /artifacts/test.xml
exit $exit_status
5 changes: 0 additions & 5 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2038,11 +2038,6 @@ func (ex *connExecutor) resetExtraTxnState(ctx context.Context, ev txnEvent, pay
ex.state.mu.Lock()
defer ex.state.mu.Unlock()
ex.state.mu.stmtCount = 0
isTracing := ex.planner.ExtendedEvalContext().Tracing.Enabled()
ex.extraTxnState.shouldLogToTelemetry, ex.extraTxnState.telemetrySkippedTxns =
ex.server.TelemetryLoggingMetrics.shouldEmitTransactionLog(isTracing,
ex.executorType == executorTypeInternal,
ex.applicationName.Load().(string))
}

// NOTE: on txnRestart we don't need to muck with the savepoints stack. It's either a
Expand Down
13 changes: 7 additions & 6 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -3231,12 +3231,6 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) {
TxnFingerprintID: appstatspb.InvalidTransactionFingerprintID,
})

isTracing := ex.planner.ExtendedEvalContext().Tracing.Enabled()
ex.extraTxnState.shouldLogToTelemetry, ex.extraTxnState.telemetrySkippedTxns =
ex.server.TelemetryLoggingMetrics.shouldEmitTransactionLog(isTracing,
ex.executorType == executorTypeInternal,
ex.applicationName.Load().(string))

ex.state.mu.RLock()
txnStart := ex.state.mu.txnStart
ex.state.mu.RUnlock()
Expand Down Expand Up @@ -3269,6 +3263,13 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) {
ex.extraTxnState.txnFinishClosure.implicit = ex.implicitTxn()
ex.extraTxnState.shouldExecuteOnTxnRestart = true

// Determine telemetry logging.
isTracing := ex.planner.ExtendedEvalContext().Tracing.Enabled()
ex.extraTxnState.shouldLogToTelemetry, ex.extraTxnState.telemetrySkippedTxns =
ex.server.TelemetryLoggingMetrics.shouldEmitTransactionLog(isTracing,
ex.executorType == executorTypeInternal,
ex.applicationName.Load().(string))

ex.statsCollector.StartTransaction()

ex.previousTransactionCommitTimestamp = hlc.Timestamp{}
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/exec_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,8 +113,8 @@ const (
// TELEMETRY. Currently the criteria is if the statement is not of type DML and is
// not BEGIN or COMMIT.
func shouldForceLogStatement(ast tree.Statement) bool {
switch ast.StatementTag() {
case "BEGIN", "COMMIT":
switch ast.(type) {
case *tree.BeginTransaction, *tree.CommitTransaction:
return false
default:
return ast.StatementType() != tree.TypeDML
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/sessiondatapb/sequence_cache_node.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,13 +44,13 @@ func NewSequenceCacheNode() *SequenceCacheNode {
func (sc *SequenceCacheNode) NextValue(
seqID catid.DescID, clientVersion uint32, fetchNextValues func() (int64, int64, int64, error),
) (int64, error) {
sc.mu.RWMutex.RLock()
sc.mu.RLock()
cacheEntry, found := sc.cache[seqID]
sc.mu.RWMutex.RUnlock()
sc.mu.RUnlock()

createSequenceCacheNodeEntry := func() {
sc.mu.RWMutex.Lock()
defer sc.mu.RWMutex.Unlock()
sc.mu.Lock()
defer sc.mu.Unlock()
// There is a hazard that multiple threads could add the entry, so check if it exists again with the writer lock
_, found = sc.cache[seqID]
if !found {
Expand Down
60 changes: 51 additions & 9 deletions pkg/sql/testdata/telemetry_logging/logging/transaction_mode
Original file line number Diff line number Diff line change
Expand Up @@ -286,10 +286,9 @@ BEGIN; SELECT * FROM t LIMIT 4; SELECT * FROM t LIMIT 5; COMMIT

subtest end

subtest txn_retry_is_sampled
subtest no_sampling_reset_on_txn_retry

# If a transaction restarts, we should redo the sampling logic. This test ensures
# that even if the first attempt of a txn is not sampled, a retry may be sampled.
# This test ensures that if a transaction is retried, the sampling decision is not reset.

exec-sql
CREATE SEQUENCE seq START WITH 1
Expand All @@ -298,7 +297,7 @@ CREATE SEQUENCE seq START WITH 1
reset-last-sampled
----

# Execute a query at time=1 so that the first attempt of the next txn is not sampled.
# Execute a query at time=1 so that the the next txn executed at time=1 is not sampled.
spy-sql unixSecs=1
SELECT 1, 2, 3
----
Expand Down Expand Up @@ -331,15 +330,59 @@ SELECT 1, 2, 3
"User": "root"
}

# The stub time will only be advanced in the retry.
# Note that the crdb_internal.force_retry will always appear in the logs due to
# it not being a DML statement.
# This transaction will not be sampled due to not enough time having elapsed. Its
# retries should also not be sampled, even though the stub time will be advanced
# for the retries such that enough time has elapsed for sampling to be enabled,
# the sampling decision should not be reset and should use the sampling decision
# initially set for this transaction.
spy-sql unixSecs=1 restartUnixSecs=2
BEGIN;
SELECT 'hello';
SELECT CASE nextval('seq') WHEN 1 THEN crdb_internal.force_retry('1s') ELSE 2 END;
COMMIT;
----


# This is the reverse scenario to the above. We will execute a txn at time=2 so that it
# is sampled (enough time elapsed). On its retry, we will not advance the stub time so that
# the retry also runs at t=2. The retry should be sampled even though not enough time has
# elapsed due to it being a retry of asampled txn.
spy-sql unixSecs=2 restartUnixSecs=2
BEGIN;
SELECT 'hello';
SELECT CASE nextval('seq') WHEN 3 THEN crdb_internal.force_retry('1s') ELSE 2 END;
COMMIT;
----
{
"ApplicationName": "telemetry-logging-datadriven",
"Database": "defaultdb",
"Distribution": "local",
"EventType": "sampled_query",
"NumRows": 1,
"OutputRowsEstimate": 1,
"PlanGist": "AgICAgYC",
"SkippedQueries": 7,
"Statement": "SELECT ‹'hello'›",
"StatementFingerprintID": 15578946620736494000,
"StmtPosInTxn": 1,
"Tag": "SELECT",
"User": "root"
}
{
"ApplicationName": "telemetry-logging-datadriven",
"Database": "defaultdb",
"Distribution": "local",
"ErrorText": "crdb_internal.force_retry(): TransactionRetryWithProtoRefreshError: forced by crdb_internal.force_retry()",
"EventType": "sampled_query",
"OutputRowsEstimate": 1,
"PlanGist": "AgICAgYC",
"SQLSTATE": "40001",
"Statement": "SELECT CASE nextval(‹'seq'›) WHEN ‹3› THEN crdb_internal.force_retry(‹'00:00:01'›) ELSE ‹2› END",
"StatementFingerprintID": 17115235937825139000,
"StmtPosInTxn": 2,
"Tag": "SELECT",
"User": "root"
}
{
"ApplicationName": "telemetry-logging-datadriven",
"Database": "defaultdb",
Expand All @@ -349,7 +392,6 @@ COMMIT;
"NumRows": 1,
"OutputRowsEstimate": 1,
"PlanGist": "AgICAgYC",
"SkippedQueries": 3,
"Statement": "SELECT ‹'hello'›",
"StatementFingerprintID": 15578946620736494000,
"StmtPosInTxn": 1,
Expand All @@ -365,7 +407,7 @@ COMMIT;
"NumRows": 1,
"OutputRowsEstimate": 1,
"PlanGist": "AgICAgYC",
"Statement": "SELECT CASE nextval(‹'seq'›) WHEN ‹1› THEN crdb_internal.force_retry(‹'00:00:01'›) ELSE ‹2› END",
"Statement": "SELECT CASE nextval(‹'seq'›) WHEN ‹3› THEN crdb_internal.force_retry(‹'00:00:01'›) ELSE ‹2› END",
"StatementFingerprintID": 9891387630896048000,
"StmtPosInTxn": 2,
"Tag": "SELECT",
Expand Down
11 changes: 9 additions & 2 deletions pkg/testutils/lint/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,8 +14,9 @@ go_library(

go_test(
name = "lint_test",
# keep
srcs = [
"lint_test.go",
"gen-lint_test.go",
"nightly_lint_test.go",
],
data = glob(["testdata/**"]) + [
Expand All @@ -24,7 +25,6 @@ go_test(
],
embed = [":lint"],
embedsrcs = ["gcassert_paths.txt"],
gotags = ["lint"],
tags = [
"integration",
"no-remote-exec",
Expand All @@ -45,3 +45,10 @@ go_test(
"@org_golang_x_tools//go/packages",
],
)

genrule(
name = "gen-lint-test",
srcs = ["lint_test.go"],
outs = ["gen-lint_test.go"],
cmd = "cat $< | grep -v '//go:build' | grep -v '// +build' > $@",
)
11 changes: 4 additions & 7 deletions pkg/testutils/lint/lint_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,11 +78,8 @@ func vetCmd(t *testing.T, dir, name string, args []string, filters []stream.Filt
var b bytes.Buffer
cmd.Stdout = &b
cmd.Stderr = &b
switch err := cmd.Run(); err.(type) {
case nil:
case *exec.ExitError:
// Non-zero exit is expected.
default:
err := cmd.Run()
if err != nil && !errors.HasType(err, (*exec.ExitError)(nil)) {
t.Fatal(err)
}
filters = append([]stream.Filter{
Expand Down Expand Up @@ -138,11 +135,11 @@ func TestLint(t *testing.T) {
pkgVar, pkgSpecified := os.LookupEnv("PKG")

var nogoConfig map[string]any
nogoJson, err := os.ReadFile(filepath.Join(crdbDir, "build", "bazelutil", "nogo_config.json"))
nogoJSON, err := os.ReadFile(filepath.Join(crdbDir, "build", "bazelutil", "nogo_config.json"))
if err != nil {
t.Fatal(err)
}
if err := json.Unmarshal(nogoJson, &nogoConfig); err != nil {
if err := json.Unmarshal(nogoJSON, &nogoConfig); err != nil {
t.Error(err)
}

Expand Down
8 changes: 4 additions & 4 deletions pkg/util/log/eventpb/eventpbgen/gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ type fieldInfo struct {
Inherited bool
IsEnum bool
AllowZeroValue bool
Nullable bool
NotNullable bool
}

var (
Expand Down Expand Up @@ -464,7 +464,7 @@ func readInput(
MixedRedactable: mixed,
IsEnum: isEnum,
AllowZeroValue: allowZeroValue,
Nullable: !notNullable,
NotNullable: notNullable,
}
curMsg.Fields = append(curMsg.Fields, fi)
curMsg.AllFields = append(curMsg.AllFields, fi)
Expand Down Expand Up @@ -718,15 +718,15 @@ func (m *{{.GoType}}) AppendJSONFields(printComma bool, b redact.RedactableBytes
}
}
{{- else if eq .FieldType "nestedMessage"}}
{{ if .Nullable -}}
{{ if not .NotNullable -}}
if m.{{.FieldName}} != nil {
{{- end }}
if printComma { b = append(b, ',')}; printComma = true
b = append(b, "\"{{.FieldName}}\":"...)
b = append(b, '{')
printComma, b = m.{{.FieldName}}.AppendJSONFields(false, b)
b = append(b, '}')
{{ if .Nullable -}}
{{ if not .NotNullable -}}
}
{{- end }}
{{- else}}
Expand Down

0 comments on commit 3e61bd3

Please sign in to comment.