cockroachdb / cockroach

CockroachDB — the cloud native, distributed SQL database designed for high availability, effortless scale, and control over data placement.
https://www.cockroachlabs.com
Other
29.9k stars 3.78k forks source link

sqlstats: crdb process halted by a coredump issue. #130290

Closed shailendra-patel closed 1 week ago

shailendra-patel commented 1 week ago

On drt-chaos and drt-large running version v24.3.0-alpha.00000000-dev-ae386bd we are observing crdb process unexpectedly halted because of a coredump.

On analyzing the coredump there is a nil pointer dereference inside the proto-generated MarshalToSizedBuffer.

gdb /home/ubuntu/cockroach core.cockroach.196793.drt-chaos-0002.1725564304
(gdb) frame 13
#13 0x0000000001e57f1a in github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights.(*Statement).MarshalToSizedBuffer (m=0x0, dAtA=..., ~r0=<optimized out>, ~r1=...)
    at github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights/bazel-out/k8-opt/bin/pkg/sql/sqlstats/insights/insights_go_proto_/github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights/insights.pb.go:710
(gdb) info args
m = 0x0
dAtA = {array = 0x0, len = 5085153, cap = 196793}
func (m *Statement) MarshalToSizedBuffer(dAtA []byte) (int, error) {
    i := len(dAtA)
    _ = i
    var l int
    _ = l
    if len(m.KVNodeIDs) > 0 {

The caller is inside this statement size, err := m.Statements[iNdEx].MarshalToSizedBuffer(dAtA[:i])

 frame 14
#14 0x0000000001e59ff9 in github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights.(*Insight).MarshalToSizedBuffer (m=0xc0b75ebe70, dAtA=..., ~r0=<optimized out>, ~r1=...)
    at github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights/bazel-out/k8-opt/bin/pkg/sql/sqlstats/insights/insights_go_proto_/github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights/insights.pb.go:950
(gdb) info args
iNdEx = 196813

Therefore, we know that m.Statements[196813] is nil.

This has happened multiple times on both cluster after September 5. On drt-chaos core dump is available in directory /mnt/data1/cores

More details in thread https://cockroachlabs.slack.com/archives/C05FHJJ0MD0/p1725625432291499

Jira issue: CRDB-41977

blathers-crl[bot] commented 1 week ago

Hi @shailendra-patel, please add branch-* labels to identify which branch(es) this C-bug affects.

:owl: Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

shailendra-patel commented 1 week ago

Setting the priority to P0 because this issue results in dead nodes in CockroachDB, requiring manual intervention to restart the database.

srosenberg commented 1 week ago

As described above, we have a nil-dereference inside the insight.Statements slice. This appears to be a new regression introduced in [1]. Specifically we have a potential data race between the statementBuf.release from the newly introduced lockingRegistry.clearSession (added in [1]), and the existing lockingRegistry.ObserveTransaction, which iterates (concurrently) over *statements [2]. Before the change statements.release() happened inside defer, i.e., after the iterator finished.

The fact that this data race was undetected over a course of several weeks is suggestive that we may not have a sufficient unit and stress test coverage of the said code.

[1] https://github.com/cockroachdb/cockroach/pull/128400 [2] https://github.com/cockroachdb/cockroach/blob/fa9c0528fc0d06be1b4cfc534ec0501448111fbe/pkg/sql/sqlstats/insights/registry.go#L115-L175

srosenberg commented 1 week ago

Also, the insightPool isn't actually helping to reduce allocation pressure because the second statement inside makeInsight [1] essentially undoes what a sync.Pool is designed to optimize, i.e., it unconditionally allocates a fresh Insight struct. I see that there is now a TODO to replace it with the statementBuf pool. That should definitely fix it, considering that pool is implemented correctly :)

[1] https://github.com/cockroachdb/cockroach/blob/fa9c0528fc0d06be1b4cfc534ec0501448111fbe/pkg/sql/sqlstats/insights/pool.go#L27-L33

xinhaoz commented 1 week ago

Thank you for looking at this! Trying to reproduce and haven't succeeded yet - I think it's correct this likely was introduced by the PR linked but I'm having trouble fully believing the explanation. clearSession and ObserveTransaction on the locking registry shouldn't be called concurrently as far as I understand this code. Aside form the store cache, the lockingRegistry is only accessed in the ingestion goroutine and the buffered events are processed synchronously.

The stack traces seem to suggest this deref is occuring in the LIstExecutionInsights grpc, which reads insights objects from LockingStore. I did add this piece of code in that same PR but the locking in that struct seems correct to me thus far. Please let me know if I'm missing something you saw in your investigation, otherwise going to continue to investigate this.

xinhaoz commented 1 week ago

I was able to repro this in a simple unit test , the issue does seem to be from a race between releaseInsight and ListExecutionInsights. Will post the RCA tomorrow with a fix!

xinhaoz commented 1 week ago

The issue is in the following block of code in the statusServer which iterates through the insights and only makes a shallow copy, sharing the slice that is later cleared when marshaling the response: https://github.com/cockroachdb/cockroach/blob/0a731c25b54db37da6af295f549f24637abd923b/pkg/server/status.go#L395-L405

Introduced recently: https://github.com/cockroachdb/cockroach/blob/77c253be537b10621da2b10cff4b67b005d0137b/pkg/sql/sqlstats/insights/pool.go#L36-L39

We can either create a new slice when creating the response in the status server code or just remove the pooling of insights altogether. I think we can do the first for now.

srosenberg commented 1 week ago

The issue is in the following block of code in the statusServer which iterates through the insights and only makes a shallow copy, sharing the slice that is later cleared when marshaling the response

Nice find! I was looking at those two, but missed the shallow copy. It's still technically a data race. Any intuition as to why none of the pre-existing stress tests picked it up?

We can either create a new slice when creating the response in the status server code

Yep, that makes sense to me; the extra short-lived allocation is not on a critical path.

xinhaoz commented 1 week ago

Any intuition as to why none of the pre-existing stress tests picked it up?

We definitely need to beef up the testing around insights edge cases. We never hit this because we don't have tests that stress / perform race detection when reading the insights cache while evicting at a high rate -- This was the test yday that detects this easily just with --race or --stress.

func TestListExecutionInsightsSafeWhileEvictingInsights(t *testing.T) {
    defer leaktest.AfterTest(t)()
    defer log.Scope(t).Close(t)
    ctx := context.Background()
    server := serverutils.StartServerOnly(t, base.TestServerArgs{})
    defer server.Stopper().Stop(ctx)

    s := server.StatusServer().(*systemStatusServer)
    insights.ExecutionInsightsCapacity.Override(ctx, &server.ClusterSettings().SV, 5)
    insights.LatencyThreshold.Override(ctx, &server.ClusterSettings().SV, 1*time.Millisecond)
    var wg sync.WaitGroup
    wg.Add(3)
    conn := sqlutils.MakeSQLRunner(server.ApplicationLayer().SQLConn(t))
    go func() {
        defer wg.Done()
        for i := 0; i < 500; i++ {
            conn.Exec(t, "SELECT * FROM system.users")
        }
    }()
    go func() {
        defer wg.Done()
        for i := 0; i < 500; i++ {
            conn.Exec(t, "SELECT * FROM system.users")
        }
    }()
    go func() {
        defer wg.Done()
        for i := 0; i < 1000; i++ {
            require.NotPanics(t, func() {
                _, err := s.ListExecutionInsights(ctx, &serverpb.ListExecutionInsightsRequest{})
                require.NoError(t, err)
            })
        }
    }()
    wg.Wait()
}