Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
85759: .*: change pebble compaction concurrency w/o restart r=bananabrick a=bananabrick

During support issues with inverted lsms, we want the ability to easily
change compaction concurrency to de-invert the lsm. This pr provides
a simple sql command to temporarily change the compaction concurrency
of a store. The compaction concurrency is changed until the sql command
is cancelled.

Release Note: None
Release justification: Useful ops tool.

86157: ui: preserve time window selection on window reload r=koorosh a=koorosh

This patch fixes recently introduced regression when
selected time window on SQL Activity or Metrics pages
was not preserved after window reload.
Now, every time timescale is changed, it's preserved
in session storage and restored on initialization of
TimeScale reducer.

Release note: None

Release justification: low risk, high benefit changes to existing functionality

Resolves: cockroachdb#85815

86166: vendor: bump Pebble to 94bbf14378db r=erikgrinaker,nicktrav a=jbowens

```
94bbf143 db: invalidate RangeKeyChanged state in SetOptions
0b6ec868 sstable: pool range key fragment iterators
be796cfc db: handle Next on exhausted prefix iterator
```

Release note: None
Release justification: Fix an iterator reuse corruption bug, reduce mvcc scan performance regression.

Close cockroachdb#86131.
Informs cockroachdb/pebble#1886.

Co-authored-by: Arjun Nair <[email protected]>
Co-authored-by: Andrii Vorobiov <[email protected]>
Co-authored-by: Jackson Owens <[email protected]>
  • Loading branch information
