Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
77349: spanconfig: add Record constructor and validation r=adityamaru a=adityamaru

This change adds a constructor method that returns
a new `spanconfig.Record` and conditionally performs
some validation if the target is a SystemTarget.

Informs: #73727

Release note: None

Release justification: low-risk updates to new functionality

77608: optbuilder: do not create invalid casts when building COALESCE and IF r=mgartner a=mgartner

The optbuilder no longer creates invalid casts when building COALESCE
and IF expressions that have children with different types. Expressions
that previously caused internal errors now result in user-facing errors.
Both UNION and CASE expressions had similar bugs that were recently
fixed in #75219 and #76193.

This commit also updates the `tree.ReType` function to return `ok=false`
if there is no valid cast to re-type the expression to the given type.
This forces callers to explicitly deal with situations where re-typing
is not possible and it ensures that the function never creates invalid
casts. This will make it easier to track down future related bugs
because internal errors should originate from the call site of
`tree.ReType` rather than from logic further along in the optimization
process (in the case of #76807 the internal error originated from the
logical props builder when it attempted to lookup the volatility of the
invalid cast).

This commit also adds special logic to make casts from any tuple type to
`types.AnyTuple` valid immutable, implicit casts. Evaluation of these
casts are no-ops. Users cannot construct these casts, but they are built
by optbuilder in some cases.

Fixes #76807

Release justification: This is a low-risk change that fixes a minor bug.

Release note (bug fix): A bug has been fixed that caused internal errors
when COALESCE and IF expressions had inner expressions with different
types that could not be cast to a common type.


77632: ui: new plan table on statement details r=maryliag a=maryliag

Previously, the Explain Plan tab on Statement Details was
showing only one plan. This commit introduces a table of plan
with their respective executions stats.
When a plan is clicked on the table, it shows the Plan and
its statistics.

Fixes #72129

