Skip to content

Commit

Permalink
kvserver: use SystemVisible for kv.raft.command.max_size
Browse files Browse the repository at this point in the history
Epic: none
Release note: None
  • Loading branch information
erikgrinaker committed Oct 3, 2023
1 parent ef2fd68 commit c515e73
Show file tree
Hide file tree
Showing 6 changed files with 29 additions and 6 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/kvserverbase/base.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,7 +298,7 @@ const (

// MaxCommandSize wraps "kv.raft.command.max_size".
var MaxCommandSize = settings.RegisterByteSizeSetting(
settings.ApplicationLevel,
settings.SystemVisible, // used by SQL/bulk to determine mutation batch sizes
"kv.raft.command.max_size",
"maximum size of a raft command",
MaxCommandSizeDefault,
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/copy/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ go_test(
"//pkg/base",
"//pkg/cli/clisqlclient",
"//pkg/kv/kvpb",
"//pkg/kv/kvserver/kvserverbase",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/security/username",
Expand Down
11 changes: 7 additions & 4 deletions pkg/sql/copy/copy_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/apd/v3"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/cli/clisqlclient"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -591,8 +592,9 @@ func TestLargeDynamicRows(t *testing.T) {
require.NoError(t, err)

// 4.0 MiB is minimum, copy sets max row size to this value / 3
err = conn.Exec(ctx, "SET CLUSTER SETTING kv.raft.command.max_size = '4.0MiB'")
require.NoError(t, err)
for _, l := range []serverutils.ApplicationLayerInterface{s, s.SystemLayer()} {
kvserverbase.MaxCommandSize.Override(ctx, &l.ClusterSettings().SV, 4<<20)
}

err = conn.Exec(ctx, "CREATE TABLE t (s STRING)")
require.NoError(t, err)
Expand All @@ -611,8 +613,9 @@ func TestLargeDynamicRows(t *testing.T) {
batchNumber = 0

// Reset and make sure we use 1 batch.
err = conn.Exec(ctx, "RESET CLUSTER SETTING kv.raft.command.max_size")
require.NoError(t, err)
for _, l := range []serverutils.ApplicationLayerInterface{s, s.SystemLayer()} {
kvserverbase.MaxCommandSize.Override(ctx, &l.ClusterSettings().SV, kvserverbase.MaxCommandSizeDefault)
}

// This won't work if the batch size gets set to less than 5. When the batch
// size is 4, the test hook will count an extra empty batch.
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/importer/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keyvisualizer"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
Expand Down Expand Up @@ -7122,7 +7123,9 @@ func TestImportRowErrorLargeRows(t *testing.T) {
defer tc.Stopper().Stop(ctx)
sqlDB := sqlutils.MakeSQLRunner(connDB)
// Our input file has an 8MB row
sqlDB.Exec(t, `SET CLUSTER SETTING kv.raft.command.max_size = '4MiB'`)
for _, l := range []serverutils.ApplicationLayerInterface{tc.Server(0), tc.Server(0).SystemLayer()} {
kvserverbase.MaxCommandSize.Override(ctx, &l.ClusterSettings().SV, 4<<20)
}
sqlDB.Exec(t, `CREATE DATABASE foo; SET DATABASE = foo`)
sqlDB.Exec(t, "CREATE TABLE simple (s string)")
defer sqlDB.Exec(t, "DROP table simple")
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/upsert
Original file line number Diff line number Diff line change
Expand Up @@ -1213,9 +1213,13 @@ statement ok
DROP TABLE target

# Regression test for UPSERT batching logic (#51391).
user host-cluster-root

statement ok
SET CLUSTER SETTING kv.raft.command.max_size='4MiB';

user root

statement ok
CREATE TABLE src (s STRING);
CREATE TABLE dest (s STRING);
Expand All @@ -1230,9 +1234,13 @@ FROM
statement ok
UPSERT INTO dest (s) (SELECT s FROM src)

user host-cluster-root

statement ok
RESET CLUSTER SETTING kv.raft.command.max_size;

user root

statement ok
DROP TABLE src;
DROP TABLE dest
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/upsert_non_metamorphic
Original file line number Diff line number Diff line change
Expand Up @@ -7,9 +7,13 @@ CREATE TABLE src (s STRING);
CREATE TABLE dest (s STRING);
INSERT INTO src SELECT repeat('a', 100000) FROM generate_series(1, 60)

user host-cluster-root

statement ok
SET CLUSTER SETTING kv.raft.command.max_size='4MiB';

user root

statement ok
SET CLUSTER SETTING sql.mutations.mutation_batch_byte_size='1MiB';

Expand All @@ -21,9 +25,13 @@ UPSERT INTO dest (s) (SELECT s FROM src)
statement ok
RESET CLUSTER SETTING sql.mutations.mutation_batch_byte_size;

user host-cluster-root

statement ok
RESET CLUSTER SETTING kv.raft.command.max_size;

user root

statement ok
DROP TABLE src;
DROP TABLE dest

0 comments on commit c515e73

Please sign in to comment.