From 09a6d87da856f6883d81d42c8a5a0b11d743e788 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Sat, 25 Aug 2018 18:15:40 -0400 Subject: [PATCH 1/6] sql: don't write/parse exprs in local mode Previously, expressions that needed to be used in processors were written to a buffer with FmtParsable and re-parsed and type checked with the parser, even if the processors were running in the local flow mode. This was inefficient. Now, we use a similar strategy to the "LocalPlanNode" system to avoid this inefficiency. When we go to save an Expr to a processor spec, if we're writing to a spec that will be inflated on the gateway, instead of formatting the Expr to text, we save the Expr in a slice and record its index. Then, at inflation time, instead of parsing the text we would have written, we simply look up the Expr in the slice we saved it to. This should eliminate one of the major inefficiences introduced by the distsql merge. Release note: None --- pkg/ccl/changefeedccl/changefeed_dist.go | 6 +- pkg/sql/backfill.go | 4 +- pkg/sql/conn_executor_exec.go | 4 +- pkg/sql/distsql_physical_planner.go | 77 ++++-- pkg/sql/distsql_physical_planner_test.go | 6 +- pkg/sql/distsql_plan_csv.go | 14 +- pkg/sql/distsql_plan_join.go | 6 +- pkg/sql/distsql_plan_scrub_physical.go | 2 +- pkg/sql/distsql_plan_window.go | 10 +- pkg/sql/distsql_running.go | 11 +- pkg/sql/distsql_running_test.go | 2 +- pkg/sql/distsqlplan/aggregator_funcs_test.go | 12 +- pkg/sql/distsqlplan/expression.go | 27 ++- pkg/sql/distsqlplan/physical_plan.go | 16 +- pkg/sql/distsqlplan/physical_plan_test.go | 8 +- pkg/sql/distsqlrun/data.pb.go | 237 ++++++++++--------- pkg/sql/distsqlrun/data.proto | 2 + pkg/sql/distsqlrun/expr.go | 12 +- pkg/sql/distsqlrun/processors.go | 2 +- pkg/sql/explain_distsql.go | 6 +- pkg/sql/plan.go | 2 +- pkg/sql/scrub_fk.go | 4 +- pkg/sql/scrub_index.go | 4 +- pkg/sql/scrub_physical.go | 4 +- pkg/sql/sem/tree/eval.go | 7 +- pkg/sql/testutils.go | 2 +- 26 files changed, 301 insertions(+), 186 deletions(-) diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go index e3d7ad94e74f..23b5bfbdd922 100644 --- a/pkg/ccl/changefeedccl/changefeed_dist.go +++ b/pkg/ccl/changefeedccl/changefeed_dist.go @@ -98,7 +98,7 @@ func distChangefeedFlow( spanPartitions = []sql.SpanPartition{{Node: gatewayNodeID, Spans: trackedSpans}} } else { // All other feeds get a ChangeAggregator local on the leaseholder. - spanPartitions, err = dsp.PartitionSpans(&planCtx, trackedSpans) + spanPartitions, err = dsp.PartitionSpans(planCtx, trackedSpans) if err != nil { return err } @@ -158,7 +158,7 @@ func distChangefeedFlow( p.ResultTypes = changefeedResultTypes p.PlanToStreamColMap = []int{1, 2, 3} - dsp.FinalizePlan(&planCtx, &p) + dsp.FinalizePlan(planCtx, &p) resultRows := makeChangefeedResultWriter(resultsCh) recv := sql.MakeDistSQLReceiver( @@ -184,7 +184,7 @@ func distChangefeedFlow( finishedSetupFn = func() { resultsCh <- tree.Datums(nil) } } - dsp.Run(&planCtx, noTxn, &p, recv, evalCtx, finishedSetupFn) + dsp.Run(planCtx, noTxn, &p, recv, evalCtx, finishedSetupFn) return resultRows.Err() } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index aab0c143dcfe..5ec48079c924 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -440,13 +440,13 @@ func (sc *SchemaChanger) distBackfill( ) planCtx := sc.distSQLPlanner.NewPlanningCtx(ctx, evalCtx, txn) plan, err := sc.distSQLPlanner.createBackfiller( - &planCtx, backfillType, *tableDesc, duration, chunkSize, spans, otherTableDescs, sc.readAsOf, + planCtx, backfillType, *tableDesc, duration, chunkSize, spans, otherTableDescs, sc.readAsOf, ) if err != nil { return err } sc.distSQLPlanner.Run( - &planCtx, + planCtx, nil, /* txn - the processors manage their own transactions */ &plan, recv, evalCtx, nil, /* finishedSetupFn */ diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 889bdb8fe77f..11dddae2bc3c 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -956,7 +956,7 @@ func (ex *connExecutor) execWithDistSQLEngine( ) evalCtx := planner.ExtendedEvalContext() - var planCtx PlanningCtx + var planCtx *PlanningCtx if distribute { planCtx = ex.server.cfg.DistSQLPlanner.NewPlanningCtx(ctx, evalCtx, planner.txn) } else { @@ -981,7 +981,7 @@ func (ex *connExecutor) execWithDistSQLEngine( // We pass in whether or not we wanted to distribute this plan, which tells // the planner whether or not to plan remote table readers. ex.server.cfg.DistSQLPlanner.PlanAndRun( - ctx, evalCtx, &planCtx, planner.txn, planner.curPlan.plan, recv) + ctx, evalCtx, planCtx, planner.txn, planner.curPlan.plan, recv) return recv.commErr } diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index b6c1e5d701cd..f1f8ced0fd73 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -492,8 +492,12 @@ type PlanningCtx struct { // keep track of whether it's valid to run a root node in a special fast path // mode. planDepth int + + localExprs []tree.TypedExpr } +var _ distsqlplan.ExprContext = &PlanningCtx{} + // EvalContext returns the associated EvalContext, or nil if there isn't one. func (p *PlanningCtx) EvalContext() *tree.EvalContext { if p.ExtendedEvalCtx == nil { @@ -502,6 +506,36 @@ func (p *PlanningCtx) EvalContext() *tree.EvalContext { return &p.ExtendedEvalCtx.EvalContext } +type ivarRemapper struct { + indexVarMap []int +} + +func (v *ivarRemapper) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) { + if ivar, ok := expr.(*tree.IndexedVar); ok { + newIvar := *ivar + newIvar.Idx = v.indexVarMap[ivar.Idx] + return false, &newIvar + } + return true, expr +} + +func (*ivarRemapper) VisitPost(expr tree.Expr) tree.Expr { return expr } + +// MaybeAddLocalExpr implements the ExprContext interface. +func (p *PlanningCtx) MaybeAddLocalExpr(expr tree.TypedExpr, indexVarMap []int) (int, bool) { + if p.isLocal { + if indexVarMap != nil { + // Remap our indexed vars + v := &ivarRemapper{indexVarMap: indexVarMap} + newExpr, _ := tree.WalkExpr(v, expr) + expr = newExpr.(tree.TypedExpr) + } + p.localExprs = append(p.localExprs, expr) + return len(p.localExprs) - 1, true + } + return -1, false +} + // sanityCheckAddresses returns an error if the same address is used by two // nodes. func (p *PlanningCtx) sanityCheckAddresses() error { @@ -839,7 +873,7 @@ func getIndexIdx(n *scanNode) (uint32, error) { // initTableReaderSpec initializes a TableReaderSpec/PostProcessSpec that // corresponds to a scanNode, except for the Spans and OutputColumns. func initTableReaderSpec( - n *scanNode, evalCtx *tree.EvalContext, indexVarMap []int, + n *scanNode, planCtx *PlanningCtx, indexVarMap []int, ) (distsqlrun.TableReaderSpec, distsqlrun.PostProcessSpec, error) { s := distsqlrun.TableReaderSpec{ Table: *n.desc, @@ -860,7 +894,7 @@ func initTableReaderSpec( return s, distsqlrun.PostProcessSpec{}, nil } - filter, err := distsqlplan.MakeExpression(n.filter, evalCtx, indexVarMap) + filter, err := distsqlplan.MakeExpression(n.filter, planCtx, indexVarMap) if err != nil { return distsqlrun.TableReaderSpec{}, distsqlrun.PostProcessSpec{}, err } @@ -1041,7 +1075,7 @@ func (dsp *DistSQLPlanner) createTableReaders( ) (PhysicalPlan, error) { scanNodeToTableOrdinalMap := getScanNodeToTableOrdinalMap(n) - spec, post, err := initTableReaderSpec(n, planCtx.EvalContext(), scanNodeToTableOrdinalMap) + spec, post, err := initTableReaderSpec(n, planCtx, scanNodeToTableOrdinalMap) if err != nil { return PhysicalPlan{}, err } @@ -1164,7 +1198,7 @@ func (dsp *DistSQLPlanner) createTableReaders( // corresponding to the render node itself. An evaluator stage is added if the // render node has any expressions which are not just simple column references. func (dsp *DistSQLPlanner) selectRenders( - p *PhysicalPlan, n *renderNode, evalCtx *tree.EvalContext, + p *PhysicalPlan, n *renderNode, planCtx *PlanningCtx, ) error { // We want to skip any unused renders. planToStreamColMap := makePlanToStreamColMap(len(n.render)) @@ -1180,7 +1214,7 @@ func (dsp *DistSQLPlanner) selectRenders( if err != nil { return err } - err = p.AddRendering(renders, evalCtx, p.PlanToStreamColMap, types) + err = p.AddRendering(renders, planCtx, p.PlanToStreamColMap, types) if err != nil { return err } @@ -1277,7 +1311,7 @@ func (dsp *DistSQLPlanner) addAggregators( aggregationsColumnTypes[i] = make([]sqlbase.ColumnType, len(fholder.arguments)) for j, argument := range fholder.arguments { var err error - aggregations[i].Arguments[j], err = distsqlplan.MakeExpression(argument, planCtx.EvalContext(), nil) + aggregations[i].Arguments[j], err = distsqlplan.MakeExpression(argument, planCtx, nil) if err != nil { return err } @@ -1656,8 +1690,7 @@ func (dsp *DistSQLPlanner) addAggregators( mappedIdx := int(finalIdxMap[finalIdx]) var err error renderExprs[i], err = distsqlplan.MakeExpression( - h.IndexedVar(mappedIdx), planCtx.EvalContext(), - nil /* indexVarMap */) + h.IndexedVar(mappedIdx), planCtx, nil /* indexVarMap */) if err != nil { return err } @@ -1677,7 +1710,7 @@ func (dsp *DistSQLPlanner) addAggregators( return err } renderExprs[i], err = distsqlplan.MakeExpression( - expr, planCtx.EvalContext(), + expr, planCtx, nil /* indexVarMap */) if err != nil { return err @@ -1808,7 +1841,7 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( } filter, err := distsqlplan.MakeExpression( - n.table.filter, planCtx.EvalContext(), nil /* indexVarMap */) + n.table.filter, planCtx, nil /* indexVarMap */) if err != nil { return PhysicalPlan{}, err } @@ -1926,7 +1959,7 @@ func (dsp *DistSQLPlanner) createPlanForLookupJoin( } var err error joinReaderSpec.OnExpr, err = distsqlplan.MakeExpression( - n.onCond, planCtx.EvalContext(), indexVarMap, + n.onCond, planCtx, indexVarMap, ) if err != nil { return PhysicalPlan{}, err @@ -2114,7 +2147,7 @@ func (dsp *DistSQLPlanner) createPlanForJoin( } post, joinToStreamColMap := joinOutColumns(n, leftPlan.PlanToStreamColMap, rightMap) - onExpr, err := remapOnExpr(planCtx.EvalContext(), n, leftPlan.PlanToStreamColMap, rightMap) + onExpr, err := remapOnExpr(planCtx, n, leftPlan.PlanToStreamColMap, rightMap) if err != nil { return PhysicalPlan{}, err } @@ -2153,7 +2186,7 @@ func (dsp *DistSQLPlanner) createPlanForJoin( var indexFilterExpr distsqlrun.Expression if lookupJoinScan.origFilter != nil { var err error - indexFilterExpr, err = distsqlplan.MakeExpression(lookupJoinScan.origFilter, planCtx.EvalContext(), nil) + indexFilterExpr, err = distsqlplan.MakeExpression(lookupJoinScan.origFilter, planCtx, nil) if err != nil { return PhysicalPlan{}, err } @@ -2303,7 +2336,7 @@ func (dsp *DistSQLPlanner) createPlanForNode( if err != nil { return PhysicalPlan{}, err } - err = dsp.selectRenders(&plan, n, planCtx.EvalContext()) + err = dsp.selectRenders(&plan, n, planCtx) if err != nil { return PhysicalPlan{}, err } @@ -2332,7 +2365,7 @@ func (dsp *DistSQLPlanner) createPlanForNode( return PhysicalPlan{}, err } - if err := plan.AddFilter(n.filter, planCtx.EvalContext(), plan.PlanToStreamColMap); err != nil { + if err := plan.AddFilter(n.filter, planCtx, plan.PlanToStreamColMap); err != nil { return PhysicalPlan{}, err } @@ -2344,7 +2377,7 @@ func (dsp *DistSQLPlanner) createPlanForNode( if err := n.evalLimit(planCtx.EvalContext()); err != nil { return PhysicalPlan{}, err } - if err := plan.AddLimit(n.count, n.offset, planCtx.EvalContext(), dsp.nodeDesc.NodeID); err != nil { + if err := plan.AddLimit(n.count, n.offset, planCtx, dsp.nodeDesc.NodeID); err != nil { return PhysicalPlan{}, err } @@ -2713,7 +2746,7 @@ func createProjectSetSpec( } for i, expr := range n.exprs { var err error - spec.Exprs[i], err = distsqlplan.MakeExpression(expr, &planCtx.ExtendedEvalCtx.EvalContext, indexVarMap) + spec.Exprs[i], err = distsqlplan.MakeExpression(expr, planCtx, indexVarMap) if err != nil { return nil, err } @@ -3079,7 +3112,7 @@ func (dsp *DistSQLPlanner) createPlanForWindow( } numWindowFuncProcessed := 0 - windowPlanState := createWindowPlanState(n, planCtx.EvalContext(), &plan) + windowPlanState := createWindowPlanState(n, planCtx, &plan) // Each iteration of this loop adds a new stage of windowers. The steps taken: // 1. find a set of unprocessed window functions that have the same PARTITION BY // clause. All of these will be computed using the single stage of windowers. @@ -3253,7 +3286,7 @@ func (dsp *DistSQLPlanner) createPlanForWindow( // NewPlanningCtx returns a new PlanningCtx. func (dsp *DistSQLPlanner) NewPlanningCtx( ctx context.Context, evalCtx *extendedEvalContext, txn *client.Txn, -) PlanningCtx { +) *PlanningCtx { planCtx := dsp.newLocalPlanningCtx(ctx, evalCtx) planCtx.spanIter = dsp.spanResolver.NewSpanResolverIterator(txn) planCtx.NodeAddresses = make(map[roachpb.NodeID]string) @@ -3265,11 +3298,13 @@ func (dsp *DistSQLPlanner) NewPlanningCtx( // used when the caller knows plans will only be run on one node. func (dsp *DistSQLPlanner) newLocalPlanningCtx( ctx context.Context, evalCtx *extendedEvalContext, -) PlanningCtx { - return PlanningCtx{ +) *PlanningCtx { + ret := &PlanningCtx{ ctx: ctx, ExtendedEvalCtx: evalCtx, } + evalCtx.LocalExprs = &ret.localExprs + return ret } // FinalizePlan adds a final "result" stage if necessary and populates the diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index b70aa5ab4581..487c3ccb6cf8 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -792,7 +792,7 @@ func TestPartitionSpans(t *testing.T) { spans = append(spans, roachpb.Span{Key: roachpb.Key(s[0]), EndKey: roachpb.Key(s[1])}) } - partitions, err := dsp.PartitionSpans(&planCtx, spans) + partitions, err := dsp.PartitionSpans(planCtx, spans) if err != nil { t.Fatal(err) } @@ -963,7 +963,7 @@ func TestPartitionSpansSkipsIncompatibleNodes(t *testing.T) { } planCtx := dsp.NewPlanningCtx(context.Background(), nil /* evalCtx */, nil /* txn */) - partitions, err := dsp.PartitionSpans(&planCtx, roachpb.Spans{span}) + partitions, err := dsp.PartitionSpans(planCtx, roachpb.Spans{span}) if err != nil { t.Fatal(err) } @@ -1054,7 +1054,7 @@ func TestPartitionSpansSkipsNodesNotInGossip(t *testing.T) { } planCtx := dsp.NewPlanningCtx(context.Background(), nil /* evalCtx */, nil /* txn */) - partitions, err := dsp.PartitionSpans(&planCtx, roachpb.Spans{span}) + partitions, err := dsp.PartitionSpans(planCtx, roachpb.Spans{span}) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/distsql_plan_csv.go b/pkg/sql/distsql_plan_csv.go index 8bd309bf8de7..a56bc4157b3e 100644 --- a/pkg/sql/distsql_plan_csv.go +++ b/pkg/sql/distsql_plan_csv.go @@ -64,7 +64,7 @@ func PlanAndRunExport( rec, err := dsp.checkSupportForNode(in) planCtx.isLocal = err != nil || rec == cannotDistribute - p, err := dsp.createPlanForNode(&planCtx, in) + p, err := dsp.createPlanForNode(planCtx, in) if err != nil { return errors.Wrap(err, "constructing distSQL plan") } @@ -78,7 +78,7 @@ func PlanAndRunExport( // columns filename/rows/bytes. p.PlanToStreamColMap = identityMap(p.PlanToStreamColMap, len(ExportPlanResultTypes)) - dsp.FinalizePlan(&planCtx, &p) + dsp.FinalizePlan(planCtx, &p) recv := MakeDistSQLReceiver( ctx, resultRows, tree.Rows, @@ -86,7 +86,7 @@ func PlanAndRunExport( evalCtx.Tracing, ) - dsp.Run(&planCtx, txn, &p, recv, evalCtx, nil /* finishedSetupFn */) + dsp.Run(planCtx, txn, &p, recv, evalCtx, nil /* finishedSetupFn */) return resultRows.Err() } @@ -194,7 +194,7 @@ func LoadCSV( // Because we're not going through the normal pathways, we have to set up // the nodeID -> nodeAddress map ourselves. for _, node := range resp.Nodes { - if err := dsp.CheckNodeHealthAndVersion(&planCtx, &node.Desc); err != nil { + if err := dsp.CheckNodeHealthAndVersion(planCtx, &node.Desc); err != nil { continue } } @@ -256,7 +256,7 @@ func LoadCSV( samples := details.Samples if samples == nil { var err error - samples, err = dsp.loadCSVSamplingPlan(ctx, job, db, evalCtx, thisNode, nodes, from, splitSize, oversample, &planCtx, inputSpecs, sstSpecs) + samples, err = dsp.loadCSVSamplingPlan(ctx, job, db, evalCtx, thisNode, nodes, from, splitSize, oversample, planCtx, inputSpecs, sstSpecs) if err != nil { return err } @@ -462,7 +462,7 @@ func LoadCSV( return err } - dsp.FinalizePlan(&planCtx, &p) + dsp.FinalizePlan(planCtx, &p) recv := MakeDistSQLReceiver( ctx, @@ -477,7 +477,7 @@ func LoadCSV( defer log.VEventf(ctx, 1, "finished job %s", job.Payload().Description) return db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { - dsp.Run(&planCtx, txn, &p, recv, evalCtx, nil /* finishedSetupFn */) + dsp.Run(planCtx, txn, &p, recv, evalCtx, nil /* finishedSetupFn */) return resultRows.Err() }) } diff --git a/pkg/sql/distsql_plan_join.go b/pkg/sql/distsql_plan_join.go index fc2662d74abc..a96fd212d07b 100644 --- a/pkg/sql/distsql_plan_join.go +++ b/pkg/sql/distsql_plan_join.go @@ -112,7 +112,7 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( joinType := n.joinType post, joinToStreamColMap := joinOutColumns(n, plans[0].PlanToStreamColMap, plans[1].PlanToStreamColMap) - onExpr, err := remapOnExpr(planCtx.EvalContext(), n, plans[0].PlanToStreamColMap, plans[1].PlanToStreamColMap) + onExpr, err := remapOnExpr(planCtx, n, plans[0].PlanToStreamColMap, plans[1].PlanToStreamColMap) if err != nil { return PhysicalPlan{}, false, err } @@ -288,7 +288,7 @@ func joinOutColumns( // join columns as described above) to values that make sense in the joiner (0 // to N-1 for the left input columns, N to N+M-1 for the right input columns). func remapOnExpr( - evalCtx *tree.EvalContext, n *joinNode, leftPlanToStreamColMap, rightPlanToStreamColMap []int, + planCtx *PlanningCtx, n *joinNode, leftPlanToStreamColMap, rightPlanToStreamColMap []int, ) (distsqlrun.Expression, error) { if n.pred.onCond == nil { return distsqlrun.Expression{}, nil @@ -309,7 +309,7 @@ func remapOnExpr( idx++ } - return distsqlplan.MakeExpression(n.pred.onCond, evalCtx, joinColMap) + return distsqlplan.MakeExpression(n.pred.onCond, planCtx, joinColMap) } // eqCols produces a slice of ordinal references for the plan columns specified diff --git a/pkg/sql/distsql_plan_scrub_physical.go b/pkg/sql/distsql_plan_scrub_physical.go index d0efbf74db16..7595f51b658d 100644 --- a/pkg/sql/distsql_plan_scrub_physical.go +++ b/pkg/sql/distsql_plan_scrub_physical.go @@ -35,7 +35,7 @@ func (dsp *DistSQLPlanner) createScrubPhysicalCheck( spans []roachpb.Span, readAsOf hlc.Timestamp, ) (PhysicalPlan, error) { - spec, _, err := initTableReaderSpec(n, planCtx.EvalContext(), nil /* indexVarMap */) + spec, _, err := initTableReaderSpec(n, planCtx, nil /* indexVarMap */) if err != nil { return PhysicalPlan{}, err } diff --git a/pkg/sql/distsql_plan_window.go b/pkg/sql/distsql_plan_window.go index 3f05570f6e4d..2d49f9bdcf7f 100644 --- a/pkg/sql/distsql_plan_window.go +++ b/pkg/sql/distsql_plan_window.go @@ -28,12 +28,12 @@ type windowPlanState struct { // they appear in n.funcs. infos []*windowFuncInfo n *windowNode - evalCtx *tree.EvalContext + planCtx *PlanningCtx plan *PhysicalPlan } func createWindowPlanState( - n *windowNode, evalCtx *tree.EvalContext, plan *PhysicalPlan, + n *windowNode, planCtx *PlanningCtx, plan *PhysicalPlan, ) *windowPlanState { infos := make([]*windowFuncInfo, 0, len(n.funcs)) for _, holder := range n.funcs { @@ -42,7 +42,7 @@ func createWindowPlanState( return &windowPlanState{ infos: infos, n: n, - evalCtx: evalCtx, + planCtx: planCtx, plan: plan, } } @@ -240,7 +240,7 @@ func (s *windowPlanState) createWindowFnSpec( if s.n.run.windowFrames[funcInProgress.funcIdx] != nil { // funcInProgress has a custom window frame. frameSpec := distsqlrun.WindowerSpec_Frame{} - if err := frameSpec.InitFromAST(s.n.run.windowFrames[funcInProgress.funcIdx], s.evalCtx); err != nil { + if err := frameSpec.InitFromAST(s.n.run.windowFrames[funcInProgress.funcIdx], s.planCtx.EvalContext()); err != nil { return distsqlrun.WindowerSpec_WindowFn{}, outputType, err } funcInProgressSpec.Frame = &frameSpec @@ -325,7 +325,7 @@ func (s *windowPlanState) addRenderingIfNecessary() error { } renderTypes = append(renderTypes, outputType) } - if err := s.plan.AddRendering(renderExprs, s.evalCtx, s.plan.PlanToStreamColMap, renderTypes); err != nil { + if err := s.plan.AddRendering(renderExprs, s.planCtx, s.plan.PlanToStreamColMap, renderTypes); err != nil { return err } s.plan.PlanToStreamColMap = identityMap(s.plan.PlanToStreamColMap, len(renderTypes)) diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 4ac2d34ab871..0a8c7bc0c053 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -617,7 +617,7 @@ func (dsp *DistSQLPlanner) PlanAndRunSubqueries( evalCtx.ActiveMemAcc = &subqueryMemAccount - var subqueryPlanCtx PlanningCtx + var subqueryPlanCtx *PlanningCtx var distributeSubquery bool if maybeDistribute { distributeSubquery = shouldDistributePlan( @@ -637,12 +637,12 @@ func (dsp *DistSQLPlanner) PlanAndRunSubqueries( // that. subqueryPlanCtx.ignoreClose = true - subqueryPhysPlan, err := dsp.createPlanForNode(&subqueryPlanCtx, subqueryPlan.plan) + subqueryPhysPlan, err := dsp.createPlanForNode(subqueryPlanCtx, subqueryPlan.plan) if err != nil { recv.SetError(err) return false } - dsp.FinalizePlan(&subqueryPlanCtx, &subqueryPhysPlan) + dsp.FinalizePlan(subqueryPlanCtx, &subqueryPhysPlan) // TODO(arjun): #28264: We set up a row container, wrap it in a row // receiver, and use it and serialize the results of the subquery. The type @@ -660,7 +660,7 @@ func (dsp *DistSQLPlanner) PlanAndRunSubqueries( subqueryRowReceiver := NewRowResultWriter(rows) subqueryRecv.resultWriter = subqueryRowReceiver subqueryPlans[planIdx].started = true - dsp.Run(&subqueryPlanCtx, planner.txn, &subqueryPhysPlan, subqueryRecv, evalCtx, nil /* finishedSetupFn */) + dsp.Run(subqueryPlanCtx, planner.txn, &subqueryPhysPlan, subqueryRecv, evalCtx, nil /* finishedSetupFn */) if subqueryRecv.commErr != nil { recv.SetError(subqueryRecv.commErr) return false @@ -742,6 +742,9 @@ func (dsp *DistSQLPlanner) PlanAndRun( recv.SetError(err) return } + if planCtx.isLocal { + //fmt.Println("Made local exprs: ", planCtx.localExprs, *planCtx.ExtendedEvalCtx.LocalExprs) + } dsp.FinalizePlan(planCtx, &physPlan) dsp.Run(planCtx, txn, &physPlan, recv, evalCtx, nil /* finishedSetupFn */) } diff --git a/pkg/sql/distsql_running_test.go b/pkg/sql/distsql_running_test.go index f2273f5755c5..43edaeb2b30a 100644 --- a/pkg/sql/distsql_running_test.go +++ b/pkg/sql/distsql_running_test.go @@ -161,7 +161,7 @@ func TestDistSQLRunningInAbortedTxn(t *testing.T) { planCtx.stmtType = recv.stmtType execCfg.DistSQLPlanner.PlanAndRun( - ctx, evalCtx, &planCtx, txn, p.curPlan.plan, recv) + ctx, evalCtx, planCtx, txn, p.curPlan.plan, recv) return rw.Err() }) if err != nil { diff --git a/pkg/sql/distsqlplan/aggregator_funcs_test.go b/pkg/sql/distsqlplan/aggregator_funcs_test.go index 062cd9172418..edb9cbd57f0c 100644 --- a/pkg/sql/distsqlplan/aggregator_funcs_test.go +++ b/pkg/sql/distsqlplan/aggregator_funcs_test.go @@ -102,6 +102,16 @@ func runTestFlow( return res } +type fakeExprContext struct{} + +func (fakeExprContext) EvalContext() *tree.EvalContext { + return &tree.EvalContext{} +} + +func (fakeExprContext) MaybeAddLocalExpr(expr tree.TypedExpr, indexVarMap []int) (int, bool) { + return 0, false +} + // checkDistAggregationInfo tests that a flow with multiple local stages and a // final stage (in accordance with per DistAggregationInfo) gets the same result // with a naive aggregation flow that has a single non-distributed stage. @@ -295,7 +305,7 @@ func checkDistAggregationInfo( t.Fatal(err) } var expr distsqlrun.Expression - expr, err = MakeExpression(renderExpr, nil, nil) + expr, err = MakeExpression(renderExpr, &fakeExprContext{}, nil) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/distsqlplan/expression.go b/pkg/sql/distsqlplan/expression.go index 74a18f3fab32..a97a29c30bd2 100644 --- a/pkg/sql/distsqlplan/expression.go +++ b/pkg/sql/distsqlplan/expression.go @@ -44,6 +44,23 @@ func exprFmtCtxBase(buf *bytes.Buffer, evalCtx *tree.EvalContext) tree.FmtCtx { return fmtCtx } +type LocalExprState struct { + expressions []tree.TypedExpr +} + +// ExprContext is an interface containing objects necessary for creating +// distsqlrun.Expressions. +type ExprContext interface { + // EvalContext returns the tree.EvalContext for planning. + EvalContext() *tree.EvalContext + + // MaybeAddLocalExpr adds an expression to the list of local expressions if + // the current plan is local, returning the index of the expression in the + // list if so. Returns false in the 2nd position of the current plan isn't + // local. + MaybeAddLocalExpr(expr tree.TypedExpr, indexVarMap []int) (idx int, ok bool) +} + // MakeExpression creates a distsqlrun.Expression. // // The distsqlrun.Expression uses the placeholder syntax (@1, @2, @3..) to refer @@ -53,12 +70,20 @@ func exprFmtCtxBase(buf *bytes.Buffer, evalCtx *tree.EvalContext) tree.FmtCtx { // remap these columns by passing an indexVarMap: an IndexedVar with index i // becomes column indexVarMap[i]. func MakeExpression( - expr tree.TypedExpr, evalCtx *tree.EvalContext, indexVarMap []int, + expr tree.TypedExpr, ctx ExprContext, indexVarMap []int, ) (distsqlrun.Expression, error) { if expr == nil { return distsqlrun.Expression{}, nil } + if idx, ok := ctx.MaybeAddLocalExpr(expr, indexVarMap); ok { + log.VEventf(ctx.EvalContext().Context, 1, "added local expr %v %d", expr, idx) + return distsqlrun.Expression{ + LocalExprIdx: uint32(idx + 1), + }, nil + } + + evalCtx := ctx.EvalContext() subqueryVisitor := &evalAndReplaceSubqueryVisitor{ evalCtx: evalCtx, } diff --git a/pkg/sql/distsqlplan/physical_plan.go b/pkg/sql/distsqlplan/physical_plan.go index 305ee293891a..09a8f2e67566 100644 --- a/pkg/sql/distsqlplan/physical_plan.go +++ b/pkg/sql/distsqlplan/physical_plan.go @@ -423,7 +423,7 @@ func exprColumn(expr tree.TypedExpr, indexVarMap []int) (int, bool) { // See MakeExpression for a description of indexVarMap. func (p *PhysicalPlan) AddRendering( exprs []tree.TypedExpr, - evalCtx *tree.EvalContext, + exprCtx ExprContext, indexVarMap []int, outTypes []sqlbase.ColumnType, ) error { @@ -481,7 +481,7 @@ func (p *PhysicalPlan) AddRendering( post.RenderExprs = make([]distsqlrun.Expression, len(exprs)) for i, e := range exprs { var err error - post.RenderExprs[i], err = MakeExpression(e, evalCtx, compositeMap) + post.RenderExprs[i], err = MakeExpression(e, exprCtx, compositeMap) if err != nil { return err } @@ -508,7 +508,7 @@ func (p *PhysicalPlan) AddRendering( if post.Projection { internalColIdx = post.OutputColumns[internalColIdx] } - newExpr, err := MakeExpression(&tree.IndexedVar{Idx: int(internalColIdx)}, evalCtx, nil) + newExpr, err := MakeExpression(&tree.IndexedVar{Idx: int(internalColIdx)}, exprCtx, nil) if err != nil { return err } @@ -592,7 +592,7 @@ func reverseProjection(outputColumns []uint32, indexVarMap []int) []int { // // See MakeExpression for a description of indexVarMap. func (p *PhysicalPlan) AddFilter( - expr tree.TypedExpr, evalCtx *tree.EvalContext, indexVarMap []int, + expr tree.TypedExpr, exprCtx ExprContext, indexVarMap []int, ) error { post := p.GetLastStagePost() if len(post.RenderExprs) > 0 || post.Offset != 0 || post.Limit != 0 { @@ -616,7 +616,7 @@ func (p *PhysicalPlan) AddFilter( if post.Projection { compositeMap = reverseProjection(post.OutputColumns, indexVarMap) } - filter, err := MakeExpression(expr, evalCtx, compositeMap) + filter, err := MakeExpression(expr, exprCtx, compositeMap) if err != nil { return err } @@ -659,7 +659,7 @@ func emptyPlan(types []sqlbase.ColumnType, node roachpb.NodeID) PhysicalPlan { // // For no limit, count should be MaxInt64. func (p *PhysicalPlan) AddLimit( - count int64, offset int64, evalCtx *tree.EvalContext, node roachpb.NodeID, + count int64, offset int64, exprCtx ExprContext, node roachpb.NodeID, ) error { if count < 0 { return errors.Errorf("negative limit") @@ -713,7 +713,7 @@ func (p *PhysicalPlan) AddLimit( } p.SetLastStagePost(post, p.ResultTypes) if limitZero { - if err := p.AddFilter(tree.DBoolFalse, evalCtx, nil); err != nil { + if err := p.AddFilter(tree.DBoolFalse, exprCtx, nil); err != nil { return err } } @@ -747,7 +747,7 @@ func (p *PhysicalPlan) AddLimit( p.ResultTypes, ) if limitZero { - if err := p.AddFilter(tree.DBoolFalse, evalCtx, nil); err != nil { + if err := p.AddFilter(tree.DBoolFalse, exprCtx, nil); err != nil { return err } } diff --git a/pkg/sql/distsqlplan/physical_plan_test.go b/pkg/sql/distsqlplan/physical_plan_test.go index 0b7f2de99ce7..58ccb863b6e0 100644 --- a/pkg/sql/distsqlplan/physical_plan_test.go +++ b/pkg/sql/distsqlplan/physical_plan_test.go @@ -206,7 +206,7 @@ func TestProjectionAndRendering(t *testing.T) { &tree.IndexedVar{Idx: 12}, &tree.IndexedVar{Idx: 13}, }, - nil, + fakeExprContext{}, []int{-1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 0, 1, 2, 3}, []sqlbase.ColumnType{strToType("A"), strToType("B"), strToType("C"), strToType("D")}, ); err != nil { @@ -230,7 +230,7 @@ func TestProjectionAndRendering(t *testing.T) { &tree.IndexedVar{Idx: 13}, &tree.IndexedVar{Idx: 12}, }, - nil, + fakeExprContext{}, []int{-1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 0, 1, 2, 3}, []sqlbase.ColumnType{strToType("B"), strToType("D"), strToType("C")}, ); err != nil { @@ -261,7 +261,7 @@ func TestProjectionAndRendering(t *testing.T) { Right: &tree.IndexedVar{Idx: 2}, }, }, - nil, + fakeExprContext{}, []int{0, 1, 2}, []sqlbase.ColumnType{strToType("X")}, ); err != nil { @@ -295,7 +295,7 @@ func TestProjectionAndRendering(t *testing.T) { }, &tree.IndexedVar{Idx: 10}, }, - nil, + fakeExprContext{}, []int{-1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 0, 1, 2}, []sqlbase.ColumnType{strToType("X"), strToType("A")}, ); err != nil { diff --git a/pkg/sql/distsqlrun/data.pb.go b/pkg/sql/distsqlrun/data.pb.go index ab3c43bb4d2f..3d4a283236eb 100644 --- a/pkg/sql/distsqlrun/data.pb.go +++ b/pkg/sql/distsqlrun/data.pb.go @@ -317,7 +317,8 @@ type Expression struct { Version string `protobuf:"bytes,1,opt,name=version" json:"version"` // SQL expressions are passed as a string, with ordinal references // (@1, @2, @3 ..) used for "input" variables. - Expr string `protobuf:"bytes,2,opt,name=expr" json:"expr"` + Expr string `protobuf:"bytes,2,opt,name=expr" json:"expr"` + LocalExprIdx uint32 `protobuf:"varint,3,opt,name=local_expr_idx,json=localExprIdx" json:"local_expr_idx"` } func (m *Expression) Reset() { *m = Expression{} } @@ -968,6 +969,9 @@ func (m *Expression) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintData(dAtA, i, uint64(len(m.Expr))) i += copy(dAtA[i:], m.Expr) + dAtA[i] = 0x18 + i++ + i = encodeVarintData(dAtA, i, uint64(m.LocalExprIdx)) return i, nil } @@ -1695,6 +1699,7 @@ func (m *Expression) Size() (n int) { n += 1 + l + sovData(uint64(l)) l = len(m.Expr) n += 1 + l + sovData(uint64(l)) + n += 1 + sovData(uint64(m.LocalExprIdx)) return n } @@ -2185,6 +2190,25 @@ func (m *Expression) Unmarshal(dAtA []byte) error { } m.Expr = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LocalExprIdx", wireType) + } + m.LocalExprIdx = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LocalExprIdx |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipData(dAtA[iNdEx:]) @@ -4464,110 +4488,111 @@ var ( func init() { proto.RegisterFile("sql/distsqlrun/data.proto", fileDescriptorData) } var fileDescriptorData = []byte{ - // 1670 bytes of a gzipped FileDescriptorProto + // 1694 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x4f, 0x6f, 0x23, 0xb7, - 0x15, 0xd7, 0xc8, 0xfa, 0xfb, 0x24, 0x6b, 0x65, 0x76, 0xb1, 0x50, 0xd5, 0xad, 0xe5, 0x4c, 0x83, - 0x62, 0x1b, 0xa4, 0xd2, 0xae, 0x73, 0x58, 0xd4, 0x39, 0x24, 0x92, 0xa5, 0xb5, 0xd4, 0x78, 0xad, - 0xed, 0xc8, 0xdb, 0x62, 0x93, 0x02, 0xd3, 0xf1, 0x90, 0x1e, 0x0f, 0x32, 0x9a, 0x19, 0x93, 0x9c, - 0x95, 0x7d, 0xe9, 0xa9, 0x1f, 0x20, 0xa7, 0xa2, 0xc7, 0x9c, 0x7a, 0xe9, 0x07, 0xe8, 0x37, 0x28, - 0xf6, 0xd8, 0x63, 0xd0, 0x02, 0x46, 0xeb, 0x5e, 0xfa, 0x15, 0x1a, 0xa0, 0x40, 0x40, 0x0e, 0xa9, - 0x3f, 0xf6, 0x3a, 0x89, 0x77, 0x2f, 0x03, 0xf2, 0xf1, 0xbd, 0x1f, 0x7f, 0xfc, 0xf1, 0xf1, 0x71, - 0x08, 0x3f, 0x64, 0xa7, 0x41, 0x07, 0xfb, 0x8c, 0xb3, 0xd3, 0x80, 0x26, 0x61, 0x07, 0x3b, 0xdc, - 0x69, 0xc7, 0x34, 0xe2, 0x11, 0x6a, 0xb8, 0x91, 0xfb, 0x39, 0x8d, 0x1c, 0xf7, 0xa4, 0xcd, 0x4e, - 0x83, 0xf6, 0xc2, 0xa9, 0xb9, 0x21, 0xad, 0xf1, 0x51, 0xc7, 0x89, 0xfd, 0xd4, 0xb9, 0x89, 0xb4, - 0x69, 0x01, 0xd0, 0xbc, 0xab, 0x6d, 0x84, 0xd2, 0x88, 0x32, 0x65, 0x6d, 0x89, 0x19, 0x63, 0x6f, - 0xe6, 0x53, 0xd2, 0x89, 0x3d, 0x39, 0xb4, 0xea, 0x70, 0x5f, 0x38, 0xb0, 0xd3, 0xe0, 0xc8, 0x61, - 0xa4, 0xc3, 0x38, 0x4d, 0x5c, 0x9e, 0x50, 0x82, 0x97, 0xc3, 0xf5, 0x28, 0x09, 0xdd, 0x08, 0x13, - 0x6c, 0x63, 0x87, 0x27, 0x53, 0xe5, 0xb0, 0x95, 0x70, 0x3f, 0xe8, 0x70, 0xea, 0xb8, 0x7e, 0xe8, - 0x75, 0x28, 0x71, 0x23, 0x2a, 0x5c, 0x58, 0xec, 0x84, 0x9a, 0x97, 0x17, 0x79, 0x91, 0x6c, 0x76, - 0x44, 0x2b, 0xb5, 0x9a, 0x7f, 0x35, 0x20, 0x3f, 0x10, 0x3c, 0x50, 0x0f, 0x4a, 0xb1, 0x67, 0x4b, - 0x4e, 0x0d, 0x63, 0xcb, 0x78, 0x50, 0xd9, 0x6e, 0xb4, 0x17, 0x5a, 0x28, 0xce, 0x6d, 0xe9, 0xdb, - 0xab, 0x5c, 0x5e, 0xb4, 0x8a, 0xcf, 0xf6, 0x64, 0x67, 0x98, 0xb1, 0x8a, 0xb1, 0x97, 0x62, 0x7c, - 0x0a, 0x1b, 0x94, 0x70, 0x7a, 0xee, 0x1c, 0x05, 0xe4, 0xf0, 0x2c, 0x94, 0xc6, 0x46, 0x56, 0x82, - 0xbd, 0xb7, 0x04, 0xa6, 0x14, 0x6a, 0x3f, 0x0f, 0x4f, 0x9c, 0x10, 0x07, 0x04, 0x5b, 0x3a, 0x48, - 0x23, 0x5e, 0x87, 0xd9, 0xc9, 0xfd, 0xe9, 0xcb, 0x56, 0xa6, 0x57, 0x82, 0x02, 0x26, 0xdc, 0xf1, - 0x03, 0xf3, 0x09, 0xc0, 0xe0, 0x2c, 0xa6, 0x84, 0x31, 0x3f, 0x0a, 0xd1, 0x26, 0x14, 0x5f, 0x12, - 0x2a, 0x9a, 0x92, 0x7c, 0xb9, 0x97, 0x7b, 0x75, 0xd1, 0xca, 0x58, 0xda, 0x88, 0x1a, 0x90, 0x23, - 0x67, 0x71, 0x4a, 0x46, 0x0f, 0x4a, 0x8b, 0xf9, 0x87, 0x2c, 0x94, 0xc6, 0x14, 0x13, 0xea, 0x87, - 0x1e, 0x1a, 0x41, 0xd1, 0x8d, 0x82, 0x64, 0x1a, 0xb2, 0x86, 0xb1, 0xb5, 0xf6, 0xa0, 0xb2, 0xfd, - 0xb3, 0xf6, 0x4d, 0xf9, 0xd0, 0xd6, 0x41, 0xed, 0x5d, 0x19, 0xa1, 0x67, 0x54, 0xf1, 0xcd, 0x3f, - 0x1b, 0x50, 0x48, 0x47, 0xd0, 0x8f, 0x25, 0xaa, 0xed, 0xe3, 0x33, 0x49, 0x6e, 0x5d, 0xb9, 0x16, - 0xdc, 0x28, 0x18, 0xe1, 0x33, 0xf4, 0x1b, 0x28, 0x63, 0x9f, 0x12, 0x97, 0x0b, 0xf6, 0x82, 0x60, - 0x6d, 0xfb, 0x83, 0xef, 0x3d, 0x6d, 0xbb, 0xaf, 0x43, 0x15, 0xea, 0x02, 0xcb, 0xdc, 0x84, 0xf2, - 0x7c, 0x14, 0x15, 0x61, 0xad, 0x3b, 0xd9, 0xad, 0x67, 0x50, 0x09, 0x72, 0xfd, 0xc1, 0x64, 0xb7, - 0x6e, 0xec, 0xe4, 0xfe, 0xfb, 0x65, 0x4b, 0x7d, 0xcd, 0xff, 0x67, 0x01, 0x4d, 0x38, 0x25, 0xce, - 0x74, 0x10, 0xe2, 0x38, 0xf2, 0x43, 0x3e, 0x89, 0x89, 0x8b, 0x3e, 0x81, 0x1c, 0x3f, 0x8f, 0x89, - 0xe4, 0x5d, 0xdb, 0x7e, 0x74, 0x33, 0xad, 0xeb, 0xb1, 0xed, 0xc3, 0xf3, 0x98, 0x68, 0xa9, 0x05, - 0x08, 0xfa, 0x05, 0x94, 0x99, 0x74, 0xb3, 0x7d, 0x2c, 0x17, 0x9a, 0xef, 0xdd, 0x17, 0xc3, 0x97, - 0x17, 0xad, 0x52, 0x1a, 0x3f, 0xea, 0x7f, 0xbd, 0xd4, 0xb6, 0x4a, 0xa9, 0xfb, 0x08, 0xa3, 0x1d, - 0xb8, 0x87, 0x49, 0x4c, 0x89, 0xeb, 0x70, 0x82, 0x6d, 0xee, 0x50, 0x8f, 0x70, 0xdb, 0xc1, 0x98, - 0x36, 0xd6, 0x96, 0x76, 0xf4, 0xee, 0xc2, 0xe7, 0x50, 0xba, 0x74, 0x31, 0xa6, 0xe8, 0x14, 0x6a, - 0x2a, 0x20, 0x8c, 0x30, 0x11, 0x73, 0xe7, 0xe4, 0xdc, 0x9f, 0xa8, 0xb9, 0xab, 0xa9, 0xef, 0x41, - 0x84, 0x89, 0x9c, 0xff, 0x03, 0xcf, 0xe7, 0x27, 0xc9, 0x51, 0xdb, 0x8d, 0xa6, 0x9d, 0xf9, 0x7a, - 0xf1, 0xd1, 0xa2, 0xdd, 0x89, 0x3f, 0xf7, 0x3a, 0x3a, 0x89, 0xd3, 0x30, 0xab, 0xca, 0x17, 0x20, - 0xd8, 0x7c, 0x08, 0x39, 0xb1, 0x7a, 0x54, 0x86, 0xfc, 0xfe, 0x78, 0xb7, 0xbb, 0x5f, 0xcf, 0x20, - 0x80, 0x82, 0x35, 0x78, 0x3a, 0x3e, 0x1c, 0xd4, 0x0d, 0xb4, 0x01, 0xeb, 0x93, 0x17, 0x07, 0xbb, - 0xb6, 0x35, 0x98, 0x3c, 0x1b, 0x1f, 0x4c, 0x06, 0xf5, 0xac, 0xf9, 0xcf, 0x2c, 0xac, 0x8f, 0xc2, - 0x38, 0xe1, 0x93, 0xf3, 0xd0, 0x95, 0xd2, 0x3f, 0x59, 0x91, 0xfe, 0xfd, 0x9b, 0xa5, 0x5f, 0x09, - 0xbb, 0xae, 0x7a, 0x1f, 0x4a, 0x91, 0xca, 0x19, 0x75, 0x16, 0xcd, 0xef, 0xce, 0x2e, 0x85, 0x30, - 0x8f, 0x44, 0xfb, 0x50, 0x4c, 0x37, 0x83, 0x35, 0xd6, 0xe4, 0xc9, 0x78, 0xff, 0x36, 0xb9, 0xa0, - 0x0f, 0x87, 0x82, 0x40, 0xbf, 0x84, 0x6a, 0x7a, 0x4e, 0x6c, 0x41, 0x91, 0x35, 0x72, 0x12, 0xf2, - 0x9d, 0x2b, 0x90, 0xaa, 0xe0, 0xa9, 0x4c, 0x5f, 0x5a, 0x58, 0xc5, 0x9d, 0x5b, 0x98, 0x69, 0x2a, - 0xad, 0xd7, 0xa1, 0xfc, 0xfc, 0x60, 0x6c, 0xf5, 0x07, 0xd6, 0xa0, 0x5f, 0xcf, 0xa0, 0x0a, 0x14, - 0x75, 0xc7, 0x30, 0xff, 0x57, 0x80, 0xfa, 0x38, 0xe1, 0x71, 0xc2, 0xad, 0x28, 0xe1, 0x84, 0x4a, - 0x81, 0x47, 0x2b, 0x02, 0x77, 0xbe, 0x45, 0x94, 0x2b, 0x91, 0xd7, 0x35, 0x5e, 0x52, 0x27, 0xfb, - 0xf6, 0xea, 0xbc, 0x03, 0xd5, 0x13, 0x87, 0x9d, 0xd8, 0xba, 0x14, 0x09, 0xc1, 0xd7, 0xad, 0x8a, - 0xb0, 0xa5, 0x52, 0x30, 0x14, 0xc0, 0x06, 0x75, 0x42, 0x8f, 0xd8, 0x54, 0xb2, 0xb2, 0x59, 0x4c, - 0x5c, 0x99, 0xd6, 0x95, 0xed, 0x9d, 0x5b, 0x2c, 0xc4, 0x12, 0x18, 0x8b, 0xbe, 0x22, 0x72, 0x87, - 0xae, 0x9a, 0xd1, 0x23, 0xd8, 0xc0, 0x3e, 0x13, 0xe5, 0xd8, 0x3e, 0x4a, 0x8e, 0x8f, 0xd3, 0x5c, - 0xca, 0x6f, 0x19, 0x0f, 0x4a, 0x2a, 0xa2, 0xae, 0x86, 0x7b, 0x7a, 0xb4, 0xf9, 0xb7, 0x35, 0xb8, - 0x73, 0x05, 0x1d, 0x7d, 0x06, 0x79, 0x71, 0x21, 0xe9, 0xda, 0xfa, 0xd1, 0x9b, 0x13, 0x6d, 0x4f, - 0x62, 0x47, 0x17, 0xbc, 0x14, 0x53, 0x88, 0x86, 0xc9, 0xb1, 0x93, 0x04, 0xdc, 0xc6, 0x84, 0xf1, - 0xb4, 0xbe, 0x58, 0x15, 0x65, 0xeb, 0x13, 0xc6, 0xd1, 0x14, 0xca, 0xf2, 0xee, 0xf4, 0x43, 0x4f, - 0x67, 0xf1, 0xe8, 0x2d, 0x38, 0xa4, 0x7b, 0x31, 0x50, 0x88, 0xba, 0xfc, 0xce, 0x67, 0x68, 0xbe, - 0x84, 0xda, 0xaa, 0x0b, 0xba, 0x0f, 0x85, 0x74, 0x4f, 0xaf, 0xdd, 0x03, 0xe2, 0x9a, 0x78, 0x02, - 0x25, 0x1d, 0xac, 0xae, 0x81, 0x77, 0x6f, 0x38, 0x10, 0x7d, 0x71, 0xf3, 0x5f, 0x99, 0x78, 0x1e, - 0xdb, 0xdc, 0x87, 0x9c, 0x90, 0x07, 0xdd, 0x85, 0x3c, 0xe3, 0x0e, 0xe5, 0x72, 0xb2, 0xaa, 0x95, - 0x76, 0x50, 0x1d, 0xd6, 0x48, 0x98, 0x96, 0xdf, 0xaa, 0x25, 0x9a, 0x82, 0x55, 0x9a, 0x79, 0xb2, - 0x96, 0xe6, 0x35, 0xab, 0xd4, 0x66, 0x7e, 0xa4, 0x8e, 0x57, 0x1d, 0xaa, 0xcf, 0xba, 0x93, 0x89, - 0x7d, 0x38, 0xb4, 0xc6, 0xcf, 0xf7, 0x86, 0x69, 0x45, 0x7b, 0x3a, 0xb2, 0xac, 0xb1, 0x55, 0x37, - 0xc4, 0x69, 0xeb, 0xbd, 0xb0, 0x87, 0xdd, 0xc9, 0xb0, 0x9e, 0x45, 0x55, 0x28, 0xf5, 0x5e, 0xd8, - 0x56, 0xf7, 0x60, 0x6f, 0x50, 0x5f, 0x33, 0xbf, 0x30, 0xa0, 0x2c, 0x09, 0x8f, 0xc2, 0xe3, 0x68, - 0x65, 0x91, 0xc6, 0x9b, 0x2f, 0x12, 0x7d, 0xa8, 0x0e, 0x6f, 0x5a, 0xd1, 0xbe, 0x77, 0xe5, 0x90, - 0x41, 0xe6, 0xef, 0xa1, 0xf6, 0x8c, 0x46, 0x38, 0x71, 0x09, 0x1d, 0x12, 0x07, 0x13, 0x8a, 0x1e, - 0x41, 0xf1, 0x38, 0x88, 0x66, 0xe2, 0x72, 0x90, 0x6a, 0xf5, 0x1a, 0xc2, 0xfd, 0x1f, 0x17, 0xad, - 0xc2, 0x93, 0x20, 0x9a, 0x8d, 0xfa, 0x97, 0xf3, 0x96, 0x55, 0x10, 0x8e, 0x23, 0xfc, 0x16, 0xb7, - 0x99, 0xf9, 0x17, 0x03, 0xaa, 0x9a, 0x40, 0xdf, 0xe1, 0x0e, 0xfa, 0x11, 0x94, 0xa9, 0x33, 0xb3, - 0x8f, 0xce, 0x39, 0x61, 0x6a, 0xbb, 0x4a, 0xd4, 0x99, 0xf5, 0x44, 0x1f, 0x59, 0x50, 0x9a, 0x12, - 0xee, 0x88, 0x7f, 0x4c, 0x55, 0x5d, 0x1e, 0xde, 0x9c, 0xb5, 0x16, 0x99, 0x46, 0x9c, 0x68, 0xf0, - 0xa7, 0x2a, 0x4e, 0xcb, 0xa7, 0x71, 0xd0, 0x7b, 0x50, 0x0b, 0x93, 0xa9, 0x4d, 0xa6, 0x31, 0x3f, - 0xb7, 0x69, 0x34, 0x63, 0x2b, 0x7b, 0x5f, 0x0d, 0x93, 0xe9, 0x40, 0x0c, 0x59, 0xd1, 0x8c, 0x99, - 0x5f, 0x19, 0x70, 0x67, 0x01, 0xc8, 0x98, 0xe3, 0x11, 0xf4, 0x31, 0x14, 0x4e, 0xa4, 0x72, 0xea, - 0x5f, 0xf1, 0xc1, 0xcd, 0x8c, 0x56, 0x95, 0xb6, 0x54, 0x1c, 0xea, 0x42, 0x81, 0x9f, 0xc7, 0x69, - 0xae, 0x8b, 0x35, 0xfd, 0xe4, 0x66, 0x84, 0x79, 0xf6, 0xe8, 0xd4, 0x4c, 0x03, 0xd1, 0xc7, 0x90, - 0x93, 0xa2, 0xac, 0x49, 0x0a, 0x3f, 0xfd, 0x6e, 0x0a, 0xfd, 0x85, 0x14, 0x32, 0xd2, 0xbc, 0xc8, - 0xc3, 0xbd, 0xd7, 0x2b, 0x86, 0x7e, 0x0b, 0x90, 0x56, 0x58, 0x3f, 0x3c, 0x8e, 0xd4, 0x2a, 0x3f, - 0xbc, 0xad, 0xee, 0x69, 0xcd, 0x10, 0xd4, 0xd9, 0x30, 0x63, 0x95, 0xa9, 0xee, 0xa1, 0xc7, 0x90, - 0x27, 0x4b, 0x7f, 0xc7, 0xad, 0x9b, 0x81, 0xf5, 0x2f, 0x71, 0xea, 0x8f, 0x3e, 0x03, 0x10, 0x7f, - 0xf9, 0xc4, 0x5e, 0x5a, 0xf9, 0xce, 0xad, 0x69, 0x1d, 0x0a, 0x08, 0xa1, 0x86, 0x60, 0xc5, 0x75, - 0x07, 0xed, 0x41, 0x8d, 0x9f, 0x85, 0xb6, 0x1b, 0x45, 0x14, 0xdb, 0x22, 0x57, 0xd4, 0x95, 0xd2, - 0x7a, 0xcd, 0xcf, 0xfb, 0xe1, 0x59, 0xb8, 0x2b, 0xfc, 0x04, 0xe6, 0x30, 0x63, 0x55, 0xf9, 0x52, - 0x1f, 0x59, 0x50, 0xa4, 0xd1, 0xcc, 0x0e, 0x93, 0xa9, 0xbc, 0x26, 0x2a, 0xdb, 0x8f, 0x6f, 0xaf, - 0x5c, 0x34, 0x3b, 0x48, 0xa6, 0xc3, 0x8c, 0x55, 0xa0, 0xb2, 0xd5, 0x1c, 0x03, 0x2c, 0xd4, 0x44, - 0xdd, 0x2b, 0xdb, 0x23, 0x52, 0xe8, 0xfe, 0x6b, 0x68, 0xce, 0x43, 0x74, 0x7d, 0x9e, 0xef, 0x41, - 0xf3, 0x77, 0x50, 0x9e, 0xeb, 0x80, 0x26, 0x70, 0xc7, 0x8d, 0x82, 0x80, 0xb8, 0x5c, 0x3d, 0x9b, - 0xf4, 0x2d, 0xb5, 0x5c, 0x9e, 0xc4, 0x23, 0xab, 0xad, 0x1e, 0x59, 0x6d, 0x4b, 0x3d, 0xb2, 0x96, - 0xae, 0xa2, 0xda, 0x1c, 0x42, 0x18, 0x59, 0x73, 0x06, 0x85, 0x74, 0x19, 0xe8, 0xe7, 0x50, 0x66, - 0x24, 0xc4, 0x84, 0xea, 0x0a, 0x53, 0xee, 0xd5, 0xe7, 0xc5, 0x42, 0x0e, 0xc8, 0x02, 0x91, 0xb6, - 0xb0, 0x78, 0x31, 0x68, 0xfd, 0xb2, 0xcb, 0x35, 0x39, 0x95, 0x02, 0xb5, 0xa0, 0x14, 0x38, 0x8c, - 0xdb, 0x53, 0xe6, 0xc9, 0x14, 0xd0, 0xd7, 0x70, 0x51, 0x58, 0x9f, 0x32, 0xaf, 0x57, 0x84, 0xfc, - 0x4b, 0x27, 0x48, 0x88, 0xf9, 0x47, 0x03, 0x1a, 0x7d, 0x9f, 0xf1, 0xc9, 0xaf, 0xf6, 0x7f, 0x9d, - 0x3e, 0x85, 0xf6, 0x22, 0xc6, 0xfc, 0x58, 0x26, 0xe1, 0xc3, 0xd5, 0x47, 0xd3, 0x7a, 0xef, 0x9e, - 0x40, 0xf9, 0xfa, 0xa2, 0x55, 0x5b, 0x0d, 0x59, 0x3c, 0xa3, 0x86, 0x70, 0x77, 0xea, 0x87, 0xb6, - 0xe3, 0xba, 0x24, 0x16, 0x42, 0xe9, 0xf0, 0xec, 0xb7, 0x86, 0xa3, 0xa9, 0x1f, 0x76, 0x55, 0x88, - 0xb2, 0x99, 0x8f, 0xe1, 0x07, 0xca, 0xab, 0x4f, 0x1d, 0x3f, 0xf4, 0x43, 0x4f, 0x52, 0xda, 0x82, - 0x12, 0x56, 0x7d, 0xc9, 0x49, 0xaf, 0x6c, 0x6e, 0xed, 0xbd, 0xfb, 0xea, 0xdf, 0x9b, 0x99, 0x57, - 0x97, 0x9b, 0xc6, 0xdf, 0x2f, 0x37, 0x8d, 0xaf, 0x2e, 0x37, 0x8d, 0x7f, 0x5d, 0x6e, 0x1a, 0x5f, - 0xfc, 0x67, 0x33, 0xf3, 0x29, 0x2c, 0x92, 0xea, 0x9b, 0x00, 0x00, 0x00, 0xff, 0xff, 0x35, 0x0b, - 0x40, 0x71, 0xe2, 0x0f, 0x00, 0x00, + 0xf9, 0xd6, 0xc8, 0xfa, 0xfb, 0x4a, 0xd6, 0xca, 0xfc, 0x2d, 0x16, 0xfa, 0xa9, 0x5b, 0xcb, 0x99, + 0x06, 0x85, 0x1b, 0xa4, 0xd2, 0xae, 0x73, 0x58, 0xd4, 0x39, 0x24, 0x92, 0xa5, 0xb5, 0xd4, 0x78, + 0xad, 0xed, 0xc8, 0xdb, 0x62, 0x93, 0x02, 0xd3, 0xf1, 0x90, 0x1e, 0x0f, 0x32, 0x9a, 0x19, 0x93, + 0x9c, 0x95, 0x7d, 0xe9, 0xa9, 0x1f, 0x20, 0xa7, 0xa2, 0xc7, 0x9c, 0x7a, 0xe9, 0x07, 0xe8, 0x37, + 0x28, 0xf6, 0xd8, 0x63, 0xd0, 0x02, 0x46, 0xeb, 0x5e, 0xfa, 0x15, 0x1a, 0xa0, 0x40, 0x41, 0x0e, + 0xa9, 0x3f, 0xf6, 0x3a, 0x89, 0xb3, 0x97, 0x01, 0xf9, 0xf2, 0x7d, 0x1f, 0x3e, 0x7c, 0xf8, 0xf2, + 0xe5, 0x10, 0xfe, 0x9f, 0x9d, 0x05, 0x1d, 0xec, 0x33, 0xce, 0xce, 0x02, 0x9a, 0x84, 0x1d, 0xec, + 0x70, 0xa7, 0x1d, 0xd3, 0x88, 0x47, 0xa8, 0xe1, 0x46, 0xee, 0xe7, 0x34, 0x72, 0xdc, 0xd3, 0x36, + 0x3b, 0x0b, 0xda, 0x0b, 0xa7, 0xe6, 0x86, 0xb4, 0xc6, 0xc7, 0x1d, 0x27, 0xf6, 0x53, 0xe7, 0x26, + 0xd2, 0xa6, 0x05, 0x40, 0xf3, 0xbe, 0xb6, 0x11, 0x4a, 0x23, 0xca, 0x94, 0xb5, 0x25, 0x66, 0x8c, + 0xbd, 0x99, 0x4f, 0x49, 0x27, 0xf6, 0xe4, 0xd0, 0xaa, 0xc3, 0x43, 0xe1, 0xc0, 0xce, 0x82, 0x63, + 0x87, 0x91, 0x0e, 0xe3, 0x34, 0x71, 0x79, 0x42, 0x09, 0x5e, 0x0e, 0xd7, 0xa3, 0x24, 0x74, 0x23, + 0x4c, 0xb0, 0x8d, 0x1d, 0x9e, 0x4c, 0x95, 0xc3, 0x56, 0xc2, 0xfd, 0xa0, 0xc3, 0xa9, 0xe3, 0xfa, + 0xa1, 0xd7, 0xa1, 0xc4, 0x8d, 0xa8, 0x70, 0x61, 0xb1, 0x13, 0x6a, 0x5e, 0x5e, 0xe4, 0x45, 0xb2, + 0xd9, 0x11, 0xad, 0xd4, 0x6a, 0xfe, 0xd9, 0x80, 0xfc, 0x40, 0xf0, 0x40, 0x3d, 0x28, 0xc5, 0x9e, + 0x2d, 0x39, 0x35, 0x8c, 0x2d, 0x63, 0xbb, 0xb2, 0xd3, 0x68, 0x2f, 0xb4, 0x50, 0x9c, 0xdb, 0xd2, + 0xb7, 0x57, 0xb9, 0xba, 0x6c, 0x15, 0x9f, 0xef, 0xcb, 0xce, 0x30, 0x63, 0x15, 0x63, 0x2f, 0xc5, + 0xf8, 0x14, 0x36, 0x28, 0xe1, 0xf4, 0xc2, 0x39, 0x0e, 0xc8, 0xd1, 0x79, 0x28, 0x8d, 0x8d, 0xac, + 0x04, 0x7b, 0x6f, 0x09, 0x4c, 0x29, 0xd4, 0x7e, 0x11, 0x9e, 0x3a, 0x21, 0x0e, 0x08, 0xb6, 0x74, + 0x90, 0x46, 0xbc, 0x09, 0xb3, 0x9b, 0xfb, 0xc3, 0x97, 0xad, 0x4c, 0xaf, 0x04, 0x05, 0x4c, 0xb8, + 0xe3, 0x07, 0x26, 0x05, 0x18, 0x9c, 0xc7, 0x94, 0x30, 0xe6, 0x47, 0x21, 0xda, 0x84, 0xe2, 0x2b, + 0x42, 0x45, 0x53, 0x92, 0x2f, 0xf7, 0x72, 0xaf, 0x2f, 0x5b, 0x19, 0x4b, 0x1b, 0x51, 0x03, 0x72, + 0xe4, 0x3c, 0x4e, 0xc9, 0xe8, 0x41, 0x69, 0x41, 0xef, 0x41, 0x2d, 0x88, 0x5c, 0x27, 0xb0, 0x45, + 0xcf, 0xf6, 0xf1, 0x79, 0x63, 0x6d, 0xcb, 0xd8, 0x5e, 0x57, 0x3e, 0x55, 0x39, 0x26, 0x26, 0x1a, + 0xe1, 0x73, 0xf3, 0x77, 0x59, 0x28, 0x8d, 0x29, 0x26, 0xd4, 0x0f, 0x3d, 0x34, 0x82, 0xa2, 0x1b, + 0x05, 0xc9, 0x34, 0x64, 0x0d, 0x63, 0x6b, 0x6d, 0xbb, 0xb2, 0xf3, 0x93, 0xf6, 0x6d, 0xb9, 0xd3, + 0xd6, 0x41, 0xed, 0x3d, 0x19, 0xa1, 0xd9, 0xa9, 0xf8, 0xe6, 0x1f, 0x0d, 0x28, 0xa4, 0x23, 0xe8, + 0x87, 0x12, 0x55, 0xf2, 0x30, 0x96, 0x78, 0x14, 0xdc, 0x28, 0x18, 0xe1, 0x73, 0xf4, 0x2b, 0x28, + 0x63, 0x9f, 0x12, 0x97, 0x8b, 0x95, 0x8a, 0xc5, 0xd4, 0x76, 0x3e, 0xf8, 0xce, 0xd3, 0xb6, 0xfb, + 0x3a, 0x54, 0xa1, 0x2e, 0xb0, 0xcc, 0x4d, 0x28, 0xcf, 0x47, 0x51, 0x11, 0xd6, 0xba, 0x93, 0xbd, + 0x7a, 0x06, 0x95, 0x20, 0xd7, 0x1f, 0x4c, 0xf6, 0xea, 0xc6, 0x6e, 0xee, 0xdf, 0x5f, 0xb6, 0xd4, + 0xd7, 0xfc, 0x6f, 0x16, 0xd0, 0x84, 0x53, 0xe2, 0x4c, 0x07, 0x21, 0x8e, 0x23, 0x3f, 0xe4, 0x93, + 0x98, 0xb8, 0xe8, 0x13, 0xc8, 0xf1, 0x8b, 0x98, 0x48, 0xde, 0xb5, 0x9d, 0xc7, 0xb7, 0xd3, 0xba, + 0x19, 0xdb, 0x3e, 0xba, 0x88, 0x89, 0xde, 0x16, 0x01, 0x82, 0x7e, 0x06, 0x65, 0x26, 0xdd, 0x6c, + 0x1f, 0xcb, 0x85, 0xe6, 0x7b, 0x0f, 0xc5, 0xf0, 0xd5, 0x65, 0xab, 0x94, 0xc6, 0x8f, 0xfa, 0x5f, + 0x2f, 0xb5, 0xad, 0x52, 0xea, 0x3e, 0xc2, 0x68, 0x17, 0x1e, 0x60, 0x12, 0x53, 0xe2, 0x3a, 0x9c, + 0x60, 0x9b, 0x3b, 0xd4, 0x23, 0xdc, 0x76, 0x30, 0xa6, 0x72, 0x67, 0xf5, 0xee, 0xdf, 0x5f, 0xf8, + 0x1c, 0x49, 0x97, 0x2e, 0xc6, 0x14, 0x9d, 0x41, 0x4d, 0x05, 0x84, 0x11, 0x26, 0x62, 0xee, 0x9c, + 0x9c, 0xfb, 0x13, 0x35, 0x77, 0x35, 0xf5, 0x3d, 0x8c, 0x30, 0x91, 0xf3, 0x7f, 0xe0, 0xf9, 0xfc, + 0x34, 0x39, 0x6e, 0xbb, 0xd1, 0xb4, 0x33, 0x5f, 0x2f, 0x3e, 0x5e, 0xb4, 0x3b, 0xf1, 0xe7, 0x5e, + 0x47, 0x27, 0x7c, 0x1a, 0x66, 0x55, 0xf9, 0x02, 0x04, 0x9b, 0x8f, 0x20, 0x27, 0x56, 0x8f, 0xca, + 0x90, 0x3f, 0x18, 0xef, 0x75, 0x0f, 0xea, 0x19, 0x04, 0x50, 0xb0, 0x06, 0xcf, 0xc6, 0x47, 0x83, + 0xba, 0x81, 0x36, 0x60, 0x7d, 0xf2, 0xf2, 0x70, 0xcf, 0xb6, 0x06, 0x93, 0xe7, 0xe3, 0xc3, 0xc9, + 0xa0, 0x9e, 0x35, 0xff, 0x9e, 0x85, 0xf5, 0x51, 0x18, 0x27, 0x7c, 0x72, 0x11, 0xba, 0x52, 0xfa, + 0xa7, 0x2b, 0xd2, 0xbf, 0x7f, 0xbb, 0xf4, 0x2b, 0x61, 0x37, 0x55, 0xef, 0x43, 0x29, 0x52, 0x39, + 0xa3, 0xce, 0xad, 0xf9, 0xed, 0xd9, 0xa5, 0x10, 0xe6, 0x91, 0xe8, 0x00, 0x8a, 0xe9, 0x66, 0xb0, + 0xc6, 0x9a, 0x3c, 0x19, 0xef, 0xdf, 0x25, 0x17, 0xf4, 0xe1, 0x50, 0x10, 0xe8, 0xe7, 0x50, 0x4d, + 0xcf, 0x89, 0x2d, 0x28, 0xb2, 0x46, 0x4e, 0x42, 0xbe, 0x73, 0x0d, 0x52, 0x15, 0x47, 0x95, 0xe9, + 0x4b, 0x0b, 0xab, 0xb8, 0x73, 0x0b, 0x33, 0x4d, 0xa5, 0xf5, 0x3a, 0x94, 0x5f, 0x1c, 0x8e, 0xad, + 0xfe, 0xc0, 0x1a, 0xf4, 0xeb, 0x19, 0x54, 0x81, 0xa2, 0xee, 0x18, 0xe6, 0x7f, 0x0a, 0x50, 0x1f, + 0x27, 0x3c, 0x4e, 0xb8, 0x15, 0x25, 0x9c, 0x50, 0x29, 0xf0, 0x68, 0x45, 0xe0, 0xce, 0x37, 0x88, + 0x72, 0x2d, 0xf2, 0xa6, 0xc6, 0x4b, 0xea, 0x64, 0xdf, 0x5e, 0x9d, 0x77, 0xa0, 0x7a, 0xea, 0xb0, + 0x53, 0x5b, 0x97, 0x22, 0x21, 0xf8, 0xba, 0x55, 0x11, 0xb6, 0x54, 0x0a, 0x86, 0x02, 0xd8, 0xa0, + 0x4e, 0xe8, 0x11, 0x9b, 0x4a, 0x56, 0x36, 0x8b, 0x89, 0x2b, 0xd3, 0xba, 0xb2, 0xb3, 0x7b, 0x87, + 0x85, 0x58, 0x02, 0x63, 0xd1, 0x57, 0x44, 0xee, 0xd1, 0x55, 0x33, 0x7a, 0x0c, 0x1b, 0xd8, 0x67, + 0xa2, 0x74, 0xdb, 0xc7, 0xc9, 0xc9, 0x49, 0x9a, 0x4b, 0xf9, 0x2d, 0x63, 0xbb, 0xa4, 0x22, 0xea, + 0x6a, 0xb8, 0xa7, 0x47, 0x9b, 0x7f, 0x59, 0x83, 0x7b, 0xd7, 0xd0, 0xd1, 0x67, 0x90, 0x17, 0x97, + 0x97, 0xae, 0xad, 0x1f, 0x7d, 0x7f, 0xa2, 0xed, 0x49, 0xec, 0xe8, 0x82, 0x97, 0x62, 0x0a, 0xd1, + 0x30, 0x39, 0x71, 0x92, 0x80, 0xdb, 0x98, 0x30, 0x9e, 0xd6, 0x17, 0xab, 0xa2, 0x6c, 0x7d, 0xc2, + 0x38, 0x9a, 0x42, 0x59, 0xde, 0xb3, 0x7e, 0xe8, 0xe9, 0x2c, 0x1e, 0xbd, 0x05, 0x87, 0x74, 0x2f, + 0x06, 0x0a, 0x51, 0x97, 0xdf, 0xf9, 0x0c, 0xcd, 0x57, 0x50, 0x5b, 0x75, 0x41, 0x0f, 0xa1, 0x90, + 0xee, 0xe9, 0x8d, 0x7b, 0x40, 0x5c, 0x13, 0x4f, 0xa1, 0xa4, 0x83, 0xd5, 0x35, 0xf0, 0xee, 0x2d, + 0x07, 0xa2, 0x2f, 0xfe, 0x12, 0xae, 0x4d, 0x3c, 0x8f, 0x6d, 0x1e, 0x40, 0x4e, 0xc8, 0x83, 0xee, + 0x43, 0x9e, 0x71, 0x87, 0x72, 0x39, 0x59, 0xd5, 0x4a, 0x3b, 0xa8, 0x0e, 0x6b, 0x24, 0x4c, 0xcb, + 0x6f, 0xd5, 0x12, 0x4d, 0xc1, 0x2a, 0xcd, 0x3c, 0x59, 0x4b, 0xf3, 0x9a, 0x55, 0x6a, 0x33, 0x3f, + 0x52, 0xc7, 0xab, 0x0e, 0xd5, 0xe7, 0xdd, 0xc9, 0xc4, 0x3e, 0x1a, 0x5a, 0xe3, 0x17, 0xfb, 0xc3, + 0xb4, 0xa2, 0x3d, 0x1b, 0x59, 0xd6, 0xd8, 0xaa, 0x1b, 0xe2, 0xb4, 0xf5, 0x5e, 0xda, 0xc3, 0xee, + 0x64, 0x58, 0xcf, 0xa2, 0x2a, 0x94, 0x7a, 0x2f, 0x6d, 0xab, 0x7b, 0xb8, 0x3f, 0xa8, 0xaf, 0x99, + 0x5f, 0x18, 0x50, 0x96, 0x84, 0x47, 0xe1, 0x49, 0xb4, 0xb2, 0x48, 0xe3, 0xfb, 0x2f, 0x12, 0x7d, + 0xa8, 0x0e, 0x6f, 0x5a, 0xd1, 0xbe, 0x73, 0xe5, 0x90, 0x41, 0xe6, 0x6f, 0xa1, 0xf6, 0x9c, 0x46, + 0x38, 0x71, 0x09, 0x1d, 0x12, 0x07, 0x13, 0x8a, 0x1e, 0x43, 0xf1, 0x24, 0x88, 0x66, 0xe2, 0x72, + 0x90, 0x6a, 0xf5, 0x1a, 0xc2, 0xfd, 0x6f, 0x97, 0xad, 0xc2, 0xd3, 0x20, 0x9a, 0x8d, 0xfa, 0x57, + 0xf3, 0x96, 0x55, 0x10, 0x8e, 0x23, 0xfc, 0x16, 0xb7, 0x99, 0xf9, 0x27, 0x03, 0xaa, 0x9a, 0x40, + 0xdf, 0xe1, 0x0e, 0xfa, 0x01, 0x94, 0xa9, 0x33, 0xb3, 0x8f, 0x2f, 0x38, 0x61, 0x6a, 0xbb, 0x4a, + 0xd4, 0x99, 0xf5, 0x44, 0x1f, 0x59, 0x50, 0x9a, 0x12, 0xee, 0x88, 0xff, 0x51, 0x55, 0x5d, 0x1e, + 0xdd, 0x9e, 0xb5, 0x16, 0x99, 0x46, 0x9c, 0x68, 0xf0, 0x67, 0x2a, 0x4e, 0xcb, 0xa7, 0x71, 0xc4, + 0x1f, 0x52, 0x98, 0x4c, 0x6d, 0x32, 0x8d, 0xf9, 0x85, 0x4d, 0xa3, 0x19, 0x5b, 0xd9, 0xfb, 0x6a, + 0x98, 0x4c, 0x07, 0x62, 0xc8, 0x8a, 0x66, 0xcc, 0xfc, 0xca, 0x80, 0x7b, 0x0b, 0x40, 0xc6, 0x1c, + 0x8f, 0xa0, 0x8f, 0xa1, 0x70, 0x2a, 0x95, 0x53, 0xff, 0x95, 0xdb, 0xb7, 0x33, 0x5a, 0x55, 0xda, + 0x52, 0x71, 0xa8, 0x0b, 0x05, 0x7e, 0x11, 0xa7, 0xb9, 0x2e, 0xd6, 0xf4, 0xa3, 0xdb, 0x11, 0xe6, + 0xd9, 0xa3, 0x53, 0x33, 0x0d, 0x44, 0x1f, 0x43, 0x4e, 0x8a, 0xb2, 0x26, 0x29, 0xfc, 0xf8, 0xdb, + 0x29, 0xf4, 0x17, 0x52, 0xc8, 0x48, 0xf3, 0x32, 0x0f, 0x0f, 0xde, 0xac, 0x18, 0xfa, 0x35, 0x40, + 0x5a, 0x61, 0xfd, 0xf0, 0x24, 0x52, 0xab, 0xfc, 0xf0, 0xae, 0xba, 0xa7, 0x35, 0x43, 0x50, 0x67, + 0xc3, 0x8c, 0x55, 0xa6, 0xba, 0x87, 0x9e, 0x40, 0x9e, 0x2c, 0xfd, 0x49, 0xb7, 0x6e, 0x07, 0xd6, + 0xbf, 0xcf, 0xa9, 0x3f, 0xfa, 0x0c, 0x40, 0xbc, 0x08, 0x88, 0xbd, 0xb4, 0xf2, 0xdd, 0x3b, 0xd3, + 0x3a, 0x12, 0x10, 0x42, 0x0d, 0xc1, 0x8a, 0xeb, 0x0e, 0xda, 0x87, 0x1a, 0x3f, 0x0f, 0x6d, 0x37, + 0x8a, 0x28, 0xb6, 0x45, 0xae, 0xa8, 0x2b, 0xa5, 0xf5, 0x86, 0x1f, 0xfd, 0xa3, 0xf3, 0x70, 0x4f, + 0xf8, 0x09, 0xcc, 0x61, 0xc6, 0xaa, 0xf2, 0xa5, 0x3e, 0xb2, 0xa0, 0x48, 0xa3, 0x99, 0x1d, 0x26, + 0x53, 0x79, 0x4d, 0x54, 0x76, 0x9e, 0xdc, 0x5d, 0xb9, 0x68, 0x76, 0x98, 0x4c, 0x87, 0x19, 0xab, + 0x40, 0x65, 0xab, 0x39, 0x06, 0x58, 0xa8, 0x89, 0xba, 0xd7, 0xb6, 0x47, 0xa4, 0xd0, 0xc3, 0x37, + 0xd0, 0x9c, 0x87, 0xe8, 0xfa, 0x3c, 0xdf, 0x83, 0xe6, 0x6f, 0xa0, 0x3c, 0xd7, 0x01, 0x4d, 0xe0, + 0x9e, 0x1b, 0x05, 0x01, 0x71, 0xb9, 0x7a, 0x62, 0xe9, 0x5b, 0x6a, 0xb9, 0x3c, 0x89, 0x07, 0x59, + 0x5b, 0x3d, 0xc8, 0xda, 0x96, 0x7a, 0x90, 0x2d, 0x5d, 0x45, 0xb5, 0x39, 0x84, 0x30, 0xb2, 0xe6, + 0x0c, 0x0a, 0xe9, 0x32, 0xd0, 0x4f, 0xa1, 0xcc, 0x48, 0x88, 0x09, 0xd5, 0x15, 0xa6, 0xdc, 0xab, + 0xcf, 0x8b, 0x85, 0x1c, 0x90, 0x05, 0x22, 0x6d, 0x61, 0xf1, 0x62, 0xd0, 0xfa, 0x65, 0x97, 0x6b, + 0x72, 0x2a, 0x05, 0x6a, 0x41, 0x29, 0x70, 0x18, 0xb7, 0xa7, 0xcc, 0x93, 0x29, 0xa0, 0xaf, 0xe1, + 0xa2, 0xb0, 0x3e, 0x63, 0x5e, 0xaf, 0x08, 0xf9, 0x57, 0x4e, 0x90, 0x10, 0xf3, 0xf7, 0x06, 0x34, + 0xfa, 0x3e, 0xe3, 0x93, 0x5f, 0x1c, 0xfc, 0x32, 0x7d, 0x36, 0xed, 0x47, 0x8c, 0xf9, 0xb1, 0x4c, + 0xc2, 0x47, 0xab, 0x0f, 0xac, 0xf5, 0xde, 0x03, 0x81, 0xf2, 0xf5, 0x65, 0xab, 0xb6, 0x1a, 0xb2, + 0x78, 0x72, 0x0d, 0xe1, 0xfe, 0xd4, 0x0f, 0x6d, 0xc7, 0x75, 0x49, 0x2c, 0x84, 0xd2, 0xe1, 0xd9, + 0x6f, 0x0c, 0x47, 0x53, 0x3f, 0xec, 0xaa, 0x10, 0x65, 0x33, 0x9f, 0xc0, 0xff, 0x29, 0xaf, 0x3e, + 0x75, 0xfc, 0xd0, 0x0f, 0x3d, 0x49, 0x69, 0x0b, 0x4a, 0x58, 0xf5, 0x25, 0x27, 0xbd, 0xb2, 0xb9, + 0xb5, 0xf7, 0xee, 0xeb, 0x7f, 0x6e, 0x66, 0x5e, 0x5f, 0x6d, 0x1a, 0x7f, 0xbd, 0xda, 0x34, 0xbe, + 0xba, 0xda, 0x34, 0xfe, 0x71, 0xb5, 0x69, 0x7c, 0xf1, 0xaf, 0xcd, 0xcc, 0xa7, 0xb0, 0x48, 0xaa, + 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x5e, 0x6f, 0xe2, 0xc0, 0x0e, 0x10, 0x00, 0x00, } diff --git a/pkg/sql/distsqlrun/data.proto b/pkg/sql/distsqlrun/data.proto index c355876b6ad8..37e91c79eb60 100644 --- a/pkg/sql/distsqlrun/data.proto +++ b/pkg/sql/distsqlrun/data.proto @@ -48,6 +48,8 @@ message Expression { // SQL expressions are passed as a string, with ordinal references // (@1, @2, @3 ..) used for "input" variables. optional string expr = 2 [(gogoproto.nullable) = false]; + + optional uint32 local_expr_idx = 3 [(gogoproto.nullable) = false]; } // Ordering defines an order - specifically a list of column indices and diff --git a/pkg/sql/distsqlrun/expr.go b/pkg/sql/distsqlrun/expr.go index 9c22b6419f48..282a257508b0 100644 --- a/pkg/sql/distsqlrun/expr.go +++ b/pkg/sql/distsqlrun/expr.go @@ -146,11 +146,21 @@ func (eh *exprHelper) init( expr Expression, types []sqlbase.ColumnType, evalCtx *tree.EvalContext, ) error { eh.evalCtx = evalCtx - if expr.Expr == "" { + if expr.Expr == "" && evalCtx.LocalExprs == nil { return nil } eh.types = types eh.vars = tree.MakeIndexedVarHelper(eh, len(types)) + + if expr.LocalExprIdx != 0 { + eh.expr = (*evalCtx.LocalExprs)[expr.LocalExprIdx-1] + if eh.expr == nil { + return errors.Errorf("programming error: local expr %d was not available in slice %v", expr.LocalExprIdx-1, evalCtx.LocalExprs) + } + // Bind IndexedVars to our eh.vars. + eh.vars.Rebind(eh.expr, true /* alsoReset */, false /* normalizeToNonNil */) + return nil + } var err error semaContext := tree.MakeSemaContext(evalCtx.SessionData.User == security.RootUser) eh.expr, err = processExpression(expr, evalCtx, &semaContext, &eh.vars) diff --git a/pkg/sql/distsqlrun/processors.go b/pkg/sql/distsqlrun/processors.go index 13e6e71309a7..adf754d79759 100644 --- a/pkg/sql/distsqlrun/processors.go +++ b/pkg/sql/distsqlrun/processors.go @@ -103,7 +103,7 @@ func (h *ProcOutputHelper) Init( } h.output = output h.numInternalCols = len(types) - if post.Filter.Expr != "" { + if post.Filter != (Expression{}) { h.filter = &exprHelper{} if err := h.filter.init(post.Filter, types, evalCtx); err != nil { return err diff --git a/pkg/sql/explain_distsql.go b/pkg/sql/explain_distsql.go index d66b800bbe91..fbf37e8c65ec 100644 --- a/pkg/sql/explain_distsql.go +++ b/pkg/sql/explain_distsql.go @@ -69,11 +69,11 @@ func (n *explainDistSQLNode) startExec(params runParams) error { planCtx.stmtType = n.stmtType planCtx.validExtendedEvalCtx = true - plan, err := distSQLPlanner.createPlanForNode(&planCtx, n.plan) + plan, err := distSQLPlanner.createPlanForNode(planCtx, n.plan) if err != nil { return err } - distSQLPlanner.FinalizePlan(&planCtx, &plan) + distSQLPlanner.FinalizePlan(planCtx, &plan) var spans []tracing.RecordedSpan if n.analyze { @@ -118,7 +118,7 @@ func (n *explainDistSQLNode) startExec(params runParams) error { params.extendedEvalCtx.Tracing, ) distSQLPlanner.Run( - &planCtx, params.p.txn, &plan, recv, params.extendedEvalCtx, nil /* finishedSetupFn */) + planCtx, params.p.txn, &plan, recv, params.extendedEvalCtx, nil /* finishedSetupFn */) n.run.executedStatement = true diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go index efb40c1244d7..88789c2ff77f 100644 --- a/pkg/sql/plan.go +++ b/pkg/sql/plan.go @@ -677,7 +677,7 @@ func (p *planner) delegateQuery( initialCheck func(ctx context.Context) error, desiredTypes []types.T, ) (planNode, error) { - // log.VEventf(ctx, 2, "delegated query: %q", sql) + log.VEventf(ctx, 2, "delegated query: %q", sql) // Prepare the sub-plan. stmt, err := parser.ParseOne(sql) diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index 2955c07b8f12..2e94ac0c2501 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -110,7 +110,7 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { } planCtx := params.extendedEvalCtx.DistSQLPlanner.NewPlanningCtx(ctx, params.extendedEvalCtx, params.p.txn) - physPlan, err := scrubPlanDistSQL(ctx, &planCtx, plan) + physPlan, err := scrubPlanDistSQL(ctx, planCtx, plan) if err != nil { return err } @@ -131,7 +131,7 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { return errors.Errorf("could not find MergeJoinerSpec in plan") } - rows, err := scrubRunDistSQL(ctx, &planCtx, params.p, physPlan, columnTypes) + rows, err := scrubRunDistSQL(ctx, planCtx, params.p, physPlan, columnTypes) if err != nil { rows.Close(ctx) return err diff --git a/pkg/sql/scrub_index.go b/pkg/sql/scrub_index.go index 14322c626ab0..0e58c8da0392 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -119,7 +119,7 @@ func (o *indexCheckOperation) Start(params runParams) error { defer plan.Close(ctx) planCtx := params.extendedEvalCtx.DistSQLPlanner.NewPlanningCtx(ctx, params.extendedEvalCtx, params.p.txn) - physPlan, err := scrubPlanDistSQL(ctx, &planCtx, plan) + physPlan, err := scrubPlanDistSQL(ctx, planCtx, plan) if err != nil { return err } @@ -140,7 +140,7 @@ func (o *indexCheckOperation) Start(params runParams) error { return errors.Errorf("could not find MergeJoinerSpec in plan") } - rows, err := scrubRunDistSQL(ctx, &planCtx, params.p, physPlan, columnTypes) + rows, err := scrubRunDistSQL(ctx, planCtx, params.p, physPlan, columnTypes) if err != nil { rows.Close(ctx) return err diff --git a/pkg/sql/scrub_physical.go b/pkg/sql/scrub_physical.go index 6c59a1c41211..b920923b0668 100644 --- a/pkg/sql/scrub_physical.go +++ b/pkg/sql/scrub_physical.go @@ -136,7 +136,7 @@ func (o *physicalCheckOperation) Start(params runParams) error { planCtx := params.extendedEvalCtx.DistSQLPlanner.NewPlanningCtx(ctx, params.extendedEvalCtx, params.p.txn) physPlan, err := params.extendedEvalCtx.DistSQLPlanner.createScrubPhysicalCheck( - &planCtx, scan, *o.tableDesc, *o.indexDesc, spans, params.p.ExecCfg().Clock.Now()) + planCtx, scan, *o.tableDesc, *o.indexDesc, spans, params.p.ExecCfg().Clock.Now()) if err != nil { return err } @@ -144,7 +144,7 @@ func (o *physicalCheckOperation) Start(params runParams) error { o.primaryColIdxs = primaryColIdxs o.columns = columns o.run.started = true - rows, err := scrubRunDistSQL(ctx, &planCtx, params.p, &physPlan, distsqlrun.ScrubTypes) + rows, err := scrubRunDistSQL(ctx, planCtx, params.p, &physPlan, distsqlrun.ScrubTypes) if err != nil { rows.Close(ctx) return err diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 55a29f1f5c2a..fe6d77b1de04 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -2455,7 +2455,7 @@ type EvalContext struct { // InternalExecutor gives access to an executor to be used for running // "internal" statements. It may seem bizarre that "expression evaluation" may - // need to run a statement, and yet many builting function do it. + // need to run a statement, and yet many builtin functions do it. // Note that the executor will be "session-bound" - it will inherit session // variables from a parent session. InternalExecutor SessionBoundInternalExecutor @@ -2489,6 +2489,11 @@ type EvalContext struct { // evaluation. It can change over the course of evaluation, such as on a // per-row basis. ActiveMemAcc *mon.BoundAccount + + // LocalExprs contains the list of typed expressions that DistSQL Expressions + // can index into when running on the gateway. See distsqlrun.Expression and + // its LocalExprIdx field. + LocalExprs *[]TypedExpr } // MakeTestingEvalContext returns an EvalContext that includes a MemoryMonitor. diff --git a/pkg/sql/testutils.go b/pkg/sql/testutils.go index 9ca7d36b6290..688d7652df17 100644 --- a/pkg/sql/testutils.go +++ b/pkg/sql/testutils.go @@ -127,6 +127,6 @@ func (dsp *DistSQLPlanner) Exec( planCtx.planner = p planCtx.stmtType = recv.stmtType - dsp.PlanAndRun(ctx, evalCtx, &planCtx, p.txn, p.curPlan.plan, recv) + dsp.PlanAndRun(ctx, evalCtx, planCtx, p.txn, p.curPlan.plan, recv) return rw.Err() } From 26d98ae83d68f98a99f852fab7da5a70f81c7c43 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Fri, 21 Sep 2018 17:39:08 -0400 Subject: [PATCH 2/6] a new approach to local exprs Rather than having to do the confusing dance of keeping around the slice of Exprs, I found a way to have protobuf not generate the type declaration for a message. Using this, I added the local tree.Expr directly to the Expression struct when necessary. This simplifies the code a lot. Leaving this as a separate commit during review, will squash later. Release note: None --- pkg/sql/distsql_physical_planner.go | 35 +-- pkg/sql/distsql_running.go | 3 - pkg/sql/distsqlplan/aggregator_funcs_test.go | 4 +- pkg/sql/distsqlplan/expression.go | 57 ++--- pkg/sql/distsqlplan/physical_plan.go | 16 +- pkg/sql/distsqlrun/aggregator.go | 22 +- pkg/sql/distsqlrun/data.go | 53 ++++ pkg/sql/distsqlrun/data.pb.go | 244 ++++++++----------- pkg/sql/distsqlrun/data.proto | 6 +- pkg/sql/distsqlrun/expr.go | 11 +- pkg/sql/distsqlrun/flow_diagram.go | 24 +- pkg/sql/distsqlrun/joinerbase.go | 2 +- pkg/sql/distsqlrun/joinreader.go | 7 +- pkg/sql/distsqlrun/readerbase.go | 2 +- pkg/sql/sem/tree/eval.go | 5 - 15 files changed, 232 insertions(+), 259 deletions(-) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index f1f8ced0fd73..72def49373aa 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -492,8 +492,6 @@ type PlanningCtx struct { // keep track of whether it's valid to run a root node in a special fast path // mode. planDepth int - - localExprs []tree.TypedExpr } var _ distsqlplan.ExprContext = &PlanningCtx{} @@ -506,34 +504,10 @@ func (p *PlanningCtx) EvalContext() *tree.EvalContext { return &p.ExtendedEvalCtx.EvalContext } -type ivarRemapper struct { - indexVarMap []int -} - -func (v *ivarRemapper) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) { - if ivar, ok := expr.(*tree.IndexedVar); ok { - newIvar := *ivar - newIvar.Idx = v.indexVarMap[ivar.Idx] - return false, &newIvar - } - return true, expr -} - -func (*ivarRemapper) VisitPost(expr tree.Expr) tree.Expr { return expr } - -// MaybeAddLocalExpr implements the ExprContext interface. -func (p *PlanningCtx) MaybeAddLocalExpr(expr tree.TypedExpr, indexVarMap []int) (int, bool) { - if p.isLocal { - if indexVarMap != nil { - // Remap our indexed vars - v := &ivarRemapper{indexVarMap: indexVarMap} - newExpr, _ := tree.WalkExpr(v, expr) - expr = newExpr.(tree.TypedExpr) - } - p.localExprs = append(p.localExprs, expr) - return len(p.localExprs) - 1, true - } - return -1, false +// IsLocal returns true if this PlanningCtx is being used to plan a query that +// has no remote flows. +func (p *PlanningCtx) IsLocal() bool { + return p.isLocal } // sanityCheckAddresses returns an error if the same address is used by two @@ -3303,7 +3277,6 @@ func (dsp *DistSQLPlanner) newLocalPlanningCtx( ctx: ctx, ExtendedEvalCtx: evalCtx, } - evalCtx.LocalExprs = &ret.localExprs return ret } diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 0a8c7bc0c053..7aba65aed0e7 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -742,9 +742,6 @@ func (dsp *DistSQLPlanner) PlanAndRun( recv.SetError(err) return } - if planCtx.isLocal { - //fmt.Println("Made local exprs: ", planCtx.localExprs, *planCtx.ExtendedEvalCtx.LocalExprs) - } dsp.FinalizePlan(planCtx, &physPlan) dsp.Run(planCtx, txn, &physPlan, recv, evalCtx, nil /* finishedSetupFn */) } diff --git a/pkg/sql/distsqlplan/aggregator_funcs_test.go b/pkg/sql/distsqlplan/aggregator_funcs_test.go index edb9cbd57f0c..071f08817c70 100644 --- a/pkg/sql/distsqlplan/aggregator_funcs_test.go +++ b/pkg/sql/distsqlplan/aggregator_funcs_test.go @@ -108,8 +108,8 @@ func (fakeExprContext) EvalContext() *tree.EvalContext { return &tree.EvalContext{} } -func (fakeExprContext) MaybeAddLocalExpr(expr tree.TypedExpr, indexVarMap []int) (int, bool) { - return 0, false +func (fakeExprContext) IsLocal() bool { + return false } // checkDistAggregationInfo tests that a flow with multiple local stages and a diff --git a/pkg/sql/distsqlplan/expression.go b/pkg/sql/distsqlplan/expression.go index a97a29c30bd2..bc62a36b827e 100644 --- a/pkg/sql/distsqlplan/expression.go +++ b/pkg/sql/distsqlplan/expression.go @@ -28,39 +28,31 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" ) -// exprFmtCtxBase produces a FmtCtx used for serializing expressions; a proper -// IndexedVar formatting function needs to be added on. It replaces placeholders -// with their values. -func exprFmtCtxBase(buf *bytes.Buffer, evalCtx *tree.EvalContext) tree.FmtCtx { - fmtCtx := tree.MakeFmtCtx(buf, tree.FmtCheckEquivalence) - fmtCtx.WithPlaceholderFormat( - func(fmtCtx *tree.FmtCtx, p *tree.Placeholder) { - d, err := p.Eval(evalCtx) - if err != nil { - panic(fmt.Sprintf("failed to serialize placeholder: %s", err)) - } - d.Format(fmtCtx) - }) - return fmtCtx -} - -type LocalExprState struct { - expressions []tree.TypedExpr -} - // ExprContext is an interface containing objects necessary for creating // distsqlrun.Expressions. type ExprContext interface { // EvalContext returns the tree.EvalContext for planning. EvalContext() *tree.EvalContext - // MaybeAddLocalExpr adds an expression to the list of local expressions if - // the current plan is local, returning the index of the expression in the - // list if so. Returns false in the 2nd position of the current plan isn't - // local. - MaybeAddLocalExpr(expr tree.TypedExpr, indexVarMap []int) (idx int, ok bool) + // IsLocal returns true if the current plan is local. + IsLocal() bool } +type ivarRemapper struct { + indexVarMap []int +} + +func (v *ivarRemapper) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) { + if ivar, ok := expr.(*tree.IndexedVar); ok { + newIvar := *ivar + newIvar.Idx = v.indexVarMap[ivar.Idx] + return false, &newIvar + } + return true, expr +} + +func (*ivarRemapper) VisitPost(expr tree.Expr) tree.Expr { return expr } + // MakeExpression creates a distsqlrun.Expression. // // The distsqlrun.Expression uses the placeholder syntax (@1, @2, @3..) to refer @@ -76,11 +68,14 @@ func MakeExpression( return distsqlrun.Expression{}, nil } - if idx, ok := ctx.MaybeAddLocalExpr(expr, indexVarMap); ok { - log.VEventf(ctx.EvalContext().Context, 1, "added local expr %v %d", expr, idx) - return distsqlrun.Expression{ - LocalExprIdx: uint32(idx + 1), - }, nil + if ctx.IsLocal() { + if indexVarMap != nil { + // Remap our indexed vars + v := &ivarRemapper{indexVarMap: indexVarMap} + newExpr, _ := tree.WalkExpr(v, expr) + expr = newExpr.(tree.TypedExpr) + } + return distsqlrun.Expression{LocalExpr: expr}, nil } evalCtx := ctx.EvalContext() @@ -94,7 +89,7 @@ func MakeExpression( } // We format the expression using the IndexedVar and Placeholder formatting interceptors. var buf bytes.Buffer - fmtCtx := exprFmtCtxBase(&buf, evalCtx) + fmtCtx := distsqlrun.ExprFmtCtxBase(&buf, evalCtx) if indexVarMap != nil { fmtCtx.WithIndexedVarFormat( func(ctx *tree.FmtCtx, idx int) { diff --git a/pkg/sql/distsqlplan/physical_plan.go b/pkg/sql/distsqlplan/physical_plan.go index 09a8f2e67566..dd5f6c206aeb 100644 --- a/pkg/sql/distsqlplan/physical_plan.go +++ b/pkg/sql/distsqlplan/physical_plan.go @@ -422,10 +422,7 @@ func exprColumn(expr tree.TypedExpr, indexVarMap []int) (int, bool) { // // See MakeExpression for a description of indexVarMap. func (p *PhysicalPlan) AddRendering( - exprs []tree.TypedExpr, - exprCtx ExprContext, - indexVarMap []int, - outTypes []sqlbase.ColumnType, + exprs []tree.TypedExpr, exprCtx ExprContext, indexVarMap []int, outTypes []sqlbase.ColumnType, ) error { // First check if we need an Evaluator, or we are just shuffling values. We // also check if the rendering is a no-op ("identity"). @@ -620,8 +617,15 @@ func (p *PhysicalPlan) AddFilter( if err != nil { return err } - if post.Filter.Expr != "" { - filter.Expr = fmt.Sprintf("(%s) AND (%s)", post.Filter.Expr, filter.Expr) + if !post.Filter.Empty() { + if filter.Expr != "" { + filter.Expr = fmt.Sprintf("(%s) AND (%s)", post.Filter.Expr, filter.Expr) + } else if filter.LocalExpr != nil { + filter.LocalExpr = tree.NewTypedAndExpr( + post.Filter.LocalExpr, + filter.LocalExpr, + ) + } } for _, pIdx := range p.ResultRouters { p.Processors[pIdx].Spec.Post.Filter = filter diff --git a/pkg/sql/distsqlrun/aggregator.go b/pkg/sql/distsqlrun/aggregator.go index 1a85db746352..9caf87272a3d 100644 --- a/pkg/sql/distsqlrun/aggregator.go +++ b/pkg/sql/distsqlrun/aggregator.go @@ -21,7 +21,6 @@ import ( "fmt" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" @@ -208,22 +207,17 @@ func (ag *aggregatorBase) init( arguments := make(tree.Datums, len(aggInfo.Arguments)) for j, argument := range aggInfo.Arguments { - expr, err := parser.ParseExpr(argument.Expr) - if err != nil { - return err - } - typedExpr, err := tree.TypeCheck(expr, &tree.SemaContext{}, types.Any) - if err != nil { - return errors.Wrap(err, expr.String()) - } - argTypes[len(aggInfo.ColIdx)+j], err = sqlbase.DatumTypeToColumnType(typedExpr.ResolvedType()) - if err != nil { - return errors.Wrap(err, expr.String()) + h := exprHelper{} + // Pass nil types and row - there are no variables in these expressions. + if err := h.init(argument, nil /* types */, flowCtx.EvalCtx); err != nil { + return errors.Wrap(err, argument.String()) } - arguments[j], err = typedExpr.Eval(ag.evalCtx) + d, err := h.eval(nil /* row */) if err != nil { - return errors.Wrap(err, expr.String()) + return errors.Wrap(err, argument.String()) } + argTypes[len(aggInfo.ColIdx)+j], err = sqlbase.DatumTypeToColumnType(d.ResolvedType()) + arguments[j] = d } aggConstructor, retType, err := GetAggregateInfo(aggInfo.Func, argTypes...) diff --git a/pkg/sql/distsqlrun/data.go b/pkg/sql/distsqlrun/data.go index 359168bee3c6..11ef962f8296 100644 --- a/pkg/sql/distsqlrun/data.go +++ b/pkg/sql/distsqlrun/data.go @@ -17,6 +17,9 @@ package distsqlrun import ( "fmt" + "bytes" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/encoding" ) @@ -67,3 +70,53 @@ func ConvertToMappedSpecOrdering( } return specOrdering } + +// ExprFmtCtxBase produces a FmtCtx used for serializing expressions; a proper +// IndexedVar formatting function needs to be added on. It replaces placeholders +// with their values. +func ExprFmtCtxBase(buf *bytes.Buffer, evalCtx *tree.EvalContext) tree.FmtCtx { + fmtCtx := tree.MakeFmtCtx(buf, tree.FmtCheckEquivalence) + fmtCtx.WithPlaceholderFormat( + func(fmtCtx *tree.FmtCtx, p *tree.Placeholder) { + d, err := p.Eval(evalCtx) + if err != nil { + panic(fmt.Sprintf("failed to serialize placeholder: %s", err)) + } + d.Format(fmtCtx) + }) + return fmtCtx +} + +// Expression is the representation of a SQL expression. +// See data.proto for the corresponding proto definition. +type Expression struct { + // Version is unused. + Version string + + // Expr, if present, is the string representation of this expression. + // SQL expressions are passed as a string, with ordinal references + // (@1, @2, @3 ..) used for "input" variables. + Expr string + + // LocalExpr is an unserialized field that's used to pass expressions to local + // flows without serializing/deserializing them. + LocalExpr tree.TypedExpr +} + +// Empty returns true if the expression has neither an Expr nor LocalExpr. +func (e *Expression) Empty() bool { + return e.Expr == "" && e.LocalExpr == nil +} + +// String implements the Stringer interface. +func (e Expression) String() string { + if e.LocalExpr != nil { + buf := bytes.Buffer{} + ctx := ExprFmtCtxBase(&buf, &tree.EvalContext{}) + ctx.FormatNode(e.LocalExpr) + return ctx.String() + } else if e.Expr != "" { + return e.Expr + } + return "none" +} diff --git a/pkg/sql/distsqlrun/data.pb.go b/pkg/sql/distsqlrun/data.pb.go index 3d4a283236eb..bf506861ea5d 100644 --- a/pkg/sql/distsqlrun/data.pb.go +++ b/pkg/sql/distsqlrun/data.pb.go @@ -312,17 +312,7 @@ func _Error_OneofSizer(msg proto.Message) (n int) { return n } -type Expression struct { - // TODO(radu): TBD how this will be used - Version string `protobuf:"bytes,1,opt,name=version" json:"version"` - // SQL expressions are passed as a string, with ordinal references - // (@1, @2, @3 ..) used for "input" variables. - Expr string `protobuf:"bytes,2,opt,name=expr" json:"expr"` - LocalExprIdx uint32 `protobuf:"varint,3,opt,name=local_expr_idx,json=localExprIdx" json:"local_expr_idx"` -} - func (m *Expression) Reset() { *m = Expression{} } -func (m *Expression) String() string { return proto.CompactTextString(m) } func (*Expression) ProtoMessage() {} func (*Expression) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{1} } @@ -969,9 +959,6 @@ func (m *Expression) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintData(dAtA, i, uint64(len(m.Expr))) i += copy(dAtA[i:], m.Expr) - dAtA[i] = 0x18 - i++ - i = encodeVarintData(dAtA, i, uint64(m.LocalExprIdx)) return i, nil } @@ -1699,7 +1686,6 @@ func (m *Expression) Size() (n int) { n += 1 + l + sovData(uint64(l)) l = len(m.Expr) n += 1 + l + sovData(uint64(l)) - n += 1 + sovData(uint64(m.LocalExprIdx)) return n } @@ -2190,25 +2176,6 @@ func (m *Expression) Unmarshal(dAtA []byte) error { } m.Expr = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LocalExprIdx", wireType) - } - m.LocalExprIdx = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowData - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.LocalExprIdx |= (uint32(b) & 0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skipData(dAtA[iNdEx:]) @@ -4488,111 +4455,110 @@ var ( func init() { proto.RegisterFile("sql/distsqlrun/data.proto", fileDescriptorData) } var fileDescriptorData = []byte{ - // 1694 bytes of a gzipped FileDescriptorProto + // 1679 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x4f, 0x6f, 0x23, 0xb7, - 0xf9, 0xd6, 0xc8, 0xfa, 0xfb, 0x4a, 0xd6, 0xca, 0xfc, 0x2d, 0x16, 0xfa, 0xa9, 0x5b, 0xcb, 0x99, - 0x06, 0x85, 0x1b, 0xa4, 0xd2, 0xae, 0x73, 0x58, 0xd4, 0x39, 0x24, 0x92, 0xa5, 0xb5, 0xd4, 0x78, - 0xad, 0xed, 0xc8, 0xdb, 0x62, 0x93, 0x02, 0xd3, 0xf1, 0x90, 0x1e, 0x0f, 0x32, 0x9a, 0x19, 0x93, - 0x9c, 0x95, 0x7d, 0xe9, 0xa9, 0x1f, 0x20, 0xa7, 0xa2, 0xc7, 0x9c, 0x7a, 0xe9, 0x07, 0xe8, 0x37, - 0x28, 0xf6, 0xd8, 0x63, 0xd0, 0x02, 0x46, 0xeb, 0x5e, 0xfa, 0x15, 0x1a, 0xa0, 0x40, 0x41, 0x0e, - 0xa9, 0x3f, 0xf6, 0x3a, 0x89, 0xb3, 0x97, 0x01, 0xf9, 0xf2, 0x7d, 0x1f, 0x3e, 0x7c, 0xf8, 0xf2, - 0xe5, 0x10, 0xfe, 0x9f, 0x9d, 0x05, 0x1d, 0xec, 0x33, 0xce, 0xce, 0x02, 0x9a, 0x84, 0x1d, 0xec, - 0x70, 0xa7, 0x1d, 0xd3, 0x88, 0x47, 0xa8, 0xe1, 0x46, 0xee, 0xe7, 0x34, 0x72, 0xdc, 0xd3, 0x36, - 0x3b, 0x0b, 0xda, 0x0b, 0xa7, 0xe6, 0x86, 0xb4, 0xc6, 0xc7, 0x1d, 0x27, 0xf6, 0x53, 0xe7, 0x26, - 0xd2, 0xa6, 0x05, 0x40, 0xf3, 0xbe, 0xb6, 0x11, 0x4a, 0x23, 0xca, 0x94, 0xb5, 0x25, 0x66, 0x8c, - 0xbd, 0x99, 0x4f, 0x49, 0x27, 0xf6, 0xe4, 0xd0, 0xaa, 0xc3, 0x43, 0xe1, 0xc0, 0xce, 0x82, 0x63, - 0x87, 0x91, 0x0e, 0xe3, 0x34, 0x71, 0x79, 0x42, 0x09, 0x5e, 0x0e, 0xd7, 0xa3, 0x24, 0x74, 0x23, - 0x4c, 0xb0, 0x8d, 0x1d, 0x9e, 0x4c, 0x95, 0xc3, 0x56, 0xc2, 0xfd, 0xa0, 0xc3, 0xa9, 0xe3, 0xfa, - 0xa1, 0xd7, 0xa1, 0xc4, 0x8d, 0xa8, 0x70, 0x61, 0xb1, 0x13, 0x6a, 0x5e, 0x5e, 0xe4, 0x45, 0xb2, - 0xd9, 0x11, 0xad, 0xd4, 0x6a, 0xfe, 0xd9, 0x80, 0xfc, 0x40, 0xf0, 0x40, 0x3d, 0x28, 0xc5, 0x9e, - 0x2d, 0x39, 0x35, 0x8c, 0x2d, 0x63, 0xbb, 0xb2, 0xd3, 0x68, 0x2f, 0xb4, 0x50, 0x9c, 0xdb, 0xd2, - 0xb7, 0x57, 0xb9, 0xba, 0x6c, 0x15, 0x9f, 0xef, 0xcb, 0xce, 0x30, 0x63, 0x15, 0x63, 0x2f, 0xc5, - 0xf8, 0x14, 0x36, 0x28, 0xe1, 0xf4, 0xc2, 0x39, 0x0e, 0xc8, 0xd1, 0x79, 0x28, 0x8d, 0x8d, 0xac, - 0x04, 0x7b, 0x6f, 0x09, 0x4c, 0x29, 0xd4, 0x7e, 0x11, 0x9e, 0x3a, 0x21, 0x0e, 0x08, 0xb6, 0x74, - 0x90, 0x46, 0xbc, 0x09, 0xb3, 0x9b, 0xfb, 0xc3, 0x97, 0xad, 0x4c, 0xaf, 0x04, 0x05, 0x4c, 0xb8, - 0xe3, 0x07, 0x26, 0x05, 0x18, 0x9c, 0xc7, 0x94, 0x30, 0xe6, 0x47, 0x21, 0xda, 0x84, 0xe2, 0x2b, - 0x42, 0x45, 0x53, 0x92, 0x2f, 0xf7, 0x72, 0xaf, 0x2f, 0x5b, 0x19, 0x4b, 0x1b, 0x51, 0x03, 0x72, - 0xe4, 0x3c, 0x4e, 0xc9, 0xe8, 0x41, 0x69, 0x41, 0xef, 0x41, 0x2d, 0x88, 0x5c, 0x27, 0xb0, 0x45, - 0xcf, 0xf6, 0xf1, 0x79, 0x63, 0x6d, 0xcb, 0xd8, 0x5e, 0x57, 0x3e, 0x55, 0x39, 0x26, 0x26, 0x1a, - 0xe1, 0x73, 0xf3, 0x77, 0x59, 0x28, 0x8d, 0x29, 0x26, 0xd4, 0x0f, 0x3d, 0x34, 0x82, 0xa2, 0x1b, - 0x05, 0xc9, 0x34, 0x64, 0x0d, 0x63, 0x6b, 0x6d, 0xbb, 0xb2, 0xf3, 0x93, 0xf6, 0x6d, 0xb9, 0xd3, - 0xd6, 0x41, 0xed, 0x3d, 0x19, 0xa1, 0xd9, 0xa9, 0xf8, 0xe6, 0x1f, 0x0d, 0x28, 0xa4, 0x23, 0xe8, - 0x87, 0x12, 0x55, 0xf2, 0x30, 0x96, 0x78, 0x14, 0xdc, 0x28, 0x18, 0xe1, 0x73, 0xf4, 0x2b, 0x28, - 0x63, 0x9f, 0x12, 0x97, 0x8b, 0x95, 0x8a, 0xc5, 0xd4, 0x76, 0x3e, 0xf8, 0xce, 0xd3, 0xb6, 0xfb, - 0x3a, 0x54, 0xa1, 0x2e, 0xb0, 0xcc, 0x4d, 0x28, 0xcf, 0x47, 0x51, 0x11, 0xd6, 0xba, 0x93, 0xbd, - 0x7a, 0x06, 0x95, 0x20, 0xd7, 0x1f, 0x4c, 0xf6, 0xea, 0xc6, 0x6e, 0xee, 0xdf, 0x5f, 0xb6, 0xd4, - 0xd7, 0xfc, 0x6f, 0x16, 0xd0, 0x84, 0x53, 0xe2, 0x4c, 0x07, 0x21, 0x8e, 0x23, 0x3f, 0xe4, 0x93, - 0x98, 0xb8, 0xe8, 0x13, 0xc8, 0xf1, 0x8b, 0x98, 0x48, 0xde, 0xb5, 0x9d, 0xc7, 0xb7, 0xd3, 0xba, - 0x19, 0xdb, 0x3e, 0xba, 0x88, 0x89, 0xde, 0x16, 0x01, 0x82, 0x7e, 0x06, 0x65, 0x26, 0xdd, 0x6c, - 0x1f, 0xcb, 0x85, 0xe6, 0x7b, 0x0f, 0xc5, 0xf0, 0xd5, 0x65, 0xab, 0x94, 0xc6, 0x8f, 0xfa, 0x5f, - 0x2f, 0xb5, 0xad, 0x52, 0xea, 0x3e, 0xc2, 0x68, 0x17, 0x1e, 0x60, 0x12, 0x53, 0xe2, 0x3a, 0x9c, - 0x60, 0x9b, 0x3b, 0xd4, 0x23, 0xdc, 0x76, 0x30, 0xa6, 0x72, 0x67, 0xf5, 0xee, 0xdf, 0x5f, 0xf8, - 0x1c, 0x49, 0x97, 0x2e, 0xc6, 0x14, 0x9d, 0x41, 0x4d, 0x05, 0x84, 0x11, 0x26, 0x62, 0xee, 0x9c, - 0x9c, 0xfb, 0x13, 0x35, 0x77, 0x35, 0xf5, 0x3d, 0x8c, 0x30, 0x91, 0xf3, 0x7f, 0xe0, 0xf9, 0xfc, - 0x34, 0x39, 0x6e, 0xbb, 0xd1, 0xb4, 0x33, 0x5f, 0x2f, 0x3e, 0x5e, 0xb4, 0x3b, 0xf1, 0xe7, 0x5e, - 0x47, 0x27, 0x7c, 0x1a, 0x66, 0x55, 0xf9, 0x02, 0x04, 0x9b, 0x8f, 0x20, 0x27, 0x56, 0x8f, 0xca, - 0x90, 0x3f, 0x18, 0xef, 0x75, 0x0f, 0xea, 0x19, 0x04, 0x50, 0xb0, 0x06, 0xcf, 0xc6, 0x47, 0x83, - 0xba, 0x81, 0x36, 0x60, 0x7d, 0xf2, 0xf2, 0x70, 0xcf, 0xb6, 0x06, 0x93, 0xe7, 0xe3, 0xc3, 0xc9, - 0xa0, 0x9e, 0x35, 0xff, 0x9e, 0x85, 0xf5, 0x51, 0x18, 0x27, 0x7c, 0x72, 0x11, 0xba, 0x52, 0xfa, - 0xa7, 0x2b, 0xd2, 0xbf, 0x7f, 0xbb, 0xf4, 0x2b, 0x61, 0x37, 0x55, 0xef, 0x43, 0x29, 0x52, 0x39, - 0xa3, 0xce, 0xad, 0xf9, 0xed, 0xd9, 0xa5, 0x10, 0xe6, 0x91, 0xe8, 0x00, 0x8a, 0xe9, 0x66, 0xb0, - 0xc6, 0x9a, 0x3c, 0x19, 0xef, 0xdf, 0x25, 0x17, 0xf4, 0xe1, 0x50, 0x10, 0xe8, 0xe7, 0x50, 0x4d, - 0xcf, 0x89, 0x2d, 0x28, 0xb2, 0x46, 0x4e, 0x42, 0xbe, 0x73, 0x0d, 0x52, 0x15, 0x47, 0x95, 0xe9, - 0x4b, 0x0b, 0xab, 0xb8, 0x73, 0x0b, 0x33, 0x4d, 0xa5, 0xf5, 0x3a, 0x94, 0x5f, 0x1c, 0x8e, 0xad, - 0xfe, 0xc0, 0x1a, 0xf4, 0xeb, 0x19, 0x54, 0x81, 0xa2, 0xee, 0x18, 0xe6, 0x7f, 0x0a, 0x50, 0x1f, - 0x27, 0x3c, 0x4e, 0xb8, 0x15, 0x25, 0x9c, 0x50, 0x29, 0xf0, 0x68, 0x45, 0xe0, 0xce, 0x37, 0x88, - 0x72, 0x2d, 0xf2, 0xa6, 0xc6, 0x4b, 0xea, 0x64, 0xdf, 0x5e, 0x9d, 0x77, 0xa0, 0x7a, 0xea, 0xb0, - 0x53, 0x5b, 0x97, 0x22, 0x21, 0xf8, 0xba, 0x55, 0x11, 0xb6, 0x54, 0x0a, 0x86, 0x02, 0xd8, 0xa0, - 0x4e, 0xe8, 0x11, 0x9b, 0x4a, 0x56, 0x36, 0x8b, 0x89, 0x2b, 0xd3, 0xba, 0xb2, 0xb3, 0x7b, 0x87, - 0x85, 0x58, 0x02, 0x63, 0xd1, 0x57, 0x44, 0xee, 0xd1, 0x55, 0x33, 0x7a, 0x0c, 0x1b, 0xd8, 0x67, - 0xa2, 0x74, 0xdb, 0xc7, 0xc9, 0xc9, 0x49, 0x9a, 0x4b, 0xf9, 0x2d, 0x63, 0xbb, 0xa4, 0x22, 0xea, - 0x6a, 0xb8, 0xa7, 0x47, 0x9b, 0x7f, 0x59, 0x83, 0x7b, 0xd7, 0xd0, 0xd1, 0x67, 0x90, 0x17, 0x97, - 0x97, 0xae, 0xad, 0x1f, 0x7d, 0x7f, 0xa2, 0xed, 0x49, 0xec, 0xe8, 0x82, 0x97, 0x62, 0x0a, 0xd1, - 0x30, 0x39, 0x71, 0x92, 0x80, 0xdb, 0x98, 0x30, 0x9e, 0xd6, 0x17, 0xab, 0xa2, 0x6c, 0x7d, 0xc2, - 0x38, 0x9a, 0x42, 0x59, 0xde, 0xb3, 0x7e, 0xe8, 0xe9, 0x2c, 0x1e, 0xbd, 0x05, 0x87, 0x74, 0x2f, - 0x06, 0x0a, 0x51, 0x97, 0xdf, 0xf9, 0x0c, 0xcd, 0x57, 0x50, 0x5b, 0x75, 0x41, 0x0f, 0xa1, 0x90, - 0xee, 0xe9, 0x8d, 0x7b, 0x40, 0x5c, 0x13, 0x4f, 0xa1, 0xa4, 0x83, 0xd5, 0x35, 0xf0, 0xee, 0x2d, - 0x07, 0xa2, 0x2f, 0xfe, 0x12, 0xae, 0x4d, 0x3c, 0x8f, 0x6d, 0x1e, 0x40, 0x4e, 0xc8, 0x83, 0xee, - 0x43, 0x9e, 0x71, 0x87, 0x72, 0x39, 0x59, 0xd5, 0x4a, 0x3b, 0xa8, 0x0e, 0x6b, 0x24, 0x4c, 0xcb, - 0x6f, 0xd5, 0x12, 0x4d, 0xc1, 0x2a, 0xcd, 0x3c, 0x59, 0x4b, 0xf3, 0x9a, 0x55, 0x6a, 0x33, 0x3f, - 0x52, 0xc7, 0xab, 0x0e, 0xd5, 0xe7, 0xdd, 0xc9, 0xc4, 0x3e, 0x1a, 0x5a, 0xe3, 0x17, 0xfb, 0xc3, - 0xb4, 0xa2, 0x3d, 0x1b, 0x59, 0xd6, 0xd8, 0xaa, 0x1b, 0xe2, 0xb4, 0xf5, 0x5e, 0xda, 0xc3, 0xee, - 0x64, 0x58, 0xcf, 0xa2, 0x2a, 0x94, 0x7a, 0x2f, 0x6d, 0xab, 0x7b, 0xb8, 0x3f, 0xa8, 0xaf, 0x99, - 0x5f, 0x18, 0x50, 0x96, 0x84, 0x47, 0xe1, 0x49, 0xb4, 0xb2, 0x48, 0xe3, 0xfb, 0x2f, 0x12, 0x7d, - 0xa8, 0x0e, 0x6f, 0x5a, 0xd1, 0xbe, 0x73, 0xe5, 0x90, 0x41, 0xe6, 0x6f, 0xa1, 0xf6, 0x9c, 0x46, - 0x38, 0x71, 0x09, 0x1d, 0x12, 0x07, 0x13, 0x8a, 0x1e, 0x43, 0xf1, 0x24, 0x88, 0x66, 0xe2, 0x72, - 0x90, 0x6a, 0xf5, 0x1a, 0xc2, 0xfd, 0x6f, 0x97, 0xad, 0xc2, 0xd3, 0x20, 0x9a, 0x8d, 0xfa, 0x57, - 0xf3, 0x96, 0x55, 0x10, 0x8e, 0x23, 0xfc, 0x16, 0xb7, 0x99, 0xf9, 0x27, 0x03, 0xaa, 0x9a, 0x40, - 0xdf, 0xe1, 0x0e, 0xfa, 0x01, 0x94, 0xa9, 0x33, 0xb3, 0x8f, 0x2f, 0x38, 0x61, 0x6a, 0xbb, 0x4a, - 0xd4, 0x99, 0xf5, 0x44, 0x1f, 0x59, 0x50, 0x9a, 0x12, 0xee, 0x88, 0xff, 0x51, 0x55, 0x5d, 0x1e, - 0xdd, 0x9e, 0xb5, 0x16, 0x99, 0x46, 0x9c, 0x68, 0xf0, 0x67, 0x2a, 0x4e, 0xcb, 0xa7, 0x71, 0xc4, - 0x1f, 0x52, 0x98, 0x4c, 0x6d, 0x32, 0x8d, 0xf9, 0x85, 0x4d, 0xa3, 0x19, 0x5b, 0xd9, 0xfb, 0x6a, - 0x98, 0x4c, 0x07, 0x62, 0xc8, 0x8a, 0x66, 0xcc, 0xfc, 0xca, 0x80, 0x7b, 0x0b, 0x40, 0xc6, 0x1c, - 0x8f, 0xa0, 0x8f, 0xa1, 0x70, 0x2a, 0x95, 0x53, 0xff, 0x95, 0xdb, 0xb7, 0x33, 0x5a, 0x55, 0xda, - 0x52, 0x71, 0xa8, 0x0b, 0x05, 0x7e, 0x11, 0xa7, 0xb9, 0x2e, 0xd6, 0xf4, 0xa3, 0xdb, 0x11, 0xe6, - 0xd9, 0xa3, 0x53, 0x33, 0x0d, 0x44, 0x1f, 0x43, 0x4e, 0x8a, 0xb2, 0x26, 0x29, 0xfc, 0xf8, 0xdb, - 0x29, 0xf4, 0x17, 0x52, 0xc8, 0x48, 0xf3, 0x32, 0x0f, 0x0f, 0xde, 0xac, 0x18, 0xfa, 0x35, 0x40, - 0x5a, 0x61, 0xfd, 0xf0, 0x24, 0x52, 0xab, 0xfc, 0xf0, 0xae, 0xba, 0xa7, 0x35, 0x43, 0x50, 0x67, - 0xc3, 0x8c, 0x55, 0xa6, 0xba, 0x87, 0x9e, 0x40, 0x9e, 0x2c, 0xfd, 0x49, 0xb7, 0x6e, 0x07, 0xd6, - 0xbf, 0xcf, 0xa9, 0x3f, 0xfa, 0x0c, 0x40, 0xbc, 0x08, 0x88, 0xbd, 0xb4, 0xf2, 0xdd, 0x3b, 0xd3, - 0x3a, 0x12, 0x10, 0x42, 0x0d, 0xc1, 0x8a, 0xeb, 0x0e, 0xda, 0x87, 0x1a, 0x3f, 0x0f, 0x6d, 0x37, - 0x8a, 0x28, 0xb6, 0x45, 0xae, 0xa8, 0x2b, 0xa5, 0xf5, 0x86, 0x1f, 0xfd, 0xa3, 0xf3, 0x70, 0x4f, - 0xf8, 0x09, 0xcc, 0x61, 0xc6, 0xaa, 0xf2, 0xa5, 0x3e, 0xb2, 0xa0, 0x48, 0xa3, 0x99, 0x1d, 0x26, - 0x53, 0x79, 0x4d, 0x54, 0x76, 0x9e, 0xdc, 0x5d, 0xb9, 0x68, 0x76, 0x98, 0x4c, 0x87, 0x19, 0xab, - 0x40, 0x65, 0xab, 0x39, 0x06, 0x58, 0xa8, 0x89, 0xba, 0xd7, 0xb6, 0x47, 0xa4, 0xd0, 0xc3, 0x37, - 0xd0, 0x9c, 0x87, 0xe8, 0xfa, 0x3c, 0xdf, 0x83, 0xe6, 0x6f, 0xa0, 0x3c, 0xd7, 0x01, 0x4d, 0xe0, - 0x9e, 0x1b, 0x05, 0x01, 0x71, 0xb9, 0x7a, 0x62, 0xe9, 0x5b, 0x6a, 0xb9, 0x3c, 0x89, 0x07, 0x59, - 0x5b, 0x3d, 0xc8, 0xda, 0x96, 0x7a, 0x90, 0x2d, 0x5d, 0x45, 0xb5, 0x39, 0x84, 0x30, 0xb2, 0xe6, - 0x0c, 0x0a, 0xe9, 0x32, 0xd0, 0x4f, 0xa1, 0xcc, 0x48, 0x88, 0x09, 0xd5, 0x15, 0xa6, 0xdc, 0xab, - 0xcf, 0x8b, 0x85, 0x1c, 0x90, 0x05, 0x22, 0x6d, 0x61, 0xf1, 0x62, 0xd0, 0xfa, 0x65, 0x97, 0x6b, - 0x72, 0x2a, 0x05, 0x6a, 0x41, 0x29, 0x70, 0x18, 0xb7, 0xa7, 0xcc, 0x93, 0x29, 0xa0, 0xaf, 0xe1, - 0xa2, 0xb0, 0x3e, 0x63, 0x5e, 0xaf, 0x08, 0xf9, 0x57, 0x4e, 0x90, 0x10, 0xf3, 0xf7, 0x06, 0x34, - 0xfa, 0x3e, 0xe3, 0x93, 0x5f, 0x1c, 0xfc, 0x32, 0x7d, 0x36, 0xed, 0x47, 0x8c, 0xf9, 0xb1, 0x4c, - 0xc2, 0x47, 0xab, 0x0f, 0xac, 0xf5, 0xde, 0x03, 0x81, 0xf2, 0xf5, 0x65, 0xab, 0xb6, 0x1a, 0xb2, - 0x78, 0x72, 0x0d, 0xe1, 0xfe, 0xd4, 0x0f, 0x6d, 0xc7, 0x75, 0x49, 0x2c, 0x84, 0xd2, 0xe1, 0xd9, - 0x6f, 0x0c, 0x47, 0x53, 0x3f, 0xec, 0xaa, 0x10, 0x65, 0x33, 0x9f, 0xc0, 0xff, 0x29, 0xaf, 0x3e, - 0x75, 0xfc, 0xd0, 0x0f, 0x3d, 0x49, 0x69, 0x0b, 0x4a, 0x58, 0xf5, 0x25, 0x27, 0xbd, 0xb2, 0xb9, - 0xb5, 0xf7, 0xee, 0xeb, 0x7f, 0x6e, 0x66, 0x5e, 0x5f, 0x6d, 0x1a, 0x7f, 0xbd, 0xda, 0x34, 0xbe, - 0xba, 0xda, 0x34, 0xfe, 0x71, 0xb5, 0x69, 0x7c, 0xf1, 0xaf, 0xcd, 0xcc, 0xa7, 0xb0, 0x48, 0xaa, - 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x5e, 0x6f, 0xe2, 0xc0, 0x0e, 0x10, 0x00, 0x00, + 0x15, 0xd7, 0xc8, 0xfa, 0xfb, 0x24, 0x6b, 0xb5, 0xec, 0x62, 0xa1, 0xaa, 0x5b, 0xcb, 0x99, 0x06, + 0xc5, 0x36, 0x48, 0xa5, 0x5d, 0xe7, 0xb0, 0xa8, 0x73, 0x48, 0x24, 0x4b, 0x6b, 0xa9, 0xf1, 0x5a, + 0xee, 0xc8, 0xdb, 0x62, 0x93, 0x02, 0xd3, 0xf1, 0x90, 0x1e, 0x0f, 0x32, 0x9a, 0x19, 0x93, 0x9c, + 0x95, 0x7d, 0xe9, 0xa9, 0x1f, 0x20, 0xa7, 0xa2, 0xc7, 0xf4, 0xd2, 0x4b, 0x3f, 0x40, 0xbf, 0x41, + 0xb1, 0xc7, 0x1e, 0x83, 0x16, 0x30, 0x5a, 0xf7, 0xd2, 0x63, 0xaf, 0x0d, 0x50, 0x20, 0x20, 0x87, + 0xd4, 0x1f, 0x7b, 0x9d, 0xc4, 0xd9, 0xcb, 0x80, 0x7c, 0x7c, 0xef, 0xc7, 0x1f, 0x7f, 0x7c, 0x7c, + 0x1c, 0xc2, 0xf7, 0xd9, 0x69, 0xd0, 0xc1, 0x3e, 0xe3, 0xec, 0x34, 0xa0, 0x49, 0xd8, 0xc1, 0x0e, + 0x77, 0xda, 0x31, 0x8d, 0x78, 0x84, 0x1a, 0x6e, 0xe4, 0x7e, 0x4a, 0x23, 0xc7, 0x3d, 0x69, 0xb3, + 0xd3, 0xa0, 0xbd, 0x70, 0x6a, 0xde, 0x95, 0xd6, 0xf8, 0xa8, 0xe3, 0xc4, 0x7e, 0xea, 0xdc, 0x44, + 0xda, 0xb4, 0x00, 0x68, 0xde, 0xd3, 0x36, 0x42, 0x69, 0x44, 0x99, 0xb2, 0xb6, 0xc4, 0x8c, 0xb1, + 0x37, 0xf3, 0x29, 0xe9, 0xc4, 0x9e, 0x1c, 0x5a, 0x75, 0x78, 0x20, 0x1c, 0xd8, 0x69, 0x70, 0xe4, + 0x30, 0xd2, 0x61, 0x9c, 0x26, 0x2e, 0x4f, 0x28, 0xc1, 0xcb, 0xe1, 0x7a, 0x94, 0x84, 0x6e, 0x84, + 0x09, 0xb6, 0xb1, 0xc3, 0x93, 0xa9, 0x72, 0xd8, 0x4c, 0xb8, 0x1f, 0x74, 0x38, 0x75, 0x5c, 0x3f, + 0xf4, 0x3a, 0x94, 0xb8, 0x11, 0x15, 0x2e, 0x2c, 0x76, 0x42, 0xcd, 0xcb, 0x8b, 0xbc, 0x48, 0x36, + 0x3b, 0xa2, 0x95, 0x5a, 0xcd, 0xbf, 0x18, 0x90, 0x1f, 0x08, 0x1e, 0xa8, 0x07, 0xa5, 0xd8, 0xb3, + 0x25, 0xa7, 0x86, 0xb1, 0x69, 0x3c, 0xac, 0x6c, 0x35, 0xda, 0x0b, 0x2d, 0x14, 0xe7, 0xb6, 0xf4, + 0xed, 0x55, 0x2e, 0x2f, 0x5a, 0xc5, 0x83, 0x5d, 0xd9, 0x19, 0x66, 0xac, 0x62, 0xec, 0xa5, 0x18, + 0x1f, 0xc3, 0x5d, 0x4a, 0x38, 0x3d, 0x77, 0x8e, 0x02, 0x72, 0x78, 0x16, 0x4a, 0x63, 0x23, 0x2b, + 0xc1, 0xde, 0x59, 0x02, 0x53, 0x0a, 0xb5, 0x9f, 0x87, 0x27, 0x4e, 0x88, 0x03, 0x82, 0x2d, 0x1d, + 0xa4, 0x11, 0xaf, 0xc3, 0x6c, 0xe7, 0xfe, 0xf0, 0x79, 0x2b, 0xd3, 0x2b, 0x41, 0x01, 0x13, 0xee, + 0xf8, 0x81, 0x79, 0x00, 0x30, 0x38, 0x8b, 0x29, 0x61, 0xcc, 0x8f, 0x42, 0xb4, 0x01, 0xc5, 0x97, + 0x84, 0x8a, 0xa6, 0x24, 0x5f, 0xee, 0xe5, 0x5e, 0x5d, 0xb4, 0x32, 0x96, 0x36, 0xa2, 0x06, 0xe4, + 0xc8, 0x59, 0x9c, 0x92, 0xd1, 0x83, 0xd2, 0xb2, 0x5d, 0x12, 0xb8, 0xff, 0xfd, 0x63, 0x2b, 0x63, + 0xfe, 0x2e, 0x0b, 0xa5, 0x31, 0xc5, 0x84, 0xfa, 0xa1, 0x87, 0x46, 0x50, 0x74, 0xa3, 0x20, 0x99, + 0x86, 0xac, 0x61, 0x6c, 0xae, 0x3d, 0xac, 0x6c, 0xfd, 0xa4, 0x7d, 0x53, 0x66, 0xb4, 0x75, 0x50, + 0x7b, 0x47, 0x46, 0xe8, 0xb9, 0x55, 0x7c, 0xf3, 0x4f, 0x06, 0x14, 0xd2, 0x11, 0xf4, 0x43, 0x89, + 0x6a, 0xfb, 0xf8, 0x4c, 0xd2, 0x5c, 0x57, 0xae, 0x05, 0x37, 0x0a, 0x46, 0xf8, 0x0c, 0xfd, 0x0a, + 0xca, 0xd8, 0xa7, 0xc4, 0xe5, 0x62, 0x1d, 0x82, 0x6a, 0x6d, 0xeb, 0xbd, 0x6f, 0x3d, 0x6d, 0xbb, + 0xaf, 0x43, 0x15, 0xea, 0x02, 0xcb, 0xdc, 0x80, 0xf2, 0x7c, 0x14, 0x15, 0x61, 0xad, 0x3b, 0xd9, + 0xa9, 0x67, 0x50, 0x09, 0x72, 0xfd, 0xc1, 0x64, 0xa7, 0x6e, 0x6c, 0xe7, 0xfe, 0xf3, 0x79, 0x4b, + 0x7d, 0xcd, 0xff, 0x67, 0x01, 0x4d, 0x38, 0x25, 0xce, 0x74, 0x10, 0xe2, 0x38, 0xf2, 0x43, 0x3e, + 0x89, 0x89, 0x8b, 0x3e, 0x82, 0x1c, 0x3f, 0x8f, 0x89, 0xe4, 0x5d, 0xdb, 0x7a, 0x7c, 0x33, 0xad, + 0xeb, 0xb1, 0xed, 0xc3, 0xf3, 0x98, 0x68, 0xd1, 0x05, 0x08, 0xfa, 0x19, 0x94, 0x99, 0x74, 0xb3, + 0x7d, 0x2c, 0x17, 0x9a, 0xef, 0x3d, 0x10, 0xc3, 0x97, 0x17, 0xad, 0x52, 0x1a, 0x3f, 0xea, 0x7f, + 0xb9, 0xd4, 0xb6, 0x4a, 0xa9, 0xfb, 0x08, 0xa3, 0x6d, 0xb8, 0x8f, 0x49, 0x4c, 0x89, 0xeb, 0x70, + 0x82, 0x6d, 0xee, 0x50, 0x8f, 0x70, 0xdb, 0xc1, 0x98, 0x36, 0xd6, 0x96, 0xf6, 0xf6, 0xde, 0xc2, + 0xe7, 0x50, 0xba, 0x74, 0x31, 0xa6, 0xe8, 0x14, 0x6a, 0x2a, 0x20, 0x8c, 0x30, 0x11, 0x73, 0xe7, + 0xe4, 0xdc, 0x1f, 0xa9, 0xb9, 0xab, 0xa9, 0xef, 0x7e, 0x84, 0x89, 0x9c, 0xff, 0x3d, 0xcf, 0xe7, + 0x27, 0xc9, 0x51, 0xdb, 0x8d, 0xa6, 0x9d, 0xf9, 0x7a, 0xf1, 0xd1, 0xa2, 0xdd, 0x89, 0x3f, 0xf5, + 0x3a, 0x3a, 0x9d, 0xd3, 0x30, 0xab, 0xca, 0x17, 0x20, 0xd8, 0x7c, 0x04, 0x39, 0xb1, 0x7a, 0x54, + 0x86, 0xfc, 0xde, 0x78, 0xa7, 0xbb, 0x57, 0xcf, 0x20, 0x80, 0x82, 0x35, 0x78, 0x36, 0x3e, 0x1c, + 0xd4, 0x0d, 0x74, 0x17, 0xd6, 0x27, 0x2f, 0xf6, 0x77, 0x6c, 0x6b, 0x30, 0x39, 0x18, 0xef, 0x4f, + 0x06, 0xf5, 0xac, 0xf9, 0x8f, 0x2c, 0xac, 0x8f, 0xc2, 0x38, 0xe1, 0x93, 0xf3, 0xd0, 0x95, 0xd2, + 0x3f, 0x5d, 0x91, 0xfe, 0xdd, 0x9b, 0xa5, 0x5f, 0x09, 0xbb, 0xae, 0x7a, 0x1f, 0x4a, 0x91, 0xca, + 0x19, 0x75, 0x2a, 0xcd, 0x6f, 0xce, 0x2e, 0x85, 0x30, 0x8f, 0x44, 0x7b, 0x50, 0x4c, 0x37, 0x83, + 0x35, 0xd6, 0xe4, 0xc9, 0x78, 0xf7, 0x36, 0xb9, 0xa0, 0x0f, 0x87, 0x82, 0x40, 0x3f, 0x87, 0x6a, + 0x7a, 0x4e, 0x6c, 0x41, 0x91, 0x35, 0x72, 0x12, 0xf2, 0xad, 0x2b, 0x90, 0xaa, 0xf4, 0xa9, 0x4c, + 0x5f, 0x5a, 0x58, 0xc5, 0x9d, 0x5b, 0x98, 0x69, 0x2a, 0xad, 0xd7, 0xa1, 0xfc, 0x7c, 0x7f, 0x6c, + 0xf5, 0x07, 0xd6, 0xa0, 0x5f, 0xcf, 0xa0, 0x0a, 0x14, 0x75, 0xc7, 0x30, 0xff, 0x57, 0x80, 0xfa, + 0x38, 0xe1, 0x71, 0xc2, 0xad, 0x28, 0xe1, 0x84, 0x4a, 0x81, 0x47, 0x2b, 0x02, 0x77, 0xbe, 0x46, + 0x94, 0x2b, 0x91, 0xd7, 0x35, 0x5e, 0x52, 0x27, 0xfb, 0xe6, 0xea, 0xbc, 0x05, 0xd5, 0x13, 0x87, + 0x9d, 0xd8, 0xba, 0x14, 0x09, 0xc1, 0xd7, 0xad, 0x8a, 0xb0, 0xa5, 0x52, 0x30, 0x14, 0xc0, 0x5d, + 0xea, 0x84, 0x1e, 0xb1, 0xa9, 0x64, 0x65, 0xb3, 0x98, 0xb8, 0x32, 0xad, 0x2b, 0x5b, 0xdb, 0xb7, + 0x58, 0x88, 0x25, 0x30, 0x16, 0x7d, 0x45, 0xe4, 0x0e, 0x5d, 0x35, 0xa3, 0xc7, 0x70, 0x17, 0xfb, + 0x4c, 0x14, 0x66, 0xfb, 0x28, 0x39, 0x3e, 0x4e, 0x73, 0x29, 0xbf, 0x69, 0x3c, 0x2c, 0xa9, 0x88, + 0xba, 0x1a, 0xee, 0xe9, 0xd1, 0xe6, 0x5f, 0xd7, 0xe0, 0xce, 0x15, 0x74, 0xf4, 0x09, 0xe4, 0xc5, + 0xd5, 0xa4, 0x6b, 0xeb, 0x07, 0xdf, 0x9d, 0x68, 0x7b, 0x12, 0x3b, 0xba, 0xe0, 0xa5, 0x98, 0x42, + 0x34, 0x4c, 0x8e, 0x9d, 0x24, 0xe0, 0x36, 0x26, 0x8c, 0xa7, 0xf5, 0xc5, 0xaa, 0x28, 0x5b, 0x9f, + 0x30, 0x8e, 0xa6, 0x50, 0x96, 0xb7, 0xa8, 0x1f, 0x7a, 0x3a, 0x8b, 0x47, 0x6f, 0xc0, 0x21, 0xdd, + 0x8b, 0x81, 0x42, 0xd4, 0xe5, 0x77, 0x3e, 0x43, 0xf3, 0x25, 0xd4, 0x56, 0x5d, 0xd0, 0x03, 0x28, + 0xa4, 0x7b, 0x7a, 0xed, 0x1e, 0x10, 0xd7, 0xc4, 0x53, 0x28, 0xe9, 0x60, 0x75, 0x0d, 0xbc, 0x7d, + 0xc3, 0x81, 0xe8, 0x8b, 0x7f, 0x80, 0x2b, 0x13, 0xcf, 0x63, 0x9b, 0x7b, 0x90, 0x13, 0xf2, 0xa0, + 0x7b, 0x90, 0x67, 0xdc, 0xa1, 0x5c, 0x4e, 0x56, 0xb5, 0xd2, 0x0e, 0xaa, 0xc3, 0x1a, 0x09, 0xd3, + 0xf2, 0x5b, 0xb5, 0x44, 0x53, 0xb0, 0x4a, 0x33, 0x4f, 0xd6, 0xd2, 0xbc, 0x66, 0x95, 0xda, 0xcc, + 0x0f, 0xd4, 0xf1, 0xaa, 0x43, 0xf5, 0xa0, 0x3b, 0x99, 0xd8, 0x87, 0x43, 0x6b, 0xfc, 0x7c, 0x77, + 0x98, 0x56, 0xb4, 0x67, 0x23, 0xcb, 0x1a, 0x5b, 0x75, 0x43, 0x9c, 0xb6, 0xde, 0x0b, 0x7b, 0xd8, + 0x9d, 0x0c, 0xeb, 0x59, 0x54, 0x85, 0x52, 0xef, 0x85, 0x6d, 0x75, 0xf7, 0x77, 0x07, 0xf5, 0x35, + 0xf3, 0x33, 0x03, 0xca, 0x92, 0xf0, 0x28, 0x3c, 0x8e, 0x56, 0x16, 0x69, 0x7c, 0xf7, 0x45, 0xa2, + 0xf7, 0xd5, 0xe1, 0x4d, 0x2b, 0xda, 0xb7, 0xae, 0x1c, 0x32, 0xc8, 0xfc, 0x2d, 0xd4, 0x0e, 0x68, + 0x84, 0x13, 0x97, 0xd0, 0x21, 0x71, 0x30, 0xa1, 0xe8, 0x31, 0x14, 0x8f, 0x83, 0x68, 0x26, 0x2e, + 0x07, 0xa9, 0x56, 0xaf, 0x21, 0xdc, 0xff, 0x7e, 0xd1, 0x2a, 0x3c, 0x0d, 0xa2, 0xd9, 0xa8, 0x7f, + 0x39, 0x6f, 0x59, 0x05, 0xe1, 0x38, 0xc2, 0x6f, 0x70, 0x9b, 0x99, 0x7f, 0x36, 0xa0, 0xaa, 0x09, + 0xf4, 0x1d, 0xee, 0xa0, 0x1f, 0x40, 0x99, 0x3a, 0x33, 0xfb, 0xe8, 0x9c, 0x13, 0xa6, 0xb6, 0xab, + 0x44, 0x9d, 0x59, 0x4f, 0xf4, 0x91, 0x05, 0xa5, 0x29, 0xe1, 0x8e, 0xf8, 0xdb, 0x54, 0xd5, 0xe5, + 0xd1, 0xcd, 0x59, 0x6b, 0x91, 0x69, 0xc4, 0x89, 0x06, 0x7f, 0xa6, 0xe2, 0xb4, 0x7c, 0x1a, 0x07, + 0xbd, 0x03, 0xb5, 0x30, 0x99, 0xda, 0x64, 0x1a, 0xf3, 0x73, 0x9b, 0x46, 0x33, 0xb6, 0xb2, 0xf7, + 0xd5, 0x30, 0x99, 0x0e, 0xc4, 0x90, 0x15, 0xcd, 0x98, 0xf9, 0x85, 0x01, 0x77, 0x16, 0x80, 0x8c, + 0x39, 0x1e, 0x41, 0x1f, 0x42, 0xe1, 0x44, 0x2a, 0xa7, 0xfe, 0x1a, 0x1f, 0xde, 0xcc, 0x68, 0x55, + 0x69, 0x4b, 0xc5, 0xa1, 0x2e, 0x14, 0xf8, 0x79, 0x9c, 0xe6, 0xba, 0x58, 0xd3, 0x8f, 0x6e, 0x46, + 0x98, 0x67, 0x8f, 0x4e, 0xcd, 0x34, 0x10, 0x7d, 0x08, 0x39, 0x29, 0xca, 0x9a, 0xa4, 0xf0, 0xe3, + 0x6f, 0xa6, 0xd0, 0x5f, 0x48, 0x21, 0x23, 0xcd, 0x8b, 0x3c, 0xdc, 0x7f, 0xbd, 0x62, 0xe8, 0xd7, + 0x00, 0x69, 0x85, 0xf5, 0xc3, 0xe3, 0x48, 0xad, 0xf2, 0xfd, 0xdb, 0xea, 0x9e, 0xd6, 0x0c, 0x41, + 0x9d, 0x0d, 0x33, 0x56, 0x99, 0xea, 0x1e, 0x7a, 0x02, 0x79, 0xb2, 0xf4, 0x9f, 0xdc, 0xba, 0x19, + 0x58, 0xff, 0x1c, 0xa7, 0xfe, 0xe8, 0x13, 0x00, 0xf1, 0xbf, 0x4f, 0xec, 0xa5, 0x95, 0x6f, 0xdf, + 0x9a, 0xd6, 0xa1, 0x80, 0x10, 0x6a, 0x08, 0x56, 0x5c, 0x77, 0xd0, 0x2e, 0xd4, 0xf8, 0x59, 0x68, + 0xbb, 0x51, 0x44, 0xb1, 0x2d, 0x72, 0x45, 0x5d, 0x29, 0xad, 0xd7, 0xfc, 0xc6, 0x1f, 0x9e, 0x85, + 0x3b, 0xc2, 0x4f, 0x60, 0x0e, 0x33, 0x56, 0x95, 0x2f, 0xf5, 0x91, 0x05, 0x45, 0x1a, 0xcd, 0xec, + 0x30, 0x99, 0xca, 0x6b, 0xa2, 0xb2, 0xf5, 0xe4, 0xf6, 0xca, 0x45, 0xb3, 0xfd, 0x64, 0x3a, 0xcc, + 0x58, 0x05, 0x2a, 0x5b, 0xcd, 0x31, 0xc0, 0x42, 0x4d, 0xd4, 0xbd, 0xb2, 0x3d, 0x22, 0x85, 0x1e, + 0xbc, 0x86, 0xe6, 0x3c, 0x44, 0xd7, 0xe7, 0xf9, 0x1e, 0x34, 0x7f, 0x03, 0xe5, 0xb9, 0x0e, 0x68, + 0x02, 0x77, 0xdc, 0x28, 0x08, 0x88, 0xcb, 0xd5, 0x03, 0x4a, 0xdf, 0x52, 0xcb, 0xe5, 0x49, 0x3c, + 0xb7, 0xda, 0xea, 0xb9, 0xd5, 0xb6, 0xd4, 0x73, 0x6b, 0xe9, 0x2a, 0xaa, 0xcd, 0x21, 0x84, 0x91, + 0x35, 0x67, 0x50, 0x48, 0x97, 0x81, 0x7e, 0x0a, 0x65, 0x46, 0x42, 0x4c, 0xa8, 0xae, 0x30, 0xe5, + 0x5e, 0x7d, 0x5e, 0x2c, 0xe4, 0x80, 0x2c, 0x10, 0x69, 0x0b, 0x8b, 0x17, 0x83, 0xd6, 0x2f, 0xbb, + 0x5c, 0x93, 0x53, 0x29, 0x50, 0x0b, 0x4a, 0x81, 0xc3, 0xb8, 0x3d, 0x65, 0x9e, 0x4c, 0x01, 0x7d, + 0x0d, 0x17, 0x85, 0xf5, 0x19, 0xf3, 0x7a, 0x45, 0xc8, 0xbf, 0x74, 0x82, 0x84, 0x98, 0xbf, 0x37, + 0xa0, 0xd1, 0xf7, 0x19, 0x9f, 0xfc, 0x62, 0xef, 0x97, 0xe9, 0xa3, 0x68, 0x37, 0x62, 0xcc, 0x8f, + 0x65, 0x12, 0x3e, 0x5a, 0x7d, 0x3e, 0xad, 0xf7, 0xee, 0x0b, 0x94, 0x2f, 0x2f, 0x5a, 0xb5, 0xd5, + 0x90, 0xc5, 0x83, 0x6a, 0x08, 0xf7, 0xa6, 0x7e, 0x68, 0x3b, 0xae, 0x4b, 0x62, 0x21, 0x94, 0x0e, + 0xcf, 0x7e, 0x6d, 0x38, 0x9a, 0xfa, 0x61, 0x57, 0x85, 0x28, 0x9b, 0xf9, 0x04, 0xbe, 0xa7, 0xbc, + 0xfa, 0xd4, 0xf1, 0x43, 0x3f, 0xf4, 0x24, 0xa5, 0x4d, 0x28, 0x61, 0xd5, 0x97, 0x9c, 0xf4, 0xca, + 0xe6, 0xd6, 0xde, 0xdb, 0xaf, 0xfe, 0xb5, 0x91, 0x79, 0x75, 0xb9, 0x61, 0xfc, 0xed, 0x72, 0xc3, + 0xf8, 0xe2, 0x72, 0xc3, 0xf8, 0xe7, 0xe5, 0x86, 0xf1, 0xd9, 0xbf, 0x37, 0x32, 0x1f, 0xc3, 0x22, + 0xa9, 0xbe, 0x0a, 0x00, 0x00, 0xff, 0xff, 0x27, 0xd4, 0x69, 0xa9, 0xec, 0x0f, 0x00, 0x00, } diff --git a/pkg/sql/distsqlrun/data.proto b/pkg/sql/distsqlrun/data.proto index 37e91c79eb60..de5370822b11 100644 --- a/pkg/sql/distsqlrun/data.proto +++ b/pkg/sql/distsqlrun/data.proto @@ -42,14 +42,16 @@ message Error { } message Expression { + // Don't generate a typedecl, so we can add the LocalExpr field. + option (gogoproto.typedecl) = false; + option (gogoproto.goproto_stringer) = false; + // TODO(radu): TBD how this will be used optional string version = 1 [(gogoproto.nullable) = false]; // SQL expressions are passed as a string, with ordinal references // (@1, @2, @3 ..) used for "input" variables. optional string expr = 2 [(gogoproto.nullable) = false]; - - optional uint32 local_expr_idx = 3 [(gogoproto.nullable) = false]; } // Ordering defines an order - specifically a list of column indices and diff --git a/pkg/sql/distsqlrun/expr.go b/pkg/sql/distsqlrun/expr.go index 282a257508b0..9af11ca245b2 100644 --- a/pkg/sql/distsqlrun/expr.go +++ b/pkg/sql/distsqlrun/expr.go @@ -145,18 +145,15 @@ func (eh *exprHelper) IndexedVarNodeFormatter(idx int) tree.NodeFormatter { func (eh *exprHelper) init( expr Expression, types []sqlbase.ColumnType, evalCtx *tree.EvalContext, ) error { - eh.evalCtx = evalCtx - if expr.Expr == "" && evalCtx.LocalExprs == nil { + if expr.Empty() { return nil } + eh.evalCtx = evalCtx eh.types = types eh.vars = tree.MakeIndexedVarHelper(eh, len(types)) - if expr.LocalExprIdx != 0 { - eh.expr = (*evalCtx.LocalExprs)[expr.LocalExprIdx-1] - if eh.expr == nil { - return errors.Errorf("programming error: local expr %d was not available in slice %v", expr.LocalExprIdx-1, evalCtx.LocalExprs) - } + if expr.LocalExpr != nil { + eh.expr = expr.LocalExpr // Bind IndexedVars to our eh.vars. eh.vars.Rebind(eh.expr, true /* alsoReset */, false /* normalizeToNonNil */) return nil diff --git a/pkg/sql/distsqlrun/flow_diagram.go b/pkg/sql/distsqlrun/flow_diagram.go index 917e88767411..bad064c15516 100644 --- a/pkg/sql/distsqlrun/flow_diagram.go +++ b/pkg/sql/distsqlrun/flow_diagram.go @@ -151,16 +151,16 @@ func (jr *JoinReaderSpec) summary() (string, []string) { if jr.LookupColumns != nil { details = append(details, fmt.Sprintf("Lookup join on: %s", colListStr(jr.LookupColumns))) } - if jr.IndexFilterExpr.Expr != "" { + if !jr.IndexFilterExpr.Empty() { // Note: The displayed IndexFilter is a bit confusing because its // IndexedVars refer to only the index column indices. This means they don't // line up with other column indices like the output columns, which refer to // the columns from both sides of the join. details = append( - details, fmt.Sprintf("IndexFilter: %s (right side only)", jr.IndexFilterExpr.Expr)) + details, fmt.Sprintf("IndexFilter: %s (right side only)", jr.IndexFilterExpr)) } - if jr.OnExpr.Expr != "" { - details = append(details, fmt.Sprintf("ON %s", jr.OnExpr.Expr)) + if !jr.OnExpr.Empty() { + details = append(details, fmt.Sprintf("ON %s", jr.OnExpr)) } return "JoinReader", details } @@ -191,8 +191,8 @@ func (hj *HashJoinerSpec) summary() (string, []string) { colListStr(hj.LeftEqColumns), colListStr(hj.RightEqColumns), )) } - if hj.OnExpr.Expr != "" { - details = append(details, fmt.Sprintf("ON %s", hj.OnExpr.Expr)) + if !hj.OnExpr.Empty() { + details = append(details, fmt.Sprintf("ON %s", hj.OnExpr)) } if hj.MergedColumns { details = append(details, fmt.Sprintf("Merged columns: %d", len(hj.LeftEqColumns))) @@ -213,8 +213,8 @@ func orderedJoinDetails( "left(%s)=right(%s)", left.diagramString(), right.diagramString(), )) - if onExpr.Expr != "" { - details = append(details, fmt.Sprintf("ON %s", onExpr.Expr)) + if !onExpr.Empty() { + details = append(details, fmt.Sprintf("ON %s", onExpr)) } return details @@ -289,7 +289,7 @@ func (d *DistinctSpec) summary() (string, []string) { func (d *ProjectSetSpec) summary() (string, []string) { var details []string for _, expr := range d.Exprs { - details = append(details, expr.Expr) + details = append(details, expr.String()) } return "ProjectSet", details } @@ -362,8 +362,8 @@ func (post *PostProcessSpec) summary() []string { // (namely InterleavedReaderJoiner) that have multiple PostProcessors. func (post *PostProcessSpec) summaryWithPrefix(prefix string) []string { var res []string - if post.Filter.Expr != "" { - res = append(res, fmt.Sprintf("%sFilter: %s", prefix, post.Filter.Expr)) + if !post.Filter.Empty() { + res = append(res, fmt.Sprintf("%sFilter: %s", prefix, post.Filter)) } if post.Projection { outputColumns := "None" @@ -381,7 +381,7 @@ func (post *PostProcessSpec) summaryWithPrefix(prefix string) []string { } // Remove any spaces in the expression (makes things more compact // and it's easier to visually separate expressions). - buf.WriteString(strings.Replace(expr.Expr, " ", "", -1)) + buf.WriteString(strings.Replace(expr.String(), " ", "", -1)) } res = append(res, buf.String()) } diff --git a/pkg/sql/distsqlrun/joinerbase.go b/pkg/sql/distsqlrun/joinerbase.go index 5463ec0f85b6..11d0b854040a 100644 --- a/pkg/sql/distsqlrun/joinerbase.go +++ b/pkg/sql/distsqlrun/joinerbase.go @@ -62,7 +62,7 @@ func (jb *joinerBase) init( jb.joinType = jType if isSetOpJoin(jb.joinType) { - if onExpr.Expr != "" { + if !onExpr.Empty() { return errors.Errorf("expected empty onExpr, got %v", onExpr.Expr) } } diff --git a/pkg/sql/distsqlrun/joinreader.go b/pkg/sql/distsqlrun/joinreader.go index de95f4e3b2a4..7eeb57092287 100644 --- a/pkg/sql/distsqlrun/joinreader.go +++ b/pkg/sql/distsqlrun/joinreader.go @@ -186,11 +186,8 @@ func newJoinReader( ); err != nil { return nil, err } - if spec.IndexFilterExpr.Expr != "" { - err := jr.indexFilter.init(spec.IndexFilterExpr, columnTypes, jr.evalCtx) - if err != nil { - return nil, err - } + if err := jr.indexFilter.init(spec.IndexFilterExpr, columnTypes, jr.evalCtx); err != nil { + return nil, err } // neededIndexColumns is the set of columns we need to fetch from jr.index. diff --git a/pkg/sql/distsqlrun/readerbase.go b/pkg/sql/distsqlrun/readerbase.go index 04c83c0a0ece..0f176e5ed352 100644 --- a/pkg/sql/distsqlrun/readerbase.go +++ b/pkg/sql/distsqlrun/readerbase.go @@ -54,7 +54,7 @@ func limitHint(specLimitHint int64, post *PostProcessSpec) (limitHint int64) { limitHint = specLimitHint + rowChannelBufSize + 1 } - if post.Filter.Expr != "" { + if !post.Filter.Empty() { // We have a filter so we will likely need to read more rows. limitHint *= 2 } diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index fe6d77b1de04..0566da9be941 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -2489,11 +2489,6 @@ type EvalContext struct { // evaluation. It can change over the course of evaluation, such as on a // per-row basis. ActiveMemAcc *mon.BoundAccount - - // LocalExprs contains the list of typed expressions that DistSQL Expressions - // can index into when running on the gateway. See distsqlrun.Expression and - // its LocalExprIdx field. - LocalExprs *[]TypedExpr } // MakeTestingEvalContext returns an EvalContext that includes a MemoryMonitor. From 50463729208f410a41222a4b929bf5e51f67077a Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Thu, 20 Sep 2018 20:46:53 -0400 Subject: [PATCH 3/6] distsql: pool some top-level large objects The following objects are now kept in memory pools and reset and released instead of letting the GC reclaim them: - TableReaderSpec - copyingRowReceiver - FlowSpec - tableReader These objects tend to have several slice fields on them. Code was changed to resize/reslice these fields instead of always making them anew, and the Release/Reset methods now 0-size-slice these fields instead of letting them be cleared. This has the effect of saving a lot of slice allocations. Release note: None --- pkg/sql/distsql_physical_planner.go | 66 ++++++++++++++--------- pkg/sql/distsql_plan_scrub_physical.go | 2 +- pkg/sql/distsql_running.go | 7 ++- pkg/sql/distsqlplan/physical_plan.go | 8 +-- pkg/sql/distsqlrun/api.go | 31 +++++++++++ pkg/sql/distsqlrun/base.go | 18 +++++++ pkg/sql/distsqlrun/flow.go | 72 +++++++++++++------------ pkg/sql/distsqlrun/flow_diagram.go | 8 +-- pkg/sql/distsqlrun/flow_diagram_test.go | 18 +++---- pkg/sql/distsqlrun/processors.go | 59 ++++++++++++++++++-- pkg/sql/distsqlrun/tablereader.go | 31 ++++++++++- pkg/sql/sqlbase/rowfetcher.go | 70 ++++++++++++++++++++---- 12 files changed, 295 insertions(+), 95 deletions(-) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 72def49373aa..d5d7b0bcb3f5 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -846,10 +846,13 @@ func getIndexIdx(n *scanNode) (uint32, error) { // initTableReaderSpec initializes a TableReaderSpec/PostProcessSpec that // corresponds to a scanNode, except for the Spans and OutputColumns. +// Only needs indexVarMap for a filter expression - if there's no filter +// expression in the scanNode, it's valid to pass a nil indexVarMap. func initTableReaderSpec( n *scanNode, planCtx *PlanningCtx, indexVarMap []int, -) (distsqlrun.TableReaderSpec, distsqlrun.PostProcessSpec, error) { - s := distsqlrun.TableReaderSpec{ +) (*distsqlrun.TableReaderSpec, distsqlrun.PostProcessSpec, error) { + s := distsqlrun.NewTableReaderSpec() + *s = distsqlrun.TableReaderSpec{ Table: *n.desc, Reverse: n.reverse, IsCheck: n.run.isCheck, @@ -857,7 +860,7 @@ func initTableReaderSpec( } indexIdx, err := getIndexIdx(n) if err != nil { - return distsqlrun.TableReaderSpec{}, distsqlrun.PostProcessSpec{}, err + return nil, distsqlrun.PostProcessSpec{}, err } s.IndexIdx = indexIdx @@ -870,7 +873,7 @@ func initTableReaderSpec( filter, err := distsqlplan.MakeExpression(n.filter, planCtx, indexVarMap) if err != nil { - return distsqlrun.TableReaderSpec{}, distsqlrun.PostProcessSpec{}, err + return nil, distsqlrun.PostProcessSpec{}, err } post := distsqlrun.PostProcessSpec{ Filter: filter, @@ -1048,17 +1051,12 @@ func (dsp *DistSQLPlanner) createTableReaders( planCtx *PlanningCtx, n *scanNode, overrideResultColumns []sqlbase.ColumnID, ) (PhysicalPlan, error) { - scanNodeToTableOrdinalMap := getScanNodeToTableOrdinalMap(n) - spec, post, err := initTableReaderSpec(n, planCtx, scanNodeToTableOrdinalMap) - if err != nil { - return PhysicalPlan{}, err - } - var spanPartitions []SpanPartition if planCtx.isLocal { spanPartitions = []SpanPartition{{dsp.nodeDesc.NodeID, n.spans}} } else if n.hardLimit == 0 && n.softLimit == 0 { // No limit - plan all table readers where their data live. + var err error spanPartitions, err = dsp.PartitionSpans(planCtx, n.spans) if err != nil { return PhysicalPlan{}, err @@ -1080,12 +1078,40 @@ func (dsp *DistSQLPlanner) createTableReaders( stageID := p.NewStageID() p.ResultRouters = make([]distsqlplan.ProcessorIdx, len(spanPartitions)) + p.Processors = make([]distsqlplan.Processor, 0, len(spanPartitions)) returnMutations := n.colCfg.visibility == publicAndNonPublicColumns + needMergeOrdering := len(p.ResultRouters) > 1 && len(n.props.ordering) > 0 + var scanNodeToTableOrdinalMap []int + if n.filter != nil || needMergeOrdering { + // If n.filter != nil, we need to make a scanNodeToTableOrdinalMap to pass + // to initTableReaderSpec, so it can properly create a filter expression. + scanNodeToTableOrdinalMap = getScanNodeToTableOrdinalMap(n) + + if needMergeOrdering { + // Make a note of the fact that we have to maintain a certain ordering + // between the parallel streams. + // + // This information is taken into account by the AddProjection call below: + // specifically, it will make sure these columns are kept even if they are + // not in the projection (e.g. "SELECT v FROM kv ORDER BY k"). + p.SetMergeOrdering(dsp.convertOrdering(n.props, scanNodeToTableOrdinalMap)) + } + } + spec, post, err := initTableReaderSpec(n, planCtx, scanNodeToTableOrdinalMap) + if err != nil { + return PhysicalPlan{}, err + } + for i, sp := range spanPartitions { - tr := &distsqlrun.TableReaderSpec{} - *tr = spec + var tr *distsqlrun.TableReaderSpec + if i == 0 { + tr = spec + } else { + tr = new(distsqlrun.TableReaderSpec) + *tr = *spec + } tr.Spans = makeTableReaderSpans(sp.Spans) proc := distsqlplan.Processor{ @@ -1101,16 +1127,6 @@ func (dsp *DistSQLPlanner) createTableReaders( p.ResultRouters[i] = pIdx } - if len(p.ResultRouters) > 1 && len(n.props.ordering) > 0 { - // Make a note of the fact that we have to maintain a certain ordering - // between the parallel streams. - // - // This information is taken into account by the AddProjection call below: - // specifically, it will make sure these columns are kept even if they are - // not in the projection (e.g. "SELECT v FROM kv ORDER BY k"). - p.SetMergeOrdering(dsp.convertOrdering(n.props, scanNodeToTableOrdinalMap)) - } - var types []sqlbase.ColumnType if returnMutations { types = make([]sqlbase.ColumnType, 0, len(n.desc.Columns)+len(n.desc.Mutations)) @@ -1140,9 +1156,9 @@ func (dsp *DistSQLPlanner) createTableReaders( } } planToStreamColMap := make([]int, len(n.cols)) - var descColumnIDs []sqlbase.ColumnID - for _, c := range n.desc.Columns { - descColumnIDs = append(descColumnIDs, c.ID) + descColumnIDs := make([]sqlbase.ColumnID, 0, len(n.desc.Columns)) + for i := range n.desc.Columns { + descColumnIDs = append(descColumnIDs, n.desc.Columns[i].ID) } if returnMutations { for _, m := range n.desc.Mutations { diff --git a/pkg/sql/distsql_plan_scrub_physical.go b/pkg/sql/distsql_plan_scrub_physical.go index 7595f51b658d..a198243bbe8f 100644 --- a/pkg/sql/distsql_plan_scrub_physical.go +++ b/pkg/sql/distsql_plan_scrub_physical.go @@ -50,7 +50,7 @@ func (dsp *DistSQLPlanner) createScrubPhysicalCheck( p.ResultRouters = make([]distsqlplan.ProcessorIdx, len(spanPartitions)) for i, sp := range spanPartitions { tr := &distsqlrun.TableReaderSpec{} - *tr = spec + *tr = *spec tr.Spans = make([]distsqlrun.TableReaderSpan, len(sp.Spans)) for j := range sp.Spans { tr.Spans[j].Span = sp.Spans[j] diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 7aba65aed0e7..d421310896ba 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -60,6 +60,8 @@ type runnerResult struct { } func (req runnerRequest) run() { + defer req.flowReq.Release() + res := runnerResult{nodeID: req.nodeID} conn, err := req.nodeDialer.Dial(req.ctx, req.nodeID) @@ -203,7 +205,7 @@ func (dsp *DistSQLPlanner) Run( continue } req := setupReq - req.Flow = flowSpec + req.Flow = *flowSpec runReq := runnerRequest{ ctx: ctx, nodeDialer: dsp.nodeDialer, @@ -238,7 +240,8 @@ func (dsp *DistSQLPlanner) Run( // Set up the flow on this node. localReq := setupReq - localReq.Flow = flows[thisNodeID] + localReq.Flow = *flows[thisNodeID] + defer localReq.Release() ctx, flow, err := dsp.distSQLSrv.SetupLocalSyncFlow(ctx, evalCtx.Mon, &localReq, recv, localState) if err != nil { recv.SetError(err) diff --git a/pkg/sql/distsqlplan/physical_plan.go b/pkg/sql/distsqlplan/physical_plan.go index dd5f6c206aeb..f95f161641bf 100644 --- a/pkg/sql/distsqlplan/physical_plan.go +++ b/pkg/sql/distsqlplan/physical_plan.go @@ -810,7 +810,7 @@ func (p *PhysicalPlan) PopulateEndpoints(nodeAddresses map[roachpb.NodeID]string // gateway is the current node's NodeID. func (p *PhysicalPlan) GenerateFlowSpecs( gateway roachpb.NodeID, -) map[roachpb.NodeID]distsqlrun.FlowSpec { +) map[roachpb.NodeID]*distsqlrun.FlowSpec { // Only generate a flow ID for a remote plan because it will need to be // referenced by remote nodes when connecting streams. This id generation is // skipped for performance reasons on local flows. @@ -818,15 +818,15 @@ func (p *PhysicalPlan) GenerateFlowSpecs( if p.remotePlan { flowID.UUID = uuid.MakeV4() } - flows := make(map[roachpb.NodeID]distsqlrun.FlowSpec) + flows := make(map[roachpb.NodeID]*distsqlrun.FlowSpec, 1) for _, proc := range p.Processors { flowSpec, ok := flows[proc.Node] if !ok { - flowSpec = distsqlrun.FlowSpec{FlowID: flowID, Gateway: gateway} + flowSpec = distsqlrun.NewFlowSpec() + flows[proc.Node] = flowSpec } flowSpec.Processors = append(flowSpec.Processors, proc.Spec) - flows[proc.Node] = flowSpec } return flows } diff --git a/pkg/sql/distsqlrun/api.go b/pkg/sql/distsqlrun/api.go index 6f79e3a64bc6..429a0afe1cfa 100644 --- a/pkg/sql/distsqlrun/api.go +++ b/pkg/sql/distsqlrun/api.go @@ -15,6 +15,8 @@ package distsqlrun import ( + "sync" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" ) @@ -60,3 +62,32 @@ func MakeEvalContext(evalCtx tree.EvalContext) EvalContext { } return res } + +var trSpecPool = sync.Pool{} + +func NewTableReaderSpec() *TableReaderSpec { + var s *TableReaderSpec + if sPtr := trSpecPool.Get(); sPtr == nil { + s = &TableReaderSpec{} + } else { + s = sPtr.(*TableReaderSpec) + } + return s +} + +func (s *TableReaderSpec) Release() { + s.Reset() + trSpecPool.Put(s) +} + +func (s *SetupFlowRequest) Release() { + if s == nil { + return + } + for i := range s.Flow.Processors { + if tr := s.Flow.Processors[i].Core.TableReader; tr != nil { + tr.Release() + } + } + s.Flow.Release() +} diff --git a/pkg/sql/distsqlrun/base.go b/pkg/sql/distsqlrun/base.go index 117fcf96ffb6..04c09e48648a 100644 --- a/pkg/sql/distsqlrun/base.go +++ b/pkg/sql/distsqlrun/base.go @@ -647,11 +647,29 @@ func (rb *RowBuffer) ConsumerClosed() { } } +var copyingRowReceiverPool = sync.Pool{ + New: func() interface{} { + return ©ingRowReceiver{} + }, +} + type copyingRowReceiver struct { RowReceiver alloc sqlbase.EncDatumRowAlloc } +func newCopyingRowReceiver(wrapped RowReceiver) *copyingRowReceiver { + ret := copyingRowReceiverPool.Get().(*copyingRowReceiver) + ret.RowReceiver = wrapped + return ret +} + +// Release implements the Releasable interface. +func (r *copyingRowReceiver) Release() { + *r = copyingRowReceiver{} + copyingRowReceiverPool.Put(r) +} + func (r *copyingRowReceiver) Push(row sqlbase.EncDatumRow, meta *ProducerMetadata) ConsumerStatus { if row != nil { row = r.alloc.CopyRow(row) diff --git a/pkg/sql/distsqlrun/flow.go b/pkg/sql/distsqlrun/flow.go index 457edf06d06b..c4f9172d41e7 100644 --- a/pkg/sql/distsqlrun/flow.go +++ b/pkg/sql/distsqlrun/flow.go @@ -338,28 +338,25 @@ func (f *Flow) makeProcessor( if len(ps.Output) != 1 { return nil, errors.Errorf("only single-output processors supported") } - outputs := make([]RowReceiver, len(ps.Output)) - for i := range ps.Output { - spec := &ps.Output[i] - if spec.Type == OutputRouterSpec_PASS_THROUGH { - // There is no entity that corresponds to a pass-through router - we just - // use its output stream directly. - if len(spec.Streams) != 1 { - return nil, errors.Errorf("expected one stream for passthrough router") - } - var err error - outputs[i], err = f.setupOutboundStream(spec.Streams[0]) - if err != nil { - return nil, err - } - continue + var output RowReceiver + spec := &ps.Output[0] + if spec.Type == OutputRouterSpec_PASS_THROUGH { + // There is no entity that corresponds to a pass-through router - we just + // use its output stream directly. + if len(spec.Streams) != 1 { + return nil, errors.Errorf("expected one stream for passthrough router") } - + var err error + output, err = f.setupOutboundStream(spec.Streams[0]) + if err != nil { + return nil, err + } + } else { r, err := f.setupRouter(spec) if err != nil { return nil, err } - outputs[i] = r + output = r f.startables = append(f.startables, r) } @@ -369,11 +366,9 @@ func (f *Flow) makeProcessor( // outputs aren't used at all if they are processors that get fused to their // upstreams, though, which means that copyingRowReceivers are only used on // non-fused processors like the output routers. + output = newCopyingRowReceiver(output) - for i := range outputs { - outputs[i] = ©ingRowReceiver{RowReceiver: outputs[i]} - } - + outputs := []RowReceiver{output} proc, err := newProcessor(ctx, &f.FlowCtx, ps.ProcessorID, &ps.Core, &ps.Post, inputs, outputs, f.localProcessors) if err != nil { return nil, err @@ -381,18 +376,16 @@ func (f *Flow) makeProcessor( // Initialize any routers (the setupRouter case above) and outboxes. types := proc.OutputTypes() - for _, o := range outputs { - rowRecv := o.(*copyingRowReceiver).RowReceiver - clearer, ok := rowRecv.(*accountClearingRowReceiver) - if ok { - rowRecv = clearer.RowReceiver - } - switch o := rowRecv.(type) { - case router: - o.init(ctx, &f.FlowCtx, types) - case *outbox: - o.init(types) - } + rowRecv := output.(*copyingRowReceiver).RowReceiver + clearer, ok := rowRecv.(*accountClearingRowReceiver) + if ok { + rowRecv = clearer.RowReceiver + } + switch o := rowRecv.(type) { + case router: + o.init(ctx, &f.FlowCtx, types) + case *outbox: + o.init(types) } return proc, nil } @@ -623,6 +616,14 @@ func (f *Flow) Wait() { } } +// Releasable is an interface for objects than can be Released back into a +// memory pool when finished. +type Releasable interface { + // Release allows this object to be returned to a memory pool. Objects must + // not be used after Release is called. + Release() +} + // Cleanup should be called when the flow completes (after all processors and // mailboxes exited). func (f *Flow) Cleanup(ctx context.Context) { @@ -632,6 +633,11 @@ func (f *Flow) Cleanup(ctx context.Context) { // This closes the account and monitor opened in ServerImpl.setupFlow. f.EvalCtx.ActiveMemAcc.Close(ctx) f.EvalCtx.Stop(ctx) + for _, p := range f.processors { + if d, ok := p.(Releasable); ok { + d.Release() + } + } if log.V(1) { log.Infof(ctx, "cleaning up") } diff --git a/pkg/sql/distsqlrun/flow_diagram.go b/pkg/sql/distsqlrun/flow_diagram.go index bad064c15516..2ceb43e30991 100644 --- a/pkg/sql/distsqlrun/flow_diagram.go +++ b/pkg/sql/distsqlrun/flow_diagram.go @@ -620,7 +620,7 @@ func generateDiagramData( // across all flows. If spans are provided, stats are extracted from the spans // and added to the plan. func GeneratePlanDiagram( - flows map[roachpb.NodeID]FlowSpec, spans []tracing.RecordedSpan, w io.Writer, + flows map[roachpb.NodeID]*FlowSpec, spans []tracing.RecordedSpan, w io.Writer, ) error { // We sort the flows by node because we want the diagram data to be // deterministic. @@ -634,7 +634,7 @@ func GeneratePlanDiagram( nodeNames := make([]string, len(nodeIDs)) for i, nVal := range nodeIDs { n := roachpb.NodeID(nVal) - flowSlice[i] = flows[n] + flowSlice[i] = *flows[n] nodeNames[i] = n.String() } @@ -649,7 +649,7 @@ func GeneratePlanDiagram( // GeneratePlanDiagramURL generates the json data for a flow diagram and a // URL which encodes the diagram. There should be one FlowSpec per node. The // function assumes that StreamIDs are unique across all flows. -func GeneratePlanDiagramURL(flows map[roachpb.NodeID]FlowSpec) (string, url.URL, error) { +func GeneratePlanDiagramURL(flows map[roachpb.NodeID]*FlowSpec) (string, url.URL, error) { return GeneratePlanDiagramURLWithSpans(flows, nil /* spans */) } @@ -657,7 +657,7 @@ func GeneratePlanDiagramURL(flows map[roachpb.NodeID]FlowSpec) (string, url.URL, // called with no spans. If spans are provided, stats are extracted and added to // the plan. func GeneratePlanDiagramURLWithSpans( - flows map[roachpb.NodeID]FlowSpec, spans []tracing.RecordedSpan, + flows map[roachpb.NodeID]*FlowSpec, spans []tracing.RecordedSpan, ) (string, url.URL, error) { var json bytes.Buffer if err := GeneratePlanDiagram(flows, spans, &json); err != nil { diff --git a/pkg/sql/distsqlrun/flow_diagram_test.go b/pkg/sql/distsqlrun/flow_diagram_test.go index 6e4d1494c5fe..4d3baa2406b3 100644 --- a/pkg/sql/distsqlrun/flow_diagram_test.go +++ b/pkg/sql/distsqlrun/flow_diagram_test.go @@ -49,7 +49,7 @@ func compareDiagrams(t *testing.T, result string, expected string) { func TestPlanDiagramIndexJoin(t *testing.T) { defer leaktest.AfterTest(t)() - flows := make(map[roachpb.NodeID]FlowSpec) + flows := make(map[roachpb.NodeID]*FlowSpec) desc := &sqlbase.TableDescriptor{ Name: "Table", @@ -60,7 +60,7 @@ func TestPlanDiagramIndexJoin(t *testing.T) { IndexIdx: 1, } - flows[1] = FlowSpec{ + flows[1] = &FlowSpec{ Processors: []ProcessorSpec{{ Core: ProcessorCoreUnion{TableReader: &tr}, Post: PostProcessSpec{ @@ -78,7 +78,7 @@ func TestPlanDiagramIndexJoin(t *testing.T) { }}, } - flows[2] = FlowSpec{ + flows[2] = &FlowSpec{ Processors: []ProcessorSpec{{ Core: ProcessorCoreUnion{TableReader: &tr}, Post: PostProcessSpec{ @@ -96,7 +96,7 @@ func TestPlanDiagramIndexJoin(t *testing.T) { }}, } - flows[3] = FlowSpec{ + flows[3] = &FlowSpec{ Processors: []ProcessorSpec{ { Core: ProcessorCoreUnion{TableReader: &tr}, @@ -174,7 +174,7 @@ func TestPlanDiagramIndexJoin(t *testing.T) { func TestPlanDiagramJoin(t *testing.T) { defer leaktest.AfterTest(t)() - flows := make(map[roachpb.NodeID]FlowSpec) + flows := make(map[roachpb.NodeID]*FlowSpec) descA := &sqlbase.TableDescriptor{Name: "TableA"} descB := &sqlbase.TableDescriptor{Name: "TableB"} @@ -190,7 +190,7 @@ func TestPlanDiagramJoin(t *testing.T) { MergedColumns: true, } - flows[1] = FlowSpec{ + flows[1] = &FlowSpec{ Processors: []ProcessorSpec{ { Core: ProcessorCoreUnion{TableReader: &trA}, @@ -211,7 +211,7 @@ func TestPlanDiagramJoin(t *testing.T) { }, } - flows[2] = FlowSpec{ + flows[2] = &FlowSpec{ Processors: []ProcessorSpec{ { Core: ProcessorCoreUnion{TableReader: &trA}, @@ -278,7 +278,7 @@ func TestPlanDiagramJoin(t *testing.T) { }, } - flows[3] = FlowSpec{ + flows[3] = &FlowSpec{ Processors: []ProcessorSpec{ { Core: ProcessorCoreUnion{TableReader: &trA}, @@ -342,7 +342,7 @@ func TestPlanDiagramJoin(t *testing.T) { }, } - flows[4] = FlowSpec{ + flows[4] = &FlowSpec{ Processors: []ProcessorSpec{{ Core: ProcessorCoreUnion{TableReader: &trB}, Post: PostProcessSpec{ diff --git a/pkg/sql/distsqlrun/processors.go b/pkg/sql/distsqlrun/processors.go index adf754d79759..b5aef3e2108c 100644 --- a/pkg/sql/distsqlrun/processors.go +++ b/pkg/sql/distsqlrun/processors.go @@ -87,6 +87,17 @@ type ProcOutputHelper struct { rowIdx uint64 } +// Reset resets this ProcOutputHelper. +func (h *ProcOutputHelper) Reset() { + if r, ok := h.output.(Releasable); ok { + r.Release() + } + *h = ProcOutputHelper{ + renderExprs: h.renderExprs[:0], + outputTypes: h.outputTypes[:0], + } +} + // Init sets up a ProcOutputHelper. The types describe the internal schema of // the processor (as described for each processor core spec); they can be // omitted if there is no filtering expression. @@ -120,14 +131,28 @@ func (h *ProcOutputHelper) Init( // nil indicates no projection; use an empty slice. h.outputCols = make([]uint32, 0) } - h.outputTypes = make([]sqlbase.ColumnType, len(h.outputCols)) + nOutputCols := len(h.outputCols) + if cap(h.outputTypes) >= nOutputCols { + h.outputTypes = h.outputTypes[:nOutputCols] + } else { + h.outputTypes = make([]sqlbase.ColumnType, nOutputCols) + } for i, c := range h.outputCols { h.outputTypes[i] = types[c] } - } else if len(post.RenderExprs) > 0 { - h.renderExprs = make([]exprHelper, len(post.RenderExprs)) - h.outputTypes = make([]sqlbase.ColumnType, len(post.RenderExprs)) + } else if nRenders := len(post.RenderExprs); nRenders > 0 { + if cap(h.renderExprs) >= nRenders { + h.renderExprs = h.renderExprs[:nRenders] + } else { + h.renderExprs = make([]exprHelper, nRenders) + } + if cap(h.outputTypes) >= nRenders { + h.outputTypes = h.outputTypes[:nRenders] + } else { + h.outputTypes = make([]sqlbase.ColumnType, nRenders) + } for i, expr := range post.RenderExprs { + h.renderExprs[i] = exprHelper{} if err := h.renderExprs[i].init(expr, types, evalCtx); err != nil { return err } @@ -539,6 +564,15 @@ type ProcessorBase struct { inputsToDrain []RowSource } +func (pb *ProcessorBase) Reset() { + pb.out.Reset() + *pb = ProcessorBase{ + out: pb.out, + trailingMeta: pb.trailingMeta[:0], + inputsToDrain: pb.inputsToDrain[:0], + } +} + // procState represents the standard states that a processor can be in. These // states are relevant when the processor is using the draining utilities in // ProcessorBase. @@ -1363,3 +1397,20 @@ func (spec WindowerSpec_Frame_Bounds) convertToAST() tree.WindowFrameBounds { func (spec *WindowerSpec_Frame) convertToAST() *tree.WindowFrame { return &tree.WindowFrame{Mode: spec.Mode.convertToAST(), Bounds: spec.Bounds.convertToAST()} } + +var flowSpecPool = sync.Pool{ + New: func() interface{} { + return &FlowSpec{} + }, +} + +func (spec *FlowSpec) Release() { + *spec = FlowSpec{ + Processors: spec.Processors[:0], + } + flowSpecPool.Put(spec) +} + +func NewFlowSpec() *FlowSpec { + return flowSpecPool.Get().(*FlowSpec) +} diff --git a/pkg/sql/distsqlrun/tablereader.go b/pkg/sql/distsqlrun/tablereader.go index 56ba193fd6e8..30835952793a 100644 --- a/pkg/sql/distsqlrun/tablereader.go +++ b/pkg/sql/distsqlrun/tablereader.go @@ -20,6 +20,8 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pkg/errors" + "sync" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util" @@ -50,6 +52,8 @@ var _ RowSource = &tableReader{} const tableReaderProcName = "table reader" +var trPool = sync.Pool{} + // newTableReader creates a tableReader. func newTableReader( flowCtx *FlowCtx, @@ -62,7 +66,13 @@ func newTableReader( return nil, errors.Errorf("attempting to create a tableReader with uninitialized NodeID") } - tr := &tableReader{} + var tr *tableReader + trPtr := trPool.Get() + if trPtr == nil { + tr = &tableReader{} + } else { + tr = trPtr.(*tableReader) + } tr.limitHint = limitHint(spec.LimitHint, post) @@ -106,7 +116,12 @@ func newTableReader( return nil, err } - tr.spans = make(roachpb.Spans, len(spec.Spans)) + nSpans := len(spec.Spans) + if cap(tr.spans) >= nSpans { + tr.spans = tr.spans[:nSpans] + } else { + tr.spans = make(roachpb.Spans, nSpans) + } for i, s := range spec.Spans { tr.spans[i] = s.Span } @@ -236,6 +251,18 @@ func (tr *tableReader) Start(ctx context.Context) context.Context { return ctx } +func (tr *tableReader) Release() { + tr.ProcessorBase.Reset() + tr.fetcher.Reset() + tr.spans = tr.spans[0:] + *tr = tableReader{ + ProcessorBase: tr.ProcessorBase, + fetcher: tr.fetcher, + spans: tr.spans, + } + trPool.Put(tr) +} + // Next is part of the RowSource interface. func (tr *tableReader) Next() (sqlbase.EncDatumRow, *ProducerMetadata) { for tr.State == StateRunning { diff --git a/pkg/sql/sqlbase/rowfetcher.go b/pkg/sql/sqlbase/rowfetcher.go index 7c28291e5f13..5ad93c460877 100644 --- a/pkg/sql/sqlbase/rowfetcher.go +++ b/pkg/sql/sqlbase/rowfetcher.go @@ -235,6 +235,12 @@ type RowFetcher struct { alloc *DatumAlloc } +func (rf *RowFetcher) Reset() { + *rf = RowFetcher{ + tables: rf.tables[:0], + } +} + // Init sets up a RowFetcher for a given table and index. If we are using a // non-primary index, tables.ValNeededForCol can only refer to columns in the // index. @@ -249,28 +255,56 @@ func (rf *RowFetcher) Init( rf.reverse = reverse rf.returnRangeInfo = returnRangeInfo rf.alloc = alloc - rf.allEquivSignatures = make(map[string]int, len(tables)) rf.isCheck = isCheck // We must always decode the index key if we need to distinguish between // rows from more than one table. - rf.mustDecodeIndexKey = len(tables) >= 2 + nTables := len(tables) + multipleTables := nTables >= 2 + rf.mustDecodeIndexKey = multipleTables + if multipleTables { + rf.allEquivSignatures = make(map[string]int, len(tables)) + } - rf.tables = make([]tableInfo, 0, len(tables)) + if cap(rf.tables) >= nTables { + rf.tables = rf.tables[:nTables] + } else { + rf.tables = make([]tableInfo, nTables) + } for tableIdx, tableArgs := range tables { - table := tableInfo{ + table := rf.tables[tableIdx] + nCols := len(tableArgs.Cols) + row := table.row + decodedRow := table.decodedRow + if cap(row) >= nCols { + row = row[:nCols] + } else { + row = make([]EncDatum, nCols) + } + if cap(decodedRow) >= nCols { + decodedRow = decodedRow[:nCols] + } else { + decodedRow = make([]tree.Datum, nCols) + } + table = tableInfo{ spans: tableArgs.Spans, desc: tableArgs.Desc, colIdxMap: tableArgs.ColIdxMap, index: tableArgs.Index, isSecondaryIndex: tableArgs.IsSecondaryIndex, cols: tableArgs.Cols, - row: make([]EncDatum, len(tableArgs.Cols)), - decodedRow: make([]tree.Datum, len(tableArgs.Cols)), + row: row, + decodedRow: decodedRow, + + // These slice fields might get re-allocated below, so reslice them from + // the old table here in case they've got enough capacity already. + indexColIdx: table.indexColIdx[0:], + keyVals: table.keyVals[0:], + extraVals: table.extraVals[0:], } var err error - if len(tables) > 1 { + if multipleTables { // We produce references to every signature's reference. equivSignatures, err := TableEquivSignatures(table.desc, table.index) if err != nil { @@ -316,7 +350,12 @@ func (rf *RowFetcher) Init( table.neededValueColsByIdx = tableArgs.ValNeededForCol.Copy() neededIndexCols := 0 - table.indexColIdx = make([]int, len(indexColumnIDs)) + nIndexCols := len(indexColumnIDs) + if cap(table.indexColIdx) >= nIndexCols { + table.indexColIdx = table.indexColIdx[:nIndexCols] + } else { + table.indexColIdx = make([]int, nIndexCols) + } for i, id := range indexColumnIDs { colIdx, ok := table.colIdxMap[id] if ok { @@ -362,7 +401,11 @@ func (rf *RowFetcher) Init( if err != nil { return err } - table.keyVals = make([]EncDatum, len(indexColumnIDs)) + if cap(table.keyVals) >= nIndexCols { + table.keyVals = table.keyVals[:nIndexCols] + } else { + table.keyVals = make([]EncDatum, nIndexCols) + } if hasExtraCols(&table) { // Unique secondary indexes have a value that is the @@ -371,7 +414,12 @@ func (rf *RowFetcher) Init( // values. If this ever changes, we'll probably have to // figure out the directions here too. table.extraTypes, err = GetColumnTypes(table.desc, table.index.ExtraColumnIDs) - table.extraVals = make([]EncDatum, len(table.index.ExtraColumnIDs)) + nExtraColumns := len(table.index.ExtraColumnIDs) + if cap(table.extraVals) >= nExtraColumns { + table.extraVals = table.extraVals[:nExtraColumns] + } else { + table.extraVals = make([]EncDatum, nExtraColumns) + } if err != nil { return err } @@ -383,7 +431,7 @@ func (rf *RowFetcher) Init( rf.maxKeysPerRow = keysPerRow } - rf.tables = append(rf.tables, table) + rf.tables[tableIdx] = table } if len(tables) == 1 { From 13a50f91e9b11b55e6fd8a012360791c90660f2a Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Fri, 21 Sep 2018 18:14:26 -0400 Subject: [PATCH 4/6] sql: pool distsql receivers Release note: None --- pkg/sql/conn_executor_exec.go | 1 + pkg/sql/distsql_running.go | 20 ++++++++++++++++++-- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 11dddae2bc3c..e8a4ed14f94e 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -954,6 +954,7 @@ func (ex *connExecutor) execWithDistSQLEngine( }, &ex.sessionTracing, ) + defer recv.Release() evalCtx := planner.ExtendedEvalContext() var planCtx *PlanningCtx diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index d421310896ba..7679c89dda85 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -20,6 +20,8 @@ import ( "fmt" + "sync" + "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -372,6 +374,12 @@ func (w *errOnlyResultWriter) IncrementRowsAffected(n int) { var _ distsqlrun.RowReceiver = &DistSQLReceiver{} +var receiverSyncPool = sync.Pool{ + New: func() interface{} { + return &DistSQLReceiver{} + }, +} + // MakeDistSQLReceiver creates a DistSQLReceiver. // // ctx is the Context that the receiver will use throughput its @@ -391,7 +399,8 @@ func MakeDistSQLReceiver( tracing *SessionTracing, ) *DistSQLReceiver { consumeCtx, cleanup := tracing.TraceExecConsume(ctx) - r := &DistSQLReceiver{ + r := receiverSyncPool.Get().(*DistSQLReceiver) + *r = DistSQLReceiver{ ctx: consumeCtx, cleanup: cleanup, resultWriter: resultWriter, @@ -420,10 +429,16 @@ func MakeDistSQLReceiver( return r } +func (r *DistSQLReceiver) Release() { + *r = DistSQLReceiver{} + receiverSyncPool.Put(r) +} + // clone clones the receiver for running subqueries. Not all fields are cloned, // only those required for running subqueries. func (r *DistSQLReceiver) clone() *DistSQLReceiver { - return &DistSQLReceiver{ + ret := receiverSyncPool.Get().(*DistSQLReceiver) + *ret = DistSQLReceiver{ ctx: r.ctx, cleanup: func() {}, rangeCache: r.rangeCache, @@ -433,6 +448,7 @@ func (r *DistSQLReceiver) clone() *DistSQLReceiver { stmtType: tree.Rows, tracing: r.tracing, } + return ret } // SetError provides a convenient way for a client to pass in an error, thus From 377b73ee2207a1959728df5b45f7a02225720626 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Fri, 21 Sep 2018 18:26:02 -0400 Subject: [PATCH 5/6] sql: pass PhysicalPlan by pointer It always gets allocated eventually anyway. An upcoming commit will introduce pooling of these objects as well. Release note: None --- pkg/sql/distsql_physical_planner.go | 146 ++++++++++++++-------------- pkg/sql/distsql_plan_csv.go | 10 +- pkg/sql/distsql_plan_join.go | 22 +++-- pkg/sql/distsql_plan_stats.go | 12 +-- pkg/sql/distsql_running.go | 8 +- pkg/sql/explain_distsql.go | 4 +- pkg/sql/scrub.go | 4 +- 7 files changed, 104 insertions(+), 102 deletions(-) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index d5d7b0bcb3f5..c7ce9cc2f0f0 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1049,7 +1049,7 @@ func (dsp *DistSQLPlanner) CheckNodeHealthAndVersion( // overridesResultColumns is optional. func (dsp *DistSQLPlanner) createTableReaders( planCtx *PlanningCtx, n *scanNode, overrideResultColumns []sqlbase.ColumnID, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { var spanPartitions []SpanPartition if planCtx.isLocal { @@ -1059,7 +1059,7 @@ func (dsp *DistSQLPlanner) createTableReaders( var err error spanPartitions, err = dsp.PartitionSpans(planCtx, n.spans) if err != nil { - return PhysicalPlan{}, err + return nil, err } } else { // If the scan is limited, use a single TableReader to avoid reading more @@ -1069,12 +1069,12 @@ func (dsp *DistSQLPlanner) createTableReaders( // side to modulate table reads. nodeID, err := dsp.getNodeIDForScan(planCtx, n.spans, n.reverse) if err != nil { - return PhysicalPlan{}, err + return nil, err } spanPartitions = []SpanPartition{{nodeID, n.spans}} } - var p PhysicalPlan + p := &PhysicalPlan{} stageID := p.NewStageID() p.ResultRouters = make([]distsqlplan.ProcessorIdx, len(spanPartitions)) @@ -1101,7 +1101,7 @@ func (dsp *DistSQLPlanner) createTableReaders( } spec, post, err := initTableReaderSpec(n, planCtx, scanNodeToTableOrdinalMap) if err != nil { - return PhysicalPlan{}, err + return nil, err } for i, sp := range spanPartitions { @@ -1818,10 +1818,10 @@ func (dsp *DistSQLPlanner) addAggregators( func (dsp *DistSQLPlanner) createPlanForIndexJoin( planCtx *PlanningCtx, n *indexJoinNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { plan, err := dsp.createTableReaders(planCtx, n.index, n.index.desc.PrimaryIndex.ColumnIDs) if err != nil { - return PhysicalPlan{}, err + return nil, err } joinReaderSpec := distsqlrun.JoinReaderSpec{ @@ -1833,7 +1833,7 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( filter, err := distsqlplan.MakeExpression( n.table.filter, planCtx, nil /* indexVarMap */) if err != nil { - return PhysicalPlan{}, err + return nil, err } post := distsqlrun.PostProcessSpec{ Filter: filter, @@ -1854,7 +1854,7 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( types, err := getTypesForPlanResult(n, plan.PlanToStreamColMap) if err != nil { - return PhysicalPlan{}, err + return nil, err } if distributeIndexJoin.Get(&dsp.st.SV) && len(plan.ResultRouters) > 1 { // Instantiate one join reader for every stream. @@ -1886,10 +1886,10 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( // converts joins to lookup joins. func (dsp *DistSQLPlanner) createPlanForLookupJoin( planCtx *PlanningCtx, n *lookupJoinNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { plan, err := dsp.createPlanForNode(planCtx, n.input) if err != nil { - return PhysicalPlan{}, err + return nil, err } joinReaderSpec := distsqlrun.JoinReaderSpec{ @@ -1898,7 +1898,7 @@ func (dsp *DistSQLPlanner) createPlanForLookupJoin( } joinReaderSpec.IndexIdx, err = getIndexIdx(n.table) if err != nil { - return PhysicalPlan{}, err + return nil, err } joinReaderSpec.LookupColumns = make([]uint32, len(n.keyCols)) for i, col := range n.keyCols { @@ -1952,7 +1952,7 @@ func (dsp *DistSQLPlanner) createPlanForLookupJoin( n.onCond, planCtx, indexVarMap, ) if err != nil { - return PhysicalPlan{}, err + return nil, err } } @@ -2005,12 +2005,12 @@ func getTypesForPlanResult(node planNode, planToStreamColMap []int) ([]sqlbase.C func (dsp *DistSQLPlanner) createPlanForJoin( planCtx *PlanningCtx, n *joinNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { // See if we can create an interleave join plan. if planInterleavedJoins.Get(&dsp.st.SV) { plan, ok, err := dsp.tryCreatePlanForInterleavedJoin(planCtx, n) if err != nil { - return PhysicalPlan{}, err + return nil, err } // An interleave join plan could be used. Return it. if ok { @@ -2039,11 +2039,11 @@ func (dsp *DistSQLPlanner) createPlanForJoin( leftPlan, err := dsp.createPlanForNode(planCtx, n.left.plan) if err != nil { - return PhysicalPlan{}, err + return nil, err } rightPlan, err := dsp.createPlanForNode(planCtx, n.right.plan) if err != nil { - return PhysicalPlan{}, err + return nil, err } // Nodes where we will run the join processors. @@ -2080,7 +2080,7 @@ func (dsp *DistSQLPlanner) createPlanForJoin( isLookupJoin = false } - var p PhysicalPlan + p := &PhysicalPlan{} var leftRouters, rightRouters []distsqlplan.ProcessorIdx if isLookupJoin { // Lookup joins only take the left side as input. The right side will @@ -2139,7 +2139,7 @@ func (dsp *DistSQLPlanner) createPlanForJoin( post, joinToStreamColMap := joinOutColumns(n, leftPlan.PlanToStreamColMap, rightMap) onExpr, err := remapOnExpr(planCtx, n, leftPlan.PlanToStreamColMap, rightMap) if err != nil { - return PhysicalPlan{}, err + return nil, err } // Create the Core spec. @@ -2164,11 +2164,11 @@ func (dsp *DistSQLPlanner) createPlanForJoin( // [1, 2]. indexColumns, err := applySortBasedOnFirst(rightEqCols, indexColumns) if err != nil { - return PhysicalPlan{}, err + return nil, err } lookupCols, err := applySortBasedOnFirst(indexColumns, leftEqCols) if err != nil { - return PhysicalPlan{}, err + return nil, err } // If the right side scan has a filter, specify this as IndexFilterExpr so @@ -2178,13 +2178,13 @@ func (dsp *DistSQLPlanner) createPlanForJoin( var err error indexFilterExpr, err = distsqlplan.MakeExpression(lookupJoinScan.origFilter, planCtx, nil) if err != nil { - return PhysicalPlan{}, err + return nil, err } } indexIdx, err := getIndexIdx(lookupJoinScan) if err != nil { - return PhysicalPlan{}, err + return nil, err } core.JoinReader = &distsqlrun.JoinReaderSpec{ Table: *(lookupJoinScan.desc), @@ -2218,7 +2218,7 @@ func (dsp *DistSQLPlanner) createPlanForJoin( p.PlanToStreamColMap = joinToStreamColMap p.ResultTypes, err = getTypesForPlanResult(n, joinToStreamColMap) if err != nil { - return PhysicalPlan{}, err + return nil, err } // Joiners may guarantee an ordering to outputs, so we ensure that @@ -2305,7 +2305,7 @@ func applySortBasedOnFirst(source, target []uint32) ([]uint32, error) { func (dsp *DistSQLPlanner) createPlanForNode( planCtx *PlanningCtx, node planNode, -) (plan PhysicalPlan, err error) { +) (plan *PhysicalPlan, err error) { planCtx.planDepth++ switch n := node.(type) { @@ -2324,51 +2324,51 @@ func (dsp *DistSQLPlanner) createPlanForNode( case *renderNode: plan, err = dsp.createPlanForNode(planCtx, n.source.plan) if err != nil { - return PhysicalPlan{}, err + return nil, err } - err = dsp.selectRenders(&plan, n, planCtx) + err = dsp.selectRenders(plan, n, planCtx) if err != nil { - return PhysicalPlan{}, err + return nil, err } case *groupNode: plan, err = dsp.createPlanForNode(planCtx, n.plan) if err != nil { - return PhysicalPlan{}, err + return nil, err } - if err := dsp.addAggregators(planCtx, &plan, n); err != nil { - return PhysicalPlan{}, err + if err := dsp.addAggregators(planCtx, plan, n); err != nil { + return nil, err } case *sortNode: plan, err = dsp.createPlanForNode(planCtx, n.plan) if err != nil { - return PhysicalPlan{}, err + return nil, err } - dsp.addSorters(&plan, n) + dsp.addSorters(plan, n) case *filterNode: plan, err = dsp.createPlanForNode(planCtx, n.source.plan) if err != nil { - return PhysicalPlan{}, err + return nil, err } if err := plan.AddFilter(n.filter, planCtx, plan.PlanToStreamColMap); err != nil { - return PhysicalPlan{}, err + return nil, err } case *limitNode: plan, err = dsp.createPlanForNode(planCtx, n.plan) if err != nil { - return PhysicalPlan{}, err + return nil, err } if err := n.evalLimit(planCtx.EvalContext()); err != nil { - return PhysicalPlan{}, err + return nil, err } if err := plan.AddLimit(n.count, n.offset, planCtx, dsp.nodeDesc.NodeID); err != nil { - return PhysicalPlan{}, err + return nil, err } case *distinctNode: @@ -2441,7 +2441,7 @@ func (dsp *DistSQLPlanner) createPlanForNode( // will create a planNodeToRowSource wrapper for the sub-tree that's not // plannable by DistSQL. If that sub-tree has DistSQL-plannable sources, they // will be planned by DistSQL and connected to the wrapper. -func (dsp *DistSQLPlanner) wrapPlan(planCtx *PlanningCtx, n planNode) (PhysicalPlan, error) { +func (dsp *DistSQLPlanner) wrapPlan(planCtx *PlanningCtx, n planNode) (*PhysicalPlan, error) { useFastPath := planCtx.planDepth == 1 && planCtx.stmtType == tree.RowsAffected // First, we search the planNode tree we're trying to wrap for the first @@ -2449,7 +2449,7 @@ func (dsp *DistSQLPlanner) wrapPlan(planCtx *PlanningCtx, n planNode) (PhysicalP // continue the DistSQL planning recursion on that planNode. seenTop := false nParents := uint32(0) - var p PhysicalPlan + p := &PhysicalPlan{} // This will be set to first DistSQL-enabled planNode we find, if any. We'll // modify its parent later to connect its source to the DistSQL-planned // subtree. @@ -2489,10 +2489,10 @@ func (dsp *DistSQLPlanner) wrapPlan(planCtx *PlanningCtx, n planNode) (PhysicalP return true, nil }, }); err != nil { - return PhysicalPlan{}, err + return nil, err } if nParents > 1 { - return PhysicalPlan{}, errors.Errorf("can't wrap plan %v %T with more than one input", n, n) + return nil, errors.Errorf("can't wrap plan %v %T with more than one input", n, n) } // Copy the evalCtx. @@ -2510,7 +2510,7 @@ func (dsp *DistSQLPlanner) wrapPlan(planCtx *PlanningCtx, n planNode) (PhysicalP useFastPath, ) if err != nil { - return PhysicalPlan{}, err + return nil, err } wrapper.firstNotWrapped = firstNotWrapped @@ -2568,7 +2568,7 @@ func (dsp *DistSQLPlanner) wrapPlan(planCtx *PlanningCtx, n planNode) (PhysicalP // and rawBytes that need to be precomputed beforehand. func (dsp *DistSQLPlanner) createValuesPlan( resultTypes []sqlbase.ColumnType, numRows int, rawBytes [][]byte, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { numColumns := len(resultTypes) s := distsqlrun.ValuesCoreSpec{ Columns: make([]distsqlrun.DatumInfo, numColumns), @@ -2595,7 +2595,7 @@ func (dsp *DistSQLPlanner) createValuesPlan( ResultTypes: resultTypes, } - return PhysicalPlan{ + return &PhysicalPlan{ PhysicalPlan: plan, PlanToStreamColMap: identityMapInPlace(make([]int, numColumns)), }, nil @@ -2603,7 +2603,7 @@ func (dsp *DistSQLPlanner) createValuesPlan( func (dsp *DistSQLPlanner) createPlanForValues( planCtx *PlanningCtx, n *valuesNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { params := runParams{ ctx: planCtx.ctx, extendedEvalCtx: planCtx.ExtendedEvalCtx, @@ -2611,11 +2611,11 @@ func (dsp *DistSQLPlanner) createPlanForValues( types, err := getTypesForPlanResult(n, nil /* planToStreamColMap */) if err != nil { - return PhysicalPlan{}, err + return nil, err } if err := n.startExec(params); err != nil { - return PhysicalPlan{}, err + return nil, err } defer n.Close(planCtx.ctx) @@ -2625,7 +2625,7 @@ func (dsp *DistSQLPlanner) createPlanForValues( rawBytes := make([][]byte, numRows) for i := 0; i < numRows; i++ { if next, err := n.Next(runParams{ctx: planCtx.ctx}); !next { - return PhysicalPlan{}, err + return nil, err } var buf []byte @@ -2635,7 +2635,7 @@ func (dsp *DistSQLPlanner) createPlanForValues( datum := sqlbase.DatumToEncDatum(types[j], datums[j]) buf, err = datum.Encode(&types[j], &a, sqlbase.DatumEncoding_VALUE, buf) if err != nil { - return PhysicalPlan{}, err + return nil, err } } rawBytes[i] = buf @@ -2646,10 +2646,10 @@ func (dsp *DistSQLPlanner) createPlanForValues( func (dsp *DistSQLPlanner) createPlanForUnary( planCtx *PlanningCtx, n *unaryNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { types, err := getTypesForPlanResult(n, nil /* planToStreamColMap */) if err != nil { - return PhysicalPlan{}, err + return nil, err } return dsp.createValuesPlan(types, 1 /* numRows */, nil /* rawBytes */) @@ -2657,10 +2657,10 @@ func (dsp *DistSQLPlanner) createPlanForUnary( func (dsp *DistSQLPlanner) createPlanForZero( planCtx *PlanningCtx, n *zeroNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { types, err := getTypesForPlanResult(n, nil /* planToStreamColMap */) if err != nil { - return PhysicalPlan{}, err + return nil, err } return dsp.createValuesPlan(types, 0 /* numRows */, nil /* rawBytes */) @@ -2699,10 +2699,10 @@ func createDistinctSpec(n *distinctNode, cols []int) *distsqlrun.DistinctSpec { func (dsp *DistSQLPlanner) createPlanForDistinct( planCtx *PlanningCtx, n *distinctNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { plan, err := dsp.createPlanForNode(planCtx, n.plan) if err != nil { - return PhysicalPlan{}, err + return nil, err } currentResultRouters := plan.ResultRouters @@ -2756,10 +2756,10 @@ func createProjectSetSpec( func (dsp *DistSQLPlanner) createPlanForProjectSet( planCtx *PlanningCtx, n *projectSetNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { plan, err := dsp.createPlanForNode(planCtx, n.source) if err != nil { - return PhysicalPlan{}, err + return nil, err } numResults := len(plan.ResultTypes) @@ -2769,7 +2769,7 @@ func (dsp *DistSQLPlanner) createPlanForProjectSet( // Create the project set processor spec. projectSetSpec, err := createProjectSetSpec(planCtx, n, indexVarMap) if err != nil { - return PhysicalPlan{}, err + return nil, err } spec := distsqlrun.ProcessorCoreUnion{ ProjectSet: projectSetSpec, @@ -2844,22 +2844,22 @@ func (dsp *DistSQLPlanner) isOnlyOnGateway(plan *PhysicalPlan) bool { // JOIN func (dsp *DistSQLPlanner) createPlanForSetOp( planCtx *PlanningCtx, n *unionNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { leftLogicalPlan := n.left leftPlan, err := dsp.createPlanForNode(planCtx, n.left) if err != nil { - return PhysicalPlan{}, err + return nil, err } rightLogicalPlan := n.right rightPlan, err := dsp.createPlanForNode(planCtx, n.right) if err != nil { - return PhysicalPlan{}, err + return nil, err } if n.inverted { leftPlan, rightPlan = rightPlan, leftPlan leftLogicalPlan, rightLogicalPlan = rightLogicalPlan, leftLogicalPlan } - childPhysicalPlans := []*PhysicalPlan{&leftPlan, &rightPlan} + childPhysicalPlans := []*PhysicalPlan{leftPlan, rightPlan} childLogicalPlans := []planNode{leftLogicalPlan, rightLogicalPlan} // Check that the left and right side PlanToStreamColMaps are equivalent. @@ -2868,7 +2868,7 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( // the unioned columns on each side, similar to how we handle mismatched // ResultTypes. if !reflect.DeepEqual(leftPlan.PlanToStreamColMap, rightPlan.PlanToStreamColMap) { - return PhysicalPlan{}, errors.Errorf( + return nil, errors.Errorf( "planToStreamColMap mismatch: %v, %v", leftPlan.PlanToStreamColMap, rightPlan.PlanToStreamColMap) } @@ -2924,7 +2924,7 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( } } - var p PhysicalPlan + p := &PhysicalPlan{} // Merge the plans' PlanToStreamColMap, which we know are equivalent. p.PlanToStreamColMap = planToStreamColMap @@ -2957,12 +2957,12 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( childLogicalPlans[side], plan.PlanToStreamColMap, ) if err != nil { - return PhysicalPlan{}, err + return nil, err } } resultTypes, err = distsqlplan.MergeResultTypes(childResultTypes[0], childResultTypes[1]) if err != nil { - return PhysicalPlan{}, err + return nil, err } } else if err != nil || !mergeOrdering.Equal(rightPlan.MergeOrdering) { // The result types or merge ordering can differ between the two sides in @@ -2981,7 +2981,7 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( // Result types should now be mergeable. resultTypes, err = distsqlplan.MergeResultTypes(leftPlan.ResultTypes, rightPlan.ResultTypes) if err != nil { - return PhysicalPlan{}, err + return nil, err } mergeOrdering = distsqlrun.Ordering{} } @@ -3095,14 +3095,14 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( // scheme found in the query's window functions. func (dsp *DistSQLPlanner) createPlanForWindow( planCtx *PlanningCtx, n *windowNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { plan, err := dsp.createPlanForNode(planCtx, n.plan) if err != nil { - return PhysicalPlan{}, err + return nil, err } numWindowFuncProcessed := 0 - windowPlanState := createWindowPlanState(n, planCtx, &plan) + windowPlanState := createWindowPlanState(n, planCtx, plan) // Each iteration of this loop adds a new stage of windowers. The steps taken: // 1. find a set of unprocessed window functions that have the same PARTITION BY // clause. All of these will be computed using the single stage of windowers. @@ -3159,7 +3159,7 @@ func (dsp *DistSQLPlanner) createPlanForWindow( windowFnSpec, outputType, err := windowPlanState.createWindowFnSpec(windowFn) if err != nil { - return PhysicalPlan{}, err + return nil, err } // Windower processor does not pass through ("consumes") all arguments of @@ -3267,7 +3267,7 @@ func (dsp *DistSQLPlanner) createPlanForWindow( // After all window functions are computed, we might need to add rendering. if err := windowPlanState.addRenderingIfNecessary(); err != nil { - return PhysicalPlan{}, err + return nil, err } return plan, nil diff --git a/pkg/sql/distsql_plan_csv.go b/pkg/sql/distsql_plan_csv.go index a56bc4157b3e..72a6cc7c34f8 100644 --- a/pkg/sql/distsql_plan_csv.go +++ b/pkg/sql/distsql_plan_csv.go @@ -78,7 +78,7 @@ func PlanAndRunExport( // columns filename/rows/bytes. p.PlanToStreamColMap = identityMap(p.PlanToStreamColMap, len(ExportPlanResultTypes)) - dsp.FinalizePlan(planCtx, &p) + dsp.FinalizePlan(planCtx, p) recv := MakeDistSQLReceiver( ctx, resultRows, tree.Rows, @@ -86,7 +86,7 @@ func PlanAndRunExport( evalCtx.Tracing, ) - dsp.Run(planCtx, txn, &p, recv, evalCtx, nil /* finishedSetupFn */) + dsp.Run(planCtx, txn, p, recv, evalCtx, nil /* finishedSetupFn */) return resultRows.Err() } @@ -360,7 +360,7 @@ func LoadCSV( // We have the split ranges. Now re-read the CSV files and route them to SST writers. - p := PhysicalPlan{} + p := &PhysicalPlan{} // This is a hardcoded two stage plan. The first stage is the mappers, // the second stage is the reducers. We have to keep track of all the mappers // we create because the reducers need to hook up a stream for each mapper. @@ -462,7 +462,7 @@ func LoadCSV( return err } - dsp.FinalizePlan(planCtx, &p) + dsp.FinalizePlan(planCtx, p) recv := MakeDistSQLReceiver( ctx, @@ -477,7 +477,7 @@ func LoadCSV( defer log.VEventf(ctx, 1, "finished job %s", job.Payload().Description) return db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { - dsp.Run(planCtx, txn, &p, recv, evalCtx, nil /* finishedSetupFn */) + dsp.Run(planCtx, txn, p, recv, evalCtx, nil /* finishedSetupFn */) return resultRows.Err() }) } diff --git a/pkg/sql/distsql_plan_join.go b/pkg/sql/distsql_plan_join.go index a96fd212d07b..f69d795bb7e5 100644 --- a/pkg/sql/distsql_plan_join.go +++ b/pkg/sql/distsql_plan_join.go @@ -38,9 +38,9 @@ var planInterleavedJoins = settings.RegisterBoolSetting( func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( planCtx *PlanningCtx, n *joinNode, -) (plan PhysicalPlan, ok bool, err error) { +) (plan *PhysicalPlan, ok bool, err error) { if !useInterleavedJoin(n) { - return PhysicalPlan{}, false, nil + return nil, false, nil } leftScan, leftOk := n.left.plan.(*scanNode) @@ -49,7 +49,7 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( // We know they are scan nodes from useInterleaveJoin, but we add // this check to prevent future panics. if !leftOk || !rightOk { - return PhysicalPlan{}, false, pgerror.NewErrorf( + return nil, false, pgerror.NewErrorf( pgerror.CodeInternalError, "left and right children of join node must be scan nodes to execute an interleaved join", ) @@ -58,7 +58,7 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( // We iterate through each table and collate their metadata for // the InterleavedReaderJoinerSpec. tables := make([]distsqlrun.InterleavedReaderJoinerSpec_Table, 2) - plans := make([]PhysicalPlan, 2) + plans := make([]*PhysicalPlan, 2) var totalLimitHint int64 for i, t := range []struct { scan *scanNode @@ -79,7 +79,7 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( // onCond and columns. var err error if plans[i], err = dsp.createTableReaders(planCtx, t.scan, nil); err != nil { - return PhysicalPlan{}, false, err + return nil, false, err } eqCols := eqCols(t.eqIndices, plans[i].PlanToStreamColMap) @@ -114,7 +114,7 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( post, joinToStreamColMap := joinOutColumns(n, plans[0].PlanToStreamColMap, plans[1].PlanToStreamColMap) onExpr, err := remapOnExpr(planCtx, n, plans[0].PlanToStreamColMap, plans[1].PlanToStreamColMap) if err != nil { - return PhysicalPlan{}, false, err + return nil, false, err } ancestor, descendant := n.interleavedNodes() @@ -122,11 +122,11 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( // We partition each set of spans to their respective nodes. ancsPartitions, err := dsp.PartitionSpans(planCtx, ancestor.spans) if err != nil { - return PhysicalPlan{}, false, err + return nil, false, err } descPartitions, err := dsp.PartitionSpans(planCtx, descendant.spans) if err != nil { - return PhysicalPlan{}, false, err + return nil, false, err } // We want to ensure that all child spans with a given interleave @@ -146,7 +146,7 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( // partitioned to node 2 and 3, then we need to move the child spans // to node 1 where the PK1 = 1 parent row is read. if descPartitions, err = alignInterleavedSpans(n, ancsPartitions, descPartitions); err != nil { - return PhysicalPlan{}, false, err + return nil, false, err } // Figure out which nodes we need to schedule a processor on. @@ -169,6 +169,8 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( ancsIdx, descIdx = 1, 0 } + plan = &PhysicalPlan{} + stageID := plan.NewStageID() // We provision a separate InterleavedReaderJoiner per node that has @@ -234,7 +236,7 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( plan.PlanToStreamColMap = joinToStreamColMap plan.ResultTypes, err = getTypesForPlanResult(n, joinToStreamColMap) if err != nil { - return PhysicalPlan{}, false, err + return nil, false, err } plan.SetMergeOrdering(dsp.convertOrdering(n.props, plan.PlanToStreamColMap)) diff --git a/pkg/sql/distsql_plan_stats.go b/pkg/sql/distsql_plan_stats.go index a70ed9d521af..4a26bb3c46a8 100644 --- a/pkg/sql/distsql_plan_stats.go +++ b/pkg/sql/distsql_plan_stats.go @@ -33,16 +33,16 @@ const histogramBuckets = 200 func (dsp *DistSQLPlanner) createStatsPlan( planCtx *PlanningCtx, desc *sqlbase.TableDescriptor, stats []requestedStat, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { // Create the table readers; for this we initialize a dummy scanNode. scan := scanNode{desc: desc} err := scan.initDescDefaults(nil /* planDependencies */, publicColumnsCfg) if err != nil { - return PhysicalPlan{}, err + return nil, err } scan.spans, err = unconstrainedSpans(desc, scan.index) if err != nil { - return PhysicalPlan{}, err + return nil, err } // Calculate the relevant columns. @@ -52,7 +52,7 @@ func (dsp *DistSQLPlanner) createStatsPlan( for _, c := range s.columns { colIdx, ok := scan.colIdxMap[c] if !ok { - return PhysicalPlan{}, errors.Errorf("unknown column ID %d", c) + return nil, errors.Errorf("unknown column ID %d", c) } if !scan.valNeededForCol.Contains(colIdx) { scan.valNeededForCol.Add(colIdx) @@ -63,7 +63,7 @@ func (dsp *DistSQLPlanner) createStatsPlan( p, err := dsp.createTableReaders(planCtx, &scan, nil /* overrideResultColumns */) if err != nil { - return PhysicalPlan{}, err + return nil, err } sketchSpecs := make([]distsqlrun.SketchSpec, len(stats)) @@ -157,7 +157,7 @@ func (dsp *DistSQLPlanner) createStatsPlan( func (dsp *DistSQLPlanner) createPlanForCreateStats( planCtx *PlanningCtx, n *createStatsNode, -) (PhysicalPlan, error) { +) (*PhysicalPlan, error) { stats := []requestedStat{ { diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 7679c89dda85..dc3afc04e503 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -661,7 +661,7 @@ func (dsp *DistSQLPlanner) PlanAndRunSubqueries( recv.SetError(err) return false } - dsp.FinalizePlan(subqueryPlanCtx, &subqueryPhysPlan) + dsp.FinalizePlan(subqueryPlanCtx, subqueryPhysPlan) // TODO(arjun): #28264: We set up a row container, wrap it in a row // receiver, and use it and serialize the results of the subquery. The type @@ -679,7 +679,7 @@ func (dsp *DistSQLPlanner) PlanAndRunSubqueries( subqueryRowReceiver := NewRowResultWriter(rows) subqueryRecv.resultWriter = subqueryRowReceiver subqueryPlans[planIdx].started = true - dsp.Run(subqueryPlanCtx, planner.txn, &subqueryPhysPlan, subqueryRecv, evalCtx, nil /* finishedSetupFn */) + dsp.Run(subqueryPlanCtx, planner.txn, subqueryPhysPlan, subqueryRecv, evalCtx, nil /* finishedSetupFn */) if subqueryRecv.commErr != nil { recv.SetError(subqueryRecv.commErr) return false @@ -761,6 +761,6 @@ func (dsp *DistSQLPlanner) PlanAndRun( recv.SetError(err) return } - dsp.FinalizePlan(planCtx, &physPlan) - dsp.Run(planCtx, txn, &physPlan, recv, evalCtx, nil /* finishedSetupFn */) + dsp.FinalizePlan(planCtx, physPlan) + dsp.Run(planCtx, txn, physPlan, recv, evalCtx, nil /* finishedSetupFn */) } diff --git a/pkg/sql/explain_distsql.go b/pkg/sql/explain_distsql.go index fbf37e8c65ec..ae0eb20d4fa1 100644 --- a/pkg/sql/explain_distsql.go +++ b/pkg/sql/explain_distsql.go @@ -73,7 +73,7 @@ func (n *explainDistSQLNode) startExec(params runParams) error { if err != nil { return err } - distSQLPlanner.FinalizePlan(planCtx, &plan) + distSQLPlanner.FinalizePlan(planCtx, plan) var spans []tracing.RecordedSpan if n.analyze { @@ -118,7 +118,7 @@ func (n *explainDistSQLNode) startExec(params runParams) error { params.extendedEvalCtx.Tracing, ) distSQLPlanner.Run( - planCtx, params.p.txn, &plan, recv, params.extendedEvalCtx, nil /* finishedSetupFn */) + planCtx, params.p.txn, plan, recv, params.extendedEvalCtx, nil /* finishedSetupFn */) n.run.executedStatement = true diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 2302a04fe948..c5c0c48b6184 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -533,8 +533,8 @@ func scrubPlanDistSQL( if err != nil { return nil, err } - planCtx.ExtendedEvalCtx.DistSQLPlanner.FinalizePlan(planCtx, &physPlan) - return &physPlan, nil + planCtx.ExtendedEvalCtx.DistSQLPlanner.FinalizePlan(planCtx, physPlan) + return physPlan, nil } // scrubRunDistSQL run a distSQLPhysicalPlan plan in distSQL. If From fe5a9e8de92499ba54ab050cf1e743c087da485e Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Sun, 23 Sep 2018 20:39:23 -0400 Subject: [PATCH 6/6] sql: pool PhysicalPlan objects Release note: None --- pkg/sql/distsql_physical_planner.go | 96 ++++++++++++++++++++--------- pkg/sql/distsql_plan_csv.go | 2 +- pkg/sql/distsql_plan_join.go | 2 +- pkg/sql/distsql_running.go | 1 + 4 files changed, 71 insertions(+), 30 deletions(-) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index c7ce9cc2f0f0..6e967a403263 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -21,6 +21,8 @@ import ( "sort" "strings" + "sync" + "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/keys" @@ -559,6 +561,18 @@ type PhysicalPlan struct { PlanToStreamColMap []int } +func (p *PhysicalPlan) Release() { + *p = PhysicalPlan{ + PhysicalPlan: distsqlplan.PhysicalPlan{ + Processors: p.Processors[:0], + ResultRouters: p.ResultRouters[:0], + ResultTypes: p.ResultTypes[:0], + }, + PlanToStreamColMap: p.PlanToStreamColMap[:0], + } + physicalPlanPool.Put(p) +} + // makePlanToStreamColMap initializes a new PhysicalPlan.PlanToStreamColMap. The // columns that are present in the result stream(s) should be set in the map. func makePlanToStreamColMap(numCols int) []int { @@ -857,6 +871,8 @@ func initTableReaderSpec( Reverse: n.reverse, IsCheck: n.run.isCheck, Visibility: n.colCfg.visibility.toDistSQLScanVisibility(), + + Spans: s.Spans[:0], } indexIdx, err := getIndexIdx(n) if err != nil { @@ -1074,11 +1090,15 @@ func (dsp *DistSQLPlanner) createTableReaders( spanPartitions = []SpanPartition{{nodeID, n.spans}} } - p := &PhysicalPlan{} + p := newPhysicalPlan() stageID := p.NewStageID() - p.ResultRouters = make([]distsqlplan.ProcessorIdx, len(spanPartitions)) - p.Processors = make([]distsqlplan.Processor, 0, len(spanPartitions)) + nPartitions := len(spanPartitions) + if cap(p.ResultRouters) >= nPartitions { + p.ResultRouters = p.ResultRouters[:nPartitions] + } else { + p.ResultRouters = make([]distsqlplan.ProcessorIdx, nPartitions) + } returnMutations := n.colCfg.visibility == publicAndNonPublicColumns @@ -1112,7 +1132,9 @@ func (dsp *DistSQLPlanner) createTableReaders( tr = new(distsqlrun.TableReaderSpec) *tr = *spec } - tr.Spans = makeTableReaderSpans(sp.Spans) + for j := range sp.Spans { + tr.Spans = append(tr.Spans, distsqlrun.TableReaderSpan{Span: sp.Spans[j]}) + } proc := distsqlplan.Processor{ Node: sp.Node, @@ -1127,11 +1149,15 @@ func (dsp *DistSQLPlanner) createTableReaders( p.ResultRouters[i] = pIdx } - var types []sqlbase.ColumnType + var nTypes int if returnMutations { - types = make([]sqlbase.ColumnType, 0, len(n.desc.Columns)+len(n.desc.Mutations)) + nTypes = len(n.desc.Columns) + len(n.desc.Mutations) } else { - types = make([]sqlbase.ColumnType, 0, len(n.desc.Columns)) + nTypes = len(n.desc.Columns) + } + types := p.ResultTypes[:0] + if cap(types) < nTypes { + types = make([]sqlbase.ColumnType, 0, nTypes) } for i := range n.desc.Columns { types = append(types, n.desc.Columns[i].Type) @@ -1155,7 +1181,13 @@ func (dsp *DistSQLPlanner) createTableReaders( outCols[i] = uint32(tableOrdinal(n.desc, id, n.colCfg.visibility)) } } - planToStreamColMap := make([]int, len(n.cols)) + + nCols := len(n.cols) + if cap(p.PlanToStreamColMap) >= nCols { + p.PlanToStreamColMap = p.PlanToStreamColMap[:nCols] + } else { + p.PlanToStreamColMap = make([]int, nCols) + } descColumnIDs := make([]sqlbase.ColumnID, 0, len(n.desc.Columns)) for i := range n.desc.Columns { descColumnIDs = append(descColumnIDs, n.desc.Columns[i].ID) @@ -1168,18 +1200,16 @@ func (dsp *DistSQLPlanner) createTableReaders( } } } - for i := range planToStreamColMap { - planToStreamColMap[i] = -1 + for i := range p.PlanToStreamColMap { + p.PlanToStreamColMap[i] = -1 for j, c := range outCols { if descColumnIDs[c] == n.cols[i].ID { - planToStreamColMap[i] = j + p.PlanToStreamColMap[i] = j break } } } p.AddProjection(outCols) - - p.PlanToStreamColMap = planToStreamColMap return p, nil } @@ -2003,6 +2033,16 @@ func getTypesForPlanResult(node planNode, planToStreamColMap []int) ([]sqlbase.C return types, nil } +var physicalPlanPool = sync.Pool{ + New: func() interface{} { + return new(PhysicalPlan) + }, +} + +func newPhysicalPlan() *PhysicalPlan { + return physicalPlanPool.Get().(*PhysicalPlan) +} + func (dsp *DistSQLPlanner) createPlanForJoin( planCtx *PlanningCtx, n *joinNode, ) (*PhysicalPlan, error) { @@ -2080,7 +2120,7 @@ func (dsp *DistSQLPlanner) createPlanForJoin( isLookupJoin = false } - p := &PhysicalPlan{} + p := newPhysicalPlan() var leftRouters, rightRouters []distsqlplan.ProcessorIdx if isLookupJoin { // Lookup joins only take the left side as input. The right side will @@ -2449,7 +2489,7 @@ func (dsp *DistSQLPlanner) wrapPlan(planCtx *PlanningCtx, n planNode) (*Physical // continue the DistSQL planning recursion on that planNode. seenTop := false nParents := uint32(0) - p := &PhysicalPlan{} + p := newPhysicalPlan() // This will be set to first DistSQL-enabled planNode we find, if any. We'll // modify its parent later to connect its source to the DistSQL-planned // subtree. @@ -2583,22 +2623,22 @@ func (dsp *DistSQLPlanner) createValuesPlan( s.RawBytes = rawBytes plan := distsqlplan.PhysicalPlan{ - Processors: []distsqlplan.Processor{{ - // TODO: find a better node to place processor at - Node: dsp.nodeDesc.NodeID, - Spec: distsqlrun.ProcessorSpec{ - Core: distsqlrun.ProcessorCoreUnion{Values: &s}, - Output: []distsqlrun.OutputRouterSpec{{Type: distsqlrun.OutputRouterSpec_PASS_THROUGH}}, - }, - }}, ResultRouters: []distsqlplan.ProcessorIdx{0}, ResultTypes: resultTypes, } + plan.AddProcessor(distsqlplan.Processor{ + // TODO: find a better node to place processor at + Node: dsp.nodeDesc.NodeID, + Spec: distsqlrun.ProcessorSpec{ + Core: distsqlrun.ProcessorCoreUnion{Values: &s}, + Output: []distsqlrun.OutputRouterSpec{{Type: distsqlrun.OutputRouterSpec_PASS_THROUGH}}, + }, + }) - return &PhysicalPlan{ - PhysicalPlan: plan, - PlanToStreamColMap: identityMapInPlace(make([]int, numColumns)), - }, nil + p := newPhysicalPlan() + p.PhysicalPlan = plan + p.PlanToStreamColMap = identityMapInPlace(make([]int, numColumns)) + return p, nil } func (dsp *DistSQLPlanner) createPlanForValues( @@ -2924,7 +2964,7 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( } } - p := &PhysicalPlan{} + p := newPhysicalPlan() // Merge the plans' PlanToStreamColMap, which we know are equivalent. p.PlanToStreamColMap = planToStreamColMap diff --git a/pkg/sql/distsql_plan_csv.go b/pkg/sql/distsql_plan_csv.go index 72a6cc7c34f8..daa6c74054ee 100644 --- a/pkg/sql/distsql_plan_csv.go +++ b/pkg/sql/distsql_plan_csv.go @@ -360,7 +360,7 @@ func LoadCSV( // We have the split ranges. Now re-read the CSV files and route them to SST writers. - p := &PhysicalPlan{} + p := newPhysicalPlan() // This is a hardcoded two stage plan. The first stage is the mappers, // the second stage is the reducers. We have to keep track of all the mappers // we create because the reducers need to hook up a stream for each mapper. diff --git a/pkg/sql/distsql_plan_join.go b/pkg/sql/distsql_plan_join.go index f69d795bb7e5..4dfd98fdd001 100644 --- a/pkg/sql/distsql_plan_join.go +++ b/pkg/sql/distsql_plan_join.go @@ -169,7 +169,7 @@ func (dsp *DistSQLPlanner) tryCreatePlanForInterleavedJoin( ancsIdx, descIdx = 1, 0 } - plan = &PhysicalPlan{} + plan = newPhysicalPlan() stageID := plan.NewStageID() diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index dc3afc04e503..0e7a082d7069 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -265,6 +265,7 @@ func (dsp *DistSQLPlanner) Run( planCtx.planner.curPlan.close(ctx) } + plan.Release() flow.Wait() flow.Cleanup(ctx) }