Page on DB Console: [video](https://www.loom.com/share/0898f48021eb4037a6f86760053a5e85)
Page on CC Console: [video](https://www.loom.com/share/84d8ec40ae7e4eb19291788721ab7133)

<img width="1058" alt="Screen Shot 2022-03-10 at 2 42 25 PM" src="https://user-images.githubusercontent.com/1017486/157742210-12b79f5d-274c-48e6-8fb6-dafc74fd25b3.png">
<img width="988" alt="Screen Shot 2022-03-10 at 2 42 36 PM" src="https://user-images.githubusercontent.com/1017486/157742211-daa2a07c-b025-4b36-a49a-4cafe7117bc8.png">



Release justification: Category 4
Release note (ui change): Explain Plan tab on Statement Details
shows statistics for all the plans executed by the selected statement
on the selected period.

77688: ci: ensure all nightlies post github issues when tests fail r=rail a=rickystewart

Release justification: ensure failing nightlies post github issues
Release note: None

Co-authored-by: Aditya Maru <adityamaru@gmail.com>
Co-authored-by: Marcus Gartner <marcus@cockroachlabs.com>
Co-authored-by: Marylia Gutierrez <marylia@cockroachlabs.com>
Co-authored-by: Ricky Stewart <ricky@cockroachlabs.com>
  • Loading branch information
5 people committed Mar 11, 2022
5 parents d68a48a + 99c9691 + e55e6b6 + a81c820 + 82e0b12 commit af747e7
Show file tree
Hide file tree
Showing 48 changed files with 1,158 additions and 391 deletions.
21 changes: 17 additions & 4 deletions build/teamcity/cockroach/nightlies/compose.sh
Original file line number Diff line number Diff line change
Expand Up @@ -4,22 +4,35 @@ set -xeuo pipefail

dir="$(dirname $(dirname $(dirname $(dirname "${0}"))))"
source "$dir/teamcity-support.sh"
source "$dir/teamcity-bazel-support.sh"

tc_start_block "Run compose tests"

bazel build //pkg/cmd/bazci --config=ci
BAZCI=$(bazel info bazel-bin --config=ci)/pkg/cmd/bazci/bazci_/bazci
bazel build //pkg/cmd/bazci //pkg/cmd/github-post //pkg/cmd/testfilter --config=ci
BAZEL_BIN=$(bazel info bazel-bin --config=ci)
BAZCI=$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci

bazel build //pkg/cmd/cockroach //pkg/compose/compare/compare:compare_test --config=ci --config=crosslinux --config=test --config=with_ui
CROSSBIN=$(bazel info bazel-bin --config=ci --config=crosslinux --config=test --config=with_ui)
COCKROACH=$CROSSBIN/pkg/cmd/cockroach/cockroach_/cockroach
COMPAREBIN=$(bazel run //pkg/compose/compare/compare:compare_test --config=ci --config=crosslinux --config=test --config=with_ui --run_under=realpath | grep '^/' | tail -n1)
ARTIFACTS_DIR=$PWD/artifacts

$BAZCI run --config=ci --config=test --artifacts_dir=$PWD/artifacts \
GO_TEST_JSON_OUTPUT_FILE=$PWD/artifacts/test.json.txt
exit_status=0
$BAZCI run --config=ci --config=test --artifacts_dir=$ARTIFACTS_DIR \
//pkg/compose:compose_test -- \
--test_env=GO_TEST_WRAP_TESTV=1 \
--test_env=GO_TEST_JSON_OUTPUT_FILE=$GO_TEST_JSON_OUTPUT_FILE \
--test_arg -cockroach --test_arg $COCKROACH \
--test_arg -compare --test_arg $COMPAREBIN \
--test_timeout=1800
--test_timeout=1800 || exit_status=$?
process_test_json \
$BAZEL_BIN/pkg/cmd/testfilter/testfilter_/testfilter \
$BAZEL_BIN/pkg/cmd/github-post/github-post_/github-post \
$ARTIFACTS_DIR \
$GO_TEST_JSON_OUTPUT_FILE \
$exit_status

tc_end_block "Run compose tests"
exit $exit_status
19 changes: 16 additions & 3 deletions build/teamcity/cockroach/nightlies/lint_urls_impl.sh
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,23 @@

set -xeuo pipefail

bazel build //pkg/cmd/bazci --config=ci
dir="$(dirname $(dirname $(dirname $(dirname "${0}"))))"
source "$dir/teamcity-bazel-support.sh" # For process_test_json

bazel build //pkg/cmd/bazci //pkg/cmd/github-post //pkg/cmd/testfilter --config=ci
BAZEL_BIN=$(bazel info bazel-bin --config=ci)
GO_TEST_JSON_OUTPUT_FILE=/artifacts/test.json.txt
exit_status=0
XML_OUTPUT_FILE=/artifacts/test.xml GO_TEST_WRAP_TESTV=1 GO_TEST_WRAP=1 bazel \
run --config=ci --config=test --define gotags=bazel,gss,nightly \
//build/bazelutil:lint
//build/bazelutil:lint || exit_status=$?
# The schema of the output test.xml will be slightly wrong -- ask `bazci` to fix
# it up.
$(bazel info bazel-bin --config=ci)/pkg/cmd/bazci/bazci_/bazci munge-test-xml /artifacts/test.xml
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci munge-test-xml /artifacts/test.xml
process_test_json \
$BAZEL_BIN/pkg/cmd/testfilter/testfilter_/testfilter \
$BAZEL_BIN/pkg/cmd/github-post/github-post_/github-post \
/artifacts \
$GO_TEST_JSON_OUTPUT_FILE \
$exit_status
exit $exit_status
45 changes: 37 additions & 8 deletions build/teamcity/cockroach/nightlies/optimizer_tests_impl.sh
Original file line number Diff line number Diff line change
Expand Up @@ -3,26 +3,55 @@
set -xeuo pipefail

dir="$(dirname $(dirname $(dirname $(dirname "${0}"))))"
source "$dir/teamcity-bazel-support.sh"
source "$dir/teamcity/util.sh"

bazel build //pkg/cmd/bazci --config=ci
bazel build //pkg/cmd/bazci //pkg/cmd/github-post //pkg/cmd/testfilter --config=ci
BAZEL_BIN=$(bazel info bazel-bin --config=ci)

tc_start_block "Run opt tests with fast_int_set_large"
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci --config=ci \
ARTIFACTS_DIR=/artifacts/fast_int_set_large
mkdir $ARTIFACTS_DIR
GO_TEST_JSON_OUTPUT_FILE=$ARTIFACTS_DIR/test.json.txt
exit_status_large=0
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci --config=ci --artifacts $ARTIFACTS_DIR \
test //pkg/sql/opt:opt_test -- \
--define gotags=bazel,crdb_test,fast_int_set_large
mkdir /artifacts/fast_int_set_large
for FILE in $(ls artifacts | grep -v '^fast_int_set_large$'); do mv /artifacts/$FILE /artifacts/fast_int_set_large; done
--define gotags=bazel,crdb_test,fast_int_set_large \
--test_env=GO_TEST_JSON_OUTPUT_FILE=$GO_TEST_JSON_OUTPUT_FILE || $exit_status_large=$?
process_test_json \
$BAZEL_BIN/pkg/cmd/testfilter/testfilter_/testfilter \
$BAZEL_BIN/pkg/cmd/github-post/github-post_/github-post \
$ARTIFACTS_DIR \
$GO_TEST_JSON_OUTPUT_FILE \
$exit_status_large
tc_end_block "Run opt tests with fast_int_set_large"

# NOTE(ricky): Running both tests in the same configuration with different
# gotags thrashes the cache. These tests are pretty quick so it shouldn't
# matter now but it is something to keep an eye on.
tc_start_block "Run opt tests with fast_int_set_small"
ARTIFACTS_DIR=/artifacts/fast_int_set_small
mkdir $ARTIFACTS_DIR
GO_TEST_JSON_OUTPUT_FILE=$ARTIFACTS_DIR/test.json.txt
exit_status_small=0
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci --config=ci \
test //pkg/sql/opt:opt_test -- \
--define gotags=bazel,crdb_test,fast_int_set_small
mkdir /artifacts/fast_int_set_small
for FILE in $(ls artifacts | grep -v '^fast_int_set'); do mv /artifacts/$FILE /artifacts/fast_int_set_small; done
--define gotags=bazel,crdb_test,fast_int_set_small \
--test_env=GO_TEST_JSON_OUTPUT_FILE=$GO_TEST_JSON_OUTPUT_FILE || $exit_status_small=$?
process_test_json \
$BAZEL_BIN/pkg/cmd/testfilter/testfilter_/testfilter \
$BAZEL_BIN/pkg/cmd/github-post/github-post_/github-post \
$ARTIFACTS_DIR \
$GO_TEST_JSON_OUTPUT_FILE \
$exit_status_large
tc_end_block "Run opt tests with fast_int_set_small"

if [ $exit_status_large -ne 0 ]
then
exit $exit_status_large
fi

if [ $exit_status_small -ne 0 ]
then
exit $exit_status_small
fi
17 changes: 15 additions & 2 deletions build/teamcity/cockroach/nightlies/random_syntax_tests_impl.sh
Original file line number Diff line number Diff line change
Expand Up @@ -2,9 +2,22 @@

set -xeuo pipefail

bazel build //pkg/cmd/bazci --config=ci
dir="$(dirname $(dirname $(dirname $(dirname "${0}"))))"
source "$dir/teamcity-bazel-support.sh"

bazel build //pkg/cmd/bazci //pkg/cmd/github-post //pkg/cmd/testfilter --config=ci
BAZEL_BIN=$(bazel info bazel-bin --config=ci)
GO_TEST_JSON_OUTPUT_FILE=/artifacts/test.json.txt
exit_status=0
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci --config=ci \
test //pkg/sql/tests:tests_test -- \
--test_arg -rsg=5m --test_arg -rsg-routines=8 --test_arg -rsg-exec-timeout=1m \
--test_timeout 3600 --test_filter 'TestRandomSyntax'
--test_timeout 3600 --test_filter 'TestRandomSyntax' \
--test_env=GO_TEST_JSON_OUTPUT_FILE=$GO_TEST_JSON_OUTPUT_FILE || exit_status=$?
process_test_json \
$BAZEL_BIN/pkg/cmd/testfilter/testfilter_/testfilter \
$BAZEL_BIN/pkg/cmd/github-post/github-post_/github-post \
/artifacts \
$GO_TEST_JSON_OUTPUT_FILE \
$exit_status
exit $exit_status
17 changes: 15 additions & 2 deletions build/teamcity/cockroach/nightlies/sqlite_logic_test_impl.sh
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,23 @@

set -xeuo pipefail

bazel build //pkg/cmd/bazci --config=ci
dir="$(dirname $(dirname $(dirname $(dirname "${0}"))))"
source "$dir/teamcity-bazel-support.sh"

bazel build //pkg/cmd/bazci //pkg/cmd/github-post //pkg/cmd/testfilter --config=ci
BAZEL_BIN=$(bazel info bazel-bin --config=ci)
GO_TEST_JSON_OUTPUT_FILE=/artifacts/test.json.txt
exit_status=0
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci --config=ci \
test //pkg/sql/logictest:logictest_test -- \
--test_arg -bigtest --test_arg -flex-types \
--define gotags=bazel,crdb_test_off --test_timeout 86400 \
--test_filter '^TestSqlLiteLogic$|^TestTenantSQLLiteLogic$'
--test_filter '^TestSqlLiteLogic$|^TestTenantSQLLiteLogic$' \
--test_env=GO_TEST_JSON_OUTPUT_FILE=$GO_TEST_JSON_OUTPUT_FILE || $exit_status=$?
process_test_json \
$BAZEL_BIN/pkg/cmd/testfilter/testfilter_/testfilter \
$BAZEL_BIN/pkg/cmd/github-post/github-post_/github-post \
/artifacts \
$GO_TEST_JSON_OUTPUT_FILE \
$exit_status
exit $exit_status
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ func TestPreSeedSpanConfigsWrittenWhenActive(t *testing.T) {
})
require.NoError(t, err)
require.Len(t, records, 1)
require.Equal(t, records[0].Target.GetSpan(), tenantSeedSpan)
require.Equal(t, records[0].GetTarget().GetSpan(), tenantSeedSpan)
}
}

