Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
30636: opt: add testing infrastructure to disable rules r=rytaft a=rytaft

This commit adds testing infrastructure for randomly disabling
some transformation rules in the optimizer. The goal is to test that
alternate plans produced by the optimizer are logically equivalent.
It can be used to test that the logic tests still pass with some
rules randomly disabled as follows:
```
  > make test PKG=./pkg/sql/logictest/... TESTS='TestLogic/local-opt/.*' \
    TESTFLAGS='-disable-opt-rule-probability=0.1'
```
This test indicates that each transformation rule may be disabled with
10% probability. The value of -disable-opt-rule-probability can be any
value between 0.0 and 1.0.

Release note: None

30689: opt: show FKs in ddl output for test catalog r=justinj a=justinj

Also remove a TODO fixed by #30072.

Release note: None

30716: storage: proactively add to replicate queue on leader acquisition r=petermattis a=petermattis

Proactively add replicas to the replicate queue on Raft leader
acquisition. This is done in order to speed up removal of a replica when
the replica to be removed is the leaseholder. When that happens the
leaseholder transfers the lease to another replica and after the lease
is transferred Raft leadership is transferred. Prior to this change the
system then had to wait for the scanner on the new leaseholder node to
pick up the replica and complete the removal. Note that we wait for Raft
leadership to transfer because removal of a replica requires the
leaseholder to also be the Raft leader due to the checks in
`filterUnremovableReplicas` which ensure we're not removing a replica
that is critical for quorum.

Fixes #30695

Release note: None

Co-authored-by: Rebecca Taft <[email protected]>
Co-authored-by: Justin Jaffray <[email protected]>
Co-authored-by: Peter Mattis <[email protected]>
  • Loading branch information
4 people committed Sep 27, 2018
4 parents a486ce7 + 97dc0f2 + 21a430a + 6c8cc10 commit c94ff23
Show file tree
Hide file tree
Showing 17 changed files with 267 additions and 121 deletions.
10 changes: 7 additions & 3 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -362,6 +362,9 @@ var (
)
sqlfmtLen = flag.Int("line-length", tree.DefaultPrettyCfg().LineWidth,
"target line length when using -rewrite-sql")
disableOptRuleProbability = flag.Float64(
"disable-opt-rule-probability", 0,
"disable transformation rules in the cost-based optimizer with the given probability.")
)

