Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

changefeedccl: flake in TestChangefeedKafkaMessageTooLarge #107591

Closed
knz opened this issue Jul 26, 2023 · 6 comments · Fixed by #107752
Closed

changefeedccl: flake in TestChangefeedKafkaMessageTooLarge #107591

knz opened this issue Jul 26, 2023 · 6 comments · Fixed by #107752
Assignees
Labels
A-cdc Change Data Capture branch-master Failures and bugs on the master branch. C-test-failure Broken test (automatically or manually discovered). T-cdc

Comments

@knz
Copy link
Contributor

knz commented Jul 26, 2023

Found here: https://teamcity.cockroachdb.com/buildConfiguration/Cockroach_BazelEssentialCi/11056281?hideProblemsFromDependencies=false&hideTestsFromDependencies=false&expandBuildChangesSection=true&expandBuildProblemsSection=true&expandBuildTestsSection=true

Failed
=== RUN   TestChangefeedKafkaMessageTooLarge/kafka/succeed_against_a_large_backfill
    helpers_test.go:745: still waiting for job status; current reverting
        --- FAIL: TestChangefeedKafkaMessageTooLarge/kafka/succeed_against_a_large_backfill (45.75s)

Jira issue: CRDB-30118

Epic CRDB-11732

@knz knz added C-test-failure Broken test (automatically or manually discovered). A-cdc Change Data Capture T-cdc labels Jul 26, 2023
@blathers-crl
Copy link

blathers-crl bot commented Jul 26, 2023

cc @cockroachdb/cdc

@knz knz added the branch-master Failures and bugs on the master branch. label Jul 26, 2023
@jayshrivastava
Copy link
Contributor

It seems that the job get's stuck in status REVERTING for 45s which causes the test to time out. This call to closeFeed hangs.

defer closeFeed(t, foo)

It looks like the cancel succeeds, setting the job state to reverting.

cockroach/pkg/jobs/adopt.go

Lines 587 to 621 in 912983c

case StatusReverting:
if err := job.WithTxn(txn).Update(ctx, func(
txn isql.Txn, md JobMetadata, ju *JobUpdater,
) error {
if !r.cancelRegisteredJobContext(id) {
// If we didn't already have a running job for this
// lease, clear out the lease here since it won't be
// cleared be cleared out on Resume exit.
//
// NB: This working as part of the update depends on
// the fact that the job struct does not have a
// claim set and thus won't validate the claim on
// update.
r.clearLeaseForJobID(id, txn, txn.KV())
}
if md.Payload.Error == "" {
// Set default cancellation reason.
md.Payload.Error = errJobCanceled.Error()
}
encodedErr := errors.EncodeError(ctx, errJobCanceled)
md.Payload.FinalResumeError = &encodedErr
ju.UpdatePayload(md.Payload)
// When we cancel a job, we want to reset its last_run and num_runs
// so that the job can be picked-up in the next adopt-loop, sooner
// than its current next-retry time.
ju.UpdateRunStats(0 /* numRuns */, r.clock.Now().GoTime() /* lastRun */)
return nil
}); err != nil {
return errors.Wrapf(err, "job %d: tried to cancel but could not mark as reverting", id)
}
log.Infof(ctx, "job %d, session id: %s canceled: the job is now reverting",
id, s.ID())
default:
return errors.AssertionFailedf("unexpected job status %s: %v", statusString, job)
}

However, it doesn't look like the registry ends up calling OnFailOrCancel.

err = resumer.OnFailOrCancel(onFailOrCancelCtx, execCtx, jobErr)

@jayshrivastava
Copy link
Contributor

jayshrivastava commented Jul 26, 2023

Right now, I suspect the kafka sink deadlocks. I printed all the gouroutines at the point where closeFeed(t, foo) fails.