Expand Down Expand Up @@ -144,7 +144,7 @@ func TestSeedTenantSpanConfigs(t *testing.T) {
})
require.NoError(t, err)
require.Len(t, records, 1)
require.Equal(t, records[0].Target.GetSpan(), tenantSeedSpan)
require.Equal(t, records[0].GetTarget().GetSpan(), tenantSeedSpan)
}
}

Expand Down Expand Up @@ -200,7 +200,7 @@ func TestSeedTenantSpanConfigsWithExistingEntry(t *testing.T) {
})
require.NoError(t, err)
require.Len(t, records, 1)
require.Equal(t, records[0].Target.GetSpan(), tenantSeedSpan)
require.Equal(t, records[0].GetTarget().GetSpan(), tenantSeedSpan)
}

// Ensure the cluster version bump goes through successfully.
Expand All @@ -215,6 +215,6 @@ func TestSeedTenantSpanConfigsWithExistingEntry(t *testing.T) {
})
require.NoError(t, err)
require.Len(t, records, 1)
require.Equal(t, records[0].Target.GetSpan(), tenantSeedSpan)
require.Equal(t, records[0].GetTarget().GetSpan(), tenantSeedSpan)
}
}
14 changes: 7 additions & 7 deletions pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,18 +208,18 @@ func TestDataDriven(t *testing.T) {
)
require.NoError(t, err)
sort.Slice(records, func(i, j int) bool {
return records[i].Target.Less(records[j].Target)
return records[i].GetTarget().Less(records[j].GetTarget())
})