4 people committed Aug 15, 2022
4 parents 3870f3a + 213da1f + 62a6b1e + 8bf2c8b commit 55e0f99
Show file tree
Hide file tree
Showing 23 changed files with 229 additions and 71 deletions.
6 changes: 3 additions & 3 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -1431,10 +1431,10 @@ def go_deps():
patches = [
"@com_github_cockroachdb_cockroach//build/patches:com_github_cockroachdb_pebble.patch",
],
sha256 = "08c72d014d11e94dba62660d804f68676b5dbf2172305ae013a4864c5fd30d2e",
strip_prefix = "github.com/cockroachdb/[email protected]20220813213520-4c5e752da1b8",
sha256 = "855751657ff6d4aa229474e378abb4417b7ece06ca08f1f12ed842553f372497",
strip_prefix = "github.com/cockroachdb/[email protected]20220815175543-94bbf14378db",
urls = [
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220813213520-4c5e752da1b8.zip",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220815175543-94bbf14378db.zip",
],
)
go_repository(
Expand Down
2 changes: 1 addition & 1 deletion build/bazelutil/distdir_files.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -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-20220813213520-4c5e752da1b8.zip": "08c72d014d11e94dba62660d804f68676b5dbf2172305ae013a4864c5fd30d2e",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220815175543-94bbf14378db.zip": "855751657ff6d4aa229474e378abb4417b7ece06ca08f1f12ed842553f372497",
"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",
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -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-20220813213520-4c5e752da1b8
github.com/cockroachdb/pebble v0.0.0-20220815175543-94bbf14378db
github.com/cockroachdb/redact v1.1.3
github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd
github.com/cockroachdb/stress v0.0.0-20220803192808-1806698b1b7b
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -467,8 +467,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-20220813213520-4c5e752da1b8 h1:on483Ok6H4Bh4BHrSc+LKlmqouIj8L/f9+U8t1bVKcs=
github.com/cockroachdb/pebble v0.0.0-20220813213520-4c5e752da1b8/go.mod h1:890yq1fUb9b6dGNwssgeUO5vQV9qfXnCPxAJhBQfXw0=
github.com/cockroachdb/pebble v0.0.0-20220815175543-94bbf14378db h1:1Yk1skUQZ4Owkq6kHXJHvmgi0ao58jLS3615GONojwk=
github.com/cockroachdb/pebble v0.0.0-20220815175543-94bbf14378db/go.mod h1:890yq1fUb9b6dGNwssgeUO5vQV9qfXnCPxAJhBQfXw0=
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=
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@ go_library(
name = "kvserver",
srcs = [
"addressing.go",
"compact_span_client.go",
"consistency_queue.go",
"debug_print.go",
"doc.go",
Expand Down Expand Up @@ -79,6 +78,7 @@ go_library(
"split_delay_helper.go",
"split_queue.go",
"split_trigger_helper.go",
"storage_engine_client.go",
"store.go",
"store_create_replica.go",
"store_init.go",
Expand Down
10 changes: 10 additions & 0 deletions pkg/kv/kvserver/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -85,3 +85,13 @@ message CompactEngineSpanRequest {

message CompactEngineSpanResponse {
}

// CompactionConcurrencyRequest increases the compaction concurrency of the store
// until the request is cancelled.
message CompactionConcurrencyRequest {
StoreRequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true];
uint64 compaction_concurrency = 2;
}

message CompactionConcurrencyResponse {
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,19 +19,18 @@ import (
"github.com/cockroachdb/errors"
)

// CompactEngineSpanClient is used to request compaction for a span
// of data on a store.
type CompactEngineSpanClient struct {
// StorageEngineClient is used to connect and make requests to a store.
type StorageEngineClient struct {
nd *nodedialer.Dialer
}

// NewCompactEngineSpanClient constructs a new CompactEngineSpanClient.
func NewCompactEngineSpanClient(nd *nodedialer.Dialer) *CompactEngineSpanClient {
return &CompactEngineSpanClient{nd: nd}
// NewStorageEngineClient constructs a new StorageEngineClient.
func NewStorageEngineClient(nd *nodedialer.Dialer) *StorageEngineClient {
return &StorageEngineClient{nd: nd}
}

// CompactEngineSpan is a tree.CompactEngineSpanFunc.
func (c *CompactEngineSpanClient) CompactEngineSpan(
func (c *StorageEngineClient) CompactEngineSpan(
ctx context.Context, nodeID, storeID int32, startKey, endKey []byte,
) error {
conn, err := c.nd.Dial(ctx, roachpb.NodeID(nodeID), rpc.DefaultClass)
Expand All @@ -49,3 +48,26 @@ func (c *CompactEngineSpanClient) CompactEngineSpan(
_, err = client.CompactEngineSpan(ctx, req)
return err
}

// SetCompactionConcurrency is a tree.CompactionConcurrencyFunc.
func (c *StorageEngineClient) SetCompactionConcurrency(
ctx context.Context, nodeID, storeID int32, compactionConcurrency uint64,
) error {
conn, err := c.nd.Dial(ctx, roachpb.NodeID(nodeID), rpc.DefaultClass)
if err != nil {
return errors.Wrapf(err, "could not dial node ID %d", nodeID)
}
client := NewPerStoreClient(conn)
req := &CompactionConcurrencyRequest{
StoreRequestHeader: StoreRequestHeader{
NodeID: roachpb.NodeID(nodeID),
StoreID: roachpb.StoreID(storeID),
},
CompactionConcurrency: compactionConcurrency,
}
_, err = client.SetCompactionConcurrency(ctx, req)
if err != nil {
return err
}
return nil
}
1 change: 1 addition & 0 deletions pkg/kv/kvserver/storage_services.proto
Original file line number Diff line number Diff line change
Expand Up @@ -39,4 +39,5 @@ service PerReplica {

service PerStore {
rpc CompactEngineSpan(cockroach.kv.kvserver.CompactEngineSpanRequest) returns (cockroach.kv.kvserver.CompactEngineSpanResponse) {}
rpc SetCompactionConcurrency(cockroach.kv.kvserver.CompactionConcurrencyRequest) returns (cockroach.kv.kvserver.CompactionConcurrencyResponse) {}
}
21 changes: 21 additions & 0 deletions pkg/kv/kvserver/stores_server.go
Original file line number Diff line number Diff line change
Expand Up @@ -166,3 +166,24 @@ func (is Server) CompactEngineSpan(
})
return resp, err
}

// SetCompactionConcurrency implements PerStoreServer. It changes the compaction
// concurrency of a store. While SetCompactionConcurrency is safe for concurrent
// use, it adds uncertainty about the compaction concurrency actually set on
// the store. It also adds uncertainty about the compaction concurrency set on
// the store once the request is cancelled.
func (is Server) SetCompactionConcurrency(
ctx context.Context, req *CompactionConcurrencyRequest,
) (*CompactionConcurrencyResponse, error) {
resp := &CompactionConcurrencyResponse{}
err := is.execStoreCommand(ctx, req.StoreRequestHeader,
func(ctx context.Context, s *Store) error {
prevConcurrency := s.Engine().SetCompactionConcurrency(req.CompactionConcurrency)

// Wait for cancellation, and once cancelled, reset the compaction concurrency.
<-ctx.Done()
s.Engine().SetCompactionConcurrency(prevConcurrency)
return nil
})
return resp, err
}
1 change: 0 additions & 1 deletion pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,6 @@ go_library(
"//pkg/util/buildutil",
"//pkg/util/contextutil",
"//pkg/util/envutil",
"//pkg/util/errorutil",
"//pkg/util/goschedstats",
"//pkg/util/grpcutil",
"//pkg/util/hlc",
Expand Down
88 changes: 39 additions & 49 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
Expand Down Expand Up @@ -578,18 +577,6 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {

gcJobNotifier := gcjobnotifier.New(cfg.Settings, cfg.systemConfigWatcher, codec, cfg.stopper)

var compactEngineSpanFunc eval.CompactEngineSpanFunc
if !codec.ForSystemTenant() {
compactEngineSpanFunc = func(
ctx context.Context, nodeID, storeID int32, startKey, endKey []byte,
) error {
return errorutil.UnsupportedWithMultiTenancy(errorutil.FeatureNotAvailableToNonSystemTenantsIssue)
}
} else {
cli := kvserver.NewCompactEngineSpanClient(cfg.nodeDialer)
compactEngineSpanFunc = cli.CompactEngineSpan
}

spanConfig := struct {
manager *spanconfigmanager.Manager
sqlTranslatorFactory *spanconfigsqltranslator.Factory
Expand Down Expand Up @@ -761,43 +748,46 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
)
contentionRegistry.Start(ctx, cfg.stopper)

storageEngineClient := kvserver.NewStorageEngineClient(cfg.nodeDialer)

*execCfg = sql.ExecutorConfig{
Settings: cfg.Settings,
NodeInfo: nodeInfo,
Codec: codec,
DefaultZoneConfig: &cfg.DefaultZoneConfig,
Locality: cfg.Locality,
AmbientCtx: cfg.AmbientCtx,
DB: cfg.db,
Gossip: cfg.gossip,
NodeLiveness: cfg.nodeLiveness,
SystemConfig: cfg.systemConfigWatcher,
MetricsRecorder: cfg.recorder,
DistSender: cfg.distSender,
RPCContext: cfg.rpcContext,
LeaseManager: leaseMgr,
TenantStatusServer: cfg.tenantStatusServer,
Clock: cfg.clock,
DistSQLSrv: distSQLServer,
NodesStatusServer: cfg.nodesStatusServer,
SQLStatusServer: cfg.sqlStatusServer,
RegionsServer: cfg.regionsServer,
SessionRegistry: cfg.sessionRegistry,
ClosedSessionCache: cfg.closedSessionCache,
ContentionRegistry: contentionRegistry,
SQLLiveness: cfg.sqlLivenessProvider,
JobRegistry: jobRegistry,
VirtualSchemas: virtualSchemas,
HistogramWindowInterval: cfg.HistogramWindowInterval(),
RangeDescriptorCache: cfg.distSender.RangeDescriptorCache(),
RoleMemberCache: sql.NewMembershipCache(serverCacheMemoryMonitor.MakeBoundAccount(), cfg.stopper),
SessionInitCache: sessioninit.NewCache(serverCacheMemoryMonitor.MakeBoundAccount(), cfg.stopper),
RootMemoryMonitor: rootSQLMemoryMonitor,
TestingKnobs: sqlExecutorTestingKnobs,
CompactEngineSpanFunc: compactEngineSpanFunc,
TraceCollector: traceCollector,
TenantUsageServer: cfg.tenantUsageServer,
KVStoresIterator: cfg.kvStoresIterator,
Settings: cfg.Settings,
NodeInfo: nodeInfo,
Codec: codec,
DefaultZoneConfig: &cfg.DefaultZoneConfig,
Locality: cfg.Locality,
AmbientCtx: cfg.AmbientCtx,
DB: cfg.db,
Gossip: cfg.gossip,
NodeLiveness: cfg.nodeLiveness,
SystemConfig: cfg.systemConfigWatcher,
MetricsRecorder: cfg.recorder,
DistSender: cfg.distSender,
RPCContext: cfg.rpcContext,
LeaseManager: leaseMgr,
TenantStatusServer: cfg.tenantStatusServer,
Clock: cfg.clock,
DistSQLSrv: distSQLServer,
NodesStatusServer: cfg.nodesStatusServer,
SQLStatusServer: cfg.sqlStatusServer,
RegionsServer: cfg.regionsServer,
SessionRegistry: cfg.sessionRegistry,
ClosedSessionCache: cfg.closedSessionCache,
ContentionRegistry: contentionRegistry,
SQLLiveness: cfg.sqlLivenessProvider,
JobRegistry: jobRegistry,
VirtualSchemas: virtualSchemas,
HistogramWindowInterval: cfg.HistogramWindowInterval(),
RangeDescriptorCache: cfg.distSender.RangeDescriptorCache(),
RoleMemberCache: sql.NewMembershipCache(serverCacheMemoryMonitor.MakeBoundAccount(), cfg.stopper),
SessionInitCache: sessioninit.NewCache(serverCacheMemoryMonitor.MakeBoundAccount(), cfg.stopper),
RootMemoryMonitor: rootSQLMemoryMonitor,
TestingKnobs: sqlExecutorTestingKnobs,
CompactEngineSpanFunc: storageEngineClient.CompactEngineSpan,
CompactionConcurrencyFunc: storageEngineClient.SetCompactionConcurrency,
TraceCollector: traceCollector,
TenantUsageServer: cfg.tenantUsageServer,
KVStoresIterator: cfg.kvStoresIterator,
SyntheticPrivilegeCache: cacheutil.NewCache(
serverCacheMemoryMonitor.MakeBoundAccount(), cfg.stopper, 1 /* numSystemTables */),

Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -1287,6 +1287,10 @@ type ExecutorConfig struct {
// perform compaction over a key span.
CompactEngineSpanFunc eval.CompactEngineSpanFunc

// CompactionConcurrencyFunc is used to inform a storage engine to change its
// compaction concurrency.
CompactionConcurrencyFunc eval.SetCompactionConcurrencyFunc

// TraceCollector is used to contact all live nodes in the cluster, and
// collect trace spans from their inflight node registries.
TraceCollector *collector.TraceCollector
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -4650,7 +4650,7 @@ FROM pg_proc p
JOIN pg_type t ON t.typinput = p.oid
WHERE t.typname = '_int4'
----
2012 array_in array_in
2013 array_in array_in

## #16285
## int2vectors should be 0-indexed
Expand Down Expand Up @@ -4688,7 +4688,7 @@ SELECT cur_max_builtin_oid FROM [SELECT max(oid) as cur_max_builtin_oid FROM pg_
query TT
SELECT proname, oid FROM pg_catalog.pg_proc WHERE oid = $cur_max_builtin_oid
----
to_regtype 2032
to_regtype 2033

## Ensure that unnest works with oid wrapper arrays

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,7 @@ func (evalCtx *extendedEvalContext) copyFromExecCfg(execCfg *ExecutorConfig) {
evalCtx.Tracer = execCfg.AmbientCtx.Tracer
evalCtx.SQLLivenessReader = execCfg.SQLLiveness
evalCtx.CompactEngineSpan = execCfg.CompactEngineSpanFunc
evalCtx.SetCompactionConcurrency = execCfg.CompactionConcurrencyFunc
evalCtx.TestingKnobs = execCfg.EvalContextTestingKnobs
evalCtx.ClusterID = execCfg.NodeInfo.LogicalClusterID()
evalCtx.ClusterName = execCfg.RPCContext.ClusterName()
Expand Down
44 changes: 44 additions & 0 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -7205,6 +7205,50 @@ that has execution latency greater than the 'minExecutionLatency'. If the
expires until the statement bundle is collected`,
},
),

"crdb_internal.set_compaction_concurrency": makeBuiltin(
tree.FunctionProperties{
Category: builtinconstants.CategorySystemRepair,
DistsqlBlocklist: true,
Undocumented: true,
},
tree.Overload{
Types: tree.ArgTypes{
{"node_id", types.Int},
{"store_id", types.Int},
{"compaction_concurrency", types.Int},
},
ReturnType: tree.FixedReturnType(types.Bool),
Fn: func(ctx *eval.Context, args tree.Datums) (tree.Datum, error) {
isAdmin, err := ctx.SessionAccessor.HasAdminRole(ctx.Context)
if err != nil {
return nil, err
}
if !isAdmin {
return nil, errInsufficientPriv
}
nodeID := int32(tree.MustBeDInt(args[0]))
storeID := int32(tree.MustBeDInt(args[1]))
compactionConcurrency := tree.MustBeDInt(args[2])
if compactionConcurrency <= 0 {
return nil, errors.AssertionFailedf("compaction_concurrency must be > 0")
}
if err = ctx.SetCompactionConcurrency(
ctx.Context, nodeID, storeID, uint64(compactionConcurrency)); err != nil {
return nil, err
}
return tree.DBoolTrue, nil
},
Info: "This function can be used to temporarily change the compaction concurrency of a " +
"given node and store. " +
"To change the compaction concurrency of a store one can do: " +
"SELECT crdb_internal.set_compaction_concurrency(<node_id>, <store_id>, <compaction_concurrency>). " +
"The store's compaction concurrency will change until the sql command is cancelled. Once cancelled " +
"the store's compaction concurrency will return to what it was previously. This command isn't safe " +
"for concurrent use.",
Volatility: volatility.Volatile,
},
),
}

var lengthImpls = func(incBitOverload bool) builtinDefinition {
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/sem/eval/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,6 +198,10 @@ type Context struct {
// CompactEngineSpan is used to force compaction of a span in a store.
CompactEngineSpan CompactEngineSpanFunc

// SetCompactionConcurrency is used to change the compaction concurrency of
// a store.
SetCompactionConcurrency SetCompactionConcurrencyFunc

// KVStoresIterator is used by various crdb_internal builtins to directly
// access stores on this node.
KVStoresIterator kvserverbase.StoresIterator
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/sem/eval/deps.go
Original file line number Diff line number Diff line change
Expand Up @@ -384,6 +384,12 @@ type CompactEngineSpanFunc func(
ctx context.Context, nodeID, storeID int32, startKey, endKey []byte,
) error

// SetCompactionConcurrencyFunc is used to change the compaction concurrency of a
// store.
type SetCompactionConcurrencyFunc func(
ctx context.Context, nodeID, storeID int32, compactionConcurrency uint64,
) error

// SessionAccessor is a limited interface to access session variables.
type SessionAccessor interface {
// SetSessionVar sets a session variable to a new value. If isLocal is true,
Expand Down
4 changes: 4 additions & 0 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -945,6 +945,10 @@ type Engine interface {
// calls to this method. Hence, this should be used with care, with only one
// caller, which is currently the admission control subsystem.
GetInternalIntervalMetrics() *pebble.InternalIntervalMetrics

// SetCompactionConcurrency is used to set the engine's compaction
// concurrency. It returns the previous compaction concurrency.
SetCompactionConcurrency(n uint64) uint64
}

// Batch is the interface for batch specific operations.
Expand Down
Loading

0 comments on commit 55e0f99

Please sign in to comment.