Skip to content

Commit 5fcc8bb

Browse files
committed
rditer: move IterateReplicaKeySpansShared
Move IterateReplicaKeySpansShared out of the deprecated engineccl package and remove the call to the deprecated utilccl.CheckEnterpriseEnabled func. Epic: none Release note: none
1 parent 15804fd commit 5fcc8bb

File tree

4 files changed

+36
-95
lines changed

4 files changed

+36
-95
lines changed

pkg/ccl/storageccl/engineccl/BUILD.bazel

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -6,27 +6,18 @@ go_library(
66
"ctr_stream.go",
77
"encrypted_fs.go",
88
"pebble_key_manager.go",
9-
"shared_storage.go",
109
],
1110
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl",
1211
visibility = ["//visibility:public"],
1312
deps = [
14-
"//pkg/ccl/utilccl",
15-
"//pkg/kv/kvserver/rditer",
16-
"//pkg/roachpb",
17-
"//pkg/settings/cluster",
18-
"//pkg/storage",
1913
"//pkg/storage/enginepb",
2014
"//pkg/storage/fs",
2115
"//pkg/storage/storageconfig",
2216
"//pkg/util/log",
2317
"//pkg/util/protoutil",
2418
"//pkg/util/syncutil",
25-
"//pkg/util/uuid",
2619
"@com_github_cockroachdb_errors//:errors",
2720
"@com_github_cockroachdb_errors//oserror",
28-
"@com_github_cockroachdb_pebble//:pebble",
29-
"@com_github_cockroachdb_pebble//rangekey",
3021
"@com_github_cockroachdb_pebble//vfs",
3122
"@com_github_cockroachdb_pebble//vfs/atomicfs",
3223
"@com_github_gogo_protobuf//proto",

pkg/ccl/storageccl/engineccl/shared_storage.go

Lines changed: 0 additions & 69 deletions
This file was deleted.

pkg/kv/kvserver/kv_snapshot_strategy.go

Lines changed: 7 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -358,12 +358,6 @@ func (kvSS *kvBatchSnapshotStrategy) Send(
358358
return nil
359359
}
360360

361-
// If snapshots containing shared files are allowed, and this range is a
362-
// non-system range, take advantage of shared storage to minimize the amount
363-
// of data we're iterating on and sending over the network.
364-
sharedReplicate := header.SharedReplicate && rditer.IterateReplicaKeySpansShared != nil
365-
externalReplicate := header.ExternalReplicate && rditer.IterateReplicaKeySpansShared != nil
366-
367361
iterateRKSpansVisitor := func(iter storage.EngineIterator, _ roachpb.Span) error {
368362
timingTag.start("iter")
369363
defer timingTag.stop("iter")
@@ -419,7 +413,7 @@ func (kvSS *kvBatchSnapshotStrategy) Send(
419413
LockTable: true,
420414
// In shared/external mode, the user span come from external SSTs and
421415
// are not iterated over here.
422-
UserKeys: !(sharedReplicate || externalReplicate),
416+
UserKeys: !(header.SharedReplicate || header.ExternalReplicate),
423417
},
424418
ReplicatedByRangeID: true,
425419
UnreplicatedByRangeID: false,
@@ -428,9 +422,12 @@ func (kvSS *kvBatchSnapshotStrategy) Send(
428422
}
429423

430424
var valBuf []byte
431-
if sharedReplicate || externalReplicate {
425+
// If snapshots containing shared files are allowed, and this range is a
426+
// non-system range, take advantage of shared storage to minimize the amount
427+
// of data we're iterating on and sending over the network.
428+
if header.SharedReplicate || header.ExternalReplicate {
432429
var sharedVisitor func(sst *pebble.SharedSSTMeta) error
433-
if sharedReplicate {
430+
if header.SharedReplicate {
434431
sharedVisitor = func(sst *pebble.SharedSSTMeta) error {
435432
sharedSSTCount++
436433
snap.sharedBackings = append(snap.sharedBackings, sst.Backing)
@@ -459,7 +456,7 @@ func (kvSS *kvBatchSnapshotStrategy) Send(
459456
}
460457
}
461458
var externalVisitor func(sst *pebble.ExternalFile) error
462-
if externalReplicate {
459+
if header.ExternalReplicate {
463460
externalVisitor = func(sst *pebble.ExternalFile) error {
464461
externalSSTCount++
465462
externalSSTs = append(externalSSTs, kvserverpb.SnapshotRequest_ExternalTable{

pkg/kv/kvserver/rditer/replica_data_iter.go

Lines changed: 29 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -454,23 +454,45 @@ func IterateReplicaKeySpans(
454454
}
455455

456456
// IterateReplicaKeySpansShared is a shared-replicate version of
457-
// IterateReplicaKeySpans. See definitions of this method for how it is
458-
// implemented.
457+
// IterateReplicaKeySpans. IterateReplicaKeySpansShared iterates over the
458+
// range's user key span, skipping any keys present in shared files. It calls
459+
// the appropriate visitor function for the type of key visited, namely, point
460+
// keys, range deletes and range keys. Shared files that are skipped during this
461+
// iteration are also surfaced through a dedicated visitor. Note that this
462+
// method only iterates over a range's user key span; IterateReplicaKeySpans
463+
// must be called to iterate over the other key spans.
459464
//
460-
// The impl of this method along with a comment is in
461-
// engineccl/shared_storage.go.
462-
var IterateReplicaKeySpansShared func(
465+
// If this method returns pebble.ErrInvalidSkipSharedIteration, only the shared
466+
// external visitors may have been invoked. In particular, no local data has
467+
// been visited yet. The above contract appears true for the current
468+
// implementation of this method, but is likely untested.
469+
//
470+
// Must use a reader with consistent iterators.
471+
func IterateReplicaKeySpansShared(
463472
ctx context.Context,
464473
desc *roachpb.RangeDescriptor,
465474
st *cluster.Settings,
466-
clusterID uuid.UUID,
475+
_ uuid.UUID,
467476
reader storage.Reader,
468477
visitPoint func(key *pebble.InternalKey, val pebble.LazyValue, info pebble.IteratorLevel) error,
469478
visitRangeDel func(start, end []byte, seqNum pebble.SeqNum) error,
470479
visitRangeKey func(start, end []byte, keys []rangekey.Key) error,
471480
visitSharedFile func(sst *pebble.SharedSSTMeta) error,
472481
visitExternalFile func(sst *pebble.ExternalFile) error,
473-
) error
482+
) error {
483+
if !reader.ConsistentIterators() {
484+
panic("reader must provide consistent iterators")
485+
}
486+
spans := Select(desc.RangeID, SelectOpts{
487+
Ranged: SelectRangedOptions{
488+
RSpan: desc.RSpan(),
489+
UserKeys: true,
490+
},
491+
})
492+
span := spans[0]
493+
return reader.ScanInternal(ctx, span.Key, span.EndKey, visitPoint, visitRangeDel,
494+
visitRangeKey, visitSharedFile, visitExternalFile)
495+
}
474496

475497
// IterateOptions instructs how points and ranges should be presented to visitor
476498
// and if iterators should be visited in forward or reverse order.

0 commit comments

Comments
 (0)