1 @ 0x49c176 0x4ad51e 0x4ad4f5 0x4c9fe5 0x4e7725 0x4ba4b4a 0x4ba4b36 0x4b9acfd 0x4b9d70a 0x4b63116 0x4b636a5 0x4b6348e 0x4b62fe5 0x4b61ff5 0x4b65d09 0x23ba4d5 0x23bf8f3 0x278d53a 0x36f42d3 0x4b5d015 0x1c4eb65 0x1b99564 0x4ce321
# labels: {"job":"CHANGEFEED id=885823367427489793", "n":"1"}
#	0x4c9fe4	sync.runtime_SemacquireMutex+0x24									GOROOT/src/runtime/sema.go:77
#	0x4e7724	sync.(*Mutex).lockSlow+0x164										GOROOT/src/sync/mutex.go:171
#	0x4ba4b49	sync.(*Mutex).Lock+0xc9											GOROOT/src/sync/mutex.go:90
#	0x4ba4b35	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*kafkaSink).Flush+0xb5				github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/sink_kafka.go:416
#	0x4b9acfc	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.errorWrapperSink.Flush+0x5c			github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/sink.go:432
#	0x4b9d709	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*safeSink).Flush+0xa9				github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/sink.go:702
#	0x4b63115	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*changeAggregator).flushBufferedEvents+0x95	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeed_processors.go:670
#	0x4b636a4	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*changeAggregator).flushFrontier+0x24		github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeed_processors.go:724
#	0x4b6348d	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*changeAggregator).noteResolvedSpan+0x34d	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeed_processors.go:698
#	0x4b62fe4	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*changeAggregator).tick+0x484			github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeed_processors.go:658
#	0x4b61ff4	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*changeAggregator).Next+0xd4			github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeed_processors.go:578
#	0x4b65d08	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*changeFrontier).Next+0xe8			github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeed_processors.go:1231
#	0x23ba4d4	github.com/cockroachdb/cockroach/pkg/sql/execinfra.Run+0x54						github.com/cockroachdb/cockroach/pkg/sql/execinfra/base.go:196
#	0x23bf8f2	github.com/cockroachdb/cockroach/pkg/sql/execinfra.(*ProcessorBaseNoHelper).Run+0x72			github.com/cockroachdb/cockroach/pkg/sql/execinfra/processorsbase.go:726
#	0x278d539	github.com/cockroachdb/cockroach/pkg/sql/flowinfra.(*FlowBase).Run+0x219				github.com/cockroachdb/cockroach/pkg/sql/flowinfra/flow.go:575
#	0x36f42d2	github.com/cockroachdb/cockroach/pkg/sql.(*DistSQLPlanner).Run+0xb92					github.com/cockroachdb/cockroach/pkg/sql/distsql_running.go:902
#	0x4b5d014	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.startDistChangefeed.func1+0x414			github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeed_dist.go:304
#	0x1c4eb64	github.com/cockroachdb/cockroach/pkg/util/ctxgroup.Group.GoCtx.func1+0x24				github.com/cockroachdb/cockroach/pkg/util/ctxgroup/ctxgroup.go:168
#	0x1b99563	golang.org/x/sync/errgroup.(*Group).Go.func1+0x63							golang.org/x/sync/errgroup/external/org_golang_x_sync/errgroup/errgroup.go:75

This is pointing here

@jayshrivastava
Copy link
Contributor

jayshrivastava commented Jul 26, 2023

I'm convinced there is a deadlock. This goroutine holds the same lock and gets stuck on a select stmt.

1 @ 0x49c176 0x4ac45c 0x4c88f0a 0x4c6b512 0x4c6b635 0x4ba655f 0x4ba5ab4 0x4ce321
# labels: {"job":"CHANGEFEED id=885823367427489793", "n":"1"}
#	0x4c88f09	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*fakeKafkaSink).Dial.func3.1+0xe9	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/testfeed_test.go:1712
#	0x4c6b511	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*syncProducerMock).SendMessage+0x31	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/sink_test.go:99
#	0x4c6b634	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*syncProducerMock).SendMessages+0x94	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/sink_test.go:108
#	0x4ba655e	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*kafkaSink).handleBufferedRetries+0x3be	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/sink_kafka.go:653
#	0x4ba5ab3	github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl.(*kafkaSink).workerLoop+0x6b3		github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/sink_kafka.go:569

This is pointing to

The lock is held here

// Once inflight messages to retry are done buffering, find a new client
// that successfully resends and continue on with it.
if isRetrying() && s.mu.inflight == 0 {
if err := s.handleBufferedRetries(retryBuf, retryErr); err != nil {
s.mu.flushErr = err
}
endInternalRetry()
}

@jayshrivastava
Copy link
Contributor

It looks like the test deadlocks bc the buffer here fills up.

feedCh := make(chan *sarama.ProducerMessage, 1024)

Even though TestChangefeedKafkaMessageTooLarge/kafka/succeed_against_a_large_backfill should be reading and writing 2000 messages, we end up writing >14689 messages. The reason is that each of the 2000 messages fails with a retryable error 20% of the time. The retry and subsequent retries also fail with a retryable error 20% of the time. This makes the total number of messages blow up and fill the buffer.

