From 8ecc6818f8cf9a281ed37a3d9d8097100dcdf027 Mon Sep 17 00:00:00 2001 From: yuzefovich Date: Thu, 5 Jul 2018 16:07:38 -0400 Subject: [PATCH] distsql: support window functions in distsql Adds support of window functions in DistSQL. A stage of windower processors is added for a particular PARTITION BY scheme, all window functions with the same partitioning are processed by that stage. We also cache sorted partitions if more than one window function has the same ORDER BY clause. A doc describing the details of the implementation is out. Note: memory accounting is not done, I plan on adding that after this PR. Resolves: #24425. Release note: None --- pkg/sql/distsql_physical_planner.go | 193 ++ pkg/sql/distsql_plan_window.go | 371 +++ pkg/sql/distsqlplan/window_frame.go | 111 + pkg/sql/distsqlrun/api.pb.go | 1 + pkg/sql/distsqlrun/flow_diagram.go | 31 + pkg/sql/distsqlrun/processors.go | 6 + pkg/sql/distsqlrun/processors.pb.go | 2161 ++++++++++++++--- pkg/sql/distsqlrun/processors.proto | 95 + pkg/sql/distsqlrun/windower.go | 682 ++++++ pkg/sql/logictest/testdata/logic_test/window | 16 +- .../testdata/planner_test/distsql_window | 64 + .../logictest/testdata/planner_test/window | 4 +- pkg/sql/sem/builtins/window_builtins.go | 10 + 13 files changed, 3378 insertions(+), 367 deletions(-) create mode 100644 pkg/sql/distsql_plan_window.go create mode 100644 pkg/sql/distsqlplan/window_frame.go create mode 100644 pkg/sql/distsqlrun/windower.go create mode 100644 pkg/sql/logictest/testdata/planner_test/distsql_window diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index a3245e7695fd..4f9bf1cd9468 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -431,6 +431,9 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio case *zeroNode: return canDistribute, nil + case *windowNode: + return dsp.checkSupportForNode(n.plan) + default: return 0, newQueryNotSupportedErrorf("unsupported node %T", node) } @@ -2271,10 +2274,17 @@ func (dsp *DistSQLPlanner) createPlanForNode( case *zeroNode: plan, err = dsp.createPlanForZero(planCtx, n) + case *windowNode: + plan, err = dsp.createPlanForWindow(planCtx, n) + default: panic(fmt.Sprintf("unsupported node type %T", n)) } + if err != nil { + return plan, err + } + if dsp.shouldPlanTestMetadata() { if err := plan.CheckLastStagePost(); err != nil { log.Fatal(planCtx.ctx, err) @@ -2813,6 +2823,189 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( return p, nil } +// createPlanForWindow creates a physical plan for computing window functions. +// We add a new stage of windower processors for each different partitioning +// scheme found in the query's window functions. +func (dsp *DistSQLPlanner) createPlanForWindow( + planCtx *planningCtx, n *windowNode, +) (physicalPlan, error) { + plan, err := dsp.createPlanForNode(planCtx, n.plan) + if err != nil { + return physicalPlan{}, err + } + + numWindowFuncProcessed := 0 + windowPlanState := createWindowPlanState(n, planCtx.EvalContext(), &plan) + // Each iteration of this loop adds a new stage of windowers. The steps taken: + // 1. find a set of unprocessed window functions that have the same PARTITION BY + // clause. All of these will be computed using the single stage of windowers. + // 2. a) populate output types of the current stage of windowers. All columns + // that are arguments to a window function in the set will be replaced by + // a single output column; all columns that are not arguments to any + // window function in the set are simply passed through. + // b) create specs for all window functions in the set. + // 3. windowers' input schema is probably different from the output one, so we + // are adjusting indices of the columns accordingly. + // 4. decide whether to put windowers on a single or on multiple nodes. + // a) if we're putting windowers on multiple nodes, we'll put them onto + // every node that participated in the previous stage. We leverage hash + // routers to partition the data based on PARTITION BY clause of window + // functions in the set. + for numWindowFuncProcessed < len(n.funcs) { + samePartitionFuncs, partitionIdxs := windowPlanState.findUnprocessedWindowFnsWithSamePartition() + numWindowFuncProcessed += len(samePartitionFuncs) + for _, f := range samePartitionFuncs { + // The output of window function f will be put at f.argIdxStart'th column. + // If it is not possible - for example, when two window functions have the + // same argIdxStart, the function that appears later in n.funcs will put + // its result at argIdxStart + 1 - later call to adjustColumnIndices will + // take care of it. + windowPlanState.infos[f.funcIdx].outputColIdx = f.argIdxStart + } + + windowerSpec := distsqlrun.WindowerSpec{ + PartitionBy: partitionIdxs, + WindowFns: make([]distsqlrun.WindowerSpec_WindowFn, len(samePartitionFuncs)), + } + + // Populating output types of this stage since windowers will likely change + // them from its input types. Let's go through an example query + // `SELECT lead(a, b, c) OVER (), avg(d) OVER, rank() OVER() FROM t`. + // 1. 'lead' takes in three arguments which are in columns [0; 3). After we + // compute it, we'll put the result in column 0 (in-place of 'a') and + // not output columns 1 and 2. + // 2. 'avg' takes in a single argument in column 3. We compute it and put + // the result in column 1 since previous window functions have produced + // only a single column so far. + // 3. 'row_number' takes no arguments. We compute it and put the result + // in column 2. + newResultTypes := make([]sqlbase.ColumnType, 0, len(plan.ResultTypes)) + // inputColIdx is the index of the column that should be processed next + // among input columns to the current stage. + inputColIdx := 0 + for windowFnSpecIdx, windowFn := range samePartitionFuncs { + // All window functions are sorted by their argIdxStart, so we copy all + // columns up to windowFn.argIdxStart (all window functions in + // samePartitionFuncs after windowFn have their arguments in later + // columns). + newResultTypes = append(newResultTypes, plan.ResultTypes[inputColIdx:windowFn.argIdxStart]...) + + windowFnSpec, outputType, err := windowPlanState.createWindowFnSpec(windowFn) + if err != nil { + return physicalPlan{}, err + } + + // Windower processor does not pass through ("consumes") all arguments of + // windowFn and puts the result of computation at windowFn.argIdxStart. + newResultTypes = append(newResultTypes, outputType) + inputColIdx = windowFn.argIdxStart + windowFn.argCount + + windowerSpec.WindowFns[windowFnSpecIdx] = windowFnSpec + } + // We keep all the columns after the last window function + // that is being processed in the current stage. + newResultTypes = append(newResultTypes, plan.ResultTypes[inputColIdx:]...) + + // We need to adjust indices since windower's output schema might not be + // equal to its input schema, and we need to maintain outputColIdx of + // processed window functions and argIdxStart of unprocessed window + // functions to point at the correct columns. + windowPlanState.adjustColumnIndices(samePartitionFuncs) + + // Check if the previous stage is all on one node. + prevStageNode := plan.Processors[plan.ResultRouters[0]].Node + for i := 1; i < len(plan.ResultRouters); i++ { + if n := plan.Processors[plan.ResultRouters[i]].Node; n != prevStageNode { + prevStageNode = 0 + break + } + } + + if len(partitionIdxs) == 0 || len(plan.ResultRouters) == 1 { + // No PARTITION BY or we have a single stream. Use a single windower. + // If the previous stage was all on a single node, put the windower + // there. Otherwise, bring the results back on this node. + node := dsp.nodeDesc.NodeID + if prevStageNode != 0 { + node = prevStageNode + } + plan.AddSingleGroupStage( + node, + distsqlrun.ProcessorCoreUnion{Windower: &windowerSpec}, + distsqlrun.PostProcessSpec{}, + newResultTypes, + ) + } else { + // Set up the output routers from the previous stage. + // We use hash routers with hashing on the columns + // from PARTITION BY clause of window functions + // we're processing in the current stage. + for _, resultProc := range plan.ResultRouters { + plan.Processors[resultProc].Spec.Output[0] = distsqlrun.OutputRouterSpec{ + Type: distsqlrun.OutputRouterSpec_BY_HASH, + HashColumns: partitionIdxs, + } + } + stageID := plan.NewStageID() + + // Get all nodes from the previous stage. + nodes := findJoinProcessorNodes(plan.ResultRouters, nil /* rightRouter */, plan.Processors, false /* includeRight */) + if len(nodes) != len(plan.ResultRouters) { + panic("unexpected number of nodes") + } + + // We put a windower on each node and we connect it + // with all hash routers from the previous stage in + // a such way that each node has its designated + // SourceRouterSlot - namely, position in which + // a node appear in nodes. + prevStageRouters := plan.ResultRouters + plan.ResultRouters = make([]distsqlplan.ProcessorIdx, 0, len(nodes)) + for bucket, nodeID := range nodes { + proc := distsqlplan.Processor{ + Node: nodeID, + Spec: distsqlrun.ProcessorSpec{ + Input: []distsqlrun.InputSyncSpec{{ + Type: distsqlrun.InputSyncSpec_UNORDERED, + ColumnTypes: plan.ResultTypes, + }}, + Core: distsqlrun.ProcessorCoreUnion{Windower: &windowerSpec}, + Post: distsqlrun.PostProcessSpec{}, + Output: []distsqlrun.OutputRouterSpec{{ + Type: distsqlrun.OutputRouterSpec_PASS_THROUGH, + }}, + StageID: stageID, + }, + } + pIdx := plan.AddProcessor(proc) + + for router := 0; router < len(nodes); router++ { + plan.Streams = append(plan.Streams, distsqlplan.Stream{ + SourceProcessor: prevStageRouters[router], + SourceRouterSlot: bucket, + DestProcessor: pIdx, + DestInput: 0, + }) + } + plan.ResultRouters = append(plan.ResultRouters, pIdx) + } + + plan.ResultTypes = newResultTypes + } + } + + // We probably added/removed columns throughout all the stages of windowers, + // so we need to update planToStreamColMap. + plan.planToStreamColMap = identityMap(plan.planToStreamColMap, len(plan.ResultTypes)) + + // After all window functions are computed, we might need to add rendering. + if err := windowPlanState.addRenderingIfNecessary(); err != nil { + return physicalPlan{}, err + } + + return plan, nil +} + func (dsp *DistSQLPlanner) newPlanningCtx( ctx context.Context, evalCtx *extendedEvalContext, txn *client.Txn, ) planningCtx { diff --git a/pkg/sql/distsql_plan_window.go b/pkg/sql/distsql_plan_window.go new file mode 100644 index 000000000000..564018de5d7a --- /dev/null +++ b/pkg/sql/distsql_plan_window.go @@ -0,0 +1,371 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package sql + +import ( + "strings" + + "github.com/cockroachdb/cockroach/pkg/sql/distsqlplan" + "github.com/cockroachdb/cockroach/pkg/sql/distsqlrun" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/pkg/errors" +) + +type windowPlanState struct { + // infos contains information about windowFuncHolders in the same order as + // they appear in n.funcs. + infos []*windowFuncInfo + n *windowNode + evalCtx *tree.EvalContext + plan *physicalPlan +} + +func createWindowPlanState( + n *windowNode, evalCtx *tree.EvalContext, plan *physicalPlan, +) *windowPlanState { + infos := make([]*windowFuncInfo, 0, len(n.funcs)) + for _, holder := range n.funcs { + infos = append(infos, &windowFuncInfo{holder: holder}) + } + return &windowPlanState{ + infos: infos, + n: n, + evalCtx: evalCtx, + plan: plan, + } +} + +// windowFuncInfo contains runtime information about a window function. +// Note: we use holder.funcIdx as the index of a particular windowFuncInfo +// among windowPlanState.infos. +type windowFuncInfo struct { + holder *windowFuncHolder + // isProcessed indicates whether holder has already been processed. It is set + // to true when holder is included in the set of window functions to be + // processed by findUnprocessedWindowFnsWithSamePartition. + isProcessed bool + // outputColIdx indicates the index of the column that contains the result of + // "computing" holder. It is first set right after holder is included in the + // set of window functions to be processed and is later might be adjusted + // with each call of adjustColumnIndices. + outputColIdx int +} + +// samePartition returns whether f and other have the same PARTITION BY clause. +func (f *windowFuncHolder) samePartition(other *windowFuncHolder) bool { + if len(f.partitionIdxs) != len(other.partitionIdxs) { + return false + } + for i, p := range f.partitionIdxs { + if p != other.partitionIdxs[i] { + return false + } + } + return true +} + +// findUnprocessedWindowFnsWithSamePartition finds a set of unprocessed window +// functions that use the same partitioning and updates their isProcessed flag +// accordingly. It returns the set of unprocessed window functions and indices +// of the columns in their PARTITION BY clause. +func (s *windowPlanState) findUnprocessedWindowFnsWithSamePartition() (samePartitionFuncs []*windowFuncHolder, partitionIdxs []uint32) { + var windowFnToProcess *windowFuncHolder + for _, windowFn := range s.infos { + if !windowFn.isProcessed { + windowFnToProcess = windowFn.holder + break + } + } + if windowFnToProcess == nil { + panic("unexpected: no unprocessed window function") + } + + partitionIdxs = make([]uint32, len(windowFnToProcess.partitionIdxs)) + for i, idx := range windowFnToProcess.partitionIdxs { + partitionIdxs[i] = uint32(idx) + } + + samePartitionFuncs = make([]*windowFuncHolder, 0, len(s.infos)-windowFnToProcess.funcIdx) + samePartitionFuncs = append(samePartitionFuncs, windowFnToProcess) + s.infos[windowFnToProcess.funcIdx].isProcessed = true + for _, windowFn := range s.infos[windowFnToProcess.funcIdx+1:] { + if windowFn.isProcessed { + continue + } + if windowFnToProcess.samePartition(windowFn.holder) { + samePartitionFuncs = append(samePartitionFuncs, windowFn.holder) + windowFn.isProcessed = true + } + } + + return samePartitionFuncs, partitionIdxs +} + +// adjustColumnIndices shifts all the indices due to window functions in +// funcsInProgress that add or remove columns. It maintains: +// 1. argIdxStart of yet unprocessed window functions in s.infos to correctly +// point at the start of their arguments; +// 2. outputColIdx of already processed (including added in the current stage) +// window function to the output column index in the output of the current +// stage of windowers; +// 3. indices of columns referred to in PARTITION BY and ORDER BY clauses of +// unprocessed window functions. +func (s *windowPlanState) adjustColumnIndices(funcsInProgress []*windowFuncHolder) { + for _, funcInProgress := range funcsInProgress { + if funcInProgress.argCount != 1 { + argShift := 1 - funcInProgress.argCount + // All window functions after funcInProgress need to be adjusted since + // funcInProgress adds/removes columns. Important assumption: all window + // functions are initially sorted by their argIdxStart, and these shifts + // keep that order intact. + // + // Some edge cases for two window functions f1 and f2 (f1 appears + // before f2 in s.infos) with f1.argIdxStart == f2.argIdxStart: + // + // 1. both f1 and f2 are in funcsInProgress: + // a) f1.argCount == 0 + // - handled correctly because we'll insert a new column at + // f1.argIdxStart, and the result of f2 will be appended right + // after f1, i.e. f2.argIdxStart == f1.argIdxStart + 1; + // b) f1.argCount > 0 + // - not possible because f2.argIdxStart would not have been + // equal to f1.argIdxStart. + // + // 2. f1 in funcsInProgress, f2 is not: + // a) f2 has been processed already, so we want to maintain + // the pointer to its output column in outputColIdx. f1 + // shifts all columns after f1.argIdxStart by argShift, + // so we need to adjust column index accordingly. + // b) f2 will be processed later, so we want to maintain + // f2.argIdxStart as an index of starting argument to f2. + // + // 3. f1 not in funcsInProgress, f2 is: + // - f2 has no influence on f1 since f1 appears before f2 in s.infos. + for _, f := range s.infos[funcInProgress.funcIdx+1:] { + if f.isProcessed { + f.outputColIdx += argShift + } else { + f.holder.argIdxStart += argShift + } + } + } + } + + // Assumption: all PARTITION BY and ORDER BY related columns come after + // columns-arguments to window functions, so we need to adjust their indices + // accordingly. + partitionOrderColShift := 0 + for _, funcInProgress := range funcsInProgress { + partitionOrderColShift += 1 - funcInProgress.argCount + } + if partitionOrderColShift != 0 { + for _, f := range s.infos { + if !f.isProcessed { + // If f has already been processed, we don't adjust its indices since + // it is not necessary. + for p := range f.holder.partitionIdxs { + f.holder.partitionIdxs[p] += partitionOrderColShift + } + oldColumnOrdering := f.holder.columnOrdering + f.holder.columnOrdering = make(sqlbase.ColumnOrdering, 0, len(oldColumnOrdering)) + for _, o := range oldColumnOrdering { + f.holder.columnOrdering = append(f.holder.columnOrdering, sqlbase.ColumnOrderInfo{ + ColIdx: o.ColIdx + partitionOrderColShift, + Direction: o.Direction, + }) + } + } + } + } +} + +func createWindowerSpecFunc(funcStr string) (distsqlrun.WindowerSpec_Func, error) { + if aggBuiltin, ok := distsqlrun.AggregatorSpec_Func_value[funcStr]; ok { + aggSpec := distsqlrun.AggregatorSpec_Func(aggBuiltin) + return distsqlrun.WindowerSpec_Func{AggregateFunc: &aggSpec}, nil + } else if winBuiltin, ok := distsqlrun.WindowerSpec_WindowFunc_value[funcStr]; ok { + winSpec := distsqlrun.WindowerSpec_WindowFunc(winBuiltin) + return distsqlrun.WindowerSpec_Func{WindowFunc: &winSpec}, nil + } else { + return distsqlrun.WindowerSpec_Func{}, errors.Errorf("unknown aggregate/window function %s", funcStr) + } +} + +func (s *windowPlanState) createWindowFnSpec( + funcInProgress *windowFuncHolder, +) (distsqlrun.WindowerSpec_WindowFn, sqlbase.ColumnType, error) { + if funcInProgress.argIdxStart+funcInProgress.argCount > len(s.plan.ResultTypes) { + return distsqlrun.WindowerSpec_WindowFn{}, sqlbase.ColumnType{}, errors.Errorf("ColIdx out of range (%d)", funcInProgress.argIdxStart+funcInProgress.argCount-1) + } + // Figure out which built-in to compute. + funcStr := strings.ToUpper(funcInProgress.expr.Func.String()) + funcSpec, err := createWindowerSpecFunc(funcStr) + if err != nil { + return distsqlrun.WindowerSpec_WindowFn{}, sqlbase.ColumnType{}, err + } + argTypes := s.plan.ResultTypes[funcInProgress.argIdxStart : funcInProgress.argIdxStart+funcInProgress.argCount] + _, outputType, err := distsqlrun.GetWindowFunctionInfo(funcSpec, argTypes...) + if err != nil { + return distsqlrun.WindowerSpec_WindowFn{}, outputType, err + } + // Populating column ordering from ORDER BY clause of funcInProgress. + ordCols := make([]distsqlrun.Ordering_Column, 0, len(funcInProgress.columnOrdering)) + for _, column := range funcInProgress.columnOrdering { + ordCols = append(ordCols, distsqlrun.Ordering_Column{ + ColIdx: uint32(column.ColIdx), + // We need this -1 because encoding.Direction has extra value "_" + // as zeroth "entry" which its proto equivalent doesn't have. + Direction: distsqlrun.Ordering_Column_Direction(column.Direction - 1), + }) + } + funcInProgressSpec := distsqlrun.WindowerSpec_WindowFn{ + Func: funcSpec, + ArgIdxStart: uint32(funcInProgress.argIdxStart), + ArgCount: uint32(funcInProgress.argCount), + Ordering: distsqlrun.Ordering{Columns: ordCols}, + } + if s.n.run.windowFrames[funcInProgress.funcIdx] != nil { + // funcInProgress has a custom window frame. + frameSpec, err := distsqlplan.ConvertToSpec(s.n.run.windowFrames[funcInProgress.funcIdx], s.evalCtx) + if err != nil { + return distsqlrun.WindowerSpec_WindowFn{}, outputType, err + } + funcInProgressSpec.Frame = &frameSpec + } + return funcInProgressSpec, outputType, nil +} + +// addRenderingIfNecessary checks whether any of the window functions' outputs +// are used in another expression and, if they are, adds rendering to the plan. +func (s *windowPlanState) addRenderingIfNecessary() error { + // numWindowFuncsAsIs is the number of window functions output of which is + // used directly (i.e. simply as an output column). Note: the same window + // function might appear multiple times in the query, but its every + // occurrence is replaced by a different windowFuncHolder. For example, on + // query like 'SELECT avg(a) OVER (), avg(a) OVER () + 1 FROM t', only the + // first window function is used "as is." + numWindowFuncsAsIs := 0 + for _, render := range s.n.windowRender { + if _, ok := render.(*windowFuncHolder); ok { + numWindowFuncsAsIs++ + } + } + if numWindowFuncsAsIs == len(s.infos) { + // All window functions' outputs are used directly, so no rendering to do. + return nil + } + + // windowNode contains render expressions that might contain: + // 1) IndexedVars that refer to columns by their indices in the full table, + // 2) IndexedVars that replaced regular aggregates that are above + // "windowing level." + // The mapping of both types IndexedVars is stored in s.n.colAndAggContainer. + // We need to make columnsMap that maps index of an indexedVar to the column + // in the output of windower processor. + + // maxColumnIdx is the largest column index referenced by any IndexedVar in + // renders of windowNode. + maxColumnIdx := -1 + for col := range s.n.colAndAggContainer.idxMap { + if col > maxColumnIdx { + maxColumnIdx = col + } + } + + // We initialize columnsMap with -1's. + columnsMap := makePlanToStreamColMap(maxColumnIdx + 1) + + // colShift refers to the number of columns added/removed because of window + // functions that take number of arguments other than one. IndexedVars from + // the container point to columns after window functions-related columns, so + // we need to shift all indices by colShift. + colShift := 0 + for _, windowFn := range s.infos { + colShift += 1 - windowFn.holder.argCount + } + for col, idx := range s.n.colAndAggContainer.idxMap { + columnsMap[col] = idx + colShift + } + + renderExprs := make([]tree.TypedExpr, len(s.n.windowRender)) + visitor := replaceWindowFuncsVisitor{ + infos: s.infos, + columnsMap: columnsMap, + } + + renderTypes := make([]sqlbase.ColumnType, 0, len(s.n.windowRender)) + for i, render := range s.n.windowRender { + if render != nil { + // render contains at least one reference to windowFuncHolder, so we need + // to walk over the render and replace all windowFuncHolders and (if found) + // IndexedVars using columnsMap and outputColIdx of windowFuncInfos. + renderExprs[i] = visitor.replace(render) + } else { + // render is nil meaning that a column is being passed through. + renderExprs[i] = tree.NewTypedOrdinalReference(visitor.colIdx, s.plan.ResultTypes[visitor.colIdx].ToDatumType()) + visitor.colIdx++ + } + outputType, err := sqlbase.DatumTypeToColumnType(renderExprs[i].ResolvedType()) + if err != nil { + return err + } + renderTypes = append(renderTypes, outputType) + } + if err := s.plan.AddRendering(renderExprs, s.evalCtx, s.plan.planToStreamColMap, renderTypes); err != nil { + return err + } + s.plan.planToStreamColMap = identityMap(s.plan.planToStreamColMap, len(renderTypes)) + return nil +} + +// replaceWindowFuncsVisitor is used to populate render expressions containing +// the results of window functions. It recurses into all expressions except for +// windowFuncHolders (which are replaced by the indices to the corresponding +// output columns) and IndexedVars (which are replaced using columnsMap). +type replaceWindowFuncsVisitor struct { + infos []*windowFuncInfo + columnsMap []int + // colIdx is the index of the current column in the output of last stage of + // windowers. It is necessary to pass through correct columns that are not + // involved in expressions that contain window functions. + colIdx int +} + +var _ tree.Visitor = &replaceWindowFuncsVisitor{} + +// VisitPre satisfies the Visitor interface. +func (v *replaceWindowFuncsVisitor) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) { + switch t := expr.(type) { + case *windowFuncHolder: + v.colIdx++ + return false, tree.NewTypedOrdinalReference(v.infos[t.funcIdx].outputColIdx, t.ResolvedType()) + case *tree.IndexedVar: + // We don't need to increment colIdx because all IndexedVar-related columns + // are the very end. + return false, tree.NewTypedOrdinalReference(v.columnsMap[t.Idx], t.ResolvedType()) + } + return true, expr +} + +// VisitPost satisfies the Visitor interface. +func (v *replaceWindowFuncsVisitor) VisitPost(expr tree.Expr) tree.Expr { + return expr +} + +func (v *replaceWindowFuncsVisitor) replace(typedExpr tree.TypedExpr) tree.TypedExpr { + expr, _ := tree.WalkExpr(v, typedExpr) + return expr.(tree.TypedExpr) +} diff --git a/pkg/sql/distsqlplan/window_frame.go b/pkg/sql/distsqlplan/window_frame.go new file mode 100644 index 000000000000..4eb7f1896ab8 --- /dev/null +++ b/pkg/sql/distsqlplan/window_frame.go @@ -0,0 +1,111 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package distsqlplan + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/distsqlrun" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/pkg/errors" +) + +func mapToSpecMode(m tree.WindowFrameMode) distsqlrun.WindowerSpec_Frame_Mode { + switch m { + case tree.RANGE: + return distsqlrun.WindowerSpec_Frame_RANGE + case tree.ROWS: + return distsqlrun.WindowerSpec_Frame_ROWS + default: + panic("unexpected WindowFrameMode") + } +} + +func mapToSpecBoundType(bt tree.WindowFrameBoundType) distsqlrun.WindowerSpec_Frame_BoundType { + switch bt { + case tree.UnboundedPreceding: + return distsqlrun.WindowerSpec_Frame_UNBOUNDED_PRECEDING + case tree.ValuePreceding: + return distsqlrun.WindowerSpec_Frame_OFFSET_PRECEDING + case tree.CurrentRow: + return distsqlrun.WindowerSpec_Frame_CURRENT_ROW + case tree.ValueFollowing: + return distsqlrun.WindowerSpec_Frame_OFFSET_FOLLOWING + case tree.UnboundedFollowing: + return distsqlrun.WindowerSpec_Frame_UNBOUNDED_FOLLOWING + default: + panic("unexpected WindowFrameBoundType") + } +} + +// If offset exprs are present, we evaluate them and save the encoded results +// in the spec. +func convertToSpecBounds( + b tree.WindowFrameBounds, evalCtx *tree.EvalContext, +) (distsqlrun.WindowerSpec_Frame_Bounds, error) { + bounds := distsqlrun.WindowerSpec_Frame_Bounds{} + if b.StartBound == nil { + return bounds, errors.Errorf("unexpected: Start Bound is nil") + } + bounds.Start = distsqlrun.WindowerSpec_Frame_Bound{ + BoundType: mapToSpecBoundType(b.StartBound.BoundType), + } + if b.StartBound.OffsetExpr != nil { + typedStartOffset := b.StartBound.OffsetExpr.(tree.TypedExpr) + dStartOffset, err := typedStartOffset.Eval(evalCtx) + if err != nil { + return bounds, err + } + if dStartOffset == tree.DNull { + return bounds, pgerror.NewErrorf(pgerror.CodeNullValueNotAllowedError, "frame starting offset must not be null") + } + startOffset := int(tree.MustBeDInt(dStartOffset)) + if startOffset < 0 { + return bounds, pgerror.NewErrorf(pgerror.CodeInvalidParameterValueError, "frame starting offset must not be negative") + } + bounds.Start.IntOffset = uint32(startOffset) + } + + if b.EndBound != nil { + bounds.End = &distsqlrun.WindowerSpec_Frame_Bound{BoundType: mapToSpecBoundType(b.EndBound.BoundType)} + if b.EndBound.OffsetExpr != nil { + typedEndOffset := b.EndBound.OffsetExpr.(tree.TypedExpr) + dEndOffset, err := typedEndOffset.Eval(evalCtx) + if err != nil { + return bounds, err + } + if dEndOffset == tree.DNull { + return bounds, pgerror.NewErrorf(pgerror.CodeNullValueNotAllowedError, "frame ending offset must not be null") + } + endOffset := int(tree.MustBeDInt(dEndOffset)) + if endOffset < 0 { + return bounds, pgerror.NewErrorf(pgerror.CodeInvalidParameterValueError, "frame ending offset must not be negative") + } + bounds.End.IntOffset = uint32(endOffset) + } + } + + return bounds, nil +} + +// ConvertToSpec produces spec based on WindowFrame. It also evaluates offset +// expressions if present in the frame. +func ConvertToSpec( + f *tree.WindowFrame, evalCtx *tree.EvalContext, +) (distsqlrun.WindowerSpec_Frame, error) { + frame := distsqlrun.WindowerSpec_Frame{Mode: mapToSpecMode(f.Mode)} + bounds, err := convertToSpecBounds(f.Bounds, evalCtx) + frame.Bounds = bounds + return frame, err +} diff --git a/pkg/sql/distsqlrun/api.pb.go b/pkg/sql/distsqlrun/api.pb.go index 25b6bf8ee722..8e16b018c87a 100644 --- a/pkg/sql/distsqlrun/api.pb.go +++ b/pkg/sql/distsqlrun/api.pb.go @@ -59,6 +59,7 @@ SampleAggregatorSpec InterleavedReaderJoinerSpec ProjectSetSpec + WindowerSpec InputStats TableReaderStats HashJoinerStats diff --git a/pkg/sql/distsqlrun/flow_diagram.go b/pkg/sql/distsqlrun/flow_diagram.go index 8f3d62aa91db..d00edac88e7b 100644 --- a/pkg/sql/distsqlrun/flow_diagram.go +++ b/pkg/sql/distsqlrun/flow_diagram.go @@ -419,6 +419,37 @@ func (s *CSVWriterSpec) summary() (string, []string) { return "CSVWriter", []string{s.Destination} } +// summary implements the diagramCellType interface. +func (w *WindowerSpec) summary() (string, []string) { + details := make([]string, 0, len(w.WindowFns)) + if len(w.PartitionBy) > 0 { + details = append(details, fmt.Sprintf("PARTITION BY: %s", colListStr(w.PartitionBy))) + } + for _, windowFn := range w.WindowFns { + var buf bytes.Buffer + if windowFn.Func.WindowFunc != nil { + buf.WriteString(windowFn.Func.WindowFunc.String()) + } else { + buf.WriteString(windowFn.Func.AggregateFunc.String()) + } + buf.WriteByte('(') + args := make([]uint32, windowFn.ArgCount) + for i := uint32(0); i < windowFn.ArgCount; i++ { + args[i] = i + windowFn.ArgIdxStart + } + buf.WriteString(colListStr(args)) + buf.WriteByte(')') + if len(windowFn.Ordering.Columns) > 0 { + buf.WriteString(" (ORDER BY ") + buf.WriteString(windowFn.Ordering.diagramString()) + buf.WriteByte(')') + } + details = append(details, buf.String()) + } + + return "Windower", details +} + type diagramCell struct { Title string `json:"title"` Details []string `json:"details"` diff --git a/pkg/sql/distsqlrun/processors.go b/pkg/sql/distsqlrun/processors.go index 36ed55fac77e..ee1505d682ee 100644 --- a/pkg/sql/distsqlrun/processors.go +++ b/pkg/sql/distsqlrun/processors.go @@ -1052,6 +1052,12 @@ func newProcessor( } return newProjectSetProcessor(flowCtx, processorID, core.ProjectSet, inputs[0], post, outputs[0]) } + if core.Windower != nil { + if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { + return nil, err + } + return newWindower(flowCtx, processorID, core.Windower, inputs[0], post, outputs[0]) + } return nil, errors.Errorf("unsupported processor core %s", core) } diff --git a/pkg/sql/distsqlrun/processors.pb.go b/pkg/sql/distsqlrun/processors.pb.go index 181db33ac1ad..d176ddc857e1 100644 --- a/pkg/sql/distsqlrun/processors.pb.go +++ b/pkg/sql/distsqlrun/processors.pb.go @@ -238,6 +238,156 @@ func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{17, 0} } +type WindowerSpec_WindowFunc int32 + +const ( + // These mirror window functions from window_builtins.go. + WindowerSpec_ROW_NUMBER WindowerSpec_WindowFunc = 0 + WindowerSpec_RANK WindowerSpec_WindowFunc = 1 + WindowerSpec_DENSE_RANK WindowerSpec_WindowFunc = 2 + WindowerSpec_PERCENT_RANK WindowerSpec_WindowFunc = 3 + WindowerSpec_CUME_DIST WindowerSpec_WindowFunc = 4 + WindowerSpec_NTILE WindowerSpec_WindowFunc = 5 + WindowerSpec_LAG WindowerSpec_WindowFunc = 6 + WindowerSpec_LEAD WindowerSpec_WindowFunc = 7 + WindowerSpec_FIRST_VALUE WindowerSpec_WindowFunc = 8 + WindowerSpec_LAST_VALUE WindowerSpec_WindowFunc = 9 + WindowerSpec_NTH_VALUE WindowerSpec_WindowFunc = 10 +) + +var WindowerSpec_WindowFunc_name = map[int32]string{ + 0: "ROW_NUMBER", + 1: "RANK", + 2: "DENSE_RANK", + 3: "PERCENT_RANK", + 4: "CUME_DIST", + 5: "NTILE", + 6: "LAG", + 7: "LEAD", + 8: "FIRST_VALUE", + 9: "LAST_VALUE", + 10: "NTH_VALUE", +} +var WindowerSpec_WindowFunc_value = map[string]int32{ + "ROW_NUMBER": 0, + "RANK": 1, + "DENSE_RANK": 2, + "PERCENT_RANK": 3, + "CUME_DIST": 4, + "NTILE": 5, + "LAG": 6, + "LEAD": 7, + "FIRST_VALUE": 8, + "LAST_VALUE": 9, + "NTH_VALUE": 10, +} + +func (x WindowerSpec_WindowFunc) Enum() *WindowerSpec_WindowFunc { + p := new(WindowerSpec_WindowFunc) + *p = x + return p +} +func (x WindowerSpec_WindowFunc) String() string { + return proto.EnumName(WindowerSpec_WindowFunc_name, int32(x)) +} +func (x *WindowerSpec_WindowFunc) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(WindowerSpec_WindowFunc_value, data, "WindowerSpec_WindowFunc") + if err != nil { + return err + } + *x = WindowerSpec_WindowFunc(value) + return nil +} +func (WindowerSpec_WindowFunc) EnumDescriptor() ([]byte, []int) { + return fileDescriptorProcessors, []int{28, 0} +} + +// Mode indicates which mode of framing is used. +type WindowerSpec_Frame_Mode int32 + +const ( + // RANGE specifies frame in terms of logical range (e.g. 1 unit cheaper). + WindowerSpec_Frame_RANGE WindowerSpec_Frame_Mode = 0 + // ROWS specifies frame in terms of physical offsets (e.g. 1 row before). + WindowerSpec_Frame_ROWS WindowerSpec_Frame_Mode = 1 +) + +var WindowerSpec_Frame_Mode_name = map[int32]string{ + 0: "RANGE", + 1: "ROWS", +} +var WindowerSpec_Frame_Mode_value = map[string]int32{ + "RANGE": 0, + "ROWS": 1, +} + +func (x WindowerSpec_Frame_Mode) Enum() *WindowerSpec_Frame_Mode { + p := new(WindowerSpec_Frame_Mode) + *p = x + return p +} +func (x WindowerSpec_Frame_Mode) String() string { + return proto.EnumName(WindowerSpec_Frame_Mode_name, int32(x)) +} +func (x *WindowerSpec_Frame_Mode) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(WindowerSpec_Frame_Mode_value, data, "WindowerSpec_Frame_Mode") + if err != nil { + return err + } + *x = WindowerSpec_Frame_Mode(value) + return nil +} +func (WindowerSpec_Frame_Mode) EnumDescriptor() ([]byte, []int) { + return fileDescriptorProcessors, []int{28, 1, 0} +} + +// BoundType indicates which type of boundary is used. +type WindowerSpec_Frame_BoundType int32 + +const ( + WindowerSpec_Frame_UNBOUNDED_PRECEDING WindowerSpec_Frame_BoundType = 0 + WindowerSpec_Frame_UNBOUNDED_FOLLOWING WindowerSpec_Frame_BoundType = 1 + // Offsets are stored within Bound. + WindowerSpec_Frame_OFFSET_PRECEDING WindowerSpec_Frame_BoundType = 2 + WindowerSpec_Frame_OFFSET_FOLLOWING WindowerSpec_Frame_BoundType = 3 + WindowerSpec_Frame_CURRENT_ROW WindowerSpec_Frame_BoundType = 4 +) + +var WindowerSpec_Frame_BoundType_name = map[int32]string{ + 0: "UNBOUNDED_PRECEDING", + 1: "UNBOUNDED_FOLLOWING", + 2: "OFFSET_PRECEDING", + 3: "OFFSET_FOLLOWING", + 4: "CURRENT_ROW", +} +var WindowerSpec_Frame_BoundType_value = map[string]int32{ + "UNBOUNDED_PRECEDING": 0, + "UNBOUNDED_FOLLOWING": 1, + "OFFSET_PRECEDING": 2, + "OFFSET_FOLLOWING": 3, + "CURRENT_ROW": 4, +} + +func (x WindowerSpec_Frame_BoundType) Enum() *WindowerSpec_Frame_BoundType { + p := new(WindowerSpec_Frame_BoundType) + *p = x + return p +} +func (x WindowerSpec_Frame_BoundType) String() string { + return proto.EnumName(WindowerSpec_Frame_BoundType_name, int32(x)) +} +func (x *WindowerSpec_Frame_BoundType) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(WindowerSpec_Frame_BoundType_value, data, "WindowerSpec_Frame_BoundType") + if err != nil { + return err + } + *x = WindowerSpec_Frame_BoundType(value) + return nil +} +func (WindowerSpec_Frame_BoundType) EnumDescriptor() ([]byte, []int) { + return fileDescriptorProcessors, []int{28, 1, 1} +} + // Each processor has the following components: // - one or more input synchronizers; each one merges rows between one or more // input streams; @@ -339,6 +489,7 @@ type ProcessorCoreUnion struct { MetadataTestReceiver *MetadataTestReceiverSpec `protobuf:"bytes,19,opt,name=metadataTestReceiver" json:"metadataTestReceiver,omitempty"` 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"` } func (m *ProcessorCoreUnion) Reset() { *m = ProcessorCoreUnion{} } @@ -1050,6 +1201,103 @@ func (m *ProjectSetSpec) String() string { return proto.CompactTextSt func (*ProjectSetSpec) ProtoMessage() {} func (*ProjectSetSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{27} } +// WindowerSpec is the specification of a processor that performs computations +// of window functions that have the same PARTITION BY clause. For a particular +// windowFn, the processor puts result at windowFn.ArgIdxStart and "consumes" +// all arguments to windowFn (windowFn.ArgCount of them). So if windowFn takes +// no arguments, an extra column is added; if windowFn takes more than one +// argument, (windowFn.ArgCount - 1) columns are removed. +type WindowerSpec struct { + // PartitionBy specifies how to partition rows for all window functions. + PartitionBy []uint32 `protobuf:"varint,1,rep,name=partitionBy" json:"partitionBy,omitempty"` + // WindowFns is the specification of all window functions to be computed. + WindowFns []WindowerSpec_WindowFn `protobuf:"bytes,2,rep,name=windowFns" json:"windowFns"` +} + +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 specifies which function to compute. It can either be built-in +// aggregate or built-in window function. +type WindowerSpec_Func struct { + AggregateFunc *AggregatorSpec_Func `protobuf:"varint,1,opt,name=aggregateFunc,enum=cockroach.sql.distsqlrun.AggregatorSpec_Func" json:"aggregateFunc,omitempty"` + WindowFunc *WindowerSpec_WindowFunc `protobuf:"varint,2,opt,name=windowFunc,enum=cockroach.sql.distsqlrun.WindowerSpec_WindowFunc" json:"windowFunc,omitempty"` +} + +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} } + +// Frame is the specification of a single window frame for a window function. +type WindowerSpec_Frame struct { + Mode WindowerSpec_Frame_Mode `protobuf:"varint,1,opt,name=mode,enum=cockroach.sql.distsqlrun.WindowerSpec_Frame_Mode" json:"mode"` + Bounds WindowerSpec_Frame_Bounds `protobuf:"bytes,2,opt,name=bounds" json:"bounds"` +} + +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} } + +// Bound specifies the type of boundary and the offset (if present). +type WindowerSpec_Frame_Bound struct { + BoundType WindowerSpec_Frame_BoundType `protobuf:"varint,1,opt,name=boundType,enum=cockroach.sql.distsqlrun.WindowerSpec_Frame_BoundType" json:"boundType"` + // For UNBOUNDED_PRECEDING, UNBOUNDED_FOLLOWING, and CURRENT_ROW offset + // is ignored. Integer offset for ROWS mode is stored in int_offset while + // an encoded datum and the type information are stored for RANGE mode + // (the support for the latter will be added soon). + IntOffset uint32 `protobuf:"varint,2,opt,name=int_offset,json=intOffset" json:"int_offset"` + TypedOffset []byte `protobuf:"bytes,3,opt,name=typed_offset,json=typedOffset" json:"typed_offset,omitempty"` + OffsetType DatumInfo `protobuf:"bytes,4,opt,name=offset_type,json=offsetType" json:"offset_type"` +} + +func (m *WindowerSpec_Frame_Bound) Reset() { *m = WindowerSpec_Frame_Bound{} } +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} +} + +// Bounds specifies boundaries of the window frame. +type WindowerSpec_Frame_Bounds struct { + // Start bound must always be present whereas end bound might be omitted. + Start WindowerSpec_Frame_Bound `protobuf:"bytes,1,opt,name=start" json:"start"` + End *WindowerSpec_Frame_Bound `protobuf:"bytes,2,opt,name=end" json:"end,omitempty"` +} + +func (m *WindowerSpec_Frame_Bounds) Reset() { *m = WindowerSpec_Frame_Bounds{} } +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} +} + +// WindowFn is the specification of a single window function. +type WindowerSpec_WindowFn struct { + // Func is which function to compute. + Func WindowerSpec_Func `protobuf:"bytes,1,opt,name=func" json:"func"` + // ArgIdxStart is which index arguments to this window function start at. + ArgIdxStart uint32 `protobuf:"varint,2,opt,name=argIdxStart" json:"argIdxStart"` + // ArgCount indicates how many arguments this window function takes in. + ArgCount uint32 `protobuf:"varint,3,opt,name=argCount" json:"argCount"` + // Ordering specifies in which order rows should be considered by this + // window function. Its contents come from ORDER BY clause of the window + // function. + Ordering Ordering `protobuf:"bytes,4,opt,name=ordering" json:"ordering"` + // Frame specifies over which frame this window function is computed. + Frame *WindowerSpec_Frame `protobuf:"bytes,5,opt,name=frame" json:"frame,omitempty"` +} + +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} +} + func init() { proto.RegisterType((*ProcessorSpec)(nil), "cockroach.sql.distsqlrun.ProcessorSpec") proto.RegisterType((*PostProcessSpec)(nil), "cockroach.sql.distsqlrun.PostProcessSpec") @@ -1082,10 +1330,19 @@ func init() { proto.RegisterType((*InterleavedReaderJoinerSpec)(nil), "cockroach.sql.distsqlrun.InterleavedReaderJoinerSpec") proto.RegisterType((*InterleavedReaderJoinerSpec_Table)(nil), "cockroach.sql.distsqlrun.InterleavedReaderJoinerSpec.Table") proto.RegisterType((*ProjectSetSpec)(nil), "cockroach.sql.distsqlrun.ProjectSetSpec") + proto.RegisterType((*WindowerSpec)(nil), "cockroach.sql.distsqlrun.WindowerSpec") + proto.RegisterType((*WindowerSpec_Func)(nil), "cockroach.sql.distsqlrun.WindowerSpec.Func") + proto.RegisterType((*WindowerSpec_Frame)(nil), "cockroach.sql.distsqlrun.WindowerSpec.Frame") + proto.RegisterType((*WindowerSpec_Frame_Bound)(nil), "cockroach.sql.distsqlrun.WindowerSpec.Frame.Bound") + proto.RegisterType((*WindowerSpec_Frame_Bounds)(nil), "cockroach.sql.distsqlrun.WindowerSpec.Frame.Bounds") + proto.RegisterType((*WindowerSpec_WindowFn)(nil), "cockroach.sql.distsqlrun.WindowerSpec.WindowFn") proto.RegisterEnum("cockroach.sql.distsqlrun.SketchType", SketchType_name, SketchType_value) proto.RegisterEnum("cockroach.sql.distsqlrun.AggregatorSpec_Func", AggregatorSpec_Func_name, AggregatorSpec_Func_value) proto.RegisterEnum("cockroach.sql.distsqlrun.AggregatorSpec_Type", AggregatorSpec_Type_name, AggregatorSpec_Type_value) proto.RegisterEnum("cockroach.sql.distsqlrun.BackfillerSpec_Type", BackfillerSpec_Type_name, BackfillerSpec_Type_value) + proto.RegisterEnum("cockroach.sql.distsqlrun.WindowerSpec_WindowFunc", WindowerSpec_WindowFunc_name, WindowerSpec_WindowFunc_value) + proto.RegisterEnum("cockroach.sql.distsqlrun.WindowerSpec_Frame_Mode", WindowerSpec_Frame_Mode_name, WindowerSpec_Frame_Mode_value) + proto.RegisterEnum("cockroach.sql.distsqlrun.WindowerSpec_Frame_BoundType", WindowerSpec_Frame_BoundType_name, WindowerSpec_Frame_BoundType_value) } func (m *ProcessorSpec) Marshal() (dAtA []byte, err error) { size := m.Size() @@ -1449,6 +1706,18 @@ func (m *ProcessorCoreUnion) MarshalTo(dAtA []byte) (int, error) { } i += n25 } + if m.Windower != nil { + dAtA[i] = 0xba + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Windower.Size())) + n26, err := m.Windower.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n26 + } return i, nil } @@ -1584,11 +1853,11 @@ func (m *TableReaderSpan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Span.Size())) - n26, err := m.Span.MarshalTo(dAtA[i:]) + n27, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n27 return i, nil } @@ -1610,11 +1879,11 @@ func (m *TableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Table.Size())) - n27, err := m.Table.MarshalTo(dAtA[i:]) + n28, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n28 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexIdx)) @@ -1670,47 +1939,47 @@ func (m *JoinReaderSpec) 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)) if len(m.LookupColumns) > 0 { - dAtA30 := make([]byte, len(m.LookupColumns)*10) - var j29 int + dAtA31 := make([]byte, len(m.LookupColumns)*10) + var j30 int for _, num := range m.LookupColumns { for num >= 1<<7 { - dAtA30[j29] = uint8(uint64(num)&0x7f | 0x80) + dAtA31[j30] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j29++ + j30++ } - dAtA30[j29] = uint8(num) - j29++ + dAtA31[j30] = uint8(num) + j30++ } dAtA[i] = 0x1a i++ - i = encodeVarintProcessors(dAtA, i, uint64(j29)) - i += copy(dAtA[i:], dAtA30[:j29]) + i = encodeVarintProcessors(dAtA, i, uint64(j30)) + i += copy(dAtA[i:], dAtA31[:j30]) } dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n31, err := m.OnExpr.MarshalTo(dAtA[i:]) + n32, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n31 + i += n32 dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexFilterExpr.Size())) - n32, err := m.IndexFilterExpr.MarshalTo(dAtA[i:]) + n33, err := m.IndexFilterExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n33 dAtA[i] = 0x30 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -1735,11 +2004,11 @@ func (m *SorterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OutputOrdering.Size())) - n33, err := m.OutputOrdering.MarshalTo(dAtA[i:]) + n34, err := m.OutputOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n34 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OrderingMatchLen)) @@ -1818,30 +2087,30 @@ func (m *ZigzagJoinerSpec) MarshalTo(dAtA []byte) (int, error) { } } if len(m.IndexIds) > 0 { - dAtA35 := make([]byte, len(m.IndexIds)*10) - var j34 int + dAtA36 := make([]byte, len(m.IndexIds)*10) + var j35 int for _, num := range m.IndexIds { for num >= 1<<7 { - dAtA35[j34] = uint8(uint64(num)&0x7f | 0x80) + dAtA36[j35] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j34++ + j35++ } - dAtA35[j34] = uint8(num) - j34++ + dAtA36[j35] = uint8(num) + j35++ } dAtA[i] = 0x1a i++ - i = encodeVarintProcessors(dAtA, i, uint64(j34)) - i += copy(dAtA[i:], dAtA35[:j34]) + i = encodeVarintProcessors(dAtA, i, uint64(j35)) + i += copy(dAtA[i:], dAtA36[:j35]) } dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n36, err := m.OnExpr.MarshalTo(dAtA[i:]) + n37, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n36 + i += n37 dAtA[i] = 0x28 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -1864,21 +2133,21 @@ func (m *Columns) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.Columns) > 0 { - dAtA38 := make([]byte, len(m.Columns)*10) - var j37 int + dAtA39 := make([]byte, len(m.Columns)*10) + var j38 int for _, num := range m.Columns { for num >= 1<<7 { - dAtA38[j37] = uint8(uint64(num)&0x7f | 0x80) + dAtA39[j38] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j37++ + j38++ } - dAtA38[j37] = uint8(num) - j37++ + dAtA39[j38] = uint8(num) + j38++ } dAtA[i] = 0xa i++ - i = encodeVarintProcessors(dAtA, i, uint64(j37)) - i += copy(dAtA[i:], dAtA38[:j37]) + i = encodeVarintProcessors(dAtA, i, uint64(j38)) + i += copy(dAtA[i:], dAtA39[:j38]) } return i, nil } @@ -1901,27 +2170,27 @@ func (m *MergeJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.LeftOrdering.Size())) - n39, err := m.LeftOrdering.MarshalTo(dAtA[i:]) + n40, err := m.LeftOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n40 dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.RightOrdering.Size())) - n40, err := m.RightOrdering.MarshalTo(dAtA[i:]) + n41, err := m.RightOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n40 + i += n41 dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n41, err := m.OnExpr.MarshalTo(dAtA[i:]) + n42, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n42 dAtA[i] = 0x30 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -1952,47 +2221,47 @@ func (m *HashJoinerSpec) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.LeftEqColumns) > 0 { - dAtA43 := make([]byte, len(m.LeftEqColumns)*10) - var j42 int + dAtA44 := make([]byte, len(m.LeftEqColumns)*10) + var j43 int for _, num := range m.LeftEqColumns { for num >= 1<<7 { - dAtA43[j42] = uint8(uint64(num)&0x7f | 0x80) + dAtA44[j43] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j42++ + j43++ } - dAtA43[j42] = uint8(num) - j42++ + dAtA44[j43] = uint8(num) + j43++ } dAtA[i] = 0xa i++ - i = encodeVarintProcessors(dAtA, i, uint64(j42)) - i += copy(dAtA[i:], dAtA43[:j42]) + i = encodeVarintProcessors(dAtA, i, uint64(j43)) + i += copy(dAtA[i:], dAtA44[:j43]) } if len(m.RightEqColumns) > 0 { - dAtA45 := make([]byte, len(m.RightEqColumns)*10) - var j44 int + dAtA46 := make([]byte, len(m.RightEqColumns)*10) + var j45 int for _, num := range m.RightEqColumns { for num >= 1<<7 { - dAtA45[j44] = uint8(uint64(num)&0x7f | 0x80) + dAtA46[j45] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j44++ + j45++ } - dAtA45[j44] = uint8(num) - j44++ + dAtA46[j45] = uint8(num) + j45++ } dAtA[i] = 0x12 i++ - i = encodeVarintProcessors(dAtA, i, uint64(j44)) - i += copy(dAtA[i:], dAtA45[:j44]) + i = encodeVarintProcessors(dAtA, i, uint64(j45)) + i += copy(dAtA[i:], dAtA46[:j45]) } dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n46, err := m.OnExpr.MarshalTo(dAtA[i:]) + n47, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n46 + i += n47 dAtA[i] = 0x30 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2023,21 +2292,21 @@ func (m *AggregatorSpec) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.GroupCols) > 0 { - dAtA48 := make([]byte, len(m.GroupCols)*10) - var j47 int + dAtA49 := make([]byte, len(m.GroupCols)*10) + var j48 int for _, num := range m.GroupCols { for num >= 1<<7 { - dAtA48[j47] = uint8(uint64(num)&0x7f | 0x80) + dAtA49[j48] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j47++ + j48++ } - dAtA48[j47] = uint8(num) - j47++ + dAtA49[j48] = uint8(num) + j48++ } dAtA[i] = 0x12 i++ - i = encodeVarintProcessors(dAtA, i, uint64(j47)) - i += copy(dAtA[i:], dAtA48[:j47]) + i = encodeVarintProcessors(dAtA, i, uint64(j48)) + i += copy(dAtA[i:], dAtA49[:j48]) } if len(m.Aggregations) > 0 { for _, msg := range m.Aggregations { @@ -2052,21 +2321,21 @@ func (m *AggregatorSpec) MarshalTo(dAtA []byte) (int, error) { } } if len(m.OrderedGroupCols) > 0 { - dAtA50 := make([]byte, len(m.OrderedGroupCols)*10) - var j49 int + dAtA51 := make([]byte, len(m.OrderedGroupCols)*10) + var j50 int for _, num := range m.OrderedGroupCols { for num >= 1<<7 { - dAtA50[j49] = uint8(uint64(num)&0x7f | 0x80) + dAtA51[j50] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j49++ + j50++ } - dAtA50[j49] = uint8(num) - j49++ + dAtA51[j50] = uint8(num) + j50++ } dAtA[i] = 0x22 i++ - i = encodeVarintProcessors(dAtA, i, uint64(j49)) - i += copy(dAtA[i:], dAtA50[:j49]) + i = encodeVarintProcessors(dAtA, i, uint64(j50)) + i += copy(dAtA[i:], dAtA51[:j50]) } dAtA[i] = 0x28 i++ @@ -2136,11 +2405,11 @@ func (m *BackfillerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Table.Size())) - n51, err := m.Table.MarshalTo(dAtA[i:]) + n52, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n52 if len(m.Spans) > 0 { for _, msg := range m.Spans { dAtA[i] = 0x1a @@ -2174,11 +2443,11 @@ func (m *BackfillerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.ReadAsOf.Size())) - n52, err := m.ReadAsOf.MarshalTo(dAtA[i:]) + n53, err := m.ReadAsOf.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n53 return i, nil } @@ -2200,11 +2469,11 @@ func (m *FlowSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.FlowID.Size())) - n53, err := m.FlowID.MarshalTo(dAtA[i:]) + n54, err := m.FlowID.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n54 if len(m.Processors) > 0 { for _, msg := range m.Processors { dAtA[i] = 0x12 @@ -2272,11 +2541,11 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Progress.Size())) - n54, err := m.Progress.MarshalTo(dAtA[i:]) + n55, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n55 if len(m.Uri) > 0 { keysForUri := make([]int32, 0, len(m.Uri)) for k := range m.Uri { @@ -2301,11 +2570,11 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Format.Size())) - n55, err := m.Format.MarshalTo(dAtA[i:]) + n56, err := m.Format.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n56 if len(m.Tables) > 0 { keysForTables := make([]string, 0, len(m.Tables)) for k := range m.Tables { @@ -2331,11 +2600,11 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(v.Size())) - n56, err := v.MarshalTo(dAtA[i:]) + n57, err := v.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n57 } } } @@ -2387,11 +2656,11 @@ func (m *SSTWriterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Progress.Size())) - n57, err := m.Progress.MarshalTo(dAtA[i:]) + n58, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n58 return i, nil } @@ -2449,11 +2718,11 @@ func (m *CSVWriterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Options.Size())) - n58, err := m.Options.MarshalTo(dAtA[i:]) + n59, err := m.Options.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n58 + i += n59 dAtA[i] = 0x20 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.ChunkRows)) @@ -2620,11 +2889,11 @@ func (m *InterleavedReaderJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n59, err := m.OnExpr.MarshalTo(dAtA[i:]) + n60, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n59 + i += n60 dAtA[i] = 0x28 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2649,30 +2918,30 @@ func (m *InterleavedReaderJoinerSpec_Table) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Desc.Size())) - n60, err := m.Desc.MarshalTo(dAtA[i:]) + n61, err := m.Desc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n60 + i += n61 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexIdx)) dAtA[i] = 0x1a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Post.Size())) - n61, err := m.Post.MarshalTo(dAtA[i:]) + n62, err := m.Post.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n61 + i += n62 dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Ordering.Size())) - n62, err := m.Ordering.MarshalTo(dAtA[i:]) + n63, err := m.Ordering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n62 + i += n63 if len(m.Spans) > 0 { for _, msg := range m.Spans { dAtA[i] = 0x2a @@ -2737,95 +3006,313 @@ func (m *ProjectSetSpec) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func encodeVarintProcessors(dAtA []byte, offset int, v uint64) int { - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ +func (m *WindowerSpec) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - dAtA[offset] = uint8(v) - return offset + 1 + return dAtA[:n], nil } -func (m *ProcessorSpec) Size() (n int) { + +func (m *WindowerSpec) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if len(m.Input) > 0 { - for _, e := range m.Input { - l = e.Size() - n += 1 + l + sovProcessors(uint64(l)) + if len(m.PartitionBy) > 0 { + for _, num := range m.PartitionBy { + dAtA[i] = 0x8 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(num)) } } - l = m.Core.Size() - n += 1 + l + sovProcessors(uint64(l)) - if len(m.Output) > 0 { - for _, e := range m.Output { - l = e.Size() - n += 1 + l + sovProcessors(uint64(l)) + if len(m.WindowFns) > 0 { + for _, msg := range m.WindowFns { + dAtA[i] = 0x12 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n } } - l = m.Post.Size() - n += 1 + l + sovProcessors(uint64(l)) - n += 1 + sovProcessors(uint64(m.StageID)) - n += 1 + sovProcessors(uint64(m.ProcessorID)) - return n + return i, nil } -func (m *PostProcessSpec) Size() (n int) { +func (m *WindowerSpec_Func) 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 *WindowerSpec_Func) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - l = m.Filter.Size() - n += 1 + l + sovProcessors(uint64(l)) - n += 2 - if len(m.OutputColumns) > 0 { - l = 0 - for _, e := range m.OutputColumns { - l += sovProcessors(uint64(e)) - } - n += 1 + sovProcessors(uint64(l)) + l + if m.AggregateFunc != nil { + dAtA[i] = 0x8 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(*m.AggregateFunc)) } - if len(m.RenderExprs) > 0 { - for _, e := range m.RenderExprs { - l = e.Size() - n += 1 + l + sovProcessors(uint64(l)) - } + if m.WindowFunc != nil { + dAtA[i] = 0x10 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(*m.WindowFunc)) } - n += 1 + sovProcessors(uint64(m.Offset)) - n += 1 + sovProcessors(uint64(m.Limit)) - return n + return i, nil } -func (m *ProcessorCoreUnion) Size() (n int) { +func (m *WindowerSpec_Frame) 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 *WindowerSpec_Frame) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if m.Noop != nil { - l = m.Noop.Size() - n += 1 + l + sovProcessors(uint64(l)) - } - if m.TableReader != nil { - l = m.TableReader.Size() - n += 1 + l + sovProcessors(uint64(l)) + dAtA[i] = 0x8 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Mode)) + dAtA[i] = 0x12 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Bounds.Size())) + n64, err := m.Bounds.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } - if m.JoinReader != nil { - l = m.JoinReader.Size() - n += 1 + l + sovProcessors(uint64(l)) + i += n64 + return i, nil +} + +func (m *WindowerSpec_Frame_Bound) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - if m.Sorter != nil { - l = m.Sorter.Size() - n += 1 + l + sovProcessors(uint64(l)) + return dAtA[:n], nil +} + +func (m *WindowerSpec_Frame_Bound) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.BoundType)) + dAtA[i] = 0x10 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.IntOffset)) + if m.TypedOffset != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintProcessors(dAtA, i, uint64(len(m.TypedOffset))) + i += copy(dAtA[i:], m.TypedOffset) } - if m.Aggregator != nil { - l = m.Aggregator.Size() - n += 1 + l + sovProcessors(uint64(l)) + dAtA[i] = 0x22 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.OffsetType.Size())) + n65, err := m.OffsetType.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } - if m.Distinct != nil { - l = m.Distinct.Size() - n += 1 + l + sovProcessors(uint64(l)) + i += n65 + return i, nil +} + +func (m *WindowerSpec_Frame_Bounds) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - if m.MergeJoiner != nil { - l = m.MergeJoiner.Size() - n += 1 + l + sovProcessors(uint64(l)) + return dAtA[:n], nil +} + +func (m *WindowerSpec_Frame_Bounds) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Start.Size())) + n66, err := m.Start.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } - if m.HashJoiner != nil { + i += n66 + if m.End != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.End.Size())) + n67, err := m.End.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n67 + } + return i, nil +} + +func (m *WindowerSpec_WindowFn) 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 *WindowerSpec_WindowFn) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Func.Size())) + n68, err := m.Func.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n68 + dAtA[i] = 0x10 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.ArgIdxStart)) + dAtA[i] = 0x18 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.ArgCount)) + dAtA[i] = 0x22 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Ordering.Size())) + n69, err := m.Ordering.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n69 + if m.Frame != nil { + dAtA[i] = 0x2a + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Frame.Size())) + n70, err := m.Frame.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n70 + } + return i, nil +} + +func encodeVarintProcessors(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *ProcessorSpec) Size() (n int) { + var l int + _ = l + if len(m.Input) > 0 { + for _, e := range m.Input { + l = e.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + } + l = m.Core.Size() + n += 1 + l + sovProcessors(uint64(l)) + if len(m.Output) > 0 { + for _, e := range m.Output { + l = e.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + } + l = m.Post.Size() + n += 1 + l + sovProcessors(uint64(l)) + n += 1 + sovProcessors(uint64(m.StageID)) + n += 1 + sovProcessors(uint64(m.ProcessorID)) + return n +} + +func (m *PostProcessSpec) Size() (n int) { + var l int + _ = l + l = m.Filter.Size() + n += 1 + l + sovProcessors(uint64(l)) + n += 2 + if len(m.OutputColumns) > 0 { + l = 0 + for _, e := range m.OutputColumns { + l += sovProcessors(uint64(e)) + } + n += 1 + sovProcessors(uint64(l)) + l + } + if len(m.RenderExprs) > 0 { + for _, e := range m.RenderExprs { + l = e.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + } + n += 1 + sovProcessors(uint64(m.Offset)) + n += 1 + sovProcessors(uint64(m.Limit)) + return n +} + +func (m *ProcessorCoreUnion) Size() (n int) { + var l int + _ = l + if m.Noop != nil { + l = m.Noop.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + if m.TableReader != nil { + l = m.TableReader.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + if m.JoinReader != nil { + l = m.JoinReader.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + if m.Sorter != nil { + l = m.Sorter.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + if m.Aggregator != nil { + l = m.Aggregator.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + if m.Distinct != nil { + l = m.Distinct.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + if m.MergeJoiner != nil { + l = m.MergeJoiner.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + if m.HashJoiner != nil { l = m.HashJoiner.Size() n += 1 + l + sovProcessors(uint64(l)) } @@ -2877,6 +3364,10 @@ func (m *ProcessorCoreUnion) Size() (n int) { l = m.ProjectSet.Size() n += 2 + l + sovProcessors(uint64(l)) } + if m.Windower != nil { + l = m.Windower.Size() + n += 2 + l + sovProcessors(uint64(l)) + } return n } @@ -3350,6 +3841,86 @@ func (m *ProjectSetSpec) Size() (n int) { return n } +func (m *WindowerSpec) Size() (n int) { + var l int + _ = l + if len(m.PartitionBy) > 0 { + for _, e := range m.PartitionBy { + n += 1 + sovProcessors(uint64(e)) + } + } + if len(m.WindowFns) > 0 { + for _, e := range m.WindowFns { + l = e.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + } + return n +} + +func (m *WindowerSpec_Func) Size() (n int) { + var l int + _ = l + if m.AggregateFunc != nil { + n += 1 + sovProcessors(uint64(*m.AggregateFunc)) + } + if m.WindowFunc != nil { + n += 1 + sovProcessors(uint64(*m.WindowFunc)) + } + return n +} + +func (m *WindowerSpec_Frame) Size() (n int) { + var l int + _ = l + n += 1 + sovProcessors(uint64(m.Mode)) + l = m.Bounds.Size() + n += 1 + l + sovProcessors(uint64(l)) + return n +} + +func (m *WindowerSpec_Frame_Bound) Size() (n int) { + var l int + _ = l + n += 1 + sovProcessors(uint64(m.BoundType)) + n += 1 + sovProcessors(uint64(m.IntOffset)) + if m.TypedOffset != nil { + l = len(m.TypedOffset) + n += 1 + l + sovProcessors(uint64(l)) + } + l = m.OffsetType.Size() + n += 1 + l + sovProcessors(uint64(l)) + return n +} + +func (m *WindowerSpec_Frame_Bounds) Size() (n int) { + var l int + _ = l + l = m.Start.Size() + n += 1 + l + sovProcessors(uint64(l)) + if m.End != nil { + l = m.End.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + return n +} + +func (m *WindowerSpec_WindowFn) Size() (n int) { + var l int + _ = l + l = m.Func.Size() + n += 1 + l + sovProcessors(uint64(l)) + n += 1 + sovProcessors(uint64(m.ArgIdxStart)) + n += 1 + sovProcessors(uint64(m.ArgCount)) + l = m.Ordering.Size() + n += 1 + l + sovProcessors(uint64(l)) + if m.Frame != nil { + l = m.Frame.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + return n +} + func sovProcessors(x uint64) (n int) { for { n++ @@ -3424,6 +3995,9 @@ func (this *ProcessorCoreUnion) GetValue() interface{} { if this.ProjectSet != nil { return this.ProjectSet } + if this.Windower != nil { + return this.Windower + } return nil } @@ -3469,6 +4043,29 @@ func (this *ProcessorCoreUnion) SetValue(value interface{}) bool { this.ZigzagJoiner = vt case *ProjectSetSpec: this.ProjectSet = vt + case *WindowerSpec: + this.Windower = vt + default: + return false + } + return true +} +func (this *WindowerSpec_Func) GetValue() interface{} { + if this.AggregateFunc != nil { + return this.AggregateFunc + } + if this.WindowFunc != nil { + return this.WindowFunc + } + return nil +} + +func (this *WindowerSpec_Func) SetValue(value interface{}) bool { + switch vt := value.(type) { + case *AggregatorSpec_Func: + this.AggregateFunc = vt + case *WindowerSpec_WindowFunc: + this.WindowFunc = vt default: return false } @@ -4604,6 +5201,39 @@ func (m *ProcessorCoreUnion) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 23: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Windower", 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.Windower == nil { + m.Windower = &WindowerSpec{} + } + if err := m.Windower.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProcessors(dAtA[iNdEx:]) @@ -9172,6 +9802,781 @@ func (m *ProjectSetSpec) Unmarshal(dAtA []byte) error { } return nil } +func (m *WindowerSpec) 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: WindowerSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WindowerSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + 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.PartitionBy = append(m.PartitionBy, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + for iNdEx < postIndex { + 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.PartitionBy = append(m.PartitionBy, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field PartitionBy", wireType) + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WindowFns", 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 + } + m.WindowFns = append(m.WindowFns, WindowerSpec_WindowFn{}) + if err := m.WindowFns[len(m.WindowFns)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 *WindowerSpec_Func) 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: Func: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Func: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AggregateFunc", wireType) + } + var v AggregatorSpec_Func + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (AggregatorSpec_Func(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.AggregateFunc = &v + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WindowFunc", wireType) + } + var v WindowerSpec_WindowFunc + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (WindowerSpec_WindowFunc(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.WindowFunc = &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 *WindowerSpec_Frame) 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: Frame: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Frame: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Mode", wireType) + } + m.Mode = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Mode |= (WindowerSpec_Frame_Mode(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Bounds", 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 err := m.Bounds.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 *WindowerSpec_Frame_Bound) 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: Bound: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Bound: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BoundType", wireType) + } + m.BoundType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BoundType |= (WindowerSpec_Frame_BoundType(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IntOffset", wireType) + } + m.IntOffset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.IntOffset |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TypedOffset", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TypedOffset = append(m.TypedOffset[:0], dAtA[iNdEx:postIndex]...) + if m.TypedOffset == nil { + m.TypedOffset = []byte{} + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OffsetType", 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 err := m.OffsetType.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 *WindowerSpec_Frame_Bounds) 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: Bounds: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Bounds: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Start", 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 err := m.Start.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field End", 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.End == nil { + m.End = &WindowerSpec_Frame_Bound{} + } + if err := m.End.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 *WindowerSpec_WindowFn) 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: WindowFn: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: WindowFn: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Func", 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 err := m.Func.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ArgIdxStart", wireType) + } + m.ArgIdxStart = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ArgIdxStart |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ArgCount", wireType) + } + m.ArgCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ArgCount |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Ordering", 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 err := m.Ordering.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Frame", 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.Frame == nil { + m.Frame = &WindowerSpec_Frame{} + } + if err := m.Frame.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 skipProcessors(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 @@ -9280,199 +10685,231 @@ var ( func init() { proto.RegisterFile("sql/distsqlrun/processors.proto", fileDescriptorProcessors) } var fileDescriptorProcessors = []byte{ - // 3100 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xcb, 0x6f, 0xdc, 0xd6, - 0xd5, 0xf7, 0x70, 0x38, 0xaf, 0x33, 0x0f, 0xd1, 0xd7, 0x4e, 0x32, 0x51, 0xf2, 0x59, 0x36, 0xe3, - 0xc4, 0x8f, 0xcf, 0x91, 0x62, 0xe7, 0x81, 0x3c, 0x91, 0xcc, 0x43, 0x92, 0x47, 0x91, 0x66, 0x1c, - 0x8e, 0xa4, 0x3c, 0x16, 0x1f, 0x3f, 0x8a, 0xbc, 0x1a, 0xd1, 0xe2, 0xf0, 0x8e, 0x49, 0x8e, 0x65, - 0x19, 0x05, 0xba, 0xea, 0xba, 0x05, 0xda, 0xa2, 0xdb, 0x6e, 0xba, 0xee, 0xae, 0xe8, 0xa6, 0x05, - 0x0a, 0x74, 0xe1, 0x65, 0xbb, 0x0b, 0x8a, 0xc2, 0x68, 0x95, 0x55, 0xff, 0x05, 0x2f, 0x8a, 0xe2, - 0xbe, 0x38, 0x1c, 0x59, 0x9a, 0x91, 0xec, 0x22, 0xd9, 0x08, 0xc3, 0xf3, 0xf8, 0xdd, 0x73, 0x0f, - 0xcf, 0x3d, 0xe7, 0xdc, 0x43, 0xc1, 0x5c, 0x78, 0xcf, 0x5b, 0x70, 0xdc, 0x30, 0x0a, 0xef, 0x79, - 0xc1, 0xd0, 0x5f, 0x18, 0x04, 0xc4, 0xc6, 0x61, 0x48, 0x82, 0x70, 0x7e, 0x10, 0x90, 0x88, 0xa0, - 0xaa, 0x4d, 0xec, 0xdd, 0x80, 0x58, 0xf6, 0xce, 0x7c, 0x78, 0xcf, 0x9b, 0x1f, 0x89, 0xce, 0x9e, - 0x65, 0xd4, 0xc1, 0xd6, 0x82, 0x35, 0x70, 0xb9, 0xf0, 0x2c, 0x92, 0x24, 0xc7, 0x8a, 0x2c, 0x41, - 0x3b, 0x2f, 0x69, 0x38, 0x08, 0x62, 0xd8, 0xd9, 0xaa, 0xa4, 0xba, 0xe4, 0xcd, 0x6d, 0x12, 0xf4, - 0xad, 0x48, 0x72, 0x5e, 0xa5, 0x16, 0x85, 0xf7, 0xbc, 0x2d, 0x2b, 0xc4, 0x0b, 0x61, 0x14, 0x0c, - 0xed, 0x68, 0x18, 0x60, 0x47, 0x70, 0xe7, 0x92, 0x5c, 0xec, 0xdb, 0xc4, 0xc1, 0x8e, 0xe9, 0x58, - 0xd1, 0xb0, 0x2f, 0x04, 0x5e, 0x49, 0x0a, 0xdc, 0x25, 0xae, 0x6f, 0x46, 0xfb, 0x03, 0x2c, 0x98, - 0x2f, 0x1f, 0xda, 0x6d, 0xc2, 0xcc, 0xea, 0x30, 0x72, 0xbd, 0x85, 0x1d, 0xcf, 0x5e, 0x88, 0xdc, - 0x3e, 0x0e, 0x23, 0xab, 0x3f, 0x90, 0x1b, 0xe8, 0x91, 0x1e, 0x61, 0x3f, 0x17, 0xe8, 0x2f, 0x4e, - 0xd5, 0x7f, 0x9a, 0x86, 0xf2, 0x1d, 0xe9, 0xac, 0xee, 0x00, 0xdb, 0xa8, 0x01, 0x19, 0xd7, 0x1f, - 0x0c, 0xa3, 0x6a, 0xea, 0x62, 0xfa, 0x6a, 0xf1, 0xd6, 0x95, 0xf9, 0xe3, 0x3c, 0x37, 0xdf, 0xa2, - 0x62, 0xdd, 0x7d, 0xdf, 0xa6, 0x7a, 0x75, 0xf5, 0xd1, 0xe3, 0xb9, 0x33, 0x06, 0xd7, 0x45, 0x4b, - 0xa0, 0xda, 0x24, 0xc0, 0x55, 0xe5, 0x62, 0xea, 0x6a, 0xf1, 0xd6, 0x8d, 0xe3, 0x31, 0xe2, 0xb5, - 0x1b, 0x24, 0xc0, 0x1b, 0xbe, 0x4b, 0x7c, 0x01, 0xc4, 0xf4, 0xd1, 0x6d, 0xc8, 0x92, 0x61, 0x44, - 0xad, 0x49, 0x33, 0x6b, 0xae, 0x1f, 0x8f, 0xd4, 0x61, 0x72, 0x06, 0x19, 0x46, 0x38, 0x48, 0x18, - 0x24, 0xf4, 0x51, 0x03, 0xd4, 0x01, 0x09, 0xa3, 0xaa, 0xca, 0x2c, 0xba, 0x36, 0xc1, 0x22, 0x12, - 0x46, 0xc2, 0xaa, 0x04, 0x0c, 0x53, 0x46, 0xd7, 0x21, 0x1f, 0x46, 0x56, 0x0f, 0x9b, 0xae, 0x53, - 0xcd, 0x5c, 0x4c, 0x5d, 0xcd, 0xd4, 0x67, 0x28, 0xf7, 0xe0, 0xf1, 0x5c, 0xae, 0x4b, 0xe9, 0xad, - 0xa6, 0x91, 0x63, 0x02, 0x2d, 0x07, 0xbd, 0x07, 0xa5, 0x38, 0x0a, 0xa9, 0x7c, 0x96, 0xc9, 0x9f, - 0x13, 0xf2, 0xc5, 0x78, 0xe3, 0xad, 0xa6, 0x51, 0x8c, 0x05, 0x5b, 0x8e, 0xfe, 0x5b, 0x05, 0x66, - 0x0e, 0xd9, 0x80, 0xea, 0x90, 0xdd, 0x76, 0xbd, 0x08, 0x07, 0xd5, 0x14, 0x33, 0xff, 0xf2, 0xf1, - 0xe6, 0x2f, 0x3e, 0x18, 0x04, 0x38, 0x0c, 0x47, 0x8e, 0x14, 0x9a, 0xe8, 0x32, 0xc0, 0x20, 0x20, - 0x77, 0xb1, 0x1d, 0xb9, 0xc4, 0x67, 0x2f, 0x26, 0x2f, 0x24, 0x12, 0x74, 0x74, 0x0d, 0x2a, 0xdc, - 0x61, 0xa6, 0x4d, 0xbc, 0x61, 0xdf, 0x0f, 0x99, 0xe3, 0xcb, 0x75, 0x45, 0x4b, 0x19, 0x65, 0xce, - 0x69, 0x70, 0x06, 0x5a, 0x83, 0x52, 0x80, 0x7d, 0x07, 0x07, 0x26, 0x7e, 0x30, 0x08, 0xc2, 0xaa, - 0xca, 0xde, 0xd0, 0x69, 0x4c, 0x2b, 0x72, 0x7d, 0x4a, 0x0f, 0xd1, 0xab, 0x90, 0x25, 0xdb, 0xdb, - 0x21, 0x8e, 0x98, 0x67, 0xd5, 0xf8, 0xf5, 0x31, 0x1a, 0x9a, 0x85, 0x8c, 0xe7, 0xf6, 0xdd, 0x88, - 0xb9, 0x51, 0x32, 0x39, 0x49, 0xff, 0x5d, 0x09, 0xd0, 0xd3, 0x71, 0x84, 0x3e, 0x04, 0xd5, 0x27, - 0x64, 0x20, 0x5c, 0xf6, 0xc6, 0xf1, 0x76, 0xb5, 0x09, 0x19, 0x50, 0x35, 0xea, 0x6a, 0x83, 0xe9, - 0xa0, 0xcf, 0xa1, 0x18, 0x59, 0x5b, 0x1e, 0x36, 0xb0, 0xe5, 0xe0, 0x40, 0x84, 0xf1, 0x84, 0xa0, - 0x59, 0x1f, 0x09, 0x33, 0x94, 0xa4, 0x36, 0xba, 0x0d, 0x40, 0x4f, 0xb0, 0xc0, 0x4a, 0x33, 0xac, - 0xab, 0xc7, 0x63, 0xad, 0xc4, 0xb2, 0x0c, 0x2a, 0xa1, 0x8b, 0x3e, 0x86, 0x6c, 0x48, 0x02, 0x1a, - 0x07, 0xea, 0xb4, 0x38, 0xe8, 0x32, 0x39, 0x86, 0x20, 0x74, 0xa8, 0x1d, 0x56, 0xaf, 0x17, 0xe0, - 0x9e, 0x15, 0x91, 0x80, 0x79, 0x79, 0xa2, 0x1d, 0xb5, 0x58, 0x96, 0xdb, 0x31, 0xd2, 0x45, 0x75, - 0xc8, 0x53, 0x41, 0xd7, 0xb7, 0xa3, 0x6a, 0x6e, 0x9a, 0x7b, 0x9b, 0x42, 0x92, 0xa1, 0xc4, 0x7a, - 0xd4, 0xc5, 0x7d, 0x1c, 0xf4, 0x30, 0xdd, 0x2e, 0x0e, 0xaa, 0xf9, 0x69, 0x2e, 0x5e, 0x1b, 0x09, - 0x73, 0x17, 0x27, 0xb4, 0xe9, 0xd6, 0x76, 0xac, 0x70, 0x47, 0x60, 0x15, 0xa6, 0x6d, 0xed, 0x76, - 0x2c, 0xcb, 0xb7, 0x36, 0xd2, 0x45, 0x9f, 0x41, 0xf6, 0xbe, 0xe5, 0x0d, 0x71, 0x58, 0x85, 0x69, - 0x28, 0x9b, 0x4c, 0x2e, 0x8e, 0x1c, 0xa1, 0x47, 0x6d, 0xd9, 0xb2, 0xec, 0xdd, 0x6d, 0xd7, 0xf3, - 0x70, 0x50, 0x2d, 0x4e, 0x43, 0xa9, 0xc7, 0xb2, 0xdc, 0x96, 0x91, 0x2e, 0x5a, 0x05, 0x08, 0xb0, - 0xe5, 0xb4, 0xfa, 0x03, 0x12, 0x44, 0xd5, 0xf2, 0xb4, 0x5c, 0x6a, 0xc4, 0xb2, 0x4d, 0x2b, 0xb2, - 0x38, 0xda, 0x48, 0x1f, 0x2d, 0x42, 0xa1, 0xdb, 0x5d, 0xff, 0x32, 0x70, 0x69, 0xfc, 0x54, 0x18, - 0xd8, 0x84, 0xe4, 0x1e, 0x8b, 0x32, 0x9c, 0x91, 0x26, 0xfa, 0x14, 0x72, 0x5d, 0xab, 0x3f, 0xa0, - 0x7b, 0x9b, 0x61, 0x20, 0xaf, 0x4f, 0x00, 0xe1, 0x82, 0x0c, 0x42, 0x6a, 0xa1, 0x6f, 0x40, 0xe3, - 0x3f, 0x47, 0x01, 0x56, 0xd5, 0x18, 0xd2, 0xfc, 0x34, 0xa4, 0x43, 0x21, 0xf9, 0x14, 0x0e, 0x22, - 0xf0, 0x92, 0xeb, 0x47, 0x38, 0xf0, 0xb0, 0x75, 0x1f, 0x3b, 0xfc, 0xd4, 0x88, 0xa0, 0x38, 0xcb, - 0x96, 0x78, 0x77, 0x52, 0x39, 0x3b, 0x52, 0x91, 0xad, 0x74, 0x1c, 0x2a, 0xfa, 0x7f, 0x40, 0x7d, - 0x1c, 0x59, 0xb4, 0x02, 0xaf, 0xe3, 0x30, 0xea, 0xb2, 0x84, 0x56, 0x45, 0x6c, 0xad, 0xb7, 0x26, - 0x05, 0xf3, 0x61, 0x1d, 0xb6, 0xcc, 0x11, 0x58, 0x68, 0x1b, 0xce, 0x27, 0xa9, 0x06, 0xb6, 0xb1, - 0x7b, 0x1f, 0x07, 0xd5, 0x73, 0x6c, 0x8d, 0x5b, 0x27, 0x5b, 0x43, 0x6a, 0xb1, 0x55, 0x8e, 0xc4, - 0xa3, 0xe1, 0xd1, 0xe8, 0x6e, 0x8a, 0xf0, 0x38, 0x3f, 0x2d, 0x3c, 0x62, 0x51, 0x1e, 0x1e, 0xf1, - 0x23, 0x6a, 0x43, 0xe9, 0xa1, 0xdb, 0x7b, 0x68, 0xf5, 0x84, 0xdb, 0x5f, 0x60, 0x48, 0x13, 0xea, - 0xf6, 0x37, 0x09, 0x69, 0x06, 0x36, 0xa6, 0x4f, 0x4f, 0x93, 0x28, 0x4f, 0x5d, 0x1c, 0x55, 0x5f, - 0x9c, 0x76, 0x9a, 0xee, 0xc4, 0xb2, 0x3c, 0xfe, 0x47, 0xba, 0x1f, 0xaa, 0x8f, 0x7e, 0x3d, 0x97, - 0x5a, 0x51, 0xf3, 0x59, 0x2d, 0xb7, 0xa2, 0xe6, 0x4b, 0x5a, 0x59, 0xaf, 0x40, 0x29, 0x99, 0xfb, - 0xf5, 0x77, 0xe0, 0xc5, 0xa3, 0x5f, 0x0c, 0x9a, 0x05, 0xc5, 0x75, 0x58, 0x25, 0x29, 0xd4, 0x41, - 0x94, 0x70, 0xa5, 0xd5, 0x34, 0x14, 0xd7, 0xd1, 0x6f, 0x43, 0xf5, 0x38, 0x57, 0xa3, 0x1b, 0x00, - 0x21, 0xaf, 0x91, 0xae, 0x13, 0xb2, 0x8e, 0xaa, 0x50, 0x2f, 0x1f, 0x3c, 0x9e, 0x2b, 0x70, 0xec, - 0x56, 0x33, 0x34, 0x0a, 0x5c, 0xa0, 0xe5, 0x84, 0xfa, 0xcf, 0x53, 0x50, 0x19, 0x4f, 0x2a, 0xa8, - 0x01, 0x39, 0x59, 0x88, 0x79, 0x3f, 0xf6, 0xda, 0x84, 0x44, 0x4b, 0xfb, 0xc7, 0x96, 0xbf, 0x4d, - 0x44, 0x79, 0x94, 0x9a, 0xe8, 0x15, 0x28, 0x04, 0xd6, 0x9e, 0xb9, 0xb5, 0x1f, 0xe1, 0xb0, 0xaa, - 0x5c, 0x4c, 0x5f, 0x2d, 0x19, 0xf9, 0xc0, 0xda, 0xab, 0xd3, 0x67, 0x34, 0x07, 0x79, 0x7f, 0xd8, - 0x37, 0x03, 0xb2, 0x17, 0xb2, 0xda, 0x24, 0x8b, 0x6b, 0xce, 0x1f, 0xf6, 0x0d, 0xb2, 0x17, 0xea, - 0x4d, 0x98, 0x19, 0x2b, 0x6f, 0x96, 0x8f, 0x6e, 0x82, 0x1a, 0x0e, 0x2c, 0x5f, 0x94, 0xd6, 0x97, - 0x12, 0x26, 0x89, 0x7e, 0x78, 0x9e, 0x8a, 0xc9, 0xd6, 0x89, 0x8a, 0xea, 0xbf, 0x51, 0x0e, 0xc1, - 0xb0, 0xb6, 0x26, 0xc3, 0xea, 0xe4, 0x31, 0x25, 0x5a, 0xb4, 0xbf, 0xbc, 0xb8, 0x36, 0x71, 0x68, - 0x07, 0xee, 0x20, 0x22, 0x81, 0x2c, 0xfe, 0x4c, 0x15, 0x5d, 0x82, 0x82, 0xeb, 0x3b, 0xf8, 0x81, - 0xe9, 0x3a, 0x0f, 0x58, 0x9d, 0x2e, 0x0b, 0x7e, 0x9e, 0x91, 0x5b, 0xce, 0x03, 0x74, 0x01, 0x72, - 0x01, 0xbe, 0x8f, 0x83, 0x10, 0xb3, 0x0d, 0xca, 0xb6, 0x47, 0x12, 0xd1, 0x22, 0x64, 0xa8, 0x89, - 0xb2, 0x83, 0x39, 0x69, 0x99, 0x8f, 0x37, 0xc8, 0xb5, 0xd1, 0x6b, 0x00, 0xac, 0x1f, 0x31, 0x77, - 0x5c, 0x9f, 0x37, 0x31, 0x69, 0x21, 0x50, 0x60, 0xf4, 0xdb, 0xae, 0x1f, 0x51, 0x6f, 0xbb, 0xa1, - 0x69, 0xef, 0x60, 0x7b, 0x97, 0xb5, 0x32, 0xb1, 0x31, 0x6e, 0xd8, 0xa0, 0x44, 0xfd, 0x89, 0x02, - 0x95, 0xf1, 0x0e, 0xe0, 0xfb, 0x72, 0xd3, 0x35, 0xa8, 0x78, 0x84, 0xec, 0x0e, 0x07, 0x47, 0xb5, - 0x7e, 0x9c, 0x23, 0x5b, 0xbf, 0x06, 0xe4, 0x88, 0xcf, 0xda, 0xbe, 0xe9, 0x8d, 0xc8, 0xd3, 0x0d, - 0x29, 0xf1, 0x29, 0x0d, 0x6d, 0xc2, 0x59, 0x6e, 0x12, 0x6f, 0x50, 0x39, 0x5c, 0xe6, 0xd4, 0x70, - 0x33, 0x0c, 0x64, 0x89, 0x61, 0x30, 0xdc, 0x0f, 0x40, 0xa5, 0x77, 0x25, 0xe6, 0xde, 0xca, 0xad, - 0xb9, 0x63, 0xbc, 0x45, 0x7d, 0xbc, 0xbe, 0x3f, 0xc0, 0x32, 0x48, 0xa9, 0x0a, 0x3d, 0x80, 0x30, - 0x6a, 0x9c, 0xd0, 0x17, 0x30, 0x23, 0x9a, 0x61, 0x12, 0x38, 0x38, 0x70, 0xfd, 0x9e, 0x78, 0x05, - 0xfa, 0x84, 0x6b, 0x88, 0x90, 0x14, 0xb8, 0xa2, 0x9b, 0x96, 0x54, 0x74, 0x0b, 0x90, 0xc4, 0x32, - 0xfb, 0x56, 0x64, 0xef, 0x98, 0x1e, 0xf6, 0xc7, 0x5e, 0x88, 0x26, 0xf9, 0x6b, 0x94, 0xbd, 0x8a, - 0x7d, 0x7d, 0x0b, 0x4a, 0xc9, 0x1e, 0x0a, 0x5d, 0x81, 0x19, 0x26, 0x83, 0x1d, 0x33, 0x99, 0x1b, - 0xca, 0x46, 0x45, 0x90, 0xe5, 0x6b, 0xba, 0x06, 0x9a, 0x6c, 0xb7, 0x62, 0x49, 0x85, 0x49, 0xce, - 0x48, 0xba, 0x10, 0xd5, 0xff, 0xa4, 0x80, 0x76, 0x38, 0x13, 0xa3, 0x26, 0x64, 0x59, 0xf4, 0xc8, - 0xdc, 0x73, 0xba, 0xc8, 0x13, 0xba, 0x68, 0x09, 0x00, 0xdf, 0x1b, 0x5b, 0xbf, 0x78, 0xeb, 0xd2, - 0x84, 0xca, 0xc2, 0x05, 0xe5, 0xd1, 0xc1, 0xf7, 0xe4, 0x6e, 0xe6, 0x46, 0x21, 0x9c, 0x0c, 0x4d, - 0x19, 0xc0, 0xff, 0xa5, 0xa8, 0x94, 0xd1, 0x93, 0x39, 0x7d, 0xf4, 0x5c, 0x81, 0x9c, 0xb4, 0xf5, - 0xd5, 0xf1, 0xb4, 0xcd, 0x2d, 0x95, 0x24, 0xfd, 0xef, 0x0a, 0xcc, 0x1c, 0x6a, 0x67, 0xd1, 0x1a, - 0x94, 0x3d, 0xbc, 0xfd, 0x1c, 0x91, 0x56, 0xa2, 0xea, 0x71, 0x9c, 0x75, 0xa0, 0x12, 0xb8, 0xbd, - 0x9d, 0x04, 0x9e, 0x72, 0x4a, 0xbc, 0x32, 0xd3, 0x8f, 0x01, 0x13, 0xce, 0xcd, 0x3c, 0xb7, 0x73, - 0x4f, 0x7f, 0x34, 0xd1, 0x35, 0x28, 0xfb, 0x43, 0xcf, 0x33, 0xf1, 0xbd, 0xa1, 0xe5, 0xb9, 0xd1, - 0x3e, 0xbb, 0x77, 0xc8, 0xec, 0x59, 0xa2, 0xac, 0x45, 0xc1, 0xd1, 0x7f, 0xa5, 0x40, 0x65, 0xbc, - 0xc3, 0x47, 0xd7, 0x61, 0x86, 0x79, 0x37, 0x11, 0x88, 0xa9, 0x44, 0x72, 0xc3, 0xdb, 0xd1, 0x62, - 0x1c, 0x67, 0x37, 0x40, 0xe3, 0xae, 0x3b, 0x14, 0xb5, 0x5c, 0x98, 0xbb, 0x75, 0x24, 0xfd, 0x43, - 0xfb, 0xe5, 0x7f, 0xa1, 0xc2, 0x2e, 0x42, 0xa3, 0x5c, 0x90, 0x74, 0x4c, 0x99, 0xf3, 0xe4, 0x29, - 0xff, 0x45, 0x16, 0x2a, 0xe3, 0x3d, 0x34, 0xba, 0x04, 0xd0, 0x0b, 0x08, 0x4f, 0xfa, 0xc9, 0x7d, - 0x16, 0x18, 0xb5, 0x41, 0xbc, 0x10, 0xfd, 0x1f, 0x94, 0xe4, 0xdd, 0xcf, 0x25, 0xa2, 0x2c, 0x14, - 0x6f, 0xbd, 0x73, 0xd2, 0x9b, 0x63, 0xfc, 0x38, 0xda, 0xf7, 0x18, 0x1e, 0x7a, 0x4b, 0xe4, 0x44, - 0xec, 0x98, 0x09, 0x53, 0xd4, 0xd8, 0x14, 0x4d, 0x70, 0x97, 0x63, 0x8b, 0x96, 0xc7, 0x0e, 0xe9, - 0x9b, 0x27, 0xb6, 0xe4, 0xb0, 0xf7, 0x66, 0xff, 0x90, 0x82, 0x62, 0xc2, 0x3c, 0x0a, 0xbc, 0x3d, - 0xf4, 0x6d, 0x76, 0xf8, 0x4e, 0x03, 0xbc, 0x34, 0xf4, 0xe3, 0x49, 0x11, 0x05, 0x40, 0x17, 0x13, - 0x37, 0xe4, 0xe4, 0xac, 0x65, 0x74, 0xff, 0xbd, 0x0c, 0x15, 0x51, 0xf8, 0x6c, 0xe2, 0xb1, 0xb2, - 0x4c, 0x93, 0x56, 0xd9, 0x28, 0x71, 0x6a, 0x83, 0x78, 0xb4, 0x28, 0xbf, 0xc4, 0x12, 0x09, 0x63, - 0x67, 0x58, 0xe6, 0xce, 0xda, 0x8c, 0xb1, 0xa2, 0xe6, 0xd3, 0x9a, 0xaa, 0xff, 0x52, 0x01, 0x95, - 0xae, 0x8d, 0x34, 0x28, 0xd5, 0xda, 0x5f, 0x9b, 0xed, 0xce, 0xba, 0xd9, 0xde, 0x58, 0x5d, 0xd5, - 0xce, 0xa0, 0x1c, 0xa4, 0x6b, 0x9b, 0xcb, 0x5a, 0x0a, 0x95, 0x20, 0x5f, 0xef, 0x74, 0x56, 0xcd, - 0x5a, 0xbb, 0xa9, 0x29, 0xa8, 0x08, 0x39, 0xf6, 0xd4, 0x31, 0xb4, 0x34, 0xaa, 0x00, 0x34, 0x3a, - 0xed, 0x46, 0x6d, 0xdd, 0xac, 0x2d, 0x2f, 0x6b, 0x2a, 0x2a, 0x40, 0xa6, 0xd1, 0xd9, 0x68, 0xaf, - 0x6b, 0x19, 0xaa, 0xbe, 0x56, 0xfb, 0x4a, 0xcb, 0xb1, 0x1f, 0xad, 0xb6, 0x96, 0x47, 0x00, 0xd9, - 0xee, 0x7a, 0xb3, 0xb9, 0xb8, 0xa9, 0x15, 0x28, 0xb1, 0xbb, 0xb1, 0xa6, 0x01, 0x85, 0xeb, 0x6e, - 0xac, 0x99, 0xad, 0xf6, 0xba, 0x56, 0xa4, 0x2b, 0x6d, 0xd6, 0x8c, 0x56, 0xad, 0xdd, 0x58, 0xd4, - 0x4a, 0x94, 0xf5, 0x55, 0xc7, 0x60, 0xc8, 0x65, 0xbe, 0xd2, 0x46, 0x7b, 0xdd, 0x34, 0x3a, 0x5f, - 0x76, 0xb5, 0x0a, 0xd3, 0xfb, 0xc2, 0x68, 0xb6, 0x96, 0x96, 0xb4, 0x19, 0x84, 0xa0, 0xb2, 0xd4, - 0x6a, 0xd7, 0x56, 0xcd, 0x58, 0x5b, 0xa3, 0x1b, 0xe2, 0x34, 0xb1, 0xe6, 0x59, 0x54, 0x86, 0x42, - 0xcd, 0x30, 0x6a, 0x5f, 0x33, 0x44, 0x44, 0x17, 0x5b, 0xe9, 0x76, 0xda, 0xec, 0xe9, 0x1c, 0x65, - 0xd2, 0xa7, 0x3a, 0x7b, 0x3c, 0xaf, 0xdf, 0x00, 0x95, 0xbe, 0x6b, 0x94, 0x07, 0xb5, 0xb6, 0xb1, - 0xde, 0xd1, 0xce, 0x30, 0xeb, 0x1b, 0xb5, 0xd5, 0x9a, 0xa1, 0xa5, 0xa8, 0x31, 0xed, 0x4e, 0xdb, - 0x14, 0xcf, 0x8a, 0xfe, 0x24, 0x0d, 0x95, 0xf1, 0x6b, 0x78, 0x1c, 0x61, 0x53, 0x03, 0x61, 0x5c, - 0xef, 0xa9, 0x08, 0x1b, 0x35, 0x6f, 0xca, 0xb3, 0x37, 0x6f, 0x71, 0x83, 0x9a, 0x7e, 0xae, 0x06, - 0xf5, 0x26, 0xe4, 0x9d, 0x61, 0xc0, 0x02, 0x9d, 0xc5, 0x5a, 0xba, 0xfe, 0x02, 0x65, 0x3f, 0x79, - 0x3c, 0x57, 0x8e, 0xdc, 0x3e, 0x9e, 0x6f, 0x0a, 0xa6, 0x11, 0x8b, 0xd1, 0x9e, 0xd6, 0xde, 0x19, - 0xfa, 0xbb, 0x66, 0xe8, 0x3e, 0xc4, 0xe3, 0x3d, 0x2d, 0xa3, 0x77, 0xdd, 0x87, 0x18, 0x75, 0xa0, - 0x44, 0xa2, 0x1d, 0x1c, 0x98, 0xa2, 0x59, 0xc8, 0x3e, 0x43, 0xb3, 0x50, 0x64, 0x08, 0xeb, 0xbc, - 0x63, 0xf8, 0x14, 0xf2, 0x01, 0xb6, 0x9c, 0x5a, 0xd8, 0xd9, 0x16, 0xe3, 0xa5, 0xff, 0x49, 0x80, - 0x0d, 0x23, 0xd7, 0x9b, 0xdf, 0xf1, 0xec, 0xf9, 0x75, 0x39, 0xe1, 0x96, 0x67, 0x4b, 0x2a, 0xe9, - 0xd7, 0xc5, 0xeb, 0x2f, 0x42, 0xae, 0xe5, 0xdf, 0xb7, 0x3c, 0xd7, 0xe1, 0x11, 0xc0, 0xf3, 0xa0, - 0x96, 0xa2, 0x81, 0xde, 0xa2, 0x6d, 0x83, 0xa6, 0xe8, 0xdf, 0xa5, 0x20, 0xbf, 0xe4, 0x91, 0x3d, - 0xf6, 0xda, 0x6f, 0x42, 0x6e, 0xdb, 0x23, 0x7b, 0xa6, 0xb8, 0xec, 0x95, 0xea, 0x55, 0x8a, 0xfc, - 0xb7, 0xc7, 0x73, 0x59, 0x2a, 0xd2, 0x6a, 0x1e, 0xc4, 0xbf, 0x8c, 0x2c, 0x15, 0x6c, 0x39, 0x68, - 0x8d, 0x5d, 0x50, 0xc5, 0xd7, 0x06, 0xd1, 0xde, 0x5c, 0x39, 0xc1, 0xc0, 0x3b, 0x31, 0x5c, 0x4e, - 0x00, 0xa0, 0x0d, 0xc8, 0xf5, 0xac, 0x08, 0xef, 0x59, 0xfb, 0xec, 0xb2, 0x92, 0xa9, 0x7f, 0x24, - 0xde, 0xd1, 0xdb, 0x3d, 0x37, 0xda, 0x19, 0x6e, 0xcd, 0xdb, 0xa4, 0xbf, 0x10, 0xa3, 0x3b, 0x5b, - 0xa3, 0xdf, 0x0b, 0x83, 0xdd, 0xde, 0x82, 0xbc, 0x7f, 0xb5, 0x89, 0xc3, 0xa6, 0xd1, 0x02, 0x4b, - 0xdf, 0x83, 0xe2, 0x0a, 0xd9, 0xba, 0x13, 0x90, 0x1e, 0xad, 0x40, 0xe8, 0x32, 0x64, 0xef, 0x92, - 0x2d, 0xb9, 0xcd, 0x74, 0xbd, 0x2c, 0xee, 0xb4, 0x99, 0x15, 0xb2, 0xd5, 0x6a, 0x1a, 0x99, 0xbb, - 0x64, 0xab, 0xe5, 0xa0, 0xab, 0x50, 0xb2, 0x89, 0x1f, 0x05, 0xee, 0xd6, 0x30, 0x1e, 0x1a, 0x2b, - 0x32, 0x85, 0x27, 0x39, 0xa8, 0x0a, 0x6a, 0xe8, 0x91, 0x48, 0x98, 0x2c, 0xef, 0x7d, 0x1e, 0x89, - 0xf4, 0x6f, 0x55, 0x40, 0x4f, 0x0f, 0xa6, 0xd0, 0xeb, 0x50, 0x0c, 0xd9, 0xf0, 0x86, 0x47, 0x96, - 0x92, 0xd0, 0x03, 0xce, 0x60, 0xa1, 0xb5, 0x0c, 0xf9, 0x81, 0xb0, 0x99, 0x15, 0xc7, 0x89, 0xd3, - 0xa6, 0xc4, 0x06, 0x65, 0x44, 0x48, 0x65, 0xb4, 0x0c, 0xe9, 0x61, 0xe0, 0x56, 0x73, 0xec, 0xf5, - 0xbc, 0x7b, 0x9a, 0x19, 0xda, 0xfc, 0x46, 0xe0, 0x2e, 0xfa, 0x51, 0xb0, 0x6f, 0x50, 0x04, 0xf4, - 0x09, 0x64, 0xf9, 0x87, 0x1e, 0x31, 0xb1, 0x9c, 0x3b, 0xe2, 0xf2, 0xdb, 0xea, 0x2c, 0xb9, 0x1e, - 0x5e, 0x62, 0x62, 0xf1, 0x14, 0x9e, 0x3d, 0xa1, 0xcd, 0xb8, 0xa5, 0x2e, 0x30, 0x53, 0xde, 0x3f, - 0x95, 0x29, 0xfc, 0x7c, 0x30, 0x6b, 0x18, 0x6e, 0x2a, 0x6e, 0xb2, 0x3f, 0x85, 0x97, 0xc3, 0x5d, - 0x77, 0x60, 0xf6, 0xdd, 0x30, 0xa4, 0x77, 0x8b, 0x6d, 0x12, 0x60, 0xb7, 0xe7, 0x9b, 0xbb, 0x78, - 0x9f, 0x4f, 0x32, 0x65, 0x01, 0x7a, 0x91, 0x8a, 0xad, 0x71, 0xa9, 0x25, 0x2e, 0xf4, 0x39, 0xde, - 0x0f, 0x67, 0x2d, 0x28, 0x26, 0xd0, 0x91, 0x06, 0xe9, 0x5d, 0xbc, 0xcf, 0x27, 0x1e, 0x06, 0xfd, - 0x89, 0x3e, 0x86, 0x0c, 0x1b, 0x70, 0x9e, 0x2e, 0x91, 0x19, 0x5c, 0xe9, 0x43, 0xe5, 0xfd, 0xd4, - 0xec, 0x7b, 0x90, 0x97, 0xbe, 0x4c, 0xe2, 0x67, 0x38, 0xfe, 0xf9, 0x24, 0x7e, 0x21, 0xa1, 0xb7, - 0xa2, 0xe6, 0x53, 0x9a, 0xc2, 0x0b, 0xde, 0x8a, 0x9a, 0x57, 0xb5, 0xcc, 0x8a, 0x9a, 0xcf, 0x68, - 0x59, 0xfd, 0xf7, 0x0a, 0x94, 0xc7, 0xc6, 0x94, 0xe8, 0x0d, 0x28, 0x3a, 0x98, 0xd6, 0x57, 0x9e, - 0xe4, 0xf8, 0xbc, 0x46, 0x24, 0x98, 0x04, 0x03, 0x5d, 0x87, 0xf2, 0x9e, 0xe5, 0x79, 0x34, 0xeb, - 0xb5, 0x2d, 0x9f, 0xf0, 0xc1, 0x87, 0xcc, 0x6c, 0xe3, 0x2c, 0xb4, 0x3a, 0x3e, 0x1d, 0x78, 0xeb, - 0x84, 0x23, 0x53, 0x36, 0x01, 0x69, 0x5b, 0x7d, 0x3c, 0x9e, 0x83, 0x93, 0x01, 0x9d, 0x79, 0x8e, - 0x80, 0xa6, 0xce, 0x94, 0x2b, 0xd0, 0xd3, 0xe7, 0x5b, 0x7d, 0x3c, 0xb6, 0x5f, 0x46, 0xa1, 0x6e, - 0xc6, 0xbe, 0xc3, 0x5c, 0x5a, 0x32, 0xe8, 0xcf, 0x15, 0x35, 0xaf, 0x68, 0x69, 0xfd, 0xcf, 0x29, - 0x28, 0x8f, 0x8d, 0xf0, 0x4e, 0xec, 0xba, 0x2b, 0x50, 0xa2, 0xc8, 0xe6, 0xc0, 0x8a, 0x22, 0x1c, - 0xf0, 0x9c, 0x10, 0x0b, 0x52, 0xce, 0x1d, 0xce, 0x40, 0x9f, 0x40, 0x8e, 0x0c, 0x64, 0xc3, 0x78, - 0x38, 0x87, 0xcb, 0x93, 0xd2, 0xe8, 0x6e, 0x76, 0xb8, 0x90, 0x9c, 0x83, 0x08, 0x9d, 0x51, 0xe5, - 0x61, 0x83, 0x29, 0xf5, 0xa9, 0xca, 0xc3, 0x46, 0x53, 0x3f, 0x51, 0x00, 0xba, 0xbb, 0x38, 0xb2, - 0x77, 0xd8, 0x1e, 0x3e, 0x87, 0x62, 0xc8, 0x9e, 0xcc, 0x44, 0xed, 0x9e, 0xf4, 0x8d, 0x84, 0x09, - 0x27, 0x4a, 0x36, 0x84, 0x31, 0x05, 0x55, 0x47, 0x57, 0x38, 0x7e, 0x67, 0x8e, 0xc7, 0x69, 0x6f, - 0x03, 0xea, 0x61, 0x1f, 0x07, 0x56, 0x84, 0xcd, 0x1d, 0x37, 0x8c, 0x48, 0x2f, 0xb0, 0xfa, 0x63, - 0xa3, 0xa5, 0xb3, 0x92, 0x7f, 0x5b, 0xb2, 0xd1, 0xfb, 0xf0, 0x42, 0x2c, 0x6b, 0xf6, 0xad, 0x07, - 0xe6, 0xd6, 0xd0, 0xde, 0xc5, 0x11, 0xdf, 0x9a, 0xbc, 0xfb, 0x9f, 0x8b, 0x45, 0xd6, 0xac, 0x07, - 0x75, 0x2e, 0x80, 0x2e, 0x41, 0x21, 0x8c, 0xac, 0xc8, 0x64, 0xaf, 0x38, 0x93, 0x70, 0x77, 0x9e, - 0x92, 0x69, 0x00, 0xe8, 0x3f, 0x82, 0x62, 0x62, 0xd4, 0x8e, 0x96, 0x20, 0xcf, 0x37, 0x12, 0xdf, - 0xdc, 0xa7, 0x3a, 0x21, 0x51, 0x8d, 0x62, 0xdd, 0xa3, 0x92, 0x74, 0xf9, 0xe9, 0x24, 0xad, 0xff, - 0x4b, 0x81, 0xf3, 0x47, 0xcd, 0xe7, 0xbf, 0x67, 0x3b, 0xd0, 0x8f, 0x01, 0xf1, 0x27, 0x79, 0x17, - 0x4a, 0x4c, 0x0a, 0xbe, 0x38, 0x78, 0x3c, 0x27, 0x3e, 0x17, 0x88, 0xdb, 0x50, 0xab, 0x19, 0x3e, - 0x79, 0x3c, 0xf7, 0xc1, 0x89, 0xaa, 0x6a, 0xe2, 0x63, 0xfc, 0xbc, 0xd4, 0x36, 0xb4, 0x70, 0x0c, - 0xce, 0x09, 0x91, 0x05, 0x79, 0x96, 0x8e, 0x69, 0x5d, 0xe5, 0xaf, 0x75, 0x49, 0x7e, 0x1e, 0x66, - 0x69, 0xb1, 0xd5, 0x3c, 0x71, 0x1d, 0x4f, 0xae, 0x48, 0xeb, 0x38, 0xc3, 0x6d, 0x39, 0xfa, 0xbf, - 0x55, 0x78, 0x65, 0xc2, 0x87, 0x0a, 0xf4, 0xf5, 0xa1, 0x91, 0xcd, 0x47, 0xcf, 0xf4, 0xbd, 0x83, - 0x67, 0xf1, 0x43, 0x73, 0x9c, 0xc4, 0x18, 0x55, 0x39, 0x6a, 0x8c, 0x3a, 0x3e, 0xff, 0x4c, 0x1f, - 0x3d, 0xff, 0xfc, 0x81, 0x67, 0x34, 0xb3, 0x7f, 0x54, 0x20, 0xc3, 0x36, 0x87, 0x3e, 0x03, 0xd5, - 0xc1, 0xa1, 0xfd, 0x4c, 0x43, 0x55, 0xa6, 0x79, 0x92, 0x99, 0xaa, 0xfc, 0xaf, 0x83, 0xf4, 0xf3, - 0xfc, 0xd7, 0x41, 0x13, 0xf2, 0xf1, 0x14, 0x47, 0x3d, 0xe5, 0x14, 0x27, 0xd6, 0x1c, 0x5d, 0x22, - 0x32, 0xcf, 0x73, 0x89, 0xd0, 0xff, 0x9a, 0x82, 0xca, 0xf8, 0x47, 0x16, 0xf4, 0x19, 0x64, 0xf8, - 0x7f, 0x00, 0xa4, 0x4e, 0xfd, 0x1f, 0x00, 0x5c, 0x11, 0xad, 0x43, 0x9c, 0x31, 0x9d, 0x29, 0x93, - 0xc2, 0xf1, 0x63, 0x98, 0x78, 0xbf, 0x5a, 0x8c, 0x20, 0x47, 0x33, 0x57, 0x40, 0xf3, 0x87, 0x7d, - 0x36, 0x4d, 0x30, 0x07, 0x38, 0x30, 0x7b, 0xd8, 0xe7, 0xd9, 0xc0, 0x28, 0xfb, 0xc3, 0x7e, 0x83, - 0x78, 0xe1, 0x1d, 0x1c, 0x2c, 0x63, 0xff, 0xba, 0x2e, 0xab, 0x08, 0x4b, 0xfc, 0xe7, 0x41, 0xbb, - 0xbd, 0xba, 0x6a, 0xde, 0x59, 0xdd, 0xe8, 0xf2, 0x3f, 0x9b, 0x37, 0xb5, 0x33, 0xf5, 0xcb, 0x8f, - 0xfe, 0x79, 0xe1, 0xcc, 0xa3, 0x83, 0x0b, 0xa9, 0xbf, 0x1c, 0x5c, 0x48, 0x7d, 0x7b, 0x70, 0x21, - 0xf5, 0x8f, 0x83, 0x0b, 0xa9, 0x9f, 0x7d, 0x77, 0xe1, 0xcc, 0x37, 0x30, 0xda, 0xe0, 0x7f, 0x02, - 0x00, 0x00, 0xff, 0xff, 0x42, 0xdf, 0xeb, 0xd9, 0x88, 0x24, 0x00, 0x00, + // 3609 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, 0x24, 0x41, 0x6e, 0x41, 0x2e, 0x73, 0xce, + 0x25, 0x87, 0x5c, 0x12, 0x20, 0x40, 0x0e, 0x3e, 0x26, 0xb7, 0x45, 0x12, 0x18, 0x89, 0xf7, 0x94, + 0xfc, 0x03, 0x01, 0x7c, 0x08, 0x16, 0xef, 0xab, 0xd9, 0xd4, 0x37, 0xed, 0xc5, 0xee, 0x45, 0x60, + 0xd7, 0xc7, 0xaf, 0xeb, 0xd5, 0xab, 0x57, 0x55, 0xaf, 0x5a, 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, 0x3f, 0xd7, 0xe8, 0x2f, 0x4e, 0xd5, 0xff, 0x2c, 0x09, 0xc5, 0x47, 0xd2, 0x59, 0xbd, 0x09, + 0x1e, 0xa2, 0x1a, 0xa4, 0x1c, 0x6f, 0x32, 0x0d, 0xcb, 0x89, 0x1b, 0xc9, 0x3b, 0xf9, 0xf5, 0xdb, + 0xab, 0xa7, 0x79, 0x6e, 0xb5, 0x45, 0xc5, 0x7a, 0x87, 0xde, 0x90, 0xea, 0x55, 0xd5, 0x97, 0xaf, + 0x56, 0x2e, 0x19, 0x5c, 0x17, 0x35, 0x41, 0x1d, 0x12, 0x1f, 0x97, 0x95, 0x1b, 0x89, 0x3b, 0xf9, + 0xf5, 0x8f, 0x4f, 0xc7, 0x88, 0xde, 0x5d, 0x23, 0x3e, 0x1e, 0x78, 0x0e, 0xf1, 0x04, 0x10, 0xd3, + 0x47, 0x0f, 0x20, 0x4d, 0xa6, 0x21, 0xb5, 0x26, 0xc9, 0xac, 0xb9, 0x77, 0x3a, 0x52, 0x97, 0xc9, + 0x19, 0x64, 0x1a, 0x62, 0x3f, 0x66, 0x90, 0xd0, 0x47, 0x35, 0x50, 0x27, 0x24, 0x08, 0xcb, 0x2a, + 0xb3, 0xe8, 0xee, 0x19, 0x16, 0x91, 0x20, 0x14, 0x56, 0xc5, 0x60, 0x98, 0x32, 0xba, 0x07, 0xd9, + 0x20, 0xb4, 0x46, 0xd8, 0x74, 0xec, 0x72, 0xea, 0x46, 0xe2, 0x4e, 0xaa, 0xba, 0x44, 0xb9, 0xaf, + 0x5f, 0xad, 0x64, 0x7a, 0x94, 0xde, 0xaa, 0x1b, 0x19, 0x26, 0xd0, 0xb2, 0xd1, 0x67, 0x50, 0x88, + 0xa2, 0x90, 0xca, 0xa7, 0x99, 0xfc, 0x15, 0x21, 0x9f, 0x8f, 0x16, 0xde, 0xaa, 0x1b, 0xf9, 0x48, + 0xb0, 0x65, 0xeb, 0x7f, 0xa7, 0xc0, 0xd2, 0x11, 0x1b, 0x50, 0x15, 0xd2, 0xbb, 0x8e, 0x1b, 0x62, + 0xbf, 0x9c, 0x60, 0xe6, 0xdf, 0x3a, 0xdd, 0xfc, 0xc6, 0xf3, 0x89, 0x8f, 0x83, 0x60, 0xe6, 0x48, + 0xa1, 0x89, 0x6e, 0x01, 0x4c, 0x7c, 0xf2, 0x2d, 0x1e, 0x86, 0x0e, 0xf1, 0xd8, 0xc6, 0x64, 0x85, + 0x44, 0x8c, 0x8e, 0xee, 0x42, 0x89, 0x3b, 0xcc, 0x1c, 0x12, 0x77, 0x3a, 0xf6, 0x02, 0xe6, 0xf8, + 0x62, 0x55, 0xd1, 0x12, 0x46, 0x91, 0x73, 0x6a, 0x9c, 0x81, 0xb6, 0xa0, 0xe0, 0x63, 0xcf, 0xc6, + 0xbe, 0x89, 0x9f, 0x4f, 0xfc, 0xa0, 0xac, 0xb2, 0x1d, 0x5a, 0xc4, 0xb4, 0x3c, 0xd7, 0xa7, 0xf4, + 0x00, 0x7d, 0x08, 0x69, 0xb2, 0xbb, 0x1b, 0xe0, 0x90, 0x79, 0x56, 0x8d, 0xb6, 0x8f, 0xd1, 0xd0, + 0x32, 0xa4, 0x5c, 0x67, 0xec, 0x84, 0xcc, 0x8d, 0x92, 0xc9, 0x49, 0xfa, 0x9b, 0x02, 0xa0, 0xe3, + 0x71, 0x84, 0xbe, 0x00, 0xd5, 0x23, 0x64, 0x22, 0x5c, 0xf6, 0xd1, 0xe9, 0x76, 0x75, 0x08, 0x99, + 0x50, 0x35, 0xea, 0x6a, 0x83, 0xe9, 0xa0, 0x87, 0x90, 0x0f, 0xad, 0x1d, 0x17, 0x1b, 0xd8, 0xb2, + 0xb1, 0x2f, 0xc2, 0xf8, 0x8c, 0xa0, 0xe9, 0xcf, 0x84, 0x19, 0x4a, 0x5c, 0x1b, 0x3d, 0x00, 0xa0, + 0x27, 0x58, 0x60, 0x25, 0x19, 0xd6, 0x9d, 0xd3, 0xb1, 0x36, 0x23, 0x59, 0x06, 0x15, 0xd3, 0x45, + 0x3f, 0x87, 0x74, 0x40, 0x7c, 0x1a, 0x07, 0xea, 0x79, 0x71, 0xd0, 0x63, 0x72, 0x0c, 0x41, 0xe8, + 0x50, 0x3b, 0xac, 0xd1, 0xc8, 0xc7, 0x23, 0x2b, 0x24, 0x3e, 0xf3, 0xf2, 0x99, 0x76, 0x54, 0x22, + 0x59, 0x6e, 0xc7, 0x4c, 0x17, 0x55, 0x21, 0x4b, 0x05, 0x1d, 0x6f, 0x18, 0x96, 0x33, 0xe7, 0xb9, + 0xb7, 0x2e, 0x24, 0x19, 0x4a, 0xa4, 0x47, 0x5d, 0x3c, 0xc6, 0xfe, 0x08, 0xd3, 0xe5, 0x62, 0xbf, + 0x9c, 0x3d, 0xcf, 0xc5, 0x5b, 0x33, 0x61, 0xee, 0xe2, 0x98, 0x36, 0x5d, 0xda, 0x9e, 0x15, 0xec, + 0x09, 0xac, 0xdc, 0x79, 0x4b, 0x7b, 0x10, 0xc9, 0xf2, 0xa5, 0xcd, 0x74, 0xd1, 0x57, 0x90, 0x7e, + 0x66, 0xb9, 0x53, 0x1c, 0x94, 0xe1, 0x3c, 0x94, 0x6d, 0x26, 0x17, 0x45, 0x8e, 0xd0, 0xa3, 0xb6, + 0xec, 0x58, 0xc3, 0xfd, 0x5d, 0xc7, 0x75, 0xb1, 0x5f, 0xce, 0x9f, 0x87, 0x52, 0x8d, 0x64, 0xb9, + 0x2d, 0x33, 0x5d, 0xd4, 0x06, 0xf0, 0xb1, 0x65, 0xb7, 0xc6, 0x13, 0xe2, 0x87, 0xe5, 0xe2, 0x79, + 0xb9, 0xd4, 0x88, 0x64, 0xeb, 0x56, 0x68, 0x71, 0xb4, 0x99, 0x3e, 0x6a, 0x40, 0xae, 0xd7, 0xeb, + 0x3f, 0xf1, 0x1d, 0x1a, 0x3f, 0x25, 0x06, 0x76, 0x46, 0x72, 0x8f, 0x44, 0x19, 0xce, 0x4c, 0x13, + 0x7d, 0x09, 0x99, 0x9e, 0x35, 0x9e, 0xd0, 0xb5, 0x2d, 0x31, 0x90, 0x1f, 0x9c, 0x01, 0xc2, 0x05, + 0x19, 0x84, 0xd4, 0x42, 0xdf, 0x80, 0xc6, 0x7f, 0xce, 0x02, 0xac, 0xac, 0x31, 0xa4, 0xd5, 0xf3, + 0x90, 0x8e, 0x84, 0xe4, 0x31, 0x1c, 0x44, 0xe0, 0x9a, 0xe3, 0x85, 0xd8, 0x77, 0xb1, 0xf5, 0x0c, + 0xdb, 0xfc, 0xd4, 0x88, 0xa0, 0xb8, 0xcc, 0x5e, 0xf1, 0x93, 0xb3, 0xca, 0xd9, 0x89, 0x8a, 0xec, + 0x4d, 0xa7, 0xa1, 0xa2, 0x3f, 0x02, 0x34, 0xc6, 0xa1, 0x45, 0x2b, 0x70, 0x1f, 0x07, 0x61, 0x8f, + 0x25, 0xb4, 0x32, 0x62, 0xef, 0xfa, 0xe4, 0xac, 0x60, 0x3e, 0xaa, 0xc3, 0x5e, 0x73, 0x02, 0x16, + 0xda, 0x85, 0xab, 0x71, 0xaa, 0x81, 0x87, 0xd8, 0x79, 0x86, 0xfd, 0xf2, 0x15, 0xf6, 0x8e, 0xf5, + 0x8b, 0xbd, 0x43, 0x6a, 0xb1, 0xb7, 0x9c, 0x88, 0x47, 0xc3, 0xa3, 0xd6, 0xdb, 0x16, 0xe1, 0x71, + 0xf5, 0xbc, 0xf0, 0x88, 0x44, 0x79, 0x78, 0x44, 0x8f, 0xa8, 0x03, 0x85, 0x17, 0xce, 0xe8, 0x85, + 0x35, 0x12, 0x6e, 0x7f, 0x8f, 0x21, 0x9d, 0x51, 0xb7, 0xbf, 0x89, 0x49, 0x33, 0xb0, 0x39, 0x7d, + 0x7a, 0x9a, 0x44, 0x79, 0xea, 0xe1, 0xb0, 0xfc, 0xfe, 0x79, 0xa7, 0xe9, 0x51, 0x24, 0xcb, 0xe3, + 0x7f, 0xa6, 0x4b, 0x93, 0xd6, 0x81, 0xe3, 0xd9, 0xe4, 0x00, 0xfb, 0xe5, 0x6b, 0xe7, 0x25, 0xad, + 0x27, 0x42, 0x92, 0x27, 0x2d, 0xa9, 0xf7, 0x85, 0xfa, 0xf2, 0x6f, 0x57, 0x12, 0x9b, 0x6a, 0x36, + 0xad, 0x65, 0x36, 0xd5, 0x6c, 0x41, 0x2b, 0xea, 0x25, 0x28, 0xc4, 0xeb, 0x87, 0xfe, 0x63, 0x78, + 0xff, 0xe4, 0xcd, 0x45, 0xcb, 0xa0, 0x38, 0x36, 0xab, 0x46, 0xb9, 0x2a, 0x88, 0x36, 0x40, 0x69, + 0xd5, 0x0d, 0xc5, 0xb1, 0xf5, 0x07, 0x50, 0x3e, 0x6d, 0xbb, 0xd0, 0xc7, 0x00, 0x01, 0xaf, 0xb3, + 0x8e, 0x1d, 0xb0, 0xae, 0x2c, 0x57, 0x2d, 0xbe, 0x7e, 0xb5, 0x92, 0xe3, 0xd8, 0xad, 0x7a, 0x60, + 0xe4, 0xb8, 0x40, 0xcb, 0x0e, 0xf4, 0xbf, 0x48, 0x40, 0x69, 0x3e, 0x31, 0xa1, 0x1a, 0x64, 0x64, + 0x31, 0xe7, 0x3d, 0xdd, 0xf7, 0xcf, 0x48, 0xd6, 0xb4, 0x07, 0x6d, 0x79, 0xbb, 0x44, 0x94, 0x58, + 0xa9, 0x89, 0x3e, 0x80, 0x9c, 0x6f, 0x1d, 0x98, 0x3b, 0x87, 0x21, 0x0e, 0xca, 0xca, 0x8d, 0xe4, + 0x9d, 0x82, 0x91, 0xf5, 0xad, 0x83, 0x2a, 0x7d, 0x46, 0x2b, 0x90, 0xf5, 0xa6, 0x63, 0xd3, 0x27, + 0x07, 0x01, 0xab, 0x6f, 0xb2, 0x40, 0x67, 0xbc, 0xe9, 0xd8, 0x20, 0x07, 0x81, 0x5e, 0x87, 0xa5, + 0xb9, 0x12, 0x69, 0x79, 0xe8, 0x3e, 0xa8, 0xc1, 0xc4, 0xf2, 0x44, 0x79, 0xbe, 0x16, 0x33, 0x49, + 0xf4, 0xd4, 0xab, 0x54, 0x4c, 0xb6, 0x5f, 0x54, 0x54, 0xff, 0x4e, 0x39, 0x02, 0xc3, 0x5a, 0xa3, + 0x14, 0xab, 0xb5, 0xa7, 0x94, 0x79, 0xd1, 0x42, 0xf3, 0x02, 0x5d, 0xc7, 0xc1, 0xd0, 0x77, 0x26, + 0x21, 0xf1, 0x65, 0x03, 0xc1, 0x54, 0xd1, 0x4d, 0xc8, 0x39, 0x9e, 0x8d, 0x9f, 0x9b, 0x8e, 0xfd, + 0x9c, 0xd5, 0xfa, 0xa2, 0xe0, 0x67, 0x19, 0xb9, 0x65, 0x3f, 0x47, 0xd7, 0x21, 0xe3, 0xe3, 0x67, + 0xd8, 0x0f, 0x30, 0x5b, 0xa0, 0x6c, 0x9d, 0x24, 0x11, 0x35, 0x20, 0x45, 0x4d, 0x94, 0x5d, 0xd0, + 0x45, 0x5b, 0x85, 0x68, 0x81, 0x5c, 0x1b, 0x7d, 0x1f, 0x80, 0xf5, 0x34, 0xe6, 0x9e, 0xe3, 0xf1, + 0x46, 0x28, 0x29, 0x04, 0x72, 0x8c, 0xfe, 0xc0, 0xf1, 0x42, 0xea, 0x6d, 0x27, 0x30, 0x87, 0x7b, + 0x78, 0xb8, 0xcf, 0xda, 0xa1, 0xc8, 0x18, 0x27, 0xa8, 0x51, 0xa2, 0xfe, 0x46, 0x81, 0xd2, 0x7c, + 0x17, 0xf1, 0xbb, 0x72, 0xd3, 0x5d, 0x28, 0xb9, 0x84, 0xec, 0x4f, 0x27, 0x27, 0xb5, 0x8f, 0x9c, + 0x23, 0xdb, 0xc7, 0x1a, 0x64, 0x88, 0xc7, 0x5a, 0xc7, 0xf3, 0x9b, 0x99, 0xe3, 0x4d, 0x2d, 0xf1, + 0x28, 0x0d, 0x6d, 0xc3, 0x65, 0x6e, 0x12, 0x6f, 0x72, 0x39, 0x5c, 0x6a, 0x61, 0xb8, 0x25, 0x06, + 0xd2, 0x64, 0x18, 0x0c, 0xf7, 0xa7, 0xa0, 0xd2, 0xfb, 0x16, 0x73, 0x6f, 0x69, 0x7d, 0xe5, 0x14, + 0x6f, 0x51, 0x1f, 0xf7, 0x0f, 0x27, 0x58, 0x06, 0x29, 0x55, 0xa1, 0x07, 0x10, 0x66, 0xcd, 0x17, + 0x7a, 0x0c, 0x4b, 0xa2, 0xa1, 0x26, 0xbe, 0x8d, 0x7d, 0xc7, 0x1b, 0x89, 0x2d, 0xd0, 0xcf, 0xb8, + 0xca, 0x08, 0x49, 0x81, 0x2b, 0x3a, 0x72, 0x49, 0x45, 0xeb, 0x80, 0x24, 0x96, 0x39, 0xb6, 0xc2, + 0xe1, 0x9e, 0xe9, 0x62, 0x6f, 0x6e, 0x43, 0x34, 0xc9, 0xdf, 0xa2, 0xec, 0x36, 0xf6, 0xf4, 0x1d, + 0x28, 0xc4, 0xfb, 0x30, 0x74, 0x1b, 0x96, 0x98, 0x0c, 0xb6, 0xcd, 0x78, 0x6e, 0x28, 0x1a, 0x25, + 0x41, 0x96, 0xdb, 0x74, 0x17, 0x34, 0xd9, 0xb2, 0x45, 0x92, 0x0a, 0x93, 0x5c, 0x92, 0x74, 0x21, + 0xaa, 0xff, 0xb3, 0x02, 0xda, 0xd1, 0x6c, 0x8e, 0xea, 0x90, 0x66, 0xd1, 0x23, 0x73, 0xcf, 0x62, + 0x91, 0x27, 0x74, 0x51, 0x13, 0x00, 0x3f, 0x9d, 0x7b, 0x7f, 0x7e, 0xfd, 0xe6, 0x19, 0xd5, 0x89, + 0x0b, 0xca, 0xa3, 0x83, 0x9f, 0xca, 0xd5, 0xac, 0xcc, 0x42, 0x38, 0x1e, 0x9a, 0x32, 0x80, 0x7f, + 0x4b, 0x51, 0x29, 0xa3, 0x27, 0xb5, 0x78, 0xf4, 0xdc, 0x86, 0x8c, 0xb4, 0xf5, 0xc3, 0xf9, 0xb4, + 0xcd, 0x2d, 0x95, 0x24, 0xfd, 0x3f, 0x15, 0x58, 0x3a, 0xd2, 0x12, 0xa3, 0x2d, 0x28, 0xba, 0x78, + 0xf7, 0x1d, 0x22, 0xad, 0x40, 0xd5, 0xa3, 0x38, 0xeb, 0x42, 0xc9, 0x77, 0x46, 0x7b, 0x31, 0x3c, + 0x65, 0x41, 0xbc, 0x22, 0xd3, 0x8f, 0x00, 0x63, 0xce, 0x4d, 0xbd, 0xb3, 0x73, 0x17, 0x3f, 0x9a, + 0xe8, 0x2e, 0x14, 0xbd, 0xa9, 0xeb, 0x9a, 0xf8, 0xe9, 0xd4, 0x72, 0x9d, 0xf0, 0x90, 0xdd, 0x5d, + 0x64, 0xf6, 0x2c, 0x50, 0x56, 0x43, 0x70, 0xf4, 0xbf, 0x56, 0xa0, 0x34, 0x7f, 0x4b, 0x40, 0xf7, + 0x60, 0x89, 0x79, 0x37, 0x16, 0x88, 0x89, 0x58, 0x72, 0xc3, 0xbb, 0x61, 0x23, 0x8a, 0xb3, 0x8f, + 0x41, 0xe3, 0xae, 0x3b, 0x12, 0xb5, 0x5c, 0x98, 0xbb, 0x75, 0x26, 0xfd, 0xfb, 0xf6, 0xcb, 0x0f, + 0xa1, 0xc4, 0x2e, 0x53, 0xb3, 0x5c, 0x10, 0x77, 0x4c, 0x91, 0xf3, 0xe4, 0x29, 0xff, 0xcb, 0x34, + 0x94, 0xe6, 0xfb, 0x70, 0x74, 0x13, 0x60, 0xe4, 0x13, 0x9e, 0xf4, 0xe3, 0xeb, 0xcc, 0x31, 0x6a, + 0x8d, 0xb8, 0x01, 0xfa, 0x43, 0x28, 0xc8, 0xfb, 0xa3, 0x43, 0x44, 0x59, 0xc8, 0xaf, 0xff, 0xf8, + 0xa2, 0xb7, 0xcf, 0xe8, 0x71, 0xb6, 0xee, 0x39, 0x3c, 0xf4, 0x89, 0xc8, 0x89, 0xd8, 0x36, 0x63, + 0xa6, 0xa8, 0x91, 0x29, 0x9a, 0xe0, 0x6e, 0x44, 0x16, 0x6d, 0xcc, 0x1d, 0xd2, 0x1f, 0x5d, 0xd8, + 0x92, 0xa3, 0xde, 0x5b, 0xfe, 0xc7, 0x04, 0xe4, 0x63, 0xe6, 0x51, 0xe0, 0xdd, 0xa9, 0x37, 0x64, + 0x87, 0x6f, 0x11, 0xe0, 0xe6, 0xd4, 0x8b, 0xa6, 0x4d, 0x14, 0x00, 0xdd, 0x88, 0xdd, 0xb2, 0xe3, + 0xf3, 0x9a, 0xd9, 0x1d, 0xfa, 0x16, 0x94, 0x44, 0xe1, 0x1b, 0x12, 0x97, 0x95, 0x65, 0x9a, 0xb4, + 0x8a, 0x46, 0x81, 0x53, 0x6b, 0xc4, 0xa5, 0x45, 0xf9, 0x1a, 0x4b, 0x24, 0x8c, 0x9d, 0x62, 0x99, + 0x3b, 0x3d, 0x64, 0x8c, 0x4d, 0x35, 0x9b, 0xd4, 0x54, 0xfd, 0xaf, 0x14, 0x50, 0xe9, 0xbb, 0x91, + 0x06, 0x85, 0x4a, 0xe7, 0x6b, 0xb3, 0xd3, 0xed, 0x9b, 0x9d, 0x41, 0xbb, 0xad, 0x5d, 0x42, 0x19, + 0x48, 0x56, 0xb6, 0x37, 0xb4, 0x04, 0x2a, 0x40, 0xb6, 0xda, 0xed, 0xb6, 0xcd, 0x4a, 0xa7, 0xae, + 0x29, 0x28, 0x0f, 0x19, 0xf6, 0xd4, 0x35, 0xb4, 0x24, 0x2a, 0x01, 0xd4, 0xba, 0x9d, 0x5a, 0xa5, + 0x6f, 0x56, 0x36, 0x36, 0x34, 0x15, 0xe5, 0x20, 0x55, 0xeb, 0x0e, 0x3a, 0x7d, 0x2d, 0x45, 0xd5, + 0xb7, 0x2a, 0xbf, 0xd4, 0x32, 0xec, 0x47, 0xab, 0xa3, 0x65, 0x11, 0x40, 0xba, 0xd7, 0xaf, 0xd7, + 0x1b, 0xdb, 0x5a, 0x8e, 0x12, 0x7b, 0x83, 0x2d, 0x0d, 0x28, 0x5c, 0x6f, 0xb0, 0x65, 0xb6, 0x3a, + 0x7d, 0x2d, 0x4f, 0xdf, 0xb4, 0x5d, 0x31, 0x5a, 0x95, 0x4e, 0xad, 0xa1, 0x15, 0x28, 0xeb, 0x97, + 0x5d, 0x83, 0x21, 0x17, 0xf9, 0x9b, 0x06, 0x9d, 0xbe, 0x69, 0x74, 0x9f, 0xf4, 0xb4, 0x12, 0xd3, + 0x7b, 0x6c, 0xd4, 0x5b, 0xcd, 0xa6, 0xb6, 0x84, 0x10, 0x94, 0x9a, 0xad, 0x4e, 0xa5, 0x6d, 0x46, + 0xda, 0x1a, 0x5d, 0x10, 0xa7, 0x89, 0x77, 0x5e, 0x46, 0x45, 0xc8, 0x55, 0x0c, 0xa3, 0xf2, 0x35, + 0x43, 0x44, 0xf4, 0x65, 0x9b, 0xbd, 0x6e, 0x87, 0x3d, 0x5d, 0xa1, 0x4c, 0xfa, 0x54, 0x65, 0x8f, + 0x57, 0xf5, 0x8f, 0x41, 0xa5, 0x7b, 0x8d, 0xb2, 0xa0, 0x56, 0x06, 0xfd, 0xae, 0x76, 0x89, 0x59, + 0x5f, 0xab, 0xb4, 0x2b, 0x86, 0x96, 0xa0, 0xc6, 0x74, 0xba, 0x1d, 0x53, 0x3c, 0x2b, 0xfa, 0x9b, + 0x24, 0x94, 0xe6, 0xaf, 0xf2, 0x51, 0x84, 0x9d, 0x1b, 0x08, 0xf3, 0x7a, 0xc7, 0x22, 0x6c, 0xd6, + 0xbc, 0x29, 0x6f, 0xdf, 0xbc, 0x45, 0x0d, 0x6a, 0xf2, 0x9d, 0x1a, 0xd4, 0xfb, 0x90, 0xb5, 0xa7, + 0x3e, 0x0b, 0x74, 0x16, 0x6b, 0xc9, 0xea, 0x7b, 0x94, 0xfd, 0xe6, 0xd5, 0x4a, 0x31, 0x74, 0xc6, + 0x78, 0xb5, 0x2e, 0x98, 0x46, 0x24, 0x46, 0x7b, 0xda, 0xe1, 0xde, 0xd4, 0xdb, 0x37, 0x03, 0xe7, + 0x05, 0x9e, 0xef, 0x69, 0x19, 0xbd, 0xe7, 0xbc, 0xc0, 0xa8, 0x0b, 0x05, 0x12, 0xee, 0x61, 0xdf, + 0x14, 0xcd, 0x42, 0xfa, 0x2d, 0x9a, 0x85, 0x3c, 0x43, 0xe8, 0xf3, 0x8e, 0xe1, 0x4b, 0xc8, 0xfa, + 0xd8, 0xb2, 0x2b, 0x41, 0x77, 0x57, 0x8c, 0xa8, 0xfe, 0x20, 0x06, 0x36, 0x0d, 0x1d, 0x77, 0x75, + 0xcf, 0x1d, 0xae, 0xf6, 0xe5, 0x94, 0x5c, 0x9e, 0x2d, 0xa9, 0xa4, 0xdf, 0x13, 0xdb, 0x9f, 0x87, + 0x4c, 0xcb, 0x7b, 0x66, 0xb9, 0x8e, 0xcd, 0x23, 0x80, 0xe7, 0x41, 0x2d, 0x41, 0x03, 0xbd, 0x45, + 0xdb, 0x06, 0x4d, 0xd1, 0x7f, 0x9d, 0x80, 0x6c, 0xd3, 0x25, 0x07, 0x6c, 0xdb, 0xef, 0x43, 0x66, + 0xd7, 0x25, 0x07, 0xa6, 0xb8, 0xec, 0x15, 0xaa, 0x65, 0x8a, 0xfc, 0xef, 0xaf, 0x56, 0xd2, 0x54, + 0xa4, 0x55, 0x7f, 0x1d, 0xfd, 0x32, 0xd2, 0x54, 0xb0, 0x65, 0xa3, 0x2d, 0x76, 0xc9, 0x15, 0x5f, + 0x2c, 0x44, 0x7b, 0x73, 0xfb, 0x02, 0x43, 0xf3, 0xd8, 0x80, 0x3a, 0x06, 0x80, 0x06, 0x90, 0x19, + 0x59, 0x21, 0x3e, 0xb0, 0x0e, 0xd9, 0x65, 0x25, 0x55, 0xfd, 0x99, 0xd8, 0xa3, 0x4f, 0x47, 0x4e, + 0xb8, 0x37, 0xdd, 0x59, 0x1d, 0x92, 0xf1, 0x5a, 0x84, 0x6e, 0xef, 0xcc, 0x7e, 0xaf, 0x4d, 0xf6, + 0x47, 0x6b, 0xf2, 0xfe, 0xd5, 0x21, 0x36, 0x9b, 0x68, 0x0b, 0x2c, 0xfd, 0x00, 0xf2, 0x9b, 0x64, + 0xe7, 0x91, 0x4f, 0x46, 0xb4, 0x02, 0xa1, 0x5b, 0x90, 0xfe, 0x96, 0xec, 0xc8, 0x65, 0x26, 0xab, + 0x45, 0x71, 0xa7, 0x4d, 0x6d, 0x92, 0x9d, 0x56, 0xdd, 0x48, 0x7d, 0x4b, 0x76, 0x5a, 0x36, 0xba, + 0x03, 0x85, 0x21, 0xf1, 0x42, 0xdf, 0xd9, 0x99, 0x46, 0x83, 0x67, 0x45, 0xa6, 0xf0, 0x38, 0x07, + 0x95, 0x41, 0x0d, 0x5c, 0x12, 0x0a, 0x93, 0xe5, 0xbd, 0xcf, 0x25, 0xa1, 0xfe, 0x2b, 0x15, 0xd0, + 0xf1, 0xe1, 0x16, 0xfa, 0x01, 0xe4, 0x03, 0x36, 0x00, 0xe2, 0x91, 0xa5, 0xc4, 0xf4, 0x80, 0x33, + 0x58, 0x68, 0x6d, 0x40, 0x76, 0x22, 0x6c, 0x66, 0xc5, 0xf1, 0xcc, 0x89, 0x55, 0x6c, 0x81, 0x32, + 0x22, 0xa4, 0x32, 0xda, 0x80, 0xe4, 0xd4, 0x77, 0xca, 0x19, 0xb6, 0x3d, 0x3f, 0x59, 0x64, 0x0e, + 0xb7, 0x3a, 0xf0, 0x9d, 0x86, 0x17, 0xfa, 0x87, 0x06, 0x45, 0x40, 0xbf, 0x80, 0x34, 0xff, 0x58, + 0x24, 0xa6, 0x9e, 0x2b, 0x27, 0x5c, 0x7e, 0x5b, 0xdd, 0xa6, 0xe3, 0xe2, 0x26, 0x13, 0x8b, 0x26, + 0xf9, 0xec, 0x09, 0x6d, 0x47, 0x2d, 0x75, 0x8e, 0x99, 0xf2, 0xf9, 0x42, 0xa6, 0xf0, 0xf3, 0xc1, + 0xac, 0x61, 0xb8, 0x89, 0xa8, 0xc9, 0xfe, 0x12, 0xbe, 0x17, 0xec, 0x3b, 0x13, 0x73, 0xec, 0x04, + 0x01, 0xbd, 0x5b, 0xec, 0x12, 0x1f, 0x3b, 0x23, 0xcf, 0xdc, 0xc7, 0x87, 0x7c, 0x1a, 0x2a, 0x0b, + 0xd0, 0xfb, 0x54, 0x6c, 0x8b, 0x4b, 0x35, 0xb9, 0xd0, 0x43, 0x7c, 0x18, 0x2c, 0x5b, 0x90, 0x8f, + 0xa1, 0x23, 0x0d, 0x92, 0xfb, 0xf8, 0x90, 0x4f, 0x3c, 0x0c, 0xfa, 0x13, 0xfd, 0x1c, 0x52, 0x6c, + 0x48, 0xba, 0x58, 0x22, 0x33, 0xb8, 0xd2, 0x17, 0xca, 0xe7, 0x89, 0xe5, 0xcf, 0x20, 0x2b, 0x7d, + 0x19, 0xc7, 0x4f, 0x71, 0xfc, 0xab, 0x71, 0xfc, 0x5c, 0x4c, 0x6f, 0x53, 0xcd, 0x26, 0x34, 0x85, + 0x17, 0xbc, 0x4d, 0x35, 0xab, 0x6a, 0xa9, 0x4d, 0x35, 0x9b, 0xd2, 0xd2, 0xfa, 0x3f, 0x28, 0x50, + 0x9c, 0x1b, 0x75, 0xa2, 0x8f, 0x20, 0x6f, 0x63, 0x5a, 0x5f, 0x79, 0x92, 0xe3, 0xf3, 0x1a, 0x91, + 0x60, 0x62, 0x0c, 0x74, 0x0f, 0x8a, 0x07, 0x96, 0xeb, 0xd2, 0xac, 0xd7, 0xb1, 0x3c, 0xc2, 0x07, + 0x1f, 0x32, 0xb3, 0xcd, 0xb3, 0x50, 0x7b, 0x7e, 0x3a, 0xf0, 0xc9, 0x05, 0xc7, 0xae, 0x6c, 0x02, + 0xd2, 0xb1, 0xc6, 0x78, 0x3e, 0x07, 0xc7, 0x03, 0x3a, 0xf5, 0x0e, 0x01, 0x4d, 0x9d, 0x29, 0xdf, + 0x40, 0x4f, 0x9f, 0x67, 0x8d, 0xf1, 0xdc, 0x7a, 0x19, 0x85, 0xba, 0x19, 0x7b, 0x36, 0x73, 0x69, + 0xc1, 0xa0, 0x3f, 0x37, 0xd5, 0xac, 0xa2, 0x25, 0xf5, 0x7f, 0x49, 0x40, 0x71, 0x6e, 0x0c, 0x78, + 0x61, 0xd7, 0xdd, 0x86, 0x02, 0x45, 0x36, 0x27, 0x56, 0x18, 0x62, 0x9f, 0xe7, 0x84, 0x48, 0x90, + 0x72, 0x1e, 0x71, 0x06, 0xfa, 0x05, 0x64, 0xc8, 0x44, 0x36, 0x8c, 0x47, 0x73, 0xb8, 0x3c, 0x29, + 0xb5, 0xde, 0x76, 0x97, 0x0b, 0xc9, 0x39, 0x88, 0xd0, 0x99, 0x55, 0x1e, 0x36, 0x98, 0x52, 0x8f, + 0x55, 0x1e, 0x36, 0x9a, 0xfa, 0x53, 0x05, 0xa0, 0xb7, 0x8f, 0xc3, 0xe1, 0x1e, 0x5b, 0xc3, 0x43, + 0xc8, 0x07, 0xec, 0xc9, 0x8c, 0xd5, 0xee, 0xb3, 0xbe, 0xb3, 0x30, 0xe1, 0x58, 0xc9, 0x86, 0x20, + 0xa2, 0xa0, 0xf2, 0xec, 0x0a, 0xc7, 0xef, 0xcc, 0xd1, 0x38, 0xed, 0x53, 0x40, 0x23, 0xec, 0x61, + 0xdf, 0x0a, 0xb1, 0xb9, 0xe7, 0x04, 0x21, 0x19, 0xf9, 0xd6, 0x78, 0x6e, 0xb4, 0x74, 0x59, 0xf2, + 0x1f, 0x48, 0x36, 0xfa, 0x1c, 0xde, 0x8b, 0x64, 0xcd, 0xb1, 0xf5, 0xdc, 0xdc, 0x99, 0x0e, 0xf7, + 0x71, 0xc8, 0x97, 0x26, 0xef, 0xfe, 0x57, 0x22, 0x91, 0x2d, 0xeb, 0x79, 0x95, 0x0b, 0xa0, 0x9b, + 0x90, 0x0b, 0x42, 0x2b, 0x34, 0xd9, 0x16, 0xa7, 0x62, 0xee, 0xce, 0x52, 0x32, 0x0d, 0x00, 0xfd, + 0x8f, 0x21, 0x1f, 0x1b, 0xd7, 0xa3, 0x26, 0x64, 0xf9, 0x42, 0xa2, 0x9b, 0xfb, 0xb9, 0x4e, 0x88, + 0x55, 0xa3, 0x48, 0xf7, 0xa4, 0x24, 0x5d, 0x3c, 0x9e, 0xa4, 0xf5, 0xff, 0x51, 0xe0, 0xea, 0x49, + 0x33, 0xfe, 0xdf, 0xb1, 0x1d, 0xe8, 0x4f, 0x00, 0xf1, 0x27, 0x79, 0x17, 0x8a, 0x4d, 0x0a, 0x1e, + 0xbf, 0x7e, 0xb5, 0x22, 0x3e, 0x39, 0x88, 0xdb, 0x50, 0xab, 0x1e, 0xbc, 0x79, 0xb5, 0xf2, 0xd3, + 0x0b, 0x55, 0xd5, 0xd8, 0x07, 0xfd, 0x55, 0xa9, 0x6d, 0x68, 0xc1, 0x1c, 0x9c, 0x1d, 0x20, 0x0b, + 0xb2, 0x2c, 0x1d, 0xd3, 0xba, 0xca, 0xb7, 0xb5, 0x29, 0x3f, 0x31, 0xb3, 0xb4, 0xd8, 0xaa, 0x5f, + 0xb8, 0x8e, 0xc7, 0xdf, 0x48, 0xeb, 0x38, 0xc3, 0x6d, 0xd9, 0xfa, 0xff, 0xab, 0xf0, 0xc1, 0x19, + 0x1f, 0x3b, 0xd0, 0xd7, 0x47, 0x46, 0x36, 0x3f, 0x7b, 0xab, 0x6f, 0x26, 0x3c, 0x8b, 0x1f, 0x99, + 0xe3, 0xc4, 0xc6, 0xa8, 0xca, 0x49, 0x63, 0xd4, 0xf9, 0xf9, 0x67, 0xf2, 0xe4, 0xf9, 0xe7, 0xef, + 0x79, 0x46, 0xb3, 0xfc, 0x4f, 0x0a, 0xa4, 0xd8, 0xe2, 0xd0, 0x57, 0xa0, 0xda, 0x38, 0x18, 0xbe, + 0xd5, 0x50, 0x95, 0x69, 0x5e, 0x64, 0xa6, 0x2a, 0xff, 0x73, 0x21, 0xf9, 0x2e, 0xff, 0xb9, 0x50, + 0x87, 0x6c, 0x34, 0xc5, 0x51, 0x17, 0x9c, 0xe2, 0x44, 0x9a, 0xb3, 0x4b, 0x44, 0xea, 0x5d, 0x2e, + 0x11, 0xfa, 0xbf, 0x25, 0xa0, 0x34, 0xff, 0xa1, 0x06, 0x7d, 0x05, 0x29, 0xfe, 0x5f, 0x04, 0x89, + 0x85, 0xff, 0x8b, 0x80, 0x2b, 0xa2, 0x3e, 0x44, 0x19, 0xd3, 0x3e, 0x67, 0x52, 0x38, 0x7f, 0x0c, + 0x63, 0xfb, 0xab, 0x45, 0x08, 0x72, 0x34, 0x73, 0x1b, 0x34, 0x6f, 0x3a, 0x66, 0xd3, 0x04, 0x73, + 0x82, 0x7d, 0x73, 0x84, 0x3d, 0x9e, 0x0d, 0x8c, 0xa2, 0x37, 0x1d, 0xd7, 0x88, 0x1b, 0x3c, 0xc2, + 0xfe, 0x06, 0xf6, 0xf4, 0xff, 0xcd, 0x43, 0x21, 0xfe, 0xd1, 0x08, 0xdd, 0x80, 0xfc, 0xc4, 0xf2, + 0x43, 0x87, 0x8d, 0x2c, 0x0e, 0xc5, 0x74, 0x35, 0x4e, 0x42, 0x3d, 0xc8, 0xf1, 0x0f, 0x4b, 0xcd, + 0xc8, 0xd2, 0xb5, 0x8b, 0x7d, 0x91, 0x12, 0x0f, 0x4d, 0xe9, 0x82, 0x19, 0xce, 0xf2, 0xdf, 0x27, + 0xc4, 0x6d, 0xbe, 0x07, 0x45, 0x39, 0x21, 0xc1, 0xcd, 0xb7, 0x9d, 0x47, 0x18, 0xf3, 0x18, 0xe8, + 0x31, 0x80, 0x78, 0x15, 0x45, 0x54, 0x18, 0xe2, 0xfd, 0xc5, 0x6c, 0xa6, 0xa8, 0x31, 0x10, 0xfe, + 0x49, 0x6d, 0xf9, 0x3f, 0x52, 0x90, 0x6a, 0xfa, 0xb4, 0xdd, 0x78, 0x08, 0xea, 0x98, 0xd8, 0xb2, + 0xf2, 0x5e, 0x14, 0x9c, 0xe9, 0xae, 0x6e, 0x11, 0x3b, 0x3a, 0xaa, 0x14, 0x04, 0x3d, 0x86, 0xf4, + 0x0e, 0x99, 0x7a, 0x76, 0x20, 0x3a, 0xce, 0x4f, 0x17, 0x82, 0xab, 0x32, 0x55, 0x99, 0x38, 0x38, + 0xd0, 0xf2, 0xff, 0x25, 0x20, 0xc5, 0x18, 0xe8, 0x1b, 0xc8, 0x31, 0x5a, 0x7f, 0xd6, 0x28, 0x7c, + 0xb6, 0x38, 0x7e, 0x2c, 0xfc, 0x66, 0x70, 0x34, 0x11, 0x3a, 0x5e, 0x68, 0x8a, 0xff, 0x88, 0x89, + 0x27, 0x86, 0x9c, 0xe3, 0x85, 0x5d, 0xfe, 0x4f, 0x31, 0x37, 0xa1, 0x40, 0x13, 0x92, 0x2d, 0xc5, + 0x92, 0xac, 0x45, 0xcb, 0x33, 0x9a, 0x10, 0xd9, 0x84, 0x3c, 0x67, 0xf2, 0x76, 0x86, 0x1f, 0xfd, + 0x05, 0xbe, 0xff, 0x01, 0xd7, 0xa6, 0x36, 0x2d, 0xff, 0x4d, 0x02, 0xd2, 0xdc, 0x25, 0xa8, 0x03, + 0xa9, 0x20, 0xb4, 0xfc, 0x50, 0x64, 0xbe, 0xf5, 0xc5, 0x97, 0x1d, 0x65, 0x04, 0x0a, 0x83, 0xea, + 0xb3, 0x1e, 0xf3, 0xad, 0xd0, 0x58, 0x5f, 0xaa, 0x7f, 0x00, 0x2a, 0x8d, 0x00, 0x7a, 0x33, 0x37, + 0x2a, 0x9d, 0x8d, 0x86, 0x76, 0x09, 0x65, 0x41, 0x65, 0xd3, 0xa2, 0x84, 0xfe, 0x02, 0x72, 0x91, + 0xbf, 0xd1, 0x35, 0xb8, 0x32, 0xe8, 0x54, 0xbb, 0x83, 0x4e, 0xbd, 0x51, 0x37, 0x1f, 0x19, 0x8d, + 0x5a, 0xa3, 0xde, 0xea, 0x6c, 0x68, 0x97, 0xe6, 0x19, 0xcd, 0x6e, 0xbb, 0xdd, 0x7d, 0x42, 0x19, + 0x09, 0x74, 0x15, 0xb4, 0x6e, 0xb3, 0xd9, 0x6b, 0xf4, 0x63, 0xe2, 0x4a, 0x8c, 0x3a, 0x93, 0x4d, + 0xa2, 0x25, 0xc8, 0xd7, 0x06, 0x86, 0xd1, 0xe0, 0xa3, 0x2a, 0x4d, 0x5d, 0xfe, 0x4e, 0x81, 0xac, + 0x3c, 0xb2, 0xa8, 0x11, 0x9b, 0x0f, 0xe6, 0xd7, 0x7f, 0x78, 0xd1, 0xc5, 0x1e, 0x9d, 0x0e, 0x7e, + 0x04, 0x79, 0xcb, 0x1f, 0xb5, 0xec, 0xe7, 0x3d, 0xb6, 0x11, 0xf1, 0x10, 0x89, 0x33, 0xd0, 0x0d, + 0xc8, 0x5a, 0xfe, 0xa8, 0x46, 0xa6, 0xa2, 0xa0, 0x46, 0x05, 0x46, 0x52, 0x7f, 0x4b, 0xb5, 0xa1, + 0x0a, 0xa9, 0x5d, 0x5f, 0xb6, 0x97, 0x67, 0xfe, 0x9f, 0xca, 0xf1, 0x4d, 0x34, 0xb8, 0xaa, 0xfe, + 0x5d, 0x02, 0x60, 0x96, 0x26, 0x50, 0x09, 0xc0, 0xe8, 0x3e, 0x31, 0x3b, 0x83, 0xad, 0x6a, 0xc3, + 0x10, 0x9b, 0x59, 0xe9, 0x3c, 0xe4, 0xd3, 0xb7, 0x7a, 0xa3, 0xd3, 0x6b, 0x98, 0xec, 0x59, 0x41, + 0x1a, 0x14, 0x1e, 0x35, 0x8c, 0x1a, 0xf3, 0x38, 0xa5, 0x24, 0x51, 0x11, 0x72, 0xb5, 0xc1, 0x56, + 0xc3, 0xac, 0xb7, 0x7a, 0x7d, 0x3e, 0x95, 0xec, 0xf4, 0x5b, 0xed, 0x06, 0x9f, 0x4a, 0xb6, 0x2b, + 0x1b, 0x5a, 0x9a, 0xc2, 0xb5, 0x1b, 0x95, 0xba, 0x96, 0xa1, 0x1b, 0xd6, 0x6c, 0x19, 0xbd, 0xbe, + 0xb9, 0x5d, 0x69, 0x0f, 0x1a, 0x5a, 0x96, 0xe2, 0xb7, 0x2b, 0xd1, 0x73, 0x8e, 0xa2, 0x75, 0xfa, + 0x0f, 0xc4, 0x23, 0xdc, 0xd3, 0xe5, 0x95, 0x81, 0x05, 0xd3, 0x55, 0xd0, 0x1e, 0xb4, 0xdb, 0xe6, + 0xa3, 0xf6, 0xa0, 0xc7, 0xff, 0x6c, 0xdf, 0xd7, 0x2e, 0x55, 0x6f, 0xbd, 0xfc, 0xef, 0xeb, 0x97, + 0x5e, 0xbe, 0xbe, 0x9e, 0xf8, 0xd7, 0xd7, 0xd7, 0x13, 0xbf, 0x7a, 0x7d, 0x3d, 0xf1, 0x5f, 0xaf, + 0xaf, 0x27, 0xfe, 0xfc, 0xd7, 0xd7, 0x2f, 0x7d, 0x03, 0x33, 0x5f, 0xfc, 0x26, 0x00, 0x00, 0xff, + 0xff, 0xd0, 0x58, 0xda, 0xbe, 0xb9, 0x2a, 0x00, 0x00, } diff --git a/pkg/sql/distsqlrun/processors.proto b/pkg/sql/distsqlrun/processors.proto index 274e769f12a6..13ad00cb49ad 100644 --- a/pkg/sql/distsqlrun/processors.proto +++ b/pkg/sql/distsqlrun/processors.proto @@ -137,6 +137,7 @@ message ProcessorCoreUnion { optional MetadataTestReceiverSpec metadataTestReceiver = 19; optional ZigzagJoinerSpec zigzagJoiner = 21; optional ProjectSetSpec projectSet = 22; + optional WindowerSpec windower = 23; reserved 6, 12; } @@ -821,3 +822,97 @@ message ProjectSetSpec { // The number of columns each expression returns. Same length as exprs. repeated uint32 num_cols_per_gen = 3; } + +// WindowerSpec is the specification of a processor that performs computations +// of window functions that have the same PARTITION BY clause. For a particular +// windowFn, the processor puts result at windowFn.ArgIdxStart and "consumes" +// all arguments to windowFn (windowFn.ArgCount of them). So if windowFn takes +// no arguments, an extra column is added; if windowFn takes more than one +// argument, (windowFn.ArgCount - 1) columns are removed. +message WindowerSpec { + enum WindowFunc { + // These mirror window functions from window_builtins.go. + ROW_NUMBER = 0; + RANK = 1; + DENSE_RANK = 2; + PERCENT_RANK = 3; + CUME_DIST = 4; + NTILE = 5; + LAG = 6; + LEAD = 7; + FIRST_VALUE = 8; + LAST_VALUE = 9; + NTH_VALUE = 10; + } + + // Func specifies which function to compute. It can either be built-in + // aggregate or built-in window function. + message Func { + option (gogoproto.onlyone) = true; + + optional AggregatorSpec.Func aggregateFunc = 1; + optional WindowFunc windowFunc = 2; + } + + // Frame is the specification of a single window frame for a window function. + message Frame { + // Mode indicates which mode of framing is used. + enum Mode { + // RANGE specifies frame in terms of logical range (e.g. 1 unit cheaper). + RANGE = 0; + // ROWS specifies frame in terms of physical offsets (e.g. 1 row before). + ROWS = 1; + } + + // BoundType indicates which type of boundary is used. + enum BoundType { + UNBOUNDED_PRECEDING = 0; + UNBOUNDED_FOLLOWING = 1; + // Offsets are stored within Bound. + OFFSET_PRECEDING = 2; // currently supported only in ROWS mode + OFFSET_FOLLOWING = 3; // currently supported only in ROWS mode + CURRENT_ROW = 4; + } + + // Bound specifies the type of boundary and the offset (if present). + message Bound { + optional BoundType boundType = 1 [(gogoproto.nullable) = false]; + // For UNBOUNDED_PRECEDING, UNBOUNDED_FOLLOWING, and CURRENT_ROW offset + // is ignored. Integer offset for ROWS mode is stored in int_offset while + // an encoded datum and the type information are stored for RANGE mode + // (the support for the latter will be added soon). + optional uint32 int_offset = 2 [(gogoproto.nullable) = false]; + optional bytes typed_offset = 3; + optional DatumInfo offset_type = 4 [(gogoproto.nullable) = false]; + } + // Bounds specifies boundaries of the window frame. + message Bounds { + // Start bound must always be present whereas end bound might be omitted. + optional Bound start = 1 [(gogoproto.nullable) = false]; + optional Bound end = 2; + } + optional Mode mode = 1 [(gogoproto.nullable) = false]; + optional Bounds bounds = 2 [(gogoproto.nullable) = false]; + } + + // WindowFn is the specification of a single window function. + message WindowFn { + // Func is which function to compute. + optional Func func = 1 [(gogoproto.nullable) = false]; + // ArgIdxStart is which index arguments to this window function start at. + optional uint32 argIdxStart = 2 [(gogoproto.nullable) = false]; + // ArgCount indicates how many arguments this window function takes in. + optional uint32 argCount = 3 [(gogoproto.nullable) = false]; + // Ordering specifies in which order rows should be considered by this + // window function. Its contents come from ORDER BY clause of the window + // function. + optional Ordering ordering = 4 [(gogoproto.nullable) = false]; + // Frame specifies over which frame this window function is computed. + optional Frame frame = 5; + } + + // PartitionBy specifies how to partition rows for all window functions. + repeated uint32 partitionBy = 1; + // WindowFns is the specification of all window functions to be computed. + repeated WindowFn windowFns = 2 [(gogoproto.nullable) = false]; +} diff --git a/pkg/sql/distsqlrun/windower.go b/pkg/sql/distsqlrun/windower.go new file mode 100644 index 000000000000..71d074a19fdc --- /dev/null +++ b/pkg/sql/distsqlrun/windower.go @@ -0,0 +1,682 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package distsqlrun + +import ( + "context" + "fmt" + "sort" + "strings" + + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sem/types" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/pkg/errors" +) + +// GetWindowFunctionInfo returns windowFunc constructor and the return type +// when given fn is applied to given inputTypes. +func GetWindowFunctionInfo( + fn WindowerSpec_Func, inputTypes ...sqlbase.ColumnType, +) ( + windowConstructor func(*tree.EvalContext) tree.WindowFunc, + returnType sqlbase.ColumnType, + err error, +) { + if fn.AggregateFunc != nil && *fn.AggregateFunc == AggregatorSpec_ANY_NOT_NULL { + // The ANY_NOT_NULL builtin does not have a fixed return type; + // handle it separately. + if len(inputTypes) != 1 { + return nil, sqlbase.ColumnType{}, errors.Errorf("any_not_null aggregate needs 1 input") + } + return builtins.NewAggregateWindowFunc(builtins.NewAnyNotNullAggregate), inputTypes[0], nil + } + datumTypes := make([]types.T, len(inputTypes)) + for i := range inputTypes { + datumTypes[i] = inputTypes[i].ToDatumType() + } + + var funcStr string + if fn.AggregateFunc != nil { + funcStr = fn.AggregateFunc.String() + } else if fn.WindowFunc != nil { + funcStr = fn.WindowFunc.String() + } else { + return nil, sqlbase.ColumnType{}, errors.Errorf( + "function is neither an aggregate nor a window function", + ) + } + _, builtins := builtins.GetBuiltinProperties(strings.ToLower(funcStr)) + for _, b := range builtins { + types := b.Types.Types() + if len(types) != len(inputTypes) { + continue + } + match := true + for i, t := range types { + if !datumTypes[i].Equivalent(t) { + match = false + break + } + } + if match { + // Found! + constructAgg := func(evalCtx *tree.EvalContext) tree.WindowFunc { + return b.WindowFunc(datumTypes, evalCtx) + } + + colTyp, err := sqlbase.DatumTypeToColumnType(b.FixedReturnType()) + if err != nil { + return nil, sqlbase.ColumnType{}, err + } + return constructAgg, colTyp, nil + } + } + return nil, sqlbase.ColumnType{}, errors.Errorf( + "no builtin aggregate/window function for %s on %v", funcStr, inputTypes, + ) +} + +// windowerState represents the state of the processor. +type windowerState int + +const ( + windowerStateUnknown windowerState = iota + // windowerAccumulating means that rows are being read from the input + // and accumulated in encodedPartitions. + windowerAccumulating + // windowerEmittingRows means that all rows have been read and + // output rows are being emitted. + windowerEmittingRows +) + +// windower is the processor that performs computation of window functions +// that have the same PARTITION BY clause. It puts the output of a window +// function windowFn at windowFn.argIdxStart and "consumes" columns +// windowFn.argIdxStart : windowFn.argIdxStart+windowFn.argCount, +// so it can both add (when argCount = 0) and remove (when argCount > 1) columns. +type windower struct { + processorBase + + // runningState represents the state of the windower. This is in addition to + // processorBase.state - the runningState is only relevant when + // processorBase.state == stateRunning. + runningState windowerState + input RowSource + inputDone bool + inputTypes []sqlbase.ColumnType + outputTypes []sqlbase.ColumnType + datumAlloc sqlbase.DatumAlloc + rowAlloc sqlbase.EncDatumRowAlloc + + scratch []byte + + cancelChecker *sqlbase.CancelChecker + + partitionBy []uint32 + encodedPartitions map[string][]sqlbase.EncDatumRow + windowFns []*windowFunc + + populated bool + bucketToPartitionIdx map[string]int + rowsInBucketEmitted map[string]int + windowValues [][][]tree.Datum + outputRow sqlbase.EncDatumRow +} + +var _ Processor = &windower{} +var _ RowSource = &windower{} + +const windowerProcName = "windower" + +func newWindower( + flowCtx *FlowCtx, + processorID int32, + spec *WindowerSpec, + input RowSource, + post *PostProcessSpec, + output RowReceiver, +) (*windower, error) { + w := &windower{ + input: input, + } + w.inputTypes = input.OutputTypes() + + windowFns := spec.WindowFns + w.encodedPartitions = make(map[string][]sqlbase.EncDatumRow) + w.partitionBy = spec.PartitionBy + w.windowFns = make([]*windowFunc, 0, len(windowFns)) + w.outputTypes = make([]sqlbase.ColumnType, 0, len(w.inputTypes)) + w.bucketToPartitionIdx = make(map[string]int) + w.rowsInBucketEmitted = make(map[string]int) + + // inputColIdx is the index of the column that should be processed next. + inputColIdx := 0 + for _, windowFn := range windowFns { + // All window functions are sorted by their argIdxStart, + // so we simply "copy" all columns up to windowFn.argIdxStart + // (all window functions in samePartitionFuncs after windowFn + // have their arguments in later columns). + w.outputTypes = append(w.outputTypes, w.inputTypes[inputColIdx:int(windowFn.ArgIdxStart)]...) + + // Check for out of bounds arguments has been done during planning step. + argTypes := w.inputTypes[windowFn.ArgIdxStart : windowFn.ArgIdxStart+windowFn.ArgCount] + windowConstructor, outputType, err := GetWindowFunctionInfo(windowFn.Func, argTypes...) + if err != nil { + return nil, err + } + // Windower processor consumes all arguments of windowFn + // and puts the result of computation of this window function + // at windowFn.argIdxStart. + w.outputTypes = append(w.outputTypes, outputType) + inputColIdx = int(windowFn.ArgIdxStart + windowFn.ArgCount) + + wf := &windowFunc{ + create: windowConstructor, + ordering: windowFn.Ordering, + argIdxStart: int(windowFn.ArgIdxStart), + argCount: int(windowFn.ArgCount), + frame: windowFn.Frame, + } + + w.windowFns = append(w.windowFns, wf) + } + // We will simply copy all columns that come after arguments + // to all window function. + w.outputTypes = append(w.outputTypes, w.inputTypes[inputColIdx:]...) + w.outputRow = make(sqlbase.EncDatumRow, len(w.outputTypes)) + + if err := w.init( + w, + post, + w.outputTypes, + flowCtx, + processorID, + output, + nil, /* memMonitor */ + procStateOpts{inputsToDrain: []RowSource{w.input}}, + ); err != nil { + return nil, err + } + + return w, nil +} + +// Start is part of the RowSource interface. +func (w *windower) Start(ctx context.Context) context.Context { + w.input.Start(ctx) + ctx = w.startInternal(ctx, windowerProcName) + w.cancelChecker = sqlbase.NewCancelChecker(ctx) + w.runningState = windowerAccumulating + return ctx +} + +// Next is part of the RowSource interface. +func (w *windower) Next() (sqlbase.EncDatumRow, *ProducerMetadata) { + for w.state == stateRunning { + var row sqlbase.EncDatumRow + var meta *ProducerMetadata + switch w.runningState { + case windowerAccumulating: + w.runningState, row, meta = w.accumulateRows() + case windowerEmittingRows: + w.runningState, row, meta = w.emitRow() + default: + log.Fatalf(w.ctx, "unsupported state: %d", w.runningState) + } + + if row == nil && meta == nil { + continue + } + return row, meta + } + return nil, w.drainHelper() +} + +// ConsumerDone is part of the RowSource interface. +func (w *windower) ConsumerDone() { + w.moveToDraining(nil /* err */) +} + +// ConsumerClosed is part of the RowSource interface. +func (w *windower) ConsumerClosed() { + // The consumer is done, Next() will not be called again. + w.internalClose() +} + +// accumulateRows continually reads rows from the input and accumulates them +// in encodedPartitions. If it encounters metadata, the metadata is immediately +// returned. Subsequent calls of this function will resume row accumulation. +func (w *windower) accumulateRows() (windowerState, sqlbase.EncDatumRow, *ProducerMetadata) { + for { + row, meta := w.input.Next() + if meta != nil { + if meta.Err != nil { + // We want to send the whole meta (below) rather than just the err, + // so we pass nil as an argument. + w.moveToDraining(nil /* err */) + return windowerStateUnknown, nil, meta + } + return windowerAccumulating, nil, meta + } + if row == nil { + log.VEvent(w.ctx, 1, "accumulation complete") + w.inputDone = true + break + } + + if len(w.partitionBy) == 0 { + w.encodedPartitions[""] = append(w.encodedPartitions[""], w.rowAlloc.CopyRow(row)) + } else { + // We need to hash the row according to partitionBy + // to figure out which partition the row belongs to. + w.scratch = w.scratch[:0] + for _, col := range w.partitionBy { + if int(col) >= len(row) { + panic(fmt.Sprintf("hash column %d, row with only %d columns", col, len(row))) + } + var err error + w.scratch, err = row[int(col)].Encode(&w.inputTypes[int(col)], &w.datumAlloc, preferredEncoding, w.scratch) + if err != nil { + return windowerStateUnknown, nil, &ProducerMetadata{Err: err} + } + } + w.encodedPartitions[string(w.scratch)] = append(w.encodedPartitions[string(w.scratch)], w.rowAlloc.CopyRow(row)) + } + } + + return windowerEmittingRows, nil, nil +} + +// decodePartitions ensures that all EncDatums of each row in each encoded +// partition are decoded. It should be called after accumulation of rows is +// complete. +func (w *windower) decodePartitions() error { + for _, encodedPartition := range w.encodedPartitions { + for _, encRow := range encodedPartition { + for i := range encRow { + if err := encRow[i].EnsureDecoded(&w.inputTypes[i], &w.datumAlloc); err != nil { + return err + } + } + } + } + return nil +} + +// emitRow emits the next row if output rows have already been populated; +// if they haven't, it first decodes all rows, computes all window functions +// over all partitions (i.e. populates outputRows), and then emits the first row. +// +// emitRow() might move to stateDraining. It might also not return a row if the +// ProcOutputHelper filtered the current row out. +func (w *windower) emitRow() (windowerState, sqlbase.EncDatumRow, *ProducerMetadata) { + if w.inputDone { + for !w.populated { + if err := w.cancelChecker.Check(); err != nil { + w.moveToDraining(err) + return windowerStateUnknown, nil, nil + } + + if err := w.decodePartitions(); err != nil { + w.moveToDraining(err) + return windowerStateUnknown, nil, nil + } + + if err := w.computeWindowFunctions(w.ctx, w.evalCtx); err != nil { + w.moveToDraining(err) + return windowerStateUnknown, nil, nil + } + w.populated = true + } + + if w.populateNextOutputRow() { + return windowerEmittingRows, w.processRowHelper(w.outputRow), nil + } + + w.moveToDraining(nil /* err */) + return windowerStateUnknown, nil, nil + } + + w.moveToDraining(nil /* err */) + return windowerStateUnknown, nil, nil +} + +// computeWindowFunctions computes all window functions over all partitions. +// The output of windowFn is put in column windowFn.argIdxStart of outputRows. +// Each window function is "responsible" for columns in the interval +// [prevWindowFn.argIdxStart + 1, curWindowFn.argIdxStart] meaning +// that non-argument columns are first copied and then the result is appended. +// After all window functions have been computed, the remaining columns +// are also simply appended to corresponding rows in outputRows. +func (w *windower) computeWindowFunctions(ctx context.Context, evalCtx *tree.EvalContext) error { + var peerGrouper peerGroupChecker + w.windowValues = make([][][]tree.Datum, len(w.windowFns)) + partitions := make([]indexedRows, len(w.encodedPartitions)) + + partitionIdx := 0 + for bucket, encodedPartition := range w.encodedPartitions { + // We want to fix some order of iteration over encoded partitions + // to be consistent. + w.bucketToPartitionIdx[bucket] = partitionIdx + rows := make([]indexedRow, 0, len(encodedPartition)) + for idx := 0; idx < len(encodedPartition); idx++ { + rows = append(rows, indexedRow{idx: idx, row: encodedPartition[idx]}) + } + partitions[partitionIdx] = indexedRows{rows: rows} + partitionIdx++ + } + + // partitionPreviouslySortedFuncIdx maps index of a window function f1 + // to the index of another window function f2 with the same ORDER BY + // clause such that f2 will have been processed before f1. + partitionPreviouslySortedFuncIdx := make([]int, len(w.windowFns)) + for i := 0; i < len(w.windowFns); i++ { + partitionPreviouslySortedFuncIdx[i] = -1 + } + sortedPartitionsCache := make([][]indexedRows, len(w.windowFns)) + shouldCacheSortedPartitions := make([]bool, len(w.windowFns)) + for windowFnIdx, windowFn := range w.windowFns { + for laterFnIdx := windowFnIdx + 1; laterFnIdx < len(w.windowFns); laterFnIdx++ { + if partitionPreviouslySortedFuncIdx[laterFnIdx] != -1 { + // We've already found equal ordering for laterFn + // among previous window functions. + continue + } + if windowFn.ordering.Equal(w.windowFns[laterFnIdx].ordering) { + partitionPreviouslySortedFuncIdx[laterFnIdx] = windowFnIdx + shouldCacheSortedPartitions[windowFnIdx] = true + } + } + } + + for windowFnIdx, windowFn := range w.windowFns { + w.windowValues[windowFnIdx] = make([][]tree.Datum, len(w.encodedPartitions)) + + frameRun := &tree.WindowFrameRun{ + ArgCount: windowFn.argCount, + ArgIdxStart: windowFn.argIdxStart, + } + + if windowFn.frame != nil { + frameRun.Frame = convertToWindowFrame(windowFn.frame) + startBound, endBound := windowFn.frame.Bounds.Start, windowFn.frame.Bounds.End + if startBound.BoundType == WindowerSpec_Frame_OFFSET_PRECEDING || + startBound.BoundType == WindowerSpec_Frame_OFFSET_FOLLOWING { + frameRun.StartBoundOffset = tree.NewDInt(tree.DInt(int(startBound.IntOffset))) + } + if endBound != nil { + if endBound.BoundType == WindowerSpec_Frame_OFFSET_PRECEDING || + endBound.BoundType == WindowerSpec_Frame_OFFSET_FOLLOWING { + frameRun.EndBoundOffset = tree.NewDInt(tree.DInt(int(endBound.IntOffset))) + } + } + } + + for partitionIdx := 0; partitionIdx < len(partitions); partitionIdx++ { + builtin := windowFn.create(evalCtx) + defer builtin.Close(ctx, evalCtx) + + partition := partitions[partitionIdx] + w.windowValues[windowFnIdx][partitionIdx] = make([]tree.Datum, partition.Len()) + + if len(windowFn.ordering.Columns) > 0 { + // If an ORDER BY clause is provided, order the partition and use the + // sorter as our peerGroupChecker. + if funcIdx := partitionPreviouslySortedFuncIdx[windowFnIdx]; funcIdx != -1 { + // We have cached sorted partitions - no need to resort them. + partition = sortedPartitionsCache[funcIdx][partitionIdx] + peerGrouper = &partitionSorter{ + evalCtx: evalCtx, + rows: partition, + ordering: windowFn.ordering, + } + } else { + sorter := &partitionSorter{ + evalCtx: evalCtx, + rows: partition, + ordering: windowFn.ordering, + } + sort.Sort(sorter) + peerGrouper = sorter + if shouldCacheSortedPartitions[windowFnIdx] { + // Later window functions will need rows in the same order, + // so we cache copies of all sorted partitions. + if sortedPartitionsCache[windowFnIdx] == nil { + sortedPartitionsCache[windowFnIdx] = make([]indexedRows, len(partitions)) + } + sortedPartitionsCache[windowFnIdx][partitionIdx] = partition.makeCopy() + } + } + } else { + // If ORDER BY clause is not provided, all rows are peers. + peerGrouper = allPeers{} + } + + frameRun.Rows = partition + frameRun.RowIdx = 0 + + if !frameRun.IsDefaultFrame() { + // We have a custom frame not equivalent to default one, so if we have + // an aggregate function, we want to reset it for each row. + // Not resetting is an optimization since we're not computing + // the result over the whole frame but only as a result of the current + // row and previous results of aggregation. + builtins.ShouldReset(builtin) + } + + for frameRun.RowIdx < partition.Len() { + // Compute the size of the current peer group. + frameRun.FirstPeerIdx = frameRun.RowIdx + frameRun.PeerRowCount = 1 + for ; frameRun.FirstPeerIdx+frameRun.PeerRowCount < frameRun.PartitionSize(); frameRun.PeerRowCount++ { + cur := frameRun.FirstPeerIdx + frameRun.PeerRowCount + if !peerGrouper.InSameGroup(cur, cur-1) { + break + } + } + + // Perform calculations on each row in the current peer group. + for ; frameRun.RowIdx < frameRun.FirstPeerIdx+frameRun.PeerRowCount; frameRun.RowIdx++ { + res, err := builtin.Compute(ctx, evalCtx, frameRun) + if err != nil { + return err + } + w.windowValues[windowFnIdx][partitionIdx][frameRun.Rows.GetRow(frameRun.RowIdx).GetIdx()] = res + } + } + } + } + + return nil +} + +// populateNextOutputRow combines results of computing window functions with +// non-argument columns of the input row to produce an output row. +func (w *windower) populateNextOutputRow() bool { + for bucket, encodedPartition := range w.encodedPartitions { + if w.rowsInBucketEmitted[bucket] == len(encodedPartition) { + // All output rows corresponding to partition 'bucket' have been fully + // emitted already, so we skip it. + continue + } + + // We reuse the same EncDatumRow since caller of Next() should've copied it. + w.outputRow = w.outputRow[:0] + // rowIdx is the index of the next row to be emitted from partition 'bucket'. + rowIdx := w.rowsInBucketEmitted[bucket] + inputRow := encodedPartition[rowIdx] + partitionIdx := w.bucketToPartitionIdx[bucket] + inputColIdx := 0 + for windowFnIdx, windowFn := range w.windowFns { + // We simply pass through columns in [inputColIdx, windowFn.argIdxStart). + w.outputRow = append(w.outputRow, inputRow[inputColIdx:windowFn.argIdxStart]...) + windowFnRes := w.windowValues[windowFnIdx][partitionIdx][rowIdx] + encWindowFnRes := sqlbase.DatumToEncDatum(w.outputTypes[len(w.outputRow)], windowFnRes) + w.outputRow = append(w.outputRow, encWindowFnRes) + // We skip all columns that were arguments to windowFn. + inputColIdx = windowFn.argIdxStart + windowFn.argCount + } + // We simply pass through all columns after all arguments to window functions. + w.outputRow = append(w.outputRow, inputRow[inputColIdx:]...) + w.rowsInBucketEmitted[bucket] = rowIdx + 1 + return true + } + return false +} + +type windowFunc struct { + create func(*tree.EvalContext) tree.WindowFunc + ordering Ordering + argIdxStart int + argCount int + frame *WindowerSpec_Frame +} + +type partitionSorter struct { + evalCtx *tree.EvalContext + rows indexedRows + ordering Ordering +} + +// partitionSorter implements the sort.Interface interface. +func (n *partitionSorter) Len() int { return n.rows.Len() } +func (n *partitionSorter) Swap(i, j int) { + n.rows.rows[i], n.rows.rows[j] = n.rows.rows[j], n.rows.rows[i] +} +func (n *partitionSorter) Less(i, j int) bool { return n.Compare(i, j) < 0 } + +// partitionSorter implements the peerGroupChecker interface. +func (n *partitionSorter) InSameGroup(i, j int) bool { return n.Compare(i, j) == 0 } + +func (n *partitionSorter) Compare(i, j int) int { + ra, rb := n.rows.rows[i], n.rows.rows[j] + for _, o := range n.ordering.Columns { + da := ra.GetDatum(int(o.ColIdx)) + db := rb.GetDatum(int(o.ColIdx)) + if c := da.Compare(n.evalCtx, db); c != 0 { + if o.Direction != Ordering_Column_ASC { + return -c + } + return c + } + } + return 0 +} + +type allPeers struct{} + +// allPeers implements the peerGroupChecker interface. +func (allPeers) InSameGroup(i, j int) bool { return true } + +// peerGroupChecker can check if a pair of row indexes within a partition are +// in the same peer group. +type peerGroupChecker interface { + InSameGroup(i, j int) bool +} + +func mapToWindowFrameMode(m WindowerSpec_Frame_Mode) tree.WindowFrameMode { + switch m { + case WindowerSpec_Frame_RANGE: + return tree.RANGE + case WindowerSpec_Frame_ROWS: + return tree.ROWS + default: + panic("unexpected WindowerSpec_Frame_Mode") + } +} + +func convertToWindowFrameBoundType(bt WindowerSpec_Frame_BoundType) tree.WindowFrameBoundType { + switch bt { + case WindowerSpec_Frame_UNBOUNDED_PRECEDING: + return tree.UnboundedPreceding + case WindowerSpec_Frame_OFFSET_PRECEDING: + return tree.ValuePreceding + case WindowerSpec_Frame_CURRENT_ROW: + return tree.CurrentRow + case WindowerSpec_Frame_OFFSET_FOLLOWING: + return tree.ValueFollowing + case WindowerSpec_Frame_UNBOUNDED_FOLLOWING: + return tree.UnboundedFollowing + default: + panic("unexpected WindowerSpec_Frame_BoundType") + } +} + +// convertToWindowFrameBounds produces tree.WindowFrameBounds based on +// WindowerSpec_Frame_Bounds. Note that it might not be fully equivalent +// to original - if offsetExprs were present in original WindowFrameBounds, +// they are not included. +func convertToWindowFrameBounds(b WindowerSpec_Frame_Bounds) tree.WindowFrameBounds { + bounds := tree.WindowFrameBounds{StartBound: &tree.WindowFrameBound{ + BoundType: convertToWindowFrameBoundType(b.Start.BoundType), + }} + if b.End != nil { + bounds.EndBound = &tree.WindowFrameBound{BoundType: convertToWindowFrameBoundType(b.End.BoundType)} + } + return bounds +} + +func convertToWindowFrame(f *WindowerSpec_Frame) *tree.WindowFrame { + return &tree.WindowFrame{Mode: mapToWindowFrameMode(f.Mode), Bounds: convertToWindowFrameBounds(f.Bounds)} +} + +// indexedRows are rows with the corresponding indices. +type indexedRows struct { + rows []indexedRow +} + +// Len implements tree.IndexedRows interface. +func (ir indexedRows) Len() int { + return len(ir.rows) +} + +// GetRow implements tree.IndexedRows interface. +func (ir indexedRows) GetRow(idx int) tree.IndexedRow { + return ir.rows[idx] +} + +func (ir indexedRows) makeCopy() indexedRows { + ret := indexedRows{rows: make([]indexedRow, ir.Len())} + copy(ret.rows, ir.rows) + return ret +} + +// indexedRow is a row with a corresponding index. +type indexedRow struct { + idx int + row sqlbase.EncDatumRow +} + +// GetIdx implements tree.IndexedRow interface. +func (ir indexedRow) GetIdx() int { + return ir.idx +} + +// GetDatum implements tree.IndexedRow interface. +func (ir indexedRow) GetDatum(colIdx int) tree.Datum { + return ir.row[colIdx].Datum +} + +// GetDatums implements tree.IndexedRow interface. +func (ir indexedRow) GetDatums(startColIdx, endColIdx int) tree.Datums { + datums := make(tree.Datums, 0, endColIdx-startColIdx) + for idx := startColIdx; idx < endColIdx; idx++ { + datums = append(datums, ir.row[idx].Datum) + } + return datums +} diff --git a/pkg/sql/logictest/testdata/logic_test/window b/pkg/sql/logictest/testdata/logic_test/window index a6b7af526172..ee24615a06cb 100644 --- a/pkg/sql/logictest/testdata/logic_test/window +++ b/pkg/sql/logictest/testdata/logic_test/window @@ -1,4 +1,4 @@ -# LogicTest: local local-opt local-parallel-stmts fakedist fakedist-opt fakedist-metadata +# LogicTest: local local-opt local-parallel-stmts fakedist fakedist-opt fakedist-metadata 5node-local 5node-dist 5node-dist-opt 5node-dist-metadata 5node-dist-disk statement ok CREATE TABLE kv ( @@ -628,6 +628,16 @@ SELECT round((avg(d) OVER wind) * max(k) + (lag(d, 1, 42.0) OVER wind) * max(d)) -385 -372 +query RR +SELECT avg(k) OVER w, avg(k) OVER w + 1 FROM kv WINDOW w AS (PARTITION BY v ORDER BY w) ORDER BY k +---- +4.6666666666666666667 5.6666666666666666667 +5.5 6.5 +5 6 +4.6666666666666666667 5.6666666666666666667 +7 8 +8 9 + statement OK INSERT INTO kv VALUES (9, 2, 9, .1, DEFAULT, DEFAULT, DEFAULT), @@ -2042,7 +2052,7 @@ Samsung Tablet 200.00 350 iPad Tablet 700.00 450 query TTRR -SELECT group_name, product_name, price, avg(price) OVER (PARTITION BY group_name ROWS (SELECT count(*) FROM PRODUCTS WHERE price = 200) PRECEDING) AS running_avg_of_three FROM products ORDER BY group_id +SELECT group_name, product_name, price, avg(price) OVER (PARTITION BY group_name ORDER BY group_id ROWS (SELECT count(*) FROM PRODUCTS WHERE price = 200) PRECEDING) AS running_avg_of_three FROM products ORDER BY group_id ---- Smartphone Microsoft Lumia 200.00 200.00 Smartphone HTC One 400.00 300.00 @@ -2177,7 +2187,7 @@ Kindle Fire 150.00 150.00 700.00 1050.00 150.00 Samsung 200.00 150.00 700.00 350.00 200.00 query RRR -SELECT avg(price) OVER w1, avg(price) OVER w2, avg(price) OVER w1 FROM products WINDOW w1 AS (PARTITION BY group_name ORDER BY group_id ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING), w2 AS (ORDER BY group_id ROWS 1 PRECEDING) +SELECT avg(price) OVER w1, avg(price) OVER w2, avg(price) OVER w1 FROM products WINDOW w1 AS (PARTITION BY group_name ORDER BY group_id ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING), w2 AS (ORDER BY group_id ROWS 1 PRECEDING) ORDER BY group_id ---- 300.00 200.00 300.00 366.66666666666666667 300.00 366.66666666666666667 diff --git a/pkg/sql/logictest/testdata/planner_test/distsql_window b/pkg/sql/logictest/testdata/planner_test/distsql_window new file mode 100644 index 000000000000..daaaaafcad0c --- /dev/null +++ b/pkg/sql/logictest/testdata/planner_test/distsql_window @@ -0,0 +1,64 @@ +# LogicTest: 5node-dist + +statement ok +CREATE TABLE data (a INT, b INT, c FLOAT, d DECIMAL, PRIMARY KEY (a, b, c, d)) + +# Split into ten parts. +statement ok +ALTER TABLE data SPLIT AT SELECT i FROM generate_series(1, 9) AS g(i) + +# Relocate the ten parts to the five nodes. +statement ok +ALTER TABLE data EXPERIMENTAL_RELOCATE + SELECT ARRAY[i%5+1], i FROM generate_series(0, 9) AS g(i) + +# Verify data placement. +query TTTI colnames +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE data] +---- +start_key end_key replicas lease_holder +NULL /1 {1} 1 +/1 /2 {2} 2 +/2 /3 {3} 3 +/3 /4 {4} 4 +/4 /5 {5} 5 +/5 /6 {1} 1 +/6 /7 {2} 2 +/7 /8 {3} 3 +/8 /9 {4} 4 +/9 NULL {5} 5 + +query T +SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(a) OVER () FROM data] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJy0kk9LwzAYxu9-CnlOCoE1bechp1530MlUPEgPsXkZhS4pSYrK6HeXpofZ0Uml9Zg_v_f3vPAcoY2iB3kgB_EGDoYYDAkYUjCskTPU1hTknLHdlx7YqE-IiKHUdeO765yhMJYgjvClrwgCz_K9oh1JRXYVgUGRl2UVNLUtD9J-ZUp6CYZt48V1xpG3DKbxp5HOyz1B8JZd0J5sjTZWkSU1MOXtSLDXUivzQXa1HqZ6erm_yfjtxRjxIAafvj1fcPt4ujZeUJtM1yYLatPp2vSfKjai3ZGrjXZ0VrXxyVFXQVJ76vvqTGMLerSmCJr-uA1cuFDkfP_K-8NGh6cQ8CfMf4XvBnB0DsdzzMkcOJ0Dr_8E5-3VdwAAAP__SlOZlw== + +query T +SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(a) OVER (PARTITION BY b ORDER BY c) FROM data] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJzElc9um0AQh-99CjSnRN3KLGDH4USj9uBD7Yi6qqKKA_GOEiSHRbuL2ijyu1dAJBea7iAtro8Yvvnz_aTxC5RS4Dp_Qg3xD-DAIAAGITCIgMEcMgaVkjvUWqrmkw5YiV8Q-wyKsqpN83PGYCcVQvwCpjB7hBi2-f0eU8wFqpkPDASavNi3bSpVPOXqORG5yYHBpjaxl3CWBCwJITswkLV5LXysd__sPeb6sV8pCSA7ZAy0yR8QYn5g_xjwWKcupRKoUPQqtVWGK3wvSiF_oprN-11vP6bb1Xa1WXs3d7GXNMa-fvtykfBL72KTfvqcejd3XhK-vxwscxw0mHbQtfwgqxn3B1--3Tvs9ebjU-RnSZFPluLitCkG400GZzEZTGby6rQmw_Emw7OYDCczuTytyWi8yegsJqPJTF7_vxv9xhgp6kqWGkddYL9ZBMUDdltrWasd3iq5a9t0j5uWa8-fQG26t68Pq7J71Qw4Ho5c4IULvHSBuW-n-ZD2_6QDOxxYYd737Q_p0CUsO0yEZYeJsOwwERaxMxFW5BLW3EW3HSZ022FCtx0mdBM7E7oXLrqvXHTbYUK3HSZ022FCN7EzoXvpovvaRbcdJnTbYUK3HSZ0EztTl_-vPw6r7-zw7ncAAAD__34O4ms= + +query T +SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(a) OVER (PARTITION BY b ORDER BY c) FROM data ORDER BY d] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJzElU9v2kAQxe_9FNacEmUr2PWagE9u1B44FCJCVUUVB4cdJUjEi9aL2ijiu1eYIGLH7DhaI47-82bevt9o9hUyrXCUPmMO8R_gwEAAgxAYSGAQwYzByug55rk22192gqH6B3GXwSJbre329YzBXBuE-BXswi4RYpimD0ucYKrQdLrAQKFNF8uizcosnlPzkqjUpsBgvLZxkHCWSJYIloQw2zDQa_tW-1Dy4SV4SvOncrHi_xmD3KaPCDHfsCMeD3XWmTYKDapSpaJK9RS_F5nSf9F0onLX22-T6XA6HI-Cm_s4SLaB3f36eZHwy-BiPPn-YxLc3AeJvLqsHOZgVBw1WmPjThuLpsMrQSbi6n1-4mizsEEqdZkUHWqDGemvetXhUSXC-vay1J43Hxx-rsHhrQ1O77SDQ4S5HxzeyuCI5uTEuciJ1shdn5YcEeaenGiFXNicXHgucmFr5PqnJUeEuScXtkJONicnz0VOtkZucFpyRJh7crL1a7am2QTzlc5ybHSDdrepoXrEXcS5Xps53ho9L9rsHseFrrgSFOZ29_XtYZjtPm0NNhdHPuK-j5h7-ebSreZVdfe9WpTE3apYOMWhu3PoFPOeu7X0Qe0WE6jdYgI1YdvLN4U6cgZO5N3zQX3tA8stJmC5xQQswraXbwpW35n3wA1r4AOLe21RQk3gItQEL8q5n3Vyk7pXKSd2KfdapvzDNv0UNLeaguZWU9AI537WSWgfVmoZWkRAc-_UKrTZ5sv_AAAA__-pavjK + +query T +SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(a) OVER (PARTITION BY b ORDER BY c), avg(a) OVER (PARTITION BY b) FROM data ORDER BY d] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJzElU9v2kwQxu_vp7DmBMq-gvUfAj45UauKQyEitFVU-eCwowSJeNF6URtFfPfKdhCx4-y4WhckLsZ-Zp59fqPZF0ilwFnyhBmEP4EDAxcYeMDABwYBxAy2Sq4wy6TKPykFU_EbwiGDdbrd6fzvmMFKKoTwBfRabxBCWCb3G1xgIlANhsBAoE7Wm6LNVq2fEvUciUQnwGC-06ETcZb_fBa5LPIg3jOQO_1a_lj1_tl5TLLHar3Ih3gfM8h08oAQ8j37wOaxzi6VSqBCUalUVKkf5Mc6FfIXqkFQ7XpztVhOl9P5zLm-C50oz-v229dexPtOb7749HnhXN85UXDRBwZX37_0IrdfO9bRsvuh5QZDt1JpVANeSzXyLt6G2RDjsZ_XIqKmgIomjSnN5P9yO-BBLc_m9n6lPW8_SPyMg8Q7G6TRqQaJSPYwSLyrQXLbk3TPSNLtjOTlqUgSyR5Iul2R9NqT9M5I0uuM5PhUJIlkDyS9rkj67Un6ZyTpd0ZyciqSRLIHkv6_uKYb-i0w28o0w1Y38DCPEMUDlnlncqdWeKPkqmhTPs4LXXGLCMx0-fb1YZqWr3KD7cWBjXhsI-ZWvrlvVvO6evhW7VbEw7rYNYo9c2fPKOYjc2vfBrVZTKA2iwnUhG0r3xTqwBg4kffIBvWlDSyzmIBlFhOwCNtWvilYY2PeEzOsiQ0sbrVFCTWBi1ATvCjndtbJTWpepZzYpdxqmfJ32_SvoJnVFDSzmoJGOLezTkJ7t1Kr0AICmnmn1qHF-__-BAAA__-fWglv + +query T +SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(a) OVER w1 AS sum_a, avg(b) OVER w2 FROM data WINDOW w1 AS (PARTITION BY b ORDER BY c), w2 AS (PARTITION BY a, d) ORDER BY sum_a ] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJzMlk9P20wQxu_vp7DmBGJf4V07_3wyqFWVQwkKtBWqcjDxCiJBNrIdtQjlu1d2iKjjMI-j3SqROBDCz7Mz-xs9fqW5SfVV8qxzin6SJEGKBAUkKCRBHZoIWmRmqvPcZOW_rIFh-psiX9BsvlgW5Z8ngqYm0xS9UjErnjRFdJvcP-mxTlKdnfskKNVFMnuqyiyy2XOSvcRpUiQkaLQsIi-WIlbVTyDK30OarASZZfFW4f3B9y_eY5I_1h8ZBzRZTQTlRfKgKZIr8cFJ35-znJss1ZlOa0-qnrLdy4_ZPDW_dHbeqVe9vhjfDm-Hoyvv8i7y4nJsN9--nsTy1DsZjT99HnuXd14cnp3u2UxHxN1aP-of9SN9tqHqHIIuvn85idV2E--nCz483Y7aNyYryspbo4zlWU2FD4uFLUaxaxBVhZ3TuDL_m8W58rfmtrt8p1Zett8BedgdkM6c6R7FDrjrR0r3OwC82OxA18kOqPYSqsNKqJxdWu8oJHTXj1TuJQRebCTsOZEwaC9hcFgJA2eX1j8KCd31IwP3EgIvNhL2nUgYtpcwPKyEobNLGxyFhO76kaF7CYEXGwkHzl9JdxQb63xh5rlu9bbpl6PS6YNezzU3y2yqrzMzrcqsP44qrnrnSHVerL99-zCcr78qD9ge7trA0reiQyu6z9OSpZUN3LOBJbguQHes6AFPq23a_5sOarC_DQcsDO46ZGEl-dIdm_3gYbAfoDLYD0CDmQEa7EfXZj94GOwHD6P9ADTYD0CD_eixlvZ5Sfs2-zGwMZyHgeGgMjAc0HZdowTgAwRFAE-jDADhhUIA4CgFAA40l40UqakqFS-65GME3Lls5Mg-qgMauI5qA9kRbtk50r2RoXvpztNId56GugMc6Q5wpHsjSOu6d4HujUzZS_dGpuylO08j3UFtpDvALTtHujfydC_deRrpztNQd4Aj3QGOdG-kav1d2-d1V41sYXWfrP77EwAA__8d9kEN + +query T +SELECT url FROM [EXPLAIN (DISTSQL) SELECT sum(a) OVER w1 AS sum_a, avg(b) OVER w2, rank() OVER w1 AS rnk, row_number() OVER w3 AS rnumber, max(a) OVER w3, min(b) OVER w2 FROM data WINDOW w1 AS (PARTITION BY b ORDER BY c), w2 AS (PARTITION BY a, d), w3 AS (PARTITION BY a, c ORDER BY b, d) ORDER BY rnk, rnumber ] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJzMmN9P4k4Uxd-_fwW5TxjnG5hpy6-nwazZkA1gqq5rNmRT6URJlJK2ZNcY_vcNRdYt4hzYW0hfjFA_c-bO9Zy58ELTKDSD4Mkk1PlOkgQpEuSQIJcEeTQSNIujsUmSKF7-yQrohb-oUxc0mc7m6fLtkaBxFBvqvFA6SR8NdegquHs0vglCE9fqJCg0aTB5zGRm8eQpiJ91GKQBCRrO005FS6GVeP2phHay310aLQRF8_RV5235u-fKQ5A85BfWHo0WI0FJGtwb6siF-GC_b-vMp1EcmtiEuZWyVTYruplMw-iniWteXvWi61_1rnrDQeXstlPRy6eX1_2qlieV6tD_dO5Xzm4runF6QoL87uBL9d37-xXZErqdq1MdqE5Ztxaa7UNQ9-vnqlbL4vq9QVV7_1BOM1eOc6hy7H3L9iHIH978GFz3z879zT4J3c562O9-q2p368N85W8luR-WtGXDl1GcmrimNk5fO6dCu6cbhnGEdoX2hG58KO7tcJ7bTvOP4tZjHUT_R7Oa8jYasH0LjdwW5O7JIcuQHLKwf8FGqZOjuDqlLEFyFFiOvW_HSw7gnHVyyIMkh9rdtqoMtlWF9b9ZatsWV6dUJbBtgeXY-3Y82wLnrG2rDmJbZ3fbOmWwrVNY_1ultm1xdUqnBLYtsBx7345nW-CctW2dg9jW3d22bhls6xbW_3apbVtcndItgW0LLMfet-PZFjhnbVv34B-vt4j7JplF08Ts9Km5vjxvE96bVXOSaB6PzUUcjTOZ1cthxmWfDEKTpKunry9609Wj5QZ3h5scWCoWzdJWQFvaaQ7c4sDSYdEsbQW0lZVmwW0OLF0WzdJWQNvZpOt_03m4vgm7VtizK3tWWDXs0g1Opthh4GugDKwJaJY2ypQmJ1PsMPC1HUaZAmiWNsqUFidT7DDwtR1GmQJoljbKlLbV2bJud7Z8d3HukyoSXEAsGrgTaaNbG-A8dThwgHuERaNrH9yAaOYAOE8dJYR8d4XuExGABj4FNAoJhPPUUUxI-_ggPZAT9gEC5QRrgAA0cipvhEA4Tx3mBGuKADRyKm-OQDhPHeYEa5QANHIqb5hAOE8d5oR9nlBgnlCseUKx5glAA6cibeA1hPPUUU4o1jwBaOBUQKOcQDhPHX6LwZonAI2-S-DNEwjnqaOcUPZ5QoF5Qu03T4wW__0OAAD__1jr_eE= + +query T +SELECT url FROM [EXPLAIN (DISTSQL) SELECT min(d) OVER w1, sum(a) OVER w2, avg(b) OVER w3 FROM data WINDOW w1 AS (PARTITION BY b ORDER BY c), w2 AS (ORDER BY c), w3 AS (PARTITION BY a, d) ORDER BY a, b ] +---- +https://cockroachdb.github.io/distsqlplan/decode.html#eJzMlV1v2jwYhs_fXxE9R1RYavzBV47c6p0mDgoVZZuqKQcptlokGiPHaKsq_vuUpBtLWvwEBSZOECFcfm5fueW8QmqUniTPOoPoO1AgwIAABwICCPQgJrC2ZqGzzNj8LyUwVj8hCgks0_XG5T_HBBbGaohewS3dSkME8-RhpWc6UdpehkBAaZcsV8WYtV0-J_ZFqsQlQGC6cVEgBZGUSPb7kxHJi-8C4i0Bs3Fvo3YTHl6CpyR7qq4t-xBvYwKZSx41RHRL9kTerbNJjVXaalVZqVilvqlvy1SZH9pe9qpTb69m8_F8PJ0E1_dRIPtA4GY86Uh6EXSms_8_zYLr-0AOuhe1zeyCshMFpTX3d19uOpI1jsX3xvIOpV49QyJHQODq6-eO5Ptni0Nm3xnr8smsVgfRJbLX_dOyslx878xeZSZtXnF6JhWnR2tO_7QVZ83dsjNxy47mdnBat7y5W34mbvnR3A5P61Y0dyvOxK04mtvRv3vtfRBjprO1STNdC_zxymG-Ea0edbnrzGzsQt9asyjGlJfTgisOTaUzV959uxin5a08YHO43wYetoFp2IpmfprW6fBvGoGZF-YVOKzD3AsLPyy8MEVG99r0xA8jPfHDSE-Q2EhPEBp51P02PRm08e2HEd9-GPGNxEZ8IzSibNjG96iNbz-M-PbDiG8kNuIbobFz8N0xeohw-u4YPcQ4QiPKERpxjiVHpGM45s3_BqnT8fa_XwEAAP__jHZv-w== diff --git a/pkg/sql/logictest/testdata/planner_test/window b/pkg/sql/logictest/testdata/planner_test/window index 6a7e509279de..d3899d512c0d 100644 --- a/pkg/sql/logictest/testdata/planner_test/window +++ b/pkg/sql/logictest/testdata/planner_test/window @@ -1,4 +1,4 @@ -# LogicTest: local +# LogicTest: local 5node-dist statement ok CREATE TABLE kv ( @@ -38,7 +38,7 @@ sort · · statement ok SET tracing = on,kv,results; SELECT k, stddev(d) OVER w FROM kv WINDOW w as (PARTITION BY v) ORDER BY variance(d) OVER w, k; 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%' ---- diff --git a/pkg/sql/sem/builtins/window_builtins.go b/pkg/sql/sem/builtins/window_builtins.go index ee859693952f..51c81386287c 100644 --- a/pkg/sql/sem/builtins/window_builtins.go +++ b/pkg/sql/sem/builtins/window_builtins.go @@ -177,6 +177,16 @@ type aggregateWindowFunc struct { peerRes tree.Datum } +// NewAggregateWindowFunc creates a constructor of aggregateWindowFunc +// with agg initialized by provided aggConstructor. +func NewAggregateWindowFunc( + aggConstructor func(*tree.EvalContext) tree.AggregateFunc, +) func(*tree.EvalContext) tree.WindowFunc { + return func(evalCtx *tree.EvalContext) tree.WindowFunc { + return &aggregateWindowFunc{agg: aggConstructor(evalCtx)} + } +} + func (w *aggregateWindowFunc) Compute( ctx context.Context, evalCtx *tree.EvalContext, wfr *tree.WindowFrameRun, ) (tree.Datum, error) {