From 3a2d49ac68b8b49dc2274faf12cc2b700ba59532 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 16 Jun 2022 19:19:57 -0700 Subject: [PATCH 1/2] sql: reuse the slice of RequestUnion objects between fetches This commit teaches `txnKVFetcher` and `txnKVStreamer` to reuse the same slice of `RequestUnion` objects between different fetches. It is now extremely easy to do given the recent refactor. We do perform memory accounting for this slice (against a memory account bound to an unlimited memory monitor). Additionally, a similar optimization is applied to how resume requests are populated by the Streamer. Release note: None --- .../kvclient/kvstreamer/requests_provider.go | 10 ++++- pkg/kv/kvclient/kvstreamer/streamer.go | 45 +++++++++++-------- pkg/sql/colfetcher/index_join.go | 1 + pkg/sql/row/kv_batch_fetcher.go | 44 +++++++++++++++--- pkg/sql/row/kv_batch_streamer.go | 24 +++++++--- pkg/sql/row/kv_fetcher.go | 3 +- pkg/sql/rowexec/joinreader.go | 4 ++ 7 files changed, 98 insertions(+), 33 deletions(-) diff --git a/pkg/kv/kvclient/kvstreamer/requests_provider.go b/pkg/kv/kvclient/kvstreamer/requests_provider.go index e416191487a8..fda9b48bec1e 100644 --- a/pkg/kv/kvclient/kvstreamer/requests_provider.go +++ b/pkg/kv/kvclient/kvstreamer/requests_provider.go @@ -79,8 +79,16 @@ type singleRangeBatch struct { // Hints.SingleRowLookup is false and some Scan requests were enqueued. subRequestIdx []int32 // reqsReservedBytes tracks the memory reservation against the budget for - // the memory usage of reqs. + // the memory usage of reqs, excluding the overhead. reqsReservedBytes int64 + // overheadAccountedFor tracks the memory reservation against the budget for + // the overhead of the reqs slice (i.e. of roachpb.RequestUnion objects). + // Since we reuse the same reqs slice for resume requests, this can be + // released only when the BatchResponse doesn't have any resume spans. + // + // RequestUnion.Size() ignores the overhead of RequestUnion object, so we + // need to account for it separately. + overheadAccountedFor int64 // minTargetBytes, if positive, indicates the minimum TargetBytes limit that // this singleRangeBatch should be sent with in order for the response to // not be empty. Note that TargetBytes of at least minTargetBytes is diff --git a/pkg/kv/kvclient/kvstreamer/streamer.go b/pkg/kv/kvclient/kvstreamer/streamer.go index 6418c7e3964b..3a7dd8ddb366 100644 --- a/pkg/kv/kvclient/kvstreamer/streamer.go +++ b/pkg/kv/kvclient/kvstreamer/streamer.go @@ -501,12 +501,13 @@ func (s *Streamer) Enqueue(ctx context.Context, reqs []roachpb.RequestUnion) (re //} r := singleRangeBatch{ - reqs: singleRangeReqs, - positions: positions, - subRequestIdx: subRequestIdx, - reqsReservedBytes: requestsMemUsage(singleRangeReqs), + reqs: singleRangeReqs, + positions: positions, + subRequestIdx: subRequestIdx, + reqsReservedBytes: requestsMemUsage(singleRangeReqs), + overheadAccountedFor: requestUnionOverhead * int64(cap(singleRangeReqs)), } - totalReqsMemUsage += r.reqsReservedBytes + totalReqsMemUsage += r.reqsReservedBytes + r.overheadAccountedFor if s.mode == OutOfOrder { // Sort all single-range requests to be in the key order. @@ -1090,6 +1091,12 @@ func (w *workerCoordinator) performRequestAsync( // non-empty responses as well as resume spans, if any. respOverestimate := targetBytes - memoryFootprintBytes reqOveraccounted := req.reqsReservedBytes - resumeReqsMemUsage + if resumeReqsMemUsage == 0 { + // There will be no resume request, so we will lose the + // reference to the req.reqs slice and can release its memory + // reservation. + reqOveraccounted += req.overheadAccountedFor + } overaccountedTotal := respOverestimate + reqOveraccounted if overaccountedTotal >= 0 { w.s.budget.release(ctx, overaccountedTotal) @@ -1213,9 +1220,6 @@ func calculateFootprint( } } } - // This addendum is the first step of requestsMemUsage() and we've already - // added the size of each resume request above. - resumeReqsMemUsage += requestUnionOverhead * int64(numIncompleteGets+numIncompleteScans) return memoryFootprintBytes, resumeReqsMemUsage, numIncompleteGets, numIncompleteScans } @@ -1223,7 +1227,7 @@ func calculateFootprint( // in the BatchResponse. The ResumeSpans, if found, are added into a new // singleRangeBatch request that is added to be picked up by the mainLoop of the // worker coordinator. This method assumes that req is no longer needed by the -// caller, so req.positions is reused for the ResumeSpans. +// caller, so the slices from req are reused for the ResumeSpans. // // It also assumes that the budget has already been reconciled with the // reservations for Results that will be created. @@ -1236,14 +1240,15 @@ func (w *workerCoordinator) processSingleRangeResults( ) error { numIncompleteRequests := numIncompleteGets + numIncompleteScans var resumeReq singleRangeBatch - // We have to allocate the new slice for requests, but we can reuse the - // positions slice. - resumeReq.reqs = make([]roachpb.RequestUnion, numIncompleteRequests) + // We have to allocate the new Get and Scan requests, but we can reuse the + // reqs and the positions slices. + resumeReq.reqs = req.reqs[:numIncompleteRequests] resumeReq.positions = req.positions[:0] resumeReq.subRequestIdx = req.subRequestIdx[:0] // We've already reconciled the budget with the actual reservation for the // requests with the ResumeSpans. resumeReq.reqsReservedBytes = resumeReqsMemUsage + resumeReq.overheadAccountedFor = req.overheadAccountedFor gets := make([]struct { req roachpb.GetRequest union roachpb.RequestUnion_Get @@ -1415,7 +1420,14 @@ func (w *workerCoordinator) processSingleRangeResults( // If we have any incomplete requests, add them back into the work // pool. - if len(resumeReq.reqs) > 0 { + if numIncompleteRequests > 0 { + // Make sure to nil out old requests that we didn't include into the + // resume request. We don't have to do this if there aren't any + // incomplete requests since req and resumeReq will be garbage collected + // on their own. + for i := numIncompleteRequests; i < len(req.reqs); i++ { + req.reqs[i] = roachpb.RequestUnion{} + } w.s.requestsToServe.add(resumeReq) } @@ -1504,12 +1516,7 @@ func init() { const requestUnionOverhead = int64(unsafe.Sizeof(roachpb.RequestUnion{})) -func requestsMemUsage(reqs []roachpb.RequestUnion) int64 { - // RequestUnion.Size() ignores the overhead of RequestUnion object, so we'll - // account for it separately first. - memUsage := requestUnionOverhead * int64(cap(reqs)) - // No need to account for elements past len(reqs) because those must be - // unset and we have already accounted for RequestUnion object above. +func requestsMemUsage(reqs []roachpb.RequestUnion) (memUsage int64) { for _, r := range reqs { memUsage += int64(r.Size()) } diff --git a/pkg/sql/colfetcher/index_join.go b/pkg/sql/colfetcher/index_join.go index 0b94d2b5b3ef..22c9c2426ab2 100644 --- a/pkg/sql/colfetcher/index_join.go +++ b/pkg/sql/colfetcher/index_join.go @@ -512,6 +512,7 @@ func NewColIndexJoin( rowcontainer.NewKVStreamerResultDiskBuffer( flowCtx.Cfg.TempStorage, diskMonitor, ), + kvFetcherMemAcc, ) } else { kvFetcher = row.NewKVFetcher( diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index 818c32afcefd..da161806f8f7 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -13,6 +13,7 @@ package row import ( "context" "time" + "unsafe" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -148,6 +149,7 @@ type txnKVFetcher struct { // least once. alreadyFetched bool batchIdx int + reqsScratch []roachpb.RequestUnion responses []roachpb.ResponseUnion remainingBatches [][]byte @@ -156,11 +158,12 @@ type txnKVFetcher struct { getResponseScratch [1]roachpb.KeyValue acc *mon.BoundAccount - // spansAccountedFor and batchResponseAccountedFor track the number of bytes - // that we've already registered with acc in regards to spans and the batch - // response, respectively. + // spansAccountedFor, batchResponseAccountedFor, and reqsScratchAccountedFor + // track the number of bytes that we've already registered with acc in + // regards to spans, the batch response, and reqsScratch, respectively. spansAccountedFor int64 batchResponseAccountedFor int64 + reqsScratchAccountedFor int64 // If set, we will use the production value for kvBatchSize. forceProductionKVBatchSize bool @@ -395,7 +398,7 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { ba.Header.TargetBytes = int64(f.batchBytesLimit) ba.Header.MaxSpanRequestKeys = int64(f.getBatchKeyLimit()) ba.AdmissionHeader = f.requestAdmissionHeader - ba.Requests = spansToRequests(f.spans.Spans, f.reverse, f.lockStrength) + ba.Requests = spansToRequests(f.spans.Spans, f.reverse, f.lockStrength, f.reqsScratch) if log.ExpensiveLogEnabled(ctx, 2) { log.VEventf(ctx, 2, "Scan %s", f.spans) @@ -472,6 +475,21 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { f.batchIdx++ f.scratchSpans.reset() f.alreadyFetched = true + // Keep the reference to the requests slice in order to reuse in the future + // after making sure to nil out the requests in order to lose references to + // the underlying Get and Scan requests which could keep large byte slices + // alive. + f.reqsScratch = ba.Requests + for i := range f.reqsScratch { + f.reqsScratch[i] = roachpb.RequestUnion{} + } + if monitoring { + reqsScratchMemUsage := requestUnionOverhead * int64(cap(f.reqsScratch)) + if err := f.acc.Resize(ctx, f.reqsScratchAccountedFor, reqsScratchMemUsage); err != nil { + return err + } + f.reqsScratchAccountedFor = reqsScratchMemUsage + } // TODO(radu): We should fetch the next chunk in the background instead of waiting for the next // call to fetch(). We can use a pool of workers to issue the KV ops which will also limit the @@ -623,7 +641,9 @@ func (f *txnKVFetcher) reset(ctx context.Context) { f.remainingBatches = nil f.spans = identifiableSpans{} f.scratchSpans = identifiableSpans{} - // Release only the allocations made by this fetcher. + // Release only the allocations made by this fetcher. Note that we're still + // keeping the reference to reqsScratch, so we don't release the allocation + // for it. f.acc.Shrink(ctx, f.batchResponseAccountedFor+f.spansAccountedFor) f.batchResponseAccountedFor, f.spansAccountedFor = 0, 0 } @@ -633,14 +653,24 @@ func (f *txnKVFetcher) close(ctx context.Context) { f.reset(ctx) } +const requestUnionOverhead = int64(unsafe.Sizeof(roachpb.RequestUnion{})) + // spansToRequests converts the provided spans to the corresponding requests. If // a span doesn't have the EndKey set, then a Get request is used for it; // otherwise, a Scan (or ReverseScan if reverse is true) request is used with // BATCH_RESPONSE format. +// +// The provided reqsScratch is reused if it has enough capacity for all spans, +// if not, a new slice is allocated. func spansToRequests( - spans roachpb.Spans, reverse bool, keyLocking lock.Strength, + spans roachpb.Spans, reverse bool, keyLocking lock.Strength, reqsScratch []roachpb.RequestUnion, ) []roachpb.RequestUnion { - reqs := make([]roachpb.RequestUnion, len(spans)) + var reqs []roachpb.RequestUnion + if cap(reqsScratch) >= len(spans) { + reqs = reqsScratch[:len(spans)] + } else { + reqs = make([]roachpb.RequestUnion, len(spans)) + } // Detect the number of gets vs scans, so we can batch allocate all of the // requests precisely. nGets := 0 diff --git a/pkg/sql/row/kv_batch_streamer.go b/pkg/sql/row/kv_batch_streamer.go index 20b948ba2888..d3451972d0fd 100644 --- a/pkg/sql/row/kv_batch_streamer.go +++ b/pkg/sql/row/kv_batch_streamer.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/errors" ) @@ -45,8 +46,11 @@ type txnKVStreamer struct { streamer *kvstreamer.Streamer keyLocking lock.Strength - spans roachpb.Spans - spanIDs []int + spans roachpb.Spans + spanIDs []int + reqsScratch []roachpb.RequestUnion + + acc *mon.BoundAccount // getResponseScratch is reused to return the result of Get requests. getResponseScratch [1]roachpb.KeyValue @@ -63,11 +67,12 @@ var _ KVBatchFetcher = &txnKVStreamer{} // newTxnKVStreamer creates a new txnKVStreamer. func newTxnKVStreamer( - streamer *kvstreamer.Streamer, lockStrength descpb.ScanLockingStrength, + streamer *kvstreamer.Streamer, lockStrength descpb.ScanLockingStrength, acc *mon.BoundAccount, ) KVBatchFetcher { return &txnKVStreamer{ streamer: streamer, keyLocking: getKeyLockingStrength(lockStrength), + acc: acc, } } @@ -86,13 +91,22 @@ func (f *txnKVStreamer) SetupNextFetch( if log.ExpensiveLogEnabled(ctx, 2) { log.VEventf(ctx, 2, "Scan %s", spans) } - reqs := spansToRequests(spans, false /* reverse */, f.keyLocking) + reqs := spansToRequests(spans, false /* reverse */, f.keyLocking, f.reqsScratch) if err := f.streamer.Enqueue(ctx, reqs); err != nil { return err } f.spans = spans f.spanIDs = spanIDs - return nil + // Keep the reference to the requests slice in order to reuse in the future + // after making sure to nil out the requests in order to lose references to + // the underlying Get and Scan requests which could keep large byte slices + // alive. + f.reqsScratch = reqs + for i := range f.reqsScratch { + f.reqsScratch[i] = roachpb.RequestUnion{} + } + reqsScratchMemUsage := requestUnionOverhead * int64(cap(f.reqsScratch)) + return f.acc.ResizeTo(ctx, reqsScratchMemUsage) } func (f *txnKVStreamer) getSpanID(resultPosition int) int { diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 6df5df898518..2f988a87c9c9 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -134,6 +134,7 @@ func NewStreamingKVFetcher( singleRowLookup bool, maxKeysPerRow int, diskBuffer kvstreamer.ResultDiskBuffer, + kvFetcherMemAcc *mon.BoundAccount, ) *KVFetcher { streamer := kvstreamer.NewStreamer( distSender, @@ -157,7 +158,7 @@ func NewStreamingKVFetcher( maxKeysPerRow, diskBuffer, ) - return newKVFetcher(newTxnKVStreamer(streamer, lockStrength)) + return newKVFetcher(newTxnKVStreamer(streamer, lockStrength, kvFetcherMemAcc)) } func newKVFetcher(batchFetcher KVBatchFetcher) *KVFetcher { diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index 65c56c2d1b6e..4d88690ac0bc 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -129,6 +129,7 @@ type joinReader struct { unlimitedMemMonitor *mon.BytesMonitor budgetAcc mon.BoundAccount diskMonitor *mon.BytesMonitor + txnKVStreamerMemAcc mon.BoundAccount } input execinfra.RowSource @@ -472,6 +473,7 @@ func newJoinReader( ) jr.streamerInfo.unlimitedMemMonitor.Start(flowCtx.EvalCtx.Ctx(), flowCtx.EvalCtx.Mon, mon.BoundAccount{}) jr.streamerInfo.budgetAcc = jr.streamerInfo.unlimitedMemMonitor.MakeBoundAccount() + jr.streamerInfo.txnKVStreamerMemAcc = jr.streamerInfo.unlimitedMemMonitor.MakeBoundAccount() var diskBuffer kvstreamer.ResultDiskBuffer if jr.maintainOrdering { @@ -496,6 +498,7 @@ func newJoinReader( singleRowLookup, int(spec.FetchSpec.MaxKeysPerRow), diskBuffer, + &jr.streamerInfo.txnKVStreamerMemAcc, ) } else { // When not using the Streamer API, we want to limit the batch size hint @@ -1090,6 +1093,7 @@ func (jr *joinReader) close() { } if jr.usesStreamer { jr.streamerInfo.budgetAcc.Close(jr.Ctx) + jr.streamerInfo.txnKVStreamerMemAcc.Close(jr.Ctx) jr.streamerInfo.unlimitedMemMonitor.Stop(jr.Ctx) if jr.streamerInfo.diskMonitor != nil { jr.streamerInfo.diskMonitor.Stop(jr.Ctx) From 55bf5b70b4e00f19d6b4dbd5724d3a5dd99bf487 Mon Sep 17 00:00:00 2001 From: Nick Travers Date: Mon, 27 Jun 2022 19:31:46 -0700 Subject: [PATCH 2/2] vendor: bump Pebble to 85bb1c759894 ``` 85bb1c75 rangekey: create `rangekey` package 0d272ec1 db: fix bug in deletion estimate computations ``` Release note: None. --- DEPS.bzl | 6 +++--- build/bazelutil/distdir_files.bzl | 2 +- go.mod | 2 +- go.sum | 4 ++-- pkg/storage/BUILD.bazel | 1 + pkg/storage/pebble.go | 3 ++- vendor | 2 +- 7 files changed, 11 insertions(+), 9 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 1141a3f1d006..7bbb66cdf6ec 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -1427,10 +1427,10 @@ def go_deps(): patches = [ "@com_github_cockroachdb_cockroach//build/patches:com_github_cockroachdb_pebble.patch", ], - sha256 = "ec4ac067a239d05ea2c7b7d0ffccdcf7546ba6835da701595217fe7dfaadcb31", - strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20220624201436-71d17c2a007b", + sha256 = "41cee4a297dc7e9242c4e821832973c4470ed86c902b979c61d4925512a8756b", + strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20220627233705-85bb1c759894", urls = [ - "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220624201436-71d17c2a007b.zip", + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220627233705-85bb1c759894.zip", ], ) go_repository( diff --git a/build/bazelutil/distdir_files.bzl b/build/bazelutil/distdir_files.bzl index f4ec26515234..3e0f1b429d5d 100644 --- a/build/bazelutil/distdir_files.bzl +++ b/build/bazelutil/distdir_files.bzl @@ -187,7 +187,7 @@ DISTDIR_FILES = { "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/go-test-teamcity/com_github_cockroachdb_go_test_teamcity-v0.0.0-20191211140407-cff980ad0a55.zip": "bac30148e525b79d004da84d16453ddd2d5cd20528e9187f1d7dac708335674b", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/gostdlib/com_github_cockroachdb_gostdlib-v1.13.0.zip": "b3d43d8f95edf65f73a5348f29e1159823cac64b148f8d3bb48340bf55d70872", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/logtags/com_github_cockroachdb_logtags-v0.0.0-20211118104740-dabe8e521a4f.zip": "1972c3f171f118add3fd9e64bcea6cbb9959a3b7fa0ada308e8a7310813fea74", - "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220624201436-71d17c2a007b.zip": "ec4ac067a239d05ea2c7b7d0ffccdcf7546ba6835da701595217fe7dfaadcb31", + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220627233705-85bb1c759894.zip": "41cee4a297dc7e9242c4e821832973c4470ed86c902b979c61d4925512a8756b", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/redact/com_github_cockroachdb_redact-v1.1.3.zip": "7778b1e4485e4f17f35e5e592d87eb99c29e173ac9507801d000ad76dd0c261e", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/returncheck/com_github_cockroachdb_returncheck-v0.0.0-20200612231554-92cdbca611dd.zip": "ce92ba4352deec995b1f2eecf16eba7f5d51f5aa245a1c362dfe24c83d31f82b", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/sentry-go/com_github_cockroachdb_sentry_go-v0.6.1-cockroachdb.2.zip": "fbb2207d02aecfdd411b1357efe1192dbb827959e36b7cab7491731ac55935c9", diff --git a/go.mod b/go.mod index 89231375b69c..afadc87c7d56 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55 github.com/cockroachdb/gostdlib v1.13.0 github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f - github.com/cockroachdb/pebble v0.0.0-20220624201436-71d17c2a007b + github.com/cockroachdb/pebble v0.0.0-20220627233705-85bb1c759894 github.com/cockroachdb/redact v1.1.3 github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd github.com/cockroachdb/stress v0.0.0-20220310203902-58fb4627376e diff --git a/go.sum b/go.sum index b1527100d7ce..b51ddfbc826d 100644 --- a/go.sum +++ b/go.sum @@ -468,8 +468,8 @@ github.com/cockroachdb/gostdlib v1.13.0/go.mod h1:eXX95p9QDrYwJfJ6AgeN9QnRa/lqqi github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f h1:6jduT9Hfc0njg5jJ1DdKCFPdMBrp/mdZfCpa5h+WM74= github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= -github.com/cockroachdb/pebble v0.0.0-20220624201436-71d17c2a007b h1:4IzbYFgF+NdvAZDMUgV1nEVszyHH3i8kgplHFfAwruk= -github.com/cockroachdb/pebble v0.0.0-20220624201436-71d17c2a007b/go.mod h1:pr479tNxFRmcfDyklTqoRMDDVmRlEbL+d7a7rhKnrI4= +github.com/cockroachdb/pebble v0.0.0-20220627233705-85bb1c759894 h1:UQ6vBnOM2Ru+UThk+GusrFYUFz8d8f+F5kFtfE2yOdQ= +github.com/cockroachdb/pebble v0.0.0-20220627233705-85bb1c759894/go.mod h1:pr479tNxFRmcfDyklTqoRMDDVmRlEbL+d7a7rhKnrI4= github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 9251142095c8..5fce60a01a30 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -80,6 +80,7 @@ go_library( "@com_github_cockroachdb_logtags//:logtags", "@com_github_cockroachdb_pebble//:pebble", "@com_github_cockroachdb_pebble//bloom", + "@com_github_cockroachdb_pebble//rangekey", "@com_github_cockroachdb_pebble//record", "@com_github_cockroachdb_pebble//sstable", "@com_github_cockroachdb_pebble//vfs", diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index e86107e12d02..8df7fc919d0b 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -49,6 +49,7 @@ import ( "github.com/cockroachdb/logtags" "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/bloom" + "github.com/cockroachdb/pebble/rangekey" "github.com/cockroachdb/pebble/sstable" "github.com/cockroachdb/pebble/vfs" "github.com/cockroachdb/redact" @@ -358,7 +359,7 @@ func (tc *pebbleDataBlockMVCCTimeIntervalRangeCollector) Add( ) error { // TODO(erikgrinaker): should reuse a buffer for keysDst, but keyspan.Key is // not exported by Pebble. - span, err := sstable.DecodeRangeKey(key, value, nil) + span, err := rangekey.Decode(key, value, nil) if err != nil { return errors.Wrapf(err, "decoding range key at %s", key) } diff --git a/vendor b/vendor index 951dbe569a7d..4801ff7a272f 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit 951dbe569a7d7eacf3aea38241ac483eaf23468a +Subproject commit 4801ff7a272f9cbfa3e1addfd2ba159d20be4fe9