github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/ccl/changefeedccl/bench_test.go (about) 1 // Copyright 2018 The Cockroach Authors. 2 // 3 // Licensed as a CockroachDB Enterprise file under the Cockroach Community 4 // License (the "License"); you may not use this file except in compliance with 5 // the License. You may obtain a copy of the License at 6 // 7 // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt 8 9 package changefeedccl 10 11 import ( 12 "bytes" 13 "context" 14 gosql "database/sql" 15 "fmt" 16 "math" 17 "sync" 18 "testing" 19 "time" 20 21 "github.com/cockroachdb/cockroach/pkg/base" 22 "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" 23 "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed" 24 "github.com/cockroachdb/cockroach/pkg/gossip" 25 "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" 26 "github.com/cockroachdb/cockroach/pkg/keys" 27 "github.com/cockroachdb/cockroach/pkg/roachpb" 28 "github.com/cockroachdb/cockroach/pkg/sql" 29 "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" 30 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 31 "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" 32 "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" 33 "github.com/cockroachdb/cockroach/pkg/util/hlc" 34 "github.com/cockroachdb/cockroach/pkg/util/leaktest" 35 "github.com/cockroachdb/cockroach/pkg/util/log" 36 "github.com/cockroachdb/cockroach/pkg/util/mon" 37 "github.com/cockroachdb/cockroach/pkg/util/span" 38 "github.com/cockroachdb/cockroach/pkg/util/syncutil" 39 "github.com/cockroachdb/cockroach/pkg/util/timeutil" 40 "github.com/cockroachdb/cockroach/pkg/workload" 41 "github.com/cockroachdb/cockroach/pkg/workload/bank" 42 "github.com/cockroachdb/errors" 43 "github.com/stretchr/testify/require" 44 ) 45 46 func BenchmarkChangefeedTicks(b *testing.B) { 47 defer leaktest.AfterTest(b)() 48 defer log.Scope(b).Close(b) 49 50 // In PR #38211, we removed the polling based data watcher in changefeeds in 51 // favor of RangeFeed. This benchmark worked by writing a bunch of data at 52 // certain timestamps and manipulating clocks at runtime so the polling 53 // grabbed a little of it at a time. There's fundamentally no way for this to 54 // work with RangeFeed without a rewrite, but it's not being used for anything 55 // right now, so the rewrite isn't worth it. We should fix this if we need to 56 // start doing changefeed perf work at some point. 57 b.Skip(`broken in #38211`) 58 59 ctx := context.Background() 60 s, sqlDBRaw, _ := serverutils.StartServer(b, base.TestServerArgs{UseDatabase: "d"}) 61 defer s.Stopper().Stop(ctx) 62 sqlDB := sqlutils.MakeSQLRunner(sqlDBRaw) 63 sqlDB.Exec(b, `CREATE DATABASE d`) 64 sqlDB.Exec(b, `SET CLUSTER SETTING changefeed.experimental_poll_interval = '0ms'`) 65 66 numRows := 1000 67 if testing.Short() { 68 numRows = 100 69 } 70 bankTable := bank.FromRows(numRows).Tables()[0] 71 timestamps, _, err := loadWorkloadBatches(sqlDBRaw, bankTable) 72 if err != nil { 73 b.Fatal(err) 74 } 75 76 runBench := func(b *testing.B, feedClock *hlc.Clock) { 77 var sinkBytes int64 78 b.ResetTimer() 79 for i := 0; i < b.N; i++ { 80 b.StartTimer() 81 sink, cancelFeed, err := createBenchmarkChangefeed(ctx, s, feedClock, `d`, `bank`) 82 require.NoError(b, err) 83 for rows := 0; rows < numRows; { 84 r, sb := sink.WaitForEmit() 85 rows += r 86 sinkBytes += sb 87 } 88 b.StopTimer() 89 if err := cancelFeed(); err != nil { 90 b.Errorf(`%+v`, err) 91 } 92 } 93 b.SetBytes(sinkBytes / int64(b.N)) 94 } 95 96 b.Run(`InitialScan`, func(b *testing.B) { 97 // Use a clock that's immediately larger than any timestamp the data was 98 // loaded at to catch it all in the initial scan. 99 runBench(b, s.Clock()) 100 }) 101 102 b.Run(`SteadyState`, func(b *testing.B) { 103 // TODO(dan): This advances the clock through the timestamps of the ingested 104 // data every time it's called, but that's a little unsatisfying. Instead, 105 // wait for each batch to come out of the feed before advancing the 106 // timestamp. 107 var feedTimeIdx int 108 feedClock := hlc.NewClock(func() int64 { 109 if feedTimeIdx < len(timestamps) { 110 feedTimeIdx++ 111 return timestamps[feedTimeIdx-1].UnixNano() 112 } 113 return timeutil.Now().UnixNano() 114 }, time.Nanosecond) 115 runBench(b, feedClock) 116 }) 117 } 118 119 type benchSink struct { 120 syncutil.Mutex 121 cond *sync.Cond 122 emits int 123 emitBytes int64 124 } 125 126 func makeBenchSink() *benchSink { 127 s := &benchSink{} 128 s.cond = sync.NewCond(&s.Mutex) 129 return s 130 } 131 132 func (s *benchSink) EmitRow( 133 ctx context.Context, _ *sqlbase.TableDescriptor, k, v []byte, _ hlc.Timestamp, 134 ) error { 135 return s.emit(int64(len(k) + len(v))) 136 } 137 func (s *benchSink) EmitResolvedTimestamp(ctx context.Context, e Encoder, ts hlc.Timestamp) error { 138 var noTopic string 139 p, err := e.EncodeResolvedTimestamp(ctx, noTopic, ts) 140 if err != nil { 141 return err 142 } 143 return s.emit(int64(len(p))) 144 } 145 func (s *benchSink) Flush(_ context.Context) error { return nil } 146 func (s *benchSink) Close() error { return nil } 147 func (s *benchSink) emit(bytes int64) error { 148 s.Lock() 149 defer s.Unlock() 150 s.emits++ 151 s.emitBytes += bytes 152 s.cond.Broadcast() 153 return nil 154 } 155 156 // WaitForEmit blocks until at least one thing is emitted by the sink. It 157 // returns the number of emitted messages and bytes since the last WaitForEmit. 158 func (s *benchSink) WaitForEmit() (int, int64) { 159 s.Lock() 160 defer s.Unlock() 161 for s.emits == 0 { 162 s.cond.Wait() 163 } 164 emits, emitBytes := s.emits, s.emitBytes 165 s.emits, s.emitBytes = 0, 0 166 return emits, emitBytes 167 } 168 169 // createBenchmarkChangefeed starts a stripped down changefeed. It watches 170 // `database.table` and outputs to `sinkURI`. The given `feedClock` is only used 171 // for the internal ExportRequest polling, so a benchmark can write data with 172 // different timestamps beforehand and simulate the changefeed going through 173 // them in steps. 174 // 175 // The returned sink can be used to count emits and the closure handed back 176 // cancels the changefeed (blocking until it's shut down) and returns an error 177 // if the changefeed had failed before the closure was called. 178 // 179 // This intentionally skips the distsql and sink parts to keep the benchmark 180 // focused on the core changefeed work, but it does include the poller. 181 func createBenchmarkChangefeed( 182 ctx context.Context, 183 s serverutils.TestServerInterface, 184 feedClock *hlc.Clock, 185 database, table string, 186 ) (*benchSink, func() error, error) { 187 tableDesc := sqlbase.GetTableDescriptor(s.DB(), keys.SystemSQLCodec, database, table) 188 spans := []roachpb.Span{tableDesc.PrimaryIndexSpan(keys.SystemSQLCodec)} 189 details := jobspb.ChangefeedDetails{ 190 Targets: jobspb.ChangefeedTargets{tableDesc.ID: jobspb.ChangefeedTarget{ 191 StatementTimeName: tableDesc.Name, 192 }}, 193 Opts: map[string]string{ 194 changefeedbase.OptEnvelope: string(changefeedbase.OptEnvelopeRow), 195 }, 196 } 197 initialHighWater := hlc.Timestamp{} 198 encoder, err := makeJSONEncoder(details.Opts) 199 if err != nil { 200 return nil, nil, err 201 } 202 sink := makeBenchSink() 203 204 settings := s.ClusterSettings() 205 metrics := MakeMetrics(base.DefaultHistogramWindowInterval()).(*Metrics) 206 buf := kvfeed.MakeChanBuffer() 207 leaseMgr := s.LeaseManager().(*lease.Manager) 208 mm := mon.MakeUnlimitedMonitor( 209 context.Background(), "test", mon.MemoryResource, 210 nil /* curCount */, nil /* maxHist */, math.MaxInt64, settings, 211 ) 212 needsInitialScan := initialHighWater.IsEmpty() 213 if needsInitialScan { 214 initialHighWater = details.StatementTime 215 } 216 _, withDiff := details.Opts[changefeedbase.OptDiff] 217 kvfeedCfg := kvfeed.Config{ 218 Settings: settings, 219 DB: s.DB(), 220 Clock: feedClock, 221 Gossip: gossip.MakeExposedGossip(s.GossipI().(*gossip.Gossip)), 222 Spans: spans, 223 Targets: details.Targets, 224 Sink: buf, 225 LeaseMgr: leaseMgr, 226 Metrics: &metrics.KVFeedMetrics, 227 MM: &mm, 228 InitialHighWater: initialHighWater, 229 WithDiff: withDiff, 230 NeedsInitialScan: needsInitialScan, 231 } 232 233 rowsFn := kvsToRows(s.ExecutorConfig().(sql.ExecutorConfig).Codec, 234 s.LeaseManager().(*lease.Manager), details, buf.Get) 235 sf := span.MakeFrontier(spans...) 236 tickFn := emitEntries(s.ClusterSettings(), details, hlc.Timestamp{}, sf, 237 encoder, sink, rowsFn, TestingKnobs{}, metrics) 238 239 ctx, cancel := context.WithCancel(ctx) 240 go func() { _ = kvfeed.Run(ctx, kvfeedCfg) }() 241 242 errCh := make(chan error, 1) 243 var wg sync.WaitGroup 244 wg.Add(1) 245 go func() { 246 defer wg.Done() 247 err := func() error { 248 sf := span.MakeFrontier(spans...) 249 for { 250 // This is basically the ChangeAggregator processor. 251 resolvedSpans, err := tickFn(ctx) 252 if err != nil { 253 return err 254 } 255 // This is basically the ChangeFrontier processor, the resolved 256 // spans are normally sent using distsql, so we're missing a bit 257 // of overhead here. 258 for _, rs := range resolvedSpans { 259 if sf.Forward(rs.Span, rs.Timestamp) { 260 frontier := sf.Frontier() 261 if err := emitResolvedTimestamp(ctx, encoder, sink, frontier); err != nil { 262 return err 263 } 264 } 265 } 266 } 267 }() 268 errCh <- err 269 }() 270 cancelFn := func() error { 271 select { 272 case err := <-errCh: 273 return err 274 default: 275 } 276 cancel() 277 wg.Wait() 278 return nil 279 } 280 return sink, cancelFn, nil 281 } 282 283 // loadWorkloadBatches inserts a workload.Table's row batches, each in one 284 // transaction. It returns the timestamps of these transactions and the byte 285 // size for use with b.SetBytes. 286 func loadWorkloadBatches(sqlDB *gosql.DB, table workload.Table) ([]time.Time, int64, error) { 287 if _, err := sqlDB.Exec(`CREATE TABLE "` + table.Name + `" ` + table.Schema); err != nil { 288 return nil, 0, err 289 } 290 291 var now time.Time 292 var timestamps []time.Time 293 var benchBytes int64 294 var numRows int 295 296 var insertStmtBuf bytes.Buffer 297 var params []interface{} 298 for batchIdx := 0; batchIdx < table.InitialRows.NumBatches; batchIdx++ { 299 if _, err := sqlDB.Exec(`BEGIN`); err != nil { 300 return nil, 0, err 301 } 302 303 params = params[:0] 304 insertStmtBuf.Reset() 305 insertStmtBuf.WriteString(`INSERT INTO "` + table.Name + `" VALUES `) 306 for _, row := range table.InitialRows.BatchRows(batchIdx) { 307 numRows++ 308 if len(params) != 0 { 309 insertStmtBuf.WriteString(`,`) 310 } 311 insertStmtBuf.WriteString(`(`) 312 for colIdx, datum := range row { 313 if colIdx != 0 { 314 insertStmtBuf.WriteString(`,`) 315 } 316 benchBytes += workload.ApproxDatumSize(datum) 317 params = append(params, datum) 318 fmt.Fprintf(&insertStmtBuf, `$%d`, len(params)) 319 } 320 insertStmtBuf.WriteString(`)`) 321 } 322 if _, err := sqlDB.Exec(insertStmtBuf.String(), params...); err != nil { 323 return nil, 0, err 324 } 325 326 if err := sqlDB.QueryRow(`SELECT transaction_timestamp(); COMMIT;`).Scan(&now); err != nil { 327 return nil, 0, err 328 } 329 timestamps = append(timestamps, now) 330 } 331 332 var totalRows int 333 if err := sqlDB.QueryRow( 334 `SELECT count(*) FROM "` + table.Name + `"`, 335 ).Scan(&totalRows); err != nil { 336 return nil, 0, err 337 } 338 if numRows != totalRows { 339 return nil, 0, errors.Errorf(`sanity check failed: expected %d rows got %d`, numRows, totalRows) 340 } 341 342 return timestamps, benchBytes, nil 343 }