Merge pull request #1894 from pingcap/hanfei/product

*: add a flag to restrict cartesian product.
This commit is contained in:
dongxu
2016-10-28 19:08:01 +08:00
committed by GitHub
3 changed files with 35 additions and 6 deletions

View File

@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/util/testkit"
@ -599,6 +600,15 @@ func (s *testSuite) TestJoin(c *C) {
result = tk.MustQuery("select a.c1 from t a , t1 b where a.c1 = b.c1 order by a.c1;")
result.Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7"))
plan.AllowCartesianProduct = false
_, err := tk.Exec("select * from t, t1")
c.Check(plan.ErrCartesianProductUnsupported.Equal(err), IsTrue)
_, err = tk.Exec("select * from t left join t1 on 1")
c.Check(plan.ErrCartesianProductUnsupported.Equal(err), IsTrue)
_, err = tk.Exec("select * from t right join t1 on 1")
c.Check(plan.ErrCartesianProductUnsupported.Equal(err), IsTrue)
plan.AllowCartesianProduct = true
}
func (s *testSuite) TestMultiJoin(c *C) {

View File

@ -23,6 +23,9 @@ import (
"github.com/pingcap/tidb/terror"
)
// AllowCartesianProduct means whether tidb allows cartesian join without equal conditions.
var AllowCartesianProduct = true
// Optimize does optimization and creates a Plan.
// The node must be prepared first.
func Optimize(ctx context.Context, node ast.Node, is infoschema.InfoSchema) (Plan, error) {
@ -49,6 +52,9 @@ func Optimize(ctx context.Context, node ast.Node, is infoschema.InfoSchema) (Pla
if err != nil {
return nil, errors.Trace(err)
}
if !AllowCartesianProduct && existsCartesianProduct(logic) {
return nil, ErrCartesianProductUnsupported
}
logic = EliminateProjection(logic)
info, err := logic.convert2PhysicalPlan(&requiredProperty{})
if err != nil {
@ -60,6 +66,17 @@ func Optimize(ctx context.Context, node ast.Node, is infoschema.InfoSchema) (Pla
}
return p, nil
}
func existsCartesianProduct(p LogicalPlan) bool {
if join, ok := p.(*Join); ok && len(join.EqualConditions) == 0 {
return join.JoinType == InnerJoin || join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin
}
for _, child := range p.GetChildren() {
if existsCartesianProduct(child.(LogicalPlan)) {
return true
}
}
return false
}
// PrepareStmt prepares a raw statement parsed from parser.
// The statement must be prepared before it can be passed to optimize function.
@ -86,12 +103,12 @@ const (
// Optimizer base errors.
var (
ErrOneColumn = terror.ClassOptimizer.New(CodeOneColumn, "Operand should contain 1 column(s)")
ErrSameColumns = terror.ClassOptimizer.New(CodeSameColumns, "Operands should contain same columns")
ErrMultiWildCard = terror.ClassOptimizer.New(CodeMultiWildCard, "wildcard field exist more than once")
ErrUnSupported = terror.ClassOptimizer.New(CodeUnsupported, "unsupported")
ErrInvalidGroupFuncUse = terror.ClassOptimizer.New(CodeInvalidGroupFuncUse, "Invalid use of group function")
ErrIllegalReference = terror.ClassOptimizer.New(CodeIllegalReference, "Illegal reference")
ErrOneColumn = terror.ClassOptimizer.New(CodeOneColumn, "Operand should contain 1 column(s)")
ErrSameColumns = terror.ClassOptimizer.New(CodeSameColumns, "Operands should contain same columns")
ErrMultiWildCard = terror.ClassOptimizer.New(CodeMultiWildCard, "wildcard field exist more than once")
ErrCartesianProductUnsupported = terror.ClassOptimizer.New(CodeUnsupported, "Cartesian product is unsupported")
ErrInvalidGroupFuncUse = terror.ClassOptimizer.New(CodeInvalidGroupFuncUse, "Invalid use of group function")
ErrIllegalReference = terror.ClassOptimizer.New(CodeIllegalReference, "Illegal reference")
)
func init() {

View File

@ -56,6 +56,7 @@ var (
reportStatus = flag.Bool("report-status", true, "If enable status report HTTP service.")
logFile = flag.String("log-file", "", "log file path")
joinCon = flag.Int("join-concurrency", 5, "the number of goroutines that participate joining.")
crossJoin = flag.Bool("cross-join", true, "whether support cartesian product or not.")
metricsAddr = flag.String("metrics-addr", "", "prometheus pushgateway address, leaves it empty will disable prometheus push.")
metricsInterval = flag.Int("metrics-interval", 15, "prometheus client push interval in second, set \"0\" to disable prometheus push.")
binlogSocket = flag.String("binlog-socket", "", "socket file to write binlog")
@ -92,6 +93,7 @@ func main() {
if joinCon != nil && *joinCon > 0 {
plan.JoinConcurrency = *joinCon
}
plan.AllowCartesianProduct = *crossJoin
// Call this before setting log level to make sure that TiDB info could be printed.
printer.PrintTiDBInfo()
log.SetLevelByString(cfg.LogLevel)