lines := make([]string, len(records))
for i, record := range records {
switch {
case record.Target.IsSpanTarget():
lines[i] = fmt.Sprintf("%-42s %s", record.Target.GetSpan(),
spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.Config))
case record.Target.IsSystemTarget():
lines[i] = fmt.Sprintf("%-42s %s", record.Target.GetSystemTarget(),
spanconfigtestutils.PrintSystemSpanConfigDiffedAgainstDefault(record.Config))
case record.GetTarget().IsSpanTarget():
lines[i] = fmt.Sprintf("%-42s %s", record.GetTarget().GetSpan(),
spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.GetConfig()))
case record.GetTarget().IsSystemTarget():
lines[i] = fmt.Sprintf("%-42s %s", record.GetTarget().GetSystemTarget(),
spanconfigtestutils.PrintSystemSpanConfigDiffedAgainstDefault(record.GetConfig()))
default:
panic("unsupported target type")
}
Expand Down
20 changes: 10 additions & 10 deletions pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,18 +161,18 @@ func TestDataDriven(t *testing.T) {
records, _, err := sqlTranslator.Translate(ctx, descIDs, generateSystemSpanConfigs)
require.NoError(t, err)
sort.Slice(records, func(i, j int) bool {
return records[i].Target.Less(records[j].Target)
return records[i].GetTarget().Less(records[j].GetTarget())
})

var output strings.Builder
for _, record := range records {
switch {
case record.Target.IsSpanTarget():
output.WriteString(fmt.Sprintf("%-42s %s\n", record.Target.GetSpan(),
spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.Config)))
case record.Target.IsSystemTarget():
output.WriteString(fmt.Sprintf("%-42s %s\n", record.Target.GetSystemTarget(),
spanconfigtestutils.PrintSystemSpanConfigDiffedAgainstDefault(record.Config)))
case record.GetTarget().IsSpanTarget():
output.WriteString(fmt.Sprintf("%-42s %s\n", record.GetTarget().GetSpan(),
spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.GetConfig())))
case record.GetTarget().IsSystemTarget():
output.WriteString(fmt.Sprintf("%-42s %s\n", record.GetTarget().GetSystemTarget(),
spanconfigtestutils.PrintSystemSpanConfigDiffedAgainstDefault(record.GetConfig())))
default:
panic("unsupported target type")
}
Expand All @@ -185,12 +185,12 @@ func TestDataDriven(t *testing.T) {
require.NoError(t, err)

sort.Slice(records, func(i, j int) bool {
return records[i].Target.Less(records[j].Target)
return records[i].GetTarget().Less(records[j].GetTarget())
})
var output strings.Builder
for _, record := range records {
output.WriteString(fmt.Sprintf("%-42s %s\n", record.Target.GetSpan(),
spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.Config)))
output.WriteString(fmt.Sprintf("%-42s %s\n", record.GetTarget().GetSpan(),
spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.GetConfig())))
}
return output.String()