I also thought about why handleBufferedRetries keeps trying to send messages even after the test has finished and asserted that the 2000 expected messages have been seen. You would think that there are no retries left to perform. My answer at the moment is that resolved messages are also in the buffer, which take up space and can get stuck retrying.

jayshrivastava added a commit to jayshrivastava/cockroach that referenced this issue Jul 27, 2023
Previously, this test would deadlock due to kafka retrying messages
too many times. These messages are stored in a buffer
of size 1024 created by the CDC testing infra: https://github.com/cockroachdb/cockroach/blob/5c3f96d38cdc3a2d953ca3ffb1e39e97d7e5110e/pkg/ccl/changefeedccl/testfeed_test.go#L1819

The test asserts that 2000 messages pass through the buffer.
When the test finishes, it stops reading from the buffer. The problem
is that due to retries, there may be more messages sent to the buffer
than that are read out of the buffer. Even after the 2000 messages
are read and the test is shutting down, the sink may be blocked
trying to put resolved messages (plus retries) in the buffer.
If this happens, the changefeed resumer (same goroutine as the kafka sink)
gets blocked and does not terminate when the job is cancelled
at the end of the test.

This change caps the number of retries at 200 for this test, so there
should be no more than 200 extra messages plus a few resolved messages
during this test. This is far less than the buffer size of 1024.

See detailed explanation in cockroachdb#107591.

Fixes: cockroachdb#107591
Epic: none
Release note: None
jayshrivastava added a commit to jayshrivastava/cockroach that referenced this issue Jul 27, 2023
Previously, this test would deadlock due to kafka retrying messages
too many times. These messages are stored in a buffer
of size 1024 created by the CDC testing infra: https://github.com/cockroachdb/cockroach/blob/5c3f96d38cdc3a2d953ca3ffb1e39e97d7e5110e/pkg/ccl/changefeedccl/testfeed_test.go#L1819

The test asserts that 2000 messages pass through the buffer.
When the test finishes, it stops reading from the buffer. The problem
is that due to retries, there may be more messages sent to the buffer
than that are read out of the buffer. Even after the 2000 messages
are read and the test is shutting down, the sink may be blocked
trying to put resolved messages (plus retries) in the buffer.
If this happens, the changefeed resumer (same goroutine as the kafka sink)
gets blocked and does not terminate when the job is cancelled
at the end of the test.

This change caps the number of retries at 200 for this test, so there
should be no more than 200 extra messages plus a few resolved messages
during this test. This is far less than the buffer size of 1024.

See detailed explanation in cockroachdb#107591.

Fixes: cockroachdb#107591
Epic: none
Release note: None
craig bot pushed a commit that referenced this issue Jul 28, 2023
…107752 #107802 #107803

106508: util/must: add runtime assertion API r=erikgrinaker a=erikgrinaker