type testClusterConfig struct {
Expand Down Expand Up @@ -937,9 +940,10 @@ func (t *logicTest) setup(cfg testClusterConfig) {
BootstrapVersion: cfg.bootstrapVersion,
},
SQLEvalContext: &tree.EvalContextTestingKnobs{
AssertBinaryExprReturnTypes: true,
AssertUnaryExprReturnTypes: true,
AssertFuncExprReturnTypes: true,
AssertBinaryExprReturnTypes: true,
AssertUnaryExprReturnTypes: true,
AssertFuncExprReturnTypes: true,
DisableOptimizerRuleProbability: *disableOptRuleProbability,
},
Upgrade: &server.UpgradeTestingKnobs{
DisableUpgrade: cfg.disableUpgrade,
Expand Down
54 changes: 53 additions & 1 deletion pkg/sql/opt/catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -308,7 +308,7 @@ type ForeignKeyReference struct {

// FormatCatalogTable nicely formats a catalog table using a treeprinter for
// debugging and testing.
func FormatCatalogTable(tab Table, tp treeprinter.Node) {
func FormatCatalogTable(cat Catalog, tab Table, tp treeprinter.Node) {
child := tp.Childf("TABLE %s", tab.Name().TableName)

var buf bytes.Buffer
Expand All @@ -321,6 +321,14 @@ func FormatCatalogTable(tab Table, tp treeprinter.Node) {
for i := 0; i < tab.IndexCount(); i++ {
formatCatalogIndex(tab.Index(i), i == PrimaryIndex, child)
}

for i := 0; i < tab.IndexCount(); i++ {
fkRef, ok := tab.Index(i).ForeignKey()

if ok {
formatCatalogFKRef(cat, tab, tab.Index(i), fkRef, child)
}
}
}

// formatCatalogIndex nicely formats a catalog index using a treeprinter for
Expand Down Expand Up @@ -356,6 +364,50 @@ func formatCatalogIndex(idx Index, isPrimary bool, tp treeprinter.Node) {
}
}

// formatColPrefix returns a string representation of the first prefixLen columns of idx.
func formatColPrefix(idx Index, prefixLen int) string {
var buf bytes.Buffer
buf.WriteByte('(')
for i := 0; i < prefixLen; i++ {
if i > 0 {
buf.WriteString(", ")
}
colName := idx.Column(i).Column.ColName()
buf.WriteString(colName.String())
}
buf.WriteByte(')')

return buf.String()
}

// formatCatalogFKRef nicely formats a catalog foreign key reference using a
// treeprinter for debugging and testing.
func formatCatalogFKRef(
cat Catalog, tab Table, idx Index, fkRef ForeignKeyReference, tp treeprinter.Node,
) {
ds, err := cat.ResolveDataSourceByID(context.TODO(), int64(fkRef.TableID))
if err != nil {
panic(err)
}

fkTable := ds.(Table)

var fkIndex Index
for j, cnt := 0, fkTable.IndexCount(); j < cnt; j++ {
if fkTable.Index(j).InternalID() == fkRef.IndexID {
fkIndex = fkTable.Index(j)
break
}
}

tp.Childf(
"FOREIGN KEY %s REFERENCES %v %s",
formatColPrefix(idx, int(fkRef.PrefixLen)),
ds.Name(),
formatColPrefix(fkIndex, int(fkRef.PrefixLen)),
)
}

func formatColumn(col Column, buf *bytes.Buffer) {
fmt.Fprintf(buf, "%s %s", col.ColName(), col.DatumType())
if !col.IsNullable() {
Expand Down
2 changes: 0 additions & 2 deletions pkg/sql/opt/norm/rules/join.opt
Original file line number Diff line number Diff line change
Expand Up @@ -238,8 +238,6 @@
# SELECT * FROM orders o LEFT JOIN customers c ON o.customer_id = c.id
# =>
# SELECT * FROM orders o INNER JOIN customers c ON o.customer_id = c.id
#
# TODO(andyk): Need to implement foreign key case.
[SimplifyLeftJoinWithFilters, Normalize]
(LeftJoin | LeftJoinApply | FullJoin | FullJoinApply
$left:*
Expand Down
16 changes: 9 additions & 7 deletions pkg/sql/opt/norm/testdata/rules/join
Original file line number Diff line number Diff line change
Expand Up @@ -31,9 +31,10 @@ TABLE c
├── INDEX secondary
│ ├── x int not null
│ └── z int not null
└── INDEX c_auto_index_fk_y_ref_a
├── y int not null
└── x int not null
├── INDEX c_auto_index_fk_y_ref_a
│ ├── y int not null
│ └── x int not null
└── FOREIGN KEY (y) REFERENCES t.public.a (k)

exec-ddl
CREATE TABLE d (x INT PRIMARY KEY, y INT NOT NULL, z INT NOT NULL, FOREIGN KEY (y,z) REFERENCES c(x,z))
Expand All @@ -44,10 +45,11 @@ TABLE d
├── z int not null
├── INDEX primary
│ └── x int not null
└── INDEX d_auto_index_fk_y_ref_c
├── y int not null
├── z int not null
└── x int not null
├── INDEX d_auto_index_fk_y_ref_c
│ ├── y int not null
│ ├── z int not null
│ └── x int not null
└── FOREIGN KEY (y, z) REFERENCES t.public.c (x, z)

exec-ddl
CREATE TABLE xy (x INT PRIMARY KEY, y INT)
Expand Down
7 changes: 4 additions & 3 deletions pkg/sql/opt/optbuilder/testdata/inner-join
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,10 @@ TABLE c
├── rowid int not null (hidden)
├── INDEX primary
│ └── rowid int not null (hidden)
└── INDEX c_auto_index_fk_x_ref_a
├── x int
└── rowid int not null (hidden)
├── INDEX c_auto_index_fk_x_ref_a
│ ├── x int
│ └── rowid int not null (hidden)
└── FOREIGN KEY (x) REFERENCES t.public.a (x)

build
SELECT * FROM a, b
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/testutils/testcat/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ func (tc *Catalog) CreateTable(stmt *tree.CreateTable) *Table {
// Update the table name to include catalog and schema if not provided.
tc.qualifyTableName(tn)

tab := &Table{TabFingerprint: tc.nextFingerprint(), TabName: *tn}
tab := &Table{TabFingerprint: tc.nextFingerprint(), TabName: *tn, Catalog: tc}

// Assume that every table in the "system" catalog is a virtual table. This
// is a simplified assumption for testing purposes.
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/testutils/testcat/test_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,7 @@ type Table struct {
Indexes []*Index
Stats TableStats
IsVirtual bool
Catalog opt.Catalog

// If Revoked is true, then the user has had privileges on the table revoked.
Revoked bool
Expand All @@ -290,7 +291,7 @@ var _ opt.Table = &Table{}

func (tt *Table) String() string {
tp := treeprinter.New()
opt.FormatCatalogTable(tt, tp)
opt.FormatCatalogTable(tt.Catalog, tt, tp)
return tp.String()
}

Expand Down
42 changes: 42 additions & 0 deletions pkg/sql/opt/xform/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,15 @@ package xform

import (
"fmt"
"math/rand"

"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/norm"
"github.com/cockroachdb/cockroach/pkg/sql/opt/props"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
)

// MatchedRuleFunc defines the callback function for the NotifyOnMatchedRule
Expand All @@ -35,6 +37,9 @@ type MatchedRuleFunc = norm.MatchedRuleFunc
// details.
type AppliedRuleFunc = norm.AppliedRuleFunc

// RuleSet efficiently stores an unordered set of RuleNames.
type RuleSet = util.FastIntSet

// Optimizer transforms an input expression tree into the logically equivalent
// output expression tree with the lowest possible execution cost.
//
Expand Down Expand Up @@ -83,6 +88,10 @@ type Optimizer struct {
// optimization rule (Normalize or Explore) has been applied by the optimizer.
// It can be set via a call to the NotifyOnAppliedRule method.
appliedRule AppliedRuleFunc

// disabledRules is a set of rules that are not allowed to run, used for
// testing.
disabledRules RuleSet
}

// Init initializes the Optimizer with a new, blank memo structure inside. This
Expand All @@ -97,6 +106,9 @@ func (o *Optimizer) Init(evalCtx *tree.EvalContext) {
o.stateMap = make(map[optStateKey]*optState)
o.matchedRule = nil
o.appliedRule = nil
if evalCtx.TestingKnobs.DisableOptimizerRuleProbability > 0 {
o.disableRules(evalCtx.TestingKnobs.DisableOptimizerRuleProbability)
}
}

// Factory returns a factory interface that the caller uses to construct an
Expand Down Expand Up @@ -662,3 +674,33 @@ func (a *optStateAlloc) allocate() *optState {
a.page = a.page[1:]
return state
}

// disableRules disables rules with the given probability for testing.
func (o *Optimizer) disableRules(probability float64) {
essentialRules := util.MakeFastIntSet(
// Needed to prevent constraint building from failing.
int(opt.NormalizeInConst),
// Needed when an index is forced.
int(opt.GenerateIndexScans),
// Needed to prevent "same fingerprint cannot map to different groups."
int(opt.PruneJoinLeftCols),
int(opt.PruneJoinRightCols),
// Needed to prevent stack overflow.
int(opt.PushFilterIntoJoinLeftAndRight),
int(opt.PruneSelectCols),
)

for i := opt.RuleName(1); i < opt.NumRuleNames; i++ {
if rand.Float64() < probability && !essentialRules.Contains(int(i)) {
o.disabledRules.Add(int(i))
}
}

o.NotifyOnMatchedRule(func(ruleName opt.RuleName) bool {
if o.disabledRules.Contains(int(ruleName)) {
log.Infof(o.evalCtx.Context, "disabled rule matched: %s", ruleName.String())
return false
}
return true
})
}
8 changes: 5 additions & 3 deletions pkg/sql/opt/xform/testdata/external/customer
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,11 @@ TABLE edges
├── INDEX primary
│ ├── src int not null
│ └── dst int not null
└── INDEX edges_auto_index_fk_dst_ref_nodes
├── dst int not null
└── src int not null
├── INDEX edges_auto_index_fk_dst_ref_nodes
│ ├── dst int not null
│ └── src int not null
├── FOREIGN KEY (src) REFERENCES t.public.nodes (id)
└── FOREIGN KEY (dst) REFERENCES t.public.nodes (id)

opt
select nodes.id,dst from nodes join edges on edges.dst=nodes.id
Expand Down
8 changes: 5 additions & 3 deletions pkg/sql/opt/xform/testdata/external/hibernate
Original file line number Diff line number Diff line change
Expand Up @@ -50,9 +50,11 @@ TABLE phone_register
├── INDEX primary
│ ├── phone_id int not null
│ └── person_id int not null
└── INDEX secondary
├── person_id int not null
└── phone_id int not null (storing)
├── INDEX secondary
│ ├── person_id int not null
│ └── phone_id int not null (storing)
├── FOREIGN KEY (phone_id) REFERENCES t.public.person (id)
└── FOREIGN KEY (person_id) REFERENCES t.public.phone (id)

opt
select
Expand Down
40 changes: 22 additions & 18 deletions pkg/sql/opt/xform/testdata/external/nova
Original file line number Diff line number Diff line change
Expand Up @@ -68,10 +68,11 @@ TABLE flavor_projects
├── updated_at timestamp
├── INDEX primary
│ └── id int not null
└── INDEX secondary
├── flavor_id int not null
├── project_id string not null
└── id int not null (storing)
├── INDEX secondary
│ ├── flavor_id int not null
│ ├── project_id string not null
│ └── id int not null (storing)
└── FOREIGN KEY (flavor_id) REFERENCES t.public.flavors (id)

exec-ddl
create table flavor_extra_specs
Expand All @@ -95,10 +96,11 @@ TABLE flavor_extra_specs
├── updated_at timestamp
├── INDEX primary
│ └── id int not null
└── INDEX flavor_extra_specs_flavor_id_key_idx
├── flavor_id int not null
├── key string not null
└── id int not null (storing)
├── INDEX flavor_extra_specs_flavor_id_key_idx
│ ├── flavor_id int not null
│ ├── key string not null
│ └── id int not null (storing)
└── FOREIGN KEY (flavor_id) REFERENCES t.public.flavors (id)

exec-ddl
create table instance_types
Expand Down Expand Up @@ -175,11 +177,12 @@ TABLE instance_type_projects
├── updated_at timestamp
├── INDEX primary
│ └── id int not null
└── INDEX secondary
├── instance_type_id int not null
├── project_id string
├── deleted bool
└── id int not null (storing)
├── INDEX secondary
│ ├── instance_type_id int not null
│ ├── project_id string
│ ├── deleted bool
│ └── id int not null (storing)
└── FOREIGN KEY (instance_type_id) REFERENCES t.public.instance_types (id)

exec-ddl
create table instance_type_extra_specs
Expand Down Expand Up @@ -212,11 +215,12 @@ TABLE instance_type_extra_specs
│ ├── instance_type_id int not null
│ ├── key string
│ └── id int not null
└── INDEX secondary
├── instance_type_id int not null
├── key string
├── deleted bool
└── id int not null (storing)
├── INDEX secondary
│ ├── instance_type_id int not null
│ ├── key string
│ ├── deleted bool
│ └── id int not null (storing)
└── FOREIGN KEY (instance_type_id) REFERENCES t.public.instance_types (id)

opt
select anon_1.flavors_created_at as anon_1_flavors_created_at,
Expand Down
Loading

0 comments on commit c94ff23

Please sign in to comment.