Expand Down
9 changes: 3 additions & 6 deletions pkg/ccl/testccl/sqlccl/tenant_gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,12 +70,9 @@ func TestGCTenantRemovesSpanConfigs(t *testing.T) {
// keyspace.
systemTarget, err := spanconfig.MakeTenantKeyspaceTarget(tenantID, tenantID)
require.NoError(t, err)
err = tenantKVAccessor.UpdateSpanConfigRecords(ctx, nil /* toDelete */, []spanconfig.Record{
{
Target: spanconfig.MakeTargetFromSystemTarget(systemTarget),
Config: roachpb.SpanConfig{}, // Doesn't matter
},
})
rec, err := spanconfig.MakeRecord(spanconfig.MakeTargetFromSystemTarget(systemTarget), roachpb.SpanConfig{})
require.NoError(t, err)
err = tenantKVAccessor.UpdateSpanConfigRecords(ctx, nil /* toDelete */, []spanconfig.Record{rec})
require.NoError(t, err)

// Ensure there are 2 configs for the tenant -- one that spans its entire
Expand Down
9 changes: 6 additions & 3 deletions pkg/kv/kvserver/client_spanconfigs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,15 +70,18 @@ func TestSpanConfigUpdateAppliedToReplica(t *testing.T) {
span := repl.Desc().RSpan().AsRawSpanWithNoLocals()
conf := roachpb.SpanConfig{NumReplicas: 5, NumVoters: 3}

add, err := spanconfig.Addition(spanconfig.MakeTargetFromSpan(span), conf)
require.NoError(t, err)
deleted, added := spanConfigStore.Apply(
ctx,
false, /* dryrun */
spanconfig.Addition(spanconfig.MakeTargetFromSpan(span), conf),
add,
)
require.Empty(t, deleted)
require.Len(t, added, 1)
require.True(t, added[0].Target.GetSpan().Equal(span))
require.True(t, added[0].Config.Equal(conf))
require.True(t, added[0].GetTarget().GetSpan().Equal(span))
addedCfg := added[0].GetConfig()
require.True(t, addedCfg.Equal(conf))

require.NotNil(t, mockSubscriber.callback)
mockSubscriber.callback(ctx, span) // invoke the callback
Expand Down
10 changes: 5 additions & 5 deletions pkg/migration/migrations/seed_tenant_span_configs.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,12 +67,12 @@ func seedTenantSpanConfigsMigration(
Key: tenantPrefix,
EndKey: tenantPrefix.Next(),
}
toUpsert := []spanconfig.Record{
{
Target: spanconfig.MakeTargetFromSpan(tenantSeedSpan),
Config: tenantSpanConfig,
},
record, err := spanconfig.MakeRecord(spanconfig.MakeTargetFromSpan(tenantSeedSpan),
tenantSpanConfig)
if err != nil {
return err
}
toUpsert := []spanconfig.Record{record}
scRecords, err := scKVAccessor.GetSpanConfigRecords(ctx, []spanconfig.Target{tenantTarget})
if err != nil {
return err
Expand Down
45 changes: 45 additions & 0 deletions pkg/roachpb/span_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@ import (
"fmt"
"strings"
"time"

"github.com/cockroachdb/errors"
)

// StoreMatchesConstraint returns whether a store's attributes or node's
Expand Down Expand Up @@ -50,6 +52,49 @@ func (s *SpanConfig) TTL() time.Duration {
return time.Duration(s.GCPolicy.TTLSeconds) * time.Second
}

// ValidateSystemTargetSpanConfig ensures that only protection policies
// (GCPolicy.ProtectionPolicies) field is set on the underlying
// roachpb.SpanConfig.
func (s *SpanConfig) ValidateSystemTargetSpanConfig() error {
if s.RangeMinBytes != 0 {
return errors.AssertionFailedf("RangeMinBytes set on system span config")
}
if s.RangeMaxBytes != 0 {
return errors.AssertionFailedf("RangeMaxBytes set on system span config")
}
if s.GCPolicy.TTLSeconds != 0 {
return errors.AssertionFailedf("TTLSeconds set on system span config")
}
if s.GCPolicy.IgnoreStrictEnforcement {
return errors.AssertionFailedf("IgnoreStrictEnforcement set on system span config")
}
if s.GlobalReads {
return errors.AssertionFailedf("GlobalReads set on system span config")
}
if s.NumReplicas != 0 {
return errors.AssertionFailedf("NumReplicas set on system span config")
}
if s.NumVoters != 0 {
return errors.AssertionFailedf("NumVoters set on system span config")
}
if len(s.Constraints) != 0 {
return errors.AssertionFailedf("Constraints set on system span config")
}
if len(s.VoterConstraints) != 0 {
return errors.AssertionFailedf("VoterConstraints set on system span config")
}
if len(s.LeasePreferences) != 0 {
return errors.AssertionFailedf("LeasePreferences set on system span config")
}
if s.RangefeedEnabled {
return errors.AssertionFailedf("RangefeedEnabled set on system span config")
}
if s.ExcludeDataFromBackup {
return errors.AssertionFailedf("ExcludeDataFromBackup set on system span config")
}
return nil
}

// GetNumVoters returns the number of voting replicas as defined in the
// span config.
// TODO(arul): We can get rid of this now that we're correctly populating
Expand Down
Loading

0 comments on commit af747e7

Please sign in to comment.