Skip to content

Commit

Permalink
Merge #83625 #83642
Browse files Browse the repository at this point in the history
83625: tracing: remove panic in Finish r=adityamaru, a=stevendanna

In #81079, we added an assertion that failed if a child recording of a
RecordingStructured span had more than one span recording. However,
this is problematic for couple of reasons:

1) The assertion was on a code path shared with RecordingVerbose
   spans; and,

2) A RecordingStructured span can have a RecordingVerbose child. The
   RecordingVerbose child is likely to have more than one span recording.

As a result, we have seen roachtests failing with

    panic: RecordingStructured has 12 recordings; expected 1

Here, we remove the assertion.

Fixes #83502

Release note: None

83642: server/systemconfigwatcher/systemconfigwatchertest: fix race r=ajwerner a=ajwerner

Fixes #83258

```
==================
WARNING: DATA RACE
Write at 0x00c001b75c00 by goroutine 264:
  github.com/cockroachdb/cockroach/pkg/roachpb.KeyValueByKey.Swap()
      github.com/cockroachdb/cockroach/pkg/roachpb/pkg/roachpb/data.go:2400 +0x190
  github.com/cockroachdb/cockroach/pkg/roachpb.(*KeyValueByKey).Swap()
      <autogenerated>:1 +0xb3
  sort.medianOfThree()
      GOROOT/src/sort/sort.go:90 +0x72
  sort.doPivot()
      GOROOT/src/sort/sort.go:114 +0x9e
  sort.quickSort()
      GOROOT/src/sort/sort.go:203 +0xac
  sort.Sort()
      GOROOT/src/sort/sort.go:231 +0x64
  github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest.runTest.func2()
      github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go:110 +0x4d1
  github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest.runTest.func3.1()
      github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go:118 +0x4e
  github.com/cockroachdb/cockroach/pkg/testutils.SucceedsWithinError.func1()
      github.com/cockroachdb/cockroach/pkg/testutils/soon.go:69 +0x7a
  github.com/cockroachdb/cockroach/pkg/util/retry.ForDuration()
      github.com/cockroachdb/cockroach/pkg/util/retry/retry.go:207 +0x191
  github.com/cockroachdb/cockroach/pkg/testutils.SucceedsWithinError()
      github.com/cockroachdb/cockroach/pkg/testutils/soon.go:75 +0x1d1
  github.com/cockroachdb/cockroach/pkg/testutils.SucceedsWithin()
      github.com/cockroachdb/cockroach/pkg/testutils/soon.go:57 +0x96
  github.com/cockroachdb/cockroach/pkg/testutils.SucceedsSoon()
      github.com/cockroachdb/cockroach/pkg/testutils/soon.go:40 +0x8d
  github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest.runTest.func3()
      github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go:117 +0x164
  github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest.runTest()
      github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go:121 +0x5cf
  github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest.TestSystemConfigWatcher.func1()
      github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/systemconfigwatchertest/test_system_config_watcher.go:62 +0xdc
  testing.tRunner()
      GOROOT/src/testing/testing.go:1259 +0x22f
  testing.(*T).Run·dwrap·21()
      GOROOT/src/testing/testing.go:1306 +0x47

Previous read at 0x00c001b75c00 by goroutine 502:
  runtime.slicecopy()
      GOROOT/src/runtime/slice.go:284 +0x0
  github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer.MergeKVs()
      github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer/kvs.go:47 +0x354
  github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher.(*Cache).handleUpdate()
      github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/cache.go:253 +0x431
  github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher.(*Cache).handleUpdate-fm()
      github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher/cache.go:232 +0x11e
  github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache.(*Watcher).handleUpdate()
      github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go:346 +0x277
  github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache.(*Watcher).Run()
      github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go:327 +0x1b04
  github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache.Start.func1()
      github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go:189 +0x3d1
  github.com/cockroachdb/cockroach/pkg/util/stop.(*Stopper).RunAsyncTaskEx.func2()
      github.com/cockroachdb/cockroach/pkg/util/stop/stopper.go:494 +0x551
```

Clone the values to make sure we don't have a race.

Release note: None

Co-authored-by: Steven Danna <[email protected]>
Co-authored-by: Andrew Werner <[email protected]>
  • Loading branch information