For details and usage examples, see the [package documentation](https://github.com/erikgrinaker/cockroach/blob/must/pkg/util/must/must.go).

---

This patch adds a convenient and canonical API for runtime assertions, inspired by the Testify package used for Go test assertions. It is intended to encourage liberal use of runtime assertions throughout the code base, by making it as easy as possible to write assertions that follow best practices. It does not attempt to reinvent the wheel, but instead builds on existing infrastructure.

Assertion failures are fatal in all non-release builds, including roachprod clusters and roachtests, to ensure they will be noticed. In release builds, they instead log the failure and report it to Sentry (if enabled), and return an assertion error to the caller for propagation. This avoids excessive disruption in production environments, where an assertion failure is often scoped to an individual RPC request, transaction, or range, and crashing the node can turn a minor problem into a full-blown outage. It is still possible to kill the node when appropriate via `log.Fatalf`, but this should be the exception rather than the norm.

It also supports expensive assertions that must be compiled out of normal dev/test/release builds for performance reasons. These are instead enabled in special test builds.

This is intended to be used instead of other existing assertion mechanisms, which have various shortcomings:

* `log.Fatalf`: kills the node even in release builds, which can cause severe disruption over often minor issues.

* `errors.AssertionFailedf`: only suitable when we have an error return path, does not fatal in non-release builds, and are not always notified in release builds.

* `logcrash.ReportOrPanic`: panics rather than fatals, which can leave the node limping along. Requires the caller to implement separate assertion handling in release builds, which is easy to forget. Also requires propagating cluster settings, which aren't always available.

* `buildutil.CrdbTestBuild`: only enabled in Go tests, not roachtests, roachprod clusters, or production clusters.

* `util.RaceEnabled`: only enabled in race builds. Expensive assertions should be possible to run without the additional overhead of the race detector.

For more details and examples, see the `must` package documentation.

Resolves #94986.
Epic: none
Release note: None

107094: streamingest: unskip TestTenantStreamingUnavailableStreamAddress r=lidorcarmel a=lidorcarmel

Changing a few things to get this test to pass under stress:
- use 50 ranges instead of 10, because there are already 50-ish system ranges,
  so if we write only 10 more ranges those might not get distributed on all
  servers.
- avoid reading from the source cluster after stopping a node, it's flaky,
  see #107499 for more info.

Epic: none
Fixes: #107023
Fixes: #106865

Release note: None

107717: server/profiler: remove `server.cpu_profile.enabled` setting r=xinhaoz a=xinhaoz

Cpu profiling can be enabled by setting the cluster setting `server.cpu_profile.cpu_usage_combined_threshold`. This makes `server.cpu_profile.enabled` redundant and makes it more difficult and confusing to enable cpu profiling. This commit removes the `server.cpu_profile.enabled` setting entirely. Note that both jdefault values for the cluster settings set profiling off.

Closes: #102024

Release note (sql change): The cluster setting
`server.cpu_profile.enabled` has been removed.
`server.cpu_profile.cpu_usage_combined_threshold` can enable and disable cpu profiling.

107720: cli: add probe_range in debug.zip r=j82w a=j82w

PR #79546 introduces `crdb_internal.probe_range`. This PR adds the `crdb_internal.probe_range` to the debug.zip. The LIMIT gives a very approximately ~1000ms*100 target on how long this can take, so that running debug.zip against an unavailable cluster won't take too long.

closes: #80360

Release note (cli change): The debug.zip now includes the `crdb_internal.probe_range` table with a limit of 100 rows to avoid the query from taking to long.

107727: server: deflake TestServerShutdownReleasesSession r=rafiss a=rafiss

The tenant was not being fully stopped, so the test could encounter flakes.

fixes #107592
Release note: None

107742: ui: show txn fingerprint details page with unspecified app r=xinhaoz a=xinhaoz

Previously, when the app was not specified in the url search params for the txn details fingerprint page, the page would fail to load. This commit allows the page to load when there is no app specified but a fingerprint id that matches the requested page in the payload. The first matching fingerprint id is loaded.

Additionally, the TransactionDetailsLink will not include the appNames search param unless the provided prop is non-nullish.

Fixes: #107731

Release note (bug fix): Txn fingerprint details page in the console UI should load with the fingerprint details even if no app is specified in the URL.




Demo:
https://www.loom.com/share/810308d3dcd74ca888c42287ebafaecf

107745: kvserver: fix test merge queue when grunning unsupported r=irfansharif a=kvoli

`TestMergeQueue/load-based-merging/switch...below-threshold` asserts that switching the split objective between CPU and QPS will not cause ranges to merge, even if their pre-switch load qualified them for merging.

This test was broken when `grunning` was unsupported, as the objective never actually switches to anything other than QPS.

Add a check for `grunning` support, and assert that a merge occurs if unsupported.

Fixes: #106937
Epic: none
Release note: None

107749: opt: add enable_durable_locking_for_serializable session variable r=DrewKimball,nvanbenschoten a=michae2

Follow-up from #105857

This commit ammends 6a3e43d to add a session variable to control whether guaranteed-durable locks are used under serializable isolation.

Informs: #100194

Epic: CRDB-25322

Release note (sql change): Add a new session variable, `enable_durable_locking_for_serializable`, which controls locking durability under serializable isolation. With this set to true, SELECT FOR UPDATE locks, SELECT FOR SHARED locks, and constraint check locks (e.g. locks acquired during foreign key checks if
`enable_implicit_fk_locking_for_serializable` is set to true) will be guaranteed-durable under serializable isolation, meaning they will always be held to transaction commit. (These locks are always guaranteed-durable under weaker isolation levels.)

By default, under serializable isolation these locks are best-effort rather than guaranteed-durable, meaning in some cases (e.g. leaseholder transfer, node loss, etc.) they could be released before transaction commit. Serializable isolation does not rely on locking for correctness, only using it to improve performance under contention, so this default is a deliberate choice to avoid the performance overhead of lock replication.

107752: changefeedccl: prevent deadlock in TestChangefeedKafkaMessageTooLarge r=miretskiy a=jayshrivastava

Previously, this test would deadlock due to kafka retrying messages too many times. These messages are stored in a buffer of size 1024 created by the CDC testing infra: https://github.com/cockroachdb/cockroach/blob/5c3f96d38cdc3a2d953ca3ffb1e39e97d7e5110e/pkg/ccl/changefeedccl/testfeed_test.go#L1819

The test asserts that 2000 messages pass through the buffer. When the test finishes, it stops reading from the buffer. The problem is that due to retries, there may be more messages sent to the buffer than that are read out of the buffer. Even after the 2000 messages are read and the test is shutting down, the sink may be blocked trying to put resolved messages (plus retries) in the buffer. If this happens, the changefeed resumer (same goroutine as the kafka sink) gets blocked and does not terminate when the job is cancelled at the end of the test.

This change caps the number of retries at 200 for this test, so there should be no more than 200 extra messages plus a few resolved messages during this test. This is far less than the buffer size of 1024.

See detailed explanation in #107591.

Fixes: #107591
Epic: none
Release note: None

107802: teamcity-trigger: don't start a job for an empty target r=healthy-pod a=rickystewart

This makes no sense, so skip these cases.

Closes: #107779
Closes: #107780
Closes: #107781

Epic: none
Release note: None

107803: githubpost: set `map` field if `null` r=healthy-pod a=rickystewart

Go is a really good language.

Informs: #107779

Epic: none
Release note: None

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
Co-authored-by: Lidor Carmel <lidor@cockroachlabs.com>
Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com>
Co-authored-by: j82w <jwilley@cockroachlabs.com>
Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
Co-authored-by: Austen McClernon <austen@cockroachlabs.com>
Co-authored-by: Michael Erickson <michae2@cockroachlabs.com>
Co-authored-by: Jayant Shrivastava <jayants@cockroachlabs.com>
Co-authored-by: Ricky Stewart <ricky@cockroachlabs.com>
@craig craig bot closed this as completed in 4f1e340 Jul 28, 2023
jayshrivastava added a commit to jayshrivastava/cockroach that referenced this issue Jul 31, 2023
Previously, this test would deadlock due to kafka retrying messages
too many times. These messages are stored in a buffer
of size 1024 created by the CDC testing infra: https://github.com/cockroachdb/cockroach/blob/5c3f96d38cdc3a2d953ca3ffb1e39e97d7e5110e/pkg/ccl/changefeedccl/testfeed_test.go#L1819

The test asserts that 2000 messages pass through the buffer.
When the test finishes, it stops reading from the buffer. The problem
is that due to retries, there may be more messages sent to the buffer
than that are read out of the buffer. Even after the 2000 messages
are read and the test is shutting down, the sink may be blocked
trying to put resolved messages (plus retries) in the buffer.
If this happens, the changefeed resumer (same goroutine as the kafka sink)
gets blocked and does not terminate when the job is cancelled
at the end of the test.

This change caps the number of retries at 200 for this test, so there
should be no more than 200 extra messages plus a few resolved messages
during this test. This is far less than the buffer size of 1024.

See detailed explanation in cockroachdb#107591.

Fixes: cockroachdb#107591
Epic: none
Release note: None
jayshrivastava added a commit to jayshrivastava/cockroach that referenced this issue Jul 31, 2023
Previously, this test would deadlock due to kafka retrying messages
too many times. These messages are stored in a buffer
of size 1024 created by the CDC testing infra: https://github.com/cockroachdb/cockroach/blob/5c3f96d38cdc3a2d953ca3ffb1e39e97d7e5110e/pkg/ccl/changefeedccl/testfeed_test.go#L1819

The test asserts that 2000 messages pass through the buffer.
When the test finishes, it stops reading from the buffer. The problem
is that due to retries, there may be more messages sent to the buffer
than that are read out of the buffer. Even after the 2000 messages
are read and the test is shutting down, the sink may be blocked
trying to put resolved messages (plus retries) in the buffer.
If this happens, the changefeed resumer (same goroutine as the kafka sink)
gets blocked and does not terminate when the job is cancelled
at the end of the test.

This change caps the number of retries at 200 for this test, so there
should be no more than 200 extra messages plus a few resolved messages
during this test. This is far less than the buffer size of 1024.

See detailed explanation in cockroachdb#107591.

Fixes: cockroachdb#107872
Fixes: cockroachdb#107591
Epic: none
Release note: None
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
A-cdc Change Data Capture branch-master Failures and bugs on the master branch. C-test-failure Broken test (automatically or manually discovered). T-cdc
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants