From 1665a1bdd3adef843df8773f047a2989510e1537 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Wed, 1 Aug 2018 10:59:34 -0400 Subject: [PATCH] sql: merge the execution engines with wrapPlan This commit merges the DistSQL and local SQL execution engines. The strategy is thus: The method to check whether DistSQL supports a planNode tree is unchanged, but the results are interpreted differently. A result that says "unsupported" is interpreted as "keep this plan on the gateway node" rather than "fall back to the local execution engine". If a planNode is unsupported by DistSQL (doesn't have a corresponding DistSQL Processor implementation), instead of giving up, the DistSQL physical planner emits a special wrapper processor (the `planNodeToRowSource` processor) that simply converts the output of the planNode into a form amenable for further consumption by DistSQL processors. The wrapped planNode isn't serialized to a processor spec - instead, since we guarantee that a plan with a wrapped planNode will be executed on the gateway node, we serialize the index of the local planNode into a slice of localPlanNodes that's passed to the execution engine in a side channel, and not serialized to a proto. When planning on the gateway, we no longer serialize the EvalContext to a proto just to deserialize it to an EvalContext again. This is both a performance optimization and a correctness fix. The expressions that didn't work in DistSQL mode didn't work because the serialized EvalContext isn't complete - we get around that by passing the full EvalContext directly to gateway flows when possible. Release note (sql change): all queries now run through the DistSQL execution engine. --- pkg/acceptance/testdata/c/test.c | 9 +- pkg/ccl/importccl/exportcsv.go | 4 - pkg/ccl/importccl/exportcsv_test.go | 16 - .../test_sql_mem_monitor.tcl | 1 + pkg/sql/conn_executor_exec.go | 56 +- pkg/sql/distinct.go | 17 +- pkg/sql/distsql_physical_planner.go | 212 ++-- pkg/sql/distsql_running.go | 47 +- pkg/sql/distsqlplan/physical_plan.go | 60 +- pkg/sql/distsqlrun/aggregator.go | 4 +- pkg/sql/distsqlrun/aggregator_test.go | 8 +- pkg/sql/distsqlrun/api.go | 10 +- pkg/sql/distsqlrun/api.pb.go | 1 + pkg/sql/distsqlrun/distinct_test.go | 4 +- pkg/sql/distsqlrun/flow.go | 52 +- pkg/sql/distsqlrun/flow_diagram.go | 5 + pkg/sql/distsqlrun/hashjoiner_test.go | 10 +- .../distsqlrun/interleaved_reader_joiner.go | 4 +- .../interleaved_reader_joiner_test.go | 6 +- pkg/sql/distsqlrun/joinreader_test.go | 8 +- pkg/sql/distsqlrun/mergejoiner_test.go | 6 +- pkg/sql/distsqlrun/noop_test.go | 2 +- pkg/sql/distsqlrun/outbox_test.go | 12 +- pkg/sql/distsqlrun/processors.go | 28 + pkg/sql/distsqlrun/processors.pb.go | 925 +++++++++++------- pkg/sql/distsqlrun/processors.proto | 12 + pkg/sql/distsqlrun/processors_test.go | 3 +- pkg/sql/distsqlrun/routers_test.go | 9 +- pkg/sql/distsqlrun/sample_aggregator_test.go | 2 +- pkg/sql/distsqlrun/sampler_test.go | 4 +- pkg/sql/distsqlrun/server.go | 185 ++-- pkg/sql/distsqlrun/sorter_test.go | 8 +- pkg/sql/distsqlrun/tablereader.go | 1 - pkg/sql/distsqlrun/tablereader_test.go | 8 +- pkg/sql/distsqlrun/utils_test.go | 4 +- pkg/sql/distsqlrun/values_test.go | 7 +- pkg/sql/distsqlrun/zigzagjoiner.go | 4 +- pkg/sql/distsqlrun/zigzagjoiner_test.go | 4 +- pkg/sql/exec_util.go | 40 +- pkg/sql/explain.go | 5 +- pkg/sql/explain_distsql.go | 43 +- .../testdata/logic_test/builtin_function | 3 +- pkg/sql/logictest/testdata/logic_test/union | 2 +- .../testdata/planner_test/distsql_auto_mode | 15 +- .../testdata/planner_test/show_trace | 10 +- .../execbuilder/testdata/distsql_auto_mode | 43 +- .../opt/exec/execbuilder/testdata/show_trace | 12 +- pkg/sql/plan_node_to_row_source.go | 130 ++- pkg/sql/planner.go | 10 +- pkg/sql/sessiondata/search_path.go | 6 + pkg/sql/sqlbase/structured.go | 2 +- pkg/sql/sqlbase/table.go | 10 + pkg/sql/testutils.go | 2 +- pkg/sql/txn_restart_test.go | 14 +- pkg/testutils/sqlutils/sql_runner_test.go | 2 +- 55 files changed, 1339 insertions(+), 768 deletions(-) diff --git a/pkg/acceptance/testdata/c/test.c b/pkg/acceptance/testdata/c/test.c index 2fd4c494bfa8..2135f5a1c3d3 100644 --- a/pkg/acceptance/testdata/c/test.c +++ b/pkg/acceptance/testdata/c/test.c @@ -141,6 +141,7 @@ int main(int argc, char *argv[]) { return 1; } + /* Always call first on any conn that is to be used with libpqtypes */ PQinitTypes(conn); @@ -161,9 +162,11 @@ int main(int argc, char *argv[]) { return 1; } - // '1401-01-19 BC' + // '1401-01-19' PGdate date; - date.isbc = 1; + // TODO(jordan): put this back when #28099 is fixed. + // date.isbc = 1; + date.isbc = 0; date.year = 1401; date.mon = 0; date.mday = 19; @@ -250,6 +253,7 @@ int main(int argc, char *argv[]) { tstz.time.usec = 0; tstz.time.withtz = 1; tstz.time.gmtoff = 0; + if (!PQputf(param, "%timestamptz", &tstz)) { fprintf(stderr, "ERROR PQputf(timestamptz): %s\n", PQgeterror()); return 1; @@ -419,6 +423,7 @@ int main(int argc, char *argv[]) { fprintf(stderr, "ERROR resultFormat=%d PQgetf(timestamptz): %s\n", resultFormat, PQgeterror()); return 1; } + if (!timestampEqual(recvtstz, tstz)) { fprintf(stderr, "resultFormat=%d expected:\n", resultFormat); timestampPrint(tstz); diff --git a/pkg/ccl/importccl/exportcsv.go b/pkg/ccl/importccl/exportcsv.go index 585397a9ae41..716e93a4c44f 100644 --- a/pkg/ccl/importccl/exportcsv.go +++ b/pkg/ccl/importccl/exportcsv.go @@ -85,10 +85,6 @@ func exportPlanHook( return nil, nil, nil, err } - if !p.DistSQLPlanner().CheckPossible(sel) { - return nil, nil, nil, errors.Errorf("unsupported EXPORT query -- as an alternative try `cockroach sql --format=csv`") - } - fn := func(ctx context.Context, plans []sql.PlanNode, resultsCh chan<- tree.Datums) error { ctx, span := tracing.ChildSpan(ctx, exportStmt.StatementTag()) defer tracing.FinishSpan(span) diff --git a/pkg/ccl/importccl/exportcsv_test.go b/pkg/ccl/importccl/exportcsv_test.go index 153165319caa..9639768aaf78 100644 --- a/pkg/ccl/importccl/exportcsv_test.go +++ b/pkg/ccl/importccl/exportcsv_test.go @@ -156,22 +156,6 @@ func TestMultiNodeExportStmt(t *testing.T) { } } -func TestExportNonTable(t *testing.T) { - defer leaktest.AfterTest(t)() - tc := testcluster.StartTestCluster(t, 3, - base.TestClusterArgs{ServerArgs: base.TestServerArgs{UseDatabase: "test"}}, - ) - defer tc.Stopper().Stop(context.Background()) - db := sqlutils.MakeSQLRunner(tc.Conns[0]) - db.Exec(t, "CREATE DATABASE test") - - if _, err := db.DB.Exec( - `EXPORT INTO CSV 'nodelocal:///series' WITH chunk_rows = '10' FROM SELECT generate_series(1, 100)`, - ); !testutils.IsError(err, "unsupported EXPORT query") { - t.Fatal(err) - } -} - func TestExportJoin(t *testing.T) { defer leaktest.AfterTest(t)() dir, cleanupDir := testutils.TempDir(t) diff --git a/pkg/cli/interactive_tests/test_sql_mem_monitor.tcl b/pkg/cli/interactive_tests/test_sql_mem_monitor.tcl index d11ac2883c99..12bf6f6f4e8b 100644 --- a/pkg/cli/interactive_tests/test_sql_mem_monitor.tcl +++ b/pkg/cli/interactive_tests/test_sql_mem_monitor.tcl @@ -77,6 +77,7 @@ expect { "out of memory" {} "cannot allocate memory" {} "std::bad_alloc" {} + "Resource temporarily unavailable" {} timeout { handle_timeout "memory allocation error" } } eexpect ":/# " diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index a4fefec98a93..83ded35a3c55 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -742,7 +742,16 @@ func (ex *connExecutor) dispatchToExecutionEngine( res.SetError(err) return nil } - defer planner.curPlan.close(ctx) + // We only need to close the plan if we don't hand it off to DistSQL. Once + // we hand it off, DistSQL will take care of closing it. + // TODO(jordan): once we add partial plan wrapping, this will need to change. + // We'll need to close all of the nodes that weren't taken over by DistSQL. + needClose := true + defer func() { + if needClose { + planner.curPlan.close(ctx) + } + }() var cols sqlbase.ResultColumns if stmt.AST.StatementType() == tree.Rows { @@ -754,29 +763,15 @@ func (ex *connExecutor) dispatchToExecutionEngine( } ex.sessionTracing.TracePlanCheckStart(ctx) - useDistSQL := false + distributePlan := false // If we use the optimizer and we are in "local" mode, don't try to // distribute. if ex.sessionData.OptimizerMode != sessiondata.OptimizerLocal { - ok, err := planner.prepareForDistSQLSupportCheck( - ctx, ex.sessionData.DistSQLMode == sessiondata.DistSQLAlways, - ) - if err != nil { - ex.sessionTracing.TracePlanCheckEnd(ctx, err, false) - res.SetError(err) - return nil - } - if ok { - useDistSQL, err = shouldUseDistSQL( - ctx, ex.sessionData.DistSQLMode, ex.server.cfg.DistSQLPlanner, planner.curPlan.plan) - if err != nil { - ex.sessionTracing.TracePlanCheckEnd(ctx, err, false) - res.SetError(err) - return nil - } - } + planner.prepareForDistSQLSupportCheck() + distributePlan = shouldDistributePlan( + ctx, ex.sessionData.DistSQLMode, ex.server.cfg.DistSQLPlanner, planner.curPlan.plan) } - ex.sessionTracing.TracePlanCheckEnd(ctx, nil, useDistSQL) + ex.sessionTracing.TracePlanCheckEnd(ctx, nil, distributePlan) if ex.server.cfg.TestingKnobs.BeforeExecute != nil { ex.server.cfg.TestingKnobs.BeforeExecute(ctx, stmt.String(), false /* isParallel */) @@ -791,12 +786,13 @@ func (ex *connExecutor) dispatchToExecutionEngine( panic(fmt.Sprintf("query %d not in registry", stmt.queryID)) } queryMeta.phase = executing - queryMeta.isDistributed = useDistSQL + queryMeta.isDistributed = distributePlan ex.mu.Unlock() - if useDistSQL { + if ex.sessionData.DistSQLMode != sessiondata.DistSQLOff { + needClose = false ex.sessionTracing.TraceExecStart(ctx, "distributed") - err = ex.execWithDistSQLEngine(ctx, planner, stmt.AST.StatementType(), res) + err = ex.execWithDistSQLEngine(ctx, planner, stmt.AST.StatementType(), res, distributePlan) } else { ex.sessionTracing.TraceExecStart(ctx, "local") err = ex.execWithLocalEngine(ctx, planner, stmt.AST.StatementType(), res) @@ -807,7 +803,7 @@ func (ex *connExecutor) dispatchToExecutionEngine( return err } ex.recordStatementSummary( - planner, stmt, useDistSQL, optimizerPlanned, + planner, stmt, distributePlan, optimizerPlanned, ex.extraTxnState.autoRetryCounter, res.RowsAffected(), res.Err(), &ex.server.EngineMetrics, ) @@ -913,7 +909,11 @@ func (ex *connExecutor) execWithLocalEngine( // If an error is returned, the connection needs to stop processing queries. // Query execution errors are written to res; they are not returned. func (ex *connExecutor) execWithDistSQLEngine( - ctx context.Context, planner *planner, stmtType tree.StatementType, res RestrictedCommandResult, + ctx context.Context, + planner *planner, + stmtType tree.StatementType, + res RestrictedCommandResult, + distribute bool, ) error { recv := makeDistSQLReceiver( ctx, res, stmtType, @@ -924,9 +924,9 @@ func (ex *connExecutor) execWithDistSQLEngine( }, &ex.sessionTracing, ) - ex.server.cfg.DistSQLPlanner.PlanAndRun( - ctx, planner.txn, planner, planner.curPlan.plan, recv, planner.ExtendedEvalContext(), - ) + // 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, planner, recv, distribute) return recv.commErr } diff --git a/pkg/sql/distinct.go b/pkg/sql/distinct.go index a665d6c8b727..b25af56e9d26 100644 --- a/pkg/sql/distinct.go +++ b/pkg/sql/distinct.go @@ -190,7 +190,7 @@ func (p *planner) distinct( func (n *distinctNode) startExec(params runParams) error { flowCtx := &distsqlrun.FlowCtx{ - EvalCtx: *params.EvalContext(), + EvalCtx: params.EvalContext(), } cols := make([]int, len(planColumns(n.plan))) @@ -200,7 +200,7 @@ func (n *distinctNode) startExec(params runParams) error { spec := createDistinctSpec(n, cols) - input, err := makePlanNodeToRowSource(n.plan, params) + input, err := makePlanNodeToRowSource(n.plan, params, false) if err != nil { return err } @@ -208,6 +208,19 @@ func (n *distinctNode) startExec(params runParams) error { return errors.New("cannot initialize a distinctNode with 0 columns") } + // Normally, startExec isn't recursive, since it's invoked for all nodes using + // the planTree walker. And as normal, the walker will startExec the source + // of this distinct. + // But, we also need to startExec our planNodeToRowSource to properly + // initialize it. That won't get touched via the planNode walker, so we have + // to do it recursively here. + if err := input.startExec(params); err != nil { + return err + } + if err := input.InitWithOutput(&distsqlrun.PostProcessSpec{}, nil); err != nil { + return err + } + post := &distsqlrun.PostProcessSpec{} // post is not used as we only use the processor for the core distinct logic. var output distsqlrun.RowReceiver // output is never used as distinct is only run as a RowSource. diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 3d1f69c714eb..b38bfaec7933 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -212,38 +212,20 @@ func (dsp *DistSQLPlanner) checkExpr(expr tree.Expr) error { return v.err } -// CheckSupport looks at a planNode tree and decides: -// - whether DistSQL is equipped to handle the query (if not, an error is -// returned). -// - whether it is recommended that the query be run with DistSQL. -func (dsp *DistSQLPlanner) CheckSupport(node planNode) (bool, error) { - rec, err := dsp.checkSupportForNode(node) - if err != nil { - return false, err - } - return (rec == shouldDistribute), nil -} - -// CheckPossible is like CheckSupport but returns whether the query *can* be run -// via distSQL, regardless of whether or not it is recommended. -func (dsp *DistSQLPlanner) CheckPossible(node planNode) bool { - _, err := dsp.checkSupportForNode(node) - return err == nil -} - type distRecommendation int const ( - // shouldNotDistribute indicates that a plan could suffer if run - // under DistSQL - shouldNotDistribute distRecommendation = iota + // cannotDistribute indicates that a plan cannot be distributed. + cannotDistribute distRecommendation = iota + + // shouldNotDistribute indicates that a plan could suffer if distributed. + shouldNotDistribute // canDistribute indicates that a plan will probably not benefit but will - // probably not suffer if run under DistSQL. + // probably not suffer if distributed. canDistribute - // shouldDistribute indicates that a plan will likely benefit if run under - // DistSQL. + // shouldDistribute indicates that a plan will likely benefit if distributed. shouldDistribute ) @@ -251,6 +233,9 @@ const ( // parts of it: if we shouldNotDistribute either part, then we // shouldNotDistribute the overall plan either. func (a distRecommendation) compose(b distRecommendation) distRecommendation { + if a == cannotDistribute || b == cannotDistribute { + return cannotDistribute + } if a == shouldNotDistribute || b == shouldNotDistribute { return shouldNotDistribute } @@ -279,21 +264,23 @@ func newQueryNotSupportedErrorf(format string, args ...interface{}) error { var mutationsNotSupportedError = newQueryNotSupportedError("mutations not supported") var setNotSupportedError = newQueryNotSupportedError("SET / SET CLUSTER SETTING should never distribute") -// checkSupportForNode returns a distRecommendation (as described above) or an -// error if the plan subtree is not supported by DistSQL. +// checkSupportForNode returns a distRecommendation (as described above) or +// cannotDistribute and an error if the plan subtree is not distributable. +// The error doesn't indicate complete failure - it's instead the reason that +// this plan couldn't be distributed. // TODO(radu): add tests for this. func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendation, error) { switch n := node.(type) { case *filterNode: if err := dsp.checkExpr(n.filter); err != nil { - return 0, err + return cannotDistribute, err } return dsp.checkSupportForNode(n.source.plan) case *renderNode: for _, e := range n.render { if err := dsp.checkExpr(e); err != nil { - return 0, err + return cannotDistribute, err } } return dsp.checkSupportForNode(n.source.plan) @@ -301,7 +288,7 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio case *sortNode: rec, err := dsp.checkSupportForNode(n.plan) if err != nil { - return 0, err + return cannotDistribute, err } // If we have to sort, distribute the query. if n.needSort { @@ -311,15 +298,15 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio case *joinNode: if err := dsp.checkExpr(n.pred.onCond); err != nil { - return 0, err + return cannotDistribute, err } recLeft, err := dsp.checkSupportForNode(n.left.plan) if err != nil { - return 0, err + return cannotDistribute, err } recRight, err := dsp.checkSupportForNode(n.right.plan) if err != nil { - return 0, err + return cannotDistribute, err } // If either the left or the right side can benefit from distribution, we // should distribute. @@ -342,7 +329,7 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio // expression or if we have a full table scan. if n.filter != nil { if err := dsp.checkExpr(n.filter); err != nil { - return 0, err + return cannotDistribute, err } rec = rec.compose(shouldDistribute) } @@ -356,33 +343,33 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio // n.table doesn't have meaningful spans, but we need to check support (e.g. // for any filtering expression). if _, err := dsp.checkSupportForNode(n.table); err != nil { - return 0, err + return cannotDistribute, err } return dsp.checkSupportForNode(n.index) case *lookupJoinNode: if err := dsp.checkExpr(n.onCond); err != nil { - return 0, err + return cannotDistribute, err } if _, err := dsp.checkSupportForNode(n.input); err != nil { - return 0, err + return cannotDistribute, err } return shouldDistribute, nil case *groupNode: rec, err := dsp.checkSupportForNode(n.plan) if err != nil { - return 0, err + return cannotDistribute, err } // Distribute aggregations if possible. return rec.compose(shouldDistribute), nil case *limitNode: if err := dsp.checkExpr(n.countExpr); err != nil { - return 0, err + return cannotDistribute, err } if err := dsp.checkExpr(n.offsetExpr); err != nil { - return 0, err + return cannotDistribute, err } return dsp.checkSupportForNode(n.plan) @@ -392,23 +379,26 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio case *unionNode: recLeft, err := dsp.checkSupportForNode(n.left) if err != nil { - return 0, err + return cannotDistribute, err } recRight, err := dsp.checkSupportForNode(n.right) if err != nil { - return 0, err + return cannotDistribute, err } return recLeft.compose(recRight), nil case *valuesNode: if !n.specifiedInQuery { - return 0, newQueryNotSupportedErrorf("unsupported node %T", node) + // This condition indicates that the valuesNode was created by planning, + // not by the user, like the way vtables are expanded into valuesNodes. We + // don't want to distribute queries like this across the network. + return cannotDistribute, newQueryNotSupportedErrorf("unsupported node %T", node) } for _, tuple := range n.tuples { for _, expr := range tuple { if err := dsp.checkExpr(expr); err != nil { - return 0, err + return cannotDistribute, err } } } @@ -419,15 +409,18 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio case *insertNode, *updateNode, *deleteNode, *upsertNode: // This is a potential hot path. - return 0, mutationsNotSupportedError + return cannotDistribute, mutationsNotSupportedError case *setVarNode, *setClusterSettingNode: // SET statements are never distributed. - return 0, setNotSupportedError + return cannotDistribute, setNotSupportedError case *projectSetNode: return dsp.checkSupportForNode(n.source) + case *unaryNode: + return canDistribute, nil + case *zeroNode: return canDistribute, nil @@ -435,7 +428,7 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio return dsp.checkSupportForNode(n.plan) default: - return 0, newQueryNotSupportedErrorf("unsupported node %T", node) + return cannotDistribute, newQueryNotSupportedErrorf("unsupported node %T", node) } } @@ -444,14 +437,30 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio type planningCtx struct { ctx context.Context extendedEvalCtx *extendedEvalContext - spanIter distsqlplan.SpanResolverIterator + // validExtendedEvalCtx is set to true if a flow can use the extendedEvalCtx + // of this planningCtx directly, without having to instantiate a new one. This + // is normally false, with the main exception of the ordinary SQL executor. + validExtendedEvalCtx bool + spanIter distsqlplan.SpanResolverIterator // nodeAddresses contains addresses for all NodeIDs that are referenced by any // physicalPlan we generate with this context. // Nodes that fail a health check have empty addresses. nodeAddresses map[roachpb.NodeID]string + + // isLocal is set to true if we're planning this query on a single node. + isLocal bool + planner *planner + stmtType tree.StatementType + // finishedPlanningFirstNode is set to false until we've finished planning the + // root of the planNode tree. It's used to keep track of whether it's valid to + // run that root node in a special fast path mode. + finishedPlanningFirstNode bool } func (p *planningCtx) EvalContext() *tree.EvalContext { + if p.extendedEvalCtx == nil { + return nil + } return &p.extendedEvalCtx.EvalContext } @@ -989,7 +998,10 @@ func (dsp *DistSQLPlanner) createTableReaders( } var spanPartitions []spanPartition - if n.hardLimit == 0 && n.softLimit == 0 { + 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. spanPartitions, err = dsp.partitionSpans(planCtx, n.spans) if err != nil { return physicalPlan{}, err @@ -2188,12 +2200,6 @@ func (dsp *DistSQLPlanner) createPlanForNode( planCtx *planningCtx, node planNode, ) (plan physicalPlan, err error) { switch n := node.(type) { - case *explainDistSQLNode: - curTol := dsp.metadataTestTolerance - dsp.metadataTestTolerance = distsqlrun.On - defer func() { dsp.metadataTestTolerance = curTol }() - plan, err = dsp.createPlanForNode(planCtx, n.plan) - case *scanNode: plan, err = dsp.createTableReaders(planCtx, n, nil) @@ -2252,7 +2258,7 @@ func (dsp *DistSQLPlanner) createPlanForNode( if err := n.evalLimit(planCtx.EvalContext()); err != nil { return physicalPlan{}, err } - if err := plan.AddLimit(n.count, n.offset, dsp.nodeDesc.NodeID); err != nil { + if err := plan.AddLimit(n.count, n.offset, planCtx.EvalContext(), dsp.nodeDesc.NodeID); err != nil { return physicalPlan{}, err } @@ -2263,7 +2269,15 @@ func (dsp *DistSQLPlanner) createPlanForNode( plan, err = dsp.createPlanForSetOp(planCtx, n) case *valuesNode: - plan, err = dsp.createPlanForValues(planCtx, n) + // Just like in checkSupportForNode, if a valuesNode wasn't specified in + // the query, it means that it was autogenerated for things that we don't + // want to be distributing, like populating values from a virtual table. So, + // we wrap the plan instead. + if !n.specifiedInQuery { + plan, err = dsp.wrapPlan(planCtx, n) + } else { + plan, err = dsp.createPlanForValues(planCtx, n) + } case *createStatsNode: plan, err = dsp.createPlanForCreateStats(planCtx, n) @@ -2271,6 +2285,9 @@ func (dsp *DistSQLPlanner) createPlanForNode( case *projectSetNode: plan, err = dsp.createPlanForProjectSet(planCtx, n) + case *unaryNode: + plan, err = dsp.createPlanForUnary(planCtx, n) + case *zeroNode: plan, err = dsp.createPlanForZero(planCtx, n) @@ -2278,7 +2295,12 @@ func (dsp *DistSQLPlanner) createPlanForNode( plan, err = dsp.createPlanForWindow(planCtx, n) default: - panic(fmt.Sprintf("unsupported node type %T", n)) + // Can't handle a node? We wrap it and continue on our way. + // TODO(jordan): this should only wrap the node itself, not all of its + // children as well. To deal with this the wrapper should use the + // planNode walker to retrieve all of the children of the current plan, + // and recurse with createPlanForNode on all of those children. + plan, err = dsp.wrapPlan(planCtx, n) } if err != nil { @@ -2303,9 +2325,71 @@ func (dsp *DistSQLPlanner) createPlanForNode( ) } + planCtx.finishedPlanningFirstNode = true + return plan, err } +// wrapPlan produces a DistSQL processor for an arbitrary planNode. This is +// invoked when a particular planNode can't be distributed for some reason. It +// will create a planNodeToRowSource wrapper, embed the planNode in that +// wrapper, and hook the wrapper up to the rest of the plan. +func (dsp *DistSQLPlanner) wrapPlan(planCtx *planningCtx, n planNode) (physicalPlan, error) { + // Copy the evalCtx. + evalCtx := *planCtx.extendedEvalCtx + // We permit the planNodeToRowSource to trigger the wrapped planNode's fast + // path if its the very first node in the flow, and if the statement type we're + // expecting is in fact RowsAffected. RowsAffected statements return a single + // row with the number of rows affected by the statement, and are the only + // types of statement where it's valid to invoke a plan's fast path. + useFastPath := !planCtx.finishedPlanningFirstNode && planCtx.stmtType == tree.RowsAffected + wrapper, err := makePlanNodeToRowSource(n, + runParams{ + extendedEvalCtx: &evalCtx, + p: planCtx.planner, + }, + useFastPath, + ) + if err != nil { + return physicalPlan{}, err + } + planCols := planColumns(n) + types := make([]sqlbase.ColumnType, len(planCols)) + for i, t := range planCols { + colTyp, err := sqlbase.DatumTypeToColumnType(t.Typ) + if err != nil { + return physicalPlan{}, err + } + types[i] = colTyp + } + idx := uint32(0) + p := physicalPlan{ + PhysicalPlan: distsqlplan.PhysicalPlan{ + LocalProcessors: []distsqlrun.LocalProcessor{wrapper}, + LocalProcessorIndexes: []*uint32{&idx}, + ResultTypes: types, + ResultRouters: make([]distsqlplan.ProcessorIdx, 1), + }, + planToStreamColMap: identityMapInPlace(make([]int, len(planCols))), + } + proc := distsqlplan.Processor{ + Node: dsp.nodeDesc.NodeID, + Spec: distsqlrun.ProcessorSpec{ + Core: distsqlrun.ProcessorCoreUnion{LocalPlanNode: &distsqlrun.LocalPlanNodeSpec{ + RowSourceIdx: &idx, + }}, + Post: distsqlrun.PostProcessSpec{}, + Output: []distsqlrun.OutputRouterSpec{{ + Type: distsqlrun.OutputRouterSpec_PASS_THROUGH, + }}, + StageID: p.NewStageID(), + }, + } + pIdx := p.AddProcessor(proc) + p.ResultRouters[0] = pIdx + return p, nil +} + // createValuesPlan creates a plan with a single Values processor // located on the gateway node and initialized with given numRows // and rawBytes that need to be precomputed beforehand. @@ -2350,7 +2434,6 @@ func (dsp *DistSQLPlanner) createPlanForValues( params := runParams{ ctx: planCtx.ctx, extendedEvalCtx: planCtx.extendedEvalCtx, - p: nil, } types, err := getTypesForPlanResult(n, nil /* planToStreamColMap */) @@ -2388,6 +2471,17 @@ func (dsp *DistSQLPlanner) createPlanForValues( return dsp.createValuesPlan(types, numRows, rawBytes) } +func (dsp *DistSQLPlanner) createPlanForUnary( + planCtx *planningCtx, n *unaryNode, +) (physicalPlan, error) { + types, err := getTypesForPlanResult(n, nil /* planToStreamColMap */) + if err != nil { + return physicalPlan{}, err + } + + return dsp.createValuesPlan(types, 1 /* numRows */, nil /* rawBytes */) +} + func (dsp *DistSQLPlanner) createPlanForZero( planCtx *planningCtx, n *zeroNode, ) (physicalPlan, error) { diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index a0805042183f..ea42aae34373 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -214,7 +214,16 @@ func (dsp *DistSQLPlanner) Run( // Set up the flow on this node. localReq := setupReq localReq.Flow = flows[thisNodeID] - ctx, flow, err := dsp.distSQLSrv.SetupSyncFlow(ctx, evalCtx.Mon, &localReq, recv) + var localState distsqlrun.LocalState + if planCtx.validExtendedEvalCtx { + localState.EvalContext = planCtx.EvalContext() + } + if planCtx.isLocal { + localState.IsLocal = true + localState.LocalProcs = plan.LocalProcessors + localState.Txn = txn + } + ctx, flow, err := dsp.distSQLSrv.SetupLocalSyncFlow(ctx, evalCtx.Mon, &localReq, recv, localState) if err != nil { recv.SetError(err) return @@ -360,15 +369,17 @@ func makeDistSQLReceiver( stmtType: stmtType, tracing: tracing, } - // When the root transaction finishes (i.e. it is abandoned, aborted, or - // committed), ensure the flow is canceled so that we don't return results to - // the client that might have missed seeing their own writes. The committed - // case shouldn't happen. + // If this receiver is part of a distributed flow and isn't using the root + // transaction, we need to sure that the flow is canceled when the root + // transaction finishes (i.e. it is abandoned, aborted, or committed), so that + // we don't return results to the client that might have missed seeing their + // own writes. The committed case shouldn't happen. This isn't necessary if + // the flow is running locally and is using the root transaction. // // TODO(andrei): Instead of doing this, should we lift this transaction // monitoring to connExecutor and have it cancel the SQL txn's context? Or for // that matter, should the TxnCoordSender cancel the context itself? - if r.txn != nil { + if r.txn != nil && r.txn.Type() == client.LeafTxn { r.txn.OnCurrentIncarnationFinish(func(err error) { r.txnAbortedErr.Store(errWrap{err: err}) }) @@ -447,8 +458,10 @@ func (r *distSQLReceiver) Push( } if r.stmtType != tree.Rows { - // We only need the row count. - r.resultWriter.IncrementRowsAffected(1) + // We only need the row count. planNodeToRowSource is set up to handle + // ensuring that the last stage in the pipeline will return a single-column + // row with the row count in it, so just grab that and exit. + r.resultWriter.IncrementRowsAffected(int(tree.MustBeDInt(row[0].Datum))) return r.status } if r.row == nil { @@ -528,15 +541,17 @@ func (r *distSQLReceiver) updateCaches(ctx context.Context, ranges []roachpb.Ran // distSQLReceiver.commErr. That can be tested to see if a client session needs // to be closed. func (dsp *DistSQLPlanner) PlanAndRun( - ctx context.Context, - txn *client.Txn, - p *planner, - plan planNode, - recv *distSQLReceiver, - evalCtx *extendedEvalContext, + ctx context.Context, p *planner, recv *distSQLReceiver, distribute bool, ) { + evalCtx := p.ExtendedEvalContext() + txn := p.txn planCtx := dsp.newPlanningCtx(ctx, evalCtx, txn) - log.VEvent(ctx, 1, "creating DistSQL plan") + planCtx.isLocal = !distribute + planCtx.planner = p + planCtx.stmtType = recv.stmtType + planCtx.validExtendedEvalCtx = true + + log.VEventf(ctx, 1, "creating DistSQL plan with distributedMode=%v", distribute) if len(p.curPlan.subqueryPlans) != 0 { err := p.curPlan.evalSubqueries(runParams{ @@ -552,7 +567,7 @@ func (dsp *DistSQLPlanner) PlanAndRun( log.VEvent(ctx, 2, "evaluated subqueries") } - physPlan, err := dsp.createPlanForNode(&planCtx, plan) + physPlan, err := dsp.createPlanForNode(&planCtx, p.curPlan.plan) if err != nil { recv.SetError(err) return diff --git a/pkg/sql/distsqlplan/physical_plan.go b/pkg/sql/distsqlplan/physical_plan.go index 2b55a164c25f..195befadf6cf 100644 --- a/pkg/sql/distsqlplan/physical_plan.go +++ b/pkg/sql/distsqlplan/physical_plan.go @@ -72,6 +72,18 @@ type PhysicalPlan struct { // Processors in the plan. Processors []Processor + // LocalProcessors contains all of the planNodeToRowSourceWrappers that were + // installed in this physical plan to wrap any planNodes that couldn't be + // properly translated into DistSQL processors. This will be empty if no + // wrapping had to happen. + LocalProcessors []distsqlrun.LocalProcessor + + // LocalProcessorIndexes contains pointers to all of the RowSourceIdx fields + // of the LocalPlanNodeSpecs that were created. This list is in the same + // order as LocalProcessors, and is kept up-to-date so that LocalPlanNodeSpecs + // always have the correct index into the LocalProcessors slice. + LocalProcessorIndexes []*uint32 + // Streams accumulates the streams in the plan - both local (intra-node) and // remote (inter-node); when we have a final plan, the streams are used to // generate processor input and output specs (see PopulateEndpoints). @@ -646,16 +658,30 @@ func emptyPlan(types []sqlbase.ColumnType, node roachpb.NodeID) PhysicalPlan { // that is placed on the given node. // // For no limit, count should be MaxInt64. -func (p *PhysicalPlan) AddLimit(count int64, offset int64, node roachpb.NodeID) error { +func (p *PhysicalPlan) AddLimit( + count int64, offset int64, evalCtx *tree.EvalContext, node roachpb.NodeID, +) error { if count < 0 { return errors.Errorf("negative limit") } if offset < 0 { return errors.Errorf("negative offset") } + // limitZero is set to true if the limit is a legitimate LIMIT 0 requested by + // the user. This needs to be tracked as a separate condition because DistSQL + // uses count=0 to mean no limit, not a limit of 0. Normally, DistSQL will + // short circuit 0-limit plans, but wrapped local planNodes sometimes need to + // be fully-executed despite having 0 limit, so if we do in fact have a + // limit-0 case when there's local planNodes around, we add an empty plan + // instead of completely eliding the 0-limit plan. + limitZero := false if count == 0 { - *p = emptyPlan(p.ResultTypes, node) - return nil + if len(p.LocalProcessors) == 0 { + *p = emptyPlan(p.ResultTypes, node) + return nil + } + count = 1 + limitZero = true } if len(p.ResultRouters) == 1 { @@ -670,8 +696,15 @@ func (p *PhysicalPlan) AddLimit(count int64, offset int64, node roachpb.NodeID) // SELECT * FROM (SELECT * FROM .. LIMIT 5) OFFSET 10 // TODO(radu): perform this optimization while propagating filters // instead of having to detect it here. - *p = emptyPlan(p.ResultTypes, node) - return nil + if len(p.LocalProcessors) == 0 { + // Even though we know there will be no results, we don't elide the + // plan if there are local processors. See comment above limitZero + // for why. + *p = emptyPlan(p.ResultTypes, node) + return nil + } + count = 1 + limitZero = true } post.Offset += uint64(offset) } @@ -679,6 +712,11 @@ func (p *PhysicalPlan) AddLimit(count int64, offset int64, node roachpb.NodeID) post.Limit = uint64(count) } p.SetLastStagePost(post, p.ResultTypes) + if limitZero { + if err := p.AddFilter(tree.DBoolFalse, evalCtx, nil); err != nil { + return err + } + } return nil } @@ -708,6 +746,11 @@ func (p *PhysicalPlan) AddLimit(count int64, offset int64, node roachpb.NodeID) post, p.ResultTypes, ) + if limitZero { + if err := p.AddFilter(tree.DBoolFalse, evalCtx, nil); err != nil { + return err + } + } return nil } @@ -810,6 +853,13 @@ func MergePlans( } mergedPlan.stageCounter = left.stageCounter + right.stageCounter + mergedPlan.LocalProcessors = append(left.LocalProcessors, right.LocalProcessors...) + mergedPlan.LocalProcessorIndexes = append(left.LocalProcessorIndexes, right.LocalProcessorIndexes...) + // Update the local processor indices in the right streams. + for i := len(left.LocalProcessorIndexes); i < len(mergedPlan.LocalProcessorIndexes); i++ { + *mergedPlan.LocalProcessorIndexes[i] += uint32(len(left.LocalProcessorIndexes)) + } + leftRouters = left.ResultRouters rightRouters = append([]ProcessorIdx(nil), right.ResultRouters...) // Update the processor indices in the right routers. diff --git a/pkg/sql/distsqlrun/aggregator.go b/pkg/sql/distsqlrun/aggregator.go index 8aae4746fcf1..ea5702dfb38a 100644 --- a/pkg/sql/distsqlrun/aggregator.go +++ b/pkg/sql/distsqlrun/aggregator.go @@ -419,7 +419,7 @@ func (ag *orderedAggregator) close() { func (ag *aggregatorBase) matchLastOrdGroupCols(row sqlbase.EncDatumRow) (bool, error) { for _, colIdx := range ag.orderedGroupCols { res, err := ag.lastOrdGroupCols[colIdx].Compare( - &ag.inputTypes[colIdx], &ag.datumAlloc, &ag.flowCtx.EvalCtx, &row[colIdx], + &ag.inputTypes[colIdx], &ag.datumAlloc, ag.flowCtx.EvalCtx, &row[colIdx], ) if res != 0 || err != nil { return false, err @@ -886,7 +886,7 @@ func (ag *aggregatorBase) createAggregateFuncs() (aggregateFuncs, error) { for i, f := range ag.funcs { // TODO(radu): we should account for the size of impl (this needs to be done // in each aggregate constructor). - bucket[i] = f.create(&ag.flowCtx.EvalCtx) + bucket[i] = f.create(ag.flowCtx.EvalCtx) } return bucket, nil } diff --git a/pkg/sql/distsqlrun/aggregator_test.go b/pkg/sql/distsqlrun/aggregator_test.go index 366ae00ddec9..835d5d083938 100644 --- a/pkg/sql/distsqlrun/aggregator_test.go +++ b/pkg/sql/distsqlrun/aggregator_test.go @@ -420,7 +420,7 @@ func TestAggregator(t *testing.T) { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } ag, err := newAggregator(&flowCtx, 0 /* processorID */, &ags, in, &PostProcessSpec{}, out) @@ -480,7 +480,7 @@ func BenchmarkAggregation(b *testing.B) { flowCtx := &FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } for _, aggFunc := range aggFuncs { @@ -523,7 +523,7 @@ func BenchmarkGrouping(b *testing.B) { flowCtx := &FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } spec := &AggregatorSpec{ GroupCols: []uint32{0}, @@ -571,7 +571,7 @@ func benchmarkAggregationWithGrouping(b *testing.B, numOrderedCols int) { flowCtx := &FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } for _, aggFunc := range aggFuncs { diff --git a/pkg/sql/distsqlrun/api.go b/pkg/sql/distsqlrun/api.go index 4afa289c62a3..6f79e3a64bc6 100644 --- a/pkg/sql/distsqlrun/api.go +++ b/pkg/sql/distsqlrun/api.go @@ -44,11 +44,11 @@ func MakeEvalContext(evalCtx tree.EvalContext) EvalContext { ExtraFloatDigits: int32(evalCtx.SessionData.DataConversion.ExtraFloatDigits), } - // Populate the search path. - iter := evalCtx.SessionData.SearchPath.Iter() - for s, ok := iter(); ok; s, ok = iter() { - res.SearchPath = append(res.SearchPath, s) - } + // Populate the search path. Make sure not to include the implicit pg_catalog, + // since the remote end already knows to add the implicit pg_catalog if + // necessary, and sending it over would make the remote end think that + // pg_catalog was explicitly included by the user. + res.SearchPath = evalCtx.SessionData.SearchPath.GetPathArray() // Populate the sequences state. latestValues, lastIncremented := evalCtx.SessionData.SequenceState.Export() diff --git a/pkg/sql/distsqlrun/api.pb.go b/pkg/sql/distsqlrun/api.pb.go index 8e16b018c87a..f20c7fab5916 100644 --- a/pkg/sql/distsqlrun/api.pb.go +++ b/pkg/sql/distsqlrun/api.pb.go @@ -44,6 +44,7 @@ SorterSpec DistinctSpec ZigzagJoinerSpec + LocalPlanNodeSpec Columns MergeJoinerSpec HashJoinerSpec diff --git a/pkg/sql/distsqlrun/distinct_test.go b/pkg/sql/distsqlrun/distinct_test.go index 7dc38fbfb677..56acafb454a8 100644 --- a/pkg/sql/distsqlrun/distinct_test.go +++ b/pkg/sql/distsqlrun/distinct_test.go @@ -120,7 +120,7 @@ func TestDistinct(t *testing.T) { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } d, err := NewDistinct(&flowCtx, 0 /* processorID */, &ds, in, &PostProcessSpec{}, out) @@ -158,7 +158,7 @@ func benchmarkDistinct(b *testing.B, orderedColumns []uint32) { flowCtx := &FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } spec := &DistinctSpec{ DistinctColumns: []uint32{0, 1}, diff --git a/pkg/sql/distsqlrun/flow.go b/pkg/sql/distsqlrun/flow.go index 79f418f5aed5..c0cf89e6b123 100644 --- a/pkg/sql/distsqlrun/flow.go +++ b/pkg/sql/distsqlrun/flow.go @@ -18,7 +18,7 @@ import ( "context" "sync" - opentracing "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "github.com/cockroachdb/cockroach/pkg/gossip" @@ -71,7 +71,7 @@ type FlowCtx struct { // // TODO(andrei): Get rid of this field and pass a non-shared EvalContext to // cores of the processors that need it. - EvalCtx tree.EvalContext + EvalCtx *tree.EvalContext // rpcCtx and nodeDialer are used by the Outboxes that may be // present in the flow for connecting to other nodes (rpcCtx for // flows initiated by 2.0 nodes that specify addresses, and @@ -123,7 +123,7 @@ type FlowCtx struct { // them at runtime to ensure expressions are evaluated with the correct indexed // var context. func (ctx *FlowCtx) NewEvalCtx() *tree.EvalContext { - evalCtx := ctx.EvalCtx + evalCtx := *ctx.EvalCtx return &evalCtx } @@ -157,6 +157,8 @@ type Flow struct { // or to the local host). syncFlowConsumer RowReceiver + localProcessors []LocalProcessor + localStreams map[StreamID]RowReceiver // inboundStreams are streams that receive data from other hosts; this map @@ -182,11 +184,17 @@ type Flow struct { spec *FlowSpec } -func newFlow(flowCtx FlowCtx, flowReg *flowRegistry, syncFlowConsumer RowReceiver) *Flow { +func newFlow( + flowCtx FlowCtx, + flowReg *flowRegistry, + syncFlowConsumer RowReceiver, + localProcessors []LocalProcessor, +) *Flow { f := &Flow{ FlowCtx: flowCtx, flowRegistry: flowReg, syncFlowConsumer: syncFlowConsumer, + localProcessors: localProcessors, } f.status = FlowNotStarted return f @@ -233,6 +241,22 @@ func (f *Flow) setupInboundStream( return nil } +// This RowReceiver clears its BoundAccount on every input row. This is useful +// for clearing the per-row memory account that's used for expression +// evaluation. +type accountClearingRowReceiver struct { + RowReceiver + ctx context.Context + acc *mon.BoundAccount +} + +func (r *accountClearingRowReceiver) Push( + row sqlbase.EncDatumRow, meta *ProducerMetadata, +) ConsumerStatus { + r.acc.Clear(r.ctx) + return r.RowReceiver.Push(row, meta) +} + // setupOutboundStream sets up an output stream; if the stream is local, the // RowChannel is looked up in the localStreams map; otherwise an outgoing // mailbox is created. @@ -240,7 +264,13 @@ func (f *Flow) setupOutboundStream(spec StreamEndpointSpec) (RowReceiver, error) sid := spec.StreamID switch spec.Type { case StreamEndpointSpec_SYNC_RESPONSE: - return f.syncFlowConsumer, nil + // Wrap the syncFlowConsumer in a row receiver that clears the row's memory + // account. + return &accountClearingRowReceiver{ + acc: f.EvalCtx.ActiveMemAcc, + ctx: f.EvalCtx.Ctx(), + RowReceiver: f.syncFlowConsumer, + }, nil case StreamEndpointSpec_REMOTE: outbox := newOutbox(&f.FlowCtx, spec.TargetNodeID, spec.DeprecatedTargetAddr, f.id, sid) @@ -330,7 +360,7 @@ func (f *Flow) makeProcessor( outputs[i] = ©ingRowReceiver{RowReceiver: outputs[i]} } - proc, err := newProcessor(ctx, &f.FlowCtx, ps.ProcessorID, &ps.Core, &ps.Post, inputs, outputs) + proc, err := newProcessor(ctx, &f.FlowCtx, ps.ProcessorID, &ps.Core, &ps.Post, inputs, outputs, f.localProcessors) if err != nil { return nil, err } @@ -338,8 +368,12 @@ func (f *Flow) makeProcessor( // Initialize any routers (the setupRouter case above) and outboxes. types := proc.OutputTypes() for _, o := range outputs { - copier := o.(*copyingRowReceiver) - switch o := copier.RowReceiver.(type) { + 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: @@ -382,7 +416,7 @@ func (f *Flow) setup(ctx context.Context, spec *FlowSpec) error { streams[i] = rowChan } var err error - sync, err = makeOrderedSync(convertToColumnOrdering(is.Ordering), &f.EvalCtx, streams) + sync, err = makeOrderedSync(convertToColumnOrdering(is.Ordering), f.EvalCtx, streams) if err != nil { return err } diff --git a/pkg/sql/distsqlrun/flow_diagram.go b/pkg/sql/distsqlrun/flow_diagram.go index d00edac88e7b..652ec6c476a7 100644 --- a/pkg/sql/distsqlrun/flow_diagram.go +++ b/pkg/sql/distsqlrun/flow_diagram.go @@ -332,6 +332,11 @@ func (is *InputSyncSpec) summary() (string, []string) { } } +// summary implements the diagramCellType interface. +func (r *LocalPlanNodeSpec) summary() (string, []string) { + return fmt.Sprintf("local planNode %d", *r.RowSourceIdx), []string{} +} + // summary implements the diagramCellType interface. func (r *OutputRouterSpec) summary() (string, []string) { switch r.Type { diff --git a/pkg/sql/distsqlrun/hashjoiner_test.go b/pkg/sql/distsqlrun/hashjoiner_test.go index 706aaeda179c..6a0fe157ed45 100644 --- a/pkg/sql/distsqlrun/hashjoiner_test.go +++ b/pkg/sql/distsqlrun/hashjoiner_test.go @@ -85,7 +85,7 @@ func TestHashJoiner(t *testing.T) { out := &RowBuffer{} flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, TempStorage: tempEngine, diskMonitor: &diskMonitor, } @@ -211,7 +211,7 @@ func TestHashJoinerError(t *testing.T) { out := &RowBuffer{} flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, TempStorage: tempEngine, diskMonitor: &diskMonitor, } @@ -351,7 +351,7 @@ func TestHashJoinerDrain(t *testing.T) { defer evalCtx.Stop(ctx) flowCtx := FlowCtx{ Settings: settings, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } post := PostProcessSpec{Projection: true, OutputColumns: outCols} @@ -474,7 +474,7 @@ func TestHashJoinerDrainAfterBuildPhaseError(t *testing.T) { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } // Disable external storage for this test to avoid initializing temp storage @@ -537,7 +537,7 @@ func BenchmarkHashJoiner(b *testing.B) { defer diskMonitor.Stop(ctx) flowCtx := &FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, diskMonitor: &diskMonitor, } tempEngine, err := engine.NewTempEngine(base.DefaultTestTempStorageConfig(st), base.DefaultTestStoreSpec) diff --git a/pkg/sql/distsqlrun/interleaved_reader_joiner.go b/pkg/sql/distsqlrun/interleaved_reader_joiner.go index 68f9cdd43507..b8478f6b4f57 100644 --- a/pkg/sql/distsqlrun/interleaved_reader_joiner.go +++ b/pkg/sql/distsqlrun/interleaved_reader_joiner.go @@ -217,7 +217,7 @@ func (irj *interleavedReaderJoiner) nextRow() (irjState, sqlbase.EncDatumRow, *P irj.tables[1].ordering, false, /* nullEquality */ &irj.alloc, - &irj.flowCtx.EvalCtx, + irj.flowCtx.EvalCtx, ) if err != nil { irj.moveToDraining(err) @@ -323,7 +323,7 @@ func newInterleavedReaderJoiner( } if err := tables[i].post.Init( - &table.Post, table.Desc.ColumnTypes(), &flowCtx.EvalCtx, nil, /*output*/ + &table.Post, table.Desc.ColumnTypes(), flowCtx.EvalCtx, nil, /*output*/ ); err != nil { return nil, errors.Wrapf(err, "failed to initialize post-processing helper") } diff --git a/pkg/sql/distsqlrun/interleaved_reader_joiner_test.go b/pkg/sql/distsqlrun/interleaved_reader_joiner_test.go index 4a9590e1d0d2..fe760348b843 100644 --- a/pkg/sql/distsqlrun/interleaved_reader_joiner_test.go +++ b/pkg/sql/distsqlrun/interleaved_reader_joiner_test.go @@ -397,7 +397,7 @@ func TestInterleavedReaderJoiner(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(s.ClusterSettings()) defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: s.ClusterSettings(), // Run in a RootTxn so that there's no txn metadata produced. txn: client.NewTxn(s.DB(), s.NodeID(), client.RootTxn), @@ -526,7 +526,7 @@ func TestInterleavedReaderJoinerErrors(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(s.ClusterSettings()) defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: s.ClusterSettings(), // Run in a RootTxn so that there's no txn metadata produced. txn: client.NewTxn(s.DB(), s.NodeID(), client.RootTxn), @@ -580,7 +580,7 @@ func TestInterleavedReaderJoinerTrailingMetadata(t *testing.T) { defer sp.Finish() flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: s.ClusterSettings(), // Run in a LeafTxn so that txn metadata is produced. txn: client.NewTxn(s.DB(), s.NodeID(), client.LeafTxn), diff --git a/pkg/sql/distsqlrun/joinreader_test.go b/pkg/sql/distsqlrun/joinreader_test.go index af4f0c556ab0..a9bbbbd519dc 100644 --- a/pkg/sql/distsqlrun/joinreader_test.go +++ b/pkg/sql/distsqlrun/joinreader_test.go @@ -251,7 +251,7 @@ func TestJoinReader(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(st) defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: st, txn: client.NewTxn(s.DB(), s.NodeID(), client.RootTxn), } @@ -380,7 +380,7 @@ INSERT INTO test.t VALUES // Initialize join reader args. indexIdx := uint32(1) // first (and only) secondary index flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: st, txn: client.NewTxn(s.DB(), s.NodeID(), client.RootTxn), } @@ -462,7 +462,7 @@ func TestJoinReaderDrain(t *testing.T) { defer sp.Finish() flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: s.ClusterSettings(), txn: client.NewTxn(s.DB(), s.NodeID(), client.LeafTxn), } @@ -552,7 +552,7 @@ func BenchmarkJoinReader(b *testing.B) { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: s.ClusterSettings(), txn: client.NewTxn(s.DB(), s.NodeID(), client.RootTxn), } diff --git a/pkg/sql/distsqlrun/mergejoiner_test.go b/pkg/sql/distsqlrun/mergejoiner_test.go index f3a1e1340124..49f113c4042e 100644 --- a/pkg/sql/distsqlrun/mergejoiner_test.go +++ b/pkg/sql/distsqlrun/mergejoiner_test.go @@ -703,7 +703,7 @@ func TestMergeJoiner(t *testing.T) { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } post := PostProcessSpec{Projection: true, OutputColumns: c.outCols} @@ -810,7 +810,7 @@ func TestConsumerClosed(t *testing.T) { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } post := PostProcessSpec{Projection: true, OutputColumns: outCols} m, err := newMergeJoiner(&flowCtx, 0 /* processorID */, &spec, leftInput, rightInput, &post, out) @@ -834,7 +834,7 @@ func BenchmarkMergeJoiner(b *testing.B) { defer evalCtx.Stop(ctx) flowCtx := &FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } spec := &MergeJoinerSpec{ diff --git a/pkg/sql/distsqlrun/noop_test.go b/pkg/sql/distsqlrun/noop_test.go index 3c0492eccc64..029e8f19489b 100644 --- a/pkg/sql/distsqlrun/noop_test.go +++ b/pkg/sql/distsqlrun/noop_test.go @@ -34,7 +34,7 @@ func BenchmarkNoop(b *testing.B) { flowCtx := &FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } post := &PostProcessSpec{} disposer := &RowDisposer{} diff --git a/pkg/sql/distsqlrun/outbox_test.go b/pkg/sql/distsqlrun/outbox_test.go index 781fd9ef75f5..335abd87c81d 100644 --- a/pkg/sql/distsqlrun/outbox_test.go +++ b/pkg/sql/distsqlrun/outbox_test.go @@ -63,7 +63,7 @@ func TestOutbox(t *testing.T) { flowCtx := FlowCtx{ Settings: st, stopper: stopper, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, nodeDialer: nodedialer.New(newInsecureRPCContext(stopper), staticAddressResolver(addr)), } flowID := FlowID{uuid.MakeV4()} @@ -214,7 +214,7 @@ func TestOutboxInitializesStreamBeforeRecevingAnyRows(t *testing.T) { flowCtx := FlowCtx{ Settings: st, stopper: stopper, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, nodeDialer: nodedialer.New(newInsecureRPCContext(stopper), staticAddressResolver(addr)), } flowID := FlowID{uuid.MakeV4()} @@ -283,7 +283,7 @@ func TestOutboxClosesWhenConsumerCloses(t *testing.T) { flowCtx := FlowCtx{ Settings: st, stopper: stopper, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, nodeDialer: nodedialer.New(newInsecureRPCContext(stopper), staticAddressResolver(addr)), } flowID := FlowID{uuid.MakeV4()} @@ -409,7 +409,7 @@ func TestOutboxCancelsFlowOnError(t *testing.T) { flowCtx := FlowCtx{ Settings: st, stopper: stopper, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, nodeDialer: nodedialer.New(newInsecureRPCContext(stopper), staticAddressResolver(addr)), } flowID := FlowID{uuid.MakeV4()} @@ -465,7 +465,7 @@ func TestOutboxCancelsFlowOnErrorLegacyInterface(t *testing.T) { flowCtx := FlowCtx{ Settings: st, stopper: stopper, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, rpcCtx: newInsecureRPCContext(stopper), } flowID := FlowID{uuid.MakeV4()} @@ -524,7 +524,7 @@ func BenchmarkOutbox(b *testing.B) { flowCtx := FlowCtx{ Settings: st, stopper: stopper, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, nodeDialer: nodedialer.New(newInsecureRPCContext(stopper), staticAddressResolver(addr)), } outbox := newOutbox(&flowCtx, staticNodeID, "", flowID, streamID) diff --git a/pkg/sql/distsqlrun/processors.go b/pkg/sql/distsqlrun/processors.go index 75f7d6c86c5b..e44787109026 100644 --- a/pkg/sql/distsqlrun/processors.go +++ b/pkg/sql/distsqlrun/processors.go @@ -87,6 +87,11 @@ type ProcOutputHelper struct { rowIdx uint64 } +// Output returns the output RowReciever of this ProcOutputHelper. +func (h *ProcOutputHelper) Output() RowReceiver { + return h.output +} + // 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. @@ -190,6 +195,11 @@ func (h *ProcOutputHelper) neededColumns() (colIdxs util.FastIntSet) { return colIdxs } +// OutputTypes returns the output types of this ProcOutputHelper. +func (h *ProcOutputHelper) OutputTypes() []sqlbase.ColumnType { + return h.outputTypes +} + // emitHelper is a utility wrapper on top of ProcOutputHelper.EmitRow(). // It takes a row to emit and, if anything happens other than the normal // situation where the emitting succeeds and the consumer still needs rows, both @@ -910,6 +920,7 @@ func newProcessor( post *PostProcessSpec, inputs []RowSource, outputs []RowReceiver, + localProcessors []LocalProcessor, ) (Processor, error) { if core.Noop != nil { if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { @@ -1058,9 +1069,26 @@ func newProcessor( } return newWindower(flowCtx, processorID, core.Windower, inputs[0], post, outputs[0]) } + if core.LocalPlanNode != nil { + if err := checkNumInOut(inputs, outputs, 0, 1); err != nil { + return nil, err + } + processor := localProcessors[*core.LocalPlanNode.RowSourceIdx] + err := processor.InitWithOutput(post, outputs[0]) + return processor, err + } return nil, errors.Errorf("unsupported processor core %s", core) } +// LocalProcessor is a RowSourcedProcessor that needs to be initialized with +// its post processing spec and output row receiver. Most processors can accept +// these objects at creation time. +type LocalProcessor interface { + RowSourcedProcessor + // InitWithOutput initializes this processor. + InitWithOutput(post *PostProcessSpec, output RowReceiver) error +} + // NewReadImportDataProcessor is externally implemented and registered by // ccl/sqlccl/csv.go. var NewReadImportDataProcessor func(*FlowCtx, int32, ReadImportDataSpec, RowReceiver) (Processor, error) diff --git a/pkg/sql/distsqlrun/processors.pb.go b/pkg/sql/distsqlrun/processors.pb.go index 9aed04a8af39..feebb041c21a 100644 --- a/pkg/sql/distsqlrun/processors.pb.go +++ b/pkg/sql/distsqlrun/processors.pb.go @@ -150,7 +150,7 @@ func (x *AggregatorSpec_Func) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Func) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{16, 0} + return fileDescriptorProcessors, []int{17, 0} } type AggregatorSpec_Type int32 @@ -196,7 +196,7 @@ func (x *AggregatorSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{16, 1} + return fileDescriptorProcessors, []int{17, 1} } type BackfillerSpec_Type int32 @@ -235,7 +235,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{17, 0} + return fileDescriptorProcessors, []int{18, 0} } type WindowerSpec_WindowFunc int32 @@ -299,7 +299,7 @@ func (x *WindowerSpec_WindowFunc) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_WindowFunc) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{28, 0} + return fileDescriptorProcessors, []int{29, 0} } // Mode indicates which mode of framing is used. @@ -338,7 +338,7 @@ func (x *WindowerSpec_Frame_Mode) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Mode) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{28, 1, 0} + return fileDescriptorProcessors, []int{29, 1, 0} } // BoundType indicates which type of boundary is used. @@ -385,7 +385,7 @@ func (x *WindowerSpec_Frame_BoundType) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_BoundType) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{28, 1, 1} + return fileDescriptorProcessors, []int{29, 1, 1} } // Each processor has the following components: @@ -490,6 +490,7 @@ type ProcessorCoreUnion struct { ZigzagJoiner *ZigzagJoinerSpec `protobuf:"bytes,21,opt,name=zigzagJoiner" json:"zigzagJoiner,omitempty"` ProjectSet *ProjectSetSpec `protobuf:"bytes,22,opt,name=projectSet" json:"projectSet,omitempty"` Windower *WindowerSpec `protobuf:"bytes,23,opt,name=windower" json:"windower,omitempty"` + LocalPlanNode *LocalPlanNodeSpec `protobuf:"bytes,24,opt,name=localPlanNode" json:"localPlanNode,omitempty"` } func (m *ProcessorCoreUnion) Reset() { *m = ProcessorCoreUnion{} } @@ -715,6 +716,22 @@ func (m *ZigzagJoinerSpec) String() string { return proto.CompactText func (*ZigzagJoinerSpec) ProtoMessage() {} func (*ZigzagJoinerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{12} } +// LocalPlanNodeSpec is the specification for a local planNode wrapping +// processor. It's created for situations where a planNode has no DistSQL +// processor equivalent, and constrains the plan to live on the gateway node. +// This spec contains just an index, which is used by the execution engine to +// find the planNode instance this processor is executing in an array of local +// planNodes. See LocalProcessors and LocalProcessorIndexes on +// distsqlplan.PhysicalPlan. +type LocalPlanNodeSpec struct { + RowSourceIdx *uint32 `protobuf:"varint,1,opt,name=RowSourceIdx" json:"RowSourceIdx,omitempty"` +} + +func (m *LocalPlanNodeSpec) Reset() { *m = LocalPlanNodeSpec{} } +func (m *LocalPlanNodeSpec) String() string { return proto.CompactTextString(m) } +func (*LocalPlanNodeSpec) ProtoMessage() {} +func (*LocalPlanNodeSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{13} } + type Columns struct { Columns []uint32 `protobuf:"varint,1,rep,packed,name=columns" json:"columns,omitempty"` } @@ -722,7 +739,7 @@ type Columns struct { func (m *Columns) Reset() { *m = Columns{} } func (m *Columns) String() string { return proto.CompactTextString(m) } func (*Columns) ProtoMessage() {} -func (*Columns) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{13} } +func (*Columns) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{14} } // MergeJoinerSpec is the specification for a merge join processor. The processor // has two inputs and one output. The inputs must have the same ordering on the @@ -766,7 +783,7 @@ type MergeJoinerSpec struct { func (m *MergeJoinerSpec) Reset() { *m = MergeJoinerSpec{} } func (m *MergeJoinerSpec) String() string { return proto.CompactTextString(m) } func (*MergeJoinerSpec) ProtoMessage() {} -func (*MergeJoinerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{14} } +func (*MergeJoinerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{15} } // HashJoinerSpec is the specification for a hash join processor. The processor // has two inputs and one output. @@ -821,7 +838,7 @@ type HashJoinerSpec struct { func (m *HashJoinerSpec) Reset() { *m = HashJoinerSpec{} } func (m *HashJoinerSpec) String() string { return proto.CompactTextString(m) } func (*HashJoinerSpec) ProtoMessage() {} -func (*HashJoinerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{15} } +func (*HashJoinerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{16} } // AggregatorSpec is the specification for an "aggregator" (processor core // type, not the logical plan computation stage). An aggregator performs @@ -843,7 +860,7 @@ type AggregatorSpec struct { func (m *AggregatorSpec) Reset() { *m = AggregatorSpec{} } func (m *AggregatorSpec) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec) ProtoMessage() {} -func (*AggregatorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{16} } +func (*AggregatorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{17} } type AggregatorSpec_Aggregation struct { Func AggregatorSpec_Func `protobuf:"varint,1,opt,name=func,enum=cockroach.sql.distsqlrun.AggregatorSpec_Func" json:"func"` @@ -869,7 +886,7 @@ func (m *AggregatorSpec_Aggregation) Reset() { *m = AggregatorSpec_Aggre func (m *AggregatorSpec_Aggregation) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec_Aggregation) ProtoMessage() {} func (*AggregatorSpec_Aggregation) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{16, 0} + return fileDescriptorProcessors, []int{17, 0} } // BackfillerSpec is the specification for a "schema change backfiller". @@ -904,7 +921,7 @@ type BackfillerSpec struct { func (m *BackfillerSpec) Reset() { *m = BackfillerSpec{} } func (m *BackfillerSpec) String() string { return proto.CompactTextString(m) } func (*BackfillerSpec) ProtoMessage() {} -func (*BackfillerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{17} } +func (*BackfillerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{18} } // FlowSpec describes a "flow" which is a subgraph of a distributed SQL // computation consisting of processors and streams. @@ -918,7 +935,7 @@ type FlowSpec struct { func (m *FlowSpec) Reset() { *m = FlowSpec{} } func (m *FlowSpec) String() string { return proto.CompactTextString(m) } func (*FlowSpec) ProtoMessage() {} -func (*FlowSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{18} } +func (*FlowSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{19} } // JobProgress identifies the job to report progress on. This reporting // happens outside this package. @@ -934,7 +951,7 @@ type JobProgress struct { func (m *JobProgress) Reset() { *m = JobProgress{} } func (m *JobProgress) String() string { return proto.CompactTextString(m) } func (*JobProgress) ProtoMessage() {} -func (*JobProgress) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{19} } +func (*JobProgress) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{20} } type ReadImportDataSpec struct { Format cockroach_roachpb4.IOFileFormat `protobuf:"bytes,8,opt,name=format" json:"format"` @@ -962,7 +979,7 @@ type ReadImportDataSpec struct { func (m *ReadImportDataSpec) Reset() { *m = ReadImportDataSpec{} } func (m *ReadImportDataSpec) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec) ProtoMessage() {} -func (*ReadImportDataSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{20} } +func (*ReadImportDataSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{21} } // SSTWriterSpec is the specification for a processor that consumes rows, uses // tempStorage to sort them, then writes them to SST files at uri. walltime is @@ -983,7 +1000,7 @@ type SSTWriterSpec struct { func (m *SSTWriterSpec) Reset() { *m = SSTWriterSpec{} } func (m *SSTWriterSpec) String() string { return proto.CompactTextString(m) } func (*SSTWriterSpec) ProtoMessage() {} -func (*SSTWriterSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{21} } +func (*SSTWriterSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{22} } type SSTWriterSpec_SpanName struct { // name is the file name that will be written by the export store. @@ -996,7 +1013,7 @@ func (m *SSTWriterSpec_SpanName) Reset() { *m = SSTWriterSpec_SpanName{} func (m *SSTWriterSpec_SpanName) String() string { return proto.CompactTextString(m) } func (*SSTWriterSpec_SpanName) ProtoMessage() {} func (*SSTWriterSpec_SpanName) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{21, 0} + return fileDescriptorProcessors, []int{22, 0} } // CSVWriterSpec is the specification for a processor that consumes rows and @@ -1015,7 +1032,7 @@ type CSVWriterSpec struct { func (m *CSVWriterSpec) Reset() { *m = CSVWriterSpec{} } func (m *CSVWriterSpec) String() string { return proto.CompactTextString(m) } func (*CSVWriterSpec) ProtoMessage() {} -func (*CSVWriterSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{22} } +func (*CSVWriterSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{23} } // SketchSpec contains the specification for a generated statistic. type SketchSpec struct { @@ -1036,7 +1053,7 @@ type SketchSpec struct { func (m *SketchSpec) Reset() { *m = SketchSpec{} } func (m *SketchSpec) String() string { return proto.CompactTextString(m) } func (*SketchSpec) ProtoMessage() {} -func (*SketchSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{23} } +func (*SketchSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{24} } // SamplerSpec is the specification of a "sampler" processor which // returns a sample (random subset) of the input columns and computes @@ -1077,7 +1094,7 @@ type SamplerSpec struct { func (m *SamplerSpec) Reset() { *m = SamplerSpec{} } func (m *SamplerSpec) String() string { return proto.CompactTextString(m) } func (*SamplerSpec) ProtoMessage() {} -func (*SamplerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{24} } +func (*SamplerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{25} } // SampleAggregatorSpec is the specification of a processor that aggregates the // results from multiple sampler processors and writes out the statistics to @@ -1108,7 +1125,7 @@ type SampleAggregatorSpec struct { func (m *SampleAggregatorSpec) Reset() { *m = SampleAggregatorSpec{} } func (m *SampleAggregatorSpec) String() string { return proto.CompactTextString(m) } func (*SampleAggregatorSpec) ProtoMessage() {} -func (*SampleAggregatorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{25} } +func (*SampleAggregatorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{26} } // InterleavedReaderJoinerSpec is the specification for a processor that performs // KV operations to retrieve rows from 2+ tables from an interleaved hierarchy, @@ -1148,7 +1165,7 @@ func (m *InterleavedReaderJoinerSpec) Reset() { *m = InterleavedReaderJo func (m *InterleavedReaderJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec) ProtoMessage() {} func (*InterleavedReaderJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{26} + return fileDescriptorProcessors, []int{27} } type InterleavedReaderJoinerSpec_Table struct { @@ -1182,7 +1199,7 @@ func (m *InterleavedReaderJoinerSpec_Table) Reset() { *m = InterleavedRe func (m *InterleavedReaderJoinerSpec_Table) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec_Table) ProtoMessage() {} func (*InterleavedReaderJoinerSpec_Table) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{26, 0} + return fileDescriptorProcessors, []int{27, 0} } // ProjectSetSpec is the specification of a processor which applies a set of @@ -1199,7 +1216,7 @@ type ProjectSetSpec struct { func (m *ProjectSetSpec) Reset() { *m = ProjectSetSpec{} } func (m *ProjectSetSpec) String() string { return proto.CompactTextString(m) } func (*ProjectSetSpec) ProtoMessage() {} -func (*ProjectSetSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{27} } +func (*ProjectSetSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{28} } // WindowerSpec is the specification of a processor that performs computations // of window functions that have the same PARTITION BY clause. For a particular @@ -1217,7 +1234,7 @@ type WindowerSpec struct { func (m *WindowerSpec) Reset() { *m = WindowerSpec{} } func (m *WindowerSpec) String() string { return proto.CompactTextString(m) } func (*WindowerSpec) ProtoMessage() {} -func (*WindowerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{28} } +func (*WindowerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{29} } // Func specifies which function to compute. It can either be built-in // aggregate or built-in window function. @@ -1229,7 +1246,7 @@ type WindowerSpec_Func struct { func (m *WindowerSpec_Func) Reset() { *m = WindowerSpec_Func{} } func (m *WindowerSpec_Func) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Func) ProtoMessage() {} -func (*WindowerSpec_Func) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{28, 0} } +func (*WindowerSpec_Func) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{29, 0} } // Frame is the specification of a single window frame for a window function. type WindowerSpec_Frame struct { @@ -1240,7 +1257,7 @@ type WindowerSpec_Frame struct { func (m *WindowerSpec_Frame) Reset() { *m = WindowerSpec_Frame{} } func (m *WindowerSpec_Frame) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame) ProtoMessage() {} -func (*WindowerSpec_Frame) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{28, 1} } +func (*WindowerSpec_Frame) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{29, 1} } // Bound specifies the type of boundary and the offset (if present). type WindowerSpec_Frame_Bound struct { @@ -1258,7 +1275,7 @@ func (m *WindowerSpec_Frame_Bound) Reset() { *m = WindowerSpec_Frame_Bou func (m *WindowerSpec_Frame_Bound) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bound) ProtoMessage() {} func (*WindowerSpec_Frame_Bound) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{28, 1, 0} + return fileDescriptorProcessors, []int{29, 1, 0} } // Bounds specifies boundaries of the window frame. @@ -1272,7 +1289,7 @@ func (m *WindowerSpec_Frame_Bounds) Reset() { *m = WindowerSpec_Frame_Bo func (m *WindowerSpec_Frame_Bounds) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bounds) ProtoMessage() {} func (*WindowerSpec_Frame_Bounds) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{28, 1, 1} + return fileDescriptorProcessors, []int{29, 1, 1} } // WindowFn is the specification of a single window function. @@ -1298,7 +1315,7 @@ func (m *WindowerSpec_WindowFn) Reset() { *m = WindowerSpec_WindowFn{} } func (m *WindowerSpec_WindowFn) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_WindowFn) ProtoMessage() {} func (*WindowerSpec_WindowFn) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{28, 2} + return fileDescriptorProcessors, []int{29, 2} } func init() { @@ -1315,6 +1332,7 @@ func init() { proto.RegisterType((*SorterSpec)(nil), "cockroach.sql.distsqlrun.SorterSpec") proto.RegisterType((*DistinctSpec)(nil), "cockroach.sql.distsqlrun.DistinctSpec") proto.RegisterType((*ZigzagJoinerSpec)(nil), "cockroach.sql.distsqlrun.ZigzagJoinerSpec") + proto.RegisterType((*LocalPlanNodeSpec)(nil), "cockroach.sql.distsqlrun.LocalPlanNodeSpec") proto.RegisterType((*Columns)(nil), "cockroach.sql.distsqlrun.Columns") proto.RegisterType((*MergeJoinerSpec)(nil), "cockroach.sql.distsqlrun.MergeJoinerSpec") proto.RegisterType((*HashJoinerSpec)(nil), "cockroach.sql.distsqlrun.HashJoinerSpec") @@ -1721,6 +1739,18 @@ func (m *ProcessorCoreUnion) MarshalTo(dAtA []byte) (int, error) { } i += n26 } + if m.LocalPlanNode != nil { + dAtA[i] = 0xc2 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.LocalPlanNode.Size())) + n27, err := m.LocalPlanNode.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n27 + } return i, nil } @@ -1856,11 +1886,11 @@ func (m *TableReaderSpan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Span.Size())) - n27, err := m.Span.MarshalTo(dAtA[i:]) + n28, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n28 return i, nil } @@ -1882,11 +1912,11 @@ func (m *TableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Table.Size())) - n28, err := m.Table.MarshalTo(dAtA[i:]) + n29, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n29 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexIdx)) @@ -1942,47 +1972,47 @@ func (m *JoinReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Table.Size())) - n29, err := m.Table.MarshalTo(dAtA[i:]) + n30, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n30 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexIdx)) if len(m.LookupColumns) > 0 { - dAtA31 := make([]byte, len(m.LookupColumns)*10) - var j30 int + dAtA32 := make([]byte, len(m.LookupColumns)*10) + var j31 int for _, num := range m.LookupColumns { for num >= 1<<7 { - dAtA31[j30] = uint8(uint64(num)&0x7f | 0x80) + dAtA32[j31] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j30++ + j31++ } - dAtA31[j30] = uint8(num) - j30++ + dAtA32[j31] = uint8(num) + j31++ } dAtA[i] = 0x1a i++ - i = encodeVarintProcessors(dAtA, i, uint64(j30)) - i += copy(dAtA[i:], dAtA31[:j30]) + i = encodeVarintProcessors(dAtA, i, uint64(j31)) + i += copy(dAtA[i:], dAtA32[:j31]) } dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n32, err := m.OnExpr.MarshalTo(dAtA[i:]) + n33, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n33 dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexFilterExpr.Size())) - n33, err := m.IndexFilterExpr.MarshalTo(dAtA[i:]) + n34, err := m.IndexFilterExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n34 dAtA[i] = 0x30 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2007,11 +2037,11 @@ func (m *SorterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OutputOrdering.Size())) - n34, err := m.OutputOrdering.MarshalTo(dAtA[i:]) + n35, err := m.OutputOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n35 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OrderingMatchLen)) @@ -2090,36 +2120,59 @@ func (m *ZigzagJoinerSpec) MarshalTo(dAtA []byte) (int, error) { } } if len(m.IndexIds) > 0 { - dAtA36 := make([]byte, len(m.IndexIds)*10) - var j35 int + dAtA37 := make([]byte, len(m.IndexIds)*10) + var j36 int for _, num := range m.IndexIds { for num >= 1<<7 { - dAtA36[j35] = uint8(uint64(num)&0x7f | 0x80) + dAtA37[j36] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j35++ + j36++ } - dAtA36[j35] = uint8(num) - j35++ + dAtA37[j36] = uint8(num) + j36++ } dAtA[i] = 0x1a i++ - i = encodeVarintProcessors(dAtA, i, uint64(j35)) - i += copy(dAtA[i:], dAtA36[:j35]) + i = encodeVarintProcessors(dAtA, i, uint64(j36)) + i += copy(dAtA[i:], dAtA37[:j36]) } dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n37, err := m.OnExpr.MarshalTo(dAtA[i:]) + n38, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n37 + i += n38 dAtA[i] = 0x28 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) return i, nil } +func (m *LocalPlanNodeSpec) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LocalPlanNodeSpec) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.RowSourceIdx != nil { + dAtA[i] = 0x8 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(*m.RowSourceIdx)) + } + return i, nil +} + func (m *Columns) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -2136,21 +2189,21 @@ func (m *Columns) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.Columns) > 0 { - dAtA39 := make([]byte, len(m.Columns)*10) - var j38 int + dAtA40 := make([]byte, len(m.Columns)*10) + var j39 int for _, num := range m.Columns { for num >= 1<<7 { - dAtA39[j38] = uint8(uint64(num)&0x7f | 0x80) + dAtA40[j39] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j38++ + j39++ } - dAtA39[j38] = uint8(num) - j38++ + dAtA40[j39] = uint8(num) + j39++ } dAtA[i] = 0xa i++ - i = encodeVarintProcessors(dAtA, i, uint64(j38)) - i += copy(dAtA[i:], dAtA39[:j38]) + i = encodeVarintProcessors(dAtA, i, uint64(j39)) + i += copy(dAtA[i:], dAtA40[:j39]) } return i, nil } @@ -2173,27 +2226,27 @@ func (m *MergeJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.LeftOrdering.Size())) - n40, err := m.LeftOrdering.MarshalTo(dAtA[i:]) + n41, err := m.LeftOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n40 + i += n41 dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.RightOrdering.Size())) - n41, err := m.RightOrdering.MarshalTo(dAtA[i:]) + n42, err := m.RightOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n42 dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n42, err := m.OnExpr.MarshalTo(dAtA[i:]) + n43, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n43 dAtA[i] = 0x30 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2224,47 +2277,47 @@ func (m *HashJoinerSpec) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.LeftEqColumns) > 0 { - dAtA44 := make([]byte, len(m.LeftEqColumns)*10) - var j43 int + dAtA45 := make([]byte, len(m.LeftEqColumns)*10) + var j44 int for _, num := range m.LeftEqColumns { for num >= 1<<7 { - dAtA44[j43] = uint8(uint64(num)&0x7f | 0x80) + dAtA45[j44] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j43++ + j44++ } - dAtA44[j43] = uint8(num) - j43++ + dAtA45[j44] = uint8(num) + j44++ } dAtA[i] = 0xa i++ - i = encodeVarintProcessors(dAtA, i, uint64(j43)) - i += copy(dAtA[i:], dAtA44[:j43]) + i = encodeVarintProcessors(dAtA, i, uint64(j44)) + i += copy(dAtA[i:], dAtA45[:j44]) } if len(m.RightEqColumns) > 0 { - dAtA46 := make([]byte, len(m.RightEqColumns)*10) - var j45 int + dAtA47 := make([]byte, len(m.RightEqColumns)*10) + var j46 int for _, num := range m.RightEqColumns { for num >= 1<<7 { - dAtA46[j45] = uint8(uint64(num)&0x7f | 0x80) + dAtA47[j46] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j45++ + j46++ } - dAtA46[j45] = uint8(num) - j45++ + dAtA47[j46] = uint8(num) + j46++ } dAtA[i] = 0x12 i++ - i = encodeVarintProcessors(dAtA, i, uint64(j45)) - i += copy(dAtA[i:], dAtA46[:j45]) + i = encodeVarintProcessors(dAtA, i, uint64(j46)) + i += copy(dAtA[i:], dAtA47[:j46]) } dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n47, err := m.OnExpr.MarshalTo(dAtA[i:]) + n48, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n47 + i += n48 dAtA[i] = 0x30 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2295,21 +2348,21 @@ func (m *AggregatorSpec) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.GroupCols) > 0 { - dAtA49 := make([]byte, len(m.GroupCols)*10) - var j48 int + dAtA50 := make([]byte, len(m.GroupCols)*10) + var j49 int for _, num := range m.GroupCols { for num >= 1<<7 { - dAtA49[j48] = uint8(uint64(num)&0x7f | 0x80) + dAtA50[j49] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j48++ + j49++ } - dAtA49[j48] = uint8(num) - j48++ + dAtA50[j49] = uint8(num) + j49++ } dAtA[i] = 0x12 i++ - i = encodeVarintProcessors(dAtA, i, uint64(j48)) - i += copy(dAtA[i:], dAtA49[:j48]) + i = encodeVarintProcessors(dAtA, i, uint64(j49)) + i += copy(dAtA[i:], dAtA50[:j49]) } if len(m.Aggregations) > 0 { for _, msg := range m.Aggregations { @@ -2324,21 +2377,21 @@ func (m *AggregatorSpec) MarshalTo(dAtA []byte) (int, error) { } } if len(m.OrderedGroupCols) > 0 { - dAtA51 := make([]byte, len(m.OrderedGroupCols)*10) - var j50 int + dAtA52 := make([]byte, len(m.OrderedGroupCols)*10) + var j51 int for _, num := range m.OrderedGroupCols { for num >= 1<<7 { - dAtA51[j50] = uint8(uint64(num)&0x7f | 0x80) + dAtA52[j51] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j50++ + j51++ } - dAtA51[j50] = uint8(num) - j50++ + dAtA52[j51] = uint8(num) + j51++ } dAtA[i] = 0x22 i++ - i = encodeVarintProcessors(dAtA, i, uint64(j50)) - i += copy(dAtA[i:], dAtA51[:j50]) + i = encodeVarintProcessors(dAtA, i, uint64(j51)) + i += copy(dAtA[i:], dAtA52[:j51]) } dAtA[i] = 0x28 i++ @@ -2408,11 +2461,11 @@ func (m *BackfillerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Table.Size())) - n52, err := m.Table.MarshalTo(dAtA[i:]) + n53, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n53 if len(m.Spans) > 0 { for _, msg := range m.Spans { dAtA[i] = 0x1a @@ -2446,11 +2499,11 @@ func (m *BackfillerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.ReadAsOf.Size())) - n53, err := m.ReadAsOf.MarshalTo(dAtA[i:]) + n54, err := m.ReadAsOf.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n54 return i, nil } @@ -2472,11 +2525,11 @@ func (m *FlowSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.FlowID.Size())) - n54, err := m.FlowID.MarshalTo(dAtA[i:]) + n55, err := m.FlowID.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n55 if len(m.Processors) > 0 { for _, msg := range m.Processors { dAtA[i] = 0x12 @@ -2544,11 +2597,11 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Progress.Size())) - n55, err := m.Progress.MarshalTo(dAtA[i:]) + n56, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n56 if len(m.Uri) > 0 { keysForUri := make([]int32, 0, len(m.Uri)) for k := range m.Uri { @@ -2573,11 +2626,11 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Format.Size())) - n56, err := m.Format.MarshalTo(dAtA[i:]) + n57, err := m.Format.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n57 if len(m.Tables) > 0 { keysForTables := make([]string, 0, len(m.Tables)) for k := range m.Tables { @@ -2603,11 +2656,11 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(v.Size())) - n57, err := v.MarshalTo(dAtA[i:]) + n58, err := v.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n58 } } } @@ -2659,11 +2712,11 @@ func (m *SSTWriterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Progress.Size())) - n58, err := m.Progress.MarshalTo(dAtA[i:]) + n59, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n58 + i += n59 return i, nil } @@ -2721,11 +2774,11 @@ func (m *CSVWriterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Options.Size())) - n59, err := m.Options.MarshalTo(dAtA[i:]) + n60, err := m.Options.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n59 + i += n60 dAtA[i] = 0x20 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.ChunkRows)) @@ -2892,11 +2945,11 @@ func (m *InterleavedReaderJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n60, err := m.OnExpr.MarshalTo(dAtA[i:]) + n61, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n60 + i += n61 dAtA[i] = 0x28 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2921,30 +2974,30 @@ func (m *InterleavedReaderJoinerSpec_Table) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Desc.Size())) - n61, err := m.Desc.MarshalTo(dAtA[i:]) + n62, err := m.Desc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n61 + i += n62 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexIdx)) dAtA[i] = 0x1a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Post.Size())) - n62, err := m.Post.MarshalTo(dAtA[i:]) + n63, err := m.Post.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n62 + i += n63 dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Ordering.Size())) - n63, err := m.Ordering.MarshalTo(dAtA[i:]) + n64, err := m.Ordering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n63 + i += n64 if len(m.Spans) > 0 { for _, msg := range m.Spans { dAtA[i] = 0x2a @@ -3095,11 +3148,11 @@ func (m *WindowerSpec_Frame) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Bounds.Size())) - n64, err := m.Bounds.MarshalTo(dAtA[i:]) + n65, err := m.Bounds.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n64 + i += n65 return i, nil } @@ -3133,11 +3186,11 @@ func (m *WindowerSpec_Frame_Bound) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OffsetType.Size())) - n65, err := m.OffsetType.MarshalTo(dAtA[i:]) + n66, err := m.OffsetType.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n65 + i += n66 return i, nil } @@ -3159,20 +3212,20 @@ func (m *WindowerSpec_Frame_Bounds) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Start.Size())) - n66, err := m.Start.MarshalTo(dAtA[i:]) + n67, err := m.Start.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n66 + i += n67 if m.End != nil { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.End.Size())) - n67, err := m.End.MarshalTo(dAtA[i:]) + n68, err := m.End.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n67 + i += n68 } return i, nil } @@ -3195,11 +3248,11 @@ func (m *WindowerSpec_WindowFn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Func.Size())) - n68, err := m.Func.MarshalTo(dAtA[i:]) + n69, err := m.Func.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n68 + i += n69 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.ArgIdxStart)) @@ -3209,20 +3262,20 @@ func (m *WindowerSpec_WindowFn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Ordering.Size())) - n69, err := m.Ordering.MarshalTo(dAtA[i:]) + n70, err := m.Ordering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n69 + i += n70 if m.Frame != nil { dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Frame.Size())) - n70, err := m.Frame.MarshalTo(dAtA[i:]) + n71, err := m.Frame.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n70 + i += n71 } dAtA[i] = 0x30 i++ @@ -3374,6 +3427,10 @@ func (m *ProcessorCoreUnion) Size() (n int) { l = m.Windower.Size() n += 2 + l + sovProcessors(uint64(l)) } + if m.LocalPlanNode != nil { + l = m.LocalPlanNode.Size() + n += 2 + l + sovProcessors(uint64(l)) + } return n } @@ -3522,6 +3579,15 @@ func (m *ZigzagJoinerSpec) Size() (n int) { return n } +func (m *LocalPlanNodeSpec) Size() (n int) { + var l int + _ = l + if m.RowSourceIdx != nil { + n += 1 + sovProcessors(uint64(*m.RowSourceIdx)) + } + return n +} + func (m *Columns) Size() (n int) { var l int _ = l @@ -4005,6 +4071,9 @@ func (this *ProcessorCoreUnion) GetValue() interface{} { if this.Windower != nil { return this.Windower } + if this.LocalPlanNode != nil { + return this.LocalPlanNode + } return nil } @@ -4052,6 +4121,8 @@ func (this *ProcessorCoreUnion) SetValue(value interface{}) bool { this.ProjectSet = vt case *WindowerSpec: this.Windower = vt + case *LocalPlanNodeSpec: + this.LocalPlanNode = vt default: return false } @@ -5241,6 +5312,39 @@ func (m *ProcessorCoreUnion) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 24: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LocalPlanNode", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.LocalPlanNode == nil { + m.LocalPlanNode = &LocalPlanNodeSpec{} + } + if err := m.LocalPlanNode.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProcessors(dAtA[iNdEx:]) @@ -6604,6 +6708,76 @@ func (m *ZigzagJoinerSpec) Unmarshal(dAtA []byte) error { } return nil } +func (m *LocalPlanNodeSpec) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LocalPlanNodeSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LocalPlanNodeSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowSourceIdx", wireType) + } + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.RowSourceIdx = &v + default: + iNdEx = preIndex + skippy, err := skipProcessors(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthProcessors + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *Columns) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -10711,232 +10885,235 @@ var ( func init() { proto.RegisterFile("sql/distsqlrun/processors.proto", fileDescriptorProcessors) } var fileDescriptorProcessors = []byte{ - // 3622 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xcd, 0x6f, 0x1b, 0x59, - 0x72, 0x37, 0x9b, 0xcd, 0xaf, 0xe2, 0x87, 0xda, 0xcf, 0x9e, 0x31, 0x57, 0x33, 0xb1, 0xec, 0x5e, - 0xef, 0xf8, 0x63, 0x67, 0xa5, 0xb1, 0x66, 0x77, 0x30, 0x3b, 0xbb, 0x8b, 0x19, 0x7e, 0xca, 0x94, - 0x29, 0xd2, 0x6e, 0x92, 0xf2, 0xce, 0x1c, 0xd2, 0x69, 0xb1, 0x9f, 0xa8, 0x1e, 0x35, 0xfb, 0xd1, - 0xdd, 0x4d, 0xcb, 0x32, 0x02, 0xe4, 0x94, 0x73, 0x02, 0x64, 0x83, 0xdc, 0x82, 0x5c, 0xf6, 0x9c, - 0x4b, 0x0e, 0xb9, 0x24, 0x40, 0x80, 0x1c, 0x7c, 0x4b, 0x72, 0x5b, 0x24, 0x81, 0x91, 0x78, 0x4f, - 0xf9, 0x0b, 0x02, 0xf8, 0x10, 0x04, 0xef, 0xab, 0xd9, 0xd4, 0x37, 0xed, 0xc5, 0xce, 0x45, 0x50, - 0xd7, 0xc7, 0xaf, 0xeb, 0xd5, 0xab, 0x57, 0x55, 0xaf, 0x9a, 0xb0, 0x12, 0x3c, 0x75, 0xd7, 0x6c, - 0x27, 0x08, 0x83, 0xa7, 0xae, 0x3f, 0xf5, 0xd6, 0x26, 0x3e, 0x19, 0xe2, 0x20, 0x20, 0x7e, 0xb0, - 0x3a, 0xf1, 0x49, 0x48, 0x50, 0x79, 0x48, 0x86, 0xfb, 0x3e, 0xb1, 0x86, 0x7b, 0xab, 0xc1, 0x53, - 0x77, 0x75, 0x26, 0xba, 0x7c, 0x99, 0x51, 0x27, 0x3b, 0x6b, 0xd6, 0xc4, 0xe1, 0xc2, 0xcb, 0x48, - 0x92, 0x6c, 0x2b, 0xb4, 0x04, 0xed, 0xaa, 0xa4, 0x61, 0xdf, 0x8f, 0x60, 0x97, 0xcb, 0x92, 0xea, - 0x90, 0x1f, 0xed, 0x12, 0x7f, 0x6c, 0x85, 0x92, 0xf3, 0x21, 0xb5, 0x28, 0x78, 0xea, 0xee, 0x58, - 0x01, 0x5e, 0x0b, 0x42, 0x7f, 0x3a, 0x0c, 0xa7, 0x3e, 0xb6, 0x05, 0x77, 0x25, 0xce, 0xc5, 0xde, - 0x90, 0xd8, 0xd8, 0x36, 0x6d, 0x2b, 0x9c, 0x8e, 0x85, 0xc0, 0x07, 0x71, 0x81, 0x6f, 0x89, 0xe3, - 0x99, 0xe1, 0xe1, 0x04, 0x0b, 0xe6, 0xf7, 0x8e, 0xac, 0x36, 0x66, 0x66, 0x79, 0x1a, 0x3a, 0xee, - 0xda, 0x9e, 0x3b, 0x5c, 0x0b, 0x9d, 0x31, 0x0e, 0x42, 0x6b, 0x3c, 0x91, 0x0b, 0x18, 0x91, 0x11, - 0x61, 0xff, 0xae, 0xd1, 0xff, 0x38, 0x55, 0xff, 0xb3, 0x24, 0x14, 0x1f, 0x49, 0x67, 0xf5, 0x26, - 0x78, 0x88, 0x6a, 0x90, 0x72, 0xbc, 0xc9, 0x34, 0x2c, 0x27, 0x6e, 0x24, 0xef, 0xe4, 0xd7, 0x6f, - 0xaf, 0x9e, 0xe6, 0xb9, 0xd5, 0x16, 0x15, 0xeb, 0x1d, 0x7a, 0x43, 0xaa, 0x57, 0x55, 0x5f, 0xbe, - 0x5a, 0xb9, 0x64, 0x70, 0x5d, 0xd4, 0x04, 0x75, 0x48, 0x7c, 0x5c, 0x56, 0x6e, 0x24, 0xee, 0xe4, - 0xd7, 0x3f, 0x3e, 0x1d, 0x23, 0x7a, 0x77, 0x8d, 0xf8, 0x78, 0xe0, 0x39, 0xc4, 0x13, 0x40, 0x4c, - 0x1f, 0x3d, 0x80, 0x34, 0x99, 0x86, 0xd4, 0x9a, 0x24, 0xb3, 0xe6, 0xde, 0xe9, 0x48, 0x5d, 0x26, - 0x67, 0x90, 0x69, 0x88, 0xfd, 0x98, 0x41, 0x42, 0x1f, 0xd5, 0x40, 0x9d, 0x90, 0x20, 0x2c, 0xab, - 0xcc, 0xa2, 0xbb, 0x67, 0x58, 0x44, 0x82, 0x50, 0x58, 0x15, 0x83, 0x61, 0xca, 0xe8, 0x1e, 0x64, - 0x83, 0xd0, 0x1a, 0x61, 0xd3, 0xb1, 0xcb, 0xa9, 0x1b, 0x89, 0x3b, 0xa9, 0xea, 0x12, 0xe5, 0xbe, - 0x7e, 0xb5, 0x92, 0xe9, 0x51, 0x7a, 0xab, 0x6e, 0x64, 0x98, 0x40, 0xcb, 0x46, 0x9f, 0x41, 0x21, - 0x8a, 0x42, 0x2a, 0x9f, 0x66, 0xf2, 0x57, 0x84, 0x7c, 0x3e, 0x5a, 0x78, 0xab, 0x6e, 0xe4, 0x23, - 0xc1, 0x96, 0xad, 0xff, 0xad, 0x02, 0x4b, 0x47, 0x6c, 0x40, 0x55, 0x48, 0xef, 0x3a, 0x6e, 0x88, - 0xfd, 0x72, 0x82, 0x99, 0x7f, 0xeb, 0x74, 0xf3, 0x1b, 0xcf, 0x27, 0x3e, 0x0e, 0x82, 0x99, 0x23, - 0x85, 0x26, 0xba, 0x05, 0x30, 0xf1, 0xc9, 0xb7, 0x78, 0x18, 0x3a, 0xc4, 0x63, 0x1b, 0x93, 0x15, - 0x12, 0x31, 0x3a, 0xba, 0x0b, 0x25, 0xee, 0x30, 0x73, 0x48, 0xdc, 0xe9, 0xd8, 0x0b, 0x98, 0xe3, - 0x8b, 0x55, 0x45, 0x4b, 0x18, 0x45, 0xce, 0xa9, 0x71, 0x06, 0xda, 0x82, 0x82, 0x8f, 0x3d, 0x1b, - 0xfb, 0x26, 0x7e, 0x3e, 0xf1, 0x83, 0xb2, 0xca, 0x76, 0x68, 0x11, 0xd3, 0xf2, 0x5c, 0x9f, 0xd2, - 0x03, 0xf4, 0x21, 0xa4, 0xc9, 0xee, 0x6e, 0x80, 0x43, 0xe6, 0x59, 0x35, 0xda, 0x3e, 0x46, 0x43, - 0xcb, 0x90, 0x72, 0x9d, 0xb1, 0x13, 0x32, 0x37, 0x4a, 0x26, 0x27, 0xe9, 0x6f, 0x0a, 0x80, 0x8e, - 0xc7, 0x11, 0xfa, 0x02, 0x54, 0x8f, 0x90, 0x89, 0x70, 0xd9, 0x47, 0xa7, 0xdb, 0xd5, 0x21, 0x64, - 0x42, 0xd5, 0xa8, 0xab, 0x0d, 0xa6, 0x83, 0x1e, 0x42, 0x3e, 0xb4, 0x76, 0x5c, 0x6c, 0x60, 0xcb, - 0xc6, 0xbe, 0x08, 0xe3, 0x33, 0x82, 0xa6, 0x3f, 0x13, 0x66, 0x28, 0x71, 0x6d, 0xf4, 0x00, 0x80, - 0x9e, 0x60, 0x81, 0x95, 0x64, 0x58, 0x77, 0x4e, 0xc7, 0xda, 0x8c, 0x64, 0x19, 0x54, 0x4c, 0x17, - 0xfd, 0x1c, 0xd2, 0x01, 0xf1, 0x69, 0x1c, 0xa8, 0xe7, 0xc5, 0x41, 0x8f, 0xc9, 0x31, 0x04, 0xa1, - 0x43, 0xed, 0xb0, 0x46, 0x23, 0x1f, 0x8f, 0xac, 0x90, 0xf8, 0xcc, 0xcb, 0x67, 0xda, 0x51, 0x89, - 0x64, 0xb9, 0x1d, 0x33, 0x5d, 0x54, 0x85, 0x2c, 0x15, 0x74, 0xbc, 0x61, 0x58, 0xce, 0x9c, 0xe7, - 0xde, 0xba, 0x90, 0x64, 0x28, 0x91, 0x1e, 0x75, 0xf1, 0x18, 0xfb, 0x23, 0x4c, 0x97, 0x8b, 0xfd, - 0x72, 0xf6, 0x3c, 0x17, 0x6f, 0xcd, 0x84, 0xb9, 0x8b, 0x63, 0xda, 0x74, 0x69, 0x7b, 0x56, 0xb0, - 0x27, 0xb0, 0x72, 0xe7, 0x2d, 0xed, 0x41, 0x24, 0xcb, 0x97, 0x36, 0xd3, 0x45, 0x5f, 0x41, 0xfa, - 0x99, 0xe5, 0x4e, 0x71, 0x50, 0x86, 0xf3, 0x50, 0xb6, 0x99, 0x5c, 0x14, 0x39, 0x42, 0x8f, 0xda, - 0xb2, 0x63, 0x0d, 0xf7, 0x77, 0x1d, 0xd7, 0xc5, 0x7e, 0x39, 0x7f, 0x1e, 0x4a, 0x35, 0x92, 0xe5, - 0xb6, 0xcc, 0x74, 0x51, 0x1b, 0xc0, 0xc7, 0x96, 0xdd, 0x1a, 0x4f, 0x88, 0x1f, 0x96, 0x8b, 0xe7, - 0xe5, 0x52, 0x23, 0x92, 0xad, 0x5b, 0xa1, 0xc5, 0xd1, 0x66, 0xfa, 0xa8, 0x01, 0xb9, 0x5e, 0xaf, - 0xff, 0xc4, 0x77, 0x68, 0xfc, 0x94, 0x18, 0xd8, 0x19, 0xc9, 0x3d, 0x12, 0x65, 0x38, 0x33, 0x4d, - 0xf4, 0x25, 0x64, 0x7a, 0xd6, 0x78, 0x42, 0xd7, 0xb6, 0xc4, 0x40, 0x7e, 0x70, 0x06, 0x08, 0x17, - 0x64, 0x10, 0x52, 0x0b, 0x7d, 0x03, 0x1a, 0xff, 0x77, 0x16, 0x60, 0x65, 0x8d, 0x21, 0xad, 0x9e, - 0x87, 0x74, 0x24, 0x24, 0x8f, 0xe1, 0x20, 0x02, 0xd7, 0x1c, 0x2f, 0xc4, 0xbe, 0x8b, 0xad, 0x67, - 0xd8, 0xe6, 0xa7, 0x46, 0x04, 0xc5, 0x65, 0xf6, 0x8a, 0x9f, 0x9c, 0x55, 0xce, 0x4e, 0x54, 0x64, - 0x6f, 0x3a, 0x0d, 0x15, 0xfd, 0x11, 0xa0, 0x31, 0x0e, 0x2d, 0x5a, 0x81, 0xfb, 0x38, 0x08, 0x7b, - 0x2c, 0xa1, 0x95, 0x11, 0x7b, 0xd7, 0x27, 0x67, 0x05, 0xf3, 0x51, 0x1d, 0xf6, 0x9a, 0x13, 0xb0, - 0xd0, 0x2e, 0x5c, 0x8d, 0x53, 0x0d, 0x3c, 0xc4, 0xce, 0x33, 0xec, 0x97, 0xaf, 0xb0, 0x77, 0xac, - 0x5f, 0xec, 0x1d, 0x52, 0x8b, 0xbd, 0xe5, 0x44, 0x3c, 0x1a, 0x1e, 0xb5, 0xde, 0xb6, 0x08, 0x8f, - 0xab, 0xe7, 0x85, 0x47, 0x24, 0xca, 0xc3, 0x23, 0x7a, 0x44, 0x1d, 0x28, 0xbc, 0x70, 0x46, 0x2f, - 0xac, 0x91, 0x70, 0xfb, 0x7b, 0x0c, 0xe9, 0x8c, 0xba, 0xfd, 0x4d, 0x4c, 0x9a, 0x81, 0xcd, 0xe9, - 0xd3, 0xd3, 0x24, 0xca, 0x53, 0x0f, 0x87, 0xe5, 0xf7, 0xcf, 0x3b, 0x4d, 0x8f, 0x22, 0x59, 0x1e, - 0xff, 0x33, 0x5d, 0x9a, 0xb4, 0x0e, 0x1c, 0xcf, 0x26, 0x07, 0xd8, 0x2f, 0x5f, 0x3b, 0x2f, 0x69, - 0x3d, 0x11, 0x92, 0x3c, 0x69, 0x49, 0xbd, 0x2f, 0xd4, 0x97, 0x7f, 0xb3, 0x92, 0xd8, 0x54, 0xb3, - 0x69, 0x2d, 0xb3, 0xa9, 0x66, 0x0b, 0x5a, 0x51, 0x2f, 0x41, 0x21, 0x5e, 0x3f, 0xf4, 0x1f, 0xc3, - 0xfb, 0x27, 0x6f, 0x2e, 0x5a, 0x06, 0xc5, 0xb1, 0x59, 0x35, 0xca, 0x55, 0x41, 0xb4, 0x01, 0x4a, - 0xab, 0x6e, 0x28, 0x8e, 0xad, 0x3f, 0x80, 0xf2, 0x69, 0xdb, 0x85, 0x3e, 0x06, 0x08, 0x78, 0x9d, - 0x75, 0xec, 0x80, 0x75, 0x65, 0xb9, 0x6a, 0xf1, 0xf5, 0xab, 0x95, 0x1c, 0xc7, 0x6e, 0xd5, 0x03, - 0x23, 0xc7, 0x05, 0x5a, 0x76, 0xa0, 0xff, 0x45, 0x02, 0x4a, 0xf3, 0x89, 0x09, 0xd5, 0x20, 0x23, - 0x8b, 0x39, 0xef, 0xe9, 0xbe, 0x7f, 0x46, 0xb2, 0xa6, 0x3d, 0x68, 0xcb, 0xdb, 0x25, 0xa2, 0xc4, - 0x4a, 0x4d, 0xf4, 0x01, 0xe4, 0x7c, 0xeb, 0xc0, 0xdc, 0x39, 0x0c, 0x71, 0x50, 0x56, 0x6e, 0x24, - 0xef, 0x14, 0x8c, 0xac, 0x6f, 0x1d, 0x54, 0xe9, 0x33, 0x5a, 0x81, 0xac, 0x37, 0x1d, 0x9b, 0x3e, - 0x39, 0x08, 0x58, 0x7d, 0x93, 0x05, 0x3a, 0xe3, 0x4d, 0xc7, 0x06, 0x39, 0x08, 0xf4, 0x3a, 0x2c, - 0xcd, 0x95, 0x48, 0xcb, 0x43, 0xf7, 0x41, 0x0d, 0x26, 0x96, 0x27, 0xca, 0xf3, 0xb5, 0x98, 0x49, - 0xa2, 0xa7, 0x5e, 0xa5, 0x62, 0xb2, 0xfd, 0xa2, 0xa2, 0xfa, 0xaf, 0x95, 0x23, 0x30, 0xac, 0x35, - 0x4a, 0xb1, 0x5a, 0x7b, 0x4a, 0x99, 0x17, 0x2d, 0x34, 0x2f, 0xd0, 0x75, 0x1c, 0x0c, 0x7d, 0x67, - 0x12, 0x12, 0x5f, 0x36, 0x10, 0x4c, 0x15, 0xdd, 0x84, 0x9c, 0xe3, 0xd9, 0xf8, 0xb9, 0xe9, 0xd8, - 0xcf, 0x59, 0xad, 0x2f, 0x0a, 0x7e, 0x96, 0x91, 0x5b, 0xf6, 0x73, 0x74, 0x1d, 0x32, 0x3e, 0x7e, - 0x86, 0xfd, 0x00, 0xb3, 0x05, 0xca, 0xd6, 0x49, 0x12, 0x51, 0x03, 0x52, 0xd4, 0x44, 0xd9, 0x05, - 0x5d, 0xb4, 0x55, 0x88, 0x16, 0xc8, 0xb5, 0xd1, 0xf7, 0x01, 0x58, 0x4f, 0x63, 0xee, 0x39, 0x1e, - 0x6f, 0x84, 0x92, 0x42, 0x20, 0xc7, 0xe8, 0x0f, 0x1c, 0x2f, 0xa4, 0xde, 0x76, 0x02, 0x73, 0xb8, - 0x87, 0x87, 0xfb, 0xac, 0x1d, 0x8a, 0x8c, 0x71, 0x82, 0x1a, 0x25, 0xea, 0x6f, 0x14, 0x28, 0xcd, - 0x77, 0x11, 0xbf, 0x2f, 0x37, 0xdd, 0x85, 0x92, 0x4b, 0xc8, 0xfe, 0x74, 0x72, 0x52, 0xfb, 0xc8, - 0x39, 0xb2, 0x7d, 0xac, 0x41, 0x86, 0x78, 0xac, 0x75, 0x3c, 0xbf, 0x99, 0x39, 0xde, 0xd4, 0x12, - 0x8f, 0xd2, 0xd0, 0x36, 0x5c, 0xe6, 0x26, 0xf1, 0x26, 0x97, 0xc3, 0xa5, 0x16, 0x86, 0x5b, 0x62, - 0x20, 0x4d, 0x86, 0xc1, 0x70, 0x7f, 0x0a, 0x2a, 0xbd, 0x6f, 0x31, 0xf7, 0x96, 0xd6, 0x57, 0x4e, - 0xf1, 0x16, 0xf5, 0x71, 0xff, 0x70, 0x82, 0x65, 0x90, 0x52, 0x15, 0x7a, 0x00, 0x61, 0xd6, 0x7c, - 0xa1, 0xc7, 0xb0, 0x24, 0x1a, 0x6a, 0xe2, 0xdb, 0xd8, 0x77, 0xbc, 0x91, 0xd8, 0x02, 0xfd, 0x8c, - 0xab, 0x8c, 0x90, 0x14, 0xb8, 0xa2, 0x23, 0x97, 0x54, 0xb4, 0x0e, 0x48, 0x62, 0x99, 0x63, 0x2b, - 0x1c, 0xee, 0x99, 0x2e, 0xf6, 0xe6, 0x36, 0x44, 0x93, 0xfc, 0x2d, 0xca, 0x6e, 0x63, 0x4f, 0xdf, - 0x81, 0x42, 0xbc, 0x0f, 0x43, 0xb7, 0x61, 0x89, 0xc9, 0x60, 0xdb, 0x8c, 0xe7, 0x86, 0xa2, 0x51, - 0x12, 0x64, 0xb9, 0x4d, 0x77, 0x41, 0x93, 0x2d, 0x5b, 0x24, 0xa9, 0x30, 0xc9, 0x25, 0x49, 0x17, - 0xa2, 0xfa, 0x3f, 0x29, 0xa0, 0x1d, 0xcd, 0xe6, 0xa8, 0x0e, 0x69, 0x16, 0x3d, 0x32, 0xf7, 0x2c, - 0x16, 0x79, 0x42, 0x17, 0x35, 0x01, 0xf0, 0xd3, 0xb9, 0xf7, 0xe7, 0xd7, 0x6f, 0x9e, 0x51, 0x9d, - 0xb8, 0xa0, 0x3c, 0x3a, 0xf8, 0xa9, 0x5c, 0xcd, 0xca, 0x2c, 0x84, 0xe3, 0xa1, 0x29, 0x03, 0xf8, - 0x77, 0x14, 0x95, 0x32, 0x7a, 0x52, 0x8b, 0x47, 0xcf, 0x6d, 0xc8, 0x48, 0x5b, 0x3f, 0x9c, 0x4f, - 0xdb, 0xdc, 0x52, 0x49, 0xd2, 0xff, 0x53, 0x81, 0xa5, 0x23, 0x2d, 0x31, 0xda, 0x82, 0xa2, 0x8b, - 0x77, 0xdf, 0x21, 0xd2, 0x0a, 0x54, 0x3d, 0x8a, 0xb3, 0x2e, 0x94, 0x7c, 0x67, 0xb4, 0x17, 0xc3, - 0x53, 0x16, 0xc4, 0x2b, 0x32, 0xfd, 0x08, 0x30, 0xe6, 0xdc, 0xd4, 0x3b, 0x3b, 0x77, 0xf1, 0xa3, - 0x89, 0xee, 0x42, 0xd1, 0x9b, 0xba, 0xae, 0x89, 0x9f, 0x4e, 0x2d, 0xd7, 0x09, 0x0f, 0xd9, 0xdd, - 0x45, 0x66, 0xcf, 0x02, 0x65, 0x35, 0x04, 0x47, 0xff, 0x2b, 0x05, 0x4a, 0xf3, 0xb7, 0x04, 0x74, - 0x0f, 0x96, 0x98, 0x77, 0x63, 0x81, 0x98, 0x88, 0x25, 0x37, 0xbc, 0x1b, 0x36, 0xa2, 0x38, 0xfb, - 0x18, 0x34, 0xee, 0xba, 0x23, 0x51, 0xcb, 0x85, 0xb9, 0x5b, 0x67, 0xd2, 0xdf, 0xb5, 0x5f, 0x7e, - 0x08, 0x25, 0x76, 0x99, 0x9a, 0xe5, 0x82, 0xb8, 0x63, 0x8a, 0x9c, 0x27, 0x4f, 0xf9, 0xaf, 0xd2, - 0x50, 0x9a, 0xef, 0xc3, 0xd1, 0x4d, 0x80, 0x91, 0x4f, 0x78, 0xd2, 0x8f, 0xaf, 0x33, 0xc7, 0xa8, - 0x35, 0xe2, 0x06, 0xe8, 0x0f, 0xa1, 0x20, 0xef, 0x8f, 0x0e, 0x11, 0x65, 0x21, 0xbf, 0xfe, 0xe3, - 0x8b, 0xde, 0x3e, 0xa3, 0xc7, 0xd9, 0xba, 0xe7, 0xf0, 0xd0, 0x27, 0x22, 0x27, 0x62, 0xdb, 0x8c, - 0x99, 0xa2, 0x46, 0xa6, 0x68, 0x82, 0xbb, 0x11, 0x59, 0xb4, 0x31, 0x77, 0x48, 0x7f, 0x74, 0x61, - 0x4b, 0x8e, 0x7a, 0x6f, 0xf9, 0x1f, 0x12, 0x90, 0x8f, 0x99, 0x47, 0x81, 0x77, 0xa7, 0xde, 0x90, - 0x1d, 0xbe, 0x45, 0x80, 0x9b, 0x53, 0x2f, 0x9a, 0x36, 0x51, 0x00, 0x74, 0x23, 0x76, 0xcb, 0x8e, - 0xcf, 0x6b, 0x66, 0x77, 0xe8, 0x5b, 0x50, 0x12, 0x85, 0x6f, 0x48, 0x5c, 0x56, 0x96, 0x69, 0xd2, - 0x2a, 0x1a, 0x05, 0x4e, 0xad, 0x11, 0x97, 0x16, 0xe5, 0x6b, 0x2c, 0x91, 0x30, 0x76, 0x8a, 0x65, - 0xee, 0xf4, 0x90, 0x31, 0x36, 0xd5, 0x6c, 0x52, 0x53, 0xf5, 0xbf, 0x54, 0x40, 0xa5, 0xef, 0x46, - 0x1a, 0x14, 0x2a, 0x9d, 0xaf, 0xcd, 0x4e, 0xb7, 0x6f, 0x76, 0x06, 0xed, 0xb6, 0x76, 0x09, 0x65, - 0x20, 0x59, 0xd9, 0xde, 0xd0, 0x12, 0xa8, 0x00, 0xd9, 0x6a, 0xb7, 0xdb, 0x36, 0x2b, 0x9d, 0xba, - 0xa6, 0xa0, 0x3c, 0x64, 0xd8, 0x53, 0xd7, 0xd0, 0x92, 0xa8, 0x04, 0x50, 0xeb, 0x76, 0x6a, 0x95, - 0xbe, 0x59, 0xd9, 0xd8, 0xd0, 0x54, 0x94, 0x83, 0x54, 0xad, 0x3b, 0xe8, 0xf4, 0xb5, 0x14, 0x55, - 0xdf, 0xaa, 0xfc, 0x52, 0xcb, 0xb0, 0x7f, 0x5a, 0x1d, 0x2d, 0x8b, 0x00, 0xd2, 0xbd, 0x7e, 0xbd, - 0xde, 0xd8, 0xd6, 0x72, 0x94, 0xd8, 0x1b, 0x6c, 0x69, 0x40, 0xe1, 0x7a, 0x83, 0x2d, 0xb3, 0xd5, - 0xe9, 0x6b, 0x79, 0xfa, 0xa6, 0xed, 0x8a, 0xd1, 0xaa, 0x74, 0x6a, 0x0d, 0xad, 0x40, 0x59, 0xbf, - 0xec, 0x1a, 0x0c, 0xb9, 0xc8, 0xdf, 0x34, 0xe8, 0xf4, 0x4d, 0xa3, 0xfb, 0xa4, 0xa7, 0x95, 0x98, - 0xde, 0x63, 0xa3, 0xde, 0x6a, 0x36, 0xb5, 0x25, 0x84, 0xa0, 0xd4, 0x6c, 0x75, 0x2a, 0x6d, 0x33, - 0xd2, 0xd6, 0xe8, 0x82, 0x38, 0x4d, 0xbc, 0xf3, 0x32, 0x2a, 0x42, 0xae, 0x62, 0x18, 0x95, 0xaf, - 0x19, 0x22, 0xa2, 0x2f, 0xdb, 0xec, 0x75, 0x3b, 0xec, 0xe9, 0x0a, 0x65, 0xd2, 0xa7, 0x2a, 0x7b, - 0xbc, 0xaa, 0x7f, 0x0c, 0x2a, 0xdd, 0x6b, 0x94, 0x05, 0xb5, 0x32, 0xe8, 0x77, 0xb5, 0x4b, 0xcc, - 0xfa, 0x5a, 0xa5, 0x5d, 0x31, 0xb4, 0x04, 0x35, 0xa6, 0xd3, 0xed, 0x98, 0xe2, 0x59, 0xd1, 0xdf, - 0x24, 0xa1, 0x34, 0x7f, 0x95, 0x8f, 0x22, 0xec, 0xdc, 0x40, 0x98, 0xd7, 0x3b, 0x16, 0x61, 0xb3, - 0xe6, 0x4d, 0x79, 0xfb, 0xe6, 0x2d, 0x6a, 0x50, 0x93, 0xef, 0xd4, 0xa0, 0xde, 0x87, 0xac, 0x3d, - 0xf5, 0x59, 0xa0, 0xb3, 0x58, 0x4b, 0x56, 0xdf, 0xa3, 0xec, 0x37, 0xaf, 0x56, 0x8a, 0xa1, 0x33, - 0xc6, 0xab, 0x75, 0xc1, 0x34, 0x22, 0x31, 0xda, 0xd3, 0x0e, 0xf7, 0xa6, 0xde, 0xbe, 0x19, 0x38, - 0x2f, 0xf0, 0x7c, 0x4f, 0xcb, 0xe8, 0x3d, 0xe7, 0x05, 0x46, 0x5d, 0x28, 0x90, 0x70, 0x0f, 0xfb, - 0xa6, 0x68, 0x16, 0xd2, 0x6f, 0xd1, 0x2c, 0xe4, 0x19, 0x42, 0x9f, 0x77, 0x0c, 0x5f, 0x42, 0xd6, - 0xc7, 0x96, 0x5d, 0x09, 0xba, 0xbb, 0x62, 0x44, 0xf5, 0x07, 0x31, 0xb0, 0x69, 0xe8, 0xb8, 0xab, - 0x7b, 0xee, 0x70, 0xb5, 0x2f, 0xa7, 0xe4, 0xf2, 0x6c, 0x49, 0x25, 0xfd, 0x9e, 0xd8, 0xfe, 0x3c, - 0x64, 0x5a, 0xde, 0x33, 0xcb, 0x75, 0x6c, 0x1e, 0x01, 0x3c, 0x0f, 0x6a, 0x09, 0x1a, 0xe8, 0x2d, - 0xda, 0x36, 0x68, 0x8a, 0xfe, 0xdb, 0x04, 0x64, 0x9b, 0x2e, 0x39, 0x60, 0xdb, 0x7e, 0x1f, 0x32, - 0xbb, 0x2e, 0x39, 0x30, 0xc5, 0x65, 0xaf, 0x50, 0x2d, 0x53, 0xe4, 0x7f, 0x7f, 0xb5, 0x92, 0xa6, - 0x22, 0xad, 0xfa, 0xeb, 0xe8, 0x3f, 0x23, 0x4d, 0x05, 0x5b, 0x36, 0xda, 0x62, 0x97, 0x5c, 0xf1, - 0xc5, 0x42, 0xb4, 0x37, 0xb7, 0x2f, 0x30, 0x34, 0x8f, 0x0d, 0xa8, 0x63, 0x00, 0x68, 0x00, 0x99, - 0x91, 0x15, 0xe2, 0x03, 0xeb, 0x90, 0x5d, 0x56, 0x52, 0xd5, 0x9f, 0x89, 0x3d, 0xfa, 0x74, 0xe4, - 0x84, 0x7b, 0xd3, 0x9d, 0xd5, 0x21, 0x19, 0xaf, 0x45, 0xe8, 0xf6, 0xce, 0xec, 0xff, 0xb5, 0xc9, - 0xfe, 0x68, 0x4d, 0xde, 0xbf, 0x3a, 0xc4, 0x66, 0x13, 0x6d, 0x81, 0xa5, 0x1f, 0x40, 0x7e, 0x93, - 0xec, 0x3c, 0xf2, 0xc9, 0x88, 0x56, 0x20, 0x74, 0x0b, 0xd2, 0xdf, 0x92, 0x1d, 0xb9, 0xcc, 0x64, - 0xb5, 0x28, 0xee, 0xb4, 0xa9, 0x4d, 0xb2, 0xd3, 0xaa, 0x1b, 0xa9, 0x6f, 0xc9, 0x4e, 0xcb, 0x46, - 0x77, 0xa0, 0x30, 0x24, 0x5e, 0xe8, 0x3b, 0x3b, 0xd3, 0x68, 0xf0, 0xac, 0xc8, 0x14, 0x1e, 0xe7, - 0xa0, 0x32, 0xa8, 0x81, 0x4b, 0x42, 0x61, 0xb2, 0xbc, 0xf7, 0xb9, 0x24, 0xd4, 0x7f, 0xa3, 0x02, - 0x3a, 0x3e, 0xdc, 0x42, 0x3f, 0x80, 0x7c, 0xc0, 0x06, 0x40, 0x3c, 0xb2, 0x94, 0x98, 0x1e, 0x70, - 0x06, 0x0b, 0xad, 0x0d, 0xc8, 0x4e, 0x84, 0xcd, 0xac, 0x38, 0x9e, 0x39, 0xb1, 0x8a, 0x2d, 0x50, - 0x46, 0x84, 0x54, 0x46, 0x1b, 0x90, 0x9c, 0xfa, 0x4e, 0x39, 0xc3, 0xb6, 0xe7, 0x27, 0x8b, 0xcc, - 0xe1, 0x56, 0x07, 0xbe, 0xd3, 0xf0, 0x42, 0xff, 0xd0, 0xa0, 0x08, 0xe8, 0x17, 0x90, 0xe6, 0x1f, - 0x8b, 0xc4, 0xd4, 0x73, 0xe5, 0x84, 0xcb, 0x6f, 0xab, 0xdb, 0x74, 0x5c, 0xdc, 0x64, 0x62, 0xd1, - 0x24, 0x9f, 0x3d, 0xa1, 0xed, 0xa8, 0xa5, 0xce, 0x31, 0x53, 0x3e, 0x5f, 0xc8, 0x14, 0x7e, 0x3e, - 0x98, 0x35, 0x0c, 0x37, 0x11, 0x35, 0xd9, 0x5f, 0xc2, 0xf7, 0x82, 0x7d, 0x67, 0x62, 0x8e, 0x9d, - 0x20, 0xa0, 0x77, 0x8b, 0x5d, 0xe2, 0x63, 0x67, 0xe4, 0x99, 0xfb, 0xf8, 0x90, 0x4f, 0x43, 0x65, - 0x01, 0x7a, 0x9f, 0x8a, 0x6d, 0x71, 0xa9, 0x26, 0x17, 0x7a, 0x88, 0x0f, 0x83, 0x65, 0x0b, 0xf2, - 0x31, 0x74, 0xa4, 0x41, 0x72, 0x1f, 0x1f, 0xf2, 0x89, 0x87, 0x41, 0xff, 0x45, 0x3f, 0x87, 0x14, - 0x1b, 0x92, 0x2e, 0x96, 0xc8, 0x0c, 0xae, 0xf4, 0x85, 0xf2, 0x79, 0x62, 0xf9, 0x33, 0xc8, 0x4a, - 0x5f, 0xc6, 0xf1, 0x53, 0x1c, 0xff, 0x6a, 0x1c, 0x3f, 0x17, 0xd3, 0xdb, 0x54, 0xb3, 0x09, 0x4d, - 0xe1, 0x05, 0x6f, 0x53, 0xcd, 0xaa, 0x5a, 0x6a, 0x53, 0xcd, 0xa6, 0xb4, 0xb4, 0xfe, 0xf7, 0x0a, - 0x14, 0xe7, 0x46, 0x9d, 0xe8, 0x23, 0xc8, 0xdb, 0x98, 0xd6, 0x57, 0x9e, 0xe4, 0xf8, 0xbc, 0x46, - 0x24, 0x98, 0x18, 0x03, 0xdd, 0x83, 0xe2, 0x81, 0xe5, 0xba, 0x34, 0xeb, 0x75, 0x2c, 0x8f, 0xf0, - 0xc1, 0x87, 0xcc, 0x6c, 0xf3, 0x2c, 0xd4, 0x9e, 0x9f, 0x0e, 0x7c, 0x72, 0xc1, 0xb1, 0x2b, 0x9b, - 0x80, 0x74, 0xac, 0x31, 0x9e, 0xcf, 0xc1, 0xf1, 0x80, 0x4e, 0xbd, 0x43, 0x40, 0x53, 0x67, 0xca, - 0x37, 0xd0, 0xd3, 0xe7, 0x59, 0x63, 0x3c, 0xb7, 0x5e, 0x46, 0xa1, 0x6e, 0xc6, 0x9e, 0xcd, 0x5c, - 0x5a, 0x30, 0xe8, 0xbf, 0x9b, 0x6a, 0x56, 0xd1, 0x92, 0xfa, 0x3f, 0x27, 0xa0, 0x38, 0x37, 0x06, - 0xbc, 0xb0, 0xeb, 0x6e, 0x43, 0x81, 0x22, 0x9b, 0x13, 0x2b, 0x0c, 0xb1, 0xcf, 0x73, 0x42, 0x24, - 0x48, 0x39, 0x8f, 0x38, 0x03, 0xfd, 0x02, 0x32, 0x64, 0x22, 0x1b, 0xc6, 0xa3, 0x39, 0x5c, 0x9e, - 0x94, 0x5a, 0x6f, 0xbb, 0xcb, 0x85, 0xe4, 0x1c, 0x44, 0xe8, 0xcc, 0x2a, 0x0f, 0x1b, 0x4c, 0xa9, - 0xc7, 0x2a, 0x0f, 0x1b, 0x4d, 0xfd, 0xa9, 0x02, 0xd0, 0xdb, 0xc7, 0xe1, 0x70, 0x8f, 0xad, 0xe1, - 0x21, 0xe4, 0x03, 0xf6, 0x64, 0xc6, 0x6a, 0xf7, 0x59, 0xdf, 0x59, 0x98, 0x70, 0xac, 0x64, 0x43, - 0x10, 0x51, 0x50, 0x79, 0x76, 0x85, 0xe3, 0x77, 0xe6, 0x68, 0x9c, 0xf6, 0x29, 0xa0, 0x11, 0xf6, - 0xb0, 0x6f, 0x85, 0xd8, 0xdc, 0x73, 0x82, 0x90, 0x8c, 0x7c, 0x6b, 0x3c, 0x37, 0x5a, 0xba, 0x2c, - 0xf9, 0x0f, 0x24, 0x1b, 0x7d, 0x0e, 0xef, 0x45, 0xb2, 0xe6, 0xd8, 0x7a, 0x6e, 0xee, 0x4c, 0x87, - 0xfb, 0x38, 0xe4, 0x4b, 0x93, 0x77, 0xff, 0x2b, 0x91, 0xc8, 0x96, 0xf5, 0xbc, 0xca, 0x05, 0xd0, - 0x4d, 0xc8, 0x05, 0xa1, 0x15, 0x9a, 0x6c, 0x8b, 0x53, 0x31, 0x77, 0x67, 0x29, 0x99, 0x06, 0x80, - 0xfe, 0xc7, 0x90, 0x8f, 0x8d, 0xeb, 0x51, 0x13, 0xb2, 0x7c, 0x21, 0xd1, 0xcd, 0xfd, 0x5c, 0x27, - 0xc4, 0xaa, 0x51, 0xa4, 0x7b, 0x52, 0x92, 0x2e, 0x1e, 0x4f, 0xd2, 0xfa, 0xff, 0x28, 0x70, 0xf5, - 0xa4, 0x19, 0xff, 0xef, 0xd9, 0x0e, 0xf4, 0x27, 0x80, 0xf8, 0x93, 0xbc, 0x0b, 0xc5, 0x26, 0x05, - 0x8f, 0x5f, 0xbf, 0x5a, 0x11, 0x9f, 0x1c, 0xc4, 0x6d, 0xa8, 0x55, 0x0f, 0xde, 0xbc, 0x5a, 0xf9, - 0xe9, 0x85, 0xaa, 0x6a, 0xec, 0x83, 0xfe, 0xaa, 0xd4, 0x36, 0xb4, 0x60, 0x0e, 0xce, 0x0e, 0x90, - 0x05, 0x59, 0x96, 0x8e, 0x69, 0x5d, 0xe5, 0xdb, 0xda, 0x94, 0x9f, 0x98, 0x59, 0x5a, 0x6c, 0xd5, - 0x2f, 0x5c, 0xc7, 0xe3, 0x6f, 0xa4, 0x75, 0x9c, 0xe1, 0xb6, 0x6c, 0xfd, 0xff, 0x54, 0xf8, 0xe0, - 0x8c, 0x8f, 0x1d, 0xe8, 0xeb, 0x23, 0x23, 0x9b, 0x9f, 0xbd, 0xd5, 0x37, 0x13, 0x9e, 0xc5, 0x8f, - 0xcc, 0x71, 0x62, 0x63, 0x54, 0xe5, 0xa4, 0x31, 0xea, 0xfc, 0xfc, 0x33, 0x79, 0xf2, 0xfc, 0xf3, - 0x3b, 0x9e, 0xd1, 0x2c, 0xff, 0xa3, 0x02, 0x29, 0xb6, 0x38, 0xf4, 0x15, 0xa8, 0x36, 0x0e, 0x86, - 0x6f, 0x35, 0x54, 0x65, 0x9a, 0x17, 0x99, 0xa9, 0xca, 0x5f, 0x2e, 0x24, 0xdf, 0xe5, 0x97, 0x0b, - 0x75, 0xc8, 0x46, 0x53, 0x1c, 0x75, 0xc1, 0x29, 0x4e, 0xa4, 0x39, 0xbb, 0x44, 0xa4, 0xde, 0xe5, - 0x12, 0xa1, 0xff, 0x5b, 0x02, 0x4a, 0xf3, 0x1f, 0x6a, 0xd0, 0x57, 0x90, 0xe2, 0xbf, 0x22, 0x48, - 0x2c, 0xfc, 0x2b, 0x02, 0xae, 0x88, 0xfa, 0x10, 0x65, 0x4c, 0xfb, 0x9c, 0x49, 0xe1, 0xfc, 0x31, - 0x8c, 0xed, 0xaf, 0x16, 0x21, 0xc8, 0xd1, 0xcc, 0x6d, 0xd0, 0xbc, 0xe9, 0x98, 0x4d, 0x13, 0xcc, - 0x09, 0xf6, 0xcd, 0x11, 0xf6, 0x78, 0x36, 0x30, 0x8a, 0xde, 0x74, 0x5c, 0x23, 0x6e, 0xf0, 0x08, - 0xfb, 0x1b, 0xd8, 0xd3, 0x7f, 0x55, 0x80, 0x42, 0xfc, 0xa3, 0x11, 0xba, 0x01, 0xf9, 0x89, 0xe5, - 0x87, 0x0e, 0x1b, 0x59, 0x1c, 0x8a, 0xe9, 0x6a, 0x9c, 0x84, 0x7a, 0x90, 0xe3, 0x1f, 0x96, 0x9a, - 0x91, 0xa5, 0x6b, 0x17, 0xfb, 0x22, 0x25, 0x1e, 0x9a, 0xd2, 0x05, 0x33, 0x9c, 0xe5, 0xbf, 0x4b, - 0x88, 0xdb, 0x7c, 0x0f, 0x8a, 0x72, 0x42, 0x82, 0x9b, 0x6f, 0x3b, 0x8f, 0x30, 0xe6, 0x31, 0xd0, - 0x63, 0x00, 0xf1, 0x2a, 0x8a, 0xa8, 0x30, 0xc4, 0xfb, 0x8b, 0xd9, 0x4c, 0x51, 0x63, 0x20, 0xfc, - 0x93, 0xda, 0xf2, 0x7f, 0xa4, 0x20, 0xd5, 0xf4, 0x69, 0xbb, 0xf1, 0x10, 0xd4, 0x31, 0xb1, 0x65, - 0xe5, 0xbd, 0x28, 0x38, 0xd3, 0x5d, 0xdd, 0x22, 0x76, 0x74, 0x54, 0x29, 0x08, 0x7a, 0x0c, 0xe9, - 0x1d, 0x32, 0xf5, 0xec, 0x40, 0x74, 0x9c, 0x9f, 0x2e, 0x04, 0x57, 0x65, 0xaa, 0x32, 0x71, 0x70, - 0xa0, 0xe5, 0xff, 0x4d, 0x40, 0x8a, 0x31, 0xd0, 0x37, 0x90, 0x63, 0xb4, 0xfe, 0xac, 0x51, 0xf8, - 0x6c, 0x71, 0xfc, 0x58, 0xf8, 0xcd, 0xe0, 0x68, 0x22, 0x74, 0xbc, 0xd0, 0x14, 0xbf, 0x88, 0x89, - 0x27, 0x86, 0x9c, 0xe3, 0x85, 0x5d, 0xfe, 0xa3, 0x98, 0x9b, 0x50, 0xa0, 0x09, 0xc9, 0x96, 0x62, - 0x49, 0xd6, 0xa2, 0xe5, 0x19, 0x4d, 0x88, 0x6c, 0x42, 0x9e, 0x33, 0x79, 0x3b, 0xc3, 0x8f, 0xfe, - 0x02, 0xdf, 0xff, 0x80, 0x6b, 0x53, 0x9b, 0x96, 0xff, 0x3a, 0x01, 0x69, 0xee, 0x12, 0xd4, 0x81, - 0x54, 0x10, 0x5a, 0x7e, 0x28, 0x32, 0xdf, 0xfa, 0xe2, 0xcb, 0x8e, 0x32, 0x02, 0x85, 0x41, 0xf5, - 0x59, 0x8f, 0xf9, 0x56, 0x68, 0xac, 0x2f, 0xd5, 0x3f, 0x00, 0x95, 0x46, 0x00, 0xbd, 0x99, 0x1b, - 0x95, 0xce, 0x46, 0x43, 0xbb, 0x84, 0xb2, 0xa0, 0xb2, 0x69, 0x51, 0x42, 0x7f, 0x01, 0xb9, 0xc8, - 0xdf, 0xe8, 0x1a, 0x5c, 0x19, 0x74, 0xaa, 0xdd, 0x41, 0xa7, 0xde, 0xa8, 0x9b, 0x8f, 0x8c, 0x46, - 0xad, 0x51, 0x6f, 0x75, 0x36, 0xb4, 0x4b, 0xf3, 0x8c, 0x66, 0xb7, 0xdd, 0xee, 0x3e, 0xa1, 0x8c, - 0x04, 0xba, 0x0a, 0x5a, 0xb7, 0xd9, 0xec, 0x35, 0xfa, 0x31, 0x71, 0x25, 0x46, 0x9d, 0xc9, 0x26, - 0xd1, 0x12, 0xe4, 0x6b, 0x03, 0xc3, 0x68, 0xf0, 0x51, 0x95, 0xa6, 0x2e, 0xff, 0x8b, 0x02, 0x59, - 0x79, 0x64, 0x51, 0x23, 0x36, 0x1f, 0xcc, 0xaf, 0xff, 0xf0, 0xa2, 0x8b, 0x3d, 0x3a, 0x1d, 0xfc, - 0x08, 0xf2, 0x96, 0x3f, 0x6a, 0xd9, 0xcf, 0x7b, 0x6c, 0x23, 0xe2, 0x21, 0x12, 0x67, 0xa0, 0x1b, - 0x90, 0xb5, 0xfc, 0x51, 0x8d, 0x4c, 0x45, 0x41, 0x8d, 0x0a, 0x8c, 0xa4, 0xfe, 0x8e, 0x6a, 0x43, - 0x15, 0x52, 0xbb, 0xbe, 0x6c, 0x2f, 0xcf, 0xfc, 0x9d, 0xca, 0xf1, 0x4d, 0x34, 0xb8, 0x2a, 0xba, - 0x03, 0x73, 0x93, 0x4b, 0xf1, 0x9b, 0x39, 0x31, 0x2c, 0x88, 0x73, 0xf4, 0x5f, 0x27, 0x00, 0x66, - 0x09, 0x05, 0x95, 0x00, 0x8c, 0xee, 0x13, 0xb3, 0x33, 0xd8, 0xaa, 0x36, 0x0c, 0xb1, 0xed, 0x95, - 0xce, 0x43, 0x3e, 0xa7, 0xab, 0x37, 0x3a, 0xbd, 0x86, 0xc9, 0x9e, 0x15, 0xa4, 0x41, 0xe1, 0x51, - 0xc3, 0xa8, 0xb1, 0xbd, 0xa1, 0x94, 0x24, 0x2a, 0x42, 0xae, 0x36, 0xd8, 0x6a, 0x98, 0xf5, 0x56, - 0xaf, 0xcf, 0xe7, 0x97, 0x9d, 0x7e, 0xab, 0xdd, 0xe0, 0xf3, 0xcb, 0x76, 0x65, 0x43, 0x4b, 0x53, - 0xb8, 0x76, 0xa3, 0x52, 0xd7, 0x32, 0x74, 0x6b, 0x9b, 0x2d, 0xa3, 0xd7, 0x37, 0xb7, 0x2b, 0xed, - 0x41, 0x43, 0xcb, 0x52, 0xfc, 0x76, 0x25, 0x7a, 0xce, 0x51, 0xb4, 0x4e, 0xff, 0x81, 0x78, 0x84, - 0x7b, 0xba, 0xbc, 0x5c, 0xb0, 0xb0, 0xbb, 0x0a, 0xda, 0x83, 0x76, 0xdb, 0x7c, 0xd4, 0x1e, 0xf4, - 0xf8, 0x9f, 0xed, 0xfb, 0xda, 0xa5, 0xea, 0xad, 0x97, 0xff, 0x7d, 0xfd, 0xd2, 0xcb, 0xd7, 0xd7, - 0x13, 0xff, 0xfa, 0xfa, 0x7a, 0xe2, 0x37, 0xaf, 0xaf, 0x27, 0xfe, 0xeb, 0xf5, 0xf5, 0xc4, 0x9f, - 0xff, 0xf6, 0xfa, 0xa5, 0x6f, 0x60, 0xe6, 0xb5, 0xff, 0x0f, 0x00, 0x00, 0xff, 0xff, 0xfe, 0x6e, - 0x64, 0xdd, 0xe3, 0x2a, 0x00, 0x00, + // 3675 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xcb, 0x6f, 0xdc, 0x68, + 0x72, 0x77, 0xb3, 0xd9, 0xaf, 0xea, 0x87, 0xe8, 0xcf, 0x9e, 0x71, 0xaf, 0x66, 0x62, 0xd9, 0x5c, + 0xef, 0xf8, 0xb1, 0xb3, 0xd2, 0xd8, 0xb3, 0x3b, 0x99, 0x9d, 0xdd, 0xc5, 0x4c, 0x3f, 0x65, 0xca, + 0xad, 0x6e, 0x99, 0xdd, 0x92, 0x77, 0xe6, 0x10, 0x86, 0x6a, 0x7e, 0x6a, 0x71, 0xc4, 0xe6, 0xd7, + 0x26, 0xd9, 0x96, 0x64, 0x04, 0xc8, 0x29, 0xe7, 0x04, 0xc8, 0x06, 0xb9, 0x05, 0xb9, 0xec, 0x39, + 0x97, 0x1c, 0x72, 0x49, 0x80, 0x00, 0x39, 0xcc, 0x25, 0x48, 0x72, 0x5b, 0x24, 0x81, 0x91, 0x78, + 0x4f, 0xf9, 0x0b, 0x02, 0xcc, 0x21, 0x08, 0xbe, 0x17, 0x9b, 0xad, 0xb7, 0xec, 0xc5, 0xee, 0xa5, + 0xd1, 0xac, 0xc7, 0x8f, 0xf5, 0xd5, 0x57, 0xac, 0xaa, 0xaf, 0x48, 0x58, 0x0a, 0x9f, 0x7b, 0x2b, + 0x8e, 0x1b, 0x46, 0xe1, 0x73, 0x2f, 0x98, 0xfa, 0x2b, 0x93, 0x80, 0x0c, 0x71, 0x18, 0x92, 0x20, + 0x5c, 0x9e, 0x04, 0x24, 0x22, 0xa8, 0x3a, 0x24, 0xc3, 0xbd, 0x80, 0xd8, 0xc3, 0xdd, 0xe5, 0xf0, + 0xb9, 0xb7, 0x3c, 0x13, 0x5d, 0xbc, 0xca, 0xa8, 0x93, 0xed, 0x15, 0x7b, 0xe2, 0x72, 0xe1, 0x45, + 0x24, 0x49, 0x8e, 0x1d, 0xd9, 0x82, 0x76, 0x5d, 0xd2, 0x70, 0x10, 0xc4, 0xb0, 0x8b, 0x55, 0x49, + 0x75, 0xc9, 0x0f, 0x76, 0x48, 0x30, 0xb6, 0x23, 0xc9, 0x79, 0x9f, 0x5a, 0x14, 0x3e, 0xf7, 0xb6, + 0xed, 0x10, 0xaf, 0x84, 0x51, 0x30, 0x1d, 0x46, 0xd3, 0x00, 0x3b, 0x82, 0xbb, 0x94, 0xe4, 0x62, + 0x7f, 0x48, 0x1c, 0xec, 0x58, 0x8e, 0x1d, 0x4d, 0xc7, 0x42, 0xe0, 0xbd, 0xa4, 0xc0, 0xd7, 0xc4, + 0xf5, 0xad, 0xe8, 0x70, 0x82, 0x05, 0xf3, 0x3b, 0x47, 0x56, 0x9b, 0x30, 0xb3, 0x3a, 0x8d, 0x5c, + 0x6f, 0x65, 0xd7, 0x1b, 0xae, 0x44, 0xee, 0x18, 0x87, 0x91, 0x3d, 0x9e, 0xc8, 0x05, 0x8c, 0xc8, + 0x88, 0xb0, 0xbf, 0x2b, 0xf4, 0x1f, 0xa7, 0xea, 0x7f, 0x9a, 0x86, 0xf2, 0x86, 0x74, 0x56, 0x7f, + 0x82, 0x87, 0xa8, 0x01, 0x19, 0xd7, 0x9f, 0x4c, 0xa3, 0x6a, 0xea, 0x56, 0xfa, 0x5e, 0xf1, 0xd1, + 0xdd, 0xe5, 0xd3, 0x3c, 0xb7, 0x6c, 0x50, 0xb1, 0xfe, 0xa1, 0x3f, 0xa4, 0x7a, 0x75, 0xf5, 0x9b, + 0x57, 0x4b, 0x57, 0x4c, 0xae, 0x8b, 0xda, 0xa0, 0x0e, 0x49, 0x80, 0xab, 0xca, 0xad, 0xd4, 0xbd, + 0xe2, 0xa3, 0x0f, 0x4f, 0xc7, 0x88, 0xef, 0xdd, 0x20, 0x01, 0xde, 0xf4, 0x5d, 0xe2, 0x0b, 0x20, + 0xa6, 0x8f, 0x1e, 0x43, 0x96, 0x4c, 0x23, 0x6a, 0x4d, 0x9a, 0x59, 0xf3, 0xe0, 0x74, 0xa4, 0x1e, + 0x93, 0x33, 0xc9, 0x34, 0xc2, 0x41, 0xc2, 0x20, 0xa1, 0x8f, 0x1a, 0xa0, 0x4e, 0x48, 0x18, 0x55, + 0x55, 0x66, 0xd1, 0xfd, 0x33, 0x2c, 0x22, 0x61, 0x24, 0xac, 0x4a, 0xc0, 0x30, 0x65, 0xf4, 0x00, + 0xf2, 0x61, 0x64, 0x8f, 0xb0, 0xe5, 0x3a, 0xd5, 0xcc, 0xad, 0xd4, 0xbd, 0x4c, 0x7d, 0x81, 0x72, + 0x5f, 0xbf, 0x5a, 0xca, 0xf5, 0x29, 0xdd, 0x68, 0x9a, 0x39, 0x26, 0x60, 0x38, 0xe8, 0x13, 0x28, + 0xc5, 0x51, 0x48, 0xe5, 0xb3, 0x4c, 0xfe, 0x9a, 0x90, 0x2f, 0xc6, 0x0b, 0x37, 0x9a, 0x66, 0x31, + 0x16, 0x34, 0x1c, 0xfd, 0x6f, 0x14, 0x58, 0x38, 0x62, 0x03, 0xaa, 0x43, 0x76, 0xc7, 0xf5, 0x22, + 0x1c, 0x54, 0x53, 0xcc, 0xfc, 0x3b, 0xa7, 0x9b, 0xdf, 0x3a, 0x98, 0x04, 0x38, 0x0c, 0x67, 0x8e, + 0x14, 0x9a, 0xe8, 0x0e, 0xc0, 0x24, 0x20, 0x5f, 0xe3, 0x61, 0xe4, 0x12, 0x9f, 0x6d, 0x4c, 0x5e, + 0x48, 0x24, 0xe8, 0xe8, 0x3e, 0x54, 0xb8, 0xc3, 0xac, 0x21, 0xf1, 0xa6, 0x63, 0x3f, 0x64, 0x8e, + 0x2f, 0xd7, 0x15, 0x2d, 0x65, 0x96, 0x39, 0xa7, 0xc1, 0x19, 0x68, 0x1d, 0x4a, 0x01, 0xf6, 0x1d, + 0x1c, 0x58, 0xf8, 0x60, 0x12, 0x84, 0x55, 0x95, 0xed, 0xd0, 0x65, 0x4c, 0x2b, 0x72, 0x7d, 0x4a, + 0x0f, 0xd1, 0xfb, 0x90, 0x25, 0x3b, 0x3b, 0x21, 0x8e, 0x98, 0x67, 0xd5, 0x78, 0xfb, 0x18, 0x0d, + 0x2d, 0x42, 0xc6, 0x73, 0xc7, 0x6e, 0xc4, 0xdc, 0x28, 0x99, 0x9c, 0xa4, 0xff, 0x73, 0x19, 0xd0, + 0xf1, 0x38, 0x42, 0x9f, 0x81, 0xea, 0x13, 0x32, 0x11, 0x2e, 0xfb, 0xe0, 0x74, 0xbb, 0xba, 0x84, + 0x4c, 0xa8, 0x1a, 0x75, 0xb5, 0xc9, 0x74, 0xd0, 0x13, 0x28, 0x46, 0xf6, 0xb6, 0x87, 0x4d, 0x6c, + 0x3b, 0x38, 0x10, 0x61, 0x7c, 0x46, 0xd0, 0x0c, 0x66, 0xc2, 0x0c, 0x25, 0xa9, 0x8d, 0x1e, 0x03, + 0xd0, 0x27, 0x58, 0x60, 0xa5, 0x19, 0xd6, 0xbd, 0xd3, 0xb1, 0xd6, 0x62, 0x59, 0x06, 0x95, 0xd0, + 0x45, 0x3f, 0x85, 0x6c, 0x48, 0x02, 0x1a, 0x07, 0xea, 0x79, 0x71, 0xd0, 0x67, 0x72, 0x0c, 0x41, + 0xe8, 0x50, 0x3b, 0xec, 0xd1, 0x28, 0xc0, 0x23, 0x3b, 0x22, 0x01, 0xf3, 0xf2, 0x99, 0x76, 0xd4, + 0x62, 0x59, 0x6e, 0xc7, 0x4c, 0x17, 0xd5, 0x21, 0x4f, 0x05, 0x5d, 0x7f, 0x18, 0x55, 0x73, 0xe7, + 0xb9, 0xb7, 0x29, 0x24, 0x19, 0x4a, 0xac, 0x47, 0x5d, 0x3c, 0xc6, 0xc1, 0x08, 0xd3, 0xe5, 0xe2, + 0xa0, 0x9a, 0x3f, 0xcf, 0xc5, 0xeb, 0x33, 0x61, 0xee, 0xe2, 0x84, 0x36, 0x5d, 0xda, 0xae, 0x1d, + 0xee, 0x0a, 0xac, 0xc2, 0x79, 0x4b, 0x7b, 0x1c, 0xcb, 0xf2, 0xa5, 0xcd, 0x74, 0xd1, 0x17, 0x90, + 0x7d, 0x61, 0x7b, 0x53, 0x1c, 0x56, 0xe1, 0x3c, 0x94, 0x2d, 0x26, 0x17, 0x47, 0x8e, 0xd0, 0xa3, + 0xb6, 0x6c, 0xdb, 0xc3, 0xbd, 0x1d, 0xd7, 0xf3, 0x70, 0x50, 0x2d, 0x9e, 0x87, 0x52, 0x8f, 0x65, + 0xb9, 0x2d, 0x33, 0x5d, 0xd4, 0x01, 0x08, 0xb0, 0xed, 0x18, 0xe3, 0x09, 0x09, 0xa2, 0x6a, 0xf9, + 0xbc, 0x5c, 0x6a, 0xc6, 0xb2, 0x4d, 0x3b, 0xb2, 0x39, 0xda, 0x4c, 0x1f, 0xb5, 0xa0, 0xd0, 0xef, + 0x0f, 0x9e, 0x05, 0x2e, 0x8d, 0x9f, 0x0a, 0x03, 0x3b, 0x23, 0xb9, 0xc7, 0xa2, 0x0c, 0x67, 0xa6, + 0x89, 0x3e, 0x87, 0x5c, 0xdf, 0x1e, 0x4f, 0xe8, 0xda, 0x16, 0x18, 0xc8, 0xf7, 0xce, 0x00, 0xe1, + 0x82, 0x0c, 0x42, 0x6a, 0xa1, 0xaf, 0x40, 0xe3, 0x7f, 0x67, 0x01, 0x56, 0xd5, 0x18, 0xd2, 0xf2, + 0x79, 0x48, 0x47, 0x42, 0xf2, 0x18, 0x0e, 0x22, 0x70, 0xc3, 0xf5, 0x23, 0x1c, 0x78, 0xd8, 0x7e, + 0x81, 0x1d, 0xfe, 0xd4, 0x88, 0xa0, 0xb8, 0xca, 0x6e, 0xf1, 0xa3, 0xb3, 0xca, 0xd9, 0x89, 0x8a, + 0xec, 0x4e, 0xa7, 0xa1, 0xa2, 0x3f, 0x04, 0x34, 0xc6, 0x91, 0x4d, 0x2b, 0xf0, 0x00, 0x87, 0x51, + 0x9f, 0x25, 0xb4, 0x2a, 0x62, 0xf7, 0xfa, 0xe8, 0xac, 0x60, 0x3e, 0xaa, 0xc3, 0x6e, 0x73, 0x02, + 0x16, 0xda, 0x81, 0xeb, 0x49, 0xaa, 0x89, 0x87, 0xd8, 0x7d, 0x81, 0x83, 0xea, 0x35, 0x76, 0x8f, + 0x47, 0x17, 0xbb, 0x87, 0xd4, 0x62, 0x77, 0x39, 0x11, 0x8f, 0x86, 0x47, 0xa3, 0xbf, 0x25, 0xc2, + 0xe3, 0xfa, 0x79, 0xe1, 0x11, 0x8b, 0xf2, 0xf0, 0x88, 0x2f, 0x51, 0x17, 0x4a, 0x2f, 0xdd, 0xd1, + 0x4b, 0x7b, 0x24, 0xdc, 0xfe, 0x0e, 0x43, 0x3a, 0xa3, 0x6e, 0x7f, 0x95, 0x90, 0x66, 0x60, 0x73, + 0xfa, 0xf4, 0x69, 0x12, 0xe5, 0xa9, 0x8f, 0xa3, 0xea, 0xbb, 0xe7, 0x3d, 0x4d, 0x1b, 0xb1, 0x2c, + 0x8f, 0xff, 0x99, 0x2e, 0x4d, 0x5a, 0xfb, 0xae, 0xef, 0x90, 0x7d, 0x1c, 0x54, 0x6f, 0x9c, 0x97, + 0xb4, 0x9e, 0x09, 0x49, 0x9e, 0xb4, 0xa4, 0x1e, 0x7a, 0x0a, 0x65, 0x8f, 0x0c, 0x6d, 0x6f, 0xc3, + 0xb3, 0xfd, 0x2e, 0x71, 0x70, 0xb5, 0xca, 0x80, 0xbe, 0x7f, 0x3a, 0x50, 0x27, 0x29, 0xce, 0xd0, + 0xe6, 0x11, 0x3e, 0x53, 0xbf, 0xf9, 0xeb, 0xa5, 0xd4, 0x9a, 0x9a, 0xcf, 0x6a, 0xb9, 0x35, 0x35, + 0x5f, 0xd2, 0xca, 0x7a, 0x05, 0x4a, 0xc9, 0x92, 0xa4, 0xff, 0x10, 0xde, 0x3d, 0x39, 0x5e, 0xd0, + 0x22, 0x28, 0xae, 0xc3, 0x0a, 0x5c, 0xa1, 0x0e, 0xa2, 0xb3, 0x50, 0x8c, 0xa6, 0xa9, 0xb8, 0x8e, + 0xfe, 0x18, 0xaa, 0xa7, 0x45, 0x00, 0xfa, 0x10, 0x20, 0xe4, 0xa5, 0xdb, 0x75, 0x42, 0xd6, 0xe8, + 0x15, 0xea, 0xe5, 0xd7, 0xaf, 0x96, 0x0a, 0x1c, 0xdb, 0x68, 0x86, 0x66, 0x81, 0x0b, 0x18, 0x4e, + 0xa8, 0xff, 0x79, 0x0a, 0x2a, 0xf3, 0xb9, 0x0e, 0x35, 0x20, 0x27, 0xfb, 0x03, 0xde, 0x26, 0x7e, + 0xf7, 0x8c, 0xfc, 0x4f, 0xdb, 0x5a, 0xc3, 0xdf, 0x21, 0xa2, 0x6a, 0x4b, 0x4d, 0xf4, 0x1e, 0x14, + 0x02, 0x7b, 0xdf, 0xda, 0x3e, 0x8c, 0x70, 0x58, 0x55, 0x6e, 0xa5, 0xef, 0x95, 0xcc, 0x7c, 0x60, + 0xef, 0xd7, 0xe9, 0x35, 0x5a, 0x82, 0xbc, 0x3f, 0x1d, 0x5b, 0x01, 0xd9, 0x0f, 0x59, 0xc9, 0x94, + 0x35, 0x3f, 0xe7, 0x4f, 0xc7, 0x26, 0xd9, 0x0f, 0xf5, 0x26, 0x2c, 0xcc, 0x55, 0x5d, 0xdb, 0x47, + 0x0f, 0x41, 0x0d, 0x27, 0xb6, 0x2f, 0x2a, 0xfe, 0x8d, 0x84, 0x49, 0xa2, 0x4d, 0x5f, 0xa6, 0x62, + 0xb2, 0xa3, 0xa3, 0xa2, 0xfa, 0x2f, 0x95, 0x23, 0x30, 0xac, 0xdb, 0xca, 0xb0, 0xf2, 0x7d, 0x4a, + 0xe7, 0x20, 0xba, 0x72, 0x5e, 0xf3, 0x9b, 0x38, 0x1c, 0x06, 0xee, 0x24, 0x22, 0x81, 0xec, 0x49, + 0x98, 0x2a, 0xba, 0x0d, 0x05, 0xd7, 0x77, 0xf0, 0x81, 0xe5, 0x3a, 0x07, 0xac, 0x7d, 0x28, 0x0b, + 0x7e, 0x9e, 0x91, 0x0d, 0xe7, 0x00, 0xdd, 0x84, 0x5c, 0x80, 0x5f, 0xe0, 0x20, 0xc4, 0x6c, 0x81, + 0xb2, 0x1b, 0x93, 0x44, 0xd4, 0x82, 0x0c, 0x35, 0x51, 0x36, 0x56, 0x17, 0xed, 0x3e, 0xe2, 0x05, + 0x72, 0x6d, 0xf4, 0x5d, 0x00, 0xd6, 0x26, 0x59, 0xbb, 0xae, 0xcf, 0x7b, 0xab, 0xb4, 0x10, 0x28, + 0x30, 0xfa, 0x63, 0xd7, 0x8f, 0xa8, 0xb7, 0xdd, 0xd0, 0x1a, 0xee, 0xe2, 0xe1, 0x1e, 0xeb, 0xb0, + 0x62, 0x63, 0xdc, 0xb0, 0x41, 0x89, 0xfa, 0xb7, 0x0a, 0x54, 0xe6, 0x1b, 0x93, 0xdf, 0x96, 0x9b, + 0xee, 0x43, 0xc5, 0x23, 0x64, 0x6f, 0x3a, 0x39, 0xa9, 0x23, 0xe5, 0x1c, 0xd9, 0x91, 0x36, 0x20, + 0x47, 0x7c, 0xd6, 0x8d, 0x9e, 0xdf, 0x1f, 0x1d, 0xef, 0x93, 0x89, 0x4f, 0x69, 0x68, 0x0b, 0xae, + 0x72, 0x93, 0x78, 0xdf, 0xcc, 0xe1, 0x32, 0x97, 0x86, 0x5b, 0x60, 0x20, 0x6d, 0x86, 0xc1, 0x70, + 0x7f, 0x0c, 0x2a, 0x3d, 0xc2, 0x31, 0xf7, 0x56, 0x1e, 0x2d, 0x9d, 0xe2, 0x2d, 0xea, 0xe3, 0xc1, + 0xe1, 0x04, 0xcb, 0x20, 0xa5, 0x2a, 0xf4, 0x01, 0x84, 0x59, 0x3f, 0x87, 0x9e, 0xc2, 0x82, 0xe8, + 0xd1, 0x49, 0xe0, 0xe0, 0xc0, 0xf5, 0x47, 0x62, 0x0b, 0xf4, 0x33, 0x4e, 0x47, 0x42, 0x52, 0xe0, + 0x8a, 0x26, 0x5f, 0x52, 0xd1, 0x23, 0x40, 0x12, 0xcb, 0x1a, 0xdb, 0xd1, 0x70, 0xd7, 0xf2, 0xb0, + 0x3f, 0xb7, 0x21, 0x9a, 0xe4, 0xaf, 0x53, 0x76, 0x07, 0xfb, 0xfa, 0x36, 0x94, 0x92, 0xad, 0x1d, + 0xba, 0x0b, 0x0b, 0x4c, 0x06, 0x3b, 0x56, 0x32, 0x37, 0x94, 0xcd, 0x8a, 0x20, 0xcb, 0x6d, 0xba, + 0x0f, 0x9a, 0xec, 0x02, 0x63, 0x49, 0x85, 0x49, 0x2e, 0x48, 0xba, 0x10, 0xd5, 0xff, 0x51, 0x01, + 0xed, 0x68, 0x81, 0x40, 0x4d, 0xc8, 0xb2, 0xe8, 0x91, 0xb9, 0xe7, 0x72, 0x91, 0x27, 0x74, 0x51, + 0x1b, 0x00, 0x3f, 0x9f, 0xbb, 0x7f, 0xf1, 0xd1, 0xed, 0x33, 0x0a, 0x1e, 0x17, 0x94, 0x8f, 0x0e, + 0x7e, 0x2e, 0x57, 0xb3, 0x34, 0x0b, 0xe1, 0x64, 0x68, 0xca, 0x00, 0xfe, 0x0d, 0x45, 0xa5, 0x8c, + 0x9e, 0xcc, 0xe5, 0xa3, 0xe7, 0xf7, 0xe1, 0xea, 0xb1, 0x22, 0x84, 0x74, 0x28, 0x99, 0x64, 0xbf, + 0x4f, 0xa6, 0xc1, 0x10, 0x1b, 0xce, 0x01, 0x0b, 0xa0, 0xb2, 0x39, 0x47, 0xd3, 0xef, 0x42, 0x4e, + 0x2e, 0xf2, 0xfd, 0xf9, 0x7c, 0xcf, 0x97, 0x28, 0x49, 0xfa, 0x7f, 0x2a, 0xb0, 0x70, 0xa4, 0x3d, + 0x47, 0xeb, 0x50, 0xf6, 0xf0, 0xce, 0x5b, 0x84, 0x68, 0x89, 0xaa, 0xc7, 0x01, 0xda, 0x83, 0x4a, + 0xe0, 0x8e, 0x76, 0x13, 0x78, 0xca, 0x25, 0xf1, 0xca, 0x4c, 0x3f, 0x06, 0x4c, 0xec, 0x4a, 0xe6, + 0xad, 0x77, 0xe5, 0xf2, 0xcf, 0x34, 0xba, 0x0f, 0x65, 0x7f, 0xea, 0x79, 0x16, 0x7e, 0x3e, 0xb5, + 0x3d, 0x37, 0x3a, 0x64, 0xe7, 0x28, 0x99, 0x76, 0x4b, 0x94, 0xd5, 0x12, 0x1c, 0xfd, 0x2f, 0x15, + 0xa8, 0xcc, 0x9f, 0x58, 0xd0, 0x03, 0x58, 0x60, 0xde, 0x4d, 0x44, 0x70, 0x2a, 0x91, 0x15, 0xf1, + 0x4e, 0xd4, 0x8a, 0x03, 0xf4, 0x43, 0xd0, 0xb8, 0xeb, 0x8e, 0x84, 0x3b, 0x17, 0xe6, 0x6e, 0x9d, + 0x49, 0xff, 0xae, 0xfd, 0xf2, 0x7d, 0xa8, 0xb0, 0x83, 0xdd, 0x2c, 0x89, 0x24, 0x1d, 0x53, 0xe6, + 0x3c, 0x99, 0x1e, 0x7e, 0x91, 0x85, 0xca, 0xfc, 0x99, 0x00, 0xdd, 0x06, 0x18, 0x05, 0x84, 0x57, + 0x8b, 0xe4, 0x3a, 0x0b, 0x8c, 0xda, 0x20, 0x5e, 0x88, 0xfe, 0x00, 0x4a, 0xf2, 0x2c, 0xeb, 0x12, + 0x51, 0x4f, 0x8a, 0x8f, 0x7e, 0x78, 0xd1, 0x93, 0x70, 0x7c, 0x39, 0x5b, 0xf7, 0x1c, 0x1e, 0xfa, + 0x48, 0x24, 0x53, 0xec, 0x58, 0x09, 0x53, 0xd4, 0xd8, 0x14, 0x4d, 0x70, 0x57, 0x63, 0x8b, 0x56, + 0xe7, 0x9e, 0xee, 0x1f, 0x5c, 0xd8, 0x92, 0xa3, 0xde, 0x5b, 0xfc, 0xfb, 0x14, 0x14, 0x13, 0xe6, + 0x51, 0xe0, 0x9d, 0xa9, 0x3f, 0x64, 0x0f, 0xdf, 0x65, 0x80, 0xdb, 0x53, 0x3f, 0x9e, 0x7c, 0x51, + 0x00, 0x74, 0x2b, 0x71, 0xe2, 0x4f, 0xce, 0x8e, 0x66, 0xe7, 0xf9, 0x3b, 0x50, 0x11, 0x15, 0x73, + 0x48, 0x3c, 0x56, 0xcf, 0x55, 0x9e, 0x53, 0x38, 0xb5, 0x41, 0x3c, 0x5a, 0xcd, 0x6f, 0xb0, 0x44, + 0xc2, 0xd8, 0x19, 0x96, 0xf2, 0xb3, 0x43, 0xc6, 0x58, 0x53, 0xf3, 0x69, 0x4d, 0xd5, 0xff, 0x42, + 0x01, 0x95, 0xde, 0x1b, 0x69, 0x50, 0xaa, 0x75, 0xbf, 0xb4, 0xba, 0xbd, 0x81, 0xd5, 0xdd, 0xec, + 0x74, 0xb4, 0x2b, 0x28, 0x07, 0xe9, 0xda, 0xd6, 0xaa, 0x96, 0x42, 0x25, 0xc8, 0xd7, 0x7b, 0xbd, + 0x8e, 0x55, 0xeb, 0x36, 0x35, 0x05, 0x15, 0x21, 0xc7, 0xae, 0x7a, 0xa6, 0x96, 0x46, 0x15, 0x80, + 0x46, 0xaf, 0xdb, 0xa8, 0x0d, 0xac, 0xda, 0xea, 0xaa, 0xa6, 0xa2, 0x02, 0x64, 0x1a, 0xbd, 0xcd, + 0xee, 0x40, 0xcb, 0x50, 0xf5, 0xf5, 0xda, 0xcf, 0xb5, 0x1c, 0xfb, 0x63, 0x74, 0xb5, 0x3c, 0x02, + 0xc8, 0xf6, 0x07, 0xcd, 0x66, 0x6b, 0x4b, 0x2b, 0x50, 0x62, 0x7f, 0x73, 0x5d, 0x03, 0x0a, 0xd7, + 0xdf, 0x5c, 0xb7, 0x8c, 0xee, 0x40, 0x2b, 0xd2, 0x3b, 0x6d, 0xd5, 0x4c, 0xa3, 0xd6, 0x6d, 0xb4, + 0xb4, 0x12, 0x65, 0xfd, 0xbc, 0x67, 0x32, 0xe4, 0x32, 0xbf, 0xd3, 0x66, 0x77, 0x60, 0x99, 0xbd, + 0x67, 0x7d, 0xad, 0xc2, 0xf4, 0x9e, 0x9a, 0x4d, 0xa3, 0xdd, 0xd6, 0x16, 0x10, 0x82, 0x4a, 0xdb, + 0xe8, 0xd6, 0x3a, 0x56, 0xac, 0xad, 0xd1, 0x05, 0x71, 0x9a, 0xb8, 0xe7, 0x55, 0x54, 0x86, 0x42, + 0xcd, 0x34, 0x6b, 0x5f, 0x32, 0x44, 0x44, 0x6f, 0xb6, 0xd6, 0xef, 0x75, 0xd9, 0xd5, 0x35, 0xca, + 0xa4, 0x57, 0x75, 0x76, 0x79, 0x5d, 0xff, 0x10, 0x54, 0xba, 0xd7, 0x28, 0x0f, 0x6a, 0x6d, 0x73, + 0xd0, 0xd3, 0xae, 0x30, 0xeb, 0x1b, 0xb5, 0x4e, 0xcd, 0xd4, 0x52, 0xd4, 0x98, 0x6e, 0xaf, 0x6b, + 0x89, 0x6b, 0x45, 0xff, 0x36, 0x0d, 0x95, 0xf9, 0xb1, 0x42, 0x1c, 0x61, 0xe7, 0x06, 0xc2, 0xbc, + 0xde, 0xb1, 0x08, 0x9b, 0x75, 0x7d, 0xca, 0x9b, 0x77, 0x7d, 0x71, 0x67, 0x9b, 0x7e, 0xab, 0xce, + 0xf6, 0x21, 0xe4, 0x9d, 0x69, 0xc0, 0x02, 0x9d, 0xc5, 0x5a, 0xba, 0xfe, 0x0e, 0x65, 0x7f, 0xfb, + 0x6a, 0xa9, 0x1c, 0xb9, 0x63, 0xbc, 0xdc, 0x14, 0x4c, 0x33, 0x16, 0xa3, 0xcd, 0xf0, 0x70, 0x77, + 0xea, 0xef, 0x59, 0xa1, 0xfb, 0x12, 0xcf, 0x37, 0xc3, 0x8c, 0xde, 0x77, 0x5f, 0x62, 0xd4, 0x83, + 0x12, 0x89, 0x76, 0x71, 0x60, 0x89, 0x2e, 0x23, 0xfb, 0x06, 0x5d, 0x46, 0x91, 0x21, 0x0c, 0x78, + 0xab, 0xf1, 0x39, 0xe4, 0x03, 0x6c, 0x3b, 0xb5, 0xb0, 0xb7, 0x23, 0xc6, 0x65, 0xbf, 0x97, 0x00, + 0x9b, 0x46, 0xae, 0xb7, 0xbc, 0xeb, 0x0d, 0x97, 0x07, 0x72, 0x62, 0x2f, 0x9f, 0x2d, 0xa9, 0xa4, + 0x3f, 0x10, 0xdb, 0x5f, 0x84, 0x9c, 0xe1, 0xbf, 0xb0, 0x3d, 0xd7, 0xe1, 0x11, 0xc0, 0xf3, 0xa0, + 0x96, 0xa2, 0x81, 0x6e, 0xd0, 0x7e, 0x43, 0x53, 0xf4, 0x5f, 0xa7, 0x20, 0xdf, 0xf6, 0xc8, 0x3e, + 0xdb, 0xf6, 0x87, 0x90, 0xdb, 0xf1, 0xc8, 0xbe, 0x25, 0x4e, 0x89, 0xa5, 0x7a, 0x95, 0x22, 0xff, + 0xfb, 0xab, 0xa5, 0x2c, 0x15, 0x31, 0x9a, 0xaf, 0xe3, 0x7f, 0x66, 0x96, 0x0a, 0x1a, 0x0e, 0x5a, + 0x67, 0x07, 0x6e, 0xf1, 0xf6, 0x44, 0xf4, 0x45, 0x77, 0x2f, 0x30, 0xc0, 0x4f, 0x0c, 0xcb, 0x13, + 0x00, 0x68, 0x13, 0x72, 0x23, 0x3b, 0xc2, 0xfb, 0xf6, 0x21, 0x3b, 0xe5, 0x64, 0xea, 0x3f, 0x11, + 0x7b, 0xf4, 0xf1, 0xc8, 0x8d, 0x76, 0xa7, 0xdb, 0xcb, 0x43, 0x32, 0x5e, 0x89, 0xd1, 0x9d, 0xed, + 0xd9, 0xff, 0x95, 0xc9, 0xde, 0x68, 0x45, 0x1e, 0xdc, 0x68, 0xe3, 0x62, 0x34, 0x4d, 0x89, 0xa5, + 0xef, 0x43, 0x71, 0x8d, 0x6c, 0x6f, 0x04, 0x64, 0x44, 0x2b, 0x10, 0xba, 0x03, 0xd9, 0xaf, 0xc9, + 0xb6, 0x5c, 0x66, 0xba, 0x5e, 0x16, 0x87, 0xe1, 0xcc, 0x1a, 0xd9, 0x36, 0x9a, 0x66, 0xe6, 0x6b, + 0xb2, 0x6d, 0x38, 0xe8, 0x1e, 0x94, 0x86, 0xc4, 0x8f, 0x02, 0x77, 0x7b, 0x1a, 0x0f, 0xc1, 0x15, + 0x99, 0xc2, 0x93, 0x1c, 0x54, 0x05, 0x35, 0xf4, 0x48, 0x24, 0x4c, 0x96, 0x07, 0x46, 0x8f, 0x44, + 0xfa, 0xaf, 0x54, 0x40, 0xc7, 0x07, 0x6d, 0xe8, 0x7b, 0x50, 0x0c, 0xd9, 0x30, 0x8a, 0x47, 0x96, + 0x92, 0xd0, 0x03, 0xce, 0x60, 0xa1, 0xb5, 0x0a, 0xf9, 0x89, 0xb0, 0x99, 0x15, 0xc7, 0x33, 0xa7, + 0x67, 0x89, 0x05, 0xca, 0x88, 0x90, 0xca, 0x68, 0x15, 0xd2, 0xd3, 0xc0, 0xad, 0xe6, 0xd8, 0xf6, + 0xfc, 0xe8, 0x32, 0x33, 0xc1, 0xe5, 0xcd, 0xc0, 0x6d, 0xf9, 0x51, 0x70, 0x68, 0x52, 0x04, 0xf4, + 0x33, 0xc8, 0xf2, 0x17, 0x57, 0x62, 0x02, 0xbb, 0x74, 0xc2, 0xa9, 0xd9, 0xe8, 0xb5, 0x5d, 0x0f, + 0xb7, 0x99, 0x58, 0xfc, 0x56, 0x81, 0x5d, 0xa1, 0xad, 0xb8, 0x17, 0x2f, 0x30, 0x53, 0x3e, 0xbd, + 0x94, 0x29, 0xfc, 0xf9, 0x60, 0xd6, 0x30, 0xdc, 0x54, 0xdc, 0x9d, 0x7f, 0x0e, 0xdf, 0x09, 0xf7, + 0xdc, 0x89, 0x35, 0x76, 0xc3, 0x90, 0x1e, 0x4a, 0x76, 0x48, 0x80, 0xdd, 0x91, 0x6f, 0xed, 0xe1, + 0x43, 0x3e, 0x99, 0x95, 0x05, 0xe8, 0x5d, 0x2a, 0xb6, 0xce, 0xa5, 0xda, 0x5c, 0xe8, 0x09, 0x3e, + 0x0c, 0x17, 0x6d, 0x28, 0x26, 0xd0, 0x91, 0x06, 0xe9, 0x3d, 0x7c, 0xc8, 0x47, 0x25, 0x26, 0xfd, + 0x8b, 0x7e, 0x0a, 0x19, 0x36, 0xb0, 0xbd, 0x5c, 0x22, 0x33, 0xb9, 0xd2, 0x67, 0xca, 0xa7, 0xa9, + 0xc5, 0x4f, 0x20, 0x2f, 0x7d, 0x99, 0xc4, 0xcf, 0x70, 0xfc, 0xeb, 0x49, 0xfc, 0x42, 0x42, 0x6f, + 0x4d, 0xcd, 0xa7, 0x34, 0x85, 0x17, 0xbc, 0x35, 0x35, 0xaf, 0x6a, 0x99, 0x35, 0x35, 0x9f, 0xd1, + 0xb2, 0xfa, 0xdf, 0x29, 0x50, 0x9e, 0x1b, 0xbb, 0xa2, 0x0f, 0xa0, 0xe8, 0x60, 0x5a, 0x5f, 0x79, + 0x92, 0xe3, 0x83, 0x1e, 0x91, 0x60, 0x12, 0x0c, 0xf4, 0x00, 0xca, 0xfb, 0xb6, 0xe7, 0xd1, 0xac, + 0xd7, 0xb5, 0x7d, 0xc2, 0x27, 0x26, 0x32, 0xb3, 0xcd, 0xb3, 0x50, 0x67, 0x7e, 0xac, 0xf0, 0xd1, + 0x05, 0x47, 0xc0, 0x6c, 0x74, 0xd2, 0xb5, 0xc7, 0x78, 0x3e, 0x07, 0x27, 0x03, 0x3a, 0xf3, 0x16, + 0x01, 0x4d, 0x9d, 0x29, 0xef, 0x40, 0x9f, 0x3e, 0xdf, 0x1e, 0xe3, 0xb9, 0xf5, 0x32, 0x0a, 0x75, + 0x33, 0xf6, 0x1d, 0xe6, 0xd2, 0x92, 0x49, 0xff, 0xae, 0xa9, 0x79, 0x45, 0x4b, 0xeb, 0xff, 0x94, + 0x82, 0xf2, 0xdc, 0x48, 0xf2, 0xc2, 0xae, 0xbb, 0x0b, 0x25, 0x8a, 0x6c, 0x4d, 0xec, 0x28, 0xc2, + 0x01, 0xcf, 0x09, 0xb1, 0x20, 0xe5, 0x6c, 0x70, 0x06, 0xfa, 0x19, 0xe4, 0xc8, 0x44, 0x36, 0x8c, + 0x47, 0x73, 0xb8, 0x7c, 0x52, 0x1a, 0xfd, 0xad, 0x1e, 0x17, 0x92, 0x03, 0x14, 0xa1, 0x33, 0xab, + 0x3c, 0x6c, 0xa2, 0xa5, 0x1e, 0xab, 0x3c, 0x6c, 0xa6, 0xf5, 0x27, 0x0a, 0x40, 0x7f, 0x0f, 0x47, + 0xc3, 0x5d, 0xb6, 0x86, 0x27, 0x50, 0x0c, 0xd9, 0x95, 0x95, 0xa8, 0xdd, 0x67, 0xbd, 0xf3, 0x61, + 0xc2, 0x89, 0x92, 0x0d, 0x61, 0x4c, 0x41, 0xd5, 0xd9, 0x11, 0x8e, 0x1f, 0xb6, 0xe3, 0x39, 0xdc, + 0xc7, 0x80, 0x46, 0xd8, 0xc7, 0x81, 0x1d, 0x61, 0x6b, 0xd7, 0x0d, 0x23, 0x32, 0x0a, 0xec, 0xf1, + 0xdc, 0x4c, 0xea, 0xaa, 0xe4, 0x3f, 0x96, 0x6c, 0xf4, 0x29, 0xbc, 0x13, 0xcb, 0x5a, 0x63, 0xfb, + 0xc0, 0xda, 0x9e, 0x0e, 0xf7, 0x70, 0xc4, 0x97, 0x26, 0x87, 0x06, 0xd7, 0x62, 0x91, 0x75, 0xfb, + 0xa0, 0xce, 0x05, 0xd0, 0x6d, 0x28, 0x84, 0x91, 0x1d, 0x59, 0x6c, 0x8b, 0x33, 0x09, 0x77, 0xe7, + 0x29, 0x99, 0x06, 0x80, 0xfe, 0x47, 0x50, 0x4c, 0xbc, 0x3a, 0x40, 0x6d, 0xc8, 0xf3, 0x85, 0xc4, + 0x47, 0xfe, 0x73, 0x9d, 0x90, 0xa8, 0x46, 0xb1, 0xee, 0x49, 0x49, 0xba, 0x7c, 0x3c, 0x49, 0xeb, + 0xff, 0xa3, 0xc0, 0xf5, 0x93, 0xde, 0x37, 0xfc, 0x96, 0xed, 0x40, 0x7f, 0x0c, 0x88, 0x5f, 0xc9, + 0xb3, 0x50, 0x62, 0xc4, 0xf0, 0xf4, 0xf5, 0xab, 0x25, 0xf1, 0xfa, 0x43, 0x9c, 0x86, 0x8c, 0x66, + 0xf8, 0xed, 0xab, 0xa5, 0x1f, 0x5f, 0xa8, 0xaa, 0x26, 0x3e, 0x2e, 0x58, 0x96, 0xda, 0xa6, 0x16, + 0xce, 0xc1, 0x39, 0x21, 0xb2, 0x21, 0xcf, 0xd2, 0x31, 0xad, 0xab, 0x7c, 0x5b, 0xdb, 0xf2, 0x75, + 0x37, 0x4b, 0x8b, 0x46, 0xf3, 0xc2, 0x75, 0x3c, 0x79, 0x47, 0x5a, 0xc7, 0x19, 0xae, 0xe1, 0xe8, + 0xff, 0xa7, 0xc2, 0x7b, 0x67, 0xbc, 0x78, 0x41, 0x5f, 0x1e, 0x99, 0xf5, 0xfc, 0xe4, 0x8d, 0xde, + 0xdf, 0xf0, 0x2c, 0x7e, 0x64, 0x00, 0x94, 0x98, 0xbf, 0x2a, 0x27, 0xcd, 0x5f, 0xe7, 0x07, 0xa7, + 0xe9, 0x93, 0x07, 0xa7, 0xbf, 0xe3, 0xe1, 0xce, 0xe2, 0x3f, 0x28, 0x90, 0x61, 0x8b, 0x43, 0x5f, + 0x80, 0xea, 0xe0, 0x70, 0xf8, 0x46, 0xd3, 0x58, 0xa6, 0x79, 0x91, 0x61, 0xac, 0xfc, 0x8a, 0x22, + 0xfd, 0x36, 0x5f, 0x51, 0x34, 0x21, 0x1f, 0x4f, 0x71, 0xd4, 0x4b, 0x4e, 0x71, 0x62, 0xcd, 0xd9, + 0x21, 0x22, 0xf3, 0x36, 0x87, 0x08, 0xfd, 0xdf, 0x52, 0x50, 0x99, 0x7f, 0x69, 0x84, 0xbe, 0x80, + 0x0c, 0xff, 0xa2, 0x21, 0x75, 0xe9, 0x2f, 0x1a, 0xb8, 0x22, 0x1a, 0x40, 0x9c, 0x31, 0x9d, 0x73, + 0x46, 0x8c, 0xf3, 0x8f, 0x61, 0x62, 0x7f, 0xb5, 0x18, 0x41, 0x8e, 0x66, 0xee, 0x82, 0xe6, 0x4f, + 0xc7, 0x6c, 0x9a, 0x60, 0x4d, 0x70, 0x60, 0x8d, 0xb0, 0xcf, 0xb3, 0x81, 0x59, 0xf6, 0xa7, 0xe3, + 0x06, 0xf1, 0xc2, 0x0d, 0x1c, 0xac, 0x62, 0x5f, 0xff, 0x45, 0x09, 0x4a, 0xc9, 0x17, 0x58, 0xe8, + 0x16, 0x14, 0x27, 0x76, 0x10, 0xb9, 0x6c, 0x64, 0x71, 0x28, 0xc6, 0xb2, 0x49, 0x12, 0xea, 0x43, + 0x81, 0xbf, 0xe4, 0x6a, 0xc7, 0x96, 0xae, 0x5c, 0xec, 0xed, 0x98, 0xb8, 0x68, 0x4b, 0x17, 0xcc, + 0x70, 0x16, 0xff, 0x36, 0x25, 0x4e, 0xf3, 0x7d, 0x28, 0xcb, 0x09, 0x09, 0x6e, 0xbf, 0xe9, 0x3c, + 0xc2, 0x9c, 0xc7, 0x40, 0x4f, 0x01, 0xc4, 0xad, 0x28, 0xa2, 0xc2, 0x10, 0x1f, 0x5e, 0xce, 0x66, + 0x8a, 0x9a, 0x00, 0xe1, 0xef, 0xe2, 0x16, 0xff, 0x23, 0x03, 0x99, 0x76, 0x40, 0xdb, 0x8d, 0x27, + 0xa0, 0x8e, 0x89, 0x23, 0x2b, 0xef, 0x45, 0xc1, 0x99, 0xee, 0xf2, 0x3a, 0x71, 0xe2, 0x47, 0x95, + 0x82, 0xa0, 0xa7, 0x90, 0xdd, 0x26, 0x53, 0xdf, 0x09, 0x45, 0xc7, 0xf9, 0xf1, 0xa5, 0xe0, 0xea, + 0x4c, 0x55, 0x26, 0x0e, 0x0e, 0xb4, 0xf8, 0xbf, 0x29, 0xc8, 0x30, 0x06, 0xfa, 0x0a, 0x0a, 0x8c, + 0x36, 0x98, 0x35, 0x0a, 0x9f, 0x5c, 0x1e, 0x3f, 0x11, 0x7e, 0x33, 0x38, 0x9a, 0x08, 0x5d, 0x3f, + 0xb2, 0xc4, 0xd7, 0x39, 0xc9, 0xc4, 0x50, 0x70, 0xfd, 0xa8, 0xc7, 0x3f, 0xd0, 0xb9, 0x0d, 0x25, + 0x9a, 0x90, 0x1c, 0x29, 0x96, 0x66, 0x2d, 0x5a, 0x91, 0xd1, 0x84, 0xc8, 0x1a, 0x14, 0x39, 0x93, + 0xb7, 0x33, 0xfc, 0xd1, 0xbf, 0xc4, 0x8b, 0x43, 0xe0, 0xda, 0xd4, 0xa6, 0xc5, 0xbf, 0x4a, 0x41, + 0x96, 0xbb, 0x04, 0x75, 0x21, 0x13, 0x46, 0x76, 0x10, 0x89, 0xcc, 0xf7, 0xe8, 0xf2, 0xcb, 0x8e, + 0x33, 0x02, 0x85, 0x41, 0xcd, 0x59, 0x8f, 0xf9, 0x46, 0x68, 0xac, 0x2f, 0xd5, 0xdf, 0x03, 0x95, + 0x46, 0x00, 0x3d, 0x99, 0x9b, 0xb5, 0xee, 0x6a, 0x4b, 0xbb, 0x82, 0xf2, 0xa0, 0xb2, 0x69, 0x51, + 0x4a, 0x7f, 0x09, 0x85, 0xd8, 0xdf, 0xe8, 0x06, 0x5c, 0xdb, 0xec, 0xd6, 0x7b, 0x9b, 0xdd, 0x66, + 0xab, 0x69, 0x6d, 0x98, 0xad, 0x46, 0xab, 0x69, 0x74, 0x57, 0xb5, 0x2b, 0xf3, 0x8c, 0x76, 0xaf, + 0xd3, 0xe9, 0x3d, 0xa3, 0x8c, 0x14, 0xba, 0x0e, 0x5a, 0xaf, 0xdd, 0xee, 0xb7, 0x06, 0x09, 0x71, + 0x25, 0x41, 0x9d, 0xc9, 0xa6, 0xd1, 0x02, 0x14, 0x1b, 0x9b, 0xa6, 0xd9, 0xe2, 0xa3, 0x2a, 0x4d, + 0x5d, 0xfc, 0x17, 0x05, 0xf2, 0xf2, 0x91, 0x45, 0xad, 0xc4, 0x7c, 0xf0, 0xcc, 0xd7, 0xd8, 0xf3, + 0x8b, 0x3d, 0x3a, 0x1d, 0xfc, 0x00, 0x8a, 0x76, 0x30, 0x32, 0x9c, 0x83, 0x3e, 0xdb, 0x88, 0x64, + 0x88, 0x24, 0x19, 0xe8, 0x16, 0xe4, 0xed, 0x60, 0xd4, 0x20, 0x53, 0x51, 0x50, 0xe3, 0x02, 0x23, + 0xa9, 0xbf, 0xa1, 0xda, 0x50, 0x87, 0xcc, 0x4e, 0x20, 0xdb, 0xcb, 0x33, 0xbf, 0x99, 0x39, 0xbe, + 0x89, 0x26, 0x57, 0x45, 0xf7, 0x60, 0x6e, 0x72, 0x29, 0xbe, 0xdf, 0x13, 0xc3, 0x82, 0x24, 0x47, + 0xff, 0x65, 0x0a, 0x60, 0x96, 0x50, 0x50, 0x05, 0xc0, 0xec, 0x3d, 0xb3, 0xba, 0x9b, 0xeb, 0xf5, + 0x96, 0x29, 0xb6, 0xbd, 0xd6, 0x7d, 0xc2, 0xe7, 0x74, 0xcd, 0x56, 0xb7, 0xdf, 0xb2, 0xd8, 0xb5, + 0x82, 0x34, 0x28, 0x6d, 0xb4, 0xcc, 0x06, 0xdb, 0x1b, 0x4a, 0x49, 0xa3, 0x32, 0x14, 0x1a, 0x9b, + 0xeb, 0x2d, 0xab, 0x69, 0xf4, 0x07, 0x7c, 0x7e, 0xd9, 0x1d, 0x18, 0x9d, 0x16, 0x9f, 0x5f, 0x76, + 0x6a, 0xab, 0x5a, 0x96, 0xc2, 0x75, 0x5a, 0xb5, 0xa6, 0x96, 0xa3, 0x5b, 0xdb, 0x36, 0xcc, 0xfe, + 0xc0, 0xda, 0xaa, 0x75, 0x36, 0x5b, 0x5a, 0x9e, 0xe2, 0x77, 0x6a, 0xf1, 0x75, 0x81, 0xa2, 0x75, + 0x07, 0x8f, 0xc5, 0x25, 0x3c, 0xd0, 0xe5, 0xe1, 0x82, 0x85, 0xdd, 0x75, 0xd0, 0x1e, 0x77, 0x3a, + 0xd6, 0x46, 0x67, 0xb3, 0xcf, 0x7f, 0xb6, 0x1e, 0x6a, 0x57, 0xea, 0x77, 0xbe, 0xf9, 0xef, 0x9b, + 0x57, 0xbe, 0x79, 0x7d, 0x33, 0xf5, 0xaf, 0xaf, 0x6f, 0xa6, 0x7e, 0xf5, 0xfa, 0x66, 0xea, 0xbf, + 0x5e, 0xdf, 0x4c, 0xfd, 0xd9, 0xaf, 0x6f, 0x5e, 0xf9, 0x0a, 0x66, 0x5e, 0xfb, 0xff, 0x00, 0x00, + 0x00, 0xff, 0xff, 0xc7, 0x87, 0xe6, 0xc8, 0x6f, 0x2b, 0x00, 0x00, } diff --git a/pkg/sql/distsqlrun/processors.proto b/pkg/sql/distsqlrun/processors.proto index 52f375d3aca8..1cfb8e79f9d8 100644 --- a/pkg/sql/distsqlrun/processors.proto +++ b/pkg/sql/distsqlrun/processors.proto @@ -138,6 +138,7 @@ message ProcessorCoreUnion { optional ZigzagJoinerSpec zigzagJoiner = 21; optional ProjectSetSpec projectSet = 22; optional WindowerSpec windower = 23; + optional LocalPlanNodeSpec localPlanNode = 24; reserved 6, 12; } @@ -323,6 +324,17 @@ message ZigzagJoinerSpec { optional sqlbase.JoinType type = 5 [(gogoproto.nullable) = false]; } +// LocalPlanNodeSpec is the specification for a local planNode wrapping +// processor. It's created for situations where a planNode has no DistSQL +// processor equivalent, and constrains the plan to live on the gateway node. +// This spec contains just an index, which is used by the execution engine to +// find the planNode instance this processor is executing in an array of local +// planNodes. See LocalProcessors and LocalProcessorIndexes on +// distsqlplan.PhysicalPlan. +message LocalPlanNodeSpec { + optional uint32 RowSourceIdx = 1; +} + message Columns { repeated uint32 columns = 1 [packed = true]; } diff --git a/pkg/sql/distsqlrun/processors_test.go b/pkg/sql/distsqlrun/processors_test.go index ccd49ec7786a..bb499b131712 100644 --- a/pkg/sql/distsqlrun/processors_test.go +++ b/pkg/sql/distsqlrun/processors_test.go @@ -399,9 +399,10 @@ func TestProcessorBaseContext(t *testing.T) { st := cluster.MakeTestingClusterSettings() runTest := func(t *testing.T, f func(noop *noopProcessor)) { + evalCtx := tree.MakeTestingEvalContext(st) flowCtx := &FlowCtx{ Settings: st, - EvalCtx: tree.MakeTestingEvalContext(st), + EvalCtx: &evalCtx, } defer flowCtx.EvalCtx.Stop(ctx) diff --git a/pkg/sql/distsqlrun/routers_test.go b/pkg/sql/distsqlrun/routers_test.go index 7d09253eddd6..2dbf036b04df 100644 --- a/pkg/sql/distsqlrun/routers_test.go +++ b/pkg/sql/distsqlrun/routers_test.go @@ -57,7 +57,7 @@ func setupRouter( } ctx := context.TODO() - flowCtx := FlowCtx{Settings: cluster.MakeTestingClusterSettings(), EvalCtx: *evalCtx} + flowCtx := FlowCtx{Settings: cluster.MakeTestingClusterSettings(), EvalCtx: evalCtx} r.init(ctx, &flowCtx, inputTypes) wg := &sync.WaitGroup{} r.start(ctx, wg, nil /* ctxCancel */) @@ -589,7 +589,8 @@ func TestRouterBlocks(t *testing.T) { } st := cluster.MakeTestingClusterSettings() ctx := context.TODO() - flowCtx := FlowCtx{Settings: st, EvalCtx: tree.MakeTestingEvalContext(st)} + evalCtx := tree.MakeTestingEvalContext(st) + flowCtx := FlowCtx{Settings: st, EvalCtx: &evalCtx} router.init(ctx, &flowCtx, colTypes) var wg sync.WaitGroup router.start(ctx, &wg, nil /* ctxCancel */) @@ -714,7 +715,7 @@ func TestRouterDiskSpill(t *testing.T) { defer evalCtx.Stop(ctx) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, TempStorage: tempEngine, diskMonitor: &diskMonitor, } @@ -801,7 +802,7 @@ func TestRouterDiskSpill(t *testing.T) { } // Verify correct order (should be the order in which we added rows). for j, c := range row { - if cmp, err := c.Compare(&intType, alloc, &flowCtx.EvalCtx, &rows[i][j]); err != nil { + if cmp, err := c.Compare(&intType, alloc, flowCtx.EvalCtx, &rows[i][j]); err != nil { t.Fatal(err) } else if cmp != 0 { t.Fatalf( diff --git a/pkg/sql/distsqlrun/sample_aggregator_test.go b/pkg/sql/distsqlrun/sample_aggregator_test.go index b47bdb0f9705..f4e226f05fea 100644 --- a/pkg/sql/distsqlrun/sample_aggregator_test.go +++ b/pkg/sql/distsqlrun/sample_aggregator_test.go @@ -44,7 +44,7 @@ func TestSampleAggregator(t *testing.T) { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, gossip: server.Gossip(), clientDB: kvDB, executor: server.InternalExecutor().(sqlutil.InternalExecutor), diff --git a/pkg/sql/distsqlrun/sampler_test.go b/pkg/sql/distsqlrun/sampler_test.go index f828f5bf6729..860b9a976d19 100644 --- a/pkg/sql/distsqlrun/sampler_test.go +++ b/pkg/sql/distsqlrun/sampler_test.go @@ -49,7 +49,7 @@ func runSampler(t *testing.T, numRows, numSamples int) []int { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } spec := &SamplerSpec{SampleSize: uint32(numSamples)} @@ -154,7 +154,7 @@ func TestSamplerSketch(t *testing.T) { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } spec := &SamplerSpec{ diff --git a/pkg/sql/distsqlrun/server.go b/pkg/sql/distsqlrun/server.go index 347fbcd01b22..80ba73a09175 100644 --- a/pkg/sql/distsqlrun/server.go +++ b/pkg/sql/distsqlrun/server.go @@ -17,9 +17,9 @@ package distsqlrun import ( "context" "io" - time "time" + "time" - opentracing "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "sync" @@ -282,6 +282,7 @@ func (ds *ServerImpl) setupFlow( parentMonitor *mon.BytesMonitor, req *SetupFlowRequest, syncFlowConsumer RowReceiver, + localState LocalState, ) (context.Context, *Flow, error) { if !FlowVerIsCompatible(req.Version, MinAcceptedVersion, Version) { err := errors.Errorf( @@ -319,6 +320,7 @@ func (ds *ServerImpl) setupFlow( monitor.Start(ctx, parentMonitor, mon.BoundAccount{}) acc := monitor.MakeBoundAccount() + txn := localState.Txn if txn := req.DeprecatedTxn; txn != nil { if req.TxnCoordMeta != nil { return nil, nil, errors.Errorf("provided both Txn and TxnCoordMeta") @@ -326,77 +328,87 @@ func (ds *ServerImpl) setupFlow( meta := roachpb.MakeTxnCoordMeta(*txn) req.TxnCoordMeta = &meta } - var txn *client.Txn if meta := req.TxnCoordMeta; meta != nil { - // The flow will run in a Txn that specifies child=true because we - // do not want each distributed Txn to heartbeat the transaction. - txn = client.NewTxnWithCoordMeta(ds.FlowDB, req.Flow.Gateway, client.LeafTxn, *meta) + if !localState.IsLocal { + // The flow will run in a Txn that specifies child=true because we + // do not want each distributed Txn to heartbeat the transaction. + txn = client.NewTxnWithCoordMeta(ds.FlowDB, req.Flow.Gateway, client.LeafTxn, *meta) + } } - location, err := timeutil.TimeZoneStringToLocation(req.EvalContext.Location) - if err != nil { - tracing.FinishSpan(sp) - return ctx, nil, err - } + var evalCtx *tree.EvalContext + if localState.EvalContext != nil { + evalCtx = localState.EvalContext + evalCtx.Mon = &monitor + evalCtx.ActiveMemAcc = &acc + evalCtx.Txn = txn + } else { + location, err := timeutil.TimeZoneStringToLocation(req.EvalContext.Location) + if err != nil { + tracing.FinishSpan(sp) + return ctx, nil, err + } - var be sessiondata.BytesEncodeFormat - switch req.EvalContext.BytesEncodeFormat { - case BytesEncodeFormat_HEX: - be = sessiondata.BytesEncodeHex - case BytesEncodeFormat_ESCAPE: - be = sessiondata.BytesEncodeEscape - case BytesEncodeFormat_BASE64: - be = sessiondata.BytesEncodeBase64 - default: - return nil, nil, errors.Errorf("unknown byte encode format: %s", - req.EvalContext.BytesEncodeFormat.String()) - } - sd := &sessiondata.SessionData{ - ApplicationName: req.EvalContext.ApplicationName, - Database: req.EvalContext.Database, - User: req.EvalContext.User, - SearchPath: sessiondata.MakeSearchPath(req.EvalContext.SearchPath), - SequenceState: sessiondata.NewSequenceState(), - DataConversion: sessiondata.DataConversionConfig{ - Location: location, - BytesEncodeFormat: be, - ExtraFloatDigits: int(req.EvalContext.ExtraFloatDigits), - }, - } - ie := &lazyInternalExecutor{ - newInternalExecutor: func() tree.SessionBoundInternalExecutor { - return ds.SessionBoundInternalExecutorFactory(ctx, sd) - }, - } + var be sessiondata.BytesEncodeFormat + switch req.EvalContext.BytesEncodeFormat { + case BytesEncodeFormat_HEX: + be = sessiondata.BytesEncodeHex + case BytesEncodeFormat_ESCAPE: + be = sessiondata.BytesEncodeEscape + case BytesEncodeFormat_BASE64: + be = sessiondata.BytesEncodeBase64 + default: + return nil, nil, errors.Errorf("unknown byte encode format: %s", + req.EvalContext.BytesEncodeFormat.String()) + } + sd := &sessiondata.SessionData{ + ApplicationName: req.EvalContext.ApplicationName, + Database: req.EvalContext.Database, + User: req.EvalContext.User, + SearchPath: sessiondata.MakeSearchPath(req.EvalContext.SearchPath), + SequenceState: sessiondata.NewSequenceState(), + DataConversion: sessiondata.DataConversionConfig{ + Location: location, + BytesEncodeFormat: be, + ExtraFloatDigits: int(req.EvalContext.ExtraFloatDigits), + }, + } + ie := &lazyInternalExecutor{ + newInternalExecutor: func() tree.SessionBoundInternalExecutor { + return ds.SessionBoundInternalExecutorFactory(ctx, sd) + }, + } - evalCtx := tree.EvalContext{ - Settings: ds.ServerConfig.Settings, - SessionData: sd, - ClusterID: ds.ServerConfig.ClusterID.Get(), - NodeID: nodeID, - ReCache: ds.regexpCache, - Mon: &monitor, - ActiveMemAcc: &acc, - // TODO(andrei): This is wrong. Each processor should override Ctx with its - // own context. - CtxProvider: simpleCtxProvider{ctx: ctx}, - Txn: txn, - Planner: &dummyEvalPlanner{}, - Sequence: &dummySequenceOperators{}, - InternalExecutor: ie, - } - evalCtx.SetStmtTimestamp(timeutil.Unix(0 /* sec */, req.EvalContext.StmtTimestampNanos)) - evalCtx.SetTxnTimestamp(timeutil.Unix(0 /* sec */, req.EvalContext.TxnTimestampNanos)) - var haveSequences bool - for _, seq := range req.EvalContext.SeqState.Seqs { - evalCtx.SessionData.SequenceState.RecordValue(seq.SeqID, seq.LatestVal) - haveSequences = true - } - if haveSequences { - evalCtx.SessionData.SequenceState.SetLastSequenceIncremented( - *req.EvalContext.SeqState.LastSeqIncremented) + evalPlanner := &dummyEvalPlanner{} + sequence := &dummySequenceOperators{} + evalCtx = &tree.EvalContext{ + Settings: ds.ServerConfig.Settings, + SessionData: sd, + ClusterID: ds.ServerConfig.ClusterID.Get(), + NodeID: nodeID, + ReCache: ds.regexpCache, + Mon: &monitor, + ActiveMemAcc: &acc, + // TODO(andrei): This is wrong. Each processor should override Ctx with its + // own context. + CtxProvider: simpleCtxProvider{ctx: ctx}, + Txn: txn, + Planner: evalPlanner, + Sequence: sequence, + InternalExecutor: ie, + } + evalCtx.SetStmtTimestamp(timeutil.Unix(0 /* sec */, req.EvalContext.StmtTimestampNanos)) + evalCtx.SetTxnTimestamp(timeutil.Unix(0 /* sec */, req.EvalContext.TxnTimestampNanos)) + var haveSequences bool + for _, seq := range req.EvalContext.SeqState.Seqs { + evalCtx.SessionData.SequenceState.RecordValue(seq.SeqID, seq.LatestVal) + haveSequences = true + } + if haveSequences { + evalCtx.SessionData.SequenceState.SetLastSequenceIncremented( + *req.EvalContext.SeqState.LastSeqIncremented) + } } - // TODO(radu): we should sanity check some of these fields. flowCtx := FlowCtx{ Settings: ds.Settings, @@ -417,17 +429,17 @@ func (ds *ServerImpl) setupFlow( JobRegistry: ds.ServerConfig.JobRegistry, traceKV: req.TraceKV, } - - ctx = flowCtx.AnnotateCtx(ctx) - - f := newFlow(flowCtx, ds.flowRegistry, syncFlowConsumer) - flowCtx.AddLogTagStr("f", f.id.Short()) + f := newFlow(flowCtx, ds.flowRegistry, syncFlowConsumer, localState.LocalProcs) if err := f.setup(ctx, &req.Flow); err != nil { log.Errorf(ctx, "error setting up flow: %s", err) tracing.FinishSpan(sp) ctx = opentracing.ContextWithSpan(ctx, nil) return ctx, nil, err } + if !f.isLocal() { + flowCtx.AddLogTagStr("f", f.id.Short()) + flowCtx.AnnotateCtx(ctx) + } return ctx, f, nil } @@ -439,7 +451,32 @@ func (ds *ServerImpl) setupFlow( func (ds *ServerImpl) SetupSyncFlow( ctx context.Context, parentMonitor *mon.BytesMonitor, req *SetupFlowRequest, output RowReceiver, ) (context.Context, *Flow, error) { - return ds.setupFlow(ds.AnnotateCtx(ctx), opentracing.SpanFromContext(ctx), parentMonitor, req, output) + return ds.setupFlow(ds.AnnotateCtx(ctx), opentracing.SpanFromContext(ctx), parentMonitor, req, output, LocalState{}) +} + +// LocalState carries information that is required to set up a flow with wrapped +// planNodes. +type LocalState struct { + // IsLocal is true if the flow is being run locally in the first place. + IsLocal bool + // LocalProcs is an array of planNodeToRowSource processors. It's in order and + // will be indexed into by the RowSourceIdx field in LocalPlanNodeSpec. + LocalProcs []LocalProcessor + EvalContext *tree.EvalContext + Txn *client.Txn +} + +// SetupLocalSyncFlow sets up a synchronous flow on the current (planning) node. +// It's used by the gateway node to set up the flows local to it. Otherwise, +// the same as SetupSyncFlow. +func (ds *ServerImpl) SetupLocalSyncFlow( + ctx context.Context, + parentMonitor *mon.BytesMonitor, + req *SetupFlowRequest, + output RowReceiver, + localState LocalState, +) (context.Context, *Flow, error) { + return ds.setupFlow(ctx, opentracing.SpanFromContext(ctx), parentMonitor, req, output, localState) } // RunSyncFlow is part of the DistSQLServer interface. @@ -488,7 +525,7 @@ func (ds *ServerImpl) SetupFlow( // Note: the passed context will be canceled when this RPC completes, so we // can't associate it with the flow. ctx = ds.AnnotateCtx(context.Background()) - ctx, f, err := ds.setupFlow(ctx, parentSpan, &ds.memMonitor, req, nil /* syncFlowConsumer */) + ctx, f, err := ds.setupFlow(ctx, parentSpan, &ds.memMonitor, req, nil /* syncFlowConsumer */, LocalState{}) if err == nil { err = ds.flowScheduler.ScheduleFlow(ctx, f) } diff --git a/pkg/sql/distsqlrun/sorter_test.go b/pkg/sql/distsqlrun/sorter_test.go index 27094593f6c1..e946c1899ef1 100644 --- a/pkg/sql/distsqlrun/sorter_test.go +++ b/pkg/sql/distsqlrun/sorter_test.go @@ -312,7 +312,7 @@ func TestSorter(t *testing.T) { diskMonitor.Start(ctx, nil /* pool */, mon.MakeStandaloneBudget(math.MaxInt64)) defer diskMonitor.Stop(ctx) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: cluster.MakeTestingClusterSettings(), TempStorage: tempEngine, diskMonitor: &diskMonitor, @@ -397,7 +397,7 @@ func BenchmarkSortAll(b *testing.B) { defer evalCtx.Stop(ctx) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) @@ -437,7 +437,7 @@ func BenchmarkSortLimit(b *testing.B) { defer evalCtx.Stop(ctx) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) @@ -481,7 +481,7 @@ func BenchmarkSortChunks(b *testing.B) { defer evalCtx.Stop(ctx) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } rng := rand.New(rand.NewSource(timeutil.Now().UnixNano())) diff --git a/pkg/sql/distsqlrun/tablereader.go b/pkg/sql/distsqlrun/tablereader.go index f881181e6918..104eb05c8923 100644 --- a/pkg/sql/distsqlrun/tablereader.go +++ b/pkg/sql/distsqlrun/tablereader.go @@ -203,7 +203,6 @@ func (tr *tableReader) Start(ctx context.Context) context.Context { // This call doesn't do much; the real "starting" is below. tr.input.Start(fetcherCtx) - // TODO(radu,andrei,knz): set the traceKV flag when requested by the session. if err := tr.fetcher.StartScan( fetcherCtx, tr.flowCtx.txn, tr.spans, true /* limit batches */, tr.limitHint, tr.flowCtx.traceKV, diff --git a/pkg/sql/distsqlrun/tablereader_test.go b/pkg/sql/distsqlrun/tablereader_test.go index 55c695c0e8f7..27798a372cc0 100644 --- a/pkg/sql/distsqlrun/tablereader_test.go +++ b/pkg/sql/distsqlrun/tablereader_test.go @@ -128,7 +128,7 @@ func TestTableReader(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(s.ClusterSettings()) defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: s.ClusterSettings(), txn: client.NewTxn(s.DB(), s.NodeID(), client.RootTxn), nodeID: s.NodeID(), @@ -212,7 +212,7 @@ ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[2], 1), (ARRAY[1], 2), (ARRAY[ defer evalCtx.Stop(context.Background()) nodeID := tc.Server(0).NodeID() flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: st, txn: client.NewTxn(tc.Server(0).DB(), nodeID, client.RootTxn), nodeID: nodeID, @@ -316,7 +316,7 @@ func TestLimitScans(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(s.ClusterSettings()) defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: s.ClusterSettings(), txn: client.NewTxn(kvDB, s.NodeID(), client.RootTxn), nodeID: s.NodeID(), @@ -409,7 +409,7 @@ func BenchmarkTableReader(b *testing.B) { defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: s.ClusterSettings(), txn: client.NewTxn(s.DB(), s.NodeID(), client.RootTxn), nodeID: s.NodeID(), diff --git a/pkg/sql/distsqlrun/utils_test.go b/pkg/sql/distsqlrun/utils_test.go index 2725734ea529..2344f60b437b 100644 --- a/pkg/sql/distsqlrun/utils_test.go +++ b/pkg/sql/distsqlrun/utils_test.go @@ -329,13 +329,13 @@ func runProcessorTest( defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, txn: txn, } p, err := newProcessor( context.Background(), &flowCtx, 0 /* processorID */, &core, &post, - []RowSource{in}, []RowReceiver{out}) + []RowSource{in}, []RowReceiver{out}, []LocalProcessor{}) if err != nil { t.Fatal(err) } diff --git a/pkg/sql/distsqlrun/values_test.go b/pkg/sql/distsqlrun/values_test.go index 95ef6263b214..bc713c3a054c 100644 --- a/pkg/sql/distsqlrun/values_test.go +++ b/pkg/sql/distsqlrun/values_test.go @@ -71,8 +71,11 @@ func TestValuesProcessor(t *testing.T) { out := &RowBuffer{} st := cluster.MakeTestingClusterSettings() + evalCtx := tree.NewTestingEvalContext(st) + defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ Settings: st, + EvalCtx: evalCtx, } v, err := newValuesProcessor(&flowCtx, 0 /* processorID */, &spec, &PostProcessSpec{}, out) @@ -97,8 +100,6 @@ func TestValuesProcessor(t *testing.T) { t.Fatalf("incorrect number of rows %d, expected %d", len(res), numRows) } - evalCtx := tree.NewTestingEvalContext(st) - defer evalCtx.Stop(context.Background()) var a sqlbase.DatumAlloc for i := 0; i < numRows; i++ { if len(res[i]) != numCols { @@ -133,7 +134,7 @@ func BenchmarkValuesProcessor(b *testing.B) { flowCtx := FlowCtx{ Settings: st, - EvalCtx: evalCtx, + EvalCtx: &evalCtx, } post := PostProcessSpec{} output := RowDisposer{} diff --git a/pkg/sql/distsqlrun/zigzagjoiner.go b/pkg/sql/distsqlrun/zigzagjoiner.go index 2af6b778ee3d..39c7f2b68405 100644 --- a/pkg/sql/distsqlrun/zigzagjoiner.go +++ b/pkg/sql/distsqlrun/zigzagjoiner.go @@ -605,7 +605,7 @@ func (z *zigzagJoiner) matchBase(curRow sqlbase.EncDatumRow, side int) (bool, er // Compare the equality columns of the baseRow to that of the curRow. da := &sqlbase.DatumAlloc{} - cmp, err := prevEqDatums.Compare(eqColTypes, da, ordering, &z.flowCtx.EvalCtx, curEqDatums) + cmp, err := prevEqDatums.Compare(eqColTypes, da, ordering, z.flowCtx.EvalCtx, curEqDatums) if err != nil { return false, err } @@ -754,7 +754,7 @@ func (z *zigzagJoiner) nextRow( return nil, z.producerMeta(err) } da := &sqlbase.DatumAlloc{} - cmp, err := prevEqCols.Compare(eqColTypes, da, ordering, &z.flowCtx.EvalCtx, currentEqCols) + cmp, err := prevEqCols.Compare(eqColTypes, da, ordering, z.flowCtx.EvalCtx, currentEqCols) if err != nil { return nil, z.producerMeta(err) } diff --git a/pkg/sql/distsqlrun/zigzagjoiner_test.go b/pkg/sql/distsqlrun/zigzagjoiner_test.go index 3cfab7676305..9b1237eb2118 100644 --- a/pkg/sql/distsqlrun/zigzagjoiner_test.go +++ b/pkg/sql/distsqlrun/zigzagjoiner_test.go @@ -495,7 +495,7 @@ func TestZigzagJoiner(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(st) defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: st, txn: client.NewTxn(s.DB(), s.NodeID(), client.RootTxn), } @@ -558,7 +558,7 @@ func TestZigzagJoinerDrain(t *testing.T) { evalCtx := tree.MakeTestingEvalContext(s.ClusterSettings()) defer evalCtx.Stop(context.Background()) flowCtx := FlowCtx{ - EvalCtx: evalCtx, + EvalCtx: &evalCtx, Settings: s.ClusterSettings(), txn: client.NewTxn(s.DB(), s.NodeID(), client.RootTxn), } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 9fe17ddc258f..3f19ff97a919 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -469,38 +469,42 @@ func countRowsAffected(params runParams, p planNode) (int, error) { return count, err } -// shouldUseDistSQL determines whether we should use DistSQL for the +func shouldDistributeGivenRecAndMode( + rec distRecommendation, mode sessiondata.DistSQLExecMode, +) bool { + switch mode { + case sessiondata.DistSQLOff: + return false + case sessiondata.DistSQLAuto: + return rec == shouldDistribute + case sessiondata.DistSQLOn, sessiondata.DistSQLAlways: + return rec != cannotDistribute + } + panic(fmt.Sprintf("unhandled distsql mode %v", mode)) +} + +// shouldDistributePlan determines whether we should distribute the // given logical plan, based on the session settings. -func shouldUseDistSQL( +func shouldDistributePlan( ctx context.Context, distSQLMode sessiondata.DistSQLExecMode, dp *DistSQLPlanner, plan planNode, -) (bool, error) { +) bool { if distSQLMode == sessiondata.DistSQLOff { - return false, nil + return false } // Don't try to run empty nodes (e.g. SET commands) with distSQL. if _, ok := plan.(*zeroNode); ok { - return false, nil + return false } - distribute, err := dp.CheckSupport(plan) + rec, err := dp.checkSupportForNode(plan) if err != nil { - // If the distSQLMode is ALWAYS, reject anything but SET. - if distSQLMode == sessiondata.DistSQLAlways && err != setNotSupportedError { - return false, err - } // Don't use distSQL for this request. log.VEventf(ctx, 1, "query not supported for distSQL: %s", err) - return false, nil - } - - if distSQLMode == sessiondata.DistSQLAuto && !distribute { - log.VEventf(ctx, 1, "not distributing query") - return false, nil + return false } - // In ON or ALWAYS mode, all supported queries are distributed. - return true, nil + return shouldDistributeGivenRecAndMode(rec, distSQLMode) } // golangFillQueryArguments transforms Go values into datums. diff --git a/pkg/sql/explain.go b/pkg/sql/explain.go index b43a3106b262..7b3bfef1e06f 100644 --- a/pkg/sql/explain.go +++ b/pkg/sql/explain.go @@ -42,8 +42,9 @@ func (p *planner) Explain(ctx context.Context, n *tree.Explain) (planNode, error return nil, err } return &explainDistSQLNode{ - plan: plan, - analyze: opts.Flags.Contains(tree.ExplainFlagAnalyze), + plan: plan, + analyze: opts.Flags.Contains(tree.ExplainFlagAnalyze), + stmtType: n.Statement.StatementType(), }, nil case tree.ExplainPlan: diff --git a/pkg/sql/explain_distsql.go b/pkg/sql/explain_distsql.go index f5301803d9f5..3b795d094f53 100644 --- a/pkg/sql/explain_distsql.go +++ b/pkg/sql/explain_distsql.go @@ -31,6 +31,8 @@ type explainDistSQLNode struct { plan planNode + stmtType tree.StatementType + // If analyze is set, plan will be executed with tracing enabled and a url // pointing to a visual query plan with statistics will be in the row // returned by the node. @@ -49,21 +51,25 @@ type explainDistSQLRun struct { } func (n *explainDistSQLNode) startExec(params runParams) error { - // Check for subqueries and trigger limit propagation. - if _, err := params.p.prepareForDistSQLSupportCheck( - params.ctx, true, /* returnError */ - ); err != nil { - return err - } + // Trigger limit propagation. + params.p.prepareForDistSQLSupportCheck() distSQLPlanner := params.extendedEvalCtx.DistSQLPlanner - auto, err := distSQLPlanner.CheckSupport(n.plan) - if err != nil { - return err - } + recommendation, _ := distSQLPlanner.checkSupportForNode(n.plan) planCtx := distSQLPlanner.newPlanningCtx(params.ctx, params.extendedEvalCtx, params.p.txn) - plan, err := distSQLPlanner.createPlanForNode(&planCtx, n) + planCtx.isLocal = !shouldDistributeGivenRecAndMode(recommendation, params.SessionData().DistSQLMode) + planCtx.planner = params.p + planCtx.stmtType = n.stmtType + planCtx.validExtendedEvalCtx = true + + // This stanza ensures that EXPLAIN(DISTSQL) won't include metadata test + // senders or receivers. + curTol := distSQLPlanner.metadataTestTolerance + distSQLPlanner.metadataTestTolerance = distsqlrun.On + defer func() { distSQLPlanner.metadataTestTolerance = curTol }() + + plan, err := distSQLPlanner.createPlanForNode(&planCtx, n.plan) if err != nil { return err } @@ -85,17 +91,16 @@ func (n *explainDistSQLNode) startExec(params runParams) error { return err } + planCtx.ctx = params.extendedEvalCtx.Tracing.ex.ctxHolder.ctx() + // Discard rows that are returned. rw := newCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { return nil }) execCfg := params.p.ExecCfg() - // tree.RowsAffected is used as the statement type passed in to the distsql - // receiver because it isn't necessary to process any result rows from the - // wrapped plan. - const stmtType = tree.RowsAffected + const stmtType = tree.Rows recv := makeDistSQLReceiver( - params.ctx, + planCtx.ctx, rw, stmtType, execCfg.RangeDescriptorCache, @@ -104,9 +109,9 @@ func (n *explainDistSQLNode) startExec(params runParams) error { func(ts hlc.Timestamp) { _ = execCfg.Clock.Update(ts) }, - params.p.ExtendedEvalContext().Tracing, + params.extendedEvalCtx.Tracing, ) - distSQLPlanner.Run(&planCtx, params.p.txn, &plan, recv, params.p.ExtendedEvalContext()) + distSQLPlanner.Run(&planCtx, params.p.txn, &plan, recv, params.extendedEvalCtx) spans = params.extendedEvalCtx.Tracing.getRecording() if err := params.extendedEvalCtx.Tracing.StopTracing(); err != nil { @@ -121,7 +126,7 @@ func (n *explainDistSQLNode) startExec(params runParams) error { } n.run.values = tree.Datums{ - tree.MakeDBool(tree.DBool(auto)), + tree.MakeDBool(tree.DBool(recommendation == shouldDistribute)), tree.NewDString(planURL.String()), tree.NewDString(planJSON), } diff --git a/pkg/sql/logictest/testdata/logic_test/builtin_function b/pkg/sql/logictest/testdata/logic_test/builtin_function index 1eea115516dc..0c9abd407d02 100644 --- a/pkg/sql/logictest/testdata/logic_test/builtin_function +++ b/pkg/sql/logictest/testdata/logic_test/builtin_function @@ -562,8 +562,9 @@ SELECT cbrt(-1.0::float), round(cbrt(27.0::float), 15), cbrt(19.3::decimal) ---- -1 3 2.6823725926296729544 + query RRRRR -SELECT ceil(-0.5::float), ceil(0.5::float), ceiling(0.5::float), ceil(0.1::decimal), ceiling(-0.9::decimal) +SELECT ceil(-0.5::float), ceil(0.5::float), ceiling(0.5::float), ceil(0.1::decimal), ceiling (-0.9::decimal) ---- -0 1 1 1 0 diff --git a/pkg/sql/logictest/testdata/logic_test/union b/pkg/sql/logictest/testdata/logic_test/union index 60f3fa84dca1..0f4b6c8b5281 100644 --- a/pkg/sql/logictest/testdata/logic_test/union +++ b/pkg/sql/logictest/testdata/logic_test/union @@ -77,7 +77,7 @@ VALUES (NULL) UNION ALL VALUES (NULL) NULL NULL -query IT +query IT rowsort SELECT x, pg_typeof(y) FROM (SELECT 1, NULL UNION ALL SELECT 2, 4) AS t(x, y) ---- 1 unknown diff --git a/pkg/sql/logictest/testdata/planner_test/distsql_auto_mode b/pkg/sql/logictest/testdata/planner_test/distsql_auto_mode index 674e7be6857b..7e0180a934be 100644 --- a/pkg/sql/logictest/testdata/planner_test/distsql_auto_mode +++ b/pkg/sql/logictest/testdata/planner_test/distsql_auto_mode @@ -118,12 +118,16 @@ SELECT automatic FROM [EXPLAIN (DISTSQL) SELECT * FROM abc WHERE b=1 AND a%2=0] true # OID cast - don't distribute (#22249). -statement error pq: cast to REGCLASS is not supported by distsql +query BT EXPLAIN (DISTSQL) SELECT t1.a FROM abc t1 INNER JOIN abc t2 on t1.a::REGCLASS = t2.a::REGCLASS; +---- +false https://cockroachdb.github.io/distsqlplan/decode.html#eJykkc1qwzAMgO97iqCzobF7Mwwyxtg6RjPS3UYObiy6QGoF2YGNkncfcQ5ZSvfT9ShZnz5JPoAji2uzRw_6FSSUAlqmCr0nHlJjwcq-g04F1K7twpAuBVTECPoAoQ4NgoYXs22wQGORFykIsBhM3cS2Ldd7wx-Z2VYgIO-CTjIJZS-AujB19MHsELTsxf-s8hKr-tY6yTpHbJHRzkTlQP5WcmL0B-PfHql2yAs1nzxfJ5nUuri7v3262WyS6yRTU_iHZZbnnLBA35LzeLzUyc7psAnaHY6X8dRxhc9MVdSMYR65mLDow_iqxmDl4lP846-wPANWx7D6EV7O4LQv-6vPAAAA___WlPjR # Query with OID expression - don't distribute (#24423). -statement error pq: OID expressions are not supported by distsql +query BT EXPLAIN (DISTSQL) SELECT 246::REGTYPE FROM abc +---- +false https://cockroachdb.github.io/distsqlplan/decode.html#eJyMjzFLBDEQhXt_hbw64K6IxVQ2h1yjx3KNSIpc8jgW9pJlkgXlyH-X3RRiIVz53pv53swVMQW-uQsz5BM9rMGsyTPnpKvVBvbhC9IZjHFeympbA5-UkCvKWCZCcHSniQNdoD50MAgsbpw27Kzjxen3izt5GAyMgSr3j0_PIjLsXo8fhx1sNUhL-eXn4s6E9NXcfsPAPKeY-af-P3JXrQHDme3PnBb1PGjyW02T79veZgTm0tK-iX1sUbX17icAAP__JLhsAQ== # Verify that EXPLAIN ANALYZE (DISTSQL) annotates plans with collected # statistics. @@ -147,10 +151,13 @@ SELECT url FROM [EXPLAIN ANALYZE (DISTSQL) SELECT DISTINCT(kw.w) FROM kv JOIN kw ---- https://cockroachdb.github.io/distsqlplan/decode.html#eJyck8FunDAQhu99CmtOrWppMZCLpUoo6qFppaZKe6s4uHi6sQIYzZgmUbTvXmEipbCBLnuD3_PBN_b4CVpv8atpkEH_BAWlhI58hcyehmgsuLIPoBMJru36MMSlhMoTgn6C4EKNoOGH-VXjDRqLtEtAgsVgXB0_25FrDD0Wd39AAvl7FoTGaqFAAgdT1yK4BrVIGCRc90GLQkF5kOD78PJDDmaPoNVBnielFqTuT5ZKF6XSRakXl771ZJHQTjzKgfxfySudfTJ8-9m7FmmXThur8Xd4W6h3H8jtb-MTyBiKWZcxO2o1UvPSMTyq5eAJrWBnUYtYAxIa8yAabDw9ip7RapEm4ou7fF6xju-e80RcnrC12Zbz_u4pIO2y6Y4U6v0JZ3zkrVa8l2zzLbYfHQfXVmGXz32HfRknId6FM-TXJC-2SN4gd75lnE_tq19OhlFFu8dx9Nn3VOE38lX8zfh6HbkYWOQwrqbjy1Ubl-Id_xdWG-B0DqercDaBkzmcrcL5OpyvwhczuDy8-RsAAP__mE7PKQ== -# Verify that EXPLAIN ANALYZE on an unsupported query returns an error. -query error unsupported node +# Verify that EXPLAIN ANALYZE on an unsupported query doesn't return an error. +statement ok EXPLAIN ANALYZE (DISTSQL) SHOW QUERIES; +statement ok +EXPLAIN ANALYZE (DISTSQL) EXPLAIN SELECT 1 + # This query verifies support for zeroNode in DistSQL. query B SELECT automatic FROM [EXPLAIN (DISTSQL) SELECT sum(k) FROM kv WHERE FALSE] diff --git a/pkg/sql/logictest/testdata/planner_test/show_trace b/pkg/sql/logictest/testdata/planner_test/show_trace index 3a514097a1b2..82af98e003a2 100644 --- a/pkg/sql/logictest/testdata/planner_test/show_trace +++ b/pkg/sql/logictest/testdata/planner_test/show_trace @@ -303,7 +303,7 @@ output row: [9] statement ok SET tracing = on,kv,results; VALUES (1, 2), (1, 1), (1, 2), (2, 1), (2, 1) UNION VALUES (1, 3), (3, 4), (1, 1); SET tracing = off -query T +query T rowsort SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'fetched:%' OR message LIKE 'output row%' ---- @@ -316,18 +316,18 @@ output row: [3 4] statement ok SET tracing = on,kv,results; SELECT * FROM abc EXCEPT SELECT * FROM abc WHERE b > 'p'; SET tracing = off -query T +query T rowsort SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'fetched:%' OR message LIKE 'output row%' ---- fetched: /abc/foo/'three' -> /3 fetched: /abc/foo/'two' -> /2 -fetched: /abc/primary/3/'three' -> NULL -fetched: /abc/primary/2/'two' -> NULL fetched: /abc/primary/1/'one' -> NULL -output row: [1 'one' NULL] +fetched: /abc/primary/2/'two' -> NULL fetched: /abc/primary/2/'two' -> NULL fetched: /abc/primary/3/'three' -> NULL +fetched: /abc/primary/3/'three' -> NULL +output row: [1 'one' NULL] statement ok SET tracing = on,kv,results; SELECT * FROM ab WHERE a > 1 INTERSECT SELECT * FROM ab WHERE b > 1; SET tracing = off diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_auto_mode b/pkg/sql/opt/exec/execbuilder/testdata/distsql_auto_mode index af4ec52007dd..06a50985a215 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_auto_mode +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_auto_mode @@ -99,29 +99,20 @@ true statement ok CREATE TABLE abc (a INT PRIMARY KEY, b INT, c INT, INDEX b (b)) -# TODO(radu): can't execbuild index join yet. -## Index join - don't distribute. -#query B -#SELECT automatic FROM [EXPLAIN (DISTSQL) SELECT * FROM abc WHERE b=1] -#---- -#false -# -## Index join with filter on result - don't distribute. -#query B -#SELECT automatic FROM [EXPLAIN (DISTSQL) SELECT * FROM abc WHERE b=1 AND c%2=0] -#---- -#false -# -## Index join with filter on index scan - distribute. -#query B -#SELECT automatic FROM [EXPLAIN (DISTSQL) SELECT * FROM abc WHERE b=1 AND a%2=0] -#---- -#true - -# OID cast - don't distribute (#22249). -statement error pq: cast to REGCLASS is not supported by distsql -EXPLAIN (DISTSQL) SELECT t1.a FROM abc t1 INNER JOIN abc t2 on t1.a::REGCLASS = t2.a::REGCLASS; - -# Query with OID expression - don't distribute (#24423). -statement error pq: OID expressions are not supported by distsql -EXPLAIN (DISTSQL) SELECT 246::REGTYPE FROM abc +# Index join - don't distribute. +query B +SELECT automatic FROM [EXPLAIN (DISTSQL) SELECT * FROM abc WHERE b=1] +---- +false + +# Index join with filter on result - don't distribute. +query B +SELECT automatic FROM [EXPLAIN (DISTSQL) SELECT * FROM abc WHERE b=1 AND c%2=0] +---- +false + +# Index join with filter on index scan - distribute. +query B +SELECT automatic FROM [EXPLAIN (DISTSQL) SELECT * FROM abc WHERE b=1 AND a%2=0] +---- +true diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace b/pkg/sql/opt/exec/execbuilder/testdata/show_trace index 22391f235044..284d1e3cf91a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace @@ -305,7 +305,7 @@ output row: [9] statement ok SET tracing = on,kv,results; VALUES (1, 2), (1, 1), (1, 2), (2, 1), (2, 1) UNION VALUES (1, 3), (3, 4), (1, 1); SET tracing = off -query T +query T rowsort SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'fetched:%' OR message LIKE 'output row%' ---- @@ -318,7 +318,7 @@ output row: [3 4] statement ok SET tracing = on,kv,results; SELECT * FROM abc EXCEPT SELECT * FROM abc WHERE b > 'p'; SET tracing = off -query T +query T rowsort SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'fetched:%' OR message LIKE 'output row%' ---- @@ -326,14 +326,14 @@ fetched: /abc/primary/1/'one' -> NULL fetched: /abc/primary/2/'two' -> NULL fetched: /abc/primary/3/'three' -> NULL fetched: /abc/primary/1/'one' -> NULL -output row: [1 'one' NULL] fetched: /abc/primary/2/'two' -> NULL fetched: /abc/primary/3/'three' -> NULL +output row: [1 'one' NULL] statement ok SET tracing = on,kv,results; SELECT * FROM ab WHERE a > 1 INTERSECT SELECT * FROM ab WHERE b > 1; SET tracing = off -query T +query T rowsort SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'fetched:%' OR message LIKE 'output row%' ---- @@ -345,10 +345,10 @@ fetched: /ab/primary/1/4 -> NULL fetched: /ab/primary/1/5 -> NULL fetched: /ab/primary/2/1 -> NULL fetched: /ab/primary/2/2 -> NULL -output row: [2 2] fetched: /ab/primary/2/6 -> NULL -output row: [2 6] fetched: /ab/primary/3/9 -> NULL +output row: [2 2] +output row: [2 6] output row: [3 9] statement ok diff --git a/pkg/sql/plan_node_to_row_source.go b/pkg/sql/plan_node_to_row_source.go index 875881b6e3cc..969e09798b30 100644 --- a/pkg/sql/plan_node_to_row_source.go +++ b/pkg/sql/plan_node_to_row_source.go @@ -17,23 +17,33 @@ package sql import ( "context" + "sync" + "github.com/cockroachdb/cockroach/pkg/sql/distsqlrun" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/log" ) type planNodeToRowSource struct { + started bool running bool + fastPath bool + node planNode params runParams outputTypes []sqlbase.ColumnType + out distsqlrun.ProcOutputHelper + // run time state machine values - ctx context.Context row sqlbase.EncDatumRow } -func makePlanNodeToRowSource(source planNode, params runParams) (*planNodeToRowSource, error) { +func makePlanNodeToRowSource( + source planNode, params runParams, fastPath bool, +) (*planNodeToRowSource, error) { nodeColumns := planColumns(source) types := make([]sqlbase.ColumnType, len(nodeColumns)) @@ -52,46 +62,120 @@ func makePlanNodeToRowSource(source planNode, params runParams) (*planNodeToRowS outputTypes: types, row: row, running: true, + fastPath: fastPath, }, nil } -var _ distsqlrun.RowSource = &planNodeToRowSource{} +var _ distsqlrun.LocalProcessor = &planNodeToRowSource{} + +// InitWithOutput implements the LocalProcessor interface. +func (p *planNodeToRowSource) InitWithOutput( + post *distsqlrun.PostProcessSpec, output distsqlrun.RowReceiver, +) error { + return p.out.Init(post, p.outputTypes, p.params.EvalContext(), output) +} func (p *planNodeToRowSource) OutputTypes() []sqlbase.ColumnType { - return p.outputTypes + return p.out.OutputTypes() } func (p *planNodeToRowSource) Start(ctx context.Context) context.Context { - p.ctx = ctx + p.params.ctx = ctx return ctx } func (p *planNodeToRowSource) internalClose() { if p.running { - p.node.Close(p.ctx) + p.node.Close(p.params.ctx) p.running = false } } +func (p *planNodeToRowSource) startExec(_ runParams) error { + // If we're getting startExec'd, it means we're running in local mode - so we + // mark ourselves already started, since local mode will have taken care of + // starting the child nodes of this node. + p.started = true + return nil +} + func (p *planNodeToRowSource) Next() (sqlbase.EncDatumRow, *distsqlrun.ProducerMetadata) { if !p.running { return nil, nil } + if !p.started { + p.started = true + // This starts all of the nodes below this node. + if err := startExec(p.params, p.node); err != nil { + p.internalClose() + return nil, &distsqlrun.ProducerMetadata{Err: err} + } - valid, err := p.node.Next(p.params) - if err != nil { - p.internalClose() - return nil, &distsqlrun.ProducerMetadata{Err: err} - } - if !valid { - p.internalClose() - return nil, nil + if p.fastPath { + var count int + // If our node is a "fast path node", it means that we're set up to just + // return a row count. So trigger the fast path and return the row count as + // a row with a single column. + if fastPath, ok := p.node.(planNodeFastPath); ok { + count, ok = fastPath.FastPathResults() + if !ok { + p.internalClose() + return nil, nil + } + if p.params.extendedEvalCtx.Tracing.Enabled() { + log.VEvent(p.params.ctx, 2, "fast path completed") + } + } else { + // If we have no fast path to trigger, fall back to counting the rows + // by Nexting our source until exhaustion. + next, err := p.node.Next(p.params) + for ; next; next, err = p.node.Next(p.params) { + // If we're tracking memory, clear the previous row's memory account. + if p.params.extendedEvalCtx.ActiveMemAcc != nil { + p.params.extendedEvalCtx.ActiveMemAcc.Clear(p.params.ctx) + } + count++ + } + if err != nil { + return nil, &distsqlrun.ProducerMetadata{Err: err} + } + } + p.internalClose() + // Return the row count the only way we can: as a single-column row with + // the count inside. + return sqlbase.EncDatumRow{sqlbase.EncDatum{Datum: tree.NewDInt(tree.DInt(count))}}, nil + } } - for i, datum := range p.node.Values() { - p.row[i] = sqlbase.DatumToEncDatum(p.outputTypes[i], datum) + for p.running { + valid, err := p.node.Next(p.params) + if err != nil { + p.internalClose() + return nil, &distsqlrun.ProducerMetadata{Err: err} + } + if !valid { + p.internalClose() + return nil, nil + } + + for i, datum := range p.node.Values() { + p.row[i] = sqlbase.DatumToEncDatum(p.outputTypes[i], datum) + } + // ProcessRow here is required to deal with projections, which won't be + // pushed into the wrapped plan. + outRow, ok, err := p.out.ProcessRow(p.params.ctx, p.row) + if err != nil { + p.internalClose() + return nil, &distsqlrun.ProducerMetadata{Err: err} + } + if !ok { + p.internalClose() + } + if outRow != nil { + return outRow, nil + } } - return p.row, nil + return nil, nil } func (p *planNodeToRowSource) ConsumerDone() { @@ -101,3 +185,15 @@ func (p *planNodeToRowSource) ConsumerDone() { func (p *planNodeToRowSource) ConsumerClosed() { p.internalClose() } + +func (p *planNodeToRowSource) Run(ctx context.Context, wg *sync.WaitGroup) { + if p.out.Output() == nil { + panic("processor output not initialized for emitting rows") + } + ctx = p.Start(ctx) + distsqlrun.Run(ctx, p, p.out.Output()) + p.internalClose() + if wg != nil { + wg.Done() + } +} diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index ca41bbd9d25d..3b2d884b1e49 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -506,16 +506,10 @@ func (p *planner) SessionData() *sessiondata.SessionData { } // prepareForDistSQLSupportCheck prepares p.curPlan.plan for a distSQL support -// check and does additional verification of the planner state. It returns -// whether the caller should go ahead and check for plan support through -// shouldUseDistSQL. If returnError is set and false is returned, an error -// explaining the failure will be returned. -func (p *planner) prepareForDistSQLSupportCheck( - ctx context.Context, returnError bool, -) (bool, error) { +// check and does additional verification of the planner state. +func (p *planner) prepareForDistSQLSupportCheck() { // Trigger limit propagation. p.setUnlimited(p.curPlan.plan) - return true, nil } // optionallyUseOptimizer will attempt to make an optimizer plan based on the diff --git a/pkg/sql/sessiondata/search_path.go b/pkg/sql/sessiondata/search_path.go index 5a15df7e80c3..6addf7feb70d 100644 --- a/pkg/sql/sessiondata/search_path.go +++ b/pkg/sql/sessiondata/search_path.go @@ -88,6 +88,12 @@ func (s SearchPath) IterWithoutImplicitPGCatalog() func() (next string, ok bool) } } +// GetPathArray returns the underlying path array of this SearchPath. The +// resultant slice is not to be modified. +func (s SearchPath) GetPathArray() []string { + return s.paths +} + func (s SearchPath) String() string { return strings.Join(s.paths, ", ") } diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index 085a126e7288..5057e3f169c1 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -2278,7 +2278,7 @@ func DatumTypeToColumnSemanticType(ptyp types.T) (ColumnType_SemanticType, error return ColumnType_UUID, nil case types.INet: return ColumnType_INET, nil - case types.Oid: + case types.Oid, types.RegClass, types.RegNamespace, types.RegProc, types.RegType, types.RegProcedure: return ColumnType_OID, nil case types.Unknown: return ColumnType_NULL, nil diff --git a/pkg/sql/sqlbase/table.go b/pkg/sql/sqlbase/table.go index dc8b5b1c96ed..5adb859f99e4 100644 --- a/pkg/sql/sqlbase/table.go +++ b/pkg/sql/sqlbase/table.go @@ -1691,6 +1691,16 @@ func decodeUntaggedDatum(a *DatumAlloc, t types.T, buf []byte) (tree.Datum, []by return a.NewDOid(tree.MakeDOid(tree.DInt(data))), b, err default: switch typ := t.(type) { + case types.TOidWrapper: + wrapped := typ.T + d, rest, err := decodeUntaggedDatum(a, wrapped, buf) + if err != nil { + return d, rest, err + } + return &tree.DOidWrapper{ + Wrapped: d, + Oid: typ.Oid(), + }, rest, nil case types.TCollatedString: b, data, err := encoding.DecodeUntaggedBytesValue(buf) return tree.NewDCollatedString(string(data), typ.Locale, &a.env), b, err diff --git a/pkg/sql/testutils.go b/pkg/sql/testutils.go index 02ddcc0638af..4a8b04cdfaaf 100644 --- a/pkg/sql/testutils.go +++ b/pkg/sql/testutils.go @@ -119,7 +119,7 @@ func (dsp *DistSQLPlanner) Exec(ctx context.Context, localPlanner interface{}, s p.ExtendedEvalContext().Tracing, ) dsp.PlanAndRun( - ctx, p.Txn(), p, p.curPlan.plan, recv, p.ExtendedEvalContext(), + ctx, p, recv, true, /* distribute */ ) return nil } diff --git a/pkg/sql/txn_restart_test.go b/pkg/sql/txn_restart_test.go index 4ebeb4ab382a..766ea78346bc 100644 --- a/pkg/sql/txn_restart_test.go +++ b/pkg/sql/txn_restart_test.go @@ -1749,14 +1749,16 @@ func TestTxnAutoRetriesDisabledAfterResultsHaveBeenSentToClient(t *testing.T) { // We'll run a statement that produces enough results to overflow the // buffers and start streaming results to the client before the retriable - // error is injected. We do this through a single statement (a UNION) - // instead of two separate statements in order to support the autoCommit - // test which needs a single statement. + // error is injected. We do this by running a generate series that blows + // up at the very end, with a CASE statement. sql := fmt.Sprintf(` %s - SELECT generate_series(1, 10000) - UNION ALL - SELECT crdb_internal.force_retry('1s'); + SELECT + CASE x + WHEN 10000 THEN crdb_internal.force_retry('1s') + ELSE x + END + FROM generate_series(1, 10000) AS t(x); %s`, prefix, suffix) _, err := sqlDB.Exec(sql) diff --git a/pkg/testutils/sqlutils/sql_runner_test.go b/pkg/testutils/sqlutils/sql_runner_test.go index d04c3243b294..e7d549854cc7 100644 --- a/pkg/testutils/sqlutils/sql_runner_test.go +++ b/pkg/testutils/sqlutils/sql_runner_test.go @@ -35,7 +35,7 @@ func TestRowsToStrMatrixError(t *testing.T) { // We'll run a query that only fails after returning some rows, so that the // error is discovered by RowsToStrMatrix below. rows, err := db.Query( - "select generate_series(1,10) union all select crdb_internal.force_error('00000', 'testing error')") + "select case x when 5 then crdb_internal.force_error('00000', 'testing error') else x end from generate_series(1,5) as v(x);") if err != nil { t.Fatal(err) }