3 people committed Jul 1, 2022
3 parents e2767f9 + c66abb3 + 7216291 commit 78caa5c
Show file tree
Hide file tree
Showing 4 changed files with 68 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_library(
"//pkg/testutils/sqlutils",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/protoutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_kr_pretty//:pretty",
"@com_github_stretchr_testify//assert",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
"github.com/kr/pretty"
"github.com/stretchr/testify/assert"
Expand Down Expand Up @@ -102,14 +103,15 @@ func runTest(
if rs == nil {
return errors.New("nil config")
}
entries := protoutil.Clone(&rs.SystemConfigEntries).(*config.SystemConfigEntries)
sc := getSystemDescriptorAndZonesSpans(ctx, t, execCfg.Codec, kvDB)
if extraRows != nil {
sc = append(sc, extraRows(t)...)
sort.Sort(roachpb.KeyValueByKey(sc))
}
sort.Sort(roachpb.KeyValueByKey(rs.Values))
if !assert.Equal(noopT{}, sc, rs.Values) {
return errors.Errorf("mismatch: %v", pretty.Diff(sc, rs.Values))
sort.Sort(roachpb.KeyValueByKey(entries.Values))
if !assert.Equal(noopT{}, sc, entries.Values) {
return errors.Errorf("mismatch: %v", pretty.Diff(sc, entries.Values))
}
return nil
}
Expand Down
11 changes: 5 additions & 6 deletions pkg/util/tracing/crdbspan.go
Original file line number Diff line number Diff line change
Expand Up @@ -546,12 +546,11 @@ func (s *crdbSpan) recordFinishedChildrenLocked(childRecording tracingpb.Recordi
// records by falling through.
fallthrough
case tracingpb.RecordingStructured:
if len(childRecording) != 1 {
panic(fmt.Sprintf("RecordingStructured has %d recordings; expected 1", len(childRecording)))
}

for i := range rootChild.StructuredRecords {
s.recordInternalLocked(&rootChild.StructuredRecords[i], &s.mu.recording.structured)
for ci := range childRecording {
child := &childRecording[ci]
for i := range child.StructuredRecords {
s.recordInternalLocked(&child.StructuredRecords[i], &s.mu.recording.structured)
}
}
case tracingpb.RecordingOff:
break
Expand Down
57 changes: 57 additions & 0 deletions pkg/util/tracing/span_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -410,6 +410,63 @@ func TestRecordingMaxSpans(t *testing.T) {
require.Len(t, root.StructuredRecords, extraChildren)
}

// TestRecordingDowngradesToStructuredIfTooBig finishes a span that has reached
// the maximum number of recorded spans and asserts that its structured
// recordings are correctly added to the parent.
func TestRecordingDowngradesToStructuredIfTooBig(t *testing.T) {
now := timeutil.Now()
clock := timeutil.NewManualTime(now)
tr := NewTracerWithOpt(context.Background(), WithTestingKnobs(TracerTestingKnobs{Clock: clock}))

s1 := tr.StartSpan("p", WithRecording(tracingpb.RecordingVerbose))
s2 := tr.StartSpan("c", WithParent(s1))
extraChildren := 10
numChildren := maxRecordedSpansPerTrace + extraChildren
payload := &types.Int32Value{Value: int32(1)}
for i := 0; i < numChildren; i++ {
child := tr.StartSpan(fmt.Sprintf("cc%d", i), WithParent(s2))
child.RecordStructured(payload)
child.Finish()
}

// We expect recordings from sp and up to the maximum number of spans and
// structured records from all spans over the max.
rec := s2.FinishAndGetConfiguredRecording()
require.Len(t, rec, maxRecordedSpansPerTrace+1)
require.Len(t, rec[0].StructuredRecords, extraChildren)

pl, err := types.MarshalAny(payload)
require.NoError(t, err)
structuredRecordSize := (&tracingpb.StructuredRecord{Time: now, Payload: pl}).MemorySize()
maxNumStructuredRecordings := maxStructuredBytesPerSpan / structuredRecordSize
if maxNumStructuredRecordings > numChildren {
maxNumStructuredRecordings = numChildren
}

// Since s2's child count exceeded the maximum, we don't expect to see any of
// its span recordings in s1. But, we should only see as many of s2's
// structured recordings as possible.
rec2 := s1.FinishAndGetConfiguredRecording()
require.Len(t, rec2, 1)
require.Len(t, rec2[0].StructuredRecords, maxNumStructuredRecordings)
}

// Test that a RecordingStructured parent does not panic when asked to ingest a
// remote verbose recording. Ingesting a recording of different type is unusual,
// since children are created with the parent's recording mode, but it can
// happen if the child's recording mode was changed dynamically.
func TestRemoteSpanWithDifferentRecordingMode(t *testing.T) {
tr := NewTracer()
s1 := tr.StartSpan("p", WithRecording(tracingpb.RecordingStructured))
s2 := tr.StartSpan("c", WithRemoteParentFromSpanMeta(s1.Meta()), WithRecording(tracingpb.RecordingVerbose))
s3 := tr.StartSpan("cc", WithParent(s2), WithRecording(tracingpb.RecordingVerbose))
s3.Finish()
r := s2.FinishAndGetConfiguredRecording()
require.NotPanics(t, func() { s1.ImportRemoteRecording(r) })
r2 := s1.FinishAndGetConfiguredRecording()
require.Len(t, r2, 1)
}

type explodyNetTr struct {
trace.Trace
}
Expand Down

0 comments on commit 78caa5c

Please sign in to comment.