From 55dd1c94bd57afa7dd978af32f7bfab989cfc2fc 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 | 444 ++++ pkg/sql/distsqlrun/api.pb.go | 1 + pkg/sql/distsqlrun/processors.go | 6 + pkg/sql/distsqlrun/processors.pb.go | 1933 ++++++++++++++--- pkg/sql/distsqlrun/processors.proto | 112 + pkg/sql/distsqlrun/windower.go | 726 +++++++ .../logictest/testdata/planner_test/window | 3 +- pkg/sql/sem/builtins/window_builtins.go | 10 + pkg/sql/targets.go | 48 +- pkg/sql/window.go | 17 +- 10 files changed, 2925 insertions(+), 375 deletions(-) create mode 100644 pkg/sql/distsqlrun/windower.go diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 20d063c1bf89..cf5341b40e06 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) } @@ -2240,6 +2243,13 @@ func (dsp *DistSQLPlanner) createPlanForNode( case *zeroNode: plan, err = dsp.createPlanForZero(planCtx, n) + case *windowNode: + plan, err = dsp.createPlanForWindow(planCtx, n) + // TODO(yuzefovich): figure out how to propagate the error correctly. + if err != nil { + return plan, err + } + default: panic(fmt.Sprintf("unsupported node type %T", n)) } @@ -2773,6 +2783,440 @@ func (dsp *DistSQLPlanner) createPlanForSetOp( return p, nil } +type windowFuncsSorter []*windowFuncHolder + +func (w windowFuncsSorter) Len() int { return len(w) } +func (w windowFuncsSorter) Swap(i, j int) { w[i], w[j] = w[j], w[i] } +func (w windowFuncsSorter) Less(i, j int) bool { + return w[i].argIdxStart < w[j].argIdxStart +} + +// createPlanForWindow creates a physical plan for computing window functions. +// We add new stage of windower processors for each different partitioning found +// in window functions. +// +// The result of a window function computation is put at windowFn.argIdxStart index, +// in a sense, we replace the first argument to window function by its output; +// also, window function "consumes" windowFn.argCount columns. +// So if a window function takes zero arguments, we'll put an extra column and +// need to shift all indices (argIdxStart, partitionIdxs, and column ordering from +// ORDER BY clause) of window functions after the current one to the "right". +// Similarly, if a window function takes more than one argument, we'll consume +// extra columns and need to shift all indices of window functions after the current +// one accordingly to the "left". +// We also adjust windowFuncOutputColumnIdx that maps already processed window function +// to the index of output column of that window function during all stages. +// +// Important assumption: arguments of all window functions and columns needed for +// ORDER BY clause of the whole query come before columns +// used to "render" PARTITION BY and ORDER BY clauses and before all columns +// referenced by IndexedVar's that correspond to columns and aggregations +// above "windowing level". (See comment for replaceIndexVarsAndAggFuncs method +// of windowNode for context.) +func (dsp *DistSQLPlanner) createPlanForWindow( + planCtx *planningCtx, n *windowNode, +) (physicalPlan, error) { + plan, err := dsp.createPlanForNode(planCtx, n.plan) + if err != nil { + return physicalPlan{}, err + } + + // Sort window functions by their argIdxStart. + sort.Sort(windowFuncsSorter(n.funcs)) + + // windowFuncOutputColumnIdx is maintained throughout all stages + // of windowers to point at the output column of the corresponding + // to a windowFuncHolder window function. + windowFuncOutputColumnIdx := make(map[*windowFuncHolder]int) + isWindowFuncProcessed, numWindowFuncProcessed := make([]bool, len(n.funcs)), 0 + samePartitionFuncs := make([]*windowFuncHolder, 0, len(n.funcs)) + samePartitionFuncsIdx := make([]int, 0, len(n.funcs)) + + for numWindowFuncProcessed < len(n.funcs) { + notProcessedIdx := -1 + for funcIdx := 0; funcIdx < len(n.funcs); funcIdx++ { + if !isWindowFuncProcessed[funcIdx] { + notProcessedIdx = funcIdx + break + } + } + if notProcessedIdx == -1 { + panic("unexpected: no unprocessed window function") + } + + partitionIdxs := make([]uint32, len(n.funcs[notProcessedIdx].partitionIdxs)) + for i, idx := range n.funcs[notProcessedIdx].partitionIdxs { + partitionIdxs[i] = uint32(idx) + } + + samePartitionFuncs = append(samePartitionFuncs[:0], n.funcs[notProcessedIdx]) + samePartitionFuncsIdx = append(samePartitionFuncsIdx[:0], notProcessedIdx) + isWindowFuncProcessed[notProcessedIdx] = true + windowFuncOutputColumnIdx[n.funcs[notProcessedIdx]] = n.funcs[notProcessedIdx].argIdxStart + numWindowFuncProcessed++ + for funcIdx := notProcessedIdx + 1; funcIdx < len(n.funcs); funcIdx++ { + if isWindowFuncProcessed[funcIdx] || len(partitionIdxs) != len(n.funcs[funcIdx].partitionIdxs) { + continue + } + samePartition := true + for i, p := range partitionIdxs { + if p != uint32(n.funcs[funcIdx].partitionIdxs[i]) { + samePartition = false + break + } + } + if samePartition { + samePartitionFuncs = append(samePartitionFuncs, n.funcs[funcIdx]) + samePartitionFuncsIdx = append(samePartitionFuncsIdx, funcIdx) + isWindowFuncProcessed[funcIdx] = true + windowFuncOutputColumnIdx[n.funcs[funcIdx]] = n.funcs[funcIdx].argIdxStart + numWindowFuncProcessed++ + } + } + + windowerSpec := distsqlrun.WindowerSpec{ + PartitionBy: partitionIdxs, + WindowFns: make([]distsqlrun.WindowerSpec_WindowFn, len(samePartitionFuncs)), + } + + 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 windowIdx, windowFn := range samePartitionFuncs { + funcStr := strings.ToUpper(windowFn.expr.Func.String()) + funcIdx, ok := distsqlrun.WindowerSpec_Func_value[funcStr] + if !ok { + return plan, errors.Errorf("unknown window function %s", funcStr) + } + + // 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). + newResultTypes = append(newResultTypes, plan.ResultTypes[inputColIdx:windowFn.argIdxStart]...) + + if windowFn.argIdxStart+windowFn.argCount > len(plan.ResultTypes) { + return physicalPlan{}, errors.Errorf("ColIdx out of range (%d)", windowFn.argIdxStart+windowFn.argCount-1) + } + argTypes := plan.ResultTypes[windowFn.argIdxStart : windowFn.argIdxStart+windowFn.argCount] + _, outputType, err := distsqlrun.GetWindowFunctionInfo(distsqlrun.WindowerSpec_Func(funcIdx), argTypes...) + if err != nil { + return physicalPlan{}, err + } + + // Windower processor consumes all arguments of windowFn + // and puts the result of computation of this window function + // at windowFn.argIdxStart. + newResultTypes = append(newResultTypes, outputType) + inputColIdx = windowFn.argIdxStart + windowFn.argCount + + // Populating column ordering from ORDER BY clause of windowFn. + ordCols := make([]distsqlrun.Ordering_Column, 0, len(windowFn.columnOrdering)) + for _, column := range windowFn.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), + }) + } + + windowFnSpec := distsqlrun.WindowerSpec_WindowFn{ + Func: distsqlrun.WindowerSpec_Func(funcIdx), + ArgIdxStart: uint32(windowFn.argIdxStart), + ArgCount: uint32(windowFn.argCount), + Ordering: distsqlrun.Ordering{Columns: ordCols}, + } + + if frame := n.run.windowFrames[windowIdx]; frame != nil { + frameSpec, err := distsqlrun.ConvertToSpec(*frame, planCtx.EvalContext()) + if err != nil { + return physicalPlan{}, err + } + windowFnSpec.Frame = &frameSpec + } + + windowerSpec.WindowFns[windowIdx] = windowFnSpec + } + + // We keep all the columns after the last window function + // that is currently being processed. + newResultTypes = append(newResultTypes, plan.ResultTypes[inputColIdx:]...) + + for i, fInProgress := range samePartitionFuncs { + if fInProgress.argCount != 1 { + argShift := 1 - fInProgress.argCount + // All window functions after fInProgress need to be adjusted + // since fInProgress 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 n.funcs) with f1.argIdxStart == f2.argIdxStart: + // + // 1. both f1 and f2 are in samePartitionFuncs: + // 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 samePartitionFuncs, f2 is not: + // a) f2 has been processed already, so we want to maintain + // f2.argIdxStart as a pointer to its output column. f1 shifts + // all columns after f1.argIdxStart by argShift, so we need + // to adjust f2.argIdxStart 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 samePartitionFuncs, f2 is: + // f2 has no influence on f1 because f1 appears before f2 + // in n.funcs. + for idx, f := range n.funcs[samePartitionFuncsIdx[i]+1:] { + funcIdx := idx + samePartitionFuncsIdx[i] + 1 + if isWindowFuncProcessed[funcIdx] { + windowFuncOutputColumnIdx[f] += argShift + } else { + f.argIdxStart += argShift + } + } + } + } + + // Assumption: all PARTITION BY and ORDER BY related columns come + // after arguments to window functions, so we need to adjust those + // indices accordingly. + partitionOrderColShift := 0 + for _, fInProgress := range samePartitionFuncs { + partitionOrderColShift += 1 - fInProgress.argCount + } + if partitionOrderColShift != 0 { + for idx, f := range n.funcs { + if !isWindowFuncProcessed[idx] { + for p := range f.partitionIdxs { + f.partitionIdxs[p] += partitionOrderColShift + } + oldColumnOrdering := f.columnOrdering + f.columnOrdering = make(sqlbase.ColumnOrdering, 0, len(oldColumnOrdering)) + for _, o := range oldColumnOrdering { + f.columnOrdering = append(f.columnOrdering, sqlbase.ColumnOrderInfo{ + ColIdx: o.ColIdx + partitionOrderColShift, + Direction: o.Direction, + }) + } + } + } + } + + // 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. + 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, plan.Processors, false) + if len(nodes) != len(plan.ResultRouters) { + panic("unexpected number of nodes") + } + + // We put a window processor on each node and we connect it + // with all hash routers from the previous stage. + 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 + } + } + + plan.planToStreamColMap = identityMap(plan.planToStreamColMap, len(plan.ResultTypes)) + + needRendering := false + // numWindowFuncsAsIs is the number of window functions + // output of which is used directly (i.e. simply as + // an output column). + numWindowFuncsAsIs := 0 + for _, render := range n.windowRender { + if _, ok := render.(*windowFuncHolder); ok { + numWindowFuncsAsIs++ + } + } + if numWindowFuncsAsIs != len(n.funcs) { + // Not all window functions' outputs are used directly, + // so we need some rendering to do. + needRendering = true + } + + if needRendering { + // windowNode contains render expressions that might contain + // 1) IndexedVar's that refer to columns by their indices in the full table + // (the corresponding map is stored in n.colContainer), and + // 2) IndexedVar's that replaced regular aggregates that are + // above "windowing level" (the corresponding map is stored + // in n.aggContainer). + // We need to make columnsMap that maps index of + // an indexedVar to the column in the output of windower + // processor. + + // Assumption: two kinds of IndexedVar's do not refer + // to the same indices. + for col := range n.colContainer.idxMap { + if _, found := n.aggContainer.idxMap[col]; found { + panic("colContainer and aggContainer refer to the same indices") + } + } + + // maxColumnIdx is the largest column index referenced by + // any IndexedVar in renders of windowNode. + maxColumnIdx := -1 + for col := range n.colContainer.idxMap { + if col > maxColumnIdx { + maxColumnIdx = col + } + } + for col := range n.aggContainer.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 + // due to window functions that take number of arguments + // other than one. IndexedVar's from both containers + // point to columns after window functions-related columns, + // so we need to shift all indices by colShift. + colShift := 0 + for _, windowFn := range n.funcs { + colShift += 1 - windowFn.argCount + } + for col, idx := range n.colContainer.idxMap { + columnsMap[col] = idx + colShift + } + for col, idx := range n.aggContainer.idxMap { + columnsMap[col] = idx + colShift + } + + renderExprs := make([]tree.TypedExpr, len(n.windowRender)) + visitor := replaceWindowFuncsVisitor{ + windowFuncToOrdinalMap: windowFuncOutputColumnIdx, + columnsMap: columnsMap, + } + + renderTypes := make([]sqlbase.ColumnType, 0, len(n.windowRender)) + for i, render := range n.windowRender { + if render != nil { + // render contains at least one reference to windowFuncHolder, + // so we need to walk over render and replace all windowFuncHolder's + // and (if found) IndexedVar's using two maps. + renderExprs[i] = visitor.replace(render) + } else { + // render is nil meaning that a column is being passed through. + renderExprs[i] = tree.NewTypedOrdinalReference(visitor.colIdx, plan.ResultTypes[visitor.colIdx].ToDatumType()) + visitor.colIdx++ + } + outputType, err := sqlbase.DatumTypeToColumnType(renderExprs[i].ResolvedType()) + if err != nil { + return physicalPlan{}, err + } + renderTypes = append(renderTypes, outputType) + } + plan.AddRendering(renderExprs, planCtx.EvalContext(), plan.planToStreamColMap, renderTypes) + plan.planToStreamColMap = identityMap(plan.planToStreamColMap, len(renderTypes)) + } + + return plan, nil +} + +type replaceWindowFuncsVisitor struct { + windowFuncToOrdinalMap map[*windowFuncHolder]int + columnsMap []int + colIdx int // index of the current column in the output of last windower processors stage +} + +var _ tree.Visitor = &replaceWindowFuncsVisitor{} + +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.windowFuncToOrdinalMap[t], 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 +} + +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) +} + func (dsp *DistSQLPlanner) newPlanningCtx( ctx context.Context, evalCtx *extendedEvalContext, txn *client.Txn, ) planningCtx { diff --git a/pkg/sql/distsqlrun/api.pb.go b/pkg/sql/distsqlrun/api.pb.go index 1c11a4a41f85..6a817a5bda87 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/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 3ab6f190f74f..f3ef3d58fc91 100644 --- a/pkg/sql/distsqlrun/processors.pb.go +++ b/pkg/sql/distsqlrun/processors.pb.go @@ -238,6 +238,217 @@ func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{17, 0} } +type WindowerSpec_Func int32 + +const ( + // These mirror aggregate functions from AggregatorSpec. + WindowerSpec_ANY_NOT_NULL WindowerSpec_Func = 0 + WindowerSpec_AVG WindowerSpec_Func = 1 + WindowerSpec_BOOL_AND WindowerSpec_Func = 2 + WindowerSpec_BOOL_OR WindowerSpec_Func = 3 + WindowerSpec_CONCAT_AGG WindowerSpec_Func = 4 + WindowerSpec_COUNT WindowerSpec_Func = 5 + WindowerSpec_MAX WindowerSpec_Func = 7 + WindowerSpec_MIN WindowerSpec_Func = 8 + WindowerSpec_STDDEV WindowerSpec_Func = 9 + WindowerSpec_SUM WindowerSpec_Func = 10 + WindowerSpec_SUM_INT WindowerSpec_Func = 11 + WindowerSpec_VARIANCE WindowerSpec_Func = 12 + WindowerSpec_XOR_AGG WindowerSpec_Func = 13 + WindowerSpec_COUNT_ROWS WindowerSpec_Func = 14 + WindowerSpec_SQRDIFF WindowerSpec_Func = 15 + WindowerSpec_FINAL_VARIANCE WindowerSpec_Func = 16 + WindowerSpec_FINAL_STDDEV WindowerSpec_Func = 17 + WindowerSpec_ARRAY_AGG WindowerSpec_Func = 18 + WindowerSpec_JSON_AGG WindowerSpec_Func = 19 + // JSONB_AGG is an alias for JSON_AGG, they do the same thing. + WindowerSpec_JSONB_AGG WindowerSpec_Func = 20 + // These mirror window functions from window_builtins.go. + WindowerSpec_ROW_NUMBER WindowerSpec_Func = 21 + WindowerSpec_RANK WindowerSpec_Func = 22 + WindowerSpec_DENSE_RANK WindowerSpec_Func = 23 + WindowerSpec_PERCENT_RANK WindowerSpec_Func = 24 + WindowerSpec_CUME_DIST WindowerSpec_Func = 25 + WindowerSpec_NTILE WindowerSpec_Func = 26 + WindowerSpec_LAG WindowerSpec_Func = 27 + WindowerSpec_LEAD WindowerSpec_Func = 28 + WindowerSpec_FIRST_VALUE WindowerSpec_Func = 29 + WindowerSpec_LAST_VALUE WindowerSpec_Func = 30 + WindowerSpec_NTH_VALUE WindowerSpec_Func = 31 +) + +var WindowerSpec_Func_name = map[int32]string{ + 0: "ANY_NOT_NULL", + 1: "AVG", + 2: "BOOL_AND", + 3: "BOOL_OR", + 4: "CONCAT_AGG", + 5: "COUNT", + 7: "MAX", + 8: "MIN", + 9: "STDDEV", + 10: "SUM", + 11: "SUM_INT", + 12: "VARIANCE", + 13: "XOR_AGG", + 14: "COUNT_ROWS", + 15: "SQRDIFF", + 16: "FINAL_VARIANCE", + 17: "FINAL_STDDEV", + 18: "ARRAY_AGG", + 19: "JSON_AGG", + 20: "JSONB_AGG", + 21: "ROW_NUMBER", + 22: "RANK", + 23: "DENSE_RANK", + 24: "PERCENT_RANK", + 25: "CUME_DIST", + 26: "NTILE", + 27: "LAG", + 28: "LEAD", + 29: "FIRST_VALUE", + 30: "LAST_VALUE", + 31: "NTH_VALUE", +} +var WindowerSpec_Func_value = map[string]int32{ + "ANY_NOT_NULL": 0, + "AVG": 1, + "BOOL_AND": 2, + "BOOL_OR": 3, + "CONCAT_AGG": 4, + "COUNT": 5, + "MAX": 7, + "MIN": 8, + "STDDEV": 9, + "SUM": 10, + "SUM_INT": 11, + "VARIANCE": 12, + "XOR_AGG": 13, + "COUNT_ROWS": 14, + "SQRDIFF": 15, + "FINAL_VARIANCE": 16, + "FINAL_STDDEV": 17, + "ARRAY_AGG": 18, + "JSON_AGG": 19, + "JSONB_AGG": 20, + "ROW_NUMBER": 21, + "RANK": 22, + "DENSE_RANK": 23, + "PERCENT_RANK": 24, + "CUME_DIST": 25, + "NTILE": 26, + "LAG": 27, + "LEAD": 28, + "FIRST_VALUE": 29, + "LAST_VALUE": 30, + "NTH_VALUE": 31, +} + +func (x WindowerSpec_Func) Enum() *WindowerSpec_Func { + p := new(WindowerSpec_Func) + *p = x + return p +} +func (x WindowerSpec_Func) String() string { + return proto.EnumName(WindowerSpec_Func_name, int32(x)) +} +func (x *WindowerSpec_Func) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(WindowerSpec_Func_value, data, "WindowerSpec_Func") + if err != nil { + return err + } + *x = WindowerSpec_Func(value) + return nil +} +func (WindowerSpec_Func) EnumDescriptor() ([]byte, []int) { + return fileDescriptorProcessors, []int{28, 0} +} + +// Mode indicates which mode of framing is used. +type WindowerSpec_Frame_Mode int32 + +const ( + // RANGE is the mode of specifying frame in terms of logical range (e.g. 100 units cheaper). + WindowerSpec_Frame_RANGE WindowerSpec_Frame_Mode = 0 + // ROWS is the mode of specifying frame in terms of physical offsets (e.g. 1 row before etc). + 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, 0, 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_VALUE_PRECEDING WindowerSpec_Frame_BoundType = 1 + WindowerSpec_Frame_CURRENT_ROW WindowerSpec_Frame_BoundType = 2 + WindowerSpec_Frame_VALUE_FOLLOWING WindowerSpec_Frame_BoundType = 3 + WindowerSpec_Frame_UNBOUNDED_FOLLOWING WindowerSpec_Frame_BoundType = 4 +) + +var WindowerSpec_Frame_BoundType_name = map[int32]string{ + 0: "UNBOUNDED_PRECEDING", + 1: "VALUE_PRECEDING", + 2: "CURRENT_ROW", + 3: "VALUE_FOLLOWING", + 4: "UNBOUNDED_FOLLOWING", +} +var WindowerSpec_Frame_BoundType_value = map[string]int32{ + "UNBOUNDED_PRECEDING": 0, + "VALUE_PRECEDING": 1, + "CURRENT_ROW": 2, + "VALUE_FOLLOWING": 3, + "UNBOUNDED_FOLLOWING": 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, 0, 1} +} + // Each processor has the following components: // - one or more input synchronizers; each one merges rows between one or more // input streams; @@ -339,6 +550,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{} } @@ -1049,6 +1261,83 @@ 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 processors that performs computations +// of window functions that have the same PARTITION BY clause. For a particular +// windowFn, the processor puts the output 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} } + +// 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, 0} } + +// Bound specifies the offset and the type of boundary. +type WindowerSpec_Frame_Bound struct { + BoundType WindowerSpec_Frame_BoundType `protobuf:"varint,1,opt,name=boundType,enum=cockroach.sql.distsqlrun.WindowerSpec_Frame_BoundType" json:"boundType"` + Offset uint32 `protobuf:"varint,2,opt,name=offset" json:"offset"` +} + +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, 0, 0} +} + +// Bounds specifies boundaries of the window frame. +type WindowerSpec_Frame_Bounds struct { + 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, 0, 1} +} + +// WindowFn is the specification of a single window function. +type WindowerSpec_WindowFn struct { + // Func is which function to compute. + Func WindowerSpec_Func `protobuf:"varint,1,opt,name=func,enum=cockroach.sql.distsqlrun.WindowerSpec_Func" json:"func"` + // ArgIdxStart indicates from which index arguments to this window function start. + 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 a window function. + Ordering Ordering `protobuf:"bytes,4,opt,name=ordering" json:"ordering"` + // Frame specifies over which frame this window function should be 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, 1} +} + func init() { proto.RegisterType((*ProcessorSpec)(nil), "cockroach.sql.distsqlrun.ProcessorSpec") proto.RegisterType((*PostProcessSpec)(nil), "cockroach.sql.distsqlrun.PostProcessSpec") @@ -1081,10 +1370,18 @@ 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_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_Func", WindowerSpec_Func_name, WindowerSpec_Func_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() @@ -1448,6 +1745,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 } @@ -1583,11 +1892,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 } @@ -1609,11 +1918,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)) @@ -1669,47 +1978,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)) @@ -1734,11 +2043,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)) @@ -1817,30 +2126,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)) @@ -1863,21 +2172,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 } @@ -1900,27 +2209,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)) @@ -1951,47 +2260,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)) @@ -2022,21 +2331,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 { @@ -2051,21 +2360,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++ @@ -2135,11 +2444,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 @@ -2173,11 +2482,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 } @@ -2199,11 +2508,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 @@ -2271,11 +2580,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 { @@ -2300,11 +2609,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 { @@ -2330,11 +2639,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 } } } @@ -2386,11 +2695,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 } @@ -2448,11 +2757,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)) @@ -2619,11 +2928,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)) @@ -2648,30 +2957,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 @@ -2736,83 +3045,254 @@ 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_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 - 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 + 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 len(m.RenderExprs) > 0 { - for _, e := range m.RenderExprs { - l = e.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 } - n += 1 + sovProcessors(uint64(m.Offset)) - n += 1 + sovProcessors(uint64(m.Limit)) - return n + return dAtA[:n], nil } -func (m *ProcessorCoreUnion) Size() (n int) { +func (m *WindowerSpec_Frame_Bound) 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)) - } - 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)) + dAtA[i] = 0x8 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.BoundType)) + dAtA[i] = 0x10 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Offset)) + 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.Aggregator != nil { + 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())) + n65, err := m.Start.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n65 + if m.End != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.End.Size())) + n66, err := m.End.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n66 + } + 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] = 0x8 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Func)) + 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())) + n67, err := m.Ordering.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n67 + if m.Frame != nil { + dAtA[i] = 0x2a + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Frame.Size())) + n68, err := m.Frame.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n68 + } + 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)) } @@ -2876,6 +3356,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 } @@ -3349,6 +3833,67 @@ 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_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.Offset)) + 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 + n += 1 + sovProcessors(uint64(m.Func)) + 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++ @@ -3423,6 +3968,9 @@ func (this *ProcessorCoreUnion) GetValue() interface{} { if this.ProjectSet != nil { return this.ProjectSet } + if this.Windower != nil { + return this.Windower + } return nil } @@ -3468,6 +4016,8 @@ func (this *ProcessorCoreUnion) SetValue(value interface{}) bool { this.ZigzagJoiner = vt case *ProjectSetSpec: this.ProjectSet = vt + case *WindowerSpec: + this.Windower = vt default: return false } @@ -4603,6 +5153,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:]) @@ -9171,6 +9754,619 @@ 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_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 Offset", wireType) + } + m.Offset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Offset |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + 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 != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Func", wireType) + } + m.Func = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Func |= (WindowerSpec_Func(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + 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 @@ -9279,199 +10475,226 @@ 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, + // 3535 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0xcd, 0x6f, 0x23, 0x57, + 0x72, 0x17, 0xc9, 0xe6, 0x57, 0xf1, 0x43, 0x3d, 0x6f, 0xc6, 0x1e, 0x5a, 0xe3, 0x95, 0xc6, 0xbd, + 0xb3, 0x9e, 0x8f, 0xf5, 0x4a, 0x1e, 0x79, 0xd7, 0xf0, 0x7a, 0x77, 0x61, 0xf3, 0x53, 0xa2, 0x86, + 0x22, 0xe5, 0x26, 0xa9, 0x59, 0xfb, 0x90, 0x4e, 0x8b, 0xfd, 0x44, 0xb5, 0xd5, 0xec, 0xc7, 0xe9, + 0x6e, 0x8e, 0xa4, 0x41, 0x80, 0x20, 0x87, 0x9c, 0x13, 0x20, 0x09, 0x72, 0x0b, 0x72, 0xd9, 0x6b, + 0x72, 0xcd, 0x25, 0x01, 0x02, 0xec, 0x61, 0x8e, 0xc9, 0x6d, 0x11, 0x04, 0x83, 0x44, 0x3e, 0xe5, + 0x5f, 0x98, 0x43, 0x10, 0xbc, 0xaf, 0x66, 0x53, 0xdf, 0x9a, 0x59, 0xac, 0x2f, 0xb9, 0x08, 0xec, + 0xfa, 0xf8, 0x75, 0xbd, 0x7a, 0xf5, 0xaa, 0xea, 0x55, 0x0b, 0x96, 0xfc, 0x67, 0xce, 0x8a, 0x65, + 0xfb, 0x81, 0xff, 0xcc, 0xf1, 0x26, 0xee, 0xca, 0xd8, 0x23, 0x03, 0xec, 0xfb, 0xc4, 0xf3, 0x97, + 0xc7, 0x1e, 0x09, 0x08, 0x2a, 0x0d, 0xc8, 0x60, 0xdf, 0x23, 0xe6, 0x60, 0x6f, 0xd9, 0x7f, 0xe6, + 0x2c, 0x4f, 0x45, 0x17, 0x6e, 0x30, 0xea, 0x78, 0x67, 0xc5, 0x1c, 0xdb, 0x5c, 0x78, 0x01, 0x49, + 0x92, 0x65, 0x06, 0xa6, 0xa0, 0xdd, 0x92, 0x34, 0xec, 0x79, 0x21, 0xec, 0x42, 0x49, 0x52, 0x6d, + 0xf2, 0x93, 0x5d, 0xe2, 0x8d, 0xcc, 0x40, 0x72, 0xde, 0xa7, 0x16, 0xf9, 0xcf, 0x9c, 0x1d, 0xd3, + 0xc7, 0x2b, 0x7e, 0xe0, 0x4d, 0x06, 0xc1, 0xc4, 0xc3, 0x96, 0xe0, 0x2e, 0x45, 0xb9, 0xd8, 0x1d, + 0x10, 0x0b, 0x5b, 0x86, 0x65, 0x06, 0x93, 0x91, 0x10, 0xb8, 0x13, 0x15, 0xf8, 0x96, 0xd8, 0xae, + 0x11, 0x1c, 0x8d, 0xb1, 0x60, 0xbe, 0x77, 0x62, 0xb5, 0x11, 0x33, 0x4b, 0x93, 0xc0, 0x76, 0x56, + 0xf6, 0x9c, 0xc1, 0x4a, 0x60, 0x8f, 0xb0, 0x1f, 0x98, 0xa3, 0xb1, 0x5c, 0xc0, 0x90, 0x0c, 0x09, + 0xfb, 0xb9, 0x42, 0x7f, 0x71, 0xaa, 0xf6, 0x17, 0x09, 0x28, 0x6c, 0x49, 0x67, 0x75, 0xc7, 0x78, + 0x80, 0xaa, 0x90, 0xb4, 0xdd, 0xf1, 0x24, 0x28, 0xc5, 0xee, 0x26, 0x1e, 0xe4, 0x56, 0xef, 0x2f, + 0x9f, 0xe7, 0xb9, 0xe5, 0x26, 0x15, 0xeb, 0x1e, 0xb9, 0x03, 0xaa, 0x57, 0x51, 0x5e, 0xbe, 0x5a, + 0x9a, 0xd3, 0xb9, 0x2e, 0x6a, 0x80, 0x32, 0x20, 0x1e, 0x2e, 0xc5, 0xef, 0xc6, 0x1e, 0xe4, 0x56, + 0x3f, 0x3a, 0x1f, 0x23, 0x7c, 0x77, 0x95, 0x78, 0xb8, 0xef, 0xda, 0xc4, 0x15, 0x40, 0x4c, 0x1f, + 0xad, 0x43, 0x8a, 0x4c, 0x02, 0x6a, 0x4d, 0x82, 0x59, 0xf3, 0xe8, 0x7c, 0xa4, 0x0e, 0x93, 0xd3, + 0xc9, 0x24, 0xc0, 0x5e, 0xc4, 0x20, 0xa1, 0x8f, 0xaa, 0xa0, 0x8c, 0x89, 0x1f, 0x94, 0x14, 0x66, + 0xd1, 0xc3, 0x0b, 0x2c, 0x22, 0x7e, 0x20, 0xac, 0x8a, 0xc0, 0x30, 0x65, 0xf4, 0x08, 0x32, 0x7e, + 0x60, 0x0e, 0xb1, 0x61, 0x5b, 0xa5, 0xe4, 0xdd, 0xd8, 0x83, 0x64, 0x65, 0x9e, 0x72, 0x8f, 0x5f, + 0x2d, 0xa5, 0xbb, 0x94, 0xde, 0xac, 0xe9, 0x69, 0x26, 0xd0, 0xb4, 0xd0, 0xa7, 0x90, 0x0f, 0xa3, + 0x90, 0xca, 0xa7, 0x98, 0xfc, 0x4d, 0x21, 0x9f, 0x0b, 0x17, 0xde, 0xac, 0xe9, 0xb9, 0x50, 0xb0, + 0x69, 0x69, 0xff, 0x18, 0x87, 0xf9, 0x13, 0x36, 0xa0, 0x0a, 0xa4, 0x76, 0x6d, 0x27, 0xc0, 0x5e, + 0x29, 0xc6, 0xcc, 0xbf, 0x77, 0xbe, 0xf9, 0xf5, 0xc3, 0xb1, 0x87, 0x7d, 0x7f, 0xea, 0x48, 0xa1, + 0x89, 0xee, 0x01, 0x8c, 0x3d, 0xf2, 0x2d, 0x1e, 0x04, 0x36, 0x71, 0xd9, 0xc6, 0x64, 0x84, 0x44, + 0x84, 0x8e, 0x1e, 0x42, 0x91, 0x3b, 0xcc, 0x18, 0x10, 0x67, 0x32, 0x72, 0x7d, 0xe6, 0xf8, 0x42, + 0x25, 0xae, 0xc6, 0xf4, 0x02, 0xe7, 0x54, 0x39, 0x03, 0x6d, 0x42, 0xde, 0xc3, 0xae, 0x85, 0x3d, + 0x03, 0x1f, 0x8e, 0x3d, 0xbf, 0xa4, 0xb0, 0x1d, 0xba, 0x8e, 0x69, 0x39, 0xae, 0x4f, 0xe9, 0x3e, + 0x7a, 0x1f, 0x52, 0x64, 0x77, 0xd7, 0xc7, 0x01, 0xf3, 0xac, 0x12, 0x6e, 0x1f, 0xa3, 0xa1, 0x05, + 0x48, 0x3a, 0xf6, 0xc8, 0x0e, 0x98, 0x1b, 0x25, 0x93, 0x93, 0xb4, 0xd7, 0x79, 0x40, 0xa7, 0xe3, + 0x08, 0x7d, 0x0e, 0x8a, 0x4b, 0xc8, 0x58, 0xb8, 0xec, 0xc3, 0xf3, 0xed, 0x6a, 0x13, 0x32, 0xa6, + 0x6a, 0xd4, 0xd5, 0x3a, 0xd3, 0x41, 0x4f, 0x20, 0x17, 0x98, 0x3b, 0x0e, 0xd6, 0xb1, 0x69, 0x61, + 0x4f, 0x84, 0xf1, 0x05, 0x41, 0xd3, 0x9b, 0x0a, 0x33, 0x94, 0xa8, 0x36, 0x5a, 0x07, 0xa0, 0x27, + 0x58, 0x60, 0x25, 0x18, 0xd6, 0x83, 0xf3, 0xb1, 0x36, 0x42, 0x59, 0x06, 0x15, 0xd1, 0x45, 0xbf, + 0x84, 0x94, 0x4f, 0x3c, 0x1a, 0x07, 0xca, 0x65, 0x71, 0xd0, 0x65, 0x72, 0x0c, 0x41, 0xe8, 0x50, + 0x3b, 0xcc, 0xe1, 0xd0, 0xc3, 0x43, 0x33, 0x20, 0x1e, 0xf3, 0xf2, 0x85, 0x76, 0x94, 0x43, 0x59, + 0x6e, 0xc7, 0x54, 0x17, 0x55, 0x20, 0x43, 0x05, 0x6d, 0x77, 0x10, 0x94, 0xd2, 0x97, 0xb9, 0xb7, + 0x26, 0x24, 0x19, 0x4a, 0xa8, 0x47, 0x5d, 0x3c, 0xc2, 0xde, 0x10, 0xd3, 0xe5, 0x62, 0xaf, 0x94, + 0xb9, 0xcc, 0xc5, 0x9b, 0x53, 0x61, 0xee, 0xe2, 0x88, 0x36, 0x5d, 0xda, 0x9e, 0xe9, 0xef, 0x09, + 0xac, 0xec, 0x65, 0x4b, 0x5b, 0x0f, 0x65, 0xf9, 0xd2, 0xa6, 0xba, 0xe8, 0x4b, 0x48, 0x3d, 0x37, + 0x9d, 0x09, 0xf6, 0x4b, 0x70, 0x19, 0xca, 0x36, 0x93, 0x0b, 0x23, 0x47, 0xe8, 0x51, 0x5b, 0x76, + 0xcc, 0xc1, 0xfe, 0xae, 0xed, 0x38, 0xd8, 0x2b, 0xe5, 0x2e, 0x43, 0xa9, 0x84, 0xb2, 0xdc, 0x96, + 0xa9, 0x2e, 0x6a, 0x01, 0x78, 0xd8, 0xb4, 0x9a, 0xa3, 0x31, 0xf1, 0x82, 0x52, 0xe1, 0xb2, 0x5c, + 0xaa, 0x87, 0xb2, 0x35, 0x33, 0x30, 0x39, 0xda, 0x54, 0x1f, 0xd5, 0x21, 0xdb, 0xed, 0xf6, 0x9e, + 0x7a, 0x36, 0x8d, 0x9f, 0x22, 0x03, 0xbb, 0x20, 0xb9, 0x87, 0xa2, 0x0c, 0x67, 0xaa, 0x89, 0xbe, + 0x80, 0x74, 0xd7, 0x1c, 0x8d, 0xe9, 0xda, 0xe6, 0x19, 0xc8, 0x8f, 0x2e, 0x00, 0xe1, 0x82, 0x0c, + 0x42, 0x6a, 0xa1, 0x6f, 0x40, 0xe5, 0x3f, 0xa7, 0x01, 0x56, 0x52, 0x19, 0xd2, 0xf2, 0x65, 0x48, + 0x27, 0x42, 0xf2, 0x14, 0x0e, 0x22, 0x70, 0xdb, 0x76, 0x03, 0xec, 0x39, 0xd8, 0x7c, 0x8e, 0x2d, + 0x7e, 0x6a, 0x44, 0x50, 0xdc, 0x60, 0xaf, 0xf8, 0xd9, 0x45, 0xe5, 0xec, 0x4c, 0x45, 0xf6, 0xa6, + 0xf3, 0x50, 0xd1, 0x1f, 0x03, 0x1a, 0xe1, 0xc0, 0xa4, 0x15, 0xb8, 0x87, 0xfd, 0xa0, 0xcb, 0x12, + 0x5a, 0x09, 0xb1, 0x77, 0x7d, 0x7c, 0x51, 0x30, 0x9f, 0xd4, 0x61, 0xaf, 0x39, 0x03, 0x0b, 0xed, + 0xc2, 0xad, 0x28, 0x55, 0xc7, 0x03, 0x6c, 0x3f, 0xc7, 0x5e, 0xe9, 0x26, 0x7b, 0xc7, 0xea, 0xd5, + 0xde, 0x21, 0xb5, 0xd8, 0x5b, 0xce, 0xc4, 0xa3, 0xe1, 0x51, 0xed, 0x6e, 0x8b, 0xf0, 0xb8, 0x75, + 0x59, 0x78, 0x84, 0xa2, 0x3c, 0x3c, 0xc2, 0x47, 0xd4, 0x86, 0xfc, 0x0b, 0x7b, 0xf8, 0xc2, 0x1c, + 0x0a, 0xb7, 0xbf, 0xc3, 0x90, 0x2e, 0xa8, 0xdb, 0xdf, 0x44, 0xa4, 0x19, 0xd8, 0x8c, 0x3e, 0x3d, + 0x4d, 0xa2, 0x3c, 0x75, 0x71, 0x50, 0x7a, 0xf7, 0xb2, 0xd3, 0xb4, 0x15, 0xca, 0xf2, 0xf8, 0x9f, + 0xea, 0xd2, 0xa4, 0x75, 0x60, 0xbb, 0x16, 0x39, 0xc0, 0x5e, 0xe9, 0xf6, 0x65, 0x49, 0xeb, 0xa9, + 0x90, 0xe4, 0x49, 0x4b, 0xea, 0x7d, 0xae, 0xbc, 0xfc, 0xfb, 0xa5, 0xd8, 0x86, 0x92, 0x49, 0xa9, + 0xe9, 0x0d, 0x25, 0x93, 0x57, 0x0b, 0x5a, 0x11, 0xf2, 0xd1, 0xfa, 0xa1, 0xfd, 0x14, 0xde, 0x3d, + 0x7b, 0x73, 0xd1, 0x02, 0xc4, 0x6d, 0x8b, 0x55, 0xa3, 0x6c, 0x05, 0x44, 0x1b, 0x10, 0x6f, 0xd6, + 0xf4, 0xb8, 0x6d, 0x69, 0xeb, 0x50, 0x3a, 0x6f, 0xbb, 0xd0, 0x47, 0x00, 0x3e, 0xaf, 0xb3, 0xb6, + 0xe5, 0xb3, 0xae, 0x2c, 0x5b, 0x29, 0x1c, 0xbf, 0x5a, 0xca, 0x72, 0xec, 0x66, 0xcd, 0xd7, 0xb3, + 0x5c, 0xa0, 0x69, 0xf9, 0xda, 0x5f, 0xc5, 0xa0, 0x38, 0x9b, 0x98, 0x50, 0x15, 0xd2, 0xb2, 0x98, + 0xf3, 0x9e, 0xee, 0x87, 0x17, 0x24, 0x6b, 0xda, 0x83, 0x36, 0xdd, 0x5d, 0x22, 0x4a, 0xac, 0xd4, + 0x44, 0x77, 0x20, 0xeb, 0x99, 0x07, 0xc6, 0xce, 0x51, 0x80, 0xfd, 0x52, 0xfc, 0x6e, 0xe2, 0x41, + 0x5e, 0xcf, 0x78, 0xe6, 0x41, 0x85, 0x3e, 0xa3, 0x25, 0xc8, 0xb8, 0x93, 0x91, 0xe1, 0x91, 0x03, + 0x9f, 0xd5, 0x37, 0x59, 0xa0, 0xd3, 0xee, 0x64, 0xa4, 0x93, 0x03, 0x5f, 0xab, 0xc1, 0xfc, 0x4c, + 0x89, 0x34, 0x5d, 0xf4, 0x18, 0x14, 0x7f, 0x6c, 0xba, 0xa2, 0x3c, 0xdf, 0x8e, 0x98, 0x24, 0x7a, + 0xea, 0x65, 0x2a, 0x26, 0xdb, 0x2f, 0x2a, 0xaa, 0xfd, 0x26, 0x7e, 0x02, 0x86, 0xb5, 0x46, 0x49, + 0x56, 0x6b, 0xcf, 0x29, 0xf3, 0xa2, 0x85, 0xe6, 0x05, 0xba, 0x86, 0xfd, 0x81, 0x67, 0x8f, 0x03, + 0xe2, 0xc9, 0x06, 0x82, 0xa9, 0xa2, 0x0f, 0x20, 0x6b, 0xbb, 0x16, 0x3e, 0x34, 0x6c, 0xeb, 0x90, + 0xd5, 0xfa, 0x82, 0xe0, 0x67, 0x18, 0xb9, 0x69, 0x1d, 0xa2, 0x45, 0x48, 0x7b, 0xf8, 0x39, 0xf6, + 0x7c, 0xcc, 0x16, 0x28, 0x5b, 0x27, 0x49, 0x44, 0x75, 0x48, 0x52, 0x13, 0x65, 0x17, 0x74, 0xd5, + 0x56, 0x21, 0x5c, 0x20, 0xd7, 0x46, 0x3f, 0x04, 0x60, 0x3d, 0x8d, 0xb1, 0x67, 0xbb, 0xbc, 0x11, + 0x4a, 0x08, 0x81, 0x2c, 0xa3, 0xaf, 0xdb, 0x6e, 0x40, 0xbd, 0x6d, 0xfb, 0xc6, 0x60, 0x0f, 0x0f, + 0xf6, 0x59, 0x3b, 0x14, 0x1a, 0x63, 0xfb, 0x55, 0x4a, 0xd4, 0x5e, 0xc7, 0xa1, 0x38, 0xdb, 0x45, + 0xfc, 0xa1, 0xdc, 0xf4, 0x10, 0x8a, 0x0e, 0x21, 0xfb, 0x93, 0xf1, 0x59, 0xed, 0x23, 0xe7, 0xc8, + 0xf6, 0xb1, 0x0a, 0x69, 0xe2, 0xb2, 0xd6, 0xf1, 0xf2, 0x66, 0xe6, 0x74, 0x53, 0x4b, 0x5c, 0x4a, + 0x43, 0xdb, 0x70, 0x83, 0x9b, 0xc4, 0x9b, 0x5c, 0x0e, 0x97, 0xbc, 0x36, 0xdc, 0x3c, 0x03, 0x69, + 0x30, 0x0c, 0x86, 0xfb, 0x73, 0x50, 0xe8, 0x7d, 0x8b, 0xb9, 0xb7, 0xb8, 0xba, 0x74, 0x8e, 0xb7, + 0xa8, 0x8f, 0x7b, 0x47, 0x63, 0x2c, 0x83, 0x94, 0xaa, 0xd0, 0x03, 0x08, 0xd3, 0xe6, 0x0b, 0x7d, + 0x05, 0xf3, 0xa2, 0xa1, 0x26, 0x9e, 0x85, 0x3d, 0xdb, 0x1d, 0x8a, 0x2d, 0xd0, 0x2e, 0xb8, 0xca, + 0x08, 0x49, 0x81, 0x2b, 0x3a, 0x72, 0x49, 0x45, 0xab, 0x80, 0x24, 0x96, 0x31, 0x32, 0x83, 0xc1, + 0x9e, 0xe1, 0x60, 0x77, 0x66, 0x43, 0x54, 0xc9, 0xdf, 0xa4, 0xec, 0x16, 0x76, 0xb5, 0x1d, 0xc8, + 0x47, 0xfb, 0x30, 0x74, 0x1f, 0xe6, 0x99, 0x0c, 0xb6, 0x8c, 0x68, 0x6e, 0x28, 0xe8, 0x45, 0x41, + 0x96, 0xdb, 0xf4, 0x10, 0x54, 0xd9, 0xb2, 0x85, 0x92, 0x71, 0x26, 0x39, 0x2f, 0xe9, 0x42, 0x54, + 0xfb, 0xd7, 0x38, 0xa8, 0x27, 0xb3, 0x39, 0xaa, 0x41, 0x8a, 0x45, 0x8f, 0xcc, 0x3d, 0xd7, 0x8b, + 0x3c, 0xa1, 0x8b, 0x1a, 0x00, 0xf8, 0xd9, 0xcc, 0xfb, 0x73, 0xab, 0x1f, 0x5c, 0x50, 0x9d, 0xb8, + 0xa0, 0x3c, 0x3a, 0xf8, 0x99, 0x5c, 0xcd, 0xd2, 0x34, 0x84, 0xa3, 0xa1, 0x29, 0x03, 0xf8, 0xf7, + 0x14, 0x95, 0x32, 0x7a, 0x92, 0xd7, 0x8f, 0x9e, 0xfb, 0x90, 0x96, 0xb6, 0xbe, 0x3f, 0x9b, 0xb6, + 0xb9, 0xa5, 0x92, 0xa4, 0xfd, 0x67, 0x1c, 0xe6, 0x4f, 0xb4, 0xc4, 0x68, 0x13, 0x0a, 0x0e, 0xde, + 0x7d, 0x8b, 0x48, 0xcb, 0x53, 0xf5, 0x30, 0xce, 0x3a, 0x50, 0xf4, 0xec, 0xe1, 0x5e, 0x04, 0x2f, + 0x7e, 0x4d, 0xbc, 0x02, 0xd3, 0x0f, 0x01, 0x23, 0xce, 0x4d, 0xbe, 0xb5, 0x73, 0xaf, 0x7f, 0x34, + 0xd1, 0x43, 0x28, 0xb8, 0x13, 0xc7, 0x31, 0xf0, 0xb3, 0x89, 0xe9, 0xd8, 0xc1, 0x11, 0xbb, 0xbb, + 0xc8, 0xec, 0x99, 0xa7, 0xac, 0xba, 0xe0, 0x68, 0x7f, 0x1b, 0x87, 0xe2, 0xec, 0x2d, 0x01, 0x3d, + 0x82, 0x79, 0xe6, 0xdd, 0x48, 0x20, 0xc6, 0x22, 0xc9, 0x0d, 0xef, 0x06, 0xf5, 0x30, 0xce, 0x3e, + 0x02, 0x95, 0xbb, 0xee, 0x44, 0xd4, 0x72, 0x61, 0xee, 0xd6, 0xa9, 0xf4, 0xf7, 0xed, 0x97, 0x1f, + 0x43, 0x91, 0x5d, 0xa6, 0xa6, 0xb9, 0x20, 0xea, 0x98, 0x02, 0xe7, 0xc9, 0x53, 0xfe, 0xd7, 0x29, + 0x28, 0xce, 0xf6, 0xe1, 0xe8, 0x03, 0x80, 0xa1, 0x47, 0x78, 0xd2, 0x8f, 0xae, 0x33, 0xcb, 0xa8, + 0x55, 0xe2, 0xf8, 0xe8, 0x8f, 0x20, 0x2f, 0xef, 0x8f, 0x36, 0x11, 0x65, 0x21, 0xb7, 0xfa, 0xd3, + 0xab, 0xde, 0x3e, 0xc3, 0xc7, 0xe9, 0xba, 0x67, 0xf0, 0xd0, 0xc7, 0x22, 0x27, 0x62, 0xcb, 0x88, + 0x98, 0xa2, 0x84, 0xa6, 0xa8, 0x82, 0xbb, 0x16, 0x5a, 0xb4, 0x36, 0x73, 0x48, 0x7f, 0x72, 0x65, + 0x4b, 0x4e, 0x7a, 0x6f, 0xe1, 0x9f, 0x63, 0x90, 0x8b, 0x98, 0x47, 0x81, 0x77, 0x27, 0xee, 0x80, + 0x1d, 0xbe, 0xeb, 0x00, 0x37, 0x26, 0x6e, 0x38, 0x6d, 0xa2, 0x00, 0xe8, 0x6e, 0xe4, 0x96, 0x1d, + 0x9d, 0xd7, 0x4c, 0xef, 0xd0, 0xf7, 0xa0, 0x28, 0x0a, 0xdf, 0x80, 0x38, 0xac, 0x2c, 0xd3, 0xa4, + 0x55, 0xd0, 0xf3, 0x9c, 0x5a, 0x25, 0x0e, 0x2d, 0xca, 0xb7, 0x59, 0x22, 0x61, 0xec, 0x24, 0xcb, + 0xdc, 0xa9, 0x01, 0x63, 0x6c, 0x28, 0x99, 0x84, 0xaa, 0x68, 0x7f, 0x13, 0x07, 0x85, 0xbe, 0x1b, + 0xa9, 0x90, 0x2f, 0xb7, 0xbf, 0x36, 0xda, 0x9d, 0x9e, 0xd1, 0xee, 0xb7, 0x5a, 0xea, 0x1c, 0x4a, + 0x43, 0xa2, 0xbc, 0xbd, 0xa6, 0xc6, 0x50, 0x1e, 0x32, 0x95, 0x4e, 0xa7, 0x65, 0x94, 0xdb, 0x35, + 0x35, 0x8e, 0x72, 0x90, 0x66, 0x4f, 0x1d, 0x5d, 0x4d, 0xa0, 0x22, 0x40, 0xb5, 0xd3, 0xae, 0x96, + 0x7b, 0x46, 0x79, 0x6d, 0x4d, 0x55, 0x50, 0x16, 0x92, 0xd5, 0x4e, 0xbf, 0xdd, 0x53, 0x93, 0x54, + 0x7d, 0xb3, 0xfc, 0x6b, 0x35, 0xcd, 0x7e, 0x34, 0xdb, 0x6a, 0x06, 0x01, 0xa4, 0xba, 0xbd, 0x5a, + 0xad, 0xbe, 0xad, 0x66, 0x29, 0xb1, 0xdb, 0xdf, 0x54, 0x81, 0xc2, 0x75, 0xfb, 0x9b, 0x46, 0xb3, + 0xdd, 0x53, 0x73, 0xf4, 0x4d, 0xdb, 0x65, 0xbd, 0x59, 0x6e, 0x57, 0xeb, 0x6a, 0x9e, 0xb2, 0x7e, + 0xdd, 0xd1, 0x19, 0x72, 0x81, 0xbf, 0xa9, 0xdf, 0xee, 0x19, 0x7a, 0xe7, 0x69, 0x57, 0x2d, 0x32, + 0xbd, 0xaf, 0xf4, 0x5a, 0xb3, 0xd1, 0x50, 0xe7, 0x11, 0x82, 0x62, 0xa3, 0xd9, 0x2e, 0xb7, 0x8c, + 0x50, 0x5b, 0xa5, 0x0b, 0xe2, 0x34, 0xf1, 0xce, 0x1b, 0xa8, 0x00, 0xd9, 0xb2, 0xae, 0x97, 0xbf, + 0x66, 0x88, 0x88, 0xbe, 0x6c, 0xa3, 0xdb, 0x69, 0xb3, 0xa7, 0x9b, 0x94, 0x49, 0x9f, 0x2a, 0xec, + 0xf1, 0x96, 0xf6, 0x11, 0x28, 0x74, 0xaf, 0x51, 0x06, 0x94, 0x72, 0xbf, 0xd7, 0x51, 0xe7, 0x98, + 0xf5, 0xd5, 0x72, 0xab, 0xac, 0xab, 0x31, 0x6a, 0x4c, 0xbb, 0xd3, 0x36, 0xc4, 0x73, 0x5c, 0x7b, + 0x9d, 0x80, 0xe2, 0xec, 0x55, 0x3e, 0x8c, 0xb0, 0x4b, 0x03, 0x61, 0x56, 0xef, 0x54, 0x84, 0x4d, + 0x9b, 0xb7, 0xf8, 0x9b, 0x37, 0x6f, 0x61, 0x83, 0x9a, 0x78, 0xab, 0x06, 0xf5, 0x31, 0x64, 0xac, + 0x89, 0xc7, 0x02, 0x9d, 0xc5, 0x5a, 0xa2, 0xf2, 0x0e, 0x65, 0xbf, 0x7e, 0xb5, 0x54, 0x08, 0xec, + 0x11, 0x5e, 0xae, 0x09, 0xa6, 0x1e, 0x8a, 0xd1, 0x9e, 0x76, 0xb0, 0x37, 0x71, 0xf7, 0x0d, 0xdf, + 0x7e, 0x81, 0x67, 0x7b, 0x5a, 0x46, 0xef, 0xda, 0x2f, 0x30, 0xea, 0x40, 0x9e, 0x04, 0x7b, 0xd8, + 0x33, 0x44, 0xb3, 0x90, 0x7a, 0x83, 0x66, 0x21, 0xc7, 0x10, 0x7a, 0xbc, 0x63, 0xf8, 0x02, 0x32, + 0x1e, 0x36, 0xad, 0xb2, 0xdf, 0xd9, 0x15, 0x23, 0xaa, 0x1f, 0x44, 0xc0, 0x26, 0x81, 0xed, 0x2c, + 0xef, 0x39, 0x83, 0xe5, 0x9e, 0x9c, 0x92, 0xcb, 0xb3, 0x25, 0x95, 0xb4, 0x47, 0x62, 0xfb, 0x73, + 0x90, 0x6e, 0xba, 0xcf, 0x4d, 0xc7, 0xb6, 0x78, 0x04, 0xf0, 0x3c, 0xa8, 0xc6, 0x68, 0xa0, 0x37, + 0x69, 0xdb, 0xa0, 0xc6, 0xb5, 0xef, 0x62, 0x90, 0x69, 0x38, 0xe4, 0x80, 0x6d, 0xfb, 0x63, 0x48, + 0xef, 0x3a, 0xe4, 0xc0, 0x10, 0x97, 0xbd, 0x7c, 0xa5, 0x44, 0x91, 0xff, 0xe3, 0xd5, 0x52, 0x8a, + 0x8a, 0x34, 0x6b, 0xc7, 0xe1, 0x2f, 0x3d, 0x45, 0x05, 0x9b, 0x16, 0xda, 0x64, 0x97, 0x5c, 0xf1, + 0xc5, 0x42, 0xb4, 0x37, 0xf7, 0xaf, 0x30, 0x34, 0x8f, 0x0c, 0xa8, 0x23, 0x00, 0xa8, 0x0f, 0xe9, + 0xa1, 0x19, 0xe0, 0x03, 0xf3, 0x88, 0x5d, 0x56, 0x92, 0x95, 0x5f, 0x88, 0x3d, 0xfa, 0x64, 0x68, + 0x07, 0x7b, 0x93, 0x9d, 0xe5, 0x01, 0x19, 0xad, 0x84, 0xe8, 0xd6, 0xce, 0xf4, 0xf7, 0xca, 0x78, + 0x7f, 0xb8, 0x22, 0xef, 0x5f, 0x6d, 0x62, 0xb1, 0x89, 0xb6, 0xc0, 0xd2, 0x0e, 0x20, 0xb7, 0x41, + 0x76, 0xb6, 0x3c, 0x32, 0xa4, 0x15, 0x08, 0xdd, 0x83, 0xd4, 0xb7, 0x64, 0x47, 0x2e, 0x33, 0x51, + 0x29, 0x88, 0x3b, 0x6d, 0x72, 0x83, 0xec, 0x34, 0x6b, 0x7a, 0xf2, 0x5b, 0xb2, 0xd3, 0xb4, 0xd0, + 0x03, 0xc8, 0x0f, 0x88, 0x1b, 0x78, 0xf6, 0xce, 0x24, 0x1c, 0x3c, 0xc7, 0x65, 0x0a, 0x8f, 0x72, + 0x50, 0x09, 0x14, 0xdf, 0x21, 0x81, 0x30, 0x59, 0xde, 0xfb, 0x1c, 0x12, 0x68, 0xbf, 0x53, 0x00, + 0x9d, 0x1e, 0x6e, 0xa1, 0x1f, 0x41, 0xce, 0x67, 0x03, 0x20, 0x1e, 0x59, 0xf1, 0x88, 0x1e, 0x70, + 0x06, 0x0b, 0xad, 0x35, 0xc8, 0x8c, 0x85, 0xcd, 0xac, 0x38, 0x5e, 0x38, 0xb1, 0x8a, 0x2c, 0x50, + 0x46, 0x84, 0x54, 0x46, 0x6b, 0x90, 0x98, 0x78, 0x76, 0x29, 0xcd, 0xb6, 0xe7, 0x67, 0xd7, 0x99, + 0xc3, 0x2d, 0xf7, 0x3d, 0xbb, 0xee, 0x06, 0xde, 0x91, 0x4e, 0x11, 0xd0, 0xaf, 0x20, 0xc5, 0x3f, + 0x16, 0x89, 0xa9, 0xe7, 0xd2, 0x19, 0x97, 0xdf, 0x66, 0xa7, 0x61, 0x3b, 0xb8, 0xc1, 0xc4, 0xc2, + 0x49, 0x3e, 0x7b, 0x42, 0xdb, 0x61, 0x4b, 0x9d, 0x65, 0xa6, 0x7c, 0x76, 0x2d, 0x53, 0xf8, 0xf9, + 0x60, 0xd6, 0x30, 0xdc, 0x58, 0xd8, 0x64, 0x7f, 0x01, 0xef, 0xf9, 0xfb, 0xf6, 0xd8, 0x18, 0xd9, + 0xbe, 0x4f, 0xef, 0x16, 0xbb, 0xc4, 0xc3, 0xf6, 0xd0, 0x35, 0xf6, 0xf1, 0x11, 0x9f, 0x86, 0xca, + 0x02, 0xf4, 0x2e, 0x15, 0xdb, 0xe4, 0x52, 0x0d, 0x2e, 0xf4, 0x04, 0x1f, 0xf9, 0x0b, 0x26, 0xe4, + 0x22, 0xe8, 0x48, 0x85, 0xc4, 0x3e, 0x3e, 0xe2, 0x13, 0x0f, 0x9d, 0xfe, 0x44, 0xbf, 0x84, 0x24, + 0x1b, 0x92, 0x5e, 0x2f, 0x91, 0xe9, 0x5c, 0xe9, 0xf3, 0xf8, 0x67, 0xb1, 0x85, 0x4f, 0x21, 0x23, + 0x7d, 0x19, 0xc5, 0x4f, 0x72, 0xfc, 0x5b, 0x51, 0xfc, 0x6c, 0x44, 0x6f, 0x43, 0xc9, 0xc4, 0xd4, + 0x38, 0x2f, 0x78, 0x1b, 0x4a, 0x46, 0x51, 0x93, 0x1b, 0x4a, 0x26, 0xa9, 0xa6, 0xb4, 0x7f, 0x8a, + 0x43, 0x61, 0x66, 0xd4, 0x89, 0x3e, 0x84, 0x9c, 0x85, 0x69, 0x7d, 0xe5, 0x49, 0x8e, 0xcf, 0x6b, + 0x44, 0x82, 0x89, 0x30, 0xd0, 0x23, 0x28, 0x1c, 0x98, 0x8e, 0x43, 0xb3, 0x5e, 0xdb, 0x74, 0x09, + 0x1f, 0x7c, 0xc8, 0xcc, 0x36, 0xcb, 0x42, 0xad, 0xd9, 0xe9, 0xc0, 0xc7, 0x57, 0x1c, 0xbb, 0xb2, + 0x09, 0x48, 0xdb, 0x1c, 0xe1, 0xd9, 0x1c, 0x1c, 0x0d, 0xe8, 0xe4, 0x5b, 0x04, 0x34, 0x75, 0xa6, + 0x7c, 0x03, 0x3d, 0x7d, 0xae, 0x39, 0xc2, 0x33, 0xeb, 0x65, 0x14, 0xea, 0x66, 0xec, 0x5a, 0xcc, + 0xa5, 0x79, 0x9d, 0xfe, 0xdc, 0x50, 0x32, 0x71, 0x35, 0xa1, 0xfd, 0x36, 0x06, 0x85, 0x99, 0x31, + 0xe0, 0x95, 0x5d, 0x77, 0x1f, 0xf2, 0x14, 0xd9, 0x18, 0x9b, 0x41, 0x80, 0x3d, 0x9e, 0x13, 0x42, + 0x41, 0xca, 0xd9, 0xe2, 0x0c, 0xf4, 0x2b, 0x48, 0x93, 0xb1, 0x6c, 0x18, 0x4f, 0xe6, 0x70, 0x79, + 0x52, 0xaa, 0xdd, 0xed, 0x0e, 0x17, 0x92, 0x73, 0x10, 0xa1, 0x33, 0xad, 0x3c, 0x6c, 0x30, 0xa5, + 0x9c, 0xaa, 0x3c, 0x6c, 0x34, 0xf5, 0xe7, 0x71, 0x80, 0xee, 0x3e, 0x0e, 0x06, 0x7b, 0x6c, 0x0d, + 0x4f, 0x20, 0xe7, 0xb3, 0x27, 0x23, 0x52, 0xbb, 0x2f, 0xfa, 0xce, 0xc2, 0x84, 0x23, 0x25, 0x1b, + 0xfc, 0x90, 0x82, 0x4a, 0xd3, 0x2b, 0x1c, 0xbf, 0x33, 0x87, 0xe3, 0xb4, 0x4f, 0x00, 0x0d, 0xb1, + 0x8b, 0x3d, 0x33, 0xc0, 0xc6, 0x9e, 0xed, 0x07, 0x64, 0xe8, 0x99, 0xa3, 0x99, 0xd1, 0xd2, 0x0d, + 0xc9, 0x5f, 0x97, 0x6c, 0xf4, 0x19, 0xbc, 0x13, 0xca, 0x1a, 0x23, 0xf3, 0xd0, 0xd8, 0x99, 0x0c, + 0xf6, 0x71, 0xc0, 0x97, 0x26, 0xef, 0xfe, 0x37, 0x43, 0x91, 0x4d, 0xf3, 0xb0, 0xc2, 0x05, 0xd0, + 0x07, 0x90, 0xf5, 0x03, 0x33, 0x30, 0xd8, 0x16, 0x27, 0x23, 0xee, 0xce, 0x50, 0x32, 0x0d, 0x00, + 0xed, 0x4f, 0x20, 0x17, 0x19, 0xd7, 0xa3, 0x06, 0x64, 0xf8, 0x42, 0xc2, 0x9b, 0xfb, 0xa5, 0x4e, + 0x88, 0x54, 0xa3, 0x50, 0xf7, 0xac, 0x24, 0x5d, 0x38, 0x9d, 0xa4, 0xb5, 0xff, 0x89, 0xc3, 0xad, + 0xb3, 0x66, 0xfc, 0x7f, 0x60, 0x3b, 0xd0, 0x9f, 0x02, 0xe2, 0x4f, 0xf2, 0x2e, 0x14, 0x99, 0x14, + 0x7c, 0x75, 0xfc, 0x6a, 0x49, 0x7c, 0x72, 0x10, 0xb7, 0xa1, 0x66, 0xcd, 0x7f, 0xfd, 0x6a, 0xe9, + 0xe7, 0x57, 0xaa, 0xaa, 0x91, 0x0f, 0xfa, 0xcb, 0x52, 0x5b, 0x57, 0xfd, 0x19, 0x38, 0xcb, 0x47, + 0x26, 0x64, 0x58, 0x3a, 0xa6, 0x75, 0x95, 0x6f, 0x6b, 0x43, 0x7e, 0x62, 0x66, 0x69, 0xb1, 0x59, + 0xbb, 0x72, 0x1d, 0x8f, 0xbe, 0x91, 0xd6, 0x71, 0x86, 0xdb, 0xb4, 0xb4, 0xff, 0x55, 0xe0, 0xce, + 0x05, 0x1f, 0x3b, 0xd0, 0xd7, 0x27, 0x46, 0x36, 0xbf, 0x78, 0xa3, 0x6f, 0x26, 0x3c, 0x8b, 0x9f, + 0x98, 0xe3, 0x44, 0xc6, 0xa8, 0xf1, 0xb3, 0xc6, 0xa8, 0xb3, 0xf3, 0xcf, 0xc4, 0xd9, 0xf3, 0xcf, + 0xef, 0x79, 0x46, 0xb3, 0xf0, 0x2f, 0x71, 0x48, 0xb2, 0xc5, 0xa1, 0x2f, 0x41, 0xb1, 0xb0, 0x3f, + 0x78, 0xa3, 0xa1, 0x2a, 0xd3, 0xbc, 0xca, 0x4c, 0x55, 0xfe, 0xe7, 0x42, 0xe2, 0x6d, 0xfe, 0x73, + 0xa1, 0x06, 0x99, 0x70, 0x8a, 0xa3, 0x5c, 0x73, 0x8a, 0x13, 0x6a, 0x4e, 0x2f, 0x11, 0xc9, 0xb7, + 0xb9, 0x44, 0x68, 0xff, 0x1e, 0x83, 0xe2, 0xec, 0x87, 0x1a, 0xf4, 0x25, 0x24, 0xf9, 0x7f, 0x11, + 0xc4, 0xae, 0xfd, 0x5f, 0x04, 0x5c, 0x11, 0xf5, 0x20, 0xcc, 0x98, 0xd6, 0x25, 0x93, 0xc2, 0xd9, + 0x63, 0x18, 0xd9, 0x5f, 0x35, 0x44, 0x90, 0xa3, 0x99, 0xfb, 0xa0, 0xba, 0x93, 0x11, 0x9b, 0x26, + 0x18, 0x63, 0xec, 0x19, 0x43, 0xec, 0xf2, 0x6c, 0xa0, 0x17, 0xdc, 0xc9, 0xa8, 0x4a, 0x1c, 0x7f, + 0x0b, 0x7b, 0x6b, 0xd8, 0xd5, 0x7e, 0x9b, 0x83, 0x7c, 0xf4, 0xa3, 0x11, 0xba, 0x0b, 0xb9, 0xb1, + 0xe9, 0x05, 0x36, 0x1b, 0x59, 0x1c, 0x89, 0xe9, 0x6a, 0x94, 0x84, 0xba, 0x90, 0xe5, 0x1f, 0x96, + 0x1a, 0xa1, 0xa5, 0x2b, 0x57, 0xfb, 0x22, 0x25, 0x1e, 0x1a, 0xd2, 0x05, 0x53, 0x9c, 0x85, 0xef, + 0x14, 0x48, 0x36, 0x3c, 0x5a, 0xb7, 0x9f, 0x80, 0x32, 0x22, 0x96, 0x2c, 0x61, 0x8f, 0xaf, 0x88, + 0xcc, 0x74, 0x97, 0x37, 0x89, 0x15, 0xc6, 0x3c, 0x05, 0x41, 0x5f, 0x41, 0x6a, 0x87, 0x4c, 0x5c, + 0xcb, 0x17, 0xad, 0xdb, 0x27, 0xd7, 0x82, 0xab, 0x30, 0x55, 0x79, 0x02, 0x39, 0xd0, 0xc2, 0x9f, + 0xc5, 0x20, 0xc9, 0x18, 0xe8, 0x1b, 0xc8, 0x32, 0x5a, 0x6f, 0x5a, 0x71, 0x3f, 0xbd, 0x3e, 0x7e, + 0x64, 0x1f, 0xa7, 0x70, 0x91, 0x7f, 0x2b, 0x89, 0x9e, 0x2e, 0x41, 0x5b, 0xf8, 0xbb, 0x18, 0xa4, + 0xb8, 0x71, 0xa8, 0x0d, 0x49, 0x3f, 0x30, 0xbd, 0x40, 0x1c, 0xe6, 0xd5, 0xeb, 0x1b, 0x10, 0x06, + 0x39, 0x85, 0x41, 0xb5, 0x69, 0xdb, 0xf4, 0x46, 0x68, 0xac, 0xd5, 0xd2, 0xee, 0x80, 0x42, 0xf7, + 0x82, 0x5e, 0x36, 0xf5, 0x72, 0x7b, 0xad, 0xae, 0xce, 0xa1, 0x0c, 0x28, 0x6c, 0x00, 0x12, 0xd3, + 0x0e, 0x21, 0x1b, 0xae, 0x1c, 0xdd, 0x86, 0x9b, 0xfd, 0x76, 0xa5, 0xd3, 0x6f, 0xd7, 0xea, 0x35, + 0x63, 0x4b, 0xaf, 0x57, 0xeb, 0xb5, 0x66, 0x7b, 0x4d, 0x9d, 0x43, 0x37, 0x61, 0x7e, 0xbb, 0xdc, + 0xea, 0xd7, 0x23, 0xc4, 0x18, 0x9a, 0x87, 0x5c, 0xb5, 0xaf, 0xeb, 0x75, 0x3e, 0x4d, 0x51, 0xe3, + 0x53, 0xa9, 0x46, 0xa7, 0xd5, 0xea, 0x3c, 0xa5, 0x52, 0x89, 0x59, 0xcc, 0x29, 0x43, 0x59, 0xf8, + 0x4d, 0x1c, 0x32, 0x32, 0x06, 0x51, 0x7d, 0x66, 0xe0, 0xf5, 0xe3, 0xab, 0x2e, 0xf5, 0xe4, 0xb8, + 0xeb, 0x43, 0xc8, 0x99, 0xde, 0xb0, 0x69, 0x1d, 0x76, 0xd9, 0x36, 0x44, 0xb7, 0x2b, 0xca, 0x40, + 0x77, 0x21, 0x63, 0x7a, 0xc3, 0x2a, 0x99, 0x88, 0x0a, 0x11, 0x66, 0x4c, 0x49, 0xfd, 0x3d, 0x25, + 0xbb, 0x0a, 0x24, 0x77, 0x3d, 0xd9, 0x2f, 0x5d, 0xf8, 0x8f, 0x17, 0xa7, 0xb7, 0x50, 0xe7, 0xaa, + 0xda, 0x3f, 0x24, 0xfe, 0x7f, 0xb6, 0x76, 0xc6, 0x6c, 0x8d, 0xbe, 0x4e, 0xef, 0x3c, 0x35, 0xda, + 0xfd, 0xcd, 0x4a, 0x5d, 0x57, 0xdf, 0x61, 0x31, 0x5d, 0x6e, 0x3f, 0x51, 0xdf, 0xa5, 0x9c, 0x5a, + 0xbd, 0xdd, 0xad, 0x1b, 0xec, 0xf9, 0x36, 0x7d, 0xcf, 0x56, 0x5d, 0xaf, 0xb2, 0x40, 0xa5, 0x94, + 0x12, 0x85, 0xaa, 0xf6, 0x37, 0xeb, 0x46, 0xad, 0xd9, 0xed, 0xa9, 0xef, 0x51, 0x9f, 0xb4, 0x7b, + 0xcd, 0x56, 0x5d, 0x5d, 0xa0, 0xab, 0x6e, 0x95, 0xd7, 0xd4, 0x3b, 0x14, 0xae, 0x55, 0x2f, 0xd7, + 0xd4, 0xf7, 0x69, 0x9c, 0x37, 0x9a, 0x7a, 0xb7, 0x67, 0xb0, 0xe0, 0x56, 0x7f, 0x40, 0xf1, 0x5b, + 0xe5, 0xf0, 0x79, 0x91, 0xa2, 0xb5, 0x7b, 0xeb, 0xe2, 0x71, 0xe9, 0x91, 0x26, 0x2f, 0x03, 0xec, + 0x4c, 0xdd, 0x02, 0x75, 0xbd, 0xd5, 0x32, 0xb6, 0x5a, 0xfd, 0x2e, 0xff, 0xb3, 0xfd, 0x58, 0x9d, + 0xab, 0xdc, 0x7b, 0xf9, 0xdf, 0x8b, 0x73, 0x2f, 0x8f, 0x17, 0x63, 0xff, 0x76, 0xbc, 0x18, 0xfb, + 0xdd, 0xf1, 0x62, 0xec, 0xbf, 0x8e, 0x17, 0x63, 0x7f, 0xf9, 0xdd, 0xe2, 0xdc, 0x37, 0x30, 0x0d, + 0x8a, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xb4, 0x50, 0x99, 0x13, 0x93, 0x2a, 0x00, 0x00, } diff --git a/pkg/sql/distsqlrun/processors.proto b/pkg/sql/distsqlrun/processors.proto index 6f0623b2bdeb..28560386e42b 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; } @@ -820,3 +821,114 @@ 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 processors that performs computations +// of window functions that have the same PARTITION BY clause. For a particular +// windowFn, the processor puts the output 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 Func { + // These mirror aggregate functions from AggregatorSpec. + ANY_NOT_NULL = 0; + AVG = 1; + BOOL_AND = 2; + BOOL_OR = 3; + CONCAT_AGG = 4; + COUNT = 5; + MAX = 7; + MIN = 8; + STDDEV = 9; + SUM = 10; + SUM_INT = 11; + VARIANCE = 12; + XOR_AGG = 13; + COUNT_ROWS = 14; + SQRDIFF = 15; + FINAL_VARIANCE = 16; + FINAL_STDDEV = 17; + ARRAY_AGG = 18; + + JSON_AGG = 19; + // JSONB_AGG is an alias for JSON_AGG, they do the same thing. + JSONB_AGG = 20; + + // These mirror window functions from window_builtins.go. + ROW_NUMBER = 21; + RANK = 22; + DENSE_RANK = 23; + PERCENT_RANK = 24; + CUME_DIST = 25; + NTILE = 26; + LAG = 27; + LEAD = 28; + FIRST_VALUE = 29; + LAST_VALUE = 30; + NTH_VALUE = 31; + } + + // 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 is the mode of specifying frame in terms of logical range (e.g. 100 units cheaper). + RANGE = 0; + // ROWS is the mode of specifying frame in terms of physical offsets (e.g. 1 row before etc). + ROWS = 1; + } + + // BoundType indicates which type of boundary is used. + enum BoundType { + UNBOUNDED_PRECEDING = 0; // represents UNBOUNDED PRECEDING type of boundary + VALUE_PRECEDING = 1; // represents 'value' PRECEDING type of boundary + CURRENT_ROW = 2; // represents CURRENT ROW type of boundary + VALUE_FOLLOWING = 3; // represents 'value' FOLLOWING type of boundary + UNBOUNDED_FOLLOWING = 4; // represents UNBOUNDED FOLLOWING type of boundary + } + + // Bound specifies the offset and the type of boundary. + message Bound { + optional BoundType boundType = 1 [(gogoproto.nullable) = false]; + + optional uint32 offset = 2 [(gogoproto.nullable) = false]; + } + + // Bounds specifies boundaries of the window frame. + message Bounds { + 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 indicates from which index arguments to this window function start. + 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 a window function. + optional Ordering ordering = 4 [(gogoproto.nullable) = false]; + + // Frame specifies over which frame this window function should be 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..6993cae48688 --- /dev/null +++ b/pkg/sql/distsqlrun/windower.go @@ -0,0 +1,726 @@ +// 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" + "hash/crc32" + "sort" + "strings" + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "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 return 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 == WindowerSpec_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() + } + + _, builtins := builtins.GetBuiltinProperties(strings.ToLower(fn.String())) + 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", fn, 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[int][]sqlbase.EncDatumRow + partitions [][]tree.IndexedRow + windowFns []*windowFunc + + populated bool + outputRows []tree.Datums + rowIter int +} + +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[int][]sqlbase.EncDatumRow) + w.partitionBy = spec.PartitionBy + w.windowFns = make([]*windowFunc, 0, len(windowFns)) + w.outputTypes = make([]sqlbase.ColumnType, 0, len(w.inputTypes)) + + // 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:]...) + + 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 { + 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[0] = append(w.encodedPartitions[0], 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 { + panic(fmt.Sprintf("unexpected error while encoding: %v", err)) + } + } + + bucket := int(crc32.Update(0, crc32Table, w.scratch)) + w.encodedPartitions[bucket] = append(w.encodedPartitions[bucket], w.rowAlloc.CopyRow(row)) + } + } + + return windowerEmittingRows, nil, nil +} + +// decodePartitions loops over all encodedPartitions and decodes each row in +// each encoded partition. It should be called after accumulation of rows is +// complete. +func (w *windower) decodePartitions() { + w.partitions = make([][]tree.IndexedRow, 0, len(w.encodedPartitions)) + rowIdx := 0 + for _, encodedPartition := range w.encodedPartitions { + partition := make([]tree.IndexedRow, 0, len(encodedPartition)) + for _, encRow := range encodedPartition { + // Set up Idx of IndexedRow as row's index among all rows of all partitions. + partition = append(partition, tree.IndexedRow{Idx: rowIdx, Row: w.decodeRow(encRow)}) + rowIdx++ + } + w.partitions = append(w.partitions, partition) + } + + // We no longer need encoded partitions, so we free up the memory. + w.encodedPartitions = nil +} + +// decodeRow decodes given encoded row. +func (w *windower) decodeRow(encRow sqlbase.EncDatumRow) tree.Datums { + row := make(tree.Datums, 0, len(encRow)) + for col, encDatum := range encRow { + if err := encDatum.EnsureDecoded(&w.inputTypes[col], &w.datumAlloc); err != nil { + panic(fmt.Sprintf("unexpected error while ensuring decoded: %v", err)) + } + row = append(row, encDatum.Datum) + } + return row +} + +// encodeRow encodes given row. +func (w *windower) encodeRow(row tree.Datums) sqlbase.EncDatumRow { + encRow := make(sqlbase.EncDatumRow, len(row)) + for idx, datum := range row { + encRow[idx] = sqlbase.DatumToEncDatum(w.outputTypes[idx], datum) + } + return encRow +} + +// 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 + } + + w.decodePartitions() + + if err := w.computeWindowFunctions(w.ctx, w.evalCtx); err != nil { + w.moveToDraining(err) + return windowerStateUnknown, nil, nil + } + w.populated = true + } + + if w.rowIter < len(w.outputRows) { + res := w.encodeRow(w.outputRows[w.rowIter]) + w.rowIter++ + return windowerEmittingRows, w.processRowHelper(res), nil + } + + w.moveToDraining(nil /* err */) + return windowerStateUnknown, nil, nil + } + + 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 { + numberOfRows := 0 + for _, partition := range w.partitions { + numberOfRows += len(partition) + } + w.outputRows = make([]tree.Datums, 0, numberOfRows) + for idx := 0; idx < numberOfRows; idx++ { + w.outputRows = append(w.outputRows, make(tree.Datums, 0, len(w.outputTypes))) + } + + var peerGrouper peerGroupChecker + + // 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([][][]tree.IndexedRow, 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 + } + } + } + + // inputColIdx is the index of the column that should be processed next. + inputColIdx := 0 + + for windowFnIdx, windowFn := range w.windowFns { + frameRun := &tree.WindowFrameRun{ + ArgCount: windowFn.argCount, + ArgIdxStart: windowFn.argIdxStart, + } + + if windowFn.frame != nil { + frameRun.Frame = convertToWindowFrame(*windowFn.frame) + frameRun.StartBoundOffset = int(windowFn.frame.Bounds.Start.Offset) + if windowFn.frame.Bounds.End != nil { + frameRun.EndBoundOffset = int(windowFn.frame.Bounds.End.Offset) + } + } + + for partitionIdx, partition := range w.partitions { + builtin := windowFn.create(evalCtx) + defer builtin.Close(ctx, evalCtx) + + 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 partitionIdx == 0 { + sortedPartitionsCache[windowFnIdx] = make([][]tree.IndexedRow, 0, len(w.partitions)) + } + sortedPartitionsCache[windowFnIdx] = append(sortedPartitionsCache[windowFnIdx], w.copyPartition(partition)) + } + } + } else if frameRun.Frame != nil && frameRun.Frame.Mode == tree.ROWS { + // If ORDER BY clause is not provided and Frame is specified with ROWS mode, + // any row has no peers. + peerGrouper = noPeers{} + } else { + // If ORDER BY clause is not provided and either no Frame is provided or Frame is + // specified with RANGE mode, 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 < len(partition) { + // 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 + } + // All window functions are sorted by their argIdxStart, + // so we simply "copy" all datums up to windowFn.ArgIdxStart + // of the current row. + currentRowIdx := frameRun.Rows[frameRun.RowIdx].Idx + w.outputRows[currentRowIdx] = append(w.outputRows[currentRowIdx], frameRun.Rows[frameRun.RowIdx].Row[inputColIdx:windowFn.argIdxStart]...) + + // We put the result of window function computation at windowFn.argIdxStart. + w.outputRows[currentRowIdx] = append(w.outputRows[currentRowIdx], res) + } + } + } + // We consume all arguments of windowFn. + inputColIdx = windowFn.argIdxStart + windowFn.argCount + } + + // We simply copy all columns that come after arguments + // to all window function. + for _, partition := range w.partitions { + for _, row := range partition { + currentRowIdx := row.Idx + w.outputRows[currentRowIdx] = append(w.outputRows[currentRowIdx], row.Row[inputColIdx:]...) + } + } + + // We no longer need partitions, so we release its underlying memory. + w.partitions = nil + + return nil +} + +// copyPartition returns a copy of the given partition. +func (w *windower) copyPartition(partition []tree.IndexedRow) []tree.IndexedRow { + ret := make([]tree.IndexedRow, 0, len(partition)) + for _, row := range partition { + ir := tree.IndexedRow{Idx: row.Idx, Row: w.datumAlloc.NewDatums(len(row.Row))} + for i, datum := range row.Row { + ir.Row[i] = datum + } + ret = append(ret, ir) + } + return ret +} + +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 []tree.IndexedRow + ordering Ordering +} + +// partitionSorter implements the sort.Interface interface. +func (n *partitionSorter) Len() int { return len(n.rows) } +func (n *partitionSorter) Swap(i, j int) { + n.rows[i], n.rows[j] = n.rows[j], n.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[i], n.rows[j] + for _, o := range n.ordering.Columns { + da := ra.Row[o.ColIdx] + db := rb.Row[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 } + +type noPeers struct{} + +// noPeers implements the peerGroupChecker interface. +func (noPeers) InSameGroup(i, j int) bool { return false } + +// 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 +} + +// mapToSpecMode maps tree.WindowFrameMode to WindowerSpec_Frame_Mode. +func mapToSpecMode(m tree.WindowFrameMode) WindowerSpec_Frame_Mode { + switch m { + case tree.RANGE: + return WindowerSpec_Frame_RANGE + case tree.ROWS: + return WindowerSpec_Frame_ROWS + default: + panic("unexpected WindowFrameMode") + } +} + +// mapToSpecBoundType maps tree.WindowFrameBoundType to WindowerSpec_Frame_BoundType. +func mapToSpecBoundType(bt tree.WindowFrameBoundType) WindowerSpec_Frame_BoundType { + switch bt { + case tree.UnboundedPreceding: + return WindowerSpec_Frame_UNBOUNDED_PRECEDING + case tree.ValuePreceding: + return WindowerSpec_Frame_VALUE_PRECEDING + case tree.CurrentRow: + return WindowerSpec_Frame_CURRENT_ROW + case tree.ValueFollowing: + return WindowerSpec_Frame_VALUE_FOLLOWING + case tree.UnboundedFollowing: + return WindowerSpec_Frame_UNBOUNDED_FOLLOWING + default: + panic("unexpected WindowFrameBoundType") + } +} + +// convertToSpecBounds produces WindowerSpec_Frame_Bounds based on tree.WindowFrameBounds. +// If offset expr's are present, it evaluates them and saves the results in the spec. +func convertToSpecBounds( + b tree.WindowFrameBounds, evalCtx *tree.EvalContext, +) (WindowerSpec_Frame_Bounds, error) { + bounds := WindowerSpec_Frame_Bounds{} + + if b.StartBound == nil { + panic("unexpected: StartBound is nil") + } + bounds.Start = 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 + } + startOffset := int(tree.MustBeDInt(dStartOffset)) + if startOffset < 0 { + return bounds, pgerror.NewErrorf(pgerror.CodeInvalidParameterValueError, "frame starting offset must not be negative") + } + bounds.Start.Offset = uint32(startOffset) + } + + if b.EndBound != nil { + bounds.End = &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 + } + endOffset := int(tree.MustBeDInt(dEndOffset)) + if endOffset < 0 { + return bounds, pgerror.NewErrorf(pgerror.CodeInvalidParameterValueError, "frame ending offset must not be negative") + } + bounds.End.Offset = uint32(endOffset) + } + } + + return bounds, nil +} + +// ConvertToSpec produces WindowerSpec_Frame based on tree.WindowFrame. +func ConvertToSpec(f tree.WindowFrame, evalCtx *tree.EvalContext) (WindowerSpec_Frame, error) { + frame := WindowerSpec_Frame{Mode: mapToSpecMode(f.Mode)} + bounds, err := convertToSpecBounds(f.Bounds, evalCtx) + frame.Bounds = bounds + return frame, err +} + +// mapToWindowFrameMode maps WindowerSpec_Frame_Mode to tree.WindowFrameMode. +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") + } +} + +// convertToWindowFrameBoundType maps WindowerSpec_Frame_BoundType to tree.WindowFrameBoundType. +func convertToWindowFrameBoundType(bt WindowerSpec_Frame_BoundType) tree.WindowFrameBoundType { + switch bt { + case WindowerSpec_Frame_UNBOUNDED_PRECEDING: + return tree.UnboundedPreceding + case WindowerSpec_Frame_VALUE_PRECEDING: + return tree.ValuePreceding + case WindowerSpec_Frame_CURRENT_ROW: + return tree.CurrentRow + case WindowerSpec_Frame_VALUE_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 - offsetExpr's are ignored since frameRun will have +// evaluated offsets. +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 +} + +// convertToWindowFrame produces tree.WindowFrame based on WindowerSpec_Frame. +func convertToWindowFrame(f WindowerSpec_Frame) *tree.WindowFrame { + return &tree.WindowFrame{Mode: mapToWindowFrameMode(f.Mode), Bounds: convertToWindowFrameBounds(f.Bounds)} +} diff --git a/pkg/sql/logictest/testdata/planner_test/window b/pkg/sql/logictest/testdata/planner_test/window index 54e3fb72ea05..c8973dd3653b 100644 --- a/pkg/sql/logictest/testdata/planner_test/window +++ b/pkg/sql/logictest/testdata/planner_test/window @@ -133,13 +133,14 @@ sort · · │ window 1 (variance((d)[decimal]) OVER (PARTITION BY (v)[int], (100)[int]))[decimal] · · │ render 1 ((k)[int] + (stddev((d)[decimal]) OVER (PARTITION BY (v)[int], ('a')[string]))[decimal])[decimal] · · │ render 2 (variance((d)[decimal]) OVER (PARTITION BY (v)[int], (100)[int]))[decimal] · · - └── render · · (k int, d decimal, d decimal, "?column?" int, "?column?" string, "?column?" int) d=d; "?column?"=CONST; "?column?"=CONST; k!=NULL; key(k) + └── render · · (k int, d decimal, d decimal, "?column?" int, "?column?" string, "?column?" int, k int) k=k; d=d; "?column?"=CONST; "?column?"=CONST; k!=NULL; key(k) │ render 0 (k)[int] · · │ render 1 (d)[decimal] · · │ render 2 (d)[decimal] · · │ render 3 (v)[int] · · │ render 4 ('a')[string] · · │ render 5 (100)[int] · · + │ render 6 (k)[int] · · └── scan · · (k int, v int, w[omitted] int, f[omitted] float, d decimal, s[omitted] string, b[omitted] bool) k!=NULL; key(k) · table kv@primary · · · spans ALL · · diff --git a/pkg/sql/sem/builtins/window_builtins.go b/pkg/sql/sem/builtins/window_builtins.go index 1488bf3b9ed6..34bfcdca07d6 100644 --- a/pkg/sql/sem/builtins/window_builtins.go +++ b/pkg/sql/sem/builtins/window_builtins.go @@ -144,6 +144,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) { diff --git a/pkg/sql/targets.go b/pkg/sql/targets.go index f3c3ffa07069..03fe521642d2 100644 --- a/pkg/sql/targets.go +++ b/pkg/sql/targets.go @@ -108,15 +108,34 @@ func (s *renderNode) isRenderEquivalent(exprStr string, j int) bool { // returned instead. func (s *renderNode) addOrReuseRender( col sqlbase.ResultColumn, expr tree.TypedExpr, reuseExistingRender bool, +) (colIdx int) { + return s.addOrReuseRenderStartingFromIdx(col, expr, reuseExistingRender, 0 /* idx */) +} + +// addOrReuseRenders adds the given result columns to the select render list and +// returns their column indices. If an expression is already rendered, and the +// reuse flag is true, no new render is added and the index of the existing +// column is returned instead. +func (s *renderNode) addOrReuseRenders( + cols sqlbase.ResultColumns, exprs []tree.TypedExpr, reuseExistingRender bool, +) (colIdxs []int) { + return s.addOrReuseRendersStartingFromIdx(cols, exprs, reuseExistingRender, 0 /* idx */) +} + +// addOrReuseRenderStartingFromIdx adds the given result column to the select +// render list and returns its column index. If the expression is already +// rendered by a render with index not smaller than idx and the reuse flag is true, +// no new render is added and the index of the existing column is returned instead. +func (s *renderNode) addOrReuseRenderStartingFromIdx( + col sqlbase.ResultColumn, expr tree.TypedExpr, reuseExistingRender bool, idx int, ) (colIdx int) { exprStr := symbolicExprStr(expr) - if reuseExistingRender && len(s.render) > 0 { - // Now, try to find an equivalent render. We use the syntax - // representation as approximation of equivalence. At this - // point the expressions must have underwent name resolution - // already so that comparison occurs after replacing column names - // to IndexedVars. - for j := range s.render { + if reuseExistingRender { + for j := idx; j < len(s.render); j++ { + // Now, try to find an equivalent render starting from idx. We use + // the syntax representation as approximation of equivalence. At this point + // the expressions must have undergone name resolution already so that + // comparison occurs after replacing column names to IndexedVars. if s.isRenderEquivalent(exprStr, j) && s.render[j].ResolvedType() == col.Typ { return j } @@ -124,20 +143,19 @@ func (s *renderNode) addOrReuseRender( } s.addRenderColumn(expr, exprStr, col) - return len(s.render) - 1 } -// addOrReuseRenders adds the given result columns to the select render list and -// returns their column indices. If an expression is already rendered, and the -// reuse flag is true, no new render is added and the index of the existing -// column is returned instead. -func (s *renderNode) addOrReuseRenders( - cols sqlbase.ResultColumns, exprs []tree.TypedExpr, reuseExistingRender bool, +// addOrReuseRendersStartingFromIdx adds the given result columns to the select +// render list and returns their column indices. If an expression is already +// rendered by a render with index not smaller than idx and the reuse flag is true, +// no new render is added and the index of the existing column is returned instead. +func (s *renderNode) addOrReuseRendersStartingFromIdx( + cols sqlbase.ResultColumns, exprs []tree.TypedExpr, reuseExistingRender bool, idx int, ) (colIdxs []int) { colIdxs = make([]int, len(cols)) for i := range cols { - colIdxs[i] = s.addOrReuseRender(cols[i], exprs[i], reuseExistingRender) + colIdxs[i] = s.addOrReuseRenderStartingFromIdx(cols[i], exprs[i], reuseExistingRender, idx) } return colIdxs } diff --git a/pkg/sql/window.go b/pkg/sql/window.go index 70768e61e721..6b1c154787b1 100644 --- a/pkg/sql/window.go +++ b/pkg/sql/window.go @@ -53,6 +53,13 @@ type windowNode struct { aggContainer windowNodeAggContainer ivarHelper *tree.IndexedVarHelper + // numRendersNotToBeReused indicates the number of renders that are being used + // as arguments to window functions plus (possibly) some columns that are simply + // being passed through windowNode (i.e. with no window functions). We do not + // want to reuse these renders because they might be "consumed" by DistSQL + // windower processors. + numRendersNotToBeReused int + run windowRun } @@ -279,6 +286,8 @@ func (p *planner) constructWindowDefinitions( } n.run.windowFrames = make([]*tree.WindowFrame, len(n.funcs)) + n.numRendersNotToBeReused = len(s.render) + // Construct window definitions for each window function application. for idx, windowFn := range n.funcs { windowDef, err := constructWindowDef(*windowFn.expr.WindowDef, namedWindowSpecs) @@ -295,7 +304,7 @@ func (p *planner) constructWindowDefinitions( return err } - colIdxs := s.addOrReuseRenders(cols, exprs, true) + colIdxs := s.addOrReuseRendersStartingFromIdx(cols, exprs, true, n.numRendersNotToBeReused) windowFn.partitionIdxs = append(windowFn.partitionIdxs, colIdxs...) } @@ -313,7 +322,7 @@ func (p *planner) constructWindowDefinitions( direction = encoding.Descending } - colIdxs := s.addOrReuseRenders(cols, exprs, true) + colIdxs := s.addOrReuseRendersStartingFromIdx(cols, exprs, true, n.numRendersNotToBeReused) for _, idx := range colIdxs { ordering := sqlbase.ColumnOrderInfo{ ColIdx: idx, @@ -480,13 +489,13 @@ func (n *windowNode) replaceIndexVarsAndAggFuncs(s *renderNode) { Name: t.String(), Typ: t.ResolvedType(), } - colIdx := s.addOrReuseRender(col, t, true) + colIdx := s.addOrReuseRenderStartingFromIdx(col, t, true, n.numRendersNotToBeReused) n.colContainer.idxMap[t.Idx] = colIdx return nil, false, ivarHelper.IndexedVar(t.Idx) case *tree.FuncExpr: // All window function applications will have been replaced by // windowFuncHolders at this point, so if we see an aggregate - // function in the window renders, it is above a window function. + // function in the window renders, it is above a windowing level. if t.GetAggregateConstructor() != nil { // We add a new render to the source renderNode for each new aggregate // function we see.