From 4f38f0e0602716e5c2c8d14b7eaef219f60e9c4a Mon Sep 17 00:00:00 2001 From: Bilal Akhtar Date: Wed, 3 Jun 2020 17:45:21 -0400 Subject: [PATCH 01/46] storage: Remove extra vars from runExportToSst A previous change (#49721) adding a benchmark for ExportToSst left in some extra unused variables in a helper method. This change cleans that up. Release note: None. --- pkg/storage/bench_pebble_test.go | 9 ++------- pkg/storage/bench_test.go | 8 +------- 2 files changed, 3 insertions(+), 14 deletions(-) diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index 286e82d2bef5..051899a2801c 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -95,7 +95,6 @@ func BenchmarkExportToSst(b *testing.B) { numKeys := []int{64, 512, 1024, 8192, 65536} numRevisions := []int{1, 10, 100} exportAllRevisions := []bool{false, true} - contention := []bool{false, true} engineMakers := []struct { name string create engineMaker @@ -111,12 +110,8 @@ func BenchmarkExportToSst(b *testing.B) { for _, numRevision := range numRevisions { b.Run(fmt.Sprintf("numRevisions=%d", numRevision), func(b *testing.B) { for _, exportAllRevisionsVal := range exportAllRevisions { - b.Run(fmt.Sprintf("exportAllRevisions=%t", exportAllRevisions), func(b *testing.B) { - for _, contentionVal := range contention { - b.Run(fmt.Sprintf("contention=%t", contentionVal), func(b *testing.B) { - runExportToSst(context.Background(), b, engineImpl.create, numKey, numRevision, exportAllRevisionsVal, contentionVal) - }) - } + b.Run(fmt.Sprintf("exportAllRevisions=%t", exportAllRevisionsVal), func(b *testing.B) { + runExportToSst(b, engineImpl.create, numKey, numRevision, exportAllRevisionsVal) }) } }) diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index e05cc4bd9c11..bb1c6f578fa9 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -985,13 +985,7 @@ func runBatchApplyBatchRepr( } func runExportToSst( - ctx context.Context, - b *testing.B, - emk engineMaker, - numKeys int, - numRevisions int, - exportAllRevisions bool, - contention bool, + b *testing.B, emk engineMaker, numKeys int, numRevisions int, exportAllRevisions bool, ) { dir, cleanup := testutils.TempDir(b) defer cleanup() From a954942f8677f08c15087d52247cc65e8cb53366 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 2 Jun 2020 14:02:03 -0700 Subject: [PATCH 02/46] sql: extract out planning of table readers into a separate method This commit extract most of the logic of `createTableReaders` into a separate method that performs the actual physical planning of table readers given already created `TableReaderSpec`s. This will allow for reuse in the follow-up work. This commit additionally slightly refactors a few other things for reuse. Release note: None --- pkg/sql/distsql_physical_planner.go | 160 ++++++++++++++++++---------- 1 file changed, 104 insertions(+), 56 deletions(-) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 63d0c841ac6e..3fb4d1135c91 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -908,38 +908,29 @@ func tableOrdinal( panic(fmt.Sprintf("column %d not in desc.Columns", colID)) } -// getScanNodeToTableOrdinalMap returns a map from scan node column ordinal to -// table reader column ordinal. Returns nil if the map is identity. -// -// scanNodes can have columns set up in a few different ways, depending on the -// colCfg. The heuristic planner always creates scanNodes with all public -// columns (even if some of them aren't even in the index we are scanning). -// The optimizer creates scanNodes with a specific set of wanted columns; in -// this case we have to create a map from scanNode column ordinal to table -// column ordinal (which is what the TableReader uses). -func getScanNodeToTableOrdinalMap(n *scanNode) []int { - if n.colCfg.wantedColumns == nil { - return nil - } - if n.colCfg.addUnwantedAsHidden { - panic("addUnwantedAsHidden not supported") - } - res := make([]int, len(n.cols)) +// toTableOrdinals returns a mapping from column ordinals in cols to table +// reader column ordinals. +func toTableOrdinals( + cols []sqlbase.ColumnDescriptor, + desc *sqlbase.ImmutableTableDescriptor, + visibility execinfrapb.ScanVisibility, +) []int { + res := make([]int, len(cols)) for i := range res { - res[i] = tableOrdinal(n.desc, n.cols[i].ID, n.colCfg.visibility) + res[i] = tableOrdinal(desc, cols[i].ID, visibility) } return res } -// getOutputColumnsFromScanNode returns the indices of the columns that are -// returned by a scanNode. +// getOutputColumnsFromColsForScan returns the indices of the columns that are +// returned by a scanNode or a tableReader. // If remap is not nil, the column ordinals are remapped accordingly. -func getOutputColumnsFromScanNode(n *scanNode, remap []int) []uint32 { - outputColumns := make([]uint32, 0, len(n.cols)) +func getOutputColumnsFromColsForScan(cols []sqlbase.ColumnDescriptor, remap []int) []uint32 { + outputColumns := make([]uint32, 0, len(cols)) // TODO(radu): if we have a scan with a filter, cols will include the // columns needed for the filter, even if they aren't needed for the next // stage. - for i := 0; i < len(n.cols); i++ { + for i := 0; i < len(cols); i++ { colIdx := i if remap != nil { colIdx = remap[i] @@ -1040,66 +1031,123 @@ func (dsp *DistSQLPlanner) CheckNodeHealthAndVersion( func (dsp *DistSQLPlanner) createTableReaders( planCtx *PlanningCtx, n *scanNode, ) (*PhysicalPlan, error) { - scanNodeToTableOrdinalMap := getScanNodeToTableOrdinalMap(n) + // scanNodeToTableOrdinalMap is a map from scan node column ordinal to + // table reader column ordinal. + // + // scanNodes can have columns set up in a few different ways, depending on the + // colCfg. The heuristic planner always creates scanNodes with all public + // columns (even if some of them aren't even in the index we are scanning). + // The optimizer creates scanNodes with a specific set of wanted columns; in + // this case we have to create a map from scanNode column ordinal to table + // column ordinal (which is what the TableReader uses). + var scanNodeToTableOrdinalMap []int + if n.colCfg.addUnwantedAsHidden { + panic("addUnwantedAsHidden not supported") + } else if n.colCfg.wantedColumns != nil { + scanNodeToTableOrdinalMap = toTableOrdinals(n.cols, n.desc, n.colCfg.visibility) + } spec, post, err := initTableReaderSpec(n, planCtx, scanNodeToTableOrdinalMap) if err != nil { return nil, err } - var spanPartitions []SpanPartition + var p PhysicalPlan + err = dsp.planTableReaders( + planCtx, + &p, + &tableReaderPlanningInfo{ + spec: spec, + post: post, + desc: n.desc, + spans: n.spans, + reverse: n.reverse, + scanVisibility: n.colCfg.visibility, + maxResults: n.maxResults, + estimatedRowCount: n.estimatedRowCount, + reqOrdering: n.reqOrdering, + cols: n.cols, + colsToTableOrdrinalMap: scanNodeToTableOrdinalMap, + }, + ) + return &p, err +} + +// tableReaderPlanningInfo is a utility struct that contains the information +// needed to perform the physical planning of table readers once the specs have +// been created. See scanNode to get more context on some of the fields. +type tableReaderPlanningInfo struct { + spec *execinfrapb.TableReaderSpec + post execinfrapb.PostProcessSpec + desc *sqlbase.ImmutableTableDescriptor + spans []roachpb.Span + reverse bool + scanVisibility execinfrapb.ScanVisibility + maxResults uint64 + estimatedRowCount uint64 + reqOrdering ReqOrdering + cols []sqlbase.ColumnDescriptor + colsToTableOrdrinalMap []int +} + +func (dsp *DistSQLPlanner) planTableReaders( + planCtx *PlanningCtx, p *PhysicalPlan, info *tableReaderPlanningInfo, +) error { + var ( + spanPartitions []SpanPartition + err error + ) if planCtx.isLocal { - spanPartitions = []SpanPartition{{dsp.nodeDesc.NodeID, n.spans}} - } else if n.hardLimit == 0 { + spanPartitions = []SpanPartition{{dsp.nodeDesc.NodeID, info.spans}} + } else if info.post.Limit == 0 { // No hard limit - plan all table readers where their data live. Note // that we're ignoring soft limits for now since the TableReader will // still read too eagerly in the soft limit case. To prevent this we'll // need a new mechanism on the execution side to modulate table reads. // TODO(yuzefovich): add that mechanism. - spanPartitions, err = dsp.PartitionSpans(planCtx, n.spans) + spanPartitions, err = dsp.PartitionSpans(planCtx, info.spans) if err != nil { - return nil, err + return err } } else { // If the scan has a hard limit, use a single TableReader to avoid // reading more rows than necessary. - nodeID, err := dsp.getNodeIDForScan(planCtx, n.spans, n.reverse) + nodeID, err := dsp.getNodeIDForScan(planCtx, info.spans, info.reverse) if err != nil { - return nil, err + return err } - spanPartitions = []SpanPartition{{nodeID, n.spans}} + spanPartitions = []SpanPartition{{nodeID, info.spans}} } - var p PhysicalPlan stageID := p.NewStageID() p.ResultRouters = make([]physicalplan.ProcessorIdx, len(spanPartitions)) p.Processors = make([]physicalplan.Processor, 0, len(spanPartitions)) - returnMutations := n.colCfg.visibility == execinfra.ScanVisibilityPublicAndNotPublic + returnMutations := info.scanVisibility == execinfra.ScanVisibilityPublicAndNotPublic for i, sp := range spanPartitions { var tr *execinfrapb.TableReaderSpec if i == 0 { // For the first span partition, we can just directly use the spec we made // above. - tr = spec + tr = info.spec } else { // For the rest, we have to copy the spec into a fresh spec. tr = physicalplan.NewTableReaderSpec() // Grab the Spans field of the new spec, and reuse it in case the pooled // TableReaderSpec we got has pre-allocated Spans memory. newSpansSlice := tr.Spans - *tr = *spec + *tr = *info.spec tr.Spans = newSpansSlice } for j := range sp.Spans { tr.Spans = append(tr.Spans, execinfrapb.TableReaderSpan{Span: sp.Spans[j]}) } - tr.MaxResults = n.maxResults - p.TotalEstimatedScannedRows += n.estimatedRowCount - if n.estimatedRowCount > p.MaxEstimatedRowCount { - p.MaxEstimatedRowCount = n.estimatedRowCount + tr.MaxResults = info.maxResults + p.TotalEstimatedScannedRows += info.estimatedRowCount + if info.estimatedRowCount > p.MaxEstimatedRowCount { + p.MaxEstimatedRowCount = info.estimatedRowCount } proc := physicalplan.Processor{ @@ -1115,47 +1163,47 @@ func (dsp *DistSQLPlanner) createTableReaders( p.ResultRouters[i] = pIdx } - if len(p.ResultRouters) > 1 && len(n.reqOrdering) > 0 { + if len(p.ResultRouters) > 1 && len(info.reqOrdering) > 0 { // Make a note of the fact that we have to maintain a certain ordering // between the parallel streams. // // This information is taken into account by the AddProjection call below: // specifically, it will make sure these columns are kept even if they are // not in the projection (e.g. "SELECT v FROM kv ORDER BY k"). - p.SetMergeOrdering(dsp.convertOrdering(n.reqOrdering, scanNodeToTableOrdinalMap)) + p.SetMergeOrdering(dsp.convertOrdering(info.reqOrdering, info.colsToTableOrdrinalMap)) } var typs []*types.T if returnMutations { - typs = make([]*types.T, 0, len(n.desc.Columns)+len(n.desc.MutationColumns())) + typs = make([]*types.T, 0, len(info.desc.Columns)+len(info.desc.MutationColumns())) } else { - typs = make([]*types.T, 0, len(n.desc.Columns)) + typs = make([]*types.T, 0, len(info.desc.Columns)) } - for i := range n.desc.Columns { - typs = append(typs, n.desc.Columns[i].Type) + for i := range info.desc.Columns { + typs = append(typs, info.desc.Columns[i].Type) } if returnMutations { - for _, col := range n.desc.MutationColumns() { + for _, col := range info.desc.MutationColumns() { typs = append(typs, col.Type) } } - p.SetLastStagePost(post, typs) + p.SetLastStagePost(info.post, typs) - outCols := getOutputColumnsFromScanNode(n, scanNodeToTableOrdinalMap) - planToStreamColMap := make([]int, len(n.cols)) - descColumnIDs := make([]sqlbase.ColumnID, 0, len(n.desc.Columns)) - for i := range n.desc.Columns { - descColumnIDs = append(descColumnIDs, n.desc.Columns[i].ID) + outCols := getOutputColumnsFromColsForScan(info.cols, info.colsToTableOrdrinalMap) + planToStreamColMap := make([]int, len(info.cols)) + descColumnIDs := make([]sqlbase.ColumnID, 0, len(info.desc.Columns)) + for i := range info.desc.Columns { + descColumnIDs = append(descColumnIDs, info.desc.Columns[i].ID) } if returnMutations { - for _, c := range n.desc.MutationColumns() { + for _, c := range info.desc.MutationColumns() { descColumnIDs = append(descColumnIDs, c.ID) } } for i := range planToStreamColMap { planToStreamColMap[i] = -1 for j, c := range outCols { - if descColumnIDs[c] == n.cols[i].ID { + if descColumnIDs[c] == info.cols[i].ID { planToStreamColMap[i] = j break } @@ -1164,7 +1212,7 @@ func (dsp *DistSQLPlanner) createTableReaders( p.AddProjection(outCols) p.PlanToStreamColMap = planToStreamColMap - return &p, nil + return nil } // selectRenders takes a PhysicalPlan that produces the results corresponding to From 3ea982a7ea49c7f12f5376dbb86af1a62c03b0cc Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 2 Jun 2020 15:07:48 -0700 Subject: [PATCH 03/46] sql: implement ConstructPlan by new factory This commit extracts the logic of `execFactory.ConstructPlan` into the function to be used by both factories. It also extracts out another small helper. Release note: None --- pkg/sql/distsql_spec_exec_factory.go | 7 ++- pkg/sql/exec_factory_util.go | 82 ++++++++++++++++++++++++++++ pkg/sql/opt_exec_factory.go | 52 +----------------- pkg/sql/plan_opt.go | 2 +- pkg/sql/scan.go | 10 ++++ 5 files changed, 100 insertions(+), 53 deletions(-) create mode 100644 pkg/sql/exec_factory_util.go diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index c07c705e5182..2c941f26d904 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -22,12 +22,13 @@ import ( ) type distSQLSpecExecFactory struct { + planner *planner } var _ exec.Factory = &distSQLSpecExecFactory{} -func newDistSQLSpecExecFactory() exec.Factory { - return &distSQLSpecExecFactory{} +func newDistSQLSpecExecFactory(p *planner) exec.Factory { + return &distSQLSpecExecFactory{planner: p} } func (e *distSQLSpecExecFactory) ConstructValues( @@ -235,7 +236,7 @@ func (e *distSQLSpecExecFactory) RenameColumns( func (e *distSQLSpecExecFactory) ConstructPlan( root exec.Node, subqueries []exec.Subquery, cascades []exec.Cascade, checks []exec.Node, ) (exec.Plan, error) { - return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning") + return constructPlan(e.planner, root, subqueries, cascades, checks) } func (e *distSQLSpecExecFactory) ConstructExplainOpt( diff --git a/pkg/sql/exec_factory_util.go b/pkg/sql/exec_factory_util.go new file mode 100644 index 000000000000..aa617877b704 --- /dev/null +++ b/pkg/sql/exec_factory_util.go @@ -0,0 +1,82 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" + "github.com/cockroachdb/cockroach/pkg/sql/rowexec" + "github.com/cockroachdb/errors" +) + +func constructPlan( + planner *planner, + root exec.Node, + subqueries []exec.Subquery, + cascades []exec.Cascade, + checks []exec.Node, +) (exec.Plan, error) { + res := &planTop{ + // TODO(radu): these fields can be modified by planning various opaque + // statements. We should have a cleaner way of plumbing these. + avoidBuffering: planner.curPlan.avoidBuffering, + auditEvents: planner.curPlan.auditEvents, + instrumentation: planner.curPlan.instrumentation, + } + assignPlan := func(plan *planMaybePhysical, node exec.Node) { + switch n := node.(type) { + case planNode: + plan.planNode = n + case planMaybePhysical: + *plan = n + default: + panic(fmt.Sprintf("unexpected node type %T", node)) + } + } + assignPlan(&res.main, root) + if len(subqueries) > 0 { + res.subqueryPlans = make([]subquery, len(subqueries)) + for i := range subqueries { + in := &subqueries[i] + out := &res.subqueryPlans[i] + out.subquery = in.ExprNode + switch in.Mode { + case exec.SubqueryExists: + out.execMode = rowexec.SubqueryExecModeExists + case exec.SubqueryOneRow: + out.execMode = rowexec.SubqueryExecModeOneRow + case exec.SubqueryAnyRows: + out.execMode = rowexec.SubqueryExecModeAllRowsNormalized + case exec.SubqueryAllRows: + out.execMode = rowexec.SubqueryExecModeAllRows + default: + return nil, errors.Errorf("invalid SubqueryMode %d", in.Mode) + } + out.expanded = true + assignPlan(&out.plan, in.Root) + } + } + if len(cascades) > 0 { + res.cascades = make([]cascadeMetadata, len(cascades)) + for i := range cascades { + res.cascades[i].Cascade = cascades[i] + } + } + if len(checks) > 0 { + res.checkPlans = make([]checkPlan, len(checks)) + for i := range checks { + assignPlan(&res.checkPlans[i].plan, checks[i]) + } + } + + return res, nil +} diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 97c62e0e4b69..e5903db2df3c 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/row" - "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/span" @@ -131,10 +130,8 @@ func (ef *execFactory) ConstructScan( scan.isFull = len(scan.spans) == 1 && scan.spans[0].EqualValue( scan.desc.IndexSpan(ef.planner.ExecCfg().Codec, scan.index.ID), ) - for i := range reqOrdering { - if reqOrdering[i].ColIdx >= len(colCfg.wantedColumns) { - return nil, errors.Errorf("invalid reqOrdering: %v", reqOrdering) - } + if err = colCfg.assertValidReqOrdering(reqOrdering); err != nil { + return nil, err } scan.reqOrdering = ReqOrdering(reqOrdering) scan.estimatedRowCount = uint64(rowCount) @@ -1048,50 +1045,7 @@ func (ef *execFactory) ConstructPlan( if spool, ok := root.(*spoolNode); ok { root = spool.source } - res := &planTop{ - // TODO(radu): these fields can be modified by planning various opaque - // statements. We should have a cleaner way of plumbing these. - avoidBuffering: ef.planner.curPlan.avoidBuffering, - auditEvents: ef.planner.curPlan.auditEvents, - instrumentation: ef.planner.curPlan.instrumentation, - } - res.main.planNode = root.(planNode) - if len(subqueries) > 0 { - res.subqueryPlans = make([]subquery, len(subqueries)) - for i := range subqueries { - in := &subqueries[i] - out := &res.subqueryPlans[i] - out.subquery = in.ExprNode - switch in.Mode { - case exec.SubqueryExists: - out.execMode = rowexec.SubqueryExecModeExists - case exec.SubqueryOneRow: - out.execMode = rowexec.SubqueryExecModeOneRow - case exec.SubqueryAnyRows: - out.execMode = rowexec.SubqueryExecModeAllRowsNormalized - case exec.SubqueryAllRows: - out.execMode = rowexec.SubqueryExecModeAllRows - default: - return nil, errors.Errorf("invalid SubqueryMode %d", in.Mode) - } - out.expanded = true - out.plan.planNode = in.Root.(planNode) - } - } - if len(cascades) > 0 { - res.cascades = make([]cascadeMetadata, len(cascades)) - for i := range cascades { - res.cascades[i].Cascade = cascades[i] - } - } - if len(checks) > 0 { - res.checkPlans = make([]checkPlan, len(checks)) - for i := range checks { - res.checkPlans[i].plan.planNode = checks[i].(planNode) - } - } - - return res, nil + return constructPlan(ef.planner, root, subqueries, cascades, checks) } // urlOutputter handles writing strings into an encoded URL for EXPLAIN (OPT, diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 6404567dece5..d03f7de94c4d 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -181,7 +181,7 @@ func (p *planner) makeOptimizerPlan(ctx context.Context) error { bld *execbuilder.Builder ) if mode := p.SessionData().ExperimentalDistSQLPlanningMode; mode != sessiondata.ExperimentalDistSQLPlanningOff { - bld = execbuilder.New(newDistSQLSpecExecFactory(), execMemo, &opc.catalog, root, p.EvalContext()) + bld = execbuilder.New(newDistSQLSpecExecFactory(p), execMemo, &opc.catalog, root, p.EvalContext()) plan, err = bld.Build() if err != nil { if mode == sessiondata.ExperimentalDistSQLPlanningAlways && diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 0af6f1466efa..777093578c28 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -133,6 +134,15 @@ type scanColumnsConfig struct { visibility execinfrapb.ScanVisibility } +func (cfg scanColumnsConfig) assertValidReqOrdering(reqOrdering exec.OutputOrdering) error { + for i := range reqOrdering { + if reqOrdering[i].ColIdx >= len(cfg.wantedColumns) { + return errors.Errorf("invalid reqOrdering: %v", reqOrdering) + } + } + return nil +} + var publicColumnsCfg = scanColumnsConfig{} func (p *planner) Scan() *scanNode { From 2949573ba1692a6531ccbb1e68607e4a3ddf9652 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Fri, 5 Jun 2020 08:07:01 -0700 Subject: [PATCH 04/46] sql: populate spatial_ref_sys table with projection entries Also fixed a bug where `\0` didn't actually output a NULL terminator (no idea why it worked). Release note (sql change): Populate the spatial_ref_sys table with support SRID entries for geospatial data types. --- pkg/geo/geoprojbase/geoprojbase.go | 9 +++- pkg/geo/geoprojbase/projections.go | 5 ++- .../logictest/testdata/logic_test/geospatial | 37 ---------------- .../testdata/logic_test/pg_extension | 43 +++++++++++++++++++ pkg/sql/pg_extension.go | 17 ++++++-- pkg/testutils/lint/lint_test.go | 1 + 6 files changed, 68 insertions(+), 44 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/pg_extension diff --git a/pkg/geo/geoprojbase/geoprojbase.go b/pkg/geo/geoprojbase/geoprojbase.go index 36d6e890b17c..68c41ecd2cd8 100644 --- a/pkg/geo/geoprojbase/geoprojbase.go +++ b/pkg/geo/geoprojbase/geoprojbase.go @@ -27,10 +27,15 @@ type Proj4Text struct { // MakeProj4Text returns a new Proj4Text with spec based on the given string. func MakeProj4Text(str string) Proj4Text { return Proj4Text{ - cStr: []byte(str + `\0`), + cStr: []byte(str + "\u0000"), } } +// String returns the string representation of the given proj text. +func (p *Proj4Text) String() string { + return string(p.cStr[:len(p.cStr)-1]) +} + // Bytes returns the raw bytes for the given proj text. func (p *Proj4Text) Bytes() []byte { return p.cStr @@ -62,6 +67,6 @@ type ProjInfo struct { // Projection returns the ProjInfo identifier for the given SRID, as well as an bool // indicating whether the projection exists. func Projection(srid geopb.SRID) (ProjInfo, bool) { - p, exists := projections[srid] + p, exists := Projections[srid] return p, exists } diff --git a/pkg/geo/geoprojbase/projections.go b/pkg/geo/geoprojbase/projections.go index cef98c17fe69..0f9f5e2450ed 100644 --- a/pkg/geo/geoprojbase/projections.go +++ b/pkg/geo/geoprojbase/projections.go @@ -12,9 +12,10 @@ package geoprojbase import "github.com/cockroachdb/cockroach/pkg/geo/geopb" -// projections is a mapping of SRID to projections. +// Projections is a mapping of SRID to projections. +// Use the `Projection` function to obtain one. // This file is not spell checked. -var projections = map[geopb.SRID]ProjInfo{ +var Projections = map[geopb.SRID]ProjInfo{ 4326: { SRID: 4326, AuthName: "EPSG", diff --git a/pkg/sql/logictest/testdata/logic_test/geospatial b/pkg/sql/logictest/testdata/logic_test/geospatial index 43446e7ed8d8..75b829a5cb78 100644 --- a/pkg/sql/logictest/testdata/logic_test/geospatial +++ b/pkg/sql/logictest/testdata/logic_test/geospatial @@ -1551,43 +1551,6 @@ MULTIPOINT (0 0, 1 1) statement error st_segmentize\(\): maximum segment length must be positive SELECT ST_Segmentize('POLYGON ((0.0 0.0, 1.0 0.0, 1.0 1.0, 0.0 1.0, 0.0 0.0))'::geometry, -1) -subtest pg_extension - -statement ok -CREATE TABLE pg_extension_test ( - a geography(point, 4326), - b geometry(linestring, 3857), - c geometry, - d geography -) - -query TTTTIIT rowsort -SELECT * FROM pg_extension.geography_columns WHERE f_table_name = 'pg_extension_test' ----- -test public pg_extension_test a 2 4326 POINT -test public pg_extension_test d NULL 0 GEOMETRY - -query TTTTIIT rowsort -SELECT * FROM pg_extension.geometry_columns WHERE f_table_name = 'pg_extension_test' ----- -test public pg_extension_test b 2 3857 LINESTRING -test public pg_extension_test c 2 0 GEOMETRY - -query TTTTIIT rowsort -SELECT * FROM geography_columns WHERE f_table_name = 'pg_extension_test' ----- -test public pg_extension_test a 2 4326 POINT -test public pg_extension_test d NULL 0 GEOMETRY - -query TTTTIIT rowsort -SELECT * FROM geometry_columns WHERE f_table_name = 'pg_extension_test' ----- -test public pg_extension_test b 2 3857 LINESTRING -test public pg_extension_test c 2 0 GEOMETRY - -statement error not yet implemented -SELECT * FROM pg_extension.spatial_ref_sys ORDER BY srid ASC - subtest st_srid statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/pg_extension b/pkg/sql/logictest/testdata/logic_test/pg_extension new file mode 100644 index 000000000000..b77d385a4b13 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/pg_extension @@ -0,0 +1,43 @@ +statement ok +CREATE TABLE pg_extension_test ( + a geography(point, 4326), + b geometry(linestring, 3857), + c geometry, + d geography +) + +query TTTTIIT rowsort +SELECT * FROM pg_extension.geography_columns WHERE f_table_name = 'pg_extension_test' +---- +test public pg_extension_test a 2 4326 POINT +test public pg_extension_test d NULL 0 GEOMETRY + +query TTTTIIT rowsort +SELECT * FROM pg_extension.geometry_columns WHERE f_table_name = 'pg_extension_test' +---- +test public pg_extension_test b 2 3857 LINESTRING +test public pg_extension_test c 2 0 GEOMETRY + +query TTTTIIT rowsort +SELECT * FROM geography_columns WHERE f_table_name = 'pg_extension_test' +---- +test public pg_extension_test a 2 4326 POINT +test public pg_extension_test d NULL 0 GEOMETRY + +query TTTTIIT rowsort +SELECT * FROM geometry_columns WHERE f_table_name = 'pg_extension_test' +---- +test public pg_extension_test b 2 3857 LINESTRING +test public pg_extension_test c 2 0 GEOMETRY + +query ITITT +SELECT * FROM pg_extension.spatial_ref_sys WHERE srid IN (3857, 4326) ORDER BY srid ASC +---- +3857 EPSG 3857 PROJCS["WGS 84 / Pseudo-Mercator",GEOGCS["WGS 84",DATUM["WGS_1984",SPHEROID["WGS 84",6378137,298.257223563,AUTHORITY["EPSG","7030"]],AUTHORITY["EPSG","6326"]],PRIMEM["Greenwich",0,AUTHORITY["EPSG","8901"]],UNIT["degree",0.0174532925199433,AUTHORITY["EPSG","9122"]],AUTHORITY["EPSG","4326"]],PROJECTION["Mercator_1SP"],PARAMETER["central_meridian",0],PARAMETER["scale_factor",1],PARAMETER["false_easting",0],PARAMETER["false_northing",0],UNIT["metre",1,AUTHORITY["EPSG","9001"]],AXIS["X",EAST],AXIS["Y",NORTH],EXTENSION["PROJ4","+proj=merc +a=6378137 +b=6378137 +lat_ts=0.0 +lon_0=0.0 +x_0=0.0 +y_0=0 +k=1.0 +units=m +nadgrids=@null +wktext +no_defs"],AUTHORITY["EPSG","3857"]] +proj=merc +a=6378137 +b=6378137 +lat_ts=0.0 +lon_0=0.0 +x_0=0.0 +y_0=0 +k=1.0 +units=m +nadgrids=@null +wktext +no_defs +4326 EPSG 4326 GEOGCS["WGS 84",DATUM["WGS_1984",SPHEROID["WGS 84",6378137,298.257223563,AUTHORITY["EPSG","7030"]],AUTHORITY["EPSG","6326"]],PRIMEM["Greenwich",0,AUTHORITY["EPSG","8901"]],UNIT["degree",0.0174532925199433,AUTHORITY["EPSG","9122"]],AUTHORITY["EPSG","4326"]] +proj=longlat +datum=WGS84 +no_defs + +query ITITT +SELECT * FROM spatial_ref_sys WHERE srid IN (3857, 4326) ORDER BY srid ASC +---- +3857 EPSG 3857 PROJCS["WGS 84 / Pseudo-Mercator",GEOGCS["WGS 84",DATUM["WGS_1984",SPHEROID["WGS 84",6378137,298.257223563,AUTHORITY["EPSG","7030"]],AUTHORITY["EPSG","6326"]],PRIMEM["Greenwich",0,AUTHORITY["EPSG","8901"]],UNIT["degree",0.0174532925199433,AUTHORITY["EPSG","9122"]],AUTHORITY["EPSG","4326"]],PROJECTION["Mercator_1SP"],PARAMETER["central_meridian",0],PARAMETER["scale_factor",1],PARAMETER["false_easting",0],PARAMETER["false_northing",0],UNIT["metre",1,AUTHORITY["EPSG","9001"]],AXIS["X",EAST],AXIS["Y",NORTH],EXTENSION["PROJ4","+proj=merc +a=6378137 +b=6378137 +lat_ts=0.0 +lon_0=0.0 +x_0=0.0 +y_0=0 +k=1.0 +units=m +nadgrids=@null +wktext +no_defs"],AUTHORITY["EPSG","3857"]] +proj=merc +a=6378137 +b=6378137 +lat_ts=0.0 +lon_0=0.0 +x_0=0.0 +y_0=0 +k=1.0 +units=m +nadgrids=@null +wktext +no_defs +4326 EPSG 4326 GEOGCS["WGS 84",DATUM["WGS_1984",SPHEROID["WGS 84",6378137,298.257223563,AUTHORITY["EPSG","7030"]],AUTHORITY["EPSG","6326"]],PRIMEM["Greenwich",0,AUTHORITY["EPSG","8901"]],UNIT["degree",0.0174532925199433,AUTHORITY["EPSG","9122"]],AUTHORITY["EPSG","4326"]] +proj=longlat +datum=WGS84 +no_defs diff --git a/pkg/sql/pg_extension.go b/pkg/sql/pg_extension.go index 97613afba5a7..07d070ad79a5 100644 --- a/pkg/sql/pg_extension.go +++ b/pkg/sql/pg_extension.go @@ -15,11 +15,11 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/geo/geopb" + "github.com/cockroachdb/cockroach/pkg/geo/geoprojbase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/errors" ) // pgExtension is virtual schema which contains virtual tables and/or views @@ -140,7 +140,18 @@ CREATE TABLE pg_extension.spatial_ref_sys ( srtext varchar(2048), proj4text varchar(2048) )`, - generator: func(ctx context.Context, p *planner, db *DatabaseDescriptor) (virtualTableGenerator, cleanupFunc, error) { - return nil, func() {}, errors.Newf("not yet implemented") + populate: func(ctx context.Context, p *planner, dbContext *DatabaseDescriptor, addRow func(...tree.Datum) error) error { + for _, projection := range geoprojbase.Projections { + if err := addRow( + tree.NewDInt(tree.DInt(projection.SRID)), + tree.NewDString(projection.AuthName), + tree.NewDInt(tree.DInt(projection.AuthSRID)), + tree.NewDString(projection.SRText), + tree.NewDString(projection.Proj4Text.String()), + ); err != nil { + return err + } + } + return nil }, } diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index ab910028e44d..77c54a628c5f 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1079,6 +1079,7 @@ func TestLint(t *testing.T) { stream.GrepNot(`^storage\/rocksdb_error_dict\.go$`), stream.GrepNot(`^workload/tpcds/tpcds.go$`), stream.GrepNot(`^geo/geoprojbase/projections.go$`), + stream.GrepNot(`^sql/logictest/testdata/logic_test/pg_extension$`), stream.Map(func(s string) string { return filepath.Join(pkgDir, s) }), From 37630738d84435d605f13e5803196d01bfae464f Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 4 Jun 2020 21:33:22 -0700 Subject: [PATCH 05/46] sql: use nicer apd.Decimal.SetInt64 in the code base This commit is automatic replacement of `apd.Decimal.SetFinite((.*), 0)` into a nicer `apd.Decimal.SetInt64($1)` which are equivalent. Release note: None --- .../execgen/cmd/execgen/overloads_bin.go | 8 ++--- .../execgen/cmd/execgen/overloads_cmp.go | 4 +-- pkg/sql/colexec/overloads_test.go | 24 +++++++------- pkg/sql/sem/builtins/aggregate_builtins.go | 22 ++++++------- pkg/sql/sem/builtins/math_builtins.go | 2 +- pkg/sql/sem/builtins/window_frame_builtins.go | 2 +- pkg/sql/sem/tree/casts.go | 6 ++-- pkg/sql/sem/tree/datum.go | 2 +- pkg/sql/sem/tree/eval.go | 32 +++++++++---------- pkg/sql/sem/tree/normalize.go | 2 +- pkg/util/json/json.go | 4 +-- 11 files changed, 54 insertions(+), 54 deletions(-) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go index 1e4cd2a469e7..d94378b98273 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go @@ -437,8 +437,8 @@ func (c intCustomizer) getBinOpAssignFunc() assignFunc { colexecerror.ExpectedError(tree.ErrDivByZero) } leftTmpDec, rightTmpDec := &_overloadHelper.tmpDec1, &_overloadHelper.tmpDec2 - leftTmpDec.SetFinite(int64({{.Left}}), 0) - rightTmpDec.SetFinite(int64({{.Right}}), 0) + leftTmpDec.SetInt64(int64({{.Left}})) + rightTmpDec.SetInt64(int64({{.Right}})) if _, err := tree.{{.Ctx}}.Quo(&{{.Target}}, leftTmpDec, rightTmpDec); err != nil { colexecerror.ExpectedError(err) } @@ -487,7 +487,7 @@ func (c decimalIntCustomizer) getBinOpAssignFunc() assignFunc { } {{end}} tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetFinite(int64({{.Right}}), 0) + tmpDec.SetInt64(int64({{.Right}})) if _, err := tree.{{.Ctx}}.{{.Op}}(&{{.Target}}, &{{.Left}}, tmpDec); err != nil { colexecerror.ExpectedError(err) } @@ -520,7 +520,7 @@ func (c intDecimalCustomizer) getBinOpAssignFunc() assignFunc { } {{end}} tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetFinite(int64({{.Left}}), 0) + tmpDec.SetInt64(int64({{.Left}})) _, err := tree.{{.Ctx}}.{{.Op}}(&{{.Target}}, tmpDec, &{{.Right}}) if err != nil { colexecerror.ExpectedError(err) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_cmp.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_cmp.go index 42a3652afec2..f7a02f9c744e 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_cmp.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_cmp.go @@ -242,7 +242,7 @@ func (c decimalIntCustomizer) getCmpOpCompareFunc() compareFunc { t := template.Must(template.New("").Parse(` { tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetFinite(int64({{.Right}}), 0) + tmpDec.SetInt64(int64({{.Right}})) {{.Target}} = tree.CompareDecimals(&{{.Left}}, tmpDec) } `)) @@ -280,7 +280,7 @@ func (c intDecimalCustomizer) getCmpOpCompareFunc() compareFunc { t := template.Must(template.New("").Parse(` { tmpDec := &_overloadHelper.tmpDec1 - tmpDec.SetFinite(int64({{.Left}}), 0) + tmpDec.SetInt64(int64({{.Left}})) {{.Target}} = tree.CompareDecimals(tmpDec, &{{.Right}}) } `)) diff --git a/pkg/sql/colexec/overloads_test.go b/pkg/sql/colexec/overloads_test.go index ade2112c6b82..4a748623e746 100644 --- a/pkg/sql/colexec/overloads_test.go +++ b/pkg/sql/colexec/overloads_test.go @@ -68,11 +68,11 @@ func TestIntegerDivision(t *testing.T) { var res apd.Decimal res = performDivInt16Int16(math.MinInt16, -1) - require.Equal(t, 0, res.Cmp(d.SetFinite(-math.MinInt16, 0))) + require.Equal(t, 0, res.Cmp(d.SetInt64(-math.MinInt16))) res = performDivInt32Int32(math.MinInt32, -1) - require.Equal(t, 0, res.Cmp(d.SetFinite(-math.MinInt32, 0))) + require.Equal(t, 0, res.Cmp(d.SetInt64(-math.MinInt32))) res = performDivInt64Int64(math.MinInt64, -1) - d.SetFinite(math.MinInt64, 0) + d.SetInt64(math.MinInt64) if _, err := tree.DecimalCtx.Neg(d, d); err != nil { t.Error(err) } @@ -83,11 +83,11 @@ func TestIntegerDivision(t *testing.T) { require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivInt64Int64(10, 0) }), tree.ErrDivByZero)) res = performDivInt16Int16(math.MaxInt16, -1) - require.Equal(t, 0, res.Cmp(d.SetFinite(-math.MaxInt16, 0))) + require.Equal(t, 0, res.Cmp(d.SetInt64(-math.MaxInt16))) res = performDivInt32Int32(math.MaxInt32, -1) - require.Equal(t, 0, res.Cmp(d.SetFinite(-math.MaxInt32, 0))) + require.Equal(t, 0, res.Cmp(d.SetInt64(-math.MaxInt32))) res = performDivInt64Int64(math.MaxInt64, -1) - require.Equal(t, 0, res.Cmp(d.SetFinite(-math.MaxInt64, 0))) + require.Equal(t, 0, res.Cmp(d.SetInt64(-math.MaxInt64))) } func TestIntegerMultiplication(t *testing.T) { @@ -145,20 +145,20 @@ func TestMixedTypeInteger(t *testing.T) { var res apd.Decimal res = performDivInt16Int32(4, 2) - require.Equal(t, 0, res.Cmp(d.SetFinite(2, 0))) + require.Equal(t, 0, res.Cmp(d.SetInt64(2))) res = performDivInt16Int64(6, 2) - require.Equal(t, 0, res.Cmp(d.SetFinite(3, 0))) + require.Equal(t, 0, res.Cmp(d.SetInt64(3))) res = performDivInt64Int32(12, 3) - require.Equal(t, 0, res.Cmp(d.SetFinite(4, 0))) + require.Equal(t, 0, res.Cmp(d.SetInt64(4))) res = performDivInt64Int16(20, 4) - require.Equal(t, 0, res.Cmp(d.SetFinite(5, 0))) + require.Equal(t, 0, res.Cmp(d.SetInt64(5))) } func TestDecimalDivByZero(t *testing.T) { defer leaktest.AfterTest(t)() nonZeroDec, zeroDec := apd.Decimal{}, apd.Decimal{} - nonZeroDec.SetFinite(4, 0) - zeroDec.SetFinite(0, 0) + nonZeroDec.SetInt64(4) + zeroDec.SetInt64(0) require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt16(nonZeroDec, 0) }), tree.ErrDivByZero)) require.True(t, errors.Is(colexecerror.CatchVectorizedRuntimeError(func() { performDivDecimalInt32(nonZeroDec, 0) }), tree.ErrDivByZero)) diff --git a/pkg/sql/sem/builtins/aggregate_builtins.go b/pkg/sql/sem/builtins/aggregate_builtins.go index 275453fee4a2..667cfcfe4319 100644 --- a/pkg/sql/sem/builtins/aggregate_builtins.go +++ b/pkg/sql/sem/builtins/aggregate_builtins.go @@ -1717,12 +1717,12 @@ func (a *intSumAggregate) Add(ctx context.Context, datum tree.Datum, _ ...tree.D // And overflow was detected; go to large integers, but keep the // sum computed so far. a.large = true - a.decSum.SetFinite(a.intSum, 0) + a.decSum.SetInt64(a.intSum) } } if a.large { - a.tmpDec.SetFinite(t, 0) + a.tmpDec.SetInt64(t) _, err := tree.ExactCtx.Add(&a.decSum, &a.decSum, &a.tmpDec) if err != nil { return err @@ -1745,7 +1745,7 @@ func (a *intSumAggregate) Result() (tree.Datum, error) { if a.large { dd.Set(&a.decSum) } else { - dd.SetFinite(a.intSum, 0) + dd.SetInt64(a.intSum) } return dd, nil } @@ -1815,7 +1815,7 @@ func (a *decimalSumAggregate) Result() (tree.Datum, error) { // Reset implements tree.AggregateFunc interface. func (a *decimalSumAggregate) Reset(ctx context.Context) { - a.sum.SetFinite(0, 0) + a.sum.SetInt64(0) a.sawNonNull = false a.reset(ctx) } @@ -1951,7 +1951,7 @@ func (a *intSqrDiffAggregate) Add(ctx context.Context, datum tree.Datum, _ ...tr return nil } - a.tmpDec.SetFinite(int64(tree.MustBeDInt(datum)), 0) + a.tmpDec.SetInt64(int64(tree.MustBeDInt(datum))) return a.agg.Add(ctx, &a.tmpDec) } @@ -2121,9 +2121,9 @@ func (a *decimalSqrDiffAggregate) Result() (tree.Datum, error) { // Reset implements tree.AggregateFunc interface. func (a *decimalSqrDiffAggregate) Reset(ctx context.Context) { - a.count.SetFinite(0, 0) - a.mean.SetFinite(0, 0) - a.sqrDiff.SetFinite(0, 0) + a.count.SetInt64(0) + a.mean.SetInt64(0) + a.sqrDiff.SetInt64(0) a.reset(ctx) } @@ -2317,9 +2317,9 @@ func (a *decimalSumSqrDiffsAggregate) Result() (tree.Datum, error) { // Reset implements tree.AggregateFunc interface. func (a *decimalSumSqrDiffsAggregate) Reset(ctx context.Context) { - a.count.SetFinite(0, 0) - a.mean.SetFinite(0, 0) - a.sqrDiff.SetFinite(0, 0) + a.count.SetInt64(0) + a.mean.SetInt64(0) + a.sqrDiff.SetInt64(0) a.reset(ctx) } diff --git a/pkg/sql/sem/builtins/math_builtins.go b/pkg/sql/sem/builtins/math_builtins.go index 03c62a3064fd..58a142870cc2 100644 --- a/pkg/sql/sem/builtins/math_builtins.go +++ b/pkg/sql/sem/builtins/math_builtins.go @@ -463,7 +463,7 @@ var mathBuiltins = map[string]builtinDefinition{ "negative.", tree.VolatilityImmutable), decimalOverload1(func(x *apd.Decimal) (tree.Datum, error) { d := &tree.DDecimal{} - d.Decimal.SetFinite(int64(x.Sign()), 0) + d.Decimal.SetInt64(int64(x.Sign())) return d, nil }, "Determines the sign of `val`: **1** for positive; **0** for 0 values; **-1** for "+ "negative.", tree.VolatilityImmutable), diff --git a/pkg/sql/sem/builtins/window_frame_builtins.go b/pkg/sql/sem/builtins/window_frame_builtins.go index a0bb103e4cee..20baa4dd37ba 100644 --- a/pkg/sql/sem/builtins/window_frame_builtins.go +++ b/pkg/sql/sem/builtins/window_frame_builtins.go @@ -403,7 +403,7 @@ func (w *avgWindowFunc) Compute( return &avg, err case *tree.DInt: dd := tree.DDecimal{} - dd.SetFinite(int64(*t), 0) + dd.SetInt64(int64(*t)) var avg tree.DDecimal count := apd.New(int64(frameSize), 0) _, err := tree.DecimalCtx.Quo(&avg.Decimal, &dd.Decimal, count) diff --git a/pkg/sql/sem/tree/casts.go b/pkg/sql/sem/tree/casts.go index 34699c9b03be..ddc76b8fac04 100644 --- a/pkg/sql/sem/tree/casts.go +++ b/pkg/sql/sem/tree/casts.go @@ -510,16 +510,16 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { switch v := d.(type) { case *DBool: if *v { - dd.SetFinite(1, 0) + dd.SetInt64(1) } case *DInt: - dd.SetFinite(int64(*v), 0) + dd.SetInt64(int64(*v)) case *DDate: // TODO(mjibson): This cast is unsupported by postgres. Should we remove ours? if !v.IsFinite() { return nil, errDecOutOfRange } - dd.SetFinite(v.UnixEpochDays(), 0) + dd.SetInt64(v.UnixEpochDays()) case *DFloat: _, err = dd.SetFloat64(float64(*v)) case *DDecimal: diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index db472fff795a..b0571b388ed4 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -971,7 +971,7 @@ func (d *DDecimal) Compare(ctx *EvalContext, other Datum) int { case *DDecimal: v = &t.Decimal case *DInt: - v.SetFinite(int64(*t), 0) + v.SetInt64(int64(*t)) case *DFloat: if _, err := v.SetFloat64(float64(*t)); err != nil { panic(errors.NewAssertionErrorWithWrappedErrf(err, "decimal compare, unexpected error")) diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 7af9197e5f0f..f9a272e4dbc7 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -593,7 +593,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ l := &left.(*DDecimal).Decimal r := MustBeDInt(right) dd := &DDecimal{} - dd.SetFinite(int64(r), 0) + dd.SetInt64(int64(r)) _, err := ExactCtx.Add(&dd.Decimal, l, &dd.Decimal) return dd, err }, @@ -607,7 +607,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ l := MustBeDInt(left) r := &right.(*DDecimal).Decimal dd := &DDecimal{} - dd.SetFinite(int64(l), 0) + dd.SetInt64(int64(l)) _, err := ExactCtx.Add(&dd.Decimal, &dd.Decimal, r) return dd, err }, @@ -888,7 +888,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ l := &left.(*DDecimal).Decimal r := MustBeDInt(right) dd := &DDecimal{} - dd.SetFinite(int64(r), 0) + dd.SetInt64(int64(r)) _, err := ExactCtx.Sub(&dd.Decimal, l, &dd.Decimal) return dd, err }, @@ -902,7 +902,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ l := MustBeDInt(left) r := &right.(*DDecimal).Decimal dd := &DDecimal{} - dd.SetFinite(int64(l), 0) + dd.SetInt64(int64(l)) _, err := ExactCtx.Sub(&dd.Decimal, &dd.Decimal, r) return dd, err }, @@ -1213,7 +1213,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ l := &left.(*DDecimal).Decimal r := MustBeDInt(right) dd := &DDecimal{} - dd.SetFinite(int64(r), 0) + dd.SetInt64(int64(r)) _, err := ExactCtx.Mul(&dd.Decimal, l, &dd.Decimal) return dd, err }, @@ -1227,7 +1227,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ l := MustBeDInt(left) r := &right.(*DDecimal).Decimal dd := &DDecimal{} - dd.SetFinite(int64(l), 0) + dd.SetInt64(int64(l)) _, err := ExactCtx.Mul(&dd.Decimal, &dd.Decimal, r) return dd, err }, @@ -1311,9 +1311,9 @@ var BinOps = map[BinaryOperator]binOpOverload{ if rInt == 0 { return nil, ErrDivByZero } - div := ctx.getTmpDec().SetFinite(int64(rInt), 0) + div := ctx.getTmpDec().SetInt64(int64(rInt)) dd := &DDecimal{} - dd.SetFinite(int64(MustBeDInt(left)), 0) + dd.SetInt64(int64(MustBeDInt(left))) _, err := DecimalCtx.Quo(&dd.Decimal, &dd.Decimal, div) return dd, err }, @@ -1359,7 +1359,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ return nil, ErrDivByZero } dd := &DDecimal{} - dd.SetFinite(int64(r), 0) + dd.SetInt64(int64(r)) _, err := DecimalCtx.Quo(&dd.Decimal, l, &dd.Decimal) return dd, err }, @@ -1376,7 +1376,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ return nil, ErrDivByZero } dd := &DDecimal{} - dd.SetFinite(int64(l), 0) + dd.SetInt64(int64(l)) _, err := DecimalCtx.Quo(&dd.Decimal, &dd.Decimal, r) return dd, err }, @@ -1465,7 +1465,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ return nil, ErrDivByZero } dd := &DDecimal{} - dd.SetFinite(int64(r), 0) + dd.SetInt64(int64(r)) _, err := HighPrecisionCtx.QuoInteger(&dd.Decimal, l, &dd.Decimal) return dd, err }, @@ -1482,7 +1482,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ return nil, ErrDivByZero } dd := &DDecimal{} - dd.SetFinite(int64(l), 0) + dd.SetInt64(int64(l)) _, err := HighPrecisionCtx.QuoInteger(&dd.Decimal, &dd.Decimal, r) return dd, err }, @@ -1545,7 +1545,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ return nil, ErrDivByZero } dd := &DDecimal{} - dd.SetFinite(int64(r), 0) + dd.SetInt64(int64(r)) _, err := HighPrecisionCtx.Rem(&dd.Decimal, l, &dd.Decimal) return dd, err }, @@ -1562,7 +1562,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ return nil, ErrDivByZero } dd := &DDecimal{} - dd.SetFinite(int64(l), 0) + dd.SetInt64(int64(l)) _, err := HighPrecisionCtx.Rem(&dd.Decimal, &dd.Decimal, r) return dd, err }, @@ -1741,7 +1741,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ l := &left.(*DDecimal).Decimal r := MustBeDInt(right) dd := &DDecimal{} - dd.SetFinite(int64(r), 0) + dd.SetInt64(int64(r)) _, err := DecimalCtx.Pow(&dd.Decimal, l, &dd.Decimal) return dd, err }, @@ -1755,7 +1755,7 @@ var BinOps = map[BinaryOperator]binOpOverload{ l := MustBeDInt(left) r := &right.(*DDecimal).Decimal dd := &DDecimal{} - dd.SetFinite(int64(l), 0) + dd.SetInt64(int64(l)) _, err := DecimalCtx.Pow(&dd.Decimal, &dd.Decimal, r) return dd, err }, diff --git a/pkg/sql/sem/tree/normalize.go b/pkg/sql/sem/tree/normalize.go index e8ce2dce7422..88c38c724d1b 100644 --- a/pkg/sql/sem/tree/normalize.go +++ b/pkg/sql/sem/tree/normalize.go @@ -968,7 +968,7 @@ func ContainsVars(expr Expr) bool { var DecimalOne DDecimal func init() { - DecimalOne.SetFinite(1, 0) + DecimalOne.SetInt64(1) } // ReType ensures that the given numeric expression evaluates diff --git a/pkg/util/json/json.go b/pkg/util/json/json.go index 15f19816bfbe..c964608dbcdd 100644 --- a/pkg/util/json/json.go +++ b/pkg/util/json/json.go @@ -942,14 +942,14 @@ func fromMap(v map[string]interface{}) (JSON, error) { // FromInt returns a JSON value given a int. func FromInt(v int) JSON { dec := apd.Decimal{} - dec.SetFinite(int64(v), 0) + dec.SetInt64(int64(v)) return jsonNumber(dec) } // FromInt64 returns a JSON value given a int64. func FromInt64(v int64) JSON { dec := apd.Decimal{} - dec.SetFinite(v, 0) + dec.SetInt64(v) return jsonNumber(dec) } From 8dfdf3a3eca4d113759c945ea1ff1169e0c49cdc Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Fri, 5 Jun 2020 10:07:28 -0700 Subject: [PATCH 06/46] geo: move TestClipRectByEWKT test to geoindex Trying to make the geo package not depend on GEOS, and this is one of the required steps. Release note: None --- pkg/geo/geo_test.go | 42 ---------------------- pkg/geo/geoindex/s2_geometry_index_test.go | 40 +++++++++++++++++++++ pkg/geo/{ => geoindex}/testdata/clip | 0 3 files changed, 40 insertions(+), 42 deletions(-) rename pkg/geo/{ => geoindex}/testdata/clip (100%) diff --git a/pkg/geo/geo_test.go b/pkg/geo/geo_test.go index 4cf21e09577d..7b142b0b5a8d 100644 --- a/pkg/geo/geo_test.go +++ b/pkg/geo/geo_test.go @@ -16,9 +16,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/geo/geopb" - "github.com/cockroachdb/cockroach/pkg/geo/geos" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/datadriven" "github.com/golang/geo/s2" "github.com/stretchr/testify/require" "github.com/twpayne/go-geom" @@ -474,42 +471,3 @@ func TestGeographyAsS2(t *testing.T) { }) } } - -func TestClipEWKBByRect(t *testing.T) { - defer leaktest.AfterTest(t)() - - var g *Geometry - var err error - datadriven.RunTest(t, "testdata/clip", func(t *testing.T, d *datadriven.TestData) string { - switch d.Cmd { - case "geometry": - g, err = ParseGeometry(d.Input) - if err != nil { - return err.Error() - } - return "" - case "clip": - var xMin, yMin, xMax, yMax int - d.ScanArgs(t, "xmin", &xMin) - d.ScanArgs(t, "ymin", &yMin) - d.ScanArgs(t, "xmax", &xMax) - d.ScanArgs(t, "ymax", &yMax) - ewkb, err := geos.ClipEWKBByRect( - g.EWKB(), float64(xMin), float64(yMin), float64(xMax), float64(yMax)) - if err != nil { - return err.Error() - } - // TODO(sumeer): - // - add WKB to WKT and print exact output - // - expand test with more inputs - return fmt.Sprintf( - "%d => %d (srid: %d)", - len(g.EWKB()), - len(ewkb), - g.SRID(), - ) - default: - return fmt.Sprintf("unknown command: %s", d.Cmd) - } - }) -} diff --git a/pkg/geo/geoindex/s2_geometry_index_test.go b/pkg/geo/geoindex/s2_geometry_index_test.go index b64c575c7f68..3611054ae8f2 100644 --- a/pkg/geo/geoindex/s2_geometry_index_test.go +++ b/pkg/geo/geoindex/s2_geometry_index_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/geo" + "github.com/cockroachdb/cockroach/pkg/geo/geos" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" ) @@ -65,3 +66,42 @@ func TestS2GeometryIndexBasic(t *testing.T) { } }) } + +func TestClipEWKBByRect(t *testing.T) { + defer leaktest.AfterTest(t)() + + var g *geo.Geometry + var err error + datadriven.RunTest(t, "testdata/clip", func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "geometry": + g, err = geo.ParseGeometry(d.Input) + if err != nil { + return err.Error() + } + return "" + case "clip": + var xMin, yMin, xMax, yMax int + d.ScanArgs(t, "xmin", &xMin) + d.ScanArgs(t, "ymin", &yMin) + d.ScanArgs(t, "xmax", &xMax) + d.ScanArgs(t, "ymax", &yMax) + ewkb, err := geos.ClipEWKBByRect( + g.EWKB(), float64(xMin), float64(yMin), float64(xMax), float64(yMax)) + if err != nil { + return err.Error() + } + // TODO(sumeer): + // - add WKB to WKT and print exact output + // - expand test with more inputs + return fmt.Sprintf( + "%d => %d (srid: %d)", + len(g.EWKB()), + len(ewkb), + g.SRID(), + ) + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) +} diff --git a/pkg/geo/testdata/clip b/pkg/geo/geoindex/testdata/clip similarity index 100% rename from pkg/geo/testdata/clip rename to pkg/geo/geoindex/testdata/clip From 13845e418807a3a1afe290636937f5ebb3f1e81c Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Sat, 30 May 2020 23:10:58 -0400 Subject: [PATCH 07/46] colexec: don't randomize batch size in benches This was making it hard to run the same benchmarks in a consistent way. Release note: None --- pkg/sql/colexec/main_test.go | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/pkg/sql/colexec/main_test.go b/pkg/sql/colexec/main_test.go index 851d2e170316..93d0c552fc0a 100644 --- a/pkg/sql/colexec/main_test.go +++ b/pkg/sql/colexec/main_test.go @@ -12,6 +12,7 @@ package colexec import ( "context" + "flag" "fmt" "os" "testing" @@ -66,12 +67,16 @@ func TestMain(m *testing.M) { testDiskAcc = &diskAcc defer testDiskAcc.Close(ctx) - // Pick a random batch size in [minBatchSize, coldata.MaxBatchSize] - // range. The randomization can be disabled using COCKROACH_RANDOMIZE_BATCH_SIZE=false. - randomBatchSize := generateBatchSize() - fmt.Printf("coldata.BatchSize() is set to %d\n", randomBatchSize) - if err := coldata.SetBatchSizeForTests(randomBatchSize); err != nil { - colexecerror.InternalError(err) + flag.Parse() + if f := flag.Lookup("test.bench"); f == nil || f.Value.String() == "" { + // If we're running benchmarks, don't set a random batch size. + // Pick a random batch size in [minBatchSize, coldata.MaxBatchSize] + // range. The randomization can be disabled using COCKROACH_RANDOMIZE_BATCH_SIZE=false. + randomBatchSize := generateBatchSize() + fmt.Printf("coldata.BatchSize() is set to %d\n", randomBatchSize) + if err := coldata.SetBatchSizeForTests(randomBatchSize); err != nil { + colexecerror.InternalError(err) + } } return m.Run() }()) From 269f6353841122945894aad4ed1ce502ca77db44 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Sat, 6 Jun 2020 15:42:07 -0400 Subject: [PATCH 08/46] colexec: remove unnecessary execgen imports The .eg.go files had unnecessary imports of execgen, which is not required in the runtime generated code, and in fact is suspicious. Release note: None --- pkg/sql/colexec/any_not_null_agg_tmpl.go | 3 --- pkg/sql/colexec/cast_tmpl.go | 3 --- pkg/sql/colexec/const_tmpl.go | 3 --- pkg/sql/colexec/distinct_tmpl.go | 3 --- pkg/sql/colexec/hash_aggregator_tmpl.go | 3 --- pkg/sql/colexec/hash_utils_tmpl.go | 3 --- pkg/sql/colexec/hashtable_tmpl.go | 4 ---- pkg/sql/colexec/mergejoinbase_tmpl.go | 3 --- pkg/sql/colexec/mergejoiner_tmpl.go | 3 --- pkg/sql/colexec/min_max_agg_tmpl.go | 3 --- pkg/sql/colexec/ordered_synchronizer_tmpl.go | 3 --- pkg/sql/colexec/proj_const_ops_tmpl.go | 3 --- pkg/sql/colexec/proj_non_const_ops_tmpl.go | 3 --- pkg/sql/colexec/rowstovec_tmpl.go | 3 --- pkg/sql/colexec/select_in_tmpl.go | 1 - pkg/sql/colexec/selection_ops_tmpl.go | 3 --- pkg/sql/colexec/sort_tmpl.go | 3 --- pkg/sql/colexec/values_differ_tmpl.go | 3 --- pkg/sql/colexec/vec_comparators_tmpl.go | 3 --- 19 files changed, 56 deletions(-) diff --git a/pkg/sql/colexec/any_not_null_agg_tmpl.go b/pkg/sql/colexec/any_not_null_agg_tmpl.go index 70e54845dc58..ffc07ab045ab 100644 --- a/pkg/sql/colexec/any_not_null_agg_tmpl.go +++ b/pkg/sql/colexec/any_not_null_agg_tmpl.go @@ -30,9 +30,6 @@ import ( "github.com/cockroachdb/errors" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/cast_tmpl.go b/pkg/sql/colexec/cast_tmpl.go index cd934bd47543..ff08c5a8ae50 100644 --- a/pkg/sql/colexec/cast_tmpl.go +++ b/pkg/sql/colexec/cast_tmpl.go @@ -36,9 +36,6 @@ import ( "github.com/cockroachdb/errors" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* type _R_GO_TYPE interface{} diff --git a/pkg/sql/colexec/const_tmpl.go b/pkg/sql/colexec/const_tmpl.go index 8d0d0e781677..db484ac2d065 100644 --- a/pkg/sql/colexec/const_tmpl.go +++ b/pkg/sql/colexec/const_tmpl.go @@ -31,9 +31,6 @@ import ( "github.com/cockroachdb/errors" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/distinct_tmpl.go b/pkg/sql/colexec/distinct_tmpl.go index 21f1482c2d67..811c23e24159 100644 --- a/pkg/sql/colexec/distinct_tmpl.go +++ b/pkg/sql/colexec/distinct_tmpl.go @@ -84,9 +84,6 @@ func NewOrderedDistinct( }, nil } -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/hash_aggregator_tmpl.go b/pkg/sql/colexec/hash_aggregator_tmpl.go index 2f8482c07a90..07dabb6418b5 100644 --- a/pkg/sql/colexec/hash_aggregator_tmpl.go +++ b/pkg/sql/colexec/hash_aggregator_tmpl.go @@ -28,9 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/hash_utils_tmpl.go b/pkg/sql/colexec/hash_utils_tmpl.go index d8e6577cd017..308b0cb49e6d 100644 --- a/pkg/sql/colexec/hash_utils_tmpl.go +++ b/pkg/sql/colexec/hash_utils_tmpl.go @@ -30,9 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // _GOTYPESLICE is a template Go type slice variable. diff --git a/pkg/sql/colexec/hashtable_tmpl.go b/pkg/sql/colexec/hashtable_tmpl.go index 8fe7c24ee659..2bcb66c9779a 100644 --- a/pkg/sql/colexec/hashtable_tmpl.go +++ b/pkg/sql/colexec/hashtable_tmpl.go @@ -21,14 +21,10 @@ package colexec import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // _LEFT_CANONICAL_TYPE_FAMILY is the template variable. diff --git a/pkg/sql/colexec/mergejoinbase_tmpl.go b/pkg/sql/colexec/mergejoinbase_tmpl.go index 4d9de321fbf9..e6d0a62f6f1b 100644 --- a/pkg/sql/colexec/mergejoinbase_tmpl.go +++ b/pkg/sql/colexec/mergejoinbase_tmpl.go @@ -28,9 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/mergejoiner_tmpl.go b/pkg/sql/colexec/mergejoiner_tmpl.go index 380939be8894..7447e369dfed 100644 --- a/pkg/sql/colexec/mergejoiner_tmpl.go +++ b/pkg/sql/colexec/mergejoiner_tmpl.go @@ -30,9 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/min_max_agg_tmpl.go b/pkg/sql/colexec/min_max_agg_tmpl.go index f770c56b8d73..d7c5cb97a3ba 100644 --- a/pkg/sql/colexec/min_max_agg_tmpl.go +++ b/pkg/sql/colexec/min_max_agg_tmpl.go @@ -31,9 +31,6 @@ import ( "github.com/cockroachdb/errors" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // Remove unused warning. var _ = colexecerror.InternalError diff --git a/pkg/sql/colexec/ordered_synchronizer_tmpl.go b/pkg/sql/colexec/ordered_synchronizer_tmpl.go index 27e820c167b5..1788ad52e862 100644 --- a/pkg/sql/colexec/ordered_synchronizer_tmpl.go +++ b/pkg/sql/colexec/ordered_synchronizer_tmpl.go @@ -36,9 +36,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/encoding" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/proj_const_ops_tmpl.go b/pkg/sql/colexec/proj_const_ops_tmpl.go index f13787878dd1..c49adb453ec5 100644 --- a/pkg/sql/colexec/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/proj_const_ops_tmpl.go @@ -33,9 +33,6 @@ import ( "github.com/cockroachdb/errors" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/proj_non_const_ops_tmpl.go index c1ba4e86f2aa..622de348fa79 100644 --- a/pkg/sql/colexec/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/proj_non_const_ops_tmpl.go @@ -33,9 +33,6 @@ import ( "github.com/cockroachdb/errors" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/rowstovec_tmpl.go b/pkg/sql/colexec/rowstovec_tmpl.go index 8bb7848b0ba0..cda93cf6fa5e 100644 --- a/pkg/sql/colexec/rowstovec_tmpl.go +++ b/pkg/sql/colexec/rowstovec_tmpl.go @@ -32,9 +32,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // _CANONICAL_TYPE_FAMILY is the template variable. diff --git a/pkg/sql/colexec/select_in_tmpl.go b/pkg/sql/colexec/select_in_tmpl.go index f8e43d1075a9..fd50d656c20b 100644 --- a/pkg/sql/colexec/select_in_tmpl.go +++ b/pkg/sql/colexec/select_in_tmpl.go @@ -35,7 +35,6 @@ import ( // Remove unused warnings. var ( - _ = execgen.UNSAFEGET _ = colexecerror.InternalError ) diff --git a/pkg/sql/colexec/selection_ops_tmpl.go b/pkg/sql/colexec/selection_ops_tmpl.go index 35ee2d7ec80e..90f5796f9f8e 100644 --- a/pkg/sql/colexec/selection_ops_tmpl.go +++ b/pkg/sql/colexec/selection_ops_tmpl.go @@ -32,9 +32,6 @@ import ( "github.com/cockroachdb/errors" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/sort_tmpl.go b/pkg/sql/colexec/sort_tmpl.go index 6e731d372dab..8209d36db5d9 100644 --- a/pkg/sql/colexec/sort_tmpl.go +++ b/pkg/sql/colexec/sort_tmpl.go @@ -31,9 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/values_differ_tmpl.go b/pkg/sql/colexec/values_differ_tmpl.go index 59e770dbd08e..1a81ea1e201e 100644 --- a/pkg/sql/colexec/values_differ_tmpl.go +++ b/pkg/sql/colexec/values_differ_tmpl.go @@ -28,9 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. diff --git a/pkg/sql/colexec/vec_comparators_tmpl.go b/pkg/sql/colexec/vec_comparators_tmpl.go index 048ca02cd963..149b323b51a4 100644 --- a/pkg/sql/colexec/vec_comparators_tmpl.go +++ b/pkg/sql/colexec/vec_comparators_tmpl.go @@ -29,9 +29,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" ) -// Remove unused warning. -var _ = execgen.UNSAFEGET - // {{/* // Declarations to make the template compile properly. From 3ccbdbbb9ea879f8be72aabd509b6039594b3cab Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Fri, 5 Jun 2020 01:34:57 -0400 Subject: [PATCH 09/46] execgen: add generic inliner This commit adds a new directive to execgen: // execgen:inline This directive causes a function that's annotated with it to be inlined into all of its callers via AST transformation. This kind of inlining is smarter than before - it permits arguments with names different from the names of the function's formal parameters, for example. This commit also changes the functions in distinct_tmpl to use this directive instead of the manual templating method. I think this is the only easy function to change. The rest have, at the simple level of difficulty, static template-time parameters, and at a harder level of difficulty, type parameters. Improving these cases holistically should come next. Release note: None --- Gopkg.lock | 18 + Gopkg.toml | 4 + pkg/sql/colexec/distinct_tmpl.go | 72 ++-- .../execgen/cmd/execgen/distinct_gen.go | 6 +- pkg/sql/colexec/execgen/cmd/execgen/main.go | 21 +- pkg/sql/colexec/execgen/datadriven_test.go | 36 ++ pkg/sql/colexec/execgen/inline.go | 365 ++++++++++++++++++ pkg/sql/colexec/execgen/inline_test.go | 102 +++++ pkg/sql/colexec/execgen/testdata/inline | 146 +++++++ pkg/sql/colexec/execgen/util_test.go | 70 ++++ pkg/testutils/lint/lint_test.go | 7 +- vendor | 2 +- 12 files changed, 788 insertions(+), 61 deletions(-) create mode 100644 pkg/sql/colexec/execgen/datadriven_test.go create mode 100644 pkg/sql/colexec/execgen/inline.go create mode 100644 pkg/sql/colexec/execgen/inline_test.go create mode 100644 pkg/sql/colexec/execgen/testdata/inline create mode 100644 pkg/sql/colexec/execgen/util_test.go diff --git a/Gopkg.lock b/Gopkg.lock index 46f4b8665436..20132c3f683d 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -534,6 +534,21 @@ revision = "20f5889cbdc3c73dbd2862796665e7c465ade7d1" version = "v1.0.8" +[[projects]] + branch = "master" + digest = "1:6c25d97f154e54290d708120e6bc72da4e8628e1467c53d8f723f1594043901d" + name = "github.com/dave/dst" + packages = [ + ".", + "decorator", + "decorator/resolver", + "decorator/resolver/gopackages", + "decorator/resolver/gotypes", + "dstutil", + ] + pruneopts = "UT" + revision = "ce1c8af3ca7fccd4405ec0594a8b68d40370dda0" + [[projects]] digest = "1:ffe9824d294da03b391f44e1ae8281281b4afc1bdaa9588c9097785e3af10cec" name = "github.com/davecgh/go-spew" @@ -2164,6 +2179,9 @@ "github.com/cockroachdb/stress", "github.com/cockroachdb/ttycolor", "github.com/codahale/hdrhistogram", + "github.com/dave/dst", + "github.com/dave/dst/decorator", + "github.com/dave/dst/dstutil", "github.com/docker/distribution/reference", "github.com/docker/docker/api/types", "github.com/docker/docker/api/types/container", diff --git a/Gopkg.toml b/Gopkg.toml index 3767d9005afb..3c496c027006 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -58,6 +58,10 @@ ignored = [ name = "github.com/docker/docker" branch = "master" +[[constraint]] + name = "github.com/dave/dst" + branch = "master" + [[constraint]] name = "github.com/maruel/panicparse" revision = "f20d4c4d746f810c9110e21928d4135e1f2a3efa" diff --git a/pkg/sql/colexec/distinct_tmpl.go b/pkg/sql/colexec/distinct_tmpl.go index 811c23e24159..987126066964 100644 --- a/pkg/sql/colexec/distinct_tmpl.go +++ b/pkg/sql/colexec/distinct_tmpl.go @@ -238,30 +238,25 @@ func (p *distinct_TYPEOp) Next(ctx context.Context) coldata.Batch { // Bounds check elimination. sel = sel[:n] if nulls != nil { - for _, checkIdx := range sel { - outputIdx := checkIdx - _CHECK_DISTINCT_WITH_NULLS(checkIdx, outputIdx, lastVal, nulls, lastValNull, col, outputCol) + for _, idx := range sel { + lastVal, lastValNull = checkDistinctWithNulls(idx, idx, lastVal, nulls, lastValNull, col, outputCol) } } else { - for _, checkIdx := range sel { - outputIdx := checkIdx - _CHECK_DISTINCT(checkIdx, outputIdx, lastVal, col, outputCol) + for _, idx := range sel { + lastVal = checkDistinct(idx, idx, lastVal, col, outputCol) } } } else { - // Bounds check elimination. col = execgen.SLICE(col, 0, n) outputCol = outputCol[:n] _ = outputCol[n-1] if nulls != nil { - for execgen.RANGE(checkIdx, col, 0, n) { - outputIdx := checkIdx - _CHECK_DISTINCT_WITH_NULLS(checkIdx, outputIdx, lastVal, nulls, lastValNull, col, outputCol) + for execgen.RANGE(idx, col, 0, n) { + lastVal, lastValNull = checkDistinctWithNulls(idx, idx, lastVal, nulls, lastValNull, col, outputCol) } } else { - for execgen.RANGE(checkIdx, col, 0, n) { - outputIdx := checkIdx - _CHECK_DISTINCT(checkIdx, outputIdx, lastVal, col, outputCol) + for execgen.RANGE(idx, col, 0, n) { + lastVal = checkDistinct(idx, idx, lastVal, col, outputCol) } } } @@ -294,11 +289,11 @@ func (p partitioner_TYPE) partitionWithOrder( outputCol[0] = true if nulls != nil { for outputIdx, checkIdx := range order { - _CHECK_DISTINCT_WITH_NULLS(checkIdx, outputIdx, lastVal, nulls, lastValNull, col, outputCol) + lastVal, lastValNull = checkDistinctWithNulls(checkIdx, outputIdx, lastVal, nulls, lastValNull, col, outputCol) } } else { for outputIdx, checkIdx := range order { - _CHECK_DISTINCT(checkIdx, outputIdx, lastVal, col, outputCol) + lastVal = checkDistinct(checkIdx, outputIdx, lastVal, col, outputCol) } } } @@ -318,14 +313,12 @@ func (p partitioner_TYPE) partition(colVec coldata.Vec, outputCol []bool, n int) outputCol = outputCol[:n] outputCol[0] = true if nulls != nil { - for execgen.RANGE(checkIdx, col, 0, n) { - outputIdx := checkIdx - _CHECK_DISTINCT_WITH_NULLS(checkIdx, outputIdx, lastVal, nulls, lastValNull, col, outputCol) + for execgen.RANGE(idx, col, 0, n) { + lastVal, lastValNull = checkDistinctWithNulls(idx, idx, lastVal, nulls, lastValNull, col, outputCol) } } else { - for execgen.RANGE(checkIdx, col, 0, n) { - outputIdx := checkIdx - _CHECK_DISTINCT(checkIdx, outputIdx, lastVal, col, outputCol) + for execgen.RANGE(idx, col, 0, n) { + lastVal = checkDistinct(idx, idx, lastVal, col, outputCol) } } } @@ -333,33 +326,27 @@ func (p partitioner_TYPE) partition(colVec coldata.Vec, outputCol []bool, n int) // {{end}} // {{end}} -// {{/* -// _CHECK_DISTINCT retrieves the value at the ith index of col, compares it +// checkDistinct retrieves the value at the ith index of col, compares it // to the passed in lastVal, and sets the ith value of outputCol to true if the // compared values were distinct. It presumes that the current batch has no null // values. -func _CHECK_DISTINCT( +// execgen:inline +func checkDistinct( checkIdx int, outputIdx int, lastVal _GOTYPE, col []_GOTYPE, outputCol []bool, -) { // */}} - - // {{define "checkDistinct" -}} - // {{with .Global}} +) _GOTYPE { v := execgen.UNSAFEGET(col, checkIdx) var unique bool _ASSIGN_NE(unique, v, lastVal, _, col, _) outputCol[outputIdx] = outputCol[outputIdx] || unique execgen.COPYVAL(lastVal, v) - // {{end}} - // {{end}} - - // {{/* -} // */}} + return lastVal +} -// {{/* -// _CHECK_DISTINCT_WITH_NULLS behaves the same as _CHECK_DISTINCT, but it also +// checkDistinctWithNulls behaves the same as checkDistinct, but it also // considers whether the previous and current values are null. It assumes that // `nulls` is non-nil. -func _CHECK_DISTINCT_WITH_NULLS( +// execgen:inline +func checkDistinctWithNulls( checkIdx int, outputIdx int, lastVal _GOTYPE, @@ -367,10 +354,7 @@ func _CHECK_DISTINCT_WITH_NULLS( lastValNull bool, col []_GOTYPE, outputCol []bool, -) { // */}} - - // {{define "checkDistinctWithNulls" -}} - // {{with .Global}} +) (lastVal _GOTYPE, lastValNull bool) { null := nulls.NullAt(checkIdx) if null { if !lastValNull { @@ -390,9 +374,5 @@ func _CHECK_DISTINCT_WITH_NULLS( } execgen.COPYVAL(lastVal, v) } - lastValNull = null - // {{end}} - // {{end}} - - // {{/* -} // */}} + return lastVal, null +} diff --git a/pkg/sql/colexec/execgen/cmd/execgen/distinct_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/distinct_gen.go index cdb7b0a6a149..ec1b23d4368a 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/distinct_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/distinct_gen.go @@ -33,11 +33,6 @@ func genDistinctOps(inputFileContents string, wr io.Writer) error { assignNeRe := makeFunctionRegex("_ASSIGN_NE", 6) s = assignNeRe.ReplaceAllString(s, makeTemplateFunctionCall("Assign", 6)) - innerLoopRe := makeFunctionRegex("_CHECK_DISTINCT", 5) - s = innerLoopRe.ReplaceAllString(s, `{{template "checkDistinct" buildDict "Global" .}}`) - - innerLoopNullsRe := makeFunctionRegex("_CHECK_DISTINCT_WITH_NULLS", 7) - s = innerLoopNullsRe.ReplaceAllString(s, `{{template "checkDistinctWithNulls" buildDict "Global" .}}`) s = replaceManipulationFuncs(s) // Now, generate the op, from the template. @@ -48,6 +43,7 @@ func genDistinctOps(inputFileContents string, wr io.Writer) error { return tmpl.Execute(wr, sameTypeComparisonOpToOverloads[tree.NE]) } + func init() { registerGenerator(genDistinctOps, "distinct.eg.go", distinctOpsTmpl) } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/main.go b/pkg/sql/colexec/execgen/cmd/execgen/main.go index eba42ab1dcd9..e0235bba19dd 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/main.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/main.go @@ -20,6 +20,7 @@ import ( "path/filepath" "regexp" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" "github.com/cockroachdb/errors" "github.com/cockroachdb/gostdlib/x/tools/imports" @@ -59,11 +60,11 @@ type entry struct { inputFile string } -func registerGenerator(g generator, filename, dep string) { - if _, ok := generators[filename]; ok { - colexecerror.InternalError(fmt.Sprintf("%s generator already registered", filename)) +func registerGenerator(g generator, outputFile, inputFile string) { + if _, ok := generators[outputFile]; ok { + colexecerror.InternalError(fmt.Sprintf("%s generator already registered", outputFile)) } - generators[filename] = entry{fn: g, inputFile: dep} + generators[outputFile] = entry{fn: g, inputFile: inputFile} } func (g *execgenTool) run(args ...string) bool { @@ -108,7 +109,6 @@ func (g *execgenTool) run(args ...string) bool { } } } - return true } @@ -119,16 +119,21 @@ func (g *execgenTool) generate(path string, entry entry) error { var buf bytes.Buffer buf.WriteString("// Code generated by execgen; DO NOT EDIT.\n") - var inputFileContents []byte + var inputFileContents string var err error if entry.inputFile != "" { - inputFileContents, err = ioutil.ReadFile(entry.inputFile) + inputFileBytes, err := ioutil.ReadFile(entry.inputFile) + if err != nil { + return err + } + // Inline functions with // execgen:inline. + inputFileContents, err = execgen.InlineFuncs(string(inputFileBytes)) if err != nil { return err } } - err = entry.fn(string(inputFileContents), &buf) + err = entry.fn(inputFileContents, &buf) if err != nil { return err } diff --git a/pkg/sql/colexec/execgen/datadriven_test.go b/pkg/sql/colexec/execgen/datadriven_test.go new file mode 100644 index 000000000000..c049fc9af53c --- /dev/null +++ b/pkg/sql/colexec/execgen/datadriven_test.go @@ -0,0 +1,36 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package execgen + +import ( + "testing" + + "github.com/cockroachdb/datadriven" +) + +// Walk walks path for datadriven files and calls RunTest on them. +func TestExecgen(t *testing.T) { + datadriven.Walk(t, "testdata", func(t *testing.T, path string) { + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "inline": + s, err := InlineFuncs(d.Input) + if err != nil { + t.Fatal(err) + } + return s + default: + t.Fatalf("unknown command: %s", d.Cmd) + return "" + } + }) + }) +} diff --git a/pkg/sql/colexec/execgen/inline.go b/pkg/sql/colexec/execgen/inline.go new file mode 100644 index 000000000000..9e81d4ad781d --- /dev/null +++ b/pkg/sql/colexec/execgen/inline.go @@ -0,0 +1,365 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package execgen + +import ( + "bytes" + "fmt" + "go/parser" + "go/token" + + "github.com/cockroachdb/errors" + "github.com/dave/dst" + "github.com/dave/dst/decorator" + "github.com/dave/dst/dstutil" +) + +// InlineFuncs takes an input file's contents and inlines all functions +// annotated with // execgen:inline into their callsites via AST manipulation. +func InlineFuncs(inputFileContents string) (string, error) { + f, err := decorator.ParseFile(token.NewFileSet(), "", inputFileContents, parser.ParseComments) + if err != nil { + return "", err + } + + templateFuncMap := make(map[string]*dst.FuncDecl) + + // First, run over the input file, searching for functions that are annotated + // with execgen:inline. + n := extractInlineFuncDecls(f, templateFuncMap) + + // Do a second pass over the AST, this time replacing calls to the inlined + // functions with the inlined function itself. + dstutil.Apply(n, func(cursor *dstutil.Cursor) bool { + n := cursor.Node() + // There are two cases. AssignStmt, which are like: + // a = foo() + // and ExprStmt, which are simply: + // foo() + // AssignStmts need to do extra work for inlining, because we have to + // simulate producing return values. + switch n := n.(type) { + case *dst.AssignStmt: + // Search for assignment function call: + // a = foo() + callExpr, ok := n.Rhs[0].(*dst.CallExpr) + if !ok { + return true + } + decl := getTemplateFunc(templateFuncMap, callExpr) + if decl == nil { + return true + } + if len(n.Rhs) > 1 { + panic("can't do template replacement with more than a single RHS to a CallExpr") + } + + // Now we've got a callExpr. We need to inline the function call, and + // convert the result into the assignment variable. + + // Produce declarations for each return value of the function to inline. + retValDeclStmt, retValNames := extractReturnValues(decl) + // inlinedStatements is a BlockStmt (a set of statements within curly + // braces) that contains the entirety of the statements that result from + // inlining the call. We make this a BlockStmt to avoid issues with + // variable shadowing. + // The first thing that goes in the BlockStmt is the ret val declarations. + // When we're done, the BlockStmt for a statement + // a, b = foo(x, y) + // where foo was defined as + // func foo(b string, c string) { ... } + // will look like: + // { + // var ( + // __retval_0 bool + // __retval_1 int + // ) + // ... + // { + // b := x + // c := y + // ... the contents of func foo() except its return ... + // { + // // If foo() had `return true, j`, we'll generate the code: + // __retval_0 = true + // __retval_1 = j + // } + // } + // a = __retval_0 + // b = __retval_1 + // } + inlinedStatements := &dst.BlockStmt{ + List: []dst.Stmt{retValDeclStmt}, + } + + // Replace return statements with assignments to the return values. + // Make a copy of the function to inline, and walk through it, replacing + // return statements at the end of the body with assignments to the return + // value declarations we made first. + body := dst.Clone(decl.Body).(*dst.BlockStmt) + body = replaceReturnStatements(decl.Name.Name, body, func(stmt *dst.ReturnStmt) dst.Stmt { + returnAssignmentSpecs := make([]dst.Stmt, len(retValNames)) + for i := range retValNames { + returnAssignmentSpecs[i] = &dst.AssignStmt{ + Lhs: []dst.Expr{dst.NewIdent(retValNames[i])}, + Tok: token.ASSIGN, + Rhs: []dst.Expr{stmt.Results[i]}, + } + } + // Replace the return with the new assignments. + return &dst.BlockStmt{List: returnAssignmentSpecs} + }) + // Reassign input parameters to formal parameters. + reassignmentStmt := getFormalParamReassignments(decl, callExpr) + inlinedStatements.List = append(inlinedStatements.List, &dst.BlockStmt{ + List: append([]dst.Stmt{reassignmentStmt}, body.List...), + }) + // Assign mangled return values to the original assignment variables. + newAssignment := dst.Clone(n).(*dst.AssignStmt) + newAssignment.Rhs = make([]dst.Expr, len(retValNames)) + for i := range retValNames { + newAssignment.Rhs[i] = dst.NewIdent(retValNames[i]) + } + inlinedStatements.List = append(inlinedStatements.List, newAssignment) + cursor.Replace(inlinedStatements) + + case *dst.ExprStmt: + // Search for raw function call: + // foo() + callExpr, ok := n.X.(*dst.CallExpr) + if !ok { + return true + } + decl := getTemplateFunc(templateFuncMap, callExpr) + if decl == nil { + return true + } + + reassignments := getFormalParamReassignments(decl, callExpr) + + // This case is simpler than the AssignStmt case. It's identical, except + // there is no mangled return value name block, nor re-assignment to + // the mangled returns after the inlined function. + funcBlock := &dst.BlockStmt{ + List: []dst.Stmt{reassignments}, + } + body := dst.Clone(decl.Body).(*dst.BlockStmt) + + // Remove return values if there are any, since we're ignoring returns + // as a raw function call. + body = replaceReturnStatements(decl.Name.Name, body, nil) + // Add the inlined function body to the block. + funcBlock.List = append(funcBlock.List, body.List...) + + cursor.Replace(funcBlock) + } + return true + }, nil) + + b := bytes.Buffer{} + _ = decorator.Fprint(&b, f) + return b.String(), nil +} + +// extractInlineFuncDecls searches the input file for functions that are +// annotated with execgen:inline, extracts them into templateFuncMap, and +// deletes them from the AST. +func extractInlineFuncDecls(f *dst.File, templateFuncMap map[string]*dst.FuncDecl) dst.Node { + return dstutil.Apply(f, func(cursor *dstutil.Cursor) bool { + n := cursor.Node() + switch n := n.(type) { + case *dst.FuncDecl: + var mustInline bool + for _, dec := range n.Decorations().Start.All() { + if dec == "// execgen:inline" { + mustInline = true + break + } + } + if !mustInline { + // Nothing to do, but recurse further. + return true + } + for _, p := range n.Type.Params.List { + if len(p.Names) > 1 { + panic("can't currently deal with multiple names per type in decls") + } + } + // Store the function in a map. + templateFuncMap[n.Name.Name] = n + // Replace the function textually with a fake constant, such as: + // `const _ = "inlined_blahFunc"`. We do this instead + // of completely deleting it to prevent "important comments" above the + // function to be deleted, such as template comments like {{end}}. This + // is kind of a quirk of the way the comments are parsed, but nonetheless + // this is an easy fix so we'll leave it for now. + cursor.Replace(&dst.GenDecl{ + Tok: token.CONST, + Specs: []dst.Spec{ + &dst.ValueSpec{ + Names: []*dst.Ident{dst.NewIdent("_")}, + Values: []dst.Expr{ + &dst.BasicLit{ + Kind: token.STRING, + Value: fmt.Sprintf(`"inlined_%s"`, n.Name.Name), + }, + }, + }, + }, + Decs: dst.GenDeclDecorations{ + NodeDecs: n.Decs.NodeDecs, + }, + }) + return false + } + return true + }, nil) +} + +// extractReturnValues generates return value variables. It will produce one +// statement per return value of the input FuncDecl. For example, for +// a FuncDecl that returns two boolean arguments, lastVal and lastValNull, +// two statements will be returned: +// var __retval_lastVal bool +// var __retval_lastValNull bool +// The second return is a slice of the names of each of the mangled return +// declarations, in this example, __retval_lastVal and __retval_lastValNull. +func extractReturnValues(decl *dst.FuncDecl) (retValDeclStmt dst.Stmt, retValNames []string) { + if decl.Type.Results == nil { + return &dst.EmptyStmt{}, nil + } + results := decl.Type.Results.List + retValNames = make([]string, len(results)) + specs := make([]dst.Spec, len(results)) + for i, result := range results { + var retvalName string + // Make a mangled name. + if len(result.Names) == 0 { + retvalName = fmt.Sprintf("__retval_%d", i) + } else { + retvalName = fmt.Sprintf("__retval_%s", result.Names[0]) + } + retValNames[i] = retvalName + specs[i] = &dst.ValueSpec{ + Names: []*dst.Ident{dst.NewIdent(retvalName)}, + Type: dst.Clone(result.Type).(dst.Expr), + } + } + return &dst.DeclStmt{ + Decl: &dst.GenDecl{ + Tok: token.VAR, + Specs: specs, + }, + }, retValNames +} + +// getFormalParamReassignments creates a new DEFINE (:=) statement per parameter +// to a FuncDecl, which makes a fresh variable with the same name as the formal +// parameter name and assigns it to the corresponding name in the CallExpr. +// +// For example, given a FuncDecl: +// +// func foo(a int, b string) { ... } +// +// and a CallExpr +// +// foo(x, y) +// +// we'll return the statement: +// +// var ( +// a int = x +// b string = y +// ) +// +// In the case where the formal parameter name is the same as the the input +// parameter name, no extra assignment is created. +func getFormalParamReassignments(decl *dst.FuncDecl, callExpr *dst.CallExpr) dst.Stmt { + formalParams := decl.Type.Params.List + reassignmentSpecs := make([]dst.Spec, 0, len(formalParams)) + for i, formalParam := range formalParams { + if inputIdent, ok := callExpr.Args[i].(*dst.Ident); ok && inputIdent.Name == formalParam.Names[0].Name { + continue + } + reassignmentSpecs = append(reassignmentSpecs, &dst.ValueSpec{ + Names: []*dst.Ident{dst.NewIdent(formalParam.Names[0].Name)}, + Type: dst.Clone(formalParam.Type).(dst.Expr), + Values: []dst.Expr{callExpr.Args[i]}, + }) + } + if len(reassignmentSpecs) == 0 { + return &dst.EmptyStmt{} + } + return &dst.DeclStmt{ + Decl: &dst.GenDecl{ + Tok: token.VAR, + Specs: reassignmentSpecs, + }, + } +} + +// replaceReturnStatements edits the input BlockStmt, from the function funcName, +// replacing ReturnStmts at the end of the BlockStmts with the results of +// applying returnEditor on the ReturnStmt or deleting them if the modifier is +// nil. +// It will panic if any return statements are not in the final position of the +// input block. +func replaceReturnStatements( + funcName string, stmt *dst.BlockStmt, returnModifier func(*dst.ReturnStmt) dst.Stmt, +) *dst.BlockStmt { + // Remove return values if there are any, since we're ignoring returns + // as a raw function call. + var seenReturn bool + return dstutil.Apply(stmt, func(cursor *dstutil.Cursor) bool { + if seenReturn { + panic(fmt.Errorf("can't inline function %s: return not at end of body (found %s)", funcName, cursor.Node())) + } + n := cursor.Node() + switch t := n.(type) { + case *dst.FuncLit: + // A FuncLit is a function literal, like: + // x := func() int { return 3 } + // We don't recurse into function literals since the return statements + // they contain aren't relevant to the inliner. + return false + case *dst.ReturnStmt: + seenReturn = true + if returnModifier == nil { + cursor.Delete() + return false + } + cursor.Replace(returnModifier(t)) + return false + } + return true + }, nil).(*dst.BlockStmt) +} + +// getTemplateFunc returns the corresponding FuncDecl for a CallExpr from the +// map, using the CallExpr's name to look up the FuncDecl from templateFuncs. +func getTemplateFunc(templateFuncs map[string]*dst.FuncDecl, n *dst.CallExpr) *dst.FuncDecl { + ident, ok := n.Fun.(*dst.Ident) + if !ok { + return nil + } + + decl, ok := templateFuncs[ident.Name] + if !ok { + return nil + } + if decl.Type.Params.NumFields() != len(n.Args) { + panic(errors.Newf( + "%s expected %d arguments, found %d", + decl.Name, decl.Type.Params.NumFields(), len(n.Args)), + ) + } + return decl +} diff --git a/pkg/sql/colexec/execgen/inline_test.go b/pkg/sql/colexec/execgen/inline_test.go new file mode 100644 index 000000000000..d550f56aca6b --- /dev/null +++ b/pkg/sql/colexec/execgen/inline_test.go @@ -0,0 +1,102 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package execgen + +import ( + "testing" + + "github.com/dave/dst" + "github.com/stretchr/testify/assert" +) + +func TestGetFormalParamReassignments(t *testing.T) { + tests := []struct { + funcDecl string + callExpr string + + expectedStmts string + }{ + { + funcDecl: `func a() {}`, + callExpr: `a()`, + expectedStmts: ``, + }, + { + funcDecl: `func a(a int) {}`, + callExpr: `a(b)`, + expectedStmts: `var a int = b`, + }, + { + funcDecl: `func a(a int, b int) {}`, + callExpr: `a(x, y)`, + expectedStmts: `var ( + a int = x + b int = y + )`, + }, + { + funcDecl: `func a(a int, b int) {}`, + callExpr: `a(a, c)`, + expectedStmts: `var b int = c`, + }, + } + for _, tt := range tests { + callExpr := parseStmts(tt.callExpr)[0].(*dst.ExprStmt).X.(*dst.CallExpr) + funcDecl := parseDecls(tt.funcDecl)[0].(*dst.FuncDecl) + stmt := getFormalParamReassignments(funcDecl, callExpr) + actual := prettyPrintStmts(stmt) + assert.Equal(t, tt.expectedStmts, actual) + } +} + +func TestExtractReturnValues(t *testing.T) { + tests := []struct { + decl string + expectedRetDecls string + }{ + { + decl: "func foo(a int) {}", + expectedRetDecls: "", + }, + { + decl: "func foo(a int) (int, string) {}", + expectedRetDecls: `var ( + __retval_0 int + __retval_1 string + )`, + }, + { + decl: "func foo(a int) int {}", + expectedRetDecls: `var __retval_0 int`, + }, + { + decl: "func foo(a int) (a int, b string) {}", + expectedRetDecls: `var ( + __retval_a int + __retval_b string + )`, + }, + } + for _, tt := range tests { + decl := parseDecls(tt.decl)[0].(*dst.FuncDecl) + retValDecl, retValNames := extractReturnValues(decl) + if _, ok := retValDecl.(*dst.EmptyStmt); ok { + assert.Equal(t, 0, len(retValNames)) + } else { + specs := retValDecl.(*dst.DeclStmt).Decl.(*dst.GenDecl).Specs + assert.Equal(t, len(specs), len(retValNames)) + for i := range retValNames { + assert.Equal(t, retValNames[i], specs[i].(*dst.ValueSpec).Names[0].Name) + } + } + assert.Equal(t, tt.expectedRetDecls, prettyPrintStmts(retValDecl)) + } +} diff --git a/pkg/sql/colexec/execgen/testdata/inline b/pkg/sql/colexec/execgen/testdata/inline new file mode 100644 index 000000000000..35ea0d5c476a --- /dev/null +++ b/pkg/sql/colexec/execgen/testdata/inline @@ -0,0 +1,146 @@ +inline +package main + +func a() { + b() +} + +func c() { + b() +} + +// execgen:inline +func b() { + foo = bar +} +---- +---- +package main + +func a() { + { + foo = bar + } +} + +func c() { + { + foo = bar + } +} + +// execgen:inline +const _ = "inlined_b" +---- +---- + +inline +package main + +func a() { + b(x, b, y) +} + +// execgen:inline +func b(a int, b int, c int) { + foo = bar +} +---- +---- +package main + +func a() { + { + var ( + a int = x + c int = y + ) + foo = bar + } +} + +// execgen:inline +const _ = "inlined_b" +---- +---- + +inline +package main + +func a() { + ret1, ret2 := b(x, b, y) +} + +func c() { + ret3, ret4 := b(x, b, y) +} + +func d() { + b(x, b, y) +} + +// execgen:inline +func b(a int, b int, c int) (int, int) { + foo = bar + return b, c +} +---- +---- +package main + +func a() { + { + var ( + __retval_0 int + __retval_1 int + ) + { + var ( + a int = x + c int = y + ) + foo = bar + { + __retval_0 = b + __retval_1 = c + } + } + ret1, ret2 := __retval_0, __retval_1 + } +} + +func c() { + { + var ( + __retval_0 int + __retval_1 int + ) + { + var ( + a int = x + c int = y + ) + foo = bar + { + __retval_0 = b + __retval_1 = c + } + } + ret3, ret4 := __retval_0, __retval_1 + } +} + +func d() { + { + var ( + a int = x + c int = y + ) + foo = bar + } +} + +// execgen:inline +const _ = "inlined_b" +---- +---- diff --git a/pkg/sql/colexec/execgen/util_test.go b/pkg/sql/colexec/execgen/util_test.go new file mode 100644 index 000000000000..8c141afd833d --- /dev/null +++ b/pkg/sql/colexec/execgen/util_test.go @@ -0,0 +1,70 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package execgen + +import ( + "fmt" + "strings" + + "github.com/dave/dst" + "github.com/dave/dst/decorator" +) + +func prettyPrintStmts(stmts ...dst.Stmt) string { + if len(stmts) == 0 { + return "" + } + f := &dst.File{ + Name: dst.NewIdent("main"), + Decls: []dst.Decl{ + &dst.FuncDecl{ + Name: dst.NewIdent("test"), + Type: &dst.FuncType{}, + Body: &dst.BlockStmt{ + List: stmts, + }, + }, + }, + } + var ret strings.Builder + _ = decorator.Fprint(&ret, f) + prelude := `package main + +func test() { +` + postlude := `} +` + s := ret.String() + return strings.TrimSpace(s[len(prelude) : len(s)-len(postlude)]) +} + +func parseStmts(stmts string) []dst.Stmt { + inputStr := fmt.Sprintf(`package main +func test() { + %s +}`, stmts) + f, err := decorator.Parse(inputStr) + if err != nil { + panic(err) + } + return f.Decls[0].(*dst.FuncDecl).Body.List +} + +func parseDecls(decls string) []dst.Decl { + inputStr := fmt.Sprintf(`package main +%s +`, decls) + f, err := decorator.Parse(inputStr) + if err != nil { + panic(err) + } + return f.Decls +} diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index ab910028e44d..f982d11a0de4 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -816,7 +816,7 @@ func TestLint(t *testing.T) { "git", "grep", "-nE", - `\.Clone\([^)]`, + `proto\.Clone\([^)]`, "--", "*.go", ":!util/protoutil/clone_test.go", @@ -1482,6 +1482,11 @@ func TestLint(t *testing.T) { ":!sql/colexecbase/colexecerror/error.go", ":!sql/colexec/execpb/stats.pb.go", ":!sql/colflow/vectorized_panic_propagation_test.go", + // This exception is because execgen itself uses panics during code + // generation - not at execution time. The (glob,exclude) directive + // (see git help gitglossary) makes * behave like a normal, single dir + // glob, and exclude is the synonym of !. + ":(glob,exclude)sql/colexec/execgen/*.go", ) if err != nil { t.Fatal(err) diff --git a/vendor b/vendor index 052117d35784..2096d93a4d7f 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit 052117d35784f8106b2beb1f09b54eeeb8946617 +Subproject commit 2096d93a4d7fed85e90e897e487366bff4813d00 From 1cee807e31951c4d4faa944e2a220bfea78f9fa7 Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Thu, 4 Jun 2020 18:08:22 -0400 Subject: [PATCH 10/46] cloud: Added support to all ExternalStorage ReadFile methods, to raise a sentinel ErrFileDoesNotExist error The `ReadFile` interface method is responsible for returning a `Reader` which can be used to stream data from an external storage source. There are also instances where we use this method to solely check for the existence (or absence) of a particular file/object, by attempting to open a stream. eg: checking for BackupManifest/BackupManifestCheckpoint before exporting data. Previously, we would treat any error returned by the storage vendor API as a signal for the file/object not existing. This change adds logic to catch the native "file does not exist" errors for each storage provider, and throw a sentinel error to users of the `ReadFile` method. This allows for more careful error handling. Relevant unit tests have also been added. Release note: None --- pkg/blobs/service.go | 11 ++++- pkg/ccl/backupccl/manifest_handling.go | 30 ++++++------ pkg/storage/cloud/azure_storage.go | 7 +++ pkg/storage/cloud/external_storage.go | 7 +++ pkg/storage/cloud/external_storage_test.go | 47 +++++++++++++++---- pkg/storage/cloud/gcs_storage.go | 7 +++ pkg/storage/cloud/gcs_storage_test.go | 41 +++++++++++++++- pkg/storage/cloud/http_storage.go | 8 +++- pkg/storage/cloud/http_storage_test.go | 2 +- pkg/storage/cloud/nodelocal_storage.go | 17 ++++++- pkg/storage/cloud/s3_storage.go | 8 ++++ pkg/storage/cloud/s3_storage_test.go | 54 ++++++++++++++++++++++ 12 files changed, 210 insertions(+), 29 deletions(-) diff --git a/pkg/blobs/service.go b/pkg/blobs/service.go index 8517a74aac27..a0eee6f3afcb 100644 --- a/pkg/blobs/service.go +++ b/pkg/blobs/service.go @@ -27,10 +27,13 @@ package blobs import ( "context" + "os" "github.com/cockroachdb/cockroach/pkg/blobs/blobspb" "github.com/cockroachdb/errors" + "google.golang.org/grpc/codes" "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" ) // Service implements the gRPC BlobService which exchanges bulk files between different nodes. @@ -89,5 +92,11 @@ func (s *Service) Delete( // Stat implements the gRPC service. func (s *Service) Stat(ctx context.Context, req *blobspb.StatRequest) (*blobspb.BlobStat, error) { - return s.localStorage.Stat(req.Filename) + resp, err := s.localStorage.Stat(req.Filename) + if os.IsNotExist(err) { + // gRPC hides the underlying golang ErrNotExist error, so we send back an + // equivalent gRPC error which can be handled gracefully on the client side. + return nil, status.Error(codes.NotFound, err.Error()) + } + return resp, err } diff --git a/pkg/ccl/backupccl/manifest_handling.go b/pkg/ccl/backupccl/manifest_handling.go index 924361a06d00..fd89dc0b95e4 100644 --- a/pkg/ccl/backupccl/manifest_handling.go +++ b/pkg/ccl/backupccl/manifest_handling.go @@ -103,8 +103,10 @@ func readBackupManifestFromStore( func containsManifest(ctx context.Context, exportStore cloud.ExternalStorage) (bool, error) { r, err := exportStore.ReadFile(ctx, BackupManifestName) if err != nil { - //nolint:returnerrcheck - return false, nil /* TODO(dt): only silence non-exists errors */ + if errors.Is(err, cloud.ErrFileDoesNotExist) { + return false, nil + } + return false, err } r.Close() return true, nil @@ -661,28 +663,30 @@ func VerifyUsableExportTarget( readable string, encryption *roachpb.FileEncryptionOptions, ) error { - if r, err := exportStore.ReadFile(ctx, BackupManifestName); err == nil { - // TODO(dt): If we audit exactly what not-exists error each ExternalStorage - // returns (and then wrap/tag them), we could narrow this check. + r, err := exportStore.ReadFile(ctx, BackupManifestName) + if err == nil { r.Close() return pgerror.Newf(pgcode.FileAlreadyExists, "%s already contains a %s file", readable, BackupManifestName) } - if r, err := exportStore.ReadFile(ctx, BackupManifestName); err == nil { - // TODO(dt): If we audit exactly what not-exists error each ExternalStorage - // returns (and then wrap/tag them), we could narrow this check. - r.Close() - return pgerror.Newf(pgcode.FileAlreadyExists, - "%s already contains a %s file", - readable, BackupManifestName) + + if !errors.Is(err, cloud.ErrFileDoesNotExist) { + return errors.Wrapf(err, "%s returned an unexpected error when checking for the existence of %s file", readable, BackupManifestName) } - if r, err := exportStore.ReadFile(ctx, BackupManifestCheckpointName); err == nil { + + r, err = exportStore.ReadFile(ctx, BackupManifestCheckpointName) + if err == nil { r.Close() return pgerror.Newf(pgcode.FileAlreadyExists, "%s already contains a %s file (is another operation already in progress?)", readable, BackupManifestCheckpointName) } + + if !errors.Is(err, cloud.ErrFileDoesNotExist) { + return errors.Wrapf(err, "%s returned an unexpected error when checking for the existence of %s file", readable, BackupManifestCheckpointName) + } + if err := writeBackupManifest( ctx, settings, exportStore, BackupManifestCheckpointName, encryption, &BackupManifest{}, ); err != nil { diff --git a/pkg/storage/cloud/azure_storage.go b/pkg/storage/cloud/azure_storage.go index 264947e613a8..12db2f8ab96e 100644 --- a/pkg/storage/cloud/azure_storage.go +++ b/pkg/storage/cloud/azure_storage.go @@ -100,6 +100,13 @@ func (s *azureStorage) ReadFile(ctx context.Context, basename string) (io.ReadCl blob := s.getBlob(basename) get, err := blob.Download(ctx, 0, 0, azblob.BlobAccessConditions{}, false) if err != nil { + if azerr := (azblob.StorageError)(nil); errors.As(err, &azerr) { + switch azerr.ServiceCode() { + // TODO(adityamaru): Investigate whether both these conditions are required. + case azblob.ServiceCodeBlobNotFound, azblob.ServiceCodeResourceNotFound: + return nil, errors.Wrapf(ErrFileDoesNotExist, "azure blob does not exist: %s", err.Error()) + } + } return nil, errors.Wrap(err, "failed to create azure reader") } reader := get.Body(azblob.RetryReaderOptions{MaxRetryRequests: 3}) diff --git a/pkg/storage/cloud/external_storage.go b/pkg/storage/cloud/external_storage.go index cfecf28b36bb..6bf9a9a82a26 100644 --- a/pkg/storage/cloud/external_storage.go +++ b/pkg/storage/cloud/external_storage.go @@ -88,6 +88,11 @@ var redactedQueryParams = map[string]struct{}{ // ErrListingUnsupported is a marker for indicating listing is unsupported. var ErrListingUnsupported = errors.New("listing is not supported") +// ErrFileDoesNotExist is a sentinel error for indicating that a specified +// bucket/object/key/file (depending on storage terminology) does not exist. +// This error is raised by the ReadFile method. +var ErrFileDoesNotExist = errors.New("external_storage: file doesn't exist") + // ExternalStorageFactory describes a factory function for ExternalStorage. type ExternalStorageFactory func(ctx context.Context, dest roachpb.ExternalStorage) (ExternalStorage, error) @@ -113,6 +118,8 @@ type ExternalStorage interface { Conf() roachpb.ExternalStorage // ReadFile should return a Reader for requested name. + // ErrFileDoesNotExist is raised if `basename` cannot be located in storage. + // This can be leveraged for an existence check. ReadFile(ctx context.Context, basename string) (io.ReadCloser, error) // WriteFile should write the content to requested name. diff --git a/pkg/storage/cloud/external_storage_test.go b/pkg/storage/cloud/external_storage_test.go index 789c6fba52f3..c779b96bd4ed 100644 --- a/pkg/storage/cloud/external_storage_test.go +++ b/pkg/storage/cloud/external_storage_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/workload" "github.com/cockroachdb/cockroach/pkg/workload/bank" + "github.com/cockroachdb/errors" "github.com/spf13/pflag" "github.com/stretchr/testify/require" "golang.org/x/oauth2/google" @@ -125,9 +126,8 @@ func testExportStoreWithExternalIOConfig( if !bytes.Equal(res, payload) { t.Fatalf("got %v expected %v", res, payload) } - if err := s.Delete(ctx, name); err != nil { - t.Fatal(err) - } + + require.NoError(t, s.Delete(ctx, name)) } }) @@ -160,9 +160,7 @@ func testExportStoreWithExternalIOConfig( if !bytes.Equal(content, testingContent) { t.Fatalf("wrong content") } - if err := s.Delete(ctx, testingFilename); err != nil { - t.Fatal(err) - } + require.NoError(t, s.Delete(ctx, testingFilename)) }) if skipSingleFile { return @@ -188,9 +186,7 @@ func testExportStoreWithExternalIOConfig( if !bytes.Equal(content, []byte("aaa")) { t.Fatalf("wrong content") } - if err := s.Delete(ctx, testingFilename); err != nil { - t.Fatal(err) - } + require.NoError(t, s.Delete(ctx, testingFilename)) }) t.Run("write-single-file-by-uri", func(t *testing.T) { const testingFilename = "B" @@ -214,9 +210,40 @@ func testExportStoreWithExternalIOConfig( if !bytes.Equal(content, []byte("bbb")) { t.Fatalf("wrong content") } - if err := s.Delete(ctx, testingFilename); err != nil { + + require.NoError(t, s.Delete(ctx, testingFilename)) + }) + // This test ensures that the ReadFile method of the ExternalStorage interface + // raises a sentinel error indicating that a requested bucket/key/file/object + // (based on the storage system) could not be found. + t.Run("file-does-not-exist", func(t *testing.T) { + const testingFilename = "A" + if err := s.WriteFile(ctx, testingFilename, bytes.NewReader([]byte("aaa"))); err != nil { + t.Fatal(err) + } + singleFile := storeFromURI(ctx, t, storeURI, clientFactory) + defer singleFile.Close() + + // Read a valid file. + res, err := singleFile.ReadFile(ctx, testingFilename) + if err != nil { t.Fatal(err) } + defer res.Close() + content, err := ioutil.ReadAll(res) + if err != nil { + t.Fatal(err) + } + // Verify the result contains what we wrote. + if !bytes.Equal(content, []byte("aaa")) { + t.Fatalf("wrong content") + } + + // Attempt to read a file which does not exist. + _, err = singleFile.ReadFile(ctx, "file_does_not_exist") + require.Error(t, err) + require.True(t, errors.Is(err, ErrFileDoesNotExist), "Expected a file does not exist error but returned %s") + require.NoError(t, s.Delete(ctx, testingFilename)) }) } diff --git a/pkg/storage/cloud/gcs_storage.go b/pkg/storage/cloud/gcs_storage.go index 98fa9fca91d4..a6d69a31e3ed 100644 --- a/pkg/storage/cloud/gcs_storage.go +++ b/pkg/storage/cloud/gcs_storage.go @@ -229,6 +229,13 @@ func (g *gcsStorage) ReadFile(ctx context.Context, basename string) (io.ReadClos object: path.Join(g.prefix, basename), } if err := reader.openStream(); err != nil { + // The Google SDK has a specialized ErrBucketDoesNotExist error, but + // the code path from this method first triggers an ErrObjectNotExist in + // both scenarios - when a Bucket does not exist or an Object does not + // exist. + if errors.Is(err, gcs.ErrObjectNotExist) { + return nil, errors.Wrapf(ErrFileDoesNotExist, "gcs object does not exist: %s", err.Error()) + } return nil, err } return reader, nil diff --git a/pkg/storage/cloud/gcs_storage_test.go b/pkg/storage/cloud/gcs_storage_test.go index e5ae5b7a7c9f..9f22b2269d1a 100644 --- a/pkg/storage/cloud/gcs_storage_test.go +++ b/pkg/storage/cloud/gcs_storage_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/sysutil" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -76,7 +77,7 @@ func (c *antagonisticConn) Read(b []byte) (int, error) { func TestAntagonisticRead(t *testing.T) { if os.Getenv("GOOGLE_APPLICATION_CREDENTIALS") == "" { // This test requires valid GS credential file. - return + t.Skip("GOOGLE_APPLICATION_CREDENTIALS env var must be set") } rnd, _ := randutil.NewPseudoRand() @@ -110,3 +111,41 @@ func TestAntagonisticRead(t *testing.T) { _, err = ioutil.ReadAll(stream) require.NoError(t, err) } + +// TestFileDoesNotExist ensures that the ReadFile method of google cloud storage +// returns a sentinel error when the `Bucket` or `Object` being read do not +// exist. +func TestFileDoesNotExist(t *testing.T) { + if os.Getenv("GOOGLE_APPLICATION_CREDENTIALS") == "" { + // This test requires valid GS credential file. + t.Skip("GOOGLE_APPLICATION_CREDENTIALS env var must be set") + } + + { + // Invalid gsFile. + gsFile := "gs://cockroach-fixtures/tpch-csv/sf-1/invalid_region.tbl?AUTH=implicit" + conf, err := ExternalStorageConfFromURI(gsFile) + require.NoError(t, err) + + s, err := MakeExternalStorage( + context.Background(), conf, base.ExternalIODirConfig{}, testSettings, nil) + require.NoError(t, err) + _, err = s.ReadFile(context.Background(), "") + require.Error(t, err, "") + require.True(t, errors.Is(err, ErrFileDoesNotExist)) + } + + { + // Invalid gsBucket. + gsFile := "gs://cockroach-fixtures-invalid/tpch-csv/sf-1/region.tbl?AUTH=implicit" + conf, err := ExternalStorageConfFromURI(gsFile) + require.NoError(t, err) + + s, err := MakeExternalStorage( + context.Background(), conf, base.ExternalIODirConfig{}, testSettings, nil) + require.NoError(t, err) + _, err = s.ReadFile(context.Background(), "") + require.Error(t, err, "") + require.True(t, errors.Is(err, ErrFileDoesNotExist)) + } +} diff --git a/pkg/storage/cloud/http_storage.go b/pkg/storage/cloud/http_storage.go index e339aaecd44e..0316f0e2e2c2 100644 --- a/pkg/storage/cloud/http_storage.go +++ b/pkg/storage/cloud/http_storage.go @@ -346,11 +346,15 @@ func (h *httpStorage) req( switch resp.StatusCode { case 200, 201, 204, 206: - // Pass. + // Pass. default: body, _ := ioutil.ReadAll(resp.Body) _ = resp.Body.Close() - return nil, errors.Errorf("error response from server: %s %q", resp.Status, body) + err := errors.Errorf("error response from server: %s %q", resp.Status, body) + if err != nil && resp.StatusCode == 404 { + err = errors.Wrapf(ErrFileDoesNotExist, "http storage file does not exist: %s", err.Error()) + } + return nil, err } return resp, nil } diff --git a/pkg/storage/cloud/http_storage_test.go b/pkg/storage/cloud/http_storage_test.go index 0a8c64ee31d7..3a54810fd1b3 100644 --- a/pkg/storage/cloud/http_storage_test.go +++ b/pkg/storage/cloud/http_storage_test.go @@ -110,7 +110,7 @@ func TestPutHttp(t *testing.T) { srv, files, cleanup := makeServer() defer cleanup() testExportStore(t, srv.String(), false) - if expected, actual := 13, files(); expected != actual { + if expected, actual := 14, files(); expected != actual { t.Fatalf("expected %d files to be written to single http store, got %d", expected, actual) } }) diff --git a/pkg/storage/cloud/nodelocal_storage.go b/pkg/storage/cloud/nodelocal_storage.go index 99a2f07ca8b6..9d4873f7845d 100644 --- a/pkg/storage/cloud/nodelocal_storage.go +++ b/pkg/storage/cloud/nodelocal_storage.go @@ -14,6 +14,7 @@ import ( "context" "fmt" "io" + "os" "path" "strings" @@ -21,6 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/errors" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) type localFileStorage struct { @@ -78,7 +81,19 @@ func (l *localFileStorage) WriteFile( } func (l *localFileStorage) ReadFile(ctx context.Context, basename string) (io.ReadCloser, error) { - return l.blobClient.ReadFile(ctx, joinRelativePath(l.base, basename)) + var err error + var reader io.ReadCloser + if reader, err = l.blobClient.ReadFile(ctx, joinRelativePath(l.base, basename)); err != nil { + // The format of the error returned by the above ReadFile call differs based + // on whether we are reading from a local or remote nodelocal store. + // The local store returns a golang native ErrNotFound, whereas the remote + // store returns a gRPC native NotFound error. + if os.IsNotExist(err) || status.Code(err) == codes.NotFound { + return nil, errors.Wrapf(ErrFileDoesNotExist, "nodelocal storage file does not exist: %s", err.Error()) + } + return nil, err + } + return reader, nil } func (l *localFileStorage) ListFiles(ctx context.Context, patternSuffix string) ([]string, error) { diff --git a/pkg/storage/cloud/s3_storage.go b/pkg/storage/cloud/s3_storage.go index 4700a7e6d550..1e7d54777ed4 100644 --- a/pkg/storage/cloud/s3_storage.go +++ b/pkg/storage/cloud/s3_storage.go @@ -18,6 +18,7 @@ import ( "strings" "github.com/aws/aws-sdk-go/aws" + "github.com/aws/aws-sdk-go/aws/awserr" "github.com/aws/aws-sdk-go/aws/session" "github.com/aws/aws-sdk-go/service/s3" "github.com/aws/aws-sdk-go/service/s3/s3manager" @@ -171,6 +172,13 @@ func (s *s3Storage) ReadFile(ctx context.Context, basename string) (io.ReadClose Key: aws.String(path.Join(s.prefix, basename)), }) if err != nil { + if aerr := (awserr.Error)(nil); errors.As(err, &aerr) { + switch aerr.Code() { + // Relevant 404 errors reported by AWS. + case s3.ErrCodeNoSuchBucket, s3.ErrCodeNoSuchKey: + return nil, errors.Wrapf(ErrFileDoesNotExist, "s3 object does not exist: %s", err.Error()) + } + } return nil, errors.Wrap(err, "failed to get s3 object") } return out.Body, nil diff --git a/pkg/storage/cloud/s3_storage_test.go b/pkg/storage/cloud/s3_storage_test.go index 0eb9bd9a4ec3..e7641f38f023 100644 --- a/pkg/storage/cloud/s3_storage_test.go +++ b/pkg/storage/cloud/s3_storage_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/blobs" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -152,3 +153,56 @@ func TestS3DisallowImplicitCredentials(t *testing.T) { require.Error(t, err) require.True(t, strings.Contains(err.Error(), "implicit")) } + +// S3 has two "does not exist" errors - ErrCodeNoSuchBucket and ErrCodeNoSuchKey. +// ErrCodeNoSuchKey is tested via the general test in external_storage_test.go. +// This test attempts to ReadFile from a bucket which does not exist. +func TestS3BucketDoesNotExist(t *testing.T) { + defer leaktest.AfterTest(t)() + + q := make(url.Values) + expect := map[string]string{ + "AWS_S3_ENDPOINT": S3EndpointParam, + "AWS_S3_ENDPOINT_KEY": S3AccessKeyParam, + "AWS_S3_ENDPOINT_REGION": S3RegionParam, + "AWS_S3_ENDPOINT_SECRET": S3SecretParam, + } + for env, param := range expect { + v := os.Getenv(env) + if v == "" { + t.Skipf("%s env var must be set", env) + } + q.Add(param, v) + } + + bucket := "invalid-bucket" + u := url.URL{ + Scheme: "s3", + Host: bucket, + Path: "backup-test", + RawQuery: q.Encode(), + } + + ctx := context.Background() + + conf, err := ExternalStorageConfFromURI(u.String()) + if err != nil { + t.Fatal(err) + } + + // Setup a sink for the given args. + clientFactory := blobs.TestBlobServiceClient(testSettings.ExternalIODir) + s, err := MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{}, testSettings, clientFactory) + if err != nil { + t.Fatal(err) + } + defer s.Close() + + if readConf := s.Conf(); readConf != conf { + t.Fatalf("conf does not roundtrip: started with %+v, got back %+v", conf, readConf) + } + + _, err = s.ReadFile(ctx, "") + require.Error(t, err, "") + require.True(t, errors.Is(err, ErrFileDoesNotExist)) +} From 4541522b9b68a1ded33d7a5400f906e9dca85741 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 8 Jun 2020 16:06:16 +0200 Subject: [PATCH 11/46] scrips/release-notes: fix the error encountered on #47761 There's at lease one PR with a missing branch tips in refs/pull. This commit makes the script tolerant of that failure. Release note: None --- scripts/release-notes.py | 28 +++++++++++++++++++++++----- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/scripts/release-notes.py b/scripts/release-notes.py index 23991b6ea8bb..5407b83f2dd6 100755 --- a/scripts/release-notes.py +++ b/scripts/release-notes.py @@ -638,11 +638,9 @@ def completenote(commit, cat, notemsg, authors, pr, title): # C, E, F, and G will each be checked. F is an ancestor of B, so it will be # excluded. E starts with "Merge", so it will not be counted. Only C and G will # have statistics included. -def analyze_pr(merge, pr): +def analyze_pr(merge, pr, parent_idx): allprs.add(pr) - refname = pull_ref_prefix + "/" + pr[1:] - tip = name_to_object(repo, refname) noteexpr = re.compile("^%s: (?P.*) r=.* a=.*" % pr[1:], flags=re.M) m = noteexpr.search(merge.message) @@ -655,6 +653,26 @@ def analyze_pr(merge, pr): title = m.group('message') title = title.strip() + try: + refname = pull_ref_prefix + "/" + pr[1:] + tip = name_to_object(repo, refname) + except exc.BadName: + # Oddly, we have at least one PR (#47761) which does not have a tip + # at /refs/pull/47761, although it's valid and merged. + # As of 2020-06-08 it's the only PR missing a branch tip there. + print("\nuh-oh! can't find PR head in repo", pr, file=sys.stderr) + # We deal with it here assuming that the order of the parents + # of the merge commit is the same as reported by the + # "Merge ..." string in the merge commit's message. + # This happens to be true of the missing PR above as well + # as for several other merge commits with more than two parents. + tip = merge.parents[parent_idx] + print("check at https://github.com/cockroachdb/cockroach/pull/%s that the last commit is %s" % (pr[1:], tip.hexsha), file=sys.stderr) + # TODO(knz): If this method is reliable, this means we don't + # need the pull tips at /refs/pull *at all* which could + # streamline the whole experience. + # This should be investigated further. + merge_base_result = repo.merge_base(merge.parents[0], tip) if len(merge_base_result) == 0: print("uh-oh! can't find merge base! pr", pr, file=sys.stderr) @@ -749,9 +767,9 @@ def analyze_standalone_commit(commit): # Analyze the commit if numbermatch is not None: prs = numbermatch.group("numbers").strip().split(" ") - for pr in prs: + for idx, pr in enumerate(prs): print(" \r%s (%s) " % (pr, ctime), end='', file=sys.stderr) - analyze_pr(commit, pr) + analyze_pr(commit, pr, idx+1) else: print(" \r%s (%s) " % (commit.hexsha[:shamin], ctime), end='', file=sys.stderr) analyze_standalone_commit(commit) From a1028aa21a3e2cfcadadd04f1d6f2878fce8c2f6 Mon Sep 17 00:00:00 2001 From: Spas Bojanov Date: Mon, 8 Jun 2020 10:22:19 -0400 Subject: [PATCH 12/46] jobs: rename Resume to Unpause Resume verb is already used to for the action of starting a job after adoption on a node. Here instead Resume is the opposite action of pausing a job. Release note: none. --- pkg/ccl/importccl/import_processor_test.go | 8 ++++---- pkg/jobs/jobs.go | 4 ++-- pkg/jobs/jobs_test.go | 8 ++++---- pkg/jobs/registry.go | 7 ++++--- pkg/sql/control_jobs.go | 2 +- 5 files changed, 15 insertions(+), 14 deletions(-) diff --git a/pkg/ccl/importccl/import_processor_test.go b/pkg/ccl/importccl/import_processor_test.go index 55cffd80f364..542c5edb34fe 100644 --- a/pkg/ccl/importccl/import_processor_test.go +++ b/pkg/ccl/importccl/import_processor_test.go @@ -659,8 +659,8 @@ func TestCSVImportCanBeResumed(t *testing.T) { resumePos := js.prog.ResumePos[0] t.Logf("Resume pos: %v\n", js.prog.ResumePos[0]) - // Resume the job and wait for it to complete. - if err := registry.Resume(ctx, nil, jobID); err != nil { + // Unpause the job and wait for it to complete. + if err := registry.Unpause(ctx, nil, jobID); err != nil { t.Fatal(err) } js = queryJobUntil(t, sqlDB.DB, jobID, func(js jobState) bool { return jobs.StatusSucceeded == js.status }) @@ -763,8 +763,8 @@ func TestCSVImportMarksFilesFullyProcessed(t *testing.T) { // Send cancellation and unblock import. proceedImport() - // Resume the job and wait for it to complete. - if err := registry.Resume(ctx, nil, jobID); err != nil { + // Unpause the job and wait for it to complete. + if err := registry.Unpause(ctx, nil, jobID); err != nil { t.Fatal(err) } js = queryJobUntil(t, sqlDB.DB, jobID, func(js jobState) bool { return jobs.StatusSucceeded == js.status }) diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go index a227bff7eac3..4e8e6009f490 100644 --- a/pkg/jobs/jobs.go +++ b/pkg/jobs/jobs.go @@ -361,11 +361,11 @@ func (j *Job) paused(ctx context.Context, fn func(context.Context, *kv.Txn) erro }) } -// resumed sets the status of the tracked job to running or reverting iff the +// unpaused sets the status of the tracked job to running or reverting iff the // job is currently paused. It does not directly resume the job; rather, it // expires the job's lease so that a Registry adoption loop detects it and // resumes it. -func (j *Job) resumed(ctx context.Context) error { +func (j *Job) unpaused(ctx context.Context) error { return j.Update(ctx, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { if md.Status == StatusRunning || md.Status == StatusReverting { // Already resumed - do nothing. diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index d1c4ca5eca1b..97337a7fc7af 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -1094,11 +1094,11 @@ func TestJobLifecycle(t *testing.T) { if err := exp.verify(job.ID(), jobs.StatusPaused); err != nil { t.Fatal(err) } - if err := registry.Resume(ctx, nil, *job.ID()); err != nil { + if err := registry.Unpause(ctx, nil, *job.ID()); err != nil { t.Fatal(err) } // Resume the job again to ensure that the resumption is idempotent. - if err := registry.Resume(ctx, nil, *job.ID()); err != nil { + if err := registry.Unpause(ctx, nil, *job.ID()); err != nil { t.Fatal(err) } if err := exp.verify(job.ID(), jobs.StatusRunning); err != nil { @@ -1172,7 +1172,7 @@ func TestJobLifecycle(t *testing.T) { if err := registry.CancelRequested(ctx, nil, *job.ID()); err != nil { t.Fatal(err) } - if err := registry.Resume(ctx, nil, *job.ID()); !testutils.IsError(err, "cannot be resumed") { + if err := registry.Unpause(ctx, nil, *job.ID()); !testutils.IsError(err, "cannot be resumed") { t.Errorf("got unexpected status '%v'", err) } } @@ -1183,7 +1183,7 @@ func TestJobLifecycle(t *testing.T) { t.Fatal(err) } expectedErr := fmt.Sprintf("job with status %s cannot be resumed", jobs.StatusSucceeded) - if err := registry.Resume(ctx, nil, *job.ID()); !testutils.IsError(err, expectedErr) { + if err := registry.Unpause(ctx, nil, *job.ID()); !testutils.IsError(err, expectedErr) { t.Errorf("expected '%s', but got '%v'", expectedErr, err) } } diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index 5abe7f03ce33..9849a053115a 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -709,13 +709,14 @@ func (r *Registry) Failed(ctx context.Context, txn *kv.Txn, id int64, causingErr return job.WithTxn(txn).failed(ctx, causingError, nil) } -// Resume resumes the paused job with id using the specified txn (may be nil). -func (r *Registry) Resume(ctx context.Context, txn *kv.Txn, id int64) error { +// Unpause changes the paused job with id to running or reverting using the +// specified txn (may be nil). +func (r *Registry) Unpause(ctx context.Context, txn *kv.Txn, id int64) error { job, _, err := r.getJobFn(ctx, txn, id) if err != nil { return err } - return job.WithTxn(txn).resumed(ctx) + return job.WithTxn(txn).unpaused(ctx) } // Resumer is a resumable job, and is associated with a Job object. Jobs can be diff --git a/pkg/sql/control_jobs.go b/pkg/sql/control_jobs.go index 1eae2545d610..3d00c259ea82 100644 --- a/pkg/sql/control_jobs.go +++ b/pkg/sql/control_jobs.go @@ -62,7 +62,7 @@ func (n *controlJobsNode) startExec(params runParams) error { case jobs.StatusPaused: err = reg.PauseRequested(params.ctx, params.p.txn, int64(jobID)) case jobs.StatusRunning: - err = reg.Resume(params.ctx, params.p.txn, int64(jobID)) + err = reg.Unpause(params.ctx, params.p.txn, int64(jobID)) case jobs.StatusCanceled: err = reg.CancelRequested(params.ctx, params.p.txn, int64(jobID)) default: From 9a4aae5194009a84ab4895b4844b9f1906c884c8 Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Wed, 3 Jun 2020 15:02:03 -0400 Subject: [PATCH 13/46] sql: disallow cross database type references Fixes #49809. This PR disallows using types from other databases in tables. This makes certain behavior (like `DROP TYPE`) more predictable in their effects, as well as unblocking some work for supporting user defined types in `cockroach dump`. Release note (sql change): Referencing types across databases has been disabled. --- pkg/sql/add_column.go | 149 +++++++++++++++++++ pkg/sql/alter_table.go | 157 ++++---------------- pkg/sql/check.go | 2 +- pkg/sql/create_table.go | 6 +- pkg/sql/logictest/testdata/logic_test/enums | 84 ++++++++--- pkg/sql/planner.go | 6 + pkg/sql/resolver.go | 21 ++- pkg/sql/sem/tree/format.go | 5 +- pkg/sql/types/types.go | 4 +- 9 files changed, 274 insertions(+), 160 deletions(-) create mode 100644 pkg/sql/add_column.go diff --git a/pkg/sql/add_column.go b/pkg/sql/add_column.go new file mode 100644 index 000000000000..b617fcb64c9e --- /dev/null +++ b/pkg/sql/add_column.go @@ -0,0 +1,149 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/schemaexpr" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/errors" +) + +// addColumnImpl performs the logic of adding a column within an ALTER TABLE. +func (p *planner) addColumnImpl( + params runParams, + n *alterTableNode, + tn *tree.TableName, + desc *sqlbase.MutableTableDescriptor, + t *tree.AlterTableAddColumn, +) error { + d := t.ColumnDef + version := params.ExecCfg().Settings.Version.ActiveVersionOrEmpty(params.ctx) + toType, err := tree.ResolveType(params.ctx, d.Type, params.p.semaCtx.GetTypeResolver()) + if err != nil { + return err + } + if supported, err := isTypeSupportedInVersion(version, toType); err != nil { + return err + } else if !supported { + return pgerror.Newf( + pgcode.FeatureNotSupported, + "type %s is not supported until version upgrade is finalized", + toType.SQLString(), + ) + } + + newDef, seqDbDesc, seqName, seqOpts, err := params.p.processSerialInColumnDef(params.ctx, d, tn) + if err != nil { + return err + } + if seqName != nil { + if err := doCreateSequence( + params, + n.n.String(), + seqDbDesc, + n.tableDesc.GetParentSchemaID(), + seqName, + n.tableDesc.Temporary, + seqOpts, + tree.AsStringWithFQNames(n.n, params.Ann()), + ); err != nil { + return err + } + } + d = newDef + incTelemetryForNewColumn(d) + + col, idx, expr, err := sqlbase.MakeColumnDefDescs(params.ctx, d, ¶ms.p.semaCtx, params.EvalContext()) + if err != nil { + return err + } + // If the new column has a DEFAULT expression that uses a sequence, add references between + // its descriptor and this column descriptor. + if d.HasDefaultExpr() { + changedSeqDescs, err := maybeAddSequenceDependencies( + params.ctx, params.p, n.tableDesc, col, expr, nil, + ) + if err != nil { + return err + } + for _, changedSeqDesc := range changedSeqDescs { + if err := params.p.writeSchemaChange( + params.ctx, changedSeqDesc, sqlbase.InvalidMutationID, tree.AsStringWithFQNames(n.n, params.Ann()), + ); err != nil { + return err + } + } + } + + // We're checking to see if a user is trying add a non-nullable column without a default to a + // non empty table by scanning the primary index span with a limit of 1 to see if any key exists. + if !col.Nullable && (col.DefaultExpr == nil && !col.IsComputed()) { + span := n.tableDesc.PrimaryIndexSpan(params.ExecCfg().Codec) + kvs, err := params.p.txn.Scan(params.ctx, span.Key, span.EndKey, 1) + if err != nil { + return err + } + if len(kvs) > 0 { + return sqlbase.NewNonNullViolationError(col.Name) + } + } + _, err = n.tableDesc.FindActiveColumnByName(string(d.Name)) + if m := n.tableDesc.FindColumnMutationByName(d.Name); m != nil { + switch m.Direction { + case sqlbase.DescriptorMutation_ADD: + return pgerror.Newf(pgcode.DuplicateColumn, + "duplicate: column %q in the middle of being added, not yet public", + col.Name) + case sqlbase.DescriptorMutation_DROP: + return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, + "column %q being dropped, try again later", col.Name) + default: + if err != nil { + return errors.AssertionFailedf( + "mutation in state %s, direction %s, and no column descriptor", + errors.Safe(m.State), errors.Safe(m.Direction)) + } + } + } + if err == nil { + if t.IfNotExists { + return nil + } + return sqlbase.NewColumnAlreadyExistsError(string(d.Name), n.tableDesc.Name) + } + + n.tableDesc.AddColumnMutation(col, sqlbase.DescriptorMutation_ADD) + if idx != nil { + if err := n.tableDesc.AddIndexMutation(idx, sqlbase.DescriptorMutation_ADD); err != nil { + return err + } + } + if d.HasColumnFamily() { + err := n.tableDesc.AddColumnToFamilyMaybeCreate( + col.Name, string(d.Family.Name), d.Family.Create, + d.Family.IfNotExists) + if err != nil { + return err + } + } + + if d.IsComputed() { + computedColValidator := schemaexpr.NewComputedColumnValidator(params.ctx, n.tableDesc, ¶ms.p.semaCtx) + if err := computedColValidator.Validate(d); err != nil { + return err + } + } + + return nil +} diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 08abe30c4a56..7626de6892ae 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -144,124 +144,13 @@ func (n *alterTableNode) startExec(params runParams) error { switch t := cmd.(type) { case *tree.AlterTableAddColumn: - d := t.ColumnDef - version := params.ExecCfg().Settings.Version.ActiveVersionOrEmpty(params.ctx) - toType, err := tree.ResolveType(params.ctx, d.Type, params.p.semaCtx.GetTypeResolver()) - if err != nil { - return err - } - if supported, err := isTypeSupportedInVersion(version, toType); err != nil { - return err - } else if !supported { - return pgerror.Newf( - pgcode.FeatureNotSupported, - "type %s is not supported until version upgrade is finalized", - toType.SQLString(), - ) - } - - newDef, seqDbDesc, seqName, seqOpts, err := params.p.processSerialInColumnDef(params.ctx, d, tn) - if err != nil { - return err - } - if seqName != nil { - if err := doCreateSequence( - params, - n.n.String(), - seqDbDesc, - n.tableDesc.GetParentSchemaID(), - seqName, - n.tableDesc.Temporary, - seqOpts, - tree.AsStringWithFQNames(n.n, params.Ann()), - ); err != nil { - return err - } - } - d = newDef - incTelemetryForNewColumn(d) - - col, idx, expr, err := sqlbase.MakeColumnDefDescs(params.ctx, d, ¶ms.p.semaCtx, params.EvalContext()) + var err error + params.p.runWithOptions(resolveFlags{contextDatabaseID: n.tableDesc.ParentID}, func() { + err = params.p.addColumnImpl(params, n, tn, n.tableDesc, t) + }) if err != nil { return err } - // If the new column has a DEFAULT expression that uses a sequence, add references between - // its descriptor and this column descriptor. - if d.HasDefaultExpr() { - changedSeqDescs, err := maybeAddSequenceDependencies( - params.ctx, params.p, n.tableDesc, col, expr, nil, - ) - if err != nil { - return err - } - for _, changedSeqDesc := range changedSeqDescs { - if err := params.p.writeSchemaChange( - params.ctx, changedSeqDesc, sqlbase.InvalidMutationID, tree.AsStringWithFQNames(n.n, params.Ann()), - ); err != nil { - return err - } - } - } - - // We're checking to see if a user is trying add a non-nullable column without a default to a - // non empty table by scanning the primary index span with a limit of 1 to see if any key exists. - if !col.Nullable && (col.DefaultExpr == nil && !col.IsComputed()) { - span := n.tableDesc.PrimaryIndexSpan(params.ExecCfg().Codec) - kvs, err := params.p.txn.Scan(params.ctx, span.Key, span.EndKey, 1) - if err != nil { - return err - } - if len(kvs) > 0 { - return sqlbase.NewNonNullViolationError(col.Name) - } - } - _, err = n.tableDesc.FindActiveColumnByName(string(d.Name)) - if m := n.tableDesc.FindColumnMutationByName(d.Name); m != nil { - switch m.Direction { - case sqlbase.DescriptorMutation_ADD: - return pgerror.Newf(pgcode.DuplicateColumn, - "duplicate: column %q in the middle of being added, not yet public", - col.Name) - case sqlbase.DescriptorMutation_DROP: - return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, - "column %q being dropped, try again later", col.Name) - default: - if err != nil { - return errors.AssertionFailedf( - "mutation in state %s, direction %s, and no column descriptor", - errors.Safe(m.State), errors.Safe(m.Direction)) - } - } - } - if err == nil { - if t.IfNotExists { - continue - } - return sqlbase.NewColumnAlreadyExistsError(string(d.Name), n.tableDesc.Name) - } - - n.tableDesc.AddColumnMutation(col, sqlbase.DescriptorMutation_ADD) - if idx != nil { - if err := n.tableDesc.AddIndexMutation(idx, sqlbase.DescriptorMutation_ADD); err != nil { - return err - } - } - if d.HasColumnFamily() { - err := n.tableDesc.AddColumnToFamilyMaybeCreate( - col.Name, string(d.Family.Name), d.Family.Create, - d.Family.IfNotExists) - if err != nil { - return err - } - } - - if d.IsComputed() { - computedColValidator := schemaexpr.NewComputedColumnValidator(params.ctx, n.tableDesc, ¶ms.p.semaCtx) - if err := computedColValidator.Validate(d); err != nil { - return err - } - } - case *tree.AlterTableAddConstraint: switch d := t.ConstraintDef.(type) { case *tree.UniqueConstraintTableDef: @@ -315,24 +204,32 @@ func (n *alterTableNode) startExec(params runParams) error { } case *tree.CheckConstraintTableDef: - info, err := n.tableDesc.GetConstraintInfo(params.ctx, nil, params.ExecCfg().Codec) - if err != nil { - return err - } - ckBuilder := schemaexpr.NewCheckConstraintBuilder(params.ctx, *tn, n.tableDesc, ¶ms.p.semaCtx) - for k := range info { - ckBuilder.MarkNameInUse(k) - } - ck, err := ckBuilder.Build(d) + var err error + params.p.runWithOptions(resolveFlags{contextDatabaseID: n.tableDesc.ParentID}, func() { + info, infoErr := n.tableDesc.GetConstraintInfo(params.ctx, nil, params.ExecCfg().Codec) + if err != nil { + err = infoErr + return + } + ckBuilder := schemaexpr.NewCheckConstraintBuilder(params.ctx, *tn, n.tableDesc, ¶ms.p.semaCtx) + for k := range info { + ckBuilder.MarkNameInUse(k) + } + ck, buildErr := ckBuilder.Build(d) + if buildErr != nil { + err = buildErr + return + } + if t.ValidationBehavior == tree.ValidationDefault { + ck.Validity = sqlbase.ConstraintValidity_Validating + } else { + ck.Validity = sqlbase.ConstraintValidity_Unvalidated + } + n.tableDesc.AddCheckMutation(ck, sqlbase.DescriptorMutation_ADD) + }) if err != nil { return err } - if t.ValidationBehavior == tree.ValidationDefault { - ck.Validity = sqlbase.ConstraintValidity_Validating - } else { - ck.Validity = sqlbase.ConstraintValidity_Unvalidated - } - n.tableDesc.AddCheckMutation(ck, sqlbase.DescriptorMutation_ADD) case *tree.ForeignKeyConstraintTableDef: for _, colName := range d.FromCols { diff --git a/pkg/sql/check.go b/pkg/sql/check.go index 1860cca6d19a..5de816ccdebd 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -54,7 +54,7 @@ func validateCheckExpr( } lim := &tree.Limit{Count: tree.NewDInt(1)} stmt := &tree.Select{Select: sel, Limit: lim} - queryStr := tree.AsStringWithFlags(stmt, tree.FmtParsable) + queryStr := tree.AsStringWithFlags(stmt, tree.FmtSerializable) log.Infof(ctx, "Validating check constraint %q with query %q", tree.SerializeForDisplay(expr), queryStr) rows, err := ie.QueryRow(ctx, "validate check constraint", txn, queryStr) diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 6e841ec7941d..18f5fa3a03b7 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1097,7 +1097,7 @@ func getFinalSourceQuery(source *tree.Select, evalCtx *tree.EvalContext) string // We use tree.FormatNode merely as a traversal method; its output buffer is // discarded immediately after the traversal because it is not needed // further. - f := tree.NewFmtCtx(tree.FmtParsable) + f := tree.NewFmtCtx(tree.FmtSerializable) f.SetReformatTableNames( func(_ *tree.FmtCtx, tn *tree.TableName) { // Persist the database prefix expansion. @@ -1113,7 +1113,7 @@ func getFinalSourceQuery(source *tree.Select, evalCtx *tree.EvalContext) string f.Close() // Substitute placeholders with their values. - ctx := tree.NewFmtCtx(tree.FmtParsable) + ctx := tree.NewFmtCtx(tree.FmtSerializable) ctx.SetPlaceholderFormat(func(ctx *tree.FmtCtx, placeholder *tree.Placeholder) { d, err := placeholder.Eval(evalCtx) if err != nil { @@ -1833,7 +1833,7 @@ func makeTableDesc( // it needs to pull in descriptors from FK depended-on tables // and interleaved parents using their current state in KV. // See the comment at the start of MakeTableDesc() and resolveFK(). - params.p.runWithOptions(resolveFlags{skipCache: true}, func() { + params.p.runWithOptions(resolveFlags{skipCache: true, contextDatabaseID: parentID}, func() { ret, err = MakeTableDesc( params.ctx, params.p.txn, diff --git a/pkg/sql/logictest/testdata/logic_test/enums b/pkg/sql/logictest/testdata/logic_test/enums index 61cc593ef01a..b5bc56c67c86 100644 --- a/pkg/sql/logictest/testdata/logic_test/enums +++ b/pkg/sql/logictest/testdata/logic_test/enums @@ -395,7 +395,7 @@ query TT SHOW CREATE t1 ---- t1 CREATE TABLE t1 ( - x test.public.greeting NULL, + x public.greeting NULL, INDEX i (x ASC), FAMILY "primary" (x, rowid) ) @@ -406,7 +406,7 @@ query T SELECT create_statement FROM crdb_internal.create_statements WHERE descriptor_name = 't1' ---- CREATE TABLE t1 ( - x test.public.greeting NULL, + x public.greeting NULL, INDEX i (x ASC), FAMILY "primary" (x, rowid) ) @@ -418,7 +418,7 @@ SELECT ARRAY['hello']::_greeting, ARRAY['hello'::greeting] {hello} {hello} # Test that we can't mix enums in an array. -query error pq: expected 'cockroach'::test.public.dbs to be of type greeting, found type dbs +query error pq: expected 'cockroach'::public.dbs to be of type greeting, found type dbs SELECT ARRAY['hello'::greeting, 'cockroach'::dbs] statement ok @@ -434,7 +434,7 @@ SELECT * FROM enum_array query TTT SELECT pg_typeof(x), pg_typeof(x[1]), pg_typeof(ARRAY['hello']::_greeting) FROM enum_array LIMIT 1 ---- -test.public.greeting[] test.public.greeting test.public.greeting[] +public.greeting[] public.greeting public.greeting[] # Ensure that the implicitly created array type will tolerate collisions. # _collision will create __collision as its implicit array type, so the @@ -492,8 +492,8 @@ SHOW CREATE enum_default ---- enum_default CREATE TABLE enum_default ( x INT8 NULL, - y test.public.greeting NULL DEFAULT 'hello':::test.public.greeting, - z BOOL NULL DEFAULT 'hello':::test.public.greeting IS OF (test.public.greeting, test.public.greeting), + y public.greeting NULL DEFAULT 'hello':::public.greeting, + z BOOL NULL DEFAULT 'hello':::public.greeting IS OF (public.greeting, public.greeting), FAMILY fam_0_x_y_z_rowid (x, y, z, rowid) ) @@ -508,8 +508,8 @@ WHERE ORDER BY column_name ---- -y 'hello':::test.public.greeting -z 'hello':::test.public.greeting IS OF (test.public.greeting, test.public.greeting) +y 'hello':::public.greeting +z 'hello':::public.greeting IS OF (public.greeting, public.greeting) # Test information_schema.columns. query TT @@ -522,8 +522,8 @@ WHERE ORDER BY column_name ---- -y 'hello':::test.public.greeting -z 'hello':::test.public.greeting IS OF (test.public.greeting, test.public.greeting) +y 'hello':::public.greeting +z 'hello':::public.greeting IS OF (public.greeting, public.greeting) # Test computed columns with enum values. statement ok @@ -547,9 +547,9 @@ SHOW CREATE enum_computed ---- enum_computed CREATE TABLE enum_computed ( x INT8 NULL, - y test.public.greeting NULL AS ('hello':::test.public.greeting) STORED, - z BOOL NULL AS (w = 'howdy':::test.public.greeting) STORED, - w test.public.greeting NULL, + y public.greeting NULL AS ('hello':::public.greeting) STORED, + z BOOL NULL AS (w = 'howdy':::public.greeting) STORED, + w public.greeting NULL, FAMILY fam_0_x_y_z_w_rowid (x, y, z, w, rowid) ) @@ -564,8 +564,8 @@ WHERE ORDER BY column_name ---- -y 'hello':::test.public.greeting -z w = 'howdy':::test.public.greeting +y 'hello':::public.greeting +z w = 'howdy':::public.greeting # Test check constraints with enum values. statement ok @@ -580,10 +580,10 @@ query TT SHOW CREATE enum_checks ---- enum_checks CREATE TABLE enum_checks ( - x test.public.greeting NULL, + x public.greeting NULL, FAMILY "primary" (x, rowid), - CONSTRAINT check_x CHECK (x = 'hello':::test.public.greeting::test.public.greeting), - CONSTRAINT "check" CHECK ('hello':::test.public.greeting = 'hello':::test.public.greeting) + CONSTRAINT check_x CHECK (x = 'hello':::public.greeting::public.greeting), + CONSTRAINT "check" CHECK ('hello':::public.greeting = 'hello':::public.greeting) ) # Ensure that we can add check constraints to tables with enums. @@ -594,11 +594,11 @@ INSERT INTO enum_checks VALUES ('hi'), ('howdy'); ALTER TABLE enum_checks ADD CHECK (x > 'hello') # Ensure that checks are validated on insert. -statement error pq: failed to satisfy CHECK constraint \(x > 'hello':::test.public.greeting\) +statement error pq: failed to satisfy CHECK constraint \(x > 'hello':::public.greeting\) INSERT INTO enum_checks VALUES ('hello') # Try adding a check that fails validation. -statement error pq: validation of CHECK "x = 'hello':::test.public.greeting" failed +statement error pq: validation of CHECK "x = 'hello':::public.greeting" failed ALTER TABLE enum_checks ADD CHECK (x = 'hello') # Check the above cases, but in a transaction. @@ -609,7 +609,7 @@ CREATE TABLE enum_checks (x greeting); INSERT INTO enum_checks VALUES ('hi'), ('howdy'); ALTER TABLE enum_checks ADD CHECK (x > 'hello') -statement error pq: failed to satisfy CHECK constraint \(x > 'hello':::test.public.greeting\) +statement error pq: failed to satisfy CHECK constraint \(x > 'hello':::public.greeting\) INSERT INTO enum_checks VALUES ('hello') statement ok @@ -621,12 +621,52 @@ CREATE TABLE enum_checks (x greeting); INSERT INTO enum_checks VALUES ('hi'), ('howdy'); # Try adding a check that fails validation. -statement error pq: validation of CHECK "x = 'hello':::test.public.greeting" failed +statement error pq: validation of CHECK "x = 'hello':::public.greeting" failed ALTER TABLE enum_checks ADD CHECK (x = 'hello') statement ok ROLLBACK +# Test that cross database type references are disallowed. +statement ok +CREATE DATABASE other; +CREATE TYPE other.t AS ENUM ('other') + +# We can still reference other databases types when creating objects +# within those databases. +statement ok +CREATE TABLE other.tt (x other.t) + +# Referencing other databases in this database's objects will error. +statement error pq: cross database type references are not supported: other.public.t +CREATE TABLE cross_error (x other.t) + +# Test that we can't hide cross database references in expressions. +statement error pq: cross database type references are not supported: other.public.t +CREATE TABLE cross_error (x BOOL DEFAULT ('other':::other.t = 'other':::other.t)) + +statement error pq: cross database type references are not supported: other.public.t +CREATE TABLE cross_error (x BOOL AS ('other':::other.t = 'other':::other.t) STORED) + +statement error pq: cross database type references are not supported: other.public.t +CREATE TABLE cross_error (x INT, CHECK ('other':::other.t = 'other':::other.t)) + +# Test that we can't add columns or checks that use these either. +statement ok +CREATE TABLE cross_error (x INT) + +statement error pq: cross database type references are not supported: other.public.t +ALTER TABLE cross_error ADD COLUMN y other.t + +statement error pq: cross database type references are not supported: other.public.t +ALTER TABLE cross_error ADD COLUMN y BOOL DEFAULT ('other':::other.t = 'other':::other.t) + +statement error pq: cross database type references are not supported: other.public.t +ALTER TABLE cross_error ADD COLUMN y BOOL AS ('other':::other.t = 'other':::other.t) STORED + +statement error pq: cross database type references are not supported: other.public.t +ALTER TABLE cross_error ADD CHECK ('other':::other.t = 'other':::other.t) + subtest schema_changes # Ensure that we can drop and create indexes on user defined type columns, diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index e6cf4d09bb3d..43827b0fa8ba 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -197,6 +197,12 @@ type planner struct { noticeSender noticeSender queryCacheSession querycache.Session + + // contextDatabaseID is the ID of a database. It is set during some name + // resolution processes to disallow cross database references. In particular, + // the type resolution steps will disallow resolution of types that have a + // parentID != contextDatabaseID when it is set. + contextDatabaseID sqlbase.ID } func (ctx *extendedEvalContext) setSessionID(sessionID ClusterWideID) { diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index e9da41b42e1e..4a1c92c5aed1 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -59,11 +59,16 @@ func (p *planner) runWithOptions(flags resolveFlags, fn func()) { defer func(prev bool) { p.avoidCachedDescriptors = prev }(p.avoidCachedDescriptors) p.avoidCachedDescriptors = true } + if flags.contextDatabaseID != sqlbase.InvalidID { + defer func(prev sqlbase.ID) { p.contextDatabaseID = prev }(p.contextDatabaseID) + p.contextDatabaseID = flags.contextDatabaseID + } fn() } type resolveFlags struct { - skipCache bool + skipCache bool + contextDatabaseID sqlbase.ID } func (p *planner) ResolveMutableTableDescriptor( @@ -156,6 +161,20 @@ func (p *planner) ResolveType( } tn := tree.MakeTypeNameFromPrefix(prefix, tree.Name(name.Object())) tdesc := desc.(*sqlbase.ImmutableTypeDescriptor) + + // Disllow cross-database type resolution. Note that we check + // p.contextDatabaseID != sqlbase.InvalidID when we have been restricted to + // accessing types in the database with ID = p.contextDatabaseID by + // p.runWithOptions. So, check to see if the resolved descriptor's parentID + // matches, unless the descriptor's parentID is invalid. This could happen + // when the type being resolved is a builtin type prefaced with a virtual + // schema like `pg_catalog.int`. Resolution for these types returns a dummy + // TypeDescriptor, so ignore those cases. + if p.contextDatabaseID != sqlbase.InvalidID && tdesc.ParentID != sqlbase.InvalidID && tdesc.ParentID != p.contextDatabaseID { + return nil, pgerror.Newf( + pgcode.FeatureNotSupported, "cross database type references are not supported: %s", tn.String()) + } + return tdesc.MakeTypesT(&tn, p.makeTypeLookupFn(ctx)) } diff --git a/pkg/sql/sem/tree/format.go b/pkg/sql/sem/tree/format.go index 7913d0a1ad29..094cbe10587d 100644 --- a/pkg/sql/sem/tree/format.go +++ b/pkg/sql/sem/tree/format.go @@ -148,7 +148,10 @@ const ( FmtPgwireText FmtFlags = fmtPgwireFormat | FmtFlags(lex.EncBareStrings) // FmtParsable instructs the pretty-printer to produce a representation that - // can be parsed into an equivalent expression. + // can be parsed into an equivalent expression. If there is a chance that the + // formatted data will be stored durably on disk or sent to other nodes, + // then this formatting directive is not appropriate, and FmtSerializable + // should be used instead. FmtParsable FmtFlags = fmtDisambiguateDatumTypes | FmtParsableNumerics // FmtSerializable instructs the pretty-printer to produce a representation diff --git a/pkg/sql/types/types.go b/pkg/sql/types/types.go index 5ac49da7a593..5b8ba6d5d29a 100644 --- a/pkg/sql/types/types.go +++ b/pkg/sql/types/types.go @@ -241,8 +241,8 @@ func (u UserDefinedTypeName) Basename() string { // FQName returns the fully qualified name. func (u UserDefinedTypeName) FQName() string { var sb strings.Builder - sb.WriteString(u.Catalog) - sb.WriteString(".") + // Note that cross-database type references are disabled, so we only + // format the qualified name with the schema. sb.WriteString(u.Schema) sb.WriteString(".") sb.WriteString(u.Name) From e14af52d3d58967843c43bdd117ee065f0299056 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Mon, 8 Jun 2020 08:01:34 -0700 Subject: [PATCH 14/46] Makefile: make PROJ compile on Publish Bleeding Edge As docgen relies on builtins which relies on proj, we are stuck in a situation in cross compilation where we are trying to install docgen with a non-native libproj.a file causing the linker to fail in `Upload Binaries`. Fix this by not compiling docgen on cross compilations. Also fix the windows compilation as PROJ will dump it into a libproj_4_9.a instead of libproj.a. Cannot see an option to fix this on the CMakeLists.txt. Also make execgen not depend on LIBPROJ. Release note: None --- Makefile | 9 +++++---- pkg/geo/geoproj/geoproj.go | 4 ++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/Makefile b/Makefile index e992478db232..729bc98d371e 100644 --- a/Makefile +++ b/Makefile @@ -482,7 +482,7 @@ LIBSNAPPY := $(SNAPPY_DIR)/libsnappy.a LIBEDIT := $(LIBEDIT_DIR)/src/.libs/libedit.a LIBROACH := $(LIBROACH_DIR)/libroach.a LIBROACHCCL := $(LIBROACH_DIR)/libroachccl.a -LIBPROJ := $(PROJ_DIR)/lib/libproj.a +LIBPROJ := $(PROJ_DIR)/lib/libproj$(if $(target-is-windows),_4_9).a LIBKRB5 := $(KRB5_DIR)/lib/libgssapi_krb5.a PROTOC := $(PROTOC_DIR)/protoc @@ -553,7 +553,7 @@ $(BASE_CGO_FLAGS_FILES): Makefile build/defs.mk.sig | bin/.submodules-initialize @echo >> $@ @echo 'package $(if $($(@D)-package),$($(@D)-package),$(notdir $(@D)))' >> $@ @echo >> $@ - @echo '// #cgo CPPFLAGS: $(addprefix -I,$(JEMALLOC_DIR)/include $(KRB_CPPFLAGS) $(GEOS_DIR)/capi $(PROJ_DIR)/lib)' >> $@ + @echo '// #cgo CPPFLAGS: $(addprefix -I,$(JEMALLOC_DIR)/include $(KRB_CPPFLAGS))' >> $@ @echo '// #cgo LDFLAGS: $(addprefix -L,$(CRYPTOPP_DIR) $(PROTOBUF_DIR) $(JEMALLOC_DIR)/lib $(SNAPPY_DIR) $(LIBEDIT_DIR)/src/.libs $(ROCKSDB_DIR) $(LIBROACH_DIR) $(KRB_DIR) $(PROJ_DIR)/lib)' >> $@ @echo 'import "C"' >> $@ @@ -972,7 +972,7 @@ buildshort: ## Build the CockroachDB binary without the admin UI. build: $(COCKROACH) buildoss: $(COCKROACHOSS) buildshort: $(COCKROACHSHORT) -build buildoss buildshort: $(DOCGEN_TARGETS) +build buildoss buildshort: $(if $(is-cross-compile),,$(DOCGEN_TARGETS)) build buildshort: $(if $(is-cross-compile),,$(SETTINGS_DOC_PAGE)) # For historical reasons, symlink cockroach to cockroachshort. @@ -1715,7 +1715,8 @@ logictest-bins := bin/logictest bin/logictestopt bin/logictestccl # Additional dependencies for binaries that depend on generated code. # # TODO(benesch): Derive this automatically. This is getting out of hand. -bin/workload bin/docgen bin/execgen bin/roachtest $(logictest-bins): $(LIBPROJ) $(CGO_FLAGS_FILES) $(SQLPARSER_TARGETS) $(PROTOBUF_TARGETS) +bin/workload bin/docgen bin/execgen bin/roachtest $(logictest-bins): $(SQLPARSER_TARGETS) $(PROTOBUF_TARGETS) +bin/workload bin/docgen bin/roachtest $(logictest-bins): $(LIBPROJ) $(CGO_FLAGS_FILES) bin/workload bin/roachtest $(logictest-bins): $(EXECGEN_TARGETS) bin/roachtest $(logictest-bins): $(C_LIBS_CCL) $(CGO_FLAGS_FILES) $(OPTGEN_TARGETS) diff --git a/pkg/geo/geoproj/geoproj.go b/pkg/geo/geoproj/geoproj.go index 9d2bdd7345bf..044dbf6255e4 100644 --- a/pkg/geo/geoproj/geoproj.go +++ b/pkg/geo/geoproj/geoproj.go @@ -13,9 +13,9 @@ package geoproj // #cgo CXXFLAGS: -std=c++14 // #cgo CPPFLAGS: -I../../../c-deps/proj/src -// #cgo LDFLAGS: -lproj +// #cgo !windows LDFLAGS: -lproj // #cgo linux LDFLAGS: -lrt -lm -lpthread -// #cgo windows LDFLAGS: -lshlwapi -lrpcrt4 +// #cgo windows LDFLAGS: -lproj_4_9 -lshlwapi -lrpcrt4 // // #include "proj.h" import "C" From d66ee36ffb5361ebf1bd4a437c0a7e28d76b16d7 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 2 Jun 2020 15:10:43 -0700 Subject: [PATCH 15/46] sql: implement ConstructScan in distsql spec factory This commit adds an implementation of `distSQLSpecExecFactory.ConstructScan` which combines the logic that performs `scanNode` creation of `execFactory.ConstructScan` and physical planning of table readers of `DistSQLPlanner.createTableReaders`. I tried to refactor the code so that there is not much duplication going on. Notably, simple projections, renders, and filters are not yet implemented. Release note: None --- pkg/sql/distsql_physical_planner.go | 14 +- pkg/sql/distsql_spec_exec_factory.go | 141 +++++++++++++++++- .../logic_test/experimental_distsql_planning | 38 ++++- pkg/sql/physicalplan/physical_plan.go | 6 +- pkg/sql/plan.go | 4 +- pkg/sql/plan_opt.go | 3 + 6 files changed, 191 insertions(+), 15 deletions(-) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 3fb4d1135c91..d5c6c90f8822 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -2116,6 +2116,14 @@ func (dsp *DistSQLPlanner) createPlanForZigzagJoin( return plan, nil } +func getTypesFromResultColumns(cols sqlbase.ResultColumns) []*types.T { + typs := make([]*types.T, len(cols)) + for i, col := range cols { + typs[i] = col.Typ + } + return typs +} + // getTypesForPlanResult returns the types of the elements in the result streams // of a plan that corresponds to a given planNode. If planToStreamColMap is nil, // a 1-1 mapping is assumed. @@ -2123,11 +2131,7 @@ func getTypesForPlanResult(node planNode, planToStreamColMap []int) ([]*types.T, nodeColumns := planColumns(node) if planToStreamColMap == nil { // No remapping. - types := make([]*types.T, len(nodeColumns)) - for i := range nodeColumns { - types[i] = nodeColumns[i].Typ - } - return types, nil + return getTypesFromResultColumns(nodeColumns), nil } numCols := 0 for _, streamCol := range planToStreamColMap { diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index 2c941f26d904..25e379e9db80 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -12,23 +12,28 @@ package sql import ( "github.com/cockroachdb/cockroach/pkg/geo/geoindex" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/span" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" ) type distSQLSpecExecFactory struct { planner *planner + dsp *DistSQLPlanner } var _ exec.Factory = &distSQLSpecExecFactory{} func newDistSQLSpecExecFactory(p *planner) exec.Factory { - return &distSQLSpecExecFactory{planner: p} + return &distSQLSpecExecFactory{planner: p, dsp: p.extendedEvalCtx.DistSQLPlanner} } func (e *distSQLSpecExecFactory) ConstructValues( @@ -37,6 +42,9 @@ func (e *distSQLSpecExecFactory) ConstructValues( return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning") } +// ConstructScan implements exec.Factory interface by combining the logic that +// performs scanNode creation of execFactory.ConstructScan and physical +// planning of table readers of DistSQLPlanner.createTableReaders. func (e *distSQLSpecExecFactory) ConstructScan( table cat.Table, index cat.Index, @@ -50,12 +58,135 @@ func (e *distSQLSpecExecFactory) ConstructScan( rowCount float64, locking *tree.LockingItem, ) (exec.Node, error) { - return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning") + if table.IsVirtualTable() { + return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning") + } + + var p PhysicalPlan + // Although we don't yet recommend distributing plans where soft limits + // propagate to scan nodes because we don't have infrastructure to only + // plan for a few ranges at a time, the propagation of the soft limits + // to scan nodes has been added in 20.1 release, so to keep the + // previous behavior we continue to ignore the soft limits for now. + // TODO(yuzefovich): pay attention to the soft limits. + recommendation := canDistribute + + // Phase 1: set up all necessary infrastructure for table reader planning + // below. This phase is equivalent to what execFactory.ConstructScan does. + tabDesc := table.(*optTable).desc + indexDesc := index.(*optIndex).desc + colCfg := makeScanColumnsConfig(table, needed) + sb := span.MakeBuilder(e.planner.ExecCfg().Codec, tabDesc.TableDesc(), indexDesc) + + // Note that initColsForScan and setting ResultColumns below are equivalent + // to what scan.initTable call does in execFactory.ConstructScan. + cols, err := initColsForScan(tabDesc, colCfg) + if err != nil { + return nil, err + } + p.ResultColumns = sqlbase.ResultColumnsFromColDescs(tabDesc.GetID(), cols) + + if indexConstraint != nil && indexConstraint.IsContradiction() { + // TODO(yuzefovich): once ConstructValues is implemented, consider + // calling it here. + physPlan, err := e.dsp.createValuesPlan( + getTypesFromResultColumns(p.ResultColumns), 0 /* numRows */, nil, /* rawBytes */ + ) + return planMaybePhysical{physPlan: physPlan, recommendation: canDistribute}, err + } + + // TODO(yuzefovich): scanNode adds "parallel" attribute in walk.go when + // scanNode.canParallelize() returns true. We should plumb that info from + // here somehow as well. + var spans roachpb.Spans + spans, err = sb.SpansFromConstraint(indexConstraint, needed, false /* forDelete */) + if err != nil { + return nil, err + } + isFullTableScan := len(spans) == 1 && spans[0].EqualValue( + tabDesc.IndexSpan(e.planner.ExecCfg().Codec, indexDesc.ID), + ) + if err = colCfg.assertValidReqOrdering(reqOrdering); err != nil { + return nil, err + } + + // Check if we are doing a full scan. + if isFullTableScan { + recommendation = recommendation.compose(shouldDistribute) + } + + // Phase 2: perform the table reader planning. This phase is equivalent to + // what DistSQLPlanner.createTableReaders does. + colsToTableOrdinalMap := toTableOrdinals(cols, tabDesc, colCfg.visibility) + trSpec := physicalplan.NewTableReaderSpec() + *trSpec = execinfrapb.TableReaderSpec{ + Table: *tabDesc.TableDesc(), + Reverse: reverse, + IsCheck: false, + Visibility: colCfg.visibility, + // Retain the capacity of the spans slice. + Spans: trSpec.Spans[:0], + } + trSpec.IndexIdx, err = getIndexIdx(indexDesc, tabDesc) + if err != nil { + return nil, err + } + if locking != nil { + trSpec.LockingStrength = sqlbase.ToScanLockingStrength(locking.Strength) + trSpec.LockingWaitPolicy = sqlbase.ToScanLockingWaitPolicy(locking.WaitPolicy) + if trSpec.LockingStrength != sqlbase.ScanLockingStrength_FOR_NONE { + // Scans that are performing row-level locking cannot currently be + // distributed because their locks would not be propagated back to + // the root transaction coordinator. + // TODO(nvanbenschoten): lift this restriction. + recommendation = cannotDistribute + } + } + + // Note that we don't do anything about the possible filter here since we + // don't know yet whether we will have it. ConstructFilter is responsible + // for pushing the filter down into the post-processing stage of this scan. + post := execinfrapb.PostProcessSpec{} + if hardLimit != 0 { + post.Limit = uint64(hardLimit) + } else if softLimit != 0 { + trSpec.LimitHint = softLimit + } + + distribute := shouldDistributeGivenRecAndMode(recommendation, e.planner.extendedEvalCtx.SessionData.DistSQLMode) + if _, singleTenant := e.planner.execCfg.NodeID.OptionalNodeID(); !singleTenant { + distribute = false + } + + evalCtx := e.planner.ExtendedEvalContext() + planCtx := e.dsp.NewPlanningCtx(evalCtx.Context, evalCtx, e.planner.txn, distribute) + err = e.dsp.planTableReaders( + planCtx, + &p, + &tableReaderPlanningInfo{ + spec: trSpec, + post: post, + desc: tabDesc, + spans: spans, + reverse: reverse, + scanVisibility: colCfg.visibility, + maxResults: maxResults, + estimatedRowCount: uint64(rowCount), + reqOrdering: ReqOrdering(reqOrdering), + cols: cols, + colsToTableOrdrinalMap: colsToTableOrdinalMap, + }, + ) + + return planMaybePhysical{physPlan: &p, recommendation: recommendation}, err } func (e *distSQLSpecExecFactory) ConstructFilter( n exec.Node, filter tree.TypedExpr, reqOrdering exec.OutputOrdering, ) (exec.Node, error) { + // TODO(yuzefovich): figure out how to push the filter into the table + // reader when it already doesn't have a filter and it doesn't have a hard + // limit. return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning") } @@ -230,7 +361,11 @@ func (e *distSQLSpecExecFactory) ConstructWindow( func (e *distSQLSpecExecFactory) RenameColumns( input exec.Node, colNames []string, ) (exec.Node, error) { - return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning") + inputCols := input.(planMaybePhysical).physPlan.ResultColumns + for i := range inputCols { + inputCols[i].Name = colNames[i] + } + return input, nil } func (e *distSQLSpecExecFactory) ConstructPlan( diff --git a/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning b/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning index c91d1626643b..10fcc5ba215e 100644 --- a/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning +++ b/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning @@ -14,9 +14,41 @@ SET CLUSTER SETTING sql.defaults.experimental_distsql_planning = on statement ok SET experimental_distsql_planning = always -# Test that a SELECT query fails but others don't. statement ok -CREATE TABLE kv (k INT PRIMARY KEY, v INT); INSERT INTO kv VALUES (1, 1), (2, 1) +CREATE TABLE kv (k INT PRIMARY KEY, v INT); INSERT INTO kv VALUES (1, 1), (2, 1), (3, 2) -statement error pq: unimplemented: experimental opt-driven distsql planning +query II colnames,rowsort SELECT * FROM kv +---- +k v +1 1 +2 1 +3 2 + +query I colnames,rowsort +SELECT k FROM kv +---- +k +1 +2 +3 + +query I colnames,rowsort +SELECT v FROM kv +---- +v +1 +1 +2 + +# Projections are not yet supported. +statement error pq: unimplemented: experimental opt-driven distsql planning +SELECT v, k FROM kv + +# Renders are not yet supported. +statement error pq: unimplemented: experimental opt-driven distsql planning +SELECT k + v FROM kv + +# Filters are not yet supported. +statement error pq: unimplemented: experimental opt-driven distsql planning +SELECT * FROM kv WHERE k > v diff --git a/pkg/sql/physicalplan/physical_plan.go b/pkg/sql/physicalplan/physical_plan.go index 553b45d2cd03..fae6264e3d67 100644 --- a/pkg/sql/physicalplan/physical_plan.go +++ b/pkg/sql/physicalplan/physical_plan.go @@ -77,7 +77,7 @@ type PhysicalPlan struct { LocalProcessors []execinfra.LocalProcessor // LocalProcessorIndexes contains pointers to all of the RowSourceIdx fields - // of the LocalPlanNodeSpecs that were created. This list is in the same + // of the LocalPlanNodeSpecs that were created. This list is in the same // order as LocalProcessors, and is kept up-to-date so that LocalPlanNodeSpecs // always have the correct index into the LocalProcessors slice. LocalProcessorIndexes []*uint32 @@ -105,6 +105,10 @@ type PhysicalPlan struct { // in-place during planning. ResultTypes []*types.T + // ResultColumns is the schema (result columns) of the rows produced by the + // ResultRouters. + ResultColumns sqlbase.ResultColumns + // MergeOrdering is the ordering guarantee for the result streams that must be // maintained when the streams eventually merge. The column indexes refer to // columns for the rows produced by ResultRouters. diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go index c4d63faa3e66..95dbc7092000 100644 --- a/pkg/sql/plan.go +++ b/pkg/sql/plan.go @@ -329,9 +329,7 @@ func (p planMaybePhysical) isPhysicalPlan() bool { func (p planMaybePhysical) planColumns() sqlbase.ResultColumns { if p.isPhysicalPlan() { - // TODO(yuzefovich): update this once we support creating table reader - // specs directly in the optimizer (see #47474). - return nil + return p.physPlan.ResultColumns } return planColumns(p.planNode) } diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index d03f7de94c4d..1e9704de3ecd 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -180,6 +180,9 @@ func (p *planner) makeOptimizerPlan(ctx context.Context) error { plan exec.Plan bld *execbuilder.Builder ) + // TODO(yuzefovich): we're creating a new exec.Factory for every query, but + // we probably could pool those allocations using sync.Pool. Investigate + // this. if mode := p.SessionData().ExperimentalDistSQLPlanningMode; mode != sessiondata.ExperimentalDistSQLPlanningOff { bld = execbuilder.New(newDistSQLSpecExecFactory(p), execMemo, &opc.catalog, root, p.EvalContext()) plan, err = bld.Build() From 15bb4ae9d87bfc85a6cb38c68fea6a23921b43e8 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 3 Jun 2020 14:01:23 -0700 Subject: [PATCH 16/46] sql: enforce the assumption that wantedColumns is not nil for scans After the examining the code closer, I see that now in all code paths we populate `scanColumnsConfig.wantedColumns` to ask for the specific columns to be scanned. Previously, this map could be left as `nil` which would mean that all columns of the table should be scanned. This was used only by the heuristic planner which has been removed, so we can now update the assumption of the scan columns config. This allows us to clean up the comments a bit. This commit also moves `makeColumnsConfig` function into the shared util file because it is used by both factories. Release note: None --- pkg/sql/distsql_physical_planner.go | 15 +++------------ pkg/sql/exec_factory_util.go | 24 ++++++++++++++++++++++++ pkg/sql/join_test.go | 7 ++++++- pkg/sql/opt_exec_factory.go | 21 --------------------- pkg/sql/scan.go | 29 +++++++---------------------- 5 files changed, 40 insertions(+), 56 deletions(-) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index d5c6c90f8822..a33887faf5de 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1031,21 +1031,12 @@ func (dsp *DistSQLPlanner) CheckNodeHealthAndVersion( func (dsp *DistSQLPlanner) createTableReaders( planCtx *PlanningCtx, n *scanNode, ) (*PhysicalPlan, error) { - // scanNodeToTableOrdinalMap is a map from scan node column ordinal to - // table reader column ordinal. - // - // scanNodes can have columns set up in a few different ways, depending on the - // colCfg. The heuristic planner always creates scanNodes with all public - // columns (even if some of them aren't even in the index we are scanning). - // The optimizer creates scanNodes with a specific set of wanted columns; in - // this case we have to create a map from scanNode column ordinal to table - // column ordinal (which is what the TableReader uses). - var scanNodeToTableOrdinalMap []int if n.colCfg.addUnwantedAsHidden { panic("addUnwantedAsHidden not supported") - } else if n.colCfg.wantedColumns != nil { - scanNodeToTableOrdinalMap = toTableOrdinals(n.cols, n.desc, n.colCfg.visibility) } + // scanNodeToTableOrdinalMap is a map from scan node column ordinal to + // table reader column ordinal. + scanNodeToTableOrdinalMap := toTableOrdinals(n.cols, n.desc, n.colCfg.visibility) spec, post, err := initTableReaderSpec(n, planCtx, scanNodeToTableOrdinalMap) if err != nil { return nil, err diff --git a/pkg/sql/exec_factory_util.go b/pkg/sql/exec_factory_util.go index aa617877b704..272873e051a9 100644 --- a/pkg/sql/exec_factory_util.go +++ b/pkg/sql/exec_factory_util.go @@ -13,8 +13,12 @@ package sql import ( "fmt" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/errors" ) @@ -80,3 +84,23 @@ func constructPlan( return res, nil } + +// makeScanColumnsConfig builds a scanColumnsConfig struct by constructing a +// list of descriptor IDs for columns in the given cols set. Columns are +// identified by their ordinal position in the table schema. +func makeScanColumnsConfig(table cat.Table, cols exec.TableColumnOrdinalSet) scanColumnsConfig { + // Set visibility=execinfra.ScanVisibilityPublicAndNotPublic, since all + // columns in the "cols" set should be projected, regardless of whether + // they're public or non-public. The caller decides which columns to + // include (or not include). Note that when wantedColumns is non-empty, + // the visibility flag will never trigger the addition of more columns. + colCfg := scanColumnsConfig{ + wantedColumns: make([]tree.ColumnID, 0, cols.Len()), + visibility: execinfra.ScanVisibilityPublicAndNotPublic, + } + for c, ok := cols.Next(0); ok; c, ok = cols.Next(c + 1) { + desc := table.Column(c).(*sqlbase.ColumnDescriptor) + colCfg.wantedColumns = append(colCfg.wantedColumns, tree.ColumnID(desc.ID)) + } + return colCfg +} diff --git a/pkg/sql/join_test.go b/pkg/sql/join_test.go index e3a9c89cfb8a..f2ea8ed7904a 100644 --- a/pkg/sql/join_test.go +++ b/pkg/sql/join_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/span" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -27,7 +28,11 @@ func newTestScanNode(kvDB *kv.DB, tableName string) (*scanNode, error) { p := planner{alloc: &sqlbase.DatumAlloc{}} scan := p.Scan() scan.desc = desc - err := scan.initDescDefaults(publicColumnsCfg) + var colCfg scanColumnsConfig + for _, col := range desc.Columns { + colCfg.wantedColumns = append(colCfg.wantedColumns, tree.ColumnID(col.ID)) + } + err := scan.initDescDefaults(colCfg) if err != nil { return nil, err } diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index e5903db2df3c..4b0a5bad6491 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -20,7 +20,6 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/geo/geoindex" - "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" @@ -2039,23 +2038,3 @@ func makeColDescList(table cat.Table, cols exec.TableColumnOrdinalSet) []sqlbase } return colDescs } - -// makeScanColumnsConfig builds a scanColumnsConfig struct by constructing a -// list of descriptor IDs for columns in the given cols set. Columns are -// identified by their ordinal position in the table schema. -func makeScanColumnsConfig(table cat.Table, cols exec.TableColumnOrdinalSet) scanColumnsConfig { - // Set visibility=execinfra.ScanVisibilityPublicAndNotPublic, since all - // columns in the "cols" set should be projected, regardless of whether - // they're public or non- public. The caller decides which columns to - // include (or not include). Note that when wantedColumns is non-empty, - // the visibility flag will never trigger the addition of more columns. - colCfg := scanColumnsConfig{ - wantedColumns: make([]tree.ColumnID, 0, cols.Len()), - visibility: execinfra.ScanVisibilityPublicAndNotPublic, - } - for c, ok := cols.Next(0); ok; c, ok = cols.Next(c + 1) { - desc := table.Column(c).(*sqlbase.ColumnDescriptor) - colCfg.wantedColumns = append(colCfg.wantedColumns, tree.ColumnID(desc.ID)) - } - return colCfg -} diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 777093578c28..94cf82f453f2 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -110,23 +110,16 @@ type scanNode struct { lockingWaitPolicy sqlbase.ScanLockingWaitPolicy } -// scanColumnsConfig controls the "schema" of a scan node. The zero value is the -// default: all "public" columns. -// Note that not all columns in the schema are read and decoded; that is further -// controlled by scanNode.valNeededForCol. +// scanColumnsConfig controls the "schema" of a scan node. type scanColumnsConfig struct { - // If set, only these columns are part of the scan node schema, in this order - // (with the caveat that the addUnwantedAsHidden flag below can add more - // columns). Non public columns can only be added if allowed by the visibility - // flag below. - // If not set, then all visible columns will be part of the scan node schema, - // as specified by the visibility flag below. The addUnwantedAsHidden flag - // is ignored in this case. + // wantedColumns contains all the columns are part of the scan node schema, + // in this order (with the caveat that the addUnwantedAsHidden flag below + // can add more columns). Non public columns can only be added if allowed + // by the visibility flag below. wantedColumns []tree.ColumnID // When set, the columns that are not in the wantedColumns list are added to - // the list of columns as hidden columns. Only useful in conjunction with - // wantedColumns. + // the list of columns as hidden columns. addUnwantedAsHidden bool // If visibility is set to execinfra.ScanVisibilityPublicAndNotPublic, then @@ -143,8 +136,6 @@ func (cfg scanColumnsConfig) assertValidReqOrdering(reqOrdering exec.OutputOrder return nil } -var publicColumnsCfg = scanColumnsConfig{} - func (p *planner) Scan() *scanNode { n := scanNodePool.Get().(*scanNode) return n @@ -291,13 +282,7 @@ func initColsForScan( desc *sqlbase.ImmutableTableDescriptor, colCfg scanColumnsConfig, ) (cols []sqlbase.ColumnDescriptor, err error) { if colCfg.wantedColumns == nil { - // Add all active and maybe mutation columns. - if colCfg.visibility == execinfra.ScanVisibilityPublic { - cols = desc.Columns - } else { - cols = desc.ReadableColumns - } - return cols, nil + return nil, errors.AssertionFailedf("unexpectedly wantedColumns is nil") } cols = make([]sqlbase.ColumnDescriptor, 0, len(desc.ReadableColumns)) From 6ff32cbb291ba1bea96d8ccec3ec66d2ec365e45 Mon Sep 17 00:00:00 2001 From: Rebecca Taft Date: Mon, 8 Jun 2020 10:18:03 -0500 Subject: [PATCH 17/46] opt: fix error caused by recursive CTE with zero rows on left side Prior to this commit, a recursive CTE in which the cardinality of the left side of the UNION ALL expression was zero would cause an error in the statistics code, "estimated row count must be non-zero". This was happening because the cardinality of the recursive CTE binding props was set to be non-zero, but the row count, which came from the left side expression, was not updated accordingly. The stats code only allows the row count to be zero if the cardinality is also zero. This commit fixes the problem by setting the row count of the binding props to 1 if the estimated row count of the left side is less than 1. Fixes #49911 Release note (bug fix): Fixed an internal planning error that occured for recursive CTEs (WITH RECURSIVE expressions) in which the left side of the UNION ALL query used in the CTE definition produced zero rows. --- pkg/sql/opt/memo/testdata/stats/with | 50 ++++++++++++++++++++++++++++ pkg/sql/opt/optbuilder/with.go | 5 +++ 2 files changed, 55 insertions(+) diff --git a/pkg/sql/opt/memo/testdata/stats/with b/pkg/sql/opt/memo/testdata/stats/with index 3d246b01e16e..ee8ee3fa5ef1 100644 --- a/pkg/sql/opt/memo/testdata/stats/with +++ b/pkg/sql/opt/memo/testdata/stats/with @@ -154,3 +154,53 @@ with &1 (t0) │ └── filters (true) └── projections └── NULL [as="?column?":27, type=unknown] + +exec-ddl +CREATE TABLE test ( + id string +) +---- + +# Regression test for #49911. Make sure there is no error if the left side of +# a recursive CTE has cardinality=0. +norm +WITH RECURSIVE hierarchy(id) as + (SELECT id FROM test WHERE id = 'foo' AND 1 != 1 UNION ALL SELECT c.id FROM test AS c, hierarchy AS p WHERE c.id = 'bar') +SELECT * FROM hierarchy +---- +project + ├── columns: id:7(string) + ├── stats: [rows=10] + ├── recursive-c-t-e + │ ├── columns: id:3(string) + │ ├── working table binding: &1 + │ ├── initial columns: test.id:1(string) + │ ├── recursive columns: c.id:4(string) + │ ├── stats: [rows=10] + │ ├── values + │ │ ├── columns: test.id:1(string!null) + │ │ ├── cardinality: [0 - 0] + │ │ ├── stats: [rows=0] + │ │ ├── key: () + │ │ └── fd: ()-->(1) + │ └── inner-join (cross) + │ ├── columns: c.id:4(string!null) + │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) + │ ├── stats: [rows=10] + │ ├── fd: ()-->(4) + │ ├── select + │ │ ├── columns: c.id:4(string!null) + │ │ ├── stats: [rows=10, distinct(4)=1, null(4)=0] + │ │ ├── fd: ()-->(4) + │ │ ├── scan c + │ │ │ ├── columns: c.id:4(string) + │ │ │ └── stats: [rows=1000, distinct(4)=100, null(4)=10] + │ │ └── filters + │ │ └── c.id:4 = 'bar' [type=bool, outer=(4), constraints=(/4: [/'bar' - /'bar']; tight), fd=()-->(4)] + │ ├── with-scan &1 (hierarchy) + │ │ ├── mapping: + │ │ ├── cardinality: [1 - ] + │ │ └── stats: [rows=1] + │ └── filters (true) + └── projections + └── id:3 [as=id:7, type=string, outer=(3)] diff --git a/pkg/sql/opt/optbuilder/with.go b/pkg/sql/opt/optbuilder/with.go index 025fc28a7170..2cb0956144fb 100644 --- a/pkg/sql/opt/optbuilder/with.go +++ b/pkg/sql/opt/optbuilder/with.go @@ -127,6 +127,11 @@ func (b *Builder) buildCTE( // We don't really know the input row count, except for the first time we run // the recursive query. We don't have anything better though. bindingProps.Stats.RowCount = initialScope.expr.Relational().Stats.RowCount + // Row count must be greater than 0 or the stats code will throw an error. + // Set it to 1 to match the cardinality. + if bindingProps.Stats.RowCount < 1 { + bindingProps.Stats.RowCount = 1 + } cteSrc.bindingProps = bindingProps cteSrc.cols = b.getCTECols(initialScope, cte.Name) From b0a478adca37e01d6d832c114660b2fe1c97a7f3 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Mon, 8 Jun 2020 12:16:52 -0400 Subject: [PATCH 18/46] lint: fix a linter message A message had an unescaped format and ended up rendered like: use YourFuncf("descriptive prefix %!s(MISSING)", ...) Release note: None --- pkg/testutils/lint/passes/fmtsafe/fmtsafe.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/testutils/lint/passes/fmtsafe/fmtsafe.go b/pkg/testutils/lint/passes/fmtsafe/fmtsafe.go index 57f10e9f7b3c..f55046ac48ee 100644 --- a/pkg/testutils/lint/passes/fmtsafe/fmtsafe.go +++ b/pkg/testutils/lint/passes/fmtsafe/fmtsafe.go @@ -223,7 +223,7 @@ func checkCallExpr(pass *analysis.Pass, enclosingFnName string, call *ast.CallEx lit := pass.TypesInfo.Types[call.Args[idx]].Value if lit != nil { - // A literal constant! All is well. + // A literal or constant! All is well. return } @@ -251,7 +251,7 @@ func checkCallExpr(pass *analysis.Pass, enclosingFnName string, call *ast.CallEx // Tip is exported for use in tests. var Tip = ` -Tip: use YourFuncf("descriptive prefix %s", ...) or list new formatting wrappers in pkg/testutils/lint/passes/fmtsafe/functions.go.` +Tip: use YourFuncf("descriptive prefix %%s", ...) or list new formatting wrappers in pkg/testutils/lint/passes/fmtsafe/functions.go.` func hasNoLintComment(pass *analysis.Pass, call *ast.CallExpr, idx int) bool { fPos, f := findContainingFile(pass, call) From 1c97090a4f2f98180b2a241714c530d2d7fd3337 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Mon, 8 Jun 2020 10:29:51 -0700 Subject: [PATCH 19/46] opt: incorporate operator volatility Incorporate volatility of operators (unary, binary, comparison) into the VolatilitySet property. Unfortunately, this modifies most plans as pretty much everything is "immutable". Perhaps once this work is behind us we will want to hide this information from plans in most cases. Release note: None --- pkg/sql/logictest/testdata/logic_test/prepare | 3 +- pkg/sql/opt/exec/execbuilder/testdata/explain | 14 +- pkg/sql/opt/memo/logical_props_builder.go | 32 ++++- pkg/sql/opt/memo/testdata/format | 40 ++++-- .../opt/memo/testdata/logprops/constraints | 38 +++-- pkg/sql/opt/memo/testdata/logprops/index-join | 7 +- pkg/sql/opt/memo/testdata/logprops/insert | 10 +- pkg/sql/opt/memo/testdata/logprops/join | 5 +- pkg/sql/opt/memo/testdata/logprops/project | 3 +- pkg/sql/opt/memo/testdata/logprops/scalar | 2 +- pkg/sql/opt/memo/testdata/logprops/set | 3 +- pkg/sql/opt/memo/testdata/logprops/update | 12 +- pkg/sql/opt/memo/testdata/logprops/upsert | 18 +-- pkg/sql/opt/memo/testdata/logprops/values | 5 +- pkg/sql/opt/memo/testdata/logprops/window | 7 +- pkg/sql/opt/memo/testdata/memo | 4 +- pkg/sql/opt/memo/testdata/stats/groupby | 6 +- pkg/sql/opt/memo/testdata/stats/index-join | 11 +- pkg/sql/opt/memo/testdata/stats/join | 28 +++- pkg/sql/opt/memo/testdata/stats/project | 4 +- pkg/sql/opt/memo/testdata/stats/scan | 12 +- pkg/sql/opt/memo/testdata/stats/select | 19 ++- pkg/sql/opt/memo/testdata/stats_quality/tpcc | 8 +- .../opt/memo/testdata/stats_quality/tpch/q01 | 7 +- .../opt/memo/testdata/stats_quality/tpch/q03 | 6 +- .../opt/memo/testdata/stats_quality/tpch/q05 | 5 +- .../opt/memo/testdata/stats_quality/tpch/q06 | 4 +- .../opt/memo/testdata/stats_quality/tpch/q07 | 2 +- .../opt/memo/testdata/stats_quality/tpch/q08 | 2 +- .../opt/memo/testdata/stats_quality/tpch/q09 | 2 +- .../opt/memo/testdata/stats_quality/tpch/q10 | 6 +- .../opt/memo/testdata/stats_quality/tpch/q11 | 2 +- .../opt/memo/testdata/stats_quality/tpch/q14 | 6 +- .../opt/memo/testdata/stats_quality/tpch/q15 | 15 +- .../opt/memo/testdata/stats_quality/tpch/q17 | 7 +- .../opt/memo/testdata/stats_quality/tpch/q19 | 4 +- .../opt/memo/testdata/stats_quality/tpch/q20 | 10 +- pkg/sql/opt/norm/testdata/rules/bool | 25 ++-- pkg/sql/opt/norm/testdata/rules/combo | 14 +- pkg/sql/opt/norm/testdata/rules/comp | 26 ++-- pkg/sql/opt/norm/testdata/rules/decorrelate | 45 +++++- .../opt/norm/testdata/rules/fold_constants | 9 +- pkg/sql/opt/norm/testdata/rules/groupby | 18 ++- pkg/sql/opt/norm/testdata/rules/inline | 68 +++++---- pkg/sql/opt/norm/testdata/rules/join | 133 +++++++++++++----- pkg/sql/opt/norm/testdata/rules/limit | 34 +++-- pkg/sql/opt/norm/testdata/rules/numeric | 40 +++--- pkg/sql/opt/norm/testdata/rules/ordering | 5 +- pkg/sql/opt/norm/testdata/rules/project | 43 ++++-- pkg/sql/opt/norm/testdata/rules/project_set | 3 + pkg/sql/opt/norm/testdata/rules/prune_cols | 128 +++++++++++------ pkg/sql/opt/norm/testdata/rules/reject_nulls | 15 +- pkg/sql/opt/norm/testdata/rules/scalar | 79 +++++++---- pkg/sql/opt/norm/testdata/rules/select | 20 ++- pkg/sql/opt/norm/testdata/rules/with | 18 ++- pkg/sql/opt/optgen/exprgen/testdata/join | 4 +- pkg/sql/opt/optgen/exprgen/testdata/values | 3 +- pkg/sql/opt/xform/testdata/coster/join | 5 +- pkg/sql/opt/xform/testdata/coster/project | 8 +- pkg/sql/opt/xform/testdata/coster/scan | 6 +- pkg/sql/opt/xform/testdata/external/customer | 4 +- pkg/sql/opt/xform/testdata/external/hibernate | 26 +++- pkg/sql/opt/xform/testdata/external/tpcc | 15 +- .../xform/testdata/external/tpcc-later-stats | 15 +- .../opt/xform/testdata/external/tpcc-no-stats | 15 +- pkg/sql/opt/xform/testdata/external/tpch | 78 +++++++--- .../opt/xform/testdata/external/tpch-no-stats | 79 ++++++++--- pkg/sql/opt/xform/testdata/external/trading | 18 ++- .../xform/testdata/external/trading-mutation | 18 ++- pkg/sql/opt/xform/testdata/physprops/ordering | 37 +++-- .../opt/xform/testdata/physprops/presentation | 3 +- pkg/sql/opt/xform/testdata/rules/computed | 4 +- pkg/sql/opt/xform/testdata/rules/groupby | 6 +- pkg/sql/opt/xform/testdata/rules/join | 10 +- pkg/sql/opt/xform/testdata/rules/select | 71 ++++++++-- 75 files changed, 1089 insertions(+), 428 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/prepare b/pkg/sql/logictest/testdata/logic_test/prepare index 1eef588e572a..b749c3602a06 100644 --- a/pkg/sql/logictest/testdata/logic_test/prepare +++ b/pkg/sql/logictest/testdata/logic_test/prepare @@ -1140,6 +1140,7 @@ EXECUTE e ---- select ├── columns: k:1 str:2 + ├── immutable ├── stats: [rows=333.333333] ├── cost: 1050.03 ├── key: (1) @@ -1153,7 +1154,7 @@ select │ ├── fd: (1)-->(2) │ └── prune: (1,2) └── filters - └── (k:1 % 2) = 1 [outer=(1)] + └── (k:1 % 2) = 1 [outer=(1), immutable] # Only root may use PREPARE AS OPT PLAN. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 97aea455bc44..2f8f2334d26d 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -987,6 +987,7 @@ EXPLAIN (OPT, VERBOSE) SELECT * FROM tc WHERE a + 2 * b > 1 ORDER BY a*b ---- sort ├── columns: a:1 b:2 [hidden: column4:4] + ├── immutable ├── stats: [rows=333.333333] ├── cost: 1179.25548 ├── fd: (1,2)-->(4) @@ -995,6 +996,7 @@ sort ├── interesting orderings: (+1) └── project ├── columns: column4:4 a:1 b:2 + ├── immutable ├── stats: [rows=333.333333] ├── cost: 1116.70667 ├── fd: (1,2)-->(4) @@ -1002,6 +1004,7 @@ sort ├── interesting orderings: (+1) ├── select │ ├── columns: a:1 b:2 + │ ├── immutable │ ├── stats: [rows=333.333333] │ ├── cost: 1110.03 │ ├── interesting orderings: (+1) @@ -1012,15 +1015,16 @@ sort │ │ ├── prune: (1,2) │ │ └── interesting orderings: (+1) │ └── filters - │ └── (a:1 + (b:2 * 2)) > 1 [outer=(1,2)] + │ └── (a:1 + (b:2 * 2)) > 1 [outer=(1,2), immutable] └── projections - └── a:1 * b:2 [as=column4:4, outer=(1,2)] + └── a:1 * b:2 [as=column4:4, outer=(1,2), immutable] query T EXPLAIN (OPT, TYPES) SELECT * FROM tc WHERE a + 2 * b > 1 ORDER BY a*b ---- sort ├── columns: a:1(int) b:2(int) [hidden: column4:4(int)] + ├── immutable ├── stats: [rows=333.333333] ├── cost: 1179.25548 ├── fd: (1,2)-->(4) @@ -1029,6 +1033,7 @@ sort ├── interesting orderings: (+1) └── project ├── columns: column4:4(int) a:1(int) b:2(int) + ├── immutable ├── stats: [rows=333.333333] ├── cost: 1116.70667 ├── fd: (1,2)-->(4) @@ -1036,6 +1041,7 @@ sort ├── interesting orderings: (+1) ├── select │ ├── columns: a:1(int) b:2(int) + │ ├── immutable │ ├── stats: [rows=333.333333] │ ├── cost: 1110.03 │ ├── interesting orderings: (+1) @@ -1046,7 +1052,7 @@ sort │ │ ├── prune: (1,2) │ │ └── interesting orderings: (+1) │ └── filters - │ └── gt [type=bool, outer=(1,2)] + │ └── gt [type=bool, outer=(1,2), immutable] │ ├── plus [type=int] │ │ ├── variable: a:1 [type=int] │ │ └── mult [type=int] @@ -1054,7 +1060,7 @@ sort │ │ └── const: 2 [type=int] │ └── const: 1 [type=int] └── projections - └── mult [as=column4:4, type=int, outer=(1,2)] + └── mult [as=column4:4, type=int, outer=(1,2), immutable] ├── variable: a:1 [type=int] └── variable: b:2 [type=int] diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 04c4de1855fe..2f22176d11d3 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -1372,6 +1372,9 @@ func BuildSharedProps(e opt.Expr, shared *props.Shared) { case *DivExpr: // Division by zero error is possible, unless the right-hand side is a // non-zero constant. + // + // TODO(radu): this case should be removed (Div should be covered by the + // binary operator logic below). var nonZero bool if c, ok := t.Right.(*ConstExpr); ok { switch v := c.Value.(type) { @@ -1413,7 +1416,34 @@ func BuildSharedProps(e opt.Expr, shared *props.Shared) { shared.VolatilitySet.Add(volatility) default: - if opt.IsMutationOp(e) { + if opt.IsUnaryOp(e) { + inputType := e.Child(0).(opt.ScalarExpr).DataType() + o, ok := FindUnaryOverload(e.Op(), inputType) + if !ok { + panic(errors.AssertionFailedf("unary overload not found (%s, %s)", e.Op(), inputType)) + } + shared.VolatilitySet.Add(o.Volatility) + } else if opt.IsComparisonOp(e) { + leftType := e.Child(0).(opt.ScalarExpr).DataType() + rightType := e.Child(1).(opt.ScalarExpr).DataType() + o, _, _, ok := FindComparisonOverload(e.Op(), leftType, rightType) + if !ok { + panic(errors.AssertionFailedf( + "comparison overload not found (%s, %s, %s)", e.Op(), leftType, rightType, + )) + } + shared.VolatilitySet.Add(o.Volatility) + } else if opt.IsBinaryOp(e) { + leftType := e.Child(0).(opt.ScalarExpr).DataType() + rightType := e.Child(1).(opt.ScalarExpr).DataType() + o, ok := FindBinaryOverload(e.Op(), leftType, rightType) + if !ok { + panic(errors.AssertionFailedf( + "binary overload not found (%s, %s, %s)", e.Op(), leftType, rightType, + )) + } + shared.VolatilitySet.Add(o.Volatility) + } else if opt.IsMutationOp(e) { shared.CanHaveSideEffects = true shared.CanMutate = true shared.VolatilitySet.AddVolatile() diff --git a/pkg/sql/opt/memo/testdata/format b/pkg/sql/opt/memo/testdata/format index db05c0fc99dc..4033c79ee52a 100644 --- a/pkg/sql/opt/memo/testdata/format +++ b/pkg/sql/opt/memo/testdata/format @@ -7,6 +7,7 @@ SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort ├── columns: "?column?":5(int) min:4(int!null) [hidden: t.public.t.a:1(int)] + ├── immutable ├── stats: [rows=98.1771622] ├── cost: 1097.87224 ├── key: (1) @@ -15,6 +16,7 @@ sort ├── prune: (1,4,5) └── project ├── columns: "?column?":5(int) t.public.t.a:1(int) min:4(int!null) + ├── immutable ├── stats: [rows=98.1771622] ├── cost: 1082.90531 ├── key: (1) @@ -23,6 +25,7 @@ sort ├── group-by │ ├── columns: t.public.t.a:1(int) min:4(int!null) │ ├── grouping columns: t.public.t.a:1(int) + │ ├── immutable │ ├── stats: [rows=98.1771622, distinct(1)=98.1771622, null(1)=1] │ ├── cost: 1080.93177 │ ├── key: (1) @@ -30,6 +33,7 @@ sort │ ├── prune: (4) │ ├── select │ │ ├── columns: t.public.t.a:1(int) t.public.t.b:2(int!null) t.public.t.k:3(int!null) + │ │ ├── immutable │ │ ├── stats: [rows=330, distinct(1)=98.1771622, null(1)=3.3, distinct(2)=100, null(2)=0] │ │ ├── cost: 1070.03 │ │ ├── key: (3) @@ -44,7 +48,7 @@ sort │ │ │ ├── prune: (1-3) │ │ │ └── interesting orderings: (+3) │ │ └── filters - │ │ └── lt [type=bool, outer=(1-3), constraints=(/2: (/NULL - ])] + │ │ └── lt [type=bool, outer=(1-3), immutable, constraints=(/2: (/NULL - ])] │ │ ├── variable: t.public.t.b:2 [type=int] │ │ └── plus [type=int] │ │ ├── variable: t.public.t.k:3 [type=int] @@ -53,7 +57,7 @@ sort │ └── min [as=min:4, type=int, outer=(2)] │ └── variable: t.public.t.b:2 [type=int] └── projections - └── plus [as="?column?":5, type=int, outer=(1)] + └── plus [as="?column?":5, type=int, outer=(1), immutable] ├── variable: t.public.t.a:1 [type=int] └── const: 1 [type=int] @@ -101,23 +105,27 @@ SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort ├── columns: "?column?":5(int) min:4(int!null) [hidden: a:1(int)] + ├── immutable ├── key: (1) ├── fd: (1)-->(4,5) ├── ordering: +1 ├── prune: (1,4,5) └── project ├── columns: "?column?":5(int) a:1(int) min:4(int!null) + ├── immutable ├── key: (1) ├── fd: (1)-->(4,5) ├── prune: (1,4,5) ├── group-by │ ├── columns: a:1(int) min:4(int!null) │ ├── grouping columns: a:1(int) + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(4) │ ├── prune: (4) │ ├── select │ │ ├── columns: a:1(int) b:2(int!null) k:3(int!null) + │ │ ├── immutable │ │ ├── key: (3) │ │ ├── fd: (3)-->(1,2) │ │ ├── interesting orderings: (+3) @@ -128,35 +136,39 @@ sort │ │ │ ├── prune: (1-3) │ │ │ └── interesting orderings: (+3) │ │ └── filters - │ │ └── b:2 < (k:3 + a:1) [type=bool, outer=(1-3), constraints=(/2: (/NULL - ])] + │ │ └── b:2 < (k:3 + a:1) [type=bool, outer=(1-3), immutable, constraints=(/2: (/NULL - ])] │ └── aggregations │ └── min [as=min:4, type=int, outer=(2)] │ └── b:2 [type=int] └── projections - └── a:1 + 1 [as="?column?":5, type=int, outer=(1)] + └── a:1 + 1 [as="?column?":5, type=int, outer=(1), immutable] opt format=(hide-stats,hide-cost,hide-qual,hide-scalars,hide-types) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort ├── columns: "?column?":5 min:4!null [hidden: a:1] + ├── immutable ├── key: (1) ├── fd: (1)-->(4,5) ├── ordering: +1 ├── prune: (1,4,5) └── project ├── columns: "?column?":5 a:1 min:4!null + ├── immutable ├── key: (1) ├── fd: (1)-->(4,5) ├── prune: (1,4,5) ├── group-by │ ├── columns: a:1 min:4!null │ ├── grouping columns: a:1 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(4) │ ├── prune: (4) │ ├── select │ │ ├── columns: a:1 b:2!null k:3!null + │ │ ├── immutable │ │ ├── key: (3) │ │ ├── fd: (3)-->(1,2) │ │ ├── interesting orderings: (+3) @@ -167,35 +179,39 @@ sort │ │ │ ├── prune: (1-3) │ │ │ └── interesting orderings: (+3) │ │ └── filters - │ │ └── b:2 < (k:3 + a:1) [outer=(1-3), constraints=(/2: (/NULL - ])] + │ │ └── b:2 < (k:3 + a:1) [outer=(1-3), immutable, constraints=(/2: (/NULL - ])] │ └── aggregations │ └── min [as=min:4, outer=(2)] │ └── b:2 └── projections - └── a:1 + 1 [as="?column?":5, outer=(1)] + └── a:1 + 1 [as="?column?":5, outer=(1), immutable] opt format=(hide-stats,hide-cost,hide-qual,hide-scalars,hide-notnull) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort ├── columns: "?column?":5(int) min:4(int) [hidden: a:1(int)] + ├── immutable ├── key: (1) ├── fd: (1)-->(4,5) ├── ordering: +1 ├── prune: (1,4,5) └── project ├── columns: "?column?":5(int) a:1(int) min:4(int) + ├── immutable ├── key: (1) ├── fd: (1)-->(4,5) ├── prune: (1,4,5) ├── group-by │ ├── columns: a:1(int) min:4(int) │ ├── grouping columns: a:1(int) + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(4) │ ├── prune: (4) │ ├── select │ │ ├── columns: a:1(int) b:2(int) k:3(int) + │ │ ├── immutable │ │ ├── key: (3) │ │ ├── fd: (3)-->(1,2) │ │ ├── interesting orderings: (+3) @@ -206,35 +222,39 @@ sort │ │ │ ├── prune: (1-3) │ │ │ └── interesting orderings: (+3) │ │ └── filters - │ │ └── b:2 < (k:3 + a:1) [type=bool, outer=(1-3), constraints=(/2: (/NULL - ])] + │ │ └── b:2 < (k:3 + a:1) [type=bool, outer=(1-3), immutable, constraints=(/2: (/NULL - ])] │ └── aggregations │ └── min [as=min:4, type=int, outer=(2)] │ └── b:2 [type=int] └── projections - └── a:1 + 1 [as="?column?":5, type=int, outer=(1)] + └── a:1 + 1 [as="?column?":5, type=int, outer=(1), immutable] opt format=(hide-stats,hide-cost,hide-qual,hide-scalars,hide-types,hide-notnull) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort ├── columns: "?column?":5 min:4 [hidden: a:1] + ├── immutable ├── key: (1) ├── fd: (1)-->(4,5) ├── ordering: +1 ├── prune: (1,4,5) └── project ├── columns: "?column?":5 a:1 min:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(4,5) ├── prune: (1,4,5) ├── group-by │ ├── columns: a:1 min:4 │ ├── grouping columns: a:1 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(4) │ ├── prune: (4) │ ├── select │ │ ├── columns: a:1 b:2 k:3 + │ │ ├── immutable │ │ ├── key: (3) │ │ ├── fd: (3)-->(1,2) │ │ ├── interesting orderings: (+3) @@ -245,12 +265,12 @@ sort │ │ │ ├── prune: (1-3) │ │ │ └── interesting orderings: (+3) │ │ └── filters - │ │ └── b:2 < (k:3 + a:1) [outer=(1-3), constraints=(/2: (/NULL - ])] + │ │ └── b:2 < (k:3 + a:1) [outer=(1-3), immutable, constraints=(/2: (/NULL - ])] │ └── aggregations │ └── min [as=min:4, outer=(2)] │ └── b:2 └── projections - └── a:1 + 1 [as="?column?":5, outer=(1)] + └── a:1 + 1 [as="?column?":5, outer=(1), immutable] opt format=(hide-miscprops,hide-physprops,hide-columns) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a diff --git a/pkg/sql/opt/memo/testdata/logprops/constraints b/pkg/sql/opt/memo/testdata/logprops/constraints index fe61562ef935..9688e792b40a 100644 --- a/pkg/sql/opt/memo/testdata/logprops/constraints +++ b/pkg/sql/opt/memo/testdata/logprops/constraints @@ -167,6 +167,7 @@ SELECT * FROM a WHERE x > 1 AND x < 5 AND x + y = 5 ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) @@ -179,7 +180,7 @@ select │ └── lt [type=bool] │ ├── variable: x:1 [type=int] │ └── const: 5 [type=int] - └── eq [type=bool, outer=(1,2)] + └── eq [type=bool, outer=(1,2), immutable] ├── plus [type=int] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] @@ -190,6 +191,7 @@ SELECT * FROM a WHERE x > 1 AND x + y >= 5 AND x + y <= 7 ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) @@ -197,12 +199,12 @@ select ├── gt [type=bool, outer=(1), constraints=(/1: [/2 - ]; tight)] │ ├── variable: x:1 [type=int] │ └── const: 1 [type=int] - ├── ge [type=bool, outer=(1,2)] + ├── ge [type=bool, outer=(1,2), immutable] │ ├── plus [type=int] │ │ ├── variable: x:1 [type=int] │ │ └── variable: y:2 [type=int] │ └── const: 5 [type=int] - └── le [type=bool, outer=(1,2)] + └── le [type=bool, outer=(1,2), immutable] ├── plus [type=int] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] @@ -357,11 +359,12 @@ SELECT * FROM a WHERE (x, y) > (1, 2) ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) └── filters - └── gt [type=bool, outer=(1,2), constraints=(/1/2: [/1/3 - ]; tight)] + └── gt [type=bool, outer=(1,2), immutable, constraints=(/1/2: [/1/3 - ]; tight)] ├── tuple [type=tuple{int, int}] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] @@ -374,11 +377,12 @@ SELECT * FROM a WHERE (x, y) >= (1, 2) ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) └── filters - └── ge [type=bool, outer=(1,2), constraints=(/1/2: [/1/2 - ]; tight)] + └── ge [type=bool, outer=(1,2), immutable, constraints=(/1/2: [/1/2 - ]; tight)] ├── tuple [type=tuple{int, int}] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] @@ -391,11 +395,12 @@ SELECT * FROM a WHERE (x, y) < (1, 2) ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) └── filters - └── lt [type=bool, outer=(1,2), constraints=(/1/2: (/NULL - /1/1]; tight)] + └── lt [type=bool, outer=(1,2), immutable, constraints=(/1/2: (/NULL - /1/1]; tight)] ├── tuple [type=tuple{int, int}] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] @@ -408,11 +413,12 @@ SELECT * FROM a WHERE (x, y) <= (1, 2) ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) └── filters - └── le [type=bool, outer=(1,2), constraints=(/1/2: (/NULL - /1/2]; tight)] + └── le [type=bool, outer=(1,2), immutable, constraints=(/1/2: (/NULL - /1/2]; tight)] ├── tuple [type=tuple{int, int}] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] @@ -426,11 +432,12 @@ SELECT * FROM a WHERE (x, y) >= (1, 2.5) ---- select ├── columns: x:1(int) y:2(int) + ├── immutable ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) └── filters - └── ge [type=bool, outer=(1,2)] + └── ge [type=bool, outer=(1,2), immutable] ├── tuple [type=tuple{int, int}] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] @@ -444,11 +451,12 @@ SELECT * FROM a WHERE (x, y) >= (1, NULL) ---- select ├── columns: x:1(int) y:2(int) + ├── immutable ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) └── filters - └── ge [type=bool, outer=(1,2)] + └── ge [type=bool, outer=(1,2), immutable] ├── tuple [type=tuple{int, int}] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] @@ -463,12 +471,13 @@ SELECT * FROM a WHERE (x, 1) >= (1, 2) ---- select ├── columns: x:1(int) y:2(int) + ├── immutable ├── prune: (2) ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) └── filters - └── ge [type=bool, outer=(1)] + └── ge [type=bool, outer=(1), immutable] ├── tuple [type=tuple{int, int}] │ ├── variable: x:1 [type=int] │ └── const: 1 [type=int] @@ -640,6 +649,7 @@ SELECT * FROM (SELECT (x, y) AS col FROM a) WHERE col > (1, 2) ---- select ├── columns: col:4(tuple{int, int}!null) + ├── immutable ├── project │ ├── columns: col:4(tuple{int, int}) │ ├── prune: (4) @@ -651,7 +661,7 @@ select │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] └── filters - └── gt [type=bool, outer=(4), constraints=(/4: [/(1, 3) - ]; tight)] + └── gt [type=bool, outer=(4), immutable, constraints=(/4: [/(1, 3) - ]; tight)] ├── variable: col:4 [type=tuple{int, int}] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] @@ -812,6 +822,7 @@ SELECT * FROM c WHERE (v, u + v) IN ((1, 2), (3, 50), (5, 100)) ---- select ├── columns: k:1(int!null) u:2(int) v:3(int!null) + ├── immutable ├── key: (1) ├── fd: (1)-->(2,3) ├── prune: (1) @@ -827,7 +838,7 @@ select │ ├── prune: (1-3) │ └── interesting orderings: (+1) (+3,+2,+1) └── filters - └── in [type=bool, outer=(2,3), constraints=(/3: [/1 - /1] [/3 - /3] [/5 - /5])] + └── in [type=bool, outer=(2,3), immutable, constraints=(/3: [/1 - /1] [/3 - /3] [/5 - /5])] ├── tuple [type=tuple{int, int}] │ ├── variable: v:3 [type=int] │ └── plus [type=int] @@ -1251,11 +1262,12 @@ SELECT * FROM a WHERE (x, y) < (1, 2) OR (x, y) > (3, 4) ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── scan a │ ├── columns: x:1(int) y:2(int) │ └── prune: (1,2) └── filters - └── or [type=bool, outer=(1,2), constraints=(/1/2: (/NULL - /1/1] [/3/5 - ]; tight)] + └── or [type=bool, outer=(1,2), immutable, constraints=(/1/2: (/NULL - /1/1] [/3/5 - ]; tight)] ├── lt [type=bool] │ ├── tuple [type=tuple{int, int}] │ │ ├── variable: x:1 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/index-join b/pkg/sql/opt/memo/testdata/logprops/index-join index 0ee2af826526..d2953cda92e8 100644 --- a/pkg/sql/opt/memo/testdata/logprops/index-join +++ b/pkg/sql/opt/memo/testdata/logprops/index-join @@ -27,6 +27,7 @@ SELECT * FROM a WHERE s = 'foo' AND x + y = 10 ---- select ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) + ├── immutable ├── key: (1) ├── fd: ()-->(3), (1)-->(2,4), (4)-->(1,2), (2,3)~~>(1,4) ├── prune: (4) @@ -44,7 +45,7 @@ select │ ├── prune: (1,3,4) │ └── interesting orderings: (+1) (-3,+4,+1) └── filters - └── eq [type=bool, outer=(1,2)] + └── eq [type=bool, outer=(1,2), immutable] ├── plus [type=int] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] @@ -55,9 +56,11 @@ SELECT y FROM a WHERE s = 'foo' AND x + y = 10 ---- project ├── columns: y:2(int) + ├── immutable ├── prune: (2) └── select ├── columns: x:1(int!null) y:2(int) s:3(string!null) + ├── immutable ├── key: (1) ├── fd: ()-->(3), (1)-->(2), (2,3)~~>(1) ├── interesting orderings: (+1) (-3) @@ -74,7 +77,7 @@ project │ ├── prune: (1,3) │ └── interesting orderings: (+1) (-3) └── filters - └── eq [type=bool, outer=(1,2)] + └── eq [type=bool, outer=(1,2), immutable] ├── plus [type=int] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/insert b/pkg/sql/opt/memo/testdata/logprops/insert index da2b0c325da0..7db1dc52f012 100644 --- a/pkg/sql/opt/memo/testdata/logprops/insert +++ b/pkg/sql/opt/memo/testdata/logprops/insert @@ -71,7 +71,7 @@ insert abcde │ └── cast: INT8 [as=column12:12, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column13:13, type=int, outer=(8,10)] + └── plus [as=column13:13, type=int, outer=(8,10), immutable] ├── plus [type=int] │ ├── variable: y:8 [type=int] │ └── variable: column10:10 [type=int] @@ -139,7 +139,7 @@ project │ └── cast: INT8 [as=column12:12, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column13:13, type=int, outer=(8,10)] + └── plus [as=column13:13, type=int, outer=(8,10), immutable] ├── plus [type=int] │ ├── variable: y:8 [type=int] │ └── variable: column10:10 [type=int] @@ -190,7 +190,7 @@ project │ └── cast: INT8 [as=column12:12, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column13:13, type=int, outer=(8,10)] + └── plus [as=column13:13, type=int, outer=(8,10), immutable] ├── plus [type=int] │ ├── variable: y:8 [type=int] │ └── variable: column10:10 [type=int] @@ -242,7 +242,7 @@ insert abcde │ └── cast: INT8 [as=column11:11, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column12:12, type=int, outer=(8,9)] + └── plus [as=column12:12, type=int, outer=(8,9), immutable] ├── plus [type=int] │ ├── variable: column2:8 [type=int] │ └── variable: column9:9 [type=int] @@ -310,7 +310,7 @@ project │ └── cast: INT8 [as=column13:13, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column14:14, type=int, outer=(10,11)] + └── plus [as=column14:14, type=int, outer=(10,11), immutable] ├── plus [type=int] │ ├── variable: int8:10 [type=int] │ └── variable: column11:11 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/join b/pkg/sql/opt/memo/testdata/logprops/join index f954aeb3caac..da58abe2557b 100644 --- a/pkg/sql/opt/memo/testdata/logprops/join +++ b/pkg/sql/opt/memo/testdata/logprops/join @@ -1989,6 +1989,7 @@ SELECT * FROM mn LEFT JOIN xysd ON y = (n * 2) ---- project ├── columns: m:1(int!null) n:2(int) x:3(int) y:4(int) s:5(string) d:6(decimal) + ├── immutable ├── key: (1,3) ├── fd: (1)-->(2), (2)~~>(1), (3)-->(4-6), (5,6)~~>(3,4) ├── prune: (1-6) @@ -1997,6 +1998,7 @@ project └── left-join (hash) ├── columns: m:1(int!null) n:2(int) x:3(int) y:4(int) s:5(string) d:6(decimal) column7:7(int) ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) + ├── immutable ├── key: (1,3) ├── fd: (1)-->(2), (2)~~>(1), (2)-->(7), (3)-->(4-6), (5,6)~~>(3,4) ├── prune: (1-3,5,6) @@ -2004,6 +2006,7 @@ project ├── interesting orderings: (+1) (+2,+1) (+3) (-5,+6,+3) ├── project │ ├── columns: column7:7(int) m:1(int!null) n:2(int) + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2), (2)~~>(1), (2)-->(7) │ ├── prune: (1,2,7) @@ -2017,7 +2020,7 @@ project │ │ ├── interesting orderings: (+1) (+2,+1) │ │ └── unfiltered-cols: (1,2) │ └── projections - │ └── mult [as=column7:7, type=int, outer=(2)] + │ └── mult [as=column7:7, type=int, outer=(2), immutable] │ ├── variable: n:2 [type=int] │ └── const: 2 [type=int] ├── scan xysd diff --git a/pkg/sql/opt/memo/testdata/logprops/project b/pkg/sql/opt/memo/testdata/logprops/project index 2ab2947a7c7c..2f2b2b08ff2a 100644 --- a/pkg/sql/opt/memo/testdata/logprops/project +++ b/pkg/sql/opt/memo/testdata/logprops/project @@ -15,6 +15,7 @@ SELECT y, x+1 AS a, 1 AS b, x FROM xysd ---- project ├── columns: y:2(int) a:5(int!null) b:6(int!null) x:1(int!null) + ├── immutable ├── key: (1) ├── fd: ()-->(6), (1)-->(2,5) ├── prune: (1,2,5,6) @@ -26,7 +27,7 @@ project │ ├── prune: (1-4) │ └── interesting orderings: (+1) (-3,+4,+1) └── projections - ├── plus [as=a:5, type=int, outer=(1)] + ├── plus [as=a:5, type=int, outer=(1), immutable] │ ├── variable: x:1 [type=int] │ └── const: 1 [type=int] └── const: 1 [as=b:6, type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/scalar b/pkg/sql/opt/memo/testdata/logprops/scalar index 6d8356b27778..b5999b24d55c 100644 --- a/pkg/sql/opt/memo/testdata/logprops/scalar +++ b/pkg/sql/opt/memo/testdata/logprops/scalar @@ -67,7 +67,7 @@ project │ ├── function: length [type=int] │ │ └── const: 'foo' [type=string] │ └── variable: y:2 [type=int] - └── mult [as=b:7, type=int, outer=(1,5)] + └── mult [as=b:7, type=int, outer=(1,5), immutable] ├── variable: rowid:5 [type=int] └── variable: x:1 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/set b/pkg/sql/opt/memo/testdata/logprops/set index 0b85ab23a170..18f8a6464a32 100644 --- a/pkg/sql/opt/memo/testdata/logprops/set +++ b/pkg/sql/opt/memo/testdata/logprops/set @@ -102,6 +102,7 @@ SELECT * FROM xy WHERE (SELECT x, u FROM uv UNION SELECT y, v FROM uv) = (1, 2) ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── key: (1) ├── fd: (1)-->(2) ├── interesting orderings: (+1) @@ -112,7 +113,7 @@ select │ ├── prune: (1,2) │ └── interesting orderings: (+1) └── filters - └── eq [type=bool, outer=(1,2), correlated-subquery] + └── eq [type=bool, outer=(1,2), immutable, correlated-subquery] ├── subquery [type=tuple{int, int}] │ └── max1-row │ ├── columns: column13:13(tuple{int, int}) diff --git a/pkg/sql/opt/memo/testdata/logprops/update b/pkg/sql/opt/memo/testdata/logprops/update index 91f726ef6902..3b755f9ebbfb 100644 --- a/pkg/sql/opt/memo/testdata/logprops/update +++ b/pkg/sql/opt/memo/testdata/logprops/update @@ -31,6 +31,7 @@ update abcde ├── volatile, side-effects, mutations └── project ├── columns: column15:15(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) b_new:13(int!null) column14:14(int!null) + ├── immutable ├── key: (11) ├── fd: ()-->(7,13,14), (11)-->(8-10,12), (9)-->(15) ├── prune: (7-15) @@ -75,7 +76,7 @@ update abcde │ └── projections │ └── const: 0 [as=column14:14, type=int] └── projections - └── plus [as=column15:15, type=int, outer=(9,13)] + └── plus [as=column15:15, type=int, outer=(9,13), immutable] ├── plus [type=int] │ ├── variable: b_new:13 [type=int] │ └── variable: c:9 [type=int] @@ -102,6 +103,7 @@ project ├── fd: ()-->(1,2), (5)-->(3,4), (3)-->(4) └── project ├── columns: column15:15(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) b_new:13(int!null) column14:14(int!null) + ├── immutable ├── key: (11) ├── fd: ()-->(7,13,14), (11)-->(8-10,12), (9)-->(15) ├── prune: (7-15) @@ -146,7 +148,7 @@ project │ └── projections │ └── const: 0 [as=column14:14, type=int] └── projections - └── plus [as=column15:15, type=int, outer=(9,13)] + └── plus [as=column15:15, type=int, outer=(9,13), immutable] ├── plus [type=int] │ ├── variable: b_new:13 [type=int] │ └── variable: c:9 [type=int] @@ -177,6 +179,7 @@ project └── project ├── columns: column15:15(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) b_new:13(int!null) column14:14(int!null) ├── cardinality: [0 - 1] + ├── immutable ├── key: () ├── fd: ()-->(7-15) ├── prune: (7-15) @@ -224,7 +227,7 @@ project │ └── projections │ └── const: 0 [as=column14:14, type=int] └── projections - └── plus [as=column15:15, type=int, outer=(9,13)] + └── plus [as=column15:15, type=int, outer=(9,13), immutable] ├── plus [type=int] │ ├── variable: b_new:13 [type=int] │ └── variable: c:9 [type=int] @@ -251,6 +254,7 @@ project ├── fd: ()-->(1), (2)==(3), (3)==(2), (5)-->(2-4), (2)-->(4) └── project ├── columns: column15:15(int!null) a:7(int!null) b:8(int!null) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) a_new:13(int!null) column14:14(int!null) + ├── immutable ├── key: (11) ├── fd: ()-->(13,14), (11)-->(7-10,12), (8)==(9), (9)==(8), (8,9)-->(15) ├── prune: (7-15) @@ -295,7 +299,7 @@ project │ └── projections │ └── const: 0 [as=column14:14, type=int] └── projections - └── plus [as=column15:15, type=int, outer=(8,9)] + └── plus [as=column15:15, type=int, outer=(8,9), immutable] ├── plus [type=int] │ ├── variable: b:8 [type=int] │ └── variable: c:9 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/upsert b/pkg/sql/opt/memo/testdata/logprops/upsert index 87f6040a1434..63a2fc45bbb4 100644 --- a/pkg/sql/opt/memo/testdata/logprops/upsert +++ b/pkg/sql/opt/memo/testdata/logprops/upsert @@ -129,7 +129,7 @@ project │ │ │ │ │ │ └── projections │ │ │ │ │ │ └── function: unique_rowid [as=column8:8, type=int, volatile, side-effects] │ │ │ │ │ └── projections - │ │ │ │ │ └── plus [as=column9:9, type=int, outer=(6)] + │ │ │ │ │ └── plus [as=column9:9, type=int, outer=(6), immutable] │ │ │ │ │ ├── variable: y:6 [type=int] │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ └── aggregations @@ -158,11 +158,11 @@ project │ │ │ └── variable: c:12 [type=int] │ │ └── projections │ │ ├── const: 1 [as=a_new:14, type=int] - │ │ └── plus [as=b_new:15, type=int, outer=(6,12)] + │ │ └── plus [as=b_new:15, type=int, outer=(6,12), immutable] │ │ ├── variable: y:6 [type=int] │ │ └── variable: c:12 [type=int] │ └── projections - │ └── plus [as=column16:16, type=int, outer=(15)] + │ └── plus [as=column16:16, type=int, outer=(15), immutable] │ ├── variable: b_new:15 [type=int] │ └── const: 1 [type=int] └── projections @@ -340,7 +340,7 @@ project │ │ │ │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ │ │ │ └── function: unique_rowid [as=column8:8, type=int, volatile, side-effects] │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ └── plus [as=column9:9, type=int, outer=(6)] + │ │ │ │ │ │ │ │ │ └── plus [as=column9:9, type=int, outer=(6), immutable] │ │ │ │ │ │ │ │ │ ├── variable: y:6 [type=int] │ │ │ │ │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ │ │ │ │ ├── scan abc @@ -514,7 +514,7 @@ project │ │ │ │ │ │ ├── const: 10 [as=column6:6, type=int] │ │ │ │ │ │ └── function: unique_rowid [as=column7:7, type=int, volatile, side-effects] │ │ │ │ │ └── projections - │ │ │ │ │ └── plus [as=column8:8, type=int, outer=(6)] + │ │ │ │ │ └── plus [as=column8:8, type=int, outer=(6), immutable] │ │ │ │ │ ├── variable: column6:6 [type=int] │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ └── aggregations @@ -541,7 +541,7 @@ project │ │ │ ├── variable: column7:7 [type=int] │ │ │ └── variable: rowid:12 [type=int] │ │ └── projections - │ │ └── plus [as=column13:13, type=int, outer=(10)] + │ │ └── plus [as=column13:13, type=int, outer=(10), immutable] │ │ ├── variable: b:10 [type=int] │ │ └── const: 1 [type=int] │ └── projections @@ -570,7 +570,7 @@ project │ │ └── variable: column7:7 [type=int] │ └── variable: rowid:12 [type=int] └── projections - └── plus [as="?column?":17, type=int, outer=(2,3)] + └── plus [as="?column?":17, type=int, outer=(2,3), immutable] ├── variable: b:2 [type=int] └── variable: c:3 [type=int] @@ -672,7 +672,7 @@ upsert abc │ │ │ │ │ │ ├── const: 10 [as=column8:8, type=int] │ │ │ │ │ │ └── function: unique_rowid [as=column9:9, type=int, volatile, side-effects] │ │ │ │ │ └── projections - │ │ │ │ │ └── plus [as=column10:10, type=int, outer=(8)] + │ │ │ │ │ └── plus [as=column10:10, type=int, outer=(8), immutable] │ │ │ │ │ ├── variable: column8:8 [type=int] │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ └── aggregations @@ -701,7 +701,7 @@ upsert abc │ │ └── projections │ │ └── const: 2 [as=b_new:15, type=int] │ └── projections - │ └── plus [as=column16:16, type=int, outer=(15)] + │ └── plus [as=column16:16, type=int, outer=(15), immutable] │ ├── variable: b_new:15 [type=int] │ └── const: 1 [type=int] └── projections diff --git a/pkg/sql/opt/memo/testdata/logprops/values b/pkg/sql/opt/memo/testdata/logprops/values index f4e4cd115e05..ebf8a5475cc3 100644 --- a/pkg/sql/opt/memo/testdata/logprops/values +++ b/pkg/sql/opt/memo/testdata/logprops/values @@ -64,6 +64,7 @@ SELECT (VALUES (x), (y+1)) FROM xy ---- project ├── columns: column1:4(int) + ├── immutable ├── prune: (4) ├── scan xy │ ├── columns: x:1(int!null) y:2(int) @@ -72,18 +73,20 @@ project │ ├── prune: (1,2) │ └── interesting orderings: (+1) └── projections - └── subquery [as=column1:4, type=int, outer=(1,2), correlated-subquery] + └── subquery [as=column1:4, type=int, outer=(1,2), immutable, correlated-subquery] └── max1-row ├── columns: column1:3(int) ├── error: "more than one row returned by a subquery used as an expression" ├── outer: (1,2) ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(3) └── values ├── columns: column1:3(int) ├── outer: (1,2) ├── cardinality: [2 - 2] + ├── immutable ├── prune: (3) ├── tuple [type=tuple{int}] │ └── variable: x:1 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/window b/pkg/sql/opt/memo/testdata/logprops/window index 7fc1ca157c55..b67e0bb8ed3c 100644 --- a/pkg/sql/opt/memo/testdata/logprops/window +++ b/pkg/sql/opt/memo/testdata/logprops/window @@ -90,6 +90,7 @@ SELECT k, (SELECT rank() OVER () + x FROM (SELECT k AS x)) FROM kv ---- project ├── columns: k:1(int!null) "?column?":11(int) + ├── immutable ├── key: (1) ├── fd: (1)-->(11) ├── prune: (1,11) @@ -101,18 +102,20 @@ project │ ├── prune: (1-7) │ └── interesting orderings: (+1) └── projections - └── subquery [as="?column?":11, type=int, outer=(1), correlated-subquery] + └── subquery [as="?column?":11, type=int, outer=(1), immutable, correlated-subquery] └── max1-row ├── columns: "?column?":10(int) ├── error: "more than one row returned by a subquery used as an expression" ├── outer: (1) ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(10) └── project ├── columns: "?column?":10(int) ├── outer: (1) ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(10) ├── prune: (10) @@ -139,7 +142,7 @@ project │ └── windows │ └── rank [as=rank:9, type=int] └── projections - └── plus [as="?column?":10, type=int, outer=(8,9)] + └── plus [as="?column?":10, type=int, outer=(8,9), immutable] ├── variable: rank:9 [type=int] └── variable: x:8 [type=int] diff --git a/pkg/sql/opt/memo/testdata/memo b/pkg/sql/opt/memo/testdata/memo index 8cf88bf5211c..6be4aa8bcb78 100644 --- a/pkg/sql/opt/memo/testdata/memo +++ b/pkg/sql/opt/memo/testdata/memo @@ -58,7 +58,7 @@ limit │ │ │ └── variable: a.x:1 [type=int] │ │ └── variable: b.x:3 [type=string] │ └── projections - │ └── plus [as=c:5, type=int, outer=(2)] + │ └── plus [as=c:5, type=int, outer=(2), immutable] │ ├── variable: y:2 [type=int] │ └── const: 1 [type=int] └── const: 10 [type=int] @@ -117,7 +117,7 @@ project │ │ └── filters (true) │ └── const: 10 [type=int] └── projections - └── plus [as=c:6, type=int, outer=(2)] + └── plus [as=c:6, type=int, outer=(2), immutable] ├── variable: y:2 [type=int] └── const: 1 [type=int] diff --git a/pkg/sql/opt/memo/testdata/stats/groupby b/pkg/sql/opt/memo/testdata/stats/groupby index 2062dd835eb7..d65bcae3ec7d 100644 --- a/pkg/sql/opt/memo/testdata/stats/groupby +++ b/pkg/sql/opt/memo/testdata/stats/groupby @@ -186,6 +186,7 @@ SELECT sum(x), s FROM a GROUP BY s HAVING sum(x) = 5 ---- select ├── columns: sum:5(decimal!null) s:4(string) + ├── immutable ├── stats: [rows=1, distinct(5)=1, null(5)=0] ├── key: (4) ├── fd: ()-->(5) @@ -209,7 +210,7 @@ select │ └── sum [as=sum:5, type=decimal, outer=(1)] │ └── x:1 [type=int] └── filters - └── sum:5 = 5 [type=bool, outer=(5), constraints=(/5: [/5 - /5]; tight), fd=()-->(5)] + └── sum:5 = 5 [type=bool, outer=(5), immutable, constraints=(/5: [/5 - /5]; tight), fd=()-->(5)] # Scalar GroupBy. build @@ -431,6 +432,7 @@ SELECT sum(x), s FROM a GROUP BY s HAVING sum(x) = 5 ---- select ├── columns: sum:5(decimal!null) s:4(string) + ├── immutable ├── stats: [rows=1, distinct(5)=1, null(5)=0] ├── key: (4) ├── fd: ()-->(5) @@ -454,7 +456,7 @@ select │ └── sum [as=sum:5, type=decimal, outer=(1)] │ └── x:1 [type=int] └── filters - └── sum:5 = 5 [type=bool, outer=(5), constraints=(/5: [/5 - /5]; tight), fd=()-->(5)] + └── sum:5 = 5 [type=bool, outer=(5), immutable, constraints=(/5: [/5 - /5]; tight), fd=()-->(5)] # Regression test for #36442. norm diff --git a/pkg/sql/opt/memo/testdata/stats/index-join b/pkg/sql/opt/memo/testdata/stats/index-join index e4684d34e85d..30baf15b0bfe 100644 --- a/pkg/sql/opt/memo/testdata/stats/index-join +++ b/pkg/sql/opt/memo/testdata/stats/index-join @@ -39,15 +39,18 @@ SELECT count(*) FROM (SELECT * FROM a WHERE s = 'foo' AND x + y = 10) GROUP BY s ---- project ├── columns: count:5(int!null) + ├── immutable ├── stats: [rows=49.2384513] └── group-by ├── columns: y:2(int) count_rows:5(int!null) ├── grouping columns: y:2(int) + ├── immutable ├── stats: [rows=49.2384513, distinct(2)=49.2384513, null(2)=0] ├── key: (2) ├── fd: (2)-->(5) ├── select │ ├── columns: x:1(int!null) y:2(int) s:3(string!null) + │ ├── immutable │ ├── stats: [rows=66.6666667, distinct(2)=49.2384513, null(2)=0, distinct(3)=1, null(3)=0] │ ├── key: (1) │ ├── fd: ()-->(3), (1)-->(2) @@ -63,7 +66,7 @@ project │ │ ├── key: (1) │ │ └── fd: ()-->(3) │ └── filters - │ └── (x:1 + y:2) = 10 [type=bool, outer=(1,2)] + │ └── (x:1 + y:2) = 10 [type=bool, outer=(1,2), immutable] └── aggregations └── count-rows [as=count_rows:5, type=int] @@ -72,6 +75,7 @@ SELECT * FROM a WHERE s = 'foo' AND x + y = 10 ---- select ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) + ├── immutable ├── stats: [rows=66.6666667, distinct(1)=66.6666667, null(1)=0, distinct(2)=49.2384513, null(2)=0, distinct(3)=1, null(3)=0, distinct(4)=57.5057212, null(4)=0, distinct(1-3)=66.6666667, null(1-3)=0] ├── key: (1) ├── fd: ()-->(3), (1)-->(2,4), (4)-->(1,2) @@ -87,7 +91,7 @@ select │ ├── key: (1) │ └── fd: ()-->(3), (1)-->(4), (4)-->(1) └── filters - └── (x:1 + y:2) = 10 [type=bool, outer=(1,2)] + └── (x:1 + y:2) = 10 [type=bool, outer=(1,2), immutable] opt colstat=1 colstat=2 colstat=3 colstat=(1,2,3) SELECT * FROM a WHERE s = 'foo' @@ -180,6 +184,7 @@ SELECT * FROM a WHERE s = 'foo' AND x + y = 10 ---- select ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) + ├── immutable ├── stats: [rows=33.3333333, distinct(1)=33.3333333, null(1)=0, distinct(2)=28.5927601, null(2)=16.6666667, distinct(3)=1, null(3)=0, distinct(4)=30.9412676, null(4)=0, distinct(1-3)=33.3333333, null(1-3)=0] ├── key: (1) ├── fd: ()-->(3), (1)-->(2,4), (4)-->(1,2) @@ -195,7 +200,7 @@ select │ ├── key: (1) │ └── fd: ()-->(3), (1)-->(4), (4)-->(1) └── filters - └── (x:1 + y:2) = 10 [type=bool, outer=(1,2)] + └── (x:1 + y:2) = 10 [type=bool, outer=(1,2), immutable] opt colstat=1 colstat=2 colstat=3 colstat=(1,2,3) SELECT * FROM a WHERE s = 'foo' diff --git a/pkg/sql/opt/memo/testdata/stats/join b/pkg/sql/opt/memo/testdata/stats/join index 79ef7c32c134..6342cc9f89ea 100644 --- a/pkg/sql/opt/memo/testdata/stats/join +++ b/pkg/sql/opt/memo/testdata/stats/join @@ -382,6 +382,7 @@ SELECT * FROM xysd JOIN uv ON x=u AND y+v=5 AND y > 0 AND y < 300 inner-join (hash) ├── columns: x:1(int!null) y:2(int!null) s:3(string) d:4(decimal!null) u:5(int!null) v:6(int!null) ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) + ├── immutable ├── stats: [rows=3333.33333, distinct(1)=500, null(1)=0, distinct(5)=500, null(5)=0] ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(5), (5)==(1) ├── select @@ -401,7 +402,7 @@ inner-join (hash) │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0] └── filters ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── (y:2 + v:6) = 5 [type=bool, outer=(2,6)] + └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] # Force column statistics calculation for semi-join. norm @@ -411,15 +412,18 @@ GROUP BY y ---- project ├── columns: count:8(int!null) + ├── immutable ├── stats: [rows=138.170075] └── group-by ├── columns: y:2(int) count_rows:8(int!null) ├── grouping columns: y:2(int) + ├── immutable ├── stats: [rows=138.170075, distinct(2)=138.170075, null(2)=0] ├── key: (2) ├── fd: (2)-->(8) ├── semi-join (hash) │ ├── columns: x:1(int!null) y:2(int) + │ ├── immutable │ ├── stats: [rows=166.666667, distinct(1)=166.666667, null(1)=0, distinct(2)=138.170075, null(2)=0] │ ├── key: (1) │ ├── fd: (1)-->(2) @@ -433,7 +437,7 @@ project │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0] │ └── filters │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6)] + │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] └── aggregations └── count-rows [as=count_rows:8, type=int] @@ -445,15 +449,18 @@ GROUP BY y ---- project ├── columns: count:8(int!null) + ├── immutable ├── stats: [rows=400] └── group-by ├── columns: y:2(int) count_rows:8(int!null) ├── grouping columns: y:2(int) + ├── immutable ├── stats: [rows=400, distinct(2)=400, null(2)=0] ├── key: (2) ├── fd: (2)-->(8) ├── anti-join (hash) │ ├── columns: x:1(int!null) y:2(int) + │ ├── immutable │ ├── stats: [rows=4833.33333, distinct(2)=400, null(2)=0] │ ├── key: (1) │ ├── fd: (1)-->(2) @@ -467,7 +474,7 @@ project │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0] │ └── filters │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6)] + │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] └── aggregations └── count-rows [as=count_rows:8, type=int] @@ -479,16 +486,19 @@ GROUP BY y ---- project ├── columns: count:8(int!null) + ├── immutable ├── stats: [rows=400] └── group-by ├── columns: y:2(int) count_rows:8(int!null) ├── grouping columns: y:2(int) + ├── immutable ├── stats: [rows=400, distinct(2)=400, null(2)=0] ├── key: (2) ├── fd: (2)-->(8) ├── left-join (hash) │ ├── columns: x:1(int!null) y:2(int) u:5(int) v:6(int) │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) + │ ├── immutable │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0, distinct(5)=500, null(5)=1666.66667] │ ├── fd: (1)-->(2) │ ├── scan xysd @@ -501,7 +511,7 @@ project │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0] │ └── filters │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6)] + │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] └── aggregations └── count-rows [as=count_rows:8, type=int] @@ -513,16 +523,19 @@ GROUP BY y ---- project ├── columns: count:8(int!null) + ├── immutable ├── stats: [rows=399.903879] └── group-by ├── columns: y:2(int) count_rows:8(int!null) ├── grouping columns: y:2(int) + ├── immutable ├── stats: [rows=399.903879, distinct(2)=399.903879, null(2)=1] ├── key: (2) ├── fd: (2)-->(8) ├── left-join (hash) │ ├── columns: x:1(int) y:2(int) u:5(int) v:6(int!null) │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + │ ├── immutable │ ├── stats: [rows=10000, distinct(1)=500, null(1)=6666.66667, distinct(2)=399.903879, null(2)=6666.66667] │ ├── fd: (1)-->(2) │ ├── scan uv @@ -535,7 +548,7 @@ project │ │ └── fd: (1)-->(2) │ └── filters │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6)] + │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] └── aggregations └── count-rows [as=count_rows:8, type=int] @@ -547,16 +560,19 @@ GROUP BY y ---- project ├── columns: count:8(int!null) + ├── immutable ├── stats: [rows=400] └── group-by ├── columns: y:2(int) count_rows:8(int!null) ├── grouping columns: y:2(int) + ├── immutable ├── stats: [rows=400, distinct(2)=400, null(2)=1] ├── key: (2) ├── fd: (2)-->(8) ├── full-join (hash) │ ├── columns: x:1(int) y:2(int) u:5(int) v:6(int) │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) + │ ├── immutable │ ├── stats: [rows=11666.6667, distinct(2)=400, null(2)=6666.66667] │ ├── fd: (1)-->(2) │ ├── scan xysd @@ -569,7 +585,7 @@ project │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0] │ └── filters │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6)] + │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] └── aggregations └── count-rows [as=count_rows:8, type=int] diff --git a/pkg/sql/opt/memo/testdata/stats/project b/pkg/sql/opt/memo/testdata/stats/project index 86009e9264d5..335e42083e69 100644 --- a/pkg/sql/opt/memo/testdata/stats/project +++ b/pkg/sql/opt/memo/testdata/stats/project @@ -135,9 +135,11 @@ SELECT * FROM (SELECT y + 3 AS v FROM a) WHERE v >= 1 AND v <= 100 ---- select ├── columns: v:5(int!null) + ├── immutable ├── stats: [rows=1000, distinct(5)=100, null(5)=0] ├── project │ ├── columns: v:5(int) + │ ├── immutable │ ├── stats: [rows=2000, distinct(5)=200, null(5)=0] │ ├── scan a │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) @@ -145,7 +147,7 @@ select │ │ ├── key: (1) │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── projections - │ └── y:2 + 3 [as=v:5, type=int, outer=(2)] + │ └── y:2 + 3 [as=v:5, type=int, outer=(2), immutable] └── filters └── (v:5 >= 1) AND (v:5 <= 100) [type=bool, outer=(5), constraints=(/5: [/1 - /100]; tight)] diff --git a/pkg/sql/opt/memo/testdata/stats/scan b/pkg/sql/opt/memo/testdata/stats/scan index 3082e5853bf4..9a198543ce12 100644 --- a/pkg/sql/opt/memo/testdata/stats/scan +++ b/pkg/sql/opt/memo/testdata/stats/scan @@ -202,6 +202,7 @@ SELECT * FROM a WHERE ((s >= 'bar' AND s <= 'foo') OR (s >= 'foobar')) AND d > 5 ---- select ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) b:5(bool) + ├── immutable ├── stats: [rows=650, distinct(3)=1, null(3)=0, distinct(4)=650, null(4)=0, distinct(3,4)=650, null(3,4)=0] ├── key: (1) ├── fd: (1)-->(2-5), (3,4)-->(1,2,5) @@ -212,13 +213,14 @@ select │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── filters ├── ((s:3 >= 'bar') AND (s:3 <= 'foo')) OR (s:3 >= 'foobar') [type=bool, outer=(3), constraints=(/3: [/'bar' - /'foo'] [/'foobar' - ]; tight)] - └── d:4 > 5.0 [type=bool, outer=(4), constraints=(/4: (/5.0 - ]; tight)] + └── d:4 > 5.0 [type=bool, outer=(4), immutable, constraints=(/4: (/5.0 - ]; tight)] opt SELECT * FROM a WHERE ((s >= 'bar' AND s <= 'foo') OR (s >= 'foobar')) AND d <= 5.0 AND s IS NOT NULL ---- select ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) b:5(bool) + ├── immutable ├── stats: [rows=650, distinct(3)=1, null(3)=0, distinct(4)=650, null(4)=0, distinct(3,4)=650, null(3,4)=0] ├── key: (1) ├── fd: (1)-->(2-5), (3,4)-->(1,2,5) @@ -229,7 +231,7 @@ select │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── filters ├── (((s:3 >= 'bar') AND (s:3 <= 'foo')) OR (s:3 >= 'foobar')) AND (s:3 IS NOT NULL) [type=bool, outer=(3), constraints=(/3: [/'bar' - /'foo'] [/'foobar' - ]; tight)] - └── d:4 <= 5.0 [type=bool, outer=(4), constraints=(/4: (/NULL - /5.0]; tight)] + └── d:4 <= 5.0 [type=bool, outer=(4), immutable, constraints=(/4: (/NULL - /5.0]; tight)] # Bump up null counts. @@ -359,11 +361,13 @@ SELECT * FROM a WHERE ((s >= 'bar' AND s <= 'foo') OR (s >= 'foobar')) AND d <= ---- index-join a ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) b:5(bool) + ├── immutable ├── stats: [rows=333.333333, distinct(3)=1, null(3)=0, distinct(4)=100, null(4)=0, distinct(3,4)=100, null(3,4)=0] ├── key: (1) ├── fd: (1)-->(2-5), (3,4)-->(1,2,5) └── select ├── columns: x:1(int!null) s:3(string!null) d:4(decimal!null) + ├── immutable ├── stats: [rows=333.333333, distinct(4)=98.265847, null(4)=0] ├── key: (1) ├── fd: (1)-->(3,4), (3,4)-->(1) @@ -376,7 +380,7 @@ index-join a │ ├── key: (1) │ └── fd: (1)-->(3,4), (3,4)-->(1) └── filters - └── d:4 <= 5.0 [type=bool, outer=(4), constraints=(/4: (/NULL - /5.0]; tight)] + └── d:4 <= 5.0 [type=bool, outer=(4), immutable, constraints=(/4: (/NULL - /5.0]; tight)] exec-ddl CREATE TABLE abcde ( @@ -720,6 +724,7 @@ SELECT * FROM hist WHERE c = 20 OR (c < 10) ---- index-join hist ├── columns: a:1(int) b:2(date) c:3(decimal!null) d:4(float) e:5(timestamp) f:6(timestamptz) g:7(string) + ├── immutable ├── stats: [rows=110, distinct(3)=10, null(3)=0] │ histogram(3)= 0 0 90 0 0 20 │ <--- 0 ---- 10 --- 20 @@ -739,6 +744,7 @@ SELECT * FROM hist WHERE c = 20 OR (c <= 10) ---- index-join hist ├── columns: a:1(int) b:2(date) c:3(decimal!null) d:4(float) e:5(timestamp) f:6(timestamptz) g:7(string) + ├── immutable ├── stats: [rows=120, distinct(3)=11, null(3)=0] │ histogram(3)= 0 0 90 10 0 20 │ <--- 0 ---- 10 --- 20 diff --git a/pkg/sql/opt/memo/testdata/stats/select b/pkg/sql/opt/memo/testdata/stats/select index 89c3e6f07eee..dec8eae7aee6 100644 --- a/pkg/sql/opt/memo/testdata/stats/select +++ b/pkg/sql/opt/memo/testdata/stats/select @@ -96,6 +96,7 @@ SELECT * FROM a WHERE x + y < 10 ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── stats: [rows=1333.33333] ├── key: (1) ├── fd: (1)-->(2) @@ -105,7 +106,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2) └── filters - └── (x:1 + y:2) < 10 [type=bool, outer=(1,2)] + └── (x:1 + y:2) < 10 [type=bool, outer=(1,2), immutable] # Remaining filter. norm @@ -538,6 +539,7 @@ SELECT * FROM order_history WHERE item_id = order_id AND customer_id % 2 = 0 ---- select ├── columns: order_id:1(int!null) item_id:2(int!null) customer_id:3(int) year:4(int) + ├── immutable ├── stats: [rows=3.267, distinct(1)=3.267, null(1)=0, distinct(2)=3.267, null(2)=0] ├── fd: (1)==(2), (2)==(1) ├── scan order_history @@ -545,7 +547,7 @@ select │ └── stats: [rows=1000, distinct(1)=100, null(1)=10, distinct(2)=100, null(2)=10] └── filters ├── item_id:2 = order_id:1 [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] - └── (customer_id:3 % 2) = 0 [type=bool, outer=(3)] + └── (customer_id:3 % 2) = 0 [type=bool, outer=(3), immutable] exec-ddl CREATE TABLE c (x INT, z INT NOT NULL, UNIQUE INDEX x_idx (x)) @@ -840,6 +842,7 @@ SELECT * FROM tjson WHERE b @> '{"a":"b"}' ---- index-join tjson ├── columns: a:1(int!null) b:2(jsonb) c:3(jsonb) + ├── immutable ├── stats: [rows=555.555556] ├── key: (1) ├── fd: (1)-->(2,3) @@ -858,6 +861,7 @@ inner-join (lookup tjson) ├── columns: a:1(int!null) b:2(jsonb) c:3(jsonb) ├── key columns: [1] = [1] ├── lookup columns are key + ├── immutable ├── stats: [rows=61.7283951] ├── key: (1) ├── fd: (1)-->(2,3) @@ -869,7 +873,7 @@ inner-join (lookup tjson) │ ├── stats: [rows=61.7283951, distinct(1)=61.7283951, null(1)=0] │ └── filters (true) └── filters - └── b:2 @> '{"a": "b", "c": "d"}' [type=bool, outer=(2)] + └── b:2 @> '{"a": "b", "c": "d"}' [type=bool, outer=(2), immutable] # Should generate a select on the table with a JSON filter, since c does not # have an inverted index. @@ -878,6 +882,7 @@ SELECT * FROM tjson WHERE c @> '{"a":"b"}' ---- select ├── columns: a:1(int!null) b:2(jsonb) c:3(jsonb) + ├── immutable ├── stats: [rows=555.555556] ├── key: (1) ├── fd: (1)-->(2,3) @@ -887,7 +892,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2,3) └── filters - └── c:3 @> '{"a": "b"}' [type=bool, outer=(3)] + └── c:3 @> '{"a": "b"}' [type=bool, outer=(3), immutable] # Should have a lower row count than the above case, due to a containment query # on 2 json paths. @@ -896,6 +901,7 @@ SELECT * FROM tjson WHERE c @> '{"a":"b", "c":"d"}' ---- select ├── columns: a:1(int!null) b:2(jsonb) c:3(jsonb) + ├── immutable ├── stats: [rows=61.7283951] ├── key: (1) ├── fd: (1)-->(2,3) @@ -905,7 +911,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2,3) └── filters - └── c:3 @> '{"a": "b", "c": "d"}' [type=bool, outer=(3)] + └── c:3 @> '{"a": "b", "c": "d"}' [type=bool, outer=(3), immutable] # Bump up null counts. exec-ddl @@ -999,6 +1005,7 @@ SELECT * FROM a WHERE x + y < 10 ---- select ├── columns: x:1(int!null) y:2(int) + ├── immutable ├── stats: [rows=1666.66667] ├── key: (1) ├── fd: (1)-->(2) @@ -1008,7 +1015,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2) └── filters - └── (x:1 + y:2) < 10 [type=bool, outer=(1,2)] + └── (x:1 + y:2) < 10 [type=bool, outer=(1,2), immutable] # Remaining filter. norm diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpcc b/pkg/sql/opt/memo/testdata/stats_quality/tpcc index b3cf06322ced..b80bffa0dc8d 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpcc +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpcc @@ -639,6 +639,7 @@ scalar-group-by ├── save-table-name: consistency_01_scalar_group_by_1 ├── columns: count:22(int!null) ├── cardinality: [1 - 1] + ├── immutable ├── stats: [rows=1, distinct(22)=1, null(22)=0] ├── key: () ├── fd: ()-->(22) @@ -647,6 +648,7 @@ scalar-group-by │ ├── columns: w_id:1(int!null) w_ytd:9(decimal!null) d_w_id:11(int!null) sum:21(decimal!null) │ ├── left ordering: +1 │ ├── right ordering: +11 + │ ├── immutable │ ├── stats: [rows=3.33333333, distinct(1)=3.33333333, null(1)=0, distinct(9)=1, null(9)=0, distinct(11)=3.33333333, null(11)=0, distinct(21)=3.33333333, null(21)=0] │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) @@ -678,7 +680,7 @@ scalar-group-by │ │ └── sum [as=sum:21, type=decimal, outer=(19)] │ │ └── d_ytd:19 [type=decimal] │ └── filters - │ └── w_ytd:9 != sum:21 [type=bool, outer=(9,21), constraints=(/9: (/NULL - ]; /21: (/NULL - ])] + │ └── w_ytd:9 != sum:21 [type=bool, outer=(9,21), immutable, constraints=(/9: (/NULL - ]; /21: (/NULL - ])] └── aggregations └── count-rows [as=count_rows:22, type=int] @@ -819,12 +821,14 @@ scalar-group-by ├── save-table-name: consistency_05_scalar_group_by_1 ├── columns: count:8(int!null) ├── cardinality: [1 - 1] + ├── immutable ├── stats: [rows=1, distinct(8)=1, null(8)=0] ├── key: () ├── fd: ()-->(8) ├── select │ ├── save-table-name: consistency_05_select_2 │ ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) max:4(int!null) min:5(int!null) count_rows:6(int!null) + │ ├── immutable │ ├── stats: [rows=33.3333333, distinct(2)=9.8265847, null(2)=0, distinct(3)=9.8265847, null(3)=0, distinct(4)=33.3333333, null(4)=0, distinct(5)=33.3333333, null(5)=0, distinct(6)=33.3333333, null(6)=0] │ ├── key: (2,3) │ ├── fd: (2,3)-->(4-6) @@ -851,7 +855,7 @@ scalar-group-by │ │ │ └── no_o_id:1 [type=int] │ │ └── count-rows [as=count_rows:6, type=int] │ └── filters - │ └── ((max:4 - min:5) - count_rows:6) != -1 [type=bool, outer=(4-6)] + │ └── ((max:4 - min:5) - count_rows:6) != -1 [type=bool, outer=(4-6), immutable] └── aggregations └── count-rows [as=count_rows:8, type=int] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q01 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q01 index 75e9beb0d2f7..e0173c7581e7 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q01 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q01 @@ -43,6 +43,7 @@ ORDER BY sort ├── save-table-name: q1_sort_1 ├── columns: l_returnflag:9(char!null) l_linestatus:10(char!null) sum_qty:17(float!null) sum_base_price:18(float!null) sum_disc_price:20(float!null) sum_charge:22(float!null) avg_qty:23(float!null) avg_price:24(float!null) avg_disc:25(float!null) count_order:26(int!null) + ├── immutable ├── stats: [rows=6, distinct(9)=3, null(9)=0, distinct(10)=2, null(10)=0, distinct(17)=6, null(17)=0, distinct(18)=6, null(18)=0, distinct(20)=6, null(20)=0, distinct(22)=6, null(22)=0, distinct(23)=6, null(23)=0, distinct(24)=6, null(24)=0, distinct(25)=6, null(25)=0, distinct(26)=6, null(26)=0, distinct(9,10)=6, null(9,10)=0] ├── key: (9,10) ├── fd: (9,10)-->(17,18,20,22-26) @@ -51,12 +52,14 @@ sort ├── save-table-name: q1_group_by_2 ├── columns: l_returnflag:9(char!null) l_linestatus:10(char!null) sum:17(float!null) sum:18(float!null) sum:20(float!null) sum:22(float!null) avg:23(float!null) avg:24(float!null) avg:25(float!null) count_rows:26(int!null) ├── grouping columns: l_returnflag:9(char!null) l_linestatus:10(char!null) + ├── immutable ├── stats: [rows=6, distinct(9)=3, null(9)=0, distinct(10)=2, null(10)=0, distinct(17)=6, null(17)=0, distinct(18)=6, null(18)=0, distinct(20)=6, null(20)=0, distinct(22)=6, null(22)=0, distinct(23)=6, null(23)=0, distinct(24)=6, null(24)=0, distinct(25)=6, null(25)=0, distinct(26)=6, null(26)=0, distinct(9,10)=6, null(9,10)=0] ├── key: (9,10) ├── fd: (9,10)-->(17,18,20,22-26) ├── project │ ├── save-table-name: q1_project_3 │ ├── columns: column19:19(float!null) column21:21(float!null) l_quantity:5(float!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_returnflag:9(char!null) l_linestatus:10(char!null) + │ ├── immutable │ ├── stats: [rows=5925056.21, distinct(5)=50, null(5)=0, distinct(6)=925955, null(6)=0, distinct(7)=11, null(7)=0, distinct(9)=3, null(9)=0, distinct(10)=2, null(10)=0, distinct(19)=5925056.21, null(19)=0, distinct(21)=5925056.21, null(21)=0, distinct(9,10)=6, null(9,10)=0] │ ├── select │ │ ├── save-table-name: q1_select_4 @@ -73,8 +76,8 @@ sort │ │ └── filters │ │ └── l_shipdate:11 <= '1998-09-02' [type=bool, outer=(11), constraints=(/11: (/NULL - /'1998-09-02']; tight)] │ └── projections - │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column19:19, type=float, outer=(6,7)] - │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column21:21, type=float, outer=(6-8)] + │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column19:19, type=float, outer=(6,7), immutable] + │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column21:21, type=float, outer=(6-8), immutable] └── aggregations ├── sum [as=sum:17, type=float, outer=(5)] │ └── l_quantity:5 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q03 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q03 index c5cabe216e39..a47e3af03dd5 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q03 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q03 @@ -45,6 +45,7 @@ limit ├── columns: l_orderkey:18(int!null) revenue:35(float!null) o_orderdate:13(date!null) o_shippriority:16(int!null) ├── internal-ordering: -35,+13 ├── cardinality: [0 - 10] + ├── immutable ├── stats: [rows=10, distinct(13)=10, null(13)=0, distinct(16)=10, null(16)=0, distinct(18)=10, null(18)=0, distinct(35)=10, null(35)=0] ├── key: (18) ├── fd: (18)-->(13,16,35) @@ -52,6 +53,7 @@ limit ├── sort │ ├── save-table-name: q3_sort_2 │ ├── columns: o_orderdate:13(date!null) o_shippriority:16(int!null) l_orderkey:18(int!null) sum:35(float!null) + │ ├── immutable │ ├── stats: [rows=359560.406, distinct(13)=359560.406, null(13)=0, distinct(16)=359560.406, null(16)=0, distinct(18)=359560.406, null(18)=0, distinct(35)=359560.406, null(35)=0] │ ├── key: (18) │ ├── fd: (18)-->(13,16,35) @@ -61,12 +63,14 @@ limit │ ├── save-table-name: q3_group_by_3 │ ├── columns: o_orderdate:13(date!null) o_shippriority:16(int!null) l_orderkey:18(int!null) sum:35(float!null) │ ├── grouping columns: l_orderkey:18(int!null) + │ ├── immutable │ ├── stats: [rows=359560.406, distinct(13)=359560.406, null(13)=0, distinct(16)=359560.406, null(16)=0, distinct(18)=359560.406, null(18)=0, distinct(35)=359560.406, null(35)=0] │ ├── key: (18) │ ├── fd: (18)-->(13,16,35) │ ├── project │ │ ├── save-table-name: q3_project_4 │ │ ├── columns: column34:34(float!null) o_orderdate:13(date!null) o_shippriority:16(int!null) l_orderkey:18(int!null) + │ │ ├── immutable │ │ ├── stats: [rows=493779.215, distinct(13)=1169, null(13)=0, distinct(16)=1, null(16)=0, distinct(18)=359560.406, null(18)=0, distinct(34)=410295.908, null(34)=0] │ │ ├── fd: (18)-->(13,16) │ │ ├── inner-join (lookup lineitem) @@ -125,7 +129,7 @@ limit │ │ │ └── filters │ │ │ └── l_shipdate:28 > '1995-03-15' [type=bool, outer=(28), constraints=(/28: [/'1995-03-16' - ]; tight)] │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column34:34, type=float, outer=(23,24)] + │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column34:34, type=float, outer=(23,24), immutable] │ └── aggregations │ ├── sum [as=sum:35, type=float, outer=(34)] │ │ └── column34:34 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q05 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q05 index f5d097cd57a0..7a315fe43b2d 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q05 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q05 @@ -50,6 +50,7 @@ ORDER BY sort ├── save-table-name: q5_sort_1 ├── columns: n_name:42(char!null) revenue:49(float!null) + ├── immutable ├── stats: [rows=5, distinct(42)=5, null(42)=0, distinct(49)=5, null(49)=0] ├── key: (42) ├── fd: (42)-->(49) @@ -58,12 +59,14 @@ sort ├── save-table-name: q5_group_by_2 ├── columns: n_name:42(char!null) sum:49(float!null) ├── grouping columns: n_name:42(char!null) + ├── immutable ├── stats: [rows=5, distinct(42)=5, null(42)=0, distinct(49)=5, null(49)=0] ├── key: (42) ├── fd: (42)-->(49) ├── project │ ├── save-table-name: q5_project_3 │ ├── columns: column48:48(float!null) n_name:42(char!null) + │ ├── immutable │ ├── stats: [rows=13445.4933, distinct(42)=5, null(42)=0, distinct(48)=13135.9517, null(48)=0] │ ├── inner-join (hash) │ │ ├── save-table-name: q5_inner_join_4 @@ -171,7 +174,7 @@ sort │ │ ├── c_custkey:1 = o_custkey:10 [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ │ └── c_nationkey:4 = s_nationkey:37 [type=bool, outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ]), fd=(4)==(37), (37)==(4)] │ └── projections - │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column48:48, type=float, outer=(23,24)] + │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column48:48, type=float, outer=(23,24), immutable] └── aggregations └── sum [as=sum:49, type=float, outer=(48)] └── column48:48 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q06 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q06 index 52ce7daf007e..e64711c89f24 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q06 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q06 @@ -34,12 +34,14 @@ scalar-group-by ├── save-table-name: q6_scalar_group_by_1 ├── columns: revenue:18(float) ├── cardinality: [1 - 1] + ├── immutable ├── stats: [rows=1, distinct(18)=1, null(18)=0] ├── key: () ├── fd: ()-->(18) ├── project │ ├── save-table-name: q6_project_2 │ ├── columns: column17:17(float!null) + │ ├── immutable │ ├── stats: [rows=34745.8339, distinct(17)=34745.8339, null(17)=0] │ ├── select │ │ ├── save-table-name: q6_select_3 @@ -66,7 +68,7 @@ scalar-group-by │ │ ├── (l_discount:7 >= 0.05) AND (l_discount:7 <= 0.07) [type=bool, outer=(7), constraints=(/7: [/0.05 - /0.07]; tight)] │ │ └── l_quantity:5 < 24.0 [type=bool, outer=(5), constraints=(/5: (/NULL - /23.999999999999996]; tight)] │ └── projections - │ └── l_extendedprice:6 * l_discount:7 [as=column17:17, type=float, outer=(6,7)] + │ └── l_extendedprice:6 * l_discount:7 [as=column17:17, type=float, outer=(6,7), immutable] └── aggregations └── sum [as=sum:18, type=float, outer=(17)] └── column17:17 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q07 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q07 index 64f7627df731..7e096bb15c1a 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q07 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q07 @@ -149,7 +149,7 @@ sort │ │ └── s_nationkey:4 = n1.n_nationkey:41 [type=bool, outer=(4,41), constraints=(/4: (/NULL - ]; /41: (/NULL - ]), fd=(4)==(41), (41)==(4)] │ └── projections │ ├── extract('year', l_shipdate:18) [as=l_year:49, type=float, outer=(18), immutable] - │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=volume:50, type=float, outer=(13,14)] + │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=volume:50, type=float, outer=(13,14), immutable] └── aggregations └── sum [as=sum:51, type=float, outer=(50)] └── volume:50 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q08 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q08 index d4c237d2939e..910a82ed9f21 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q08 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q08 @@ -246,7 +246,7 @@ sort │ │ │ │ └── p_partkey:1 = l_partkey:18 [type=bool, outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ]), fd=(1)==(18), (18)==(1)] │ │ │ └── projections │ │ │ ├── extract('year', o_orderdate:37) [as=o_year:61, type=float, outer=(37), immutable] - │ │ │ └── l_extendedprice:22 * (1.0 - l_discount:23) [as=volume:62, type=float, outer=(22,23)] + │ │ │ └── l_extendedprice:22 * (1.0 - l_discount:23) [as=volume:62, type=float, outer=(22,23), immutable] │ │ └── projections │ │ └── CASE WHEN n2.n_name:55 = 'BRAZIL' THEN volume:62 ELSE 0.0 END [as=column63:63, type=float, outer=(55,62)] │ └── aggregations diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q09 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q09 index a9221e0136c2..7237c589ceb7 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q09 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q09 @@ -159,7 +159,7 @@ sort │ │ └── p_name:2 LIKE '%green%' [type=bool, outer=(2), constraints=(/2: (/NULL - ])] │ └── projections │ ├── extract('year', o_orderdate:42) [as=o_year:51, type=float, outer=(42), immutable] - │ └── (l_extendedprice:22 * (1.0 - l_discount:23)) - (ps_supplycost:36 * l_quantity:21) [as=amount:52, type=float, outer=(21-23,36)] + │ └── (l_extendedprice:22 * (1.0 - l_discount:23)) - (ps_supplycost:36 * l_quantity:21) [as=amount:52, type=float, outer=(21-23,36), immutable] └── aggregations └── sum [as=sum:53, type=float, outer=(52)] └── amount:52 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q10 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q10 index 9d7b50fb3b8e..99a705233001 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q10 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q10 @@ -57,6 +57,7 @@ limit ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) revenue:39(float!null) c_acctbal:6(float!null) n_name:35(char!null) c_address:3(varchar!null) c_phone:5(char!null) c_comment:8(varchar!null) ├── internal-ordering: -39 ├── cardinality: [0 - 20] + ├── immutable ├── stats: [rows=20, distinct(1)=20, null(1)=0, distinct(2)=20, null(2)=0, distinct(3)=20, null(3)=0, distinct(5)=20, null(5)=0, distinct(6)=20, null(6)=0, distinct(8)=20, null(8)=0, distinct(35)=20, null(35)=0, distinct(39)=20, null(39)=0] ├── key: (1) ├── fd: (1)-->(2,3,5,6,8,35,39) @@ -64,6 +65,7 @@ limit ├── sort │ ├── save-table-name: q10_sort_2 │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) n_name:35(char!null) sum:39(float!null) + │ ├── immutable │ ├── stats: [rows=42917.9526, distinct(1)=42917.9526, null(1)=0, distinct(2)=42917.9526, null(2)=0, distinct(3)=42917.9526, null(3)=0, distinct(5)=42917.9526, null(5)=0, distinct(6)=42917.9526, null(6)=0, distinct(8)=42917.9526, null(8)=0, distinct(35)=42917.9526, null(35)=0, distinct(39)=42917.9526, null(39)=0] │ ├── key: (1) │ ├── fd: (1)-->(2,3,5,6,8,35,39) @@ -73,12 +75,14 @@ limit │ ├── save-table-name: q10_group_by_3 │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) n_name:35(char!null) sum:39(float!null) │ ├── grouping columns: c_custkey:1(int!null) + │ ├── immutable │ ├── stats: [rows=42917.9526, distinct(1)=42917.9526, null(1)=0, distinct(2)=42917.9526, null(2)=0, distinct(3)=42917.9526, null(3)=0, distinct(5)=42917.9526, null(5)=0, distinct(6)=42917.9526, null(6)=0, distinct(8)=42917.9526, null(8)=0, distinct(35)=42917.9526, null(35)=0, distinct(39)=42917.9526, null(39)=0] │ ├── key: (1) │ ├── fd: (1)-->(2,3,5,6,8,35,39) │ ├── project │ │ ├── save-table-name: q10_project_4 │ │ ├── columns: column38:38(float!null) c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) n_name:35(char!null) + │ │ ├── immutable │ │ ├── stats: [rows=91240.8317, distinct(1)=42917.9526, null(1)=0, distinct(2)=68356.4353, null(2)=0, distinct(3)=68348.5807, null(3)=0, distinct(5)=68356.4353, null(5)=0, distinct(6)=67126.327, null(6)=0, distinct(8)=68271.7501, null(8)=0, distinct(35)=25, null(35)=0, distinct(38)=88236.775, null(38)=0] │ │ ├── fd: (1)-->(2,3,5,6,8,35) │ │ ├── inner-join (hash) @@ -143,7 +147,7 @@ limit │ │ │ └── filters │ │ │ └── c_nationkey:4 = n_nationkey:34 [type=bool, outer=(4,34), constraints=(/4: (/NULL - ]; /34: (/NULL - ]), fd=(4)==(34), (34)==(4)] │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column38:38, type=float, outer=(23,24)] + │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column38:38, type=float, outer=(23,24), immutable] │ └── aggregations │ ├── sum [as=sum:39, type=float, outer=(38)] │ │ └── column38:38 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q11 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q11 index a11abe0e488b..81260c52ed6c 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q11 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q11 @@ -188,7 +188,7 @@ sort │ └── sum [as=sum:36, type=float, outer=(35)] │ └── column35:35 [type=float] └── projections - └── sum:36 * 0.0001 [as="?column?":37, type=float, outer=(36)] + └── sum:36 * 0.0001 [as="?column?":37, type=float, outer=(36), immutable] stats table=q11_sort_1 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q14 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q14 index 3a2ab8e835e8..d7ed576ff4c6 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q14 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q14 @@ -42,12 +42,14 @@ project │ ├── save-table-name: q14_scalar_group_by_2 │ ├── columns: sum:27(float) sum:29(float) │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── stats: [rows=1, distinct(27)=1, null(27)=0, distinct(29)=1, null(29)=0, distinct(27,29)=1, null(27,29)=0] │ ├── key: () │ ├── fd: ()-->(27,29) │ ├── project │ │ ├── save-table-name: q14_project_3 │ │ ├── columns: column26:26(float!null) column28:28(float!null) + │ │ ├── immutable │ │ ├── stats: [rows=82726.8788, distinct(26)=82726.8788, null(26)=0, distinct(28)=52210.2591, null(28)=0] │ │ ├── inner-join (hash) │ │ │ ├── save-table-name: q14_inner_join_4 @@ -83,8 +85,8 @@ project │ │ │ └── filters │ │ │ └── l_partkey:2 = p_partkey:17 [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] │ │ └── projections - │ │ ├── CASE WHEN p_type:21 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column26:26, type=float, outer=(6,7,21)] - │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, type=float, outer=(6,7)] + │ │ ├── CASE WHEN p_type:21 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column26:26, type=float, outer=(6,7,21), immutable] + │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, type=float, outer=(6,7), immutable] │ └── aggregations │ ├── sum [as=sum:27, type=float, outer=(26)] │ │ └── column26:26 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q15 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q15 index e068989ff7dc..e18e4f141b92 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q15 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q15 @@ -53,6 +53,7 @@ ORDER BY project ├── save-table-name: q15_project_1 ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_phone:5(char!null) total_revenue:25(float!null) + ├── immutable ├── stats: [rows=3333.33333, distinct(1)=3306.66667, null(1)=0, distinct(2)=2834.3606, null(2)=0, distinct(3)=2834.80729, null(3)=0, distinct(5)=2834.80729, null(5)=0, distinct(25)=2100.04396, null(25)=0] ├── key: (1) ├── fd: (1)-->(2,3,5,25) @@ -62,6 +63,7 @@ project ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_phone:5(char!null) l_suppkey:10(int!null) sum:25(float!null) ├── left ordering: +1 ├── right ordering: +10 + ├── immutable ├── stats: [rows=3333.33333, distinct(1)=3306.66667, null(1)=0, distinct(2)=2834.3606, null(2)=0, distinct(3)=2834.80729, null(3)=0, distinct(5)=2834.80729, null(5)=0, distinct(10)=3306.66667, null(10)=0, distinct(25)=2100.04396, null(25)=0] ├── key: (10) ├── fd: (1)-->(2,3,5), (10)-->(25), (1)==(10), (10)==(1) @@ -78,6 +80,7 @@ project ├── sort │ ├── save-table-name: q15_sort_4 │ ├── columns: l_suppkey:10(int!null) sum:25(float!null) + │ ├── immutable │ ├── stats: [rows=3306.66667, distinct(10)=3306.66667, null(10)=0, distinct(25)=3306.66667, null(25)=0] │ ├── key: (10) │ ├── fd: (10)-->(25) @@ -85,6 +88,7 @@ project │ └── select │ ├── save-table-name: q15_select_5 │ ├── columns: l_suppkey:10(int!null) sum:25(float!null) + │ ├── immutable │ ├── stats: [rows=3306.66667, distinct(10)=3306.66667, null(10)=0, distinct(25)=3306.66667, null(25)=0] │ ├── key: (10) │ ├── fd: (10)-->(25) @@ -92,12 +96,14 @@ project │ │ ├── save-table-name: q15_group_by_6 │ │ ├── columns: l_suppkey:10(int!null) sum:25(float!null) │ │ ├── grouping columns: l_suppkey:10(int!null) + │ │ ├── immutable │ │ ├── stats: [rows=9920, distinct(10)=9920, null(10)=0, distinct(25)=9920, null(25)=0] │ │ ├── key: (10) │ │ ├── fd: (10)-->(25) │ │ ├── project │ │ │ ├── save-table-name: q15_project_7 │ │ │ ├── columns: column24:24(float!null) l_suppkey:10(int!null) + │ │ │ ├── immutable │ │ │ ├── stats: [rows=259635.063, distinct(10)=9920, null(10)=0, distinct(24)=259635.063, null(24)=0] │ │ │ ├── index-join lineitem │ │ │ │ ├── save-table-name: q15_index_join_8 @@ -117,18 +123,19 @@ project │ │ │ │ ├── key: (8,11) │ │ │ │ └── fd: (8,11)-->(18) │ │ │ └── projections - │ │ │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=column24:24, type=float, outer=(13,14)] + │ │ │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=column24:24, type=float, outer=(13,14), immutable] │ │ └── aggregations │ │ └── sum [as=sum:25, type=float, outer=(24)] │ │ └── column24:24 [type=float] │ └── filters - │ └── eq [type=bool, outer=(25), subquery, constraints=(/25: (/NULL - ])] + │ └── eq [type=bool, outer=(25), immutable, subquery, constraints=(/25: (/NULL - ])] │ ├── sum:25 [type=float] │ └── subquery [type=float] │ └── scalar-group-by │ ├── save-table-name: q15_scalar_group_by_10 │ ├── columns: max:44(float) │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── stats: [rows=1, distinct(44)=1, null(44)=0] │ ├── key: () │ ├── fd: ()-->(44) @@ -136,12 +143,14 @@ project │ │ ├── save-table-name: q15_group_by_11 │ │ ├── columns: l_suppkey:28(int!null) sum:43(float!null) │ │ ├── grouping columns: l_suppkey:28(int!null) + │ │ ├── immutable │ │ ├── stats: [rows=9920, distinct(28)=9920, null(28)=0, distinct(43)=9920, null(43)=0] │ │ ├── key: (28) │ │ ├── fd: (28)-->(43) │ │ ├── project │ │ │ ├── save-table-name: q15_project_12 │ │ │ ├── columns: column42:42(float!null) l_suppkey:28(int!null) + │ │ │ ├── immutable │ │ │ ├── stats: [rows=259635.063, distinct(28)=9920, null(28)=0, distinct(42)=259635.063, null(42)=0] │ │ │ ├── index-join lineitem │ │ │ │ ├── save-table-name: q15_index_join_13 @@ -161,7 +170,7 @@ project │ │ │ │ ├── key: (26,29) │ │ │ │ └── fd: (26,29)-->(36) │ │ │ └── projections - │ │ │ └── l_extendedprice:31 * (1.0 - l_discount:32) [as=column42:42, type=float, outer=(31,32)] + │ │ │ └── l_extendedprice:31 * (1.0 - l_discount:32) [as=column42:42, type=float, outer=(31,32), immutable] │ │ └── aggregations │ │ └── sum [as=sum:43, type=float, outer=(42)] │ │ └── column42:42 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q17 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q17 index c1bda172c749..4d46d9034226 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q17 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q17 @@ -43,6 +43,7 @@ project ├── save-table-name: q17_project_1 ├── columns: avg_yearly:45(float) ├── cardinality: [1 - 1] + ├── immutable ├── stats: [rows=1, distinct(45)=1, null(45)=0] ├── key: () ├── fd: ()-->(45) @@ -50,6 +51,7 @@ project │ ├── save-table-name: q17_scalar_group_by_2 │ ├── columns: sum:44(float) │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── stats: [rows=1, distinct(44)=1, null(44)=0] │ ├── key: () │ ├── fd: ()-->(44) @@ -58,18 +60,21 @@ project │ │ ├── columns: l_partkey:2(int!null) l_quantity:5(float!null) l_extendedprice:6(float!null) p_partkey:17(int!null) "?column?":43(float!null) │ │ ├── key columns: [1 4] = [1 4] │ │ ├── lookup columns are key + │ │ ├── immutable │ │ ├── stats: [rows=2008.02163, distinct(2)=199.999619, null(2)=0, distinct(5)=50, null(5)=0, distinct(6)=2005.84759, null(6)=0, distinct(17)=199.999619, null(17)=0, distinct(43)=199.999619, null(43)=0] │ │ ├── fd: (17)-->(43), (2)==(17), (17)==(2) │ │ ├── inner-join (lookup lineitem@l_pk) │ │ │ ├── save-table-name: q17_lookup_join_4 │ │ │ ├── columns: l_orderkey:1(int!null) l_partkey:2(int!null) l_linenumber:4(int!null) p_partkey:17(int!null) "?column?":43(float) │ │ │ ├── key columns: [17] = [2] + │ │ │ ├── immutable │ │ │ ├── stats: [rows=6024.06489, distinct(1)=6012.21509, null(1)=0, distinct(2)=199.999619, null(2)=0, distinct(4)=7, null(4)=0, distinct(17)=199.999619, null(17)=0, distinct(43)=199.999619, null(43)=0] │ │ │ ├── key: (1,4) │ │ │ ├── fd: (17)-->(43), (1,4)-->(2), (2)==(17), (17)==(2) │ │ │ ├── project │ │ │ │ ├── save-table-name: q17_project_5 │ │ │ │ ├── columns: "?column?":43(float) p_partkey:17(int!null) + │ │ │ │ ├── immutable │ │ │ │ ├── stats: [rows=199.999619, distinct(17)=199.999619, null(17)=0, distinct(43)=199.999619, null(43)=0] │ │ │ │ ├── key: (17) │ │ │ │ ├── fd: (17)-->(43) @@ -122,7 +127,7 @@ project │ │ │ │ │ └── avg [as=avg:42, type=float, outer=(30)] │ │ │ │ │ └── l_quantity:30 [type=float] │ │ │ │ └── projections - │ │ │ │ └── avg:42 * 0.2 [as="?column?":43, type=float, outer=(42)] + │ │ │ │ └── avg:42 * 0.2 [as="?column?":43, type=float, outer=(42), immutable] │ │ │ └── filters (true) │ │ └── filters │ │ └── l_quantity:5 < "?column?":43 [type=bool, outer=(5,43), constraints=(/5: (/NULL - ]; /43: (/NULL - ])] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q19 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q19 index 8d019234afb4..479cc01ca175 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q19 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q19 @@ -57,12 +57,14 @@ scalar-group-by ├── save-table-name: q19_scalar_group_by_1 ├── columns: revenue:27(float) ├── cardinality: [1 - 1] + ├── immutable ├── stats: [rows=1, distinct(27)=1, null(27)=0] ├── key: () ├── fd: ()-->(27) ├── project │ ├── save-table-name: q19_project_2 │ ├── columns: column26:26(float!null) + │ ├── immutable │ ├── stats: [rows=71.4087386, distinct(26)=71.402791, null(26)=0] │ ├── inner-join (hash) │ │ ├── save-table-name: q19_inner_join_3 @@ -104,7 +106,7 @@ scalar-group-by │ │ ├── p_partkey:17 = l_partkey:2 [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] │ │ └── ((((((p_brand:20 = 'Brand#12') AND (p_container:23 IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity:5 >= 1.0)) AND (l_quantity:5 <= 11.0)) AND (p_size:22 <= 5)) OR (((((p_brand:20 = 'Brand#23') AND (p_container:23 IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity:5 >= 10.0)) AND (l_quantity:5 <= 20.0)) AND (p_size:22 <= 10))) OR (((((p_brand:20 = 'Brand#34') AND (p_container:23 IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity:5 >= 20.0)) AND (l_quantity:5 <= 30.0)) AND (p_size:22 <= 15)) [type=bool, outer=(5,20,22,23), constraints=(/5: [/1.0 - /30.0]; /20: [/'Brand#12' - /'Brand#12'] [/'Brand#23' - /'Brand#23'] [/'Brand#34' - /'Brand#34']; /22: (/NULL - /15]; /23: [/'LG BOX' - /'LG BOX'] [/'LG CASE' - /'LG CASE'] [/'LG PACK' - /'LG PACK'] [/'LG PKG' - /'LG PKG'] [/'MED BAG' - /'MED BAG'] [/'MED BOX' - /'MED BOX'] [/'MED PACK' - /'MED PACK'] [/'MED PKG' - /'MED PKG'] [/'SM BOX' - /'SM BOX'] [/'SM CASE' - /'SM CASE'] [/'SM PACK' - /'SM PACK'] [/'SM PKG' - /'SM PKG'])] │ └── projections - │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column26:26, type=float, outer=(6,7)] + │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column26:26, type=float, outer=(6,7), immutable] └── aggregations └── sum [as=sum:27, type=float, outer=(26)] └── column26:26 [type=float] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q20 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q20 index 9344c49d8598..e27e2b35670f 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q20 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q20 @@ -60,22 +60,26 @@ ORDER BY sort ├── save-table-name: q20_sort_1 ├── columns: s_name:2(char!null) s_address:3(varchar!null) + ├── immutable ├── stats: [rows=392.749612, distinct(2)=392.742232, null(2)=0, distinct(3)=392.749612, null(3)=0] ├── ordering: +2 └── project ├── save-table-name: q20_project_2 ├── columns: s_name:2(char!null) s_address:3(varchar!null) + ├── immutable ├── stats: [rows=392.749612, distinct(2)=392.742232, null(2)=0, distinct(3)=392.749612, null(3)=0] └── inner-join (hash) ├── save-table-name: q20_inner_join_3 ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_nationkey:4(int!null) n_nationkey:8(int!null) n_name:9(char!null) ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + ├── immutable ├── stats: [rows=392.749612, distinct(1)=392.685411, null(1)=0, distinct(2)=392.742232, null(2)=0, distinct(3)=392.749612, null(3)=0, distinct(4)=1, null(4)=0, distinct(8)=1, null(8)=0, distinct(9)=1, null(9)=0] ├── key: (1) ├── fd: ()-->(9), (1)-->(2-4), (4)==(8), (8)==(4) ├── semi-join (hash) │ ├── save-table-name: q20_semi_join_4 │ ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_nationkey:4(int!null) + │ ├── immutable │ ├── stats: [rows=9818.7403, distinct(1)=9740.19038, null(1)=0, distinct(2)=9809.64703, null(2)=0, distinct(3)=9818.7403, null(3)=0, distinct(4)=25, null(4)=0] │ ├── key: (1) │ ├── fd: (1)-->(2-4) @@ -92,11 +96,13 @@ sort │ ├── project │ │ ├── save-table-name: q20_project_6 │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) + │ │ ├── immutable │ │ ├── stats: [rows=36952.1991, distinct(12)=22217.3354, null(12)=0, distinct(13)=9740.19038, null(13)=0] │ │ ├── key: (12,13) │ │ └── project │ │ ├── save-table-name: q20_project_7 │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) p_partkey:17(int!null) + │ │ ├── immutable │ │ ├── stats: [rows=36960.327, distinct(12)=22217.3354, null(12)=0, distinct(13)=9681.00153, null(13)=0, distinct(17)=22217.3354, null(17)=0] │ │ ├── key: (13,17) │ │ ├── fd: (12)==(17), (17)==(12) @@ -104,12 +110,14 @@ sort │ │ ├── save-table-name: q20_inner_join_8 │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) ps_availqty:14(int!null) p_partkey:17(int!null) p_name:18(varchar!null) sum:42(float) │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ ├── immutable │ │ ├── stats: [rows=36960.327, distinct(12)=22217.3354, null(12)=0, distinct(13)=9681.00153, null(13)=0, distinct(14)=34508.432, null(14)=0, distinct(17)=22217.3354, null(17)=0, distinct(18)=17907.1379, null(18)=0, distinct(42)=34508.432, null(42)=0] │ │ ├── key: (13,17) │ │ ├── fd: (12,13)-->(14,42), (17)-->(18), (12)==(17), (17)==(12) │ │ ├── select │ │ │ ├── save-table-name: q20_select_9 │ │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) ps_availqty:14(int!null) sum:42(float) + │ │ │ ├── immutable │ │ │ ├── stats: [rows=266100.667, distinct(12)=159991.77, null(12)=0, distinct(13)=9920, null(13)=0, distinct(14)=266100.667, null(14)=0, distinct(42)=266100.667, null(42)=0] │ │ │ ├── key: (12,13) │ │ │ ├── fd: (12,13)-->(14,42) @@ -161,7 +169,7 @@ sort │ │ │ │ └── const-agg [as=ps_availqty:14, type=int, outer=(14)] │ │ │ │ └── ps_availqty:14 [type=int] │ │ │ └── filters - │ │ │ └── ps_availqty:14 > (sum:42 * 0.5) [type=bool, outer=(14,42), constraints=(/14: (/NULL - ])] + │ │ │ └── ps_availqty:14 > (sum:42 * 0.5) [type=bool, outer=(14,42), immutable, constraints=(/14: (/NULL - ])] │ │ ├── select │ │ │ ├── save-table-name: q20_select_15 │ │ │ ├── columns: p_partkey:17(int!null) p_name:18(varchar!null) diff --git a/pkg/sql/opt/norm/testdata/rules/bool b/pkg/sql/opt/norm/testdata/rules/bool index 9f2e763947f1..cd353bfcc6aa 100644 --- a/pkg/sql/opt/norm/testdata/rules/bool +++ b/pkg/sql/opt/norm/testdata/rules/bool @@ -386,6 +386,7 @@ SELECT * FROM a WHERE NOT(s ~ 'foo') AND NOT(s !~ 'foo') AND NOT(s ~* 'foo') AND ---- select ├── columns: k:1!null i:2 f:3 s:4!null j:5 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-5) ├── scan a @@ -393,10 +394,10 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters - ├── s:4 !~ 'foo' [outer=(4), constraints=(/4: (/NULL - ])] - ├── s:4 ~ 'foo' [outer=(4), constraints=(/4: (/NULL - ])] - ├── s:4 !~* 'foo' [outer=(4), constraints=(/4: (/NULL - ])] - └── s:4 ~* 'foo' [outer=(4), constraints=(/4: (/NULL - ])] + ├── s:4 !~ 'foo' [outer=(4), immutable, constraints=(/4: (/NULL - ])] + ├── s:4 ~ 'foo' [outer=(4), immutable, constraints=(/4: (/NULL - ])] + ├── s:4 !~* 'foo' [outer=(4), immutable, constraints=(/4: (/NULL - ])] + └── s:4 ~* 'foo' [outer=(4), immutable, constraints=(/4: (/NULL - ])] norm expect-not=NegateComparison SELECT * FROM a WHERE @@ -408,6 +409,7 @@ SELECT * FROM a WHERE ---- select ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-5) ├── scan a @@ -415,12 +417,12 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters - ├── NOT ('[1, 2]' @> j:5) [outer=(5)] - ├── NOT ('[3, 4]' @> j:5) [outer=(5)] - ├── NOT (j:5 ? 'foo') [outer=(5)] - ├── NOT (j:5 ?| ARRAY['foo']) [outer=(5)] - ├── NOT (j:5 ?& ARRAY['foo']) [outer=(5)] - └── NOT (ARRAY[i:2] && ARRAY[1]) [outer=(2)] + ├── NOT ('[1, 2]' @> j:5) [outer=(5), immutable] + ├── NOT ('[3, 4]' @> j:5) [outer=(5), immutable] + ├── NOT (j:5 ? 'foo') [outer=(5), immutable] + ├── NOT (j:5 ?| ARRAY['foo']) [outer=(5), immutable] + ├── NOT (j:5 ?& ARRAY['foo']) [outer=(5), immutable] + └── NOT (ARRAY[i:2] && ARRAY[1]) [outer=(2), immutable] # -------------------------------------------------- # EliminateNot @@ -476,6 +478,7 @@ SELECT * FROM a WHERE NOT (k >= i OR i < f OR k + i < f) ---- select ├── columns: k:1!null i:2!null f:3!null s:4 j:5 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-5) ├── scan a @@ -485,7 +488,7 @@ select └── filters ├── k:1 < i:2 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ])] ├── i:2 >= f:3 [outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ])] - └── f:3 <= (k:1 + i:2) [outer=(1-3), constraints=(/3: (/NULL - ])] + └── f:3 <= (k:1 + i:2) [outer=(1-3), immutable, constraints=(/3: (/NULL - ])] norm expect=(NegateOr,NegateComparison) SELECT * FROM a WHERE NOT (k >= i OR i < f OR (i > 10 OR i < 5 OR f > 1)) diff --git a/pkg/sql/opt/norm/testdata/rules/combo b/pkg/sql/opt/norm/testdata/rules/combo index c4e4d7d80482..1c04865b0adf 100644 --- a/pkg/sql/opt/norm/testdata/rules/combo +++ b/pkg/sql/opt/norm/testdata/rules/combo @@ -25,8 +25,10 @@ Initial expression ================================================================================ project ├── columns: s:4 + ├── immutable └── inner-join (cross) ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 + ├── immutable ├── key: (1,6) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)-->(7) ├── scan a @@ -38,16 +40,18 @@ Initial expression │ ├── key: (6) │ └── fd: (6)-->(7) └── filters - └── (k:1 = x:6) AND ((i:2 + 1) = 10) [outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── (k:1 = x:6) AND ((i:2 + 1) = 10) [outer=(1,2,6), immutable, constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ================================================================================ NormalizeCmpPlusConst Cost: 15470.07 ================================================================================ project ├── columns: s:4 + ├── immutable └── inner-join (cross) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 + ├── immutable ├── key: (1,6) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)-->(7) ├── scan a @@ -59,16 +63,18 @@ NormalizeCmpPlusConst │ ├── key: (6) │ └── fd: (6)-->(7) └── filters - - └── (k:1 = x:6) AND ((i:2 + 1) = 10) [outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - + └── (k:1 = x:6) AND (i:2 = (10 - 1)) [outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] + - └── (k:1 = x:6) AND ((i:2 + 1) = 10) [outer=(1,2,6), immutable, constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + + └── (k:1 = x:6) AND (i:2 = (10 - 1)) [outer=(1,2,6), immutable, constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] ================================================================================ FoldBinary Cost: 12203.40 ================================================================================ project ├── columns: s:4 + - ├── immutable └── inner-join (cross) ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 + - ├── immutable ├── key: (1,6) - ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)-->(7) + ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5), (6)-->(7) @@ -81,7 +87,7 @@ FoldBinary │ ├── key: (6) │ └── fd: (6)-->(7) └── filters - - └── (k:1 = x:6) AND (i:2 = (10 - 1)) [outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] + - └── (k:1 = x:6) AND (i:2 = (10 - 1)) [outer=(1,2,6), immutable, constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] + └── (k:1 = x:6) AND (i:2 = 9) [outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: [/9 - /9]; /6: (/NULL - ]), fd=()-->(2)] ================================================================================ SimplifyJoinFilters diff --git a/pkg/sql/opt/norm/testdata/rules/comp b/pkg/sql/opt/norm/testdata/rules/comp index d6819f776580..1f3276a540f4 100644 --- a/pkg/sql/opt/norm/testdata/rules/comp +++ b/pkg/sql/opt/norm/testdata/rules/comp @@ -13,6 +13,7 @@ SELECT * FROM a WHERE 1+ik AND k/2>=i ---- select ├── columns: k:1!null i:2!null f:3 s:4 j:5 d:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-6) ├── scan a @@ -20,9 +21,9 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters - ├── k:1 > (i:2 + 1) [outer=(1,2), constraints=(/1: (/NULL - ])] - ├── i:2 >= (k:1 - 1) [outer=(1,2), constraints=(/2: (/NULL - ])] - ├── k:1 < (i:2 * i:2) [outer=(1,2), constraints=(/1: (/NULL - ])] + ├── k:1 > (i:2 + 1) [outer=(1,2), immutable, constraints=(/1: (/NULL - ])] + ├── i:2 >= (k:1 - 1) [outer=(1,2), immutable, constraints=(/2: (/NULL - ])] + ├── k:1 < (i:2 * i:2) [outer=(1,2), immutable, constraints=(/1: (/NULL - ])] └── i:2 <= (k:1 / 2) [outer=(1,2), constraints=(/2: (/NULL - ])] # -------------------------------------------------- @@ -33,6 +34,7 @@ SELECT * FROM a WHERE 5+1i AND 'foo'>=s ---- select ├── columns: k:1!null i:2!null f:3 s:4!null j:5 d:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-6) ├── scan a @@ -40,8 +42,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters - ├── (i:2 + k:1) > 6 [outer=(1,2)] - ├── (i:2 * 2) >= 8.3333333333333333333 [outer=(2)] + ├── (i:2 + k:1) > 6 [outer=(1,2), immutable] + ├── (i:2 * 2) >= 8.3333333333333333333 [outer=(2), immutable] ├── i:2 < 5 [outer=(2), constraints=(/2: (/NULL - /4]; tight)] └── s:4 <= 'foo' [outer=(4), constraints=(/4: (/NULL - /'foo']; tight)] @@ -50,6 +52,7 @@ SELECT * FROM a WHERE length('foo')+1(2-6) ├── scan a @@ -57,8 +60,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters - ├── (i:2 + k:1) > 4 [outer=(1,2)] - └── (i:2 * 2) >= 3 [outer=(2)] + ├── (i:2 + k:1) > 4 [outer=(1,2), immutable] + └── (i:2 * 2) >= 3 [outer=(2), immutable] # Impure function should not be considered constant. norm expect-not=CommuteConstInequality @@ -102,7 +105,7 @@ select └── filters ├── (i:2 >= 2) AND (i:2 > 6) [outer=(2), constraints=(/2: [/7 - ]; tight)] ├── k:1 = 1 [outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] - ├── (f:3 + f:3) < 3.0 [outer=(3)] + ├── (f:3 + f:3) < 3.0 [outer=(3), immutable] └── i:2::INTERVAL >= '01:00:00' [outer=(2), immutable] # Try case that should not match pattern because Minus overload is not defined. @@ -148,7 +151,7 @@ select └── filters ├── (i:2 >= 4) AND (i:2 < 14) [outer=(2), constraints=(/2: [/4 - /13]; tight)] ├── k:1 = 3 [outer=(1), constraints=(/1: [/3 - /3]; tight), fd=()-->(1)] - ├── (f:3 + f:3) < 7.0 [outer=(3)] + ├── (f:3 + f:3) < 7.0 [outer=(3), immutable] ├── (f:3 + i:2::FLOAT8) >= 110.0 [outer=(2,3), immutable] └── d:6 >= '2018-09-30' [outer=(6), constraints=(/6: [/'2018-09-30' - ]; tight)] @@ -194,7 +197,7 @@ select └── filters ├── (i:2 >= -2) AND (i:2 > 10) [outer=(2), constraints=(/2: [/11 - ]; tight)] ├── k:1 = -1 [outer=(1), constraints=(/1: [/-1 - /-1]; tight), fd=()-->(1)] - ├── (f:3 + f:3) > -3.0 [outer=(3)] + ├── (f:3 + f:3) > -3.0 [outer=(3), immutable] └── (f:3 + i:2::FLOAT8) <= -90.0 [outer=(2,3), immutable] # Try case that should not match pattern because Minus overload is not defined. @@ -203,6 +206,7 @@ SELECT * FROM a WHERE '[1, 2]'::json - i = '[1]' ---- select ├── columns: k:1!null i:2 f:3 s:4 j:5 d:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-6) ├── scan a @@ -210,7 +214,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters - └── ('[1, 2]' - i:2) = '[1]' [outer=(2)] + └── ('[1, 2]' - i:2) = '[1]' [outer=(2), immutable] # -------------------------------------------------- # NormalizeTupleEquality diff --git a/pkg/sql/opt/norm/testdata/rules/decorrelate b/pkg/sql/opt/norm/testdata/rules/decorrelate index 4afcc5791203..dc659a87c012 100644 --- a/pkg/sql/opt/norm/testdata/rules/decorrelate +++ b/pkg/sql/opt/norm/testdata/rules/decorrelate @@ -1051,18 +1051,22 @@ SELECT i*i/5=ANY(SELECT y FROM xy WHERE x=k) AS r FROM a ---- project ├── columns: r:8 + ├── immutable ├── group-by │ ├── columns: k:1!null scalar:9 bool_or:11 │ ├── grouping columns: k:1!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(9,11) │ ├── left-join (hash) │ │ ├── columns: k:1!null x:6 y:7 scalar:9 notnull:10 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) + │ │ ├── immutable │ │ ├── key: (1) │ │ ├── fd: (1)-->(6,7,9,10), (6)-->(7), (7)~~>(10) │ │ ├── project │ │ │ ├── columns: scalar:9 k:1!null + │ │ │ ├── immutable │ │ │ ├── key: (1) │ │ │ ├── fd: (1)-->(9) │ │ │ ├── scan a @@ -1070,7 +1074,7 @@ project │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ └── projections - │ │ │ └── (i:2 * i:2) / 5 [as=scalar:9, outer=(2)] + │ │ │ └── (i:2 * i:2) / 5 [as=scalar:9, outer=(2), immutable] │ │ ├── project │ │ │ ├── columns: notnull:10!null x:6!null y:7 │ │ │ ├── key: (6) @@ -1090,7 +1094,7 @@ project │ └── const-agg [as=scalar:9, outer=(9)] │ └── scalar:9 └── projections - └── CASE WHEN bool_or:11 AND (scalar:9 IS NOT NULL) THEN true WHEN bool_or:11 IS NULL THEN false END [as=r:8, outer=(9,11)] + └── CASE WHEN bool_or:11 AND (scalar:9 IS NOT NULL) THEN true WHEN bool_or:11 IS NULL THEN false END [as=r:8, outer=(9,11), immutable] # -------------------------------------------------- # TryDecorrelateProject @@ -1197,9 +1201,11 @@ WHERE EXISTS ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── semi-join-apply ├── columns: k:1!null i:2 + ├── immutable ├── key: (1) ├── fd: (1)-->(2) ├── scan a @@ -1210,12 +1216,14 @@ project │ ├── columns: x:6!null plus:10 │ ├── outer: (2) │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) + │ ├── immutable │ ├── scan xy │ │ ├── columns: x:6!null │ │ └── key: (6) │ ├── project │ │ ├── columns: plus:10!null │ │ ├── outer: (2) + │ │ ├── immutable │ │ ├── select │ │ │ ├── columns: u:8!null │ │ │ ├── outer: (2) @@ -1226,7 +1234,7 @@ project │ │ │ └── filters │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ └── projections - │ │ └── u:8 + 1 [as=plus:10, outer=(8)] + │ │ └── u:8 + 1 [as=plus:10, outer=(8), immutable] │ └── filters │ └── x:6 = plus:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] └── filters (true) @@ -1241,9 +1249,11 @@ WHERE EXISTS ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── semi-join-apply ├── columns: k:1!null i:2 + ├── immutable ├── key: (1) ├── fd: (1)-->(2) ├── scan a @@ -1254,12 +1264,14 @@ project │ ├── columns: x:6 plus:10 │ ├── outer: (2) │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) + │ ├── immutable │ ├── scan xy │ │ ├── columns: x:6!null │ │ └── key: (6) │ ├── project │ │ ├── columns: plus:10!null │ │ ├── outer: (2) + │ │ ├── immutable │ │ ├── select │ │ │ ├── columns: u:8!null │ │ │ ├── outer: (2) @@ -1270,7 +1282,7 @@ project │ │ │ └── filters │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ └── projections - │ │ └── u:8 + 1 [as=plus:10, outer=(8)] + │ │ └── u:8 + 1 [as=plus:10, outer=(8), immutable] │ └── filters │ └── x:6 = plus:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] └── filters (true) @@ -1283,14 +1295,17 @@ SELECT (SELECT sum(y + v) FROM xy, uv WHERE x=u AND x=k) FROM a ---- project ├── columns: sum:12 + ├── immutable ├── group-by │ ├── columns: k:1!null sum:11 │ ├── grouping columns: k:1!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(11) │ ├── left-join (hash) │ │ ├── columns: k:1!null x:6 column10:10 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) + │ │ ├── immutable │ │ ├── key: (1) │ │ ├── fd: (6)-->(10), (1)-->(6,10) │ │ ├── scan a @@ -1298,6 +1313,7 @@ project │ │ │ └── key: (1) │ │ ├── project │ │ │ ├── columns: column10:10 x:6!null + │ │ │ ├── immutable │ │ │ ├── key: (6) │ │ │ ├── fd: (6)-->(10) │ │ │ ├── inner-join (hash) @@ -1316,7 +1332,7 @@ project │ │ │ │ └── filters │ │ │ │ └── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] │ │ │ └── projections - │ │ │ └── y:7 + v:9 [as=column10:10, outer=(7,9)] + │ │ │ └── y:7 + v:9 [as=column10:10, outer=(7,9), immutable] │ │ └── filters │ │ └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] │ └── aggregations @@ -4075,18 +4091,22 @@ SELECT i*i/100 < ALL(SELECT y FROM xy WHERE x=k) AS r, s FROM a ---- project ├── columns: r:8 s:4 + ├── immutable ├── group-by │ ├── columns: k:1!null s:4 scalar:9 bool_or:11 │ ├── grouping columns: k:1!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(4,9,11) │ ├── left-join (hash) │ │ ├── columns: k:1!null s:4 x:6 y:7 scalar:9 notnull:10 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) + │ │ ├── immutable │ │ ├── key: (1) │ │ ├── fd: (1)-->(4,6,7,9,10), (6)-->(7), (7)~~>(10) │ │ ├── project │ │ │ ├── columns: scalar:9 k:1!null s:4 + │ │ │ ├── immutable │ │ │ ├── key: (1) │ │ │ ├── fd: (1)-->(4,9) │ │ │ ├── scan a @@ -4094,7 +4114,7 @@ project │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2,4) │ │ │ └── projections - │ │ │ └── (i:2 * i:2) / 100 [as=scalar:9, outer=(2)] + │ │ │ └── (i:2 * i:2) / 100 [as=scalar:9, outer=(2), immutable] │ │ ├── project │ │ │ ├── columns: notnull:10!null x:6!null y:7 │ │ │ ├── key: (6) @@ -4116,7 +4136,7 @@ project │ └── const-agg [as=scalar:9, outer=(9)] │ └── scalar:9 └── projections - └── NOT CASE WHEN bool_or:11 AND (scalar:9 IS NOT NULL) THEN true WHEN bool_or:11 IS NULL THEN false END [as=r:8, outer=(9,11)] + └── NOT CASE WHEN bool_or:11 AND (scalar:9 IS NOT NULL) THEN true WHEN bool_or:11 IS NULL THEN false END [as=r:8, outer=(9,11), immutable] # Regress issue #32270: Panic when expression contains both correlated and # uncorrelated subquery. @@ -4458,8 +4478,10 @@ SELECT i, y FROM a INNER JOIN xy ON (SELECT k+1) = x ---- project ├── columns: i:2 y:7 + ├── immutable └── inner-join-apply ├── columns: k:1!null i:2 x:6!null y:7 "?column?":8 + ├── immutable ├── key: (1,6) ├── fd: (1)-->(2), (1,6)-->(7,8), (6)==(8), (8)==(6) ├── scan a @@ -4470,6 +4492,7 @@ project │ ├── columns: x:6!null y:7 "?column?":8 │ ├── outer: (1) │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + │ ├── immutable │ ├── key: (6) │ ├── fd: ()-->(8), (6)-->(7) │ ├── scan xy @@ -4480,6 +4503,7 @@ project │ │ ├── columns: "?column?":8 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] + │ │ ├── immutable │ │ ├── key: () │ │ ├── fd: ()-->(8) │ │ └── (k:1 + 1,) @@ -4608,14 +4632,17 @@ SELECT (VALUES ((SELECT i+1 AS r)), (10), ((SELECT k+1 AS s))) FROM a ---- project ├── columns: column1:9 + ├── immutable ├── ensure-distinct-on │ ├── columns: k:1!null column1:8 │ ├── grouping columns: k:1!null │ ├── error: "more than one row returned by a subquery used as an expression" + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(8) │ ├── inner-join-apply │ │ ├── columns: k:1!null i:2 r:6 s:7 column1:8 + │ │ ├── immutable │ │ ├── fd: (1)-->(2) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 @@ -4625,18 +4652,21 @@ project │ │ │ ├── columns: r:6 s:7 column1:8 │ │ │ ├── outer: (1,2) │ │ │ ├── cardinality: [3 - 3] + │ │ │ ├── immutable │ │ │ ├── fd: ()-->(6,7) │ │ │ ├── inner-join (cross) │ │ │ │ ├── columns: r:6 s:7 │ │ │ │ ├── outer: (1,2) │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) + │ │ │ │ ├── immutable │ │ │ │ ├── key: () │ │ │ │ ├── fd: ()-->(6,7) │ │ │ │ ├── values │ │ │ │ │ ├── columns: r:6 │ │ │ │ │ ├── outer: (2) │ │ │ │ │ ├── cardinality: [1 - 1] + │ │ │ │ │ ├── immutable │ │ │ │ │ ├── key: () │ │ │ │ │ ├── fd: ()-->(6) │ │ │ │ │ └── (i:2 + 1,) @@ -4644,6 +4674,7 @@ project │ │ │ │ │ ├── columns: s:7 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [1 - 1] + │ │ │ │ │ ├── immutable │ │ │ │ │ ├── key: () │ │ │ │ │ ├── fd: ()-->(7) │ │ │ │ │ └── (k:1 + 1,) diff --git a/pkg/sql/opt/norm/testdata/rules/fold_constants b/pkg/sql/opt/norm/testdata/rules/fold_constants index d4fc7fcd99a6..720fd2d269c1 100644 --- a/pkg/sql/opt/norm/testdata/rules/fold_constants +++ b/pkg/sql/opt/norm/testdata/rules/fold_constants @@ -243,6 +243,7 @@ SELECT 9223372036854775800::INT + 9223372036854775800::INT values ├── columns: "?column?":1 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(1) └── (9223372036854775800 + 9223372036854775800,) @@ -265,6 +266,7 @@ SELECT (-9223372036854775800)::INT - 9223372036854775800::INT values ├── columns: "?column?":1 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(1) └── (-9223372036854775800 - 9223372036854775800,) @@ -287,6 +289,7 @@ SELECT 9223372036854775800::INT * 9223372036854775800::INT values ├── columns: "?column?":1 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(1) └── (9223372036854775800 * 9223372036854775800,) @@ -332,6 +335,7 @@ SELECT B'01' # B'11001001010101' values ├── columns: "?column?":1 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(1) └── (B'01' # B'11001001010101',) @@ -354,6 +358,7 @@ SELECT B'01' | B'11001001010101' values ├── columns: "?column?":1 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(1) └── (B'01' | B'11001001010101',) @@ -455,6 +460,7 @@ SELECT -((-9223372036854775808)::int) values ├── columns: "?column?":1(int) ├── cardinality: [1 - 1] + ├── immutable ├── stats: [rows=1] ├── cost: 0.02 ├── key: () @@ -845,10 +851,11 @@ SELECT ARRAY[i, i + 1][2] FROM a ---- project ├── columns: array:7 + ├── immutable ├── scan a │ └── columns: i:2 └── projections - └── i:2 + 1 [as=array:7, outer=(2)] + └── i:2 + 1 [as=array:7, outer=(2), immutable] # Fold when input is a DArray constant. norm expect=FoldIndirection diff --git a/pkg/sql/opt/norm/testdata/rules/groupby b/pkg/sql/opt/norm/testdata/rules/groupby index 0da07bcde89c..6b91051f0623 100644 --- a/pkg/sql/opt/norm/testdata/rules/groupby +++ b/pkg/sql/opt/norm/testdata/rules/groupby @@ -700,26 +700,30 @@ SELECT (SELECT y FROM xy WHERE x+y=k) FROM a ---- project ├── columns: y:8 + ├── immutable ├── ensure-distinct-on │ ├── columns: k:1!null xy.y:7 │ ├── grouping columns: k:1!null │ ├── error: "more than one row returned by a subquery used as an expression" + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(7) │ ├── left-join (hash) │ │ ├── columns: k:1!null xy.y:7 column9:9 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) + │ │ ├── immutable │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── project │ │ │ ├── columns: column9:9 xy.y:7 + │ │ │ ├── immutable │ │ │ ├── scan xy │ │ │ │ ├── columns: x:6!null xy.y:7 │ │ │ │ ├── key: (6) │ │ │ │ └── fd: (6)-->(7) │ │ │ └── projections - │ │ │ └── x:6 + xy.y:7 [as=column9:9, outer=(6,7)] + │ │ │ └── x:6 + xy.y:7 [as=column9:9, outer=(6,7), immutable] │ │ └── filters │ │ └── k:1 = column9:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] │ └── aggregations @@ -857,18 +861,21 @@ SELECT min(s) FROM (SELECT i+1 AS i2, s FROM a) GROUP BY i2 ---- project ├── columns: min:7!null + ├── immutable └── group-by ├── columns: i2:6!null min:7!null ├── grouping columns: i2:6!null + ├── immutable ├── key: (6) ├── fd: (6)-->(7) ├── project │ ├── columns: i2:6!null s:4!null + │ ├── immutable │ ├── scan a │ │ ├── columns: i:2!null s:4!null │ │ └── key: (2,4) │ └── projections - │ └── i:2 + 1 [as=i2:6, outer=(2)] + │ └── i:2 + 1 [as=i2:6, outer=(2), immutable] └── aggregations └── min [as=min:7, outer=(4)] └── s:4 @@ -1925,6 +1932,7 @@ WHERE x > 100 OR b > 100 ---- project ├── columns: x:1!null y:2!null z:3!null a:4 b:5 c:6 "?column?":7!null + ├── immutable ├── fd: (1)-->(7) ├── select │ ├── columns: column1:1!null column2:2!null column3:3!null a:4 b:5 c:6 @@ -1947,7 +1955,7 @@ project │ └── filters │ └── (column1:1 > 100) OR (b:5 > 100) [outer=(1,5)] └── projections - └── column1:1 + 1 [as="?column?":7, outer=(1)] + └── column1:1 + 1 [as="?column?":7, outer=(1), immutable] # Right input of left join does not have a key, so left side may have dups. norm expect-not=EliminateDistinctOnValues @@ -2025,11 +2033,13 @@ distinct-on ├── columns: x:1!null y:2!null ├── grouping columns: y:2!null ├── cardinality: [1 - 2] + ├── immutable ├── key: (2) ├── fd: (1)-->(2), (2)-->(1) ├── project │ ├── columns: y:2!null column1:1!null │ ├── cardinality: [2 - 2] + │ ├── immutable │ ├── fd: (1)-->(2) │ ├── values │ │ ├── columns: column1:1!null @@ -2037,7 +2047,7 @@ distinct-on │ │ ├── (1,) │ │ └── (2,) │ └── projections - │ └── column1:1 + 1 [as=y:2, outer=(1)] + │ └── column1:1 + 1 [as=y:2, outer=(1), immutable] └── aggregations └── first-agg [as=column1:1, outer=(1)] └── column1:1 diff --git a/pkg/sql/opt/norm/testdata/rules/inline b/pkg/sql/opt/norm/testdata/rules/inline index 3f1bddef930c..062a8bfaf354 100644 --- a/pkg/sql/opt/norm/testdata/rules/inline +++ b/pkg/sql/opt/norm/testdata/rules/inline @@ -103,7 +103,7 @@ select │ ├── (0.00,) │ └── (0.000,) └── filters - ├── column1:1 = 0 [outer=(1), constraints=(/1: [/0 - /0]; tight), fd=()-->(1)] + ├── column1:1 = 0 [outer=(1), immutable, constraints=(/1: [/0 - /0]; tight), fd=()-->(1)] └── column1:1::STRING = '0.00' [outer=(1), immutable] # The rule should trigger, but not inline the composite type. @@ -122,7 +122,7 @@ select │ ├── (0.00, 'b') │ └── (0.000, 'b') └── filters - ├── column1:1 = 0 [outer=(1), constraints=(/1: [/0 - /0]; tight), fd=()-->(1)] + ├── column1:1 = 0 [outer=(1), immutable, constraints=(/1: [/0 - /0]; tight), fd=()-->(1)] ├── column1:1::STRING = '0.00' [outer=(1), immutable] └── column2:2 = 'b' [outer=(2), constraints=(/2: [/'b' - /'b']; tight), fd=()-->(2)] @@ -196,7 +196,7 @@ SELECT one+two+three+four FROM (VALUES (1, $1:::int, 2, $2:::int)) AS t(one, two project ├── columns: "?column?":5 ├── cardinality: [1 - 1] - ├── has-placeholder + ├── immutable, has-placeholder ├── key: () ├── fd: ()-->(5) ├── values @@ -207,7 +207,7 @@ project │ ├── fd: ()-->(2,4) │ └── ($1, $2) └── projections - └── column4:4 + ((column2:2 + 1) + 2) [as="?column?":5, outer=(2,4)] + └── column4:4 + ((column2:2 + 1) + 2) [as="?column?":5, outer=(2,4), immutable] # Multiple constant columns, multiple refs to each, interspersed with other # columns. @@ -258,13 +258,14 @@ SELECT one+two FROM (VALUES (1, 2), (3, 4)) AS t(one, two) project ├── columns: "?column?":3!null ├── cardinality: [2 - 2] + ├── immutable ├── values │ ├── columns: column1:1!null column2:2!null │ ├── cardinality: [2 - 2] │ ├── (1, 2) │ └── (3, 4) └── projections - └── column1:1 + column2:2 [as="?column?":3, outer=(1,2)] + └── column1:1 + column2:2 [as="?column?":3, outer=(1,2), immutable] # -------------------------------------------------- # InlineSelectConstants @@ -456,16 +457,18 @@ SELECT * FROM (SELECT k*2+1 AS expr FROM a) a WHERE expr > 10 ---- project ├── columns: expr:6!null + ├── immutable ├── select │ ├── columns: k:1!null + │ ├── immutable │ ├── key: (1) │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ └── filters - │ └── (k:1 * 2) > 9 [outer=(1)] + │ └── (k:1 * 2) > 9 [outer=(1), immutable] └── projections - └── (k:1 * 2) + 1 [as=expr:6, outer=(1)] + └── (k:1 * 2) + 1 [as=expr:6, outer=(1), immutable] # Inline boolean logic. norm expect=PushSelectIntoInlinableProject @@ -505,14 +508,16 @@ SELECT * FROM (SELECT f+1 AS expr FROM a) a WHERE expr=expr ---- project ├── columns: expr:6 + ├── immutable ├── select │ ├── columns: f:3 + │ ├── immutable │ ├── scan a │ │ └── columns: f:3 │ └── filters - │ └── (f:3 + 1.0) IS DISTINCT FROM CAST(NULL AS FLOAT8) [outer=(3)] + │ └── (f:3 + 1.0) IS DISTINCT FROM CAST(NULL AS FLOAT8) [outer=(3), immutable] └── projections - └── f:3 + 1.0 [as=expr:6, outer=(3)] + └── f:3 + 1.0 [as=expr:6, outer=(3), immutable] # Use outer references in both inlined expression and in referencing expression. norm expect=PushSelectIntoInlinableProject @@ -520,6 +525,7 @@ SELECT * FROM a WHERE EXISTS(SELECT * FROM (SELECT (x-i) AS expr FROM xy) WHERE ---- semi-join (cross) ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-5) ├── scan a @@ -530,7 +536,7 @@ semi-join (cross) │ ├── columns: x:6!null │ └── key: (6) └── filters - └── (x:6 - i:2) > (i:2 * i:2) [outer=(2,6)] + └── (x:6 - i:2) > (i:2 * i:2) [outer=(2,6), immutable] exec-ddl CREATE TABLE crdb_internal.zones ( @@ -596,13 +602,14 @@ project │ ├── limit hint: 107.00 │ ├── project │ │ ├── columns: c0:6!null c1:7!null + │ │ ├── immutable │ │ ├── limit hint: 321.00 │ │ ├── scan crdb_internal.public.zones │ │ │ ├── columns: crdb_internal.public.zones.zone_id:1!null │ │ │ └── limit hint: 321.00 │ │ └── projections - │ │ ├── crdb_internal.public.zones.zone_id:1 + 1 [as=c0:6, outer=(1)] - │ │ └── crdb_internal.public.zones.zone_id:1 + 2 [as=c1:7, outer=(1)] + │ │ ├── crdb_internal.public.zones.zone_id:1 + 1 [as=c0:6, outer=(1), immutable] + │ │ └── crdb_internal.public.zones.zone_id:1 + 2 [as=c1:7, outer=(1), immutable] │ └── filters │ └── le [outer=(6,7), stable+volatile, side-effects, correlated-subquery] │ ├── case @@ -636,13 +643,14 @@ SELECT NOT(expr), i+1 AS r FROM (SELECT k=1 AS expr, i FROM a) ---- project ├── columns: "?column?":7!null r:8 + ├── immutable ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections ├── k:1 != 1 [as="?column?":7, outer=(1)] - └── i:2 + 1 [as=r:8, outer=(2)] + └── i:2 + 1 [as=r:8, outer=(2), immutable] # Multiple synthesized column references to same inner passthrough column # (should still inline). @@ -651,14 +659,15 @@ SELECT x+1, x+2, y1+2 FROM (SELECT x, y+1 AS y1 FROM xy) ---- project ├── columns: "?column?":4!null "?column?":5!null "?column?":6 + ├── immutable ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - ├── x:1 + 1 [as="?column?":4, outer=(1)] - ├── x:1 + 2 [as="?column?":5, outer=(1)] - └── (y:2 + 1) + 2 [as="?column?":6, outer=(2)] + ├── x:1 + 1 [as="?column?":4, outer=(1), immutable] + ├── x:1 + 2 [as="?column?":5, outer=(1), immutable] + └── (y:2 + 1) + 2 [as="?column?":6, outer=(2), immutable] # Synthesized and passthrough references to same inner passthrough column # (should still inline). @@ -667,6 +676,7 @@ SELECT x+y1 FROM (SELECT x, y+1 AS y1 FROM xy) ORDER BY x ---- project ├── columns: "?column?":4 [hidden: x:1!null] + ├── immutable ├── key: (1) ├── fd: (1)-->(4) ├── ordering: +1 @@ -676,7 +686,7 @@ project │ ├── fd: (1)-->(2) │ └── ordering: +1 └── projections - └── x:1 + (y:2 + 1) [as="?column?":4, outer=(1,2)] + └── x:1 + (y:2 + 1) [as="?column?":4, outer=(1,2), immutable] # Inline multiple expressions. norm expect=InlineProjectInProject @@ -684,13 +694,14 @@ SELECT expr+1 AS r, i, expr2 || 'bar' AS s FROM (SELECT k+1 AS expr, s || 'foo' ---- project ├── columns: r:8!null i:2 s:9 + ├── immutable ├── scan a │ ├── columns: k:1!null i:2 a.s:4 │ ├── key: (1) │ └── fd: (1)-->(2,4) └── projections - ├── (k:1 + 1) + 1 [as=r:8, outer=(1)] - └── (a.s:4 || 'foo') || 'bar' [as=s:9, outer=(4)] + ├── (k:1 + 1) + 1 [as=r:8, outer=(1), immutable] + └── (a.s:4 || 'foo') || 'bar' [as=s:9, outer=(4), immutable] # Don't inline when there are multiple references. norm expect-not=InlineProjectInProject @@ -698,16 +709,18 @@ SELECT expr, expr*2 AS r FROM (SELECT k+1 AS expr FROM a) ---- project ├── columns: expr:6!null r:7!null + ├── immutable ├── fd: (6)-->(7) ├── project │ ├── columns: expr:6!null + │ ├── immutable │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ └── projections - │ └── k:1 + 1 [as=expr:6, outer=(1)] + │ └── k:1 + 1 [as=expr:6, outer=(1), immutable] └── projections - └── expr:6 * 2 [as=r:7, outer=(6)] + └── expr:6 * 2 [as=r:7, outer=(6), immutable] # Uncorrelated subquery should not block inlining. norm expect=InlineProjectInProject @@ -715,6 +728,7 @@ SELECT EXISTS(SELECT * FROM xy WHERE x=1 OR x=2), expr*2 AS r FROM (SELECT k+1 A ---- project ├── columns: exists:9 r:10!null + ├── immutable ├── fd: ()-->(9) ├── scan a │ ├── columns: k:1!null @@ -740,7 +754,7 @@ project │ │ └── filters │ │ └── (x:7 = 1) OR (x:7 = 2) [outer=(7), constraints=(/7: [/1 - /1] [/2 - /2]; tight)] │ └── 1 - └── (k:1 + 1) * 2 [as=r:10, outer=(1)] + └── (k:1 + 1) * 2 [as=r:10, outer=(1), immutable] # Correlated subquery should be hoisted as usual. norm expect=InlineProjectInProject @@ -748,26 +762,31 @@ SELECT EXISTS(SELECT * FROM xy WHERE expr<0) FROM (SELECT k+1 AS expr FROM a) ---- project ├── columns: exists:9!null + ├── immutable ├── group-by │ ├── columns: true_agg:11 rownum:13!null │ ├── grouping columns: rownum:13!null + │ ├── immutable │ ├── key: (13) │ ├── fd: (13)-->(11) │ ├── left-join (cross) │ │ ├── columns: expr:6!null true:10 rownum:13!null │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) + │ │ ├── immutable │ │ ├── fd: (13)-->(6) │ │ ├── ordinality │ │ │ ├── columns: expr:6!null rownum:13!null + │ │ │ ├── immutable │ │ │ ├── key: (13) │ │ │ ├── fd: (13)-->(6) │ │ │ └── project │ │ │ ├── columns: expr:6!null + │ │ │ ├── immutable │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null │ │ │ │ └── key: (1) │ │ │ └── projections - │ │ │ └── k:1 + 1 [as=expr:6, outer=(1)] + │ │ │ └── k:1 + 1 [as=expr:6, outer=(1), immutable] │ │ ├── project │ │ │ ├── columns: true:10!null │ │ │ ├── fd: ()-->(10) @@ -788,6 +807,7 @@ SELECT c FROM (SELECT k+2 AS c FROM a) AS t WHERE c > 2; ---- project ├── columns: c:6!null + ├── immutable ├── select │ ├── columns: k:1!null │ ├── key: (1) @@ -797,4 +817,4 @@ project │ └── filters │ └── k:1 > 0 [outer=(1), constraints=(/1: [/1 - ]; tight)] └── projections - └── k:1 + 2 [as=c:6, outer=(1)] + └── k:1 + 2 [as=c:6, outer=(1), immutable] diff --git a/pkg/sql/opt/norm/testdata/rules/join b/pkg/sql/opt/norm/testdata/rules/join index baa909153d90..99c972d98f35 100644 --- a/pkg/sql/opt/norm/testdata/rules/join +++ b/pkg/sql/opt/norm/testdata/rules/join @@ -346,10 +346,12 @@ SELECT * FROM a INNER JOIN b ON a.k=b.x AND a.k + b.y > 5 AND b.x * a.i = 3 inner-join (hash) ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) + ├── immutable ├── key: (6) ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) ├── select │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2-5) │ ├── scan a @@ -357,9 +359,10 @@ inner-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ └── filters - │ └── (k:1 * i:2) = 3 [outer=(1,2)] + │ └── (k:1 * i:2) = 3 [outer=(1,2), immutable] ├── select │ ├── columns: x:6!null y:7 + │ ├── immutable │ ├── key: (6) │ ├── fd: (6)-->(7) │ ├── scan b @@ -367,7 +370,7 @@ inner-join (hash) │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters - │ └── (x:6 + y:7) > 5 [outer=(6,7)] + │ └── (x:6 + y:7) > 5 [outer=(6,7), immutable] └── filters └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] @@ -415,10 +418,12 @@ SELECT * FROM a WHERE EXISTS( ---- semi-join (hash) ├── columns: k:1!null i:2 f:3!null s:4 j:5 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-5) ├── select │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2-5) │ ├── scan a @@ -426,9 +431,10 @@ semi-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ └── filters - │ └── (k:1 * i:2) = 3 [outer=(1,2)] + │ └── (k:1 * i:2) = 3 [outer=(1,2), immutable] ├── select │ ├── columns: x:6!null y:7 + │ ├── immutable │ ├── key: (6) │ ├── fd: (6)-->(7) │ ├── scan b @@ -436,7 +442,7 @@ semi-join (hash) │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters - │ └── (x:6 + y:7) > 5 [outer=(6,7)] + │ └── (x:6 + y:7) > 5 [outer=(6,7), immutable] └── filters └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] @@ -480,6 +486,7 @@ SELECT * FROM a LEFT JOIN b ON a.k=b.x AND a.k + b.y > 5 AND b.x * a.i = 3 left-join (hash) ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6 y:7 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) + ├── immutable ├── key: (1) ├── fd: (1)-->(2-7), (6)-->(7) ├── scan a @@ -488,6 +495,7 @@ left-join (hash) │ └── fd: (1)-->(2-5) ├── select │ ├── columns: x:6!null y:7 + │ ├── immutable │ ├── key: (6) │ ├── fd: (6)-->(7) │ ├── scan b @@ -495,10 +503,10 @@ left-join (hash) │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters - │ └── (x:6 + y:7) > 5 [outer=(6,7)] + │ └── (x:6 + y:7) > 5 [outer=(6,7), immutable] └── filters ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── (x:6 * i:2) = 3 [outer=(2,6)] + └── (x:6 * i:2) = 3 [outer=(2,6), immutable] norm expect=MapFilterIntoJoinRight expect-not=PushFilterIntoJoinLeftAndRight SELECT * FROM a LEFT JOIN b ON a.k=b.x AND a.k > 5 AND b.x IN (3, 7, 10) @@ -533,6 +541,7 @@ SELECT * FROM a FULL JOIN b ON a.k=b.x AND a.k + b.y > 5 AND b.x * a.i = 3 full-join (hash) ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) + ├── immutable ├── key: (1,6) ├── fd: (1)-->(2-5), (6)-->(7) ├── scan a @@ -545,8 +554,8 @@ full-join (hash) │ └── fd: (6)-->(7) └── filters ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── (k:1 + y:7) > 5 [outer=(1,7)] - └── (x:6 * i:2) = 3 [outer=(2,6)] + ├── (k:1 + y:7) > 5 [outer=(1,7), immutable] + └── (x:6 * i:2) = 3 [outer=(2,6), immutable] norm expect-not=(PushFilterIntoJoinLeftAndRight,MapFilterIntoJoinLeft,MapFilterIntoJoinRight) SELECT * FROM a FULL JOIN b ON a.k=b.x AND a.k > 5 AND b.x IN (3, 7, 10) @@ -577,6 +586,7 @@ SELECT * FROM a WHERE NOT EXISTS( ---- anti-join (hash) ├── columns: k:1!null i:2 f:3!null s:4 j:5 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-5) ├── scan a @@ -585,6 +595,7 @@ anti-join (hash) │ └── fd: (1)-->(2-5) ├── select │ ├── columns: x:6!null y:7 + │ ├── immutable │ ├── key: (6) │ ├── fd: (6)-->(7) │ ├── scan b @@ -592,10 +603,10 @@ anti-join (hash) │ │ ├── key: (6) │ │ └── fd: (6)-->(7) │ └── filters - │ └── (x:6 + y:7) > 5 [outer=(6,7)] + │ └── (x:6 + y:7) > 5 [outer=(6,7), immutable] └── filters ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── (x:6 * i:2) = 3 [outer=(2,6)] + └── (x:6 * i:2) = 3 [outer=(2,6), immutable] norm expect=MapFilterIntoJoinRight expect-not=PushFilterIntoJoinLeftAndRight SELECT * FROM a WHERE NOT EXISTS( @@ -629,10 +640,12 @@ SELECT * FROM a JOIN b ON a.k = b.x AND b.x * a.i = (SELECT min(b.x) FROM b) inner-join (hash) ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) + ├── immutable ├── key: (6) ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) ├── select │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2-5) │ ├── scan a @@ -640,7 +653,7 @@ inner-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ └── filters - │ └── eq [outer=(1,2), subquery] + │ └── eq [outer=(1,2), immutable, subquery] │ ├── k:1 * i:2 │ └── subquery │ └── scalar-group-by @@ -667,10 +680,12 @@ SELECT * FROM a JOIN b ON a.k = b.x AND b.x * a.i = (SELECT a.k * b.y FROM b) ---- project ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 + ├── immutable ├── key: (6) ├── fd: (1)-->(2-5), (1,6)-->(7), (1)==(6), (6)==(1) └── inner-join-apply ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 "?column?":10 + ├── immutable ├── key: (6) ├── fd: (1)-->(2-5), (1,6)-->(7,10), (1)==(6), (6)==(1) ├── scan a @@ -682,12 +697,14 @@ project │ ├── grouping columns: x:6!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── outer: (1) + │ ├── immutable │ ├── key: (6) │ ├── fd: (6)-->(7,10) │ ├── left-join (cross) │ │ ├── columns: x:6!null y:7 "?column?":10 │ │ ├── outer: (1) │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) + │ │ ├── immutable │ │ ├── fd: (6)-->(7) │ │ ├── scan b │ │ │ ├── columns: x:6!null y:7 @@ -696,10 +713,11 @@ project │ │ ├── project │ │ │ ├── columns: "?column?":10 │ │ │ ├── outer: (1) + │ │ │ ├── immutable │ │ │ ├── scan b │ │ │ │ └── columns: y:9 │ │ │ └── projections - │ │ │ └── k:1 * y:9 [as="?column?":10, outer=(1,9)] + │ │ │ └── k:1 * y:9 [as="?column?":10, outer=(1,9), immutable] │ │ └── filters (true) │ └── aggregations │ ├── const-agg [as=y:7, outer=(7)] @@ -708,7 +726,7 @@ project │ └── "?column?":10 └── filters ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── "?column?":10 = (x:6 * i:2) [outer=(2,6,10), constraints=(/10: (/NULL - ])] + └── "?column?":10 = (x:6 * i:2) [outer=(2,6,10), immutable, constraints=(/10: (/NULL - ])] # Ensure that we do not map filters for types with composite key encoding. norm expect-not=(PushFilterIntoJoinLeftAndRight,MapFilterIntoJoinLeft,MapFilterIntoJoinRight) @@ -737,7 +755,7 @@ inner-join (hash) │ ├── (1.00,) │ └── (2.00,) └── filters - └── column1:1 = column1:2 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] + └── column1:1 = column1:2 [outer=(1,2), immutable, constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] # Optimization does not apply if equality is only on one side. norm expect-not=(PushFilterIntoJoinLeftAndRight,MapFilterIntoJoinLeft,MapFilterIntoJoinRight) @@ -745,6 +763,7 @@ SELECT * FROM a INNER JOIN b ON b.y=b.x AND a.k=a.i AND a.k + b.y > 5 AND b.x * ---- inner-join (cross) ├── columns: k:1!null i:2!null f:3!null s:4 j:5 x:6!null y:7!null + ├── immutable ├── key: (1,6) ├── fd: (1)-->(3-5), (1)==(2), (2)==(1), (6)==(7), (7)==(6) ├── select @@ -768,8 +787,8 @@ inner-join (cross) │ └── filters │ └── y:7 = x:6 [outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] └── filters - ├── (k:1 + y:7) > 5 [outer=(1,7)] - └── (x:6 * i:2) = 3 [outer=(2,6)] + ├── (k:1 + y:7) > 5 [outer=(1,7), immutable] + └── (x:6 * i:2) = 3 [outer=(2,6), immutable] # Ensure that MapFilterIntoJoinRight doesn't cause cycle with decorrelation. norm expect=MapFilterIntoJoinRight @@ -783,14 +802,17 @@ FROM c ---- project ├── columns: x:13 + ├── immutable ├── ensure-distinct-on │ ├── columns: c.x:1!null b.x:4 │ ├── grouping columns: c.x:1!null │ ├── error: "more than one row returned by a subquery used as an expression" + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(4) │ ├── left-join-apply │ │ ├── columns: c.x:1!null b.x:4 k:8 + │ │ ├── immutable │ │ ├── fd: (4)==(8), (8)==(4) │ │ ├── scan c │ │ │ ├── columns: c.x:1!null @@ -799,6 +821,7 @@ project │ │ │ ├── columns: b.x:4!null k:8!null │ │ │ ├── outer: (1) │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ │ ├── immutable │ │ │ ├── fd: (4)==(8), (8)==(4) │ │ │ ├── full-join (cross) │ │ │ │ ├── columns: b.x:4 @@ -812,12 +835,13 @@ project │ │ │ │ └── c.x:1 = 5 [outer=(1), constraints=(/1: [/5 - /5]; tight), fd=()-->(1)] │ │ │ ├── select │ │ │ │ ├── columns: k:8!null + │ │ │ │ ├── immutable │ │ │ │ ├── key: (8) │ │ │ │ ├── scan a │ │ │ │ │ ├── columns: k:8!null │ │ │ │ │ └── key: (8) │ │ │ │ └── filters - │ │ │ │ └── (k:8 + k:8) < 5 [outer=(8)] + │ │ │ │ └── (k:8 + k:8) < 5 [outer=(8), immutable] │ │ │ └── filters │ │ │ └── k:8 = b.x:4 [outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ]), fd=(4)==(8), (8)==(4)] │ │ └── filters (true) @@ -838,14 +862,17 @@ FROM c ---- project ├── columns: x:13 + ├── immutable ├── ensure-distinct-on │ ├── columns: c.x:1!null b.x:9 │ ├── grouping columns: c.x:1!null │ ├── error: "more than one row returned by a subquery used as an expression" + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(9) │ ├── left-join-apply │ │ ├── columns: c.x:1!null k:4 b.x:9 + │ │ ├── immutable │ │ ├── fd: (4)==(9), (9)==(4) │ │ ├── scan c │ │ │ ├── columns: c.x:1!null @@ -854,15 +881,17 @@ project │ │ │ ├── columns: k:4!null b.x:9!null │ │ │ ├── outer: (1) │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) + │ │ │ ├── immutable │ │ │ ├── fd: (4)==(9), (9)==(4) │ │ │ ├── select │ │ │ │ ├── columns: k:4!null + │ │ │ │ ├── immutable │ │ │ │ ├── key: (4) │ │ │ │ ├── scan a │ │ │ │ │ ├── columns: k:4!null │ │ │ │ │ └── key: (4) │ │ │ │ └── filters - │ │ │ │ └── (k:4 + k:4) < 5 [outer=(4)] + │ │ │ │ └── (k:4 + k:4) < 5 [outer=(4), immutable] │ │ │ ├── full-join (cross) │ │ │ │ ├── columns: b.x:9 │ │ │ │ ├── outer: (1) @@ -897,7 +926,7 @@ SELECT * FROM t1, t2 WHERE a = b AND age(b, TIMESTAMPTZ '2017-01-01') > INTERVAL ---- inner-join (cross) ├── columns: a:1!null b:3!null - ├── immutable, side-effects + ├── stable, side-effects ├── fd: (1)==(3), (3)==(1) ├── scan t1 │ └── columns: a:1 @@ -909,7 +938,7 @@ inner-join (cross) │ └── filters │ └── age(b:3, '2017-01-01 00:00:00+00:00') > '1 day' [outer=(3), immutable, side-effects] └── filters - └── a:1 = b:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── a:1 = b:3 [outer=(1,3), stable, constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] # Regression for issue 28818. Try to trigger undetectable cycle between the # PushFilterIntoJoinLeftAndRight and TryDecorrelateSelect rules. @@ -1891,10 +1920,12 @@ SELECT * FROM a WHERE (SELECT sum(column1) FROM (VALUES (k), (1))) = 1 ---- project ├── columns: k:1!null i:2 f:3!null s:4 j:5 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-5) └── select ├── columns: k:1!null i:2 f:3!null s:4 j:5 sum:7!null + ├── immutable ├── key: (1) ├── fd: ()-->(7), (1)-->(2-5) ├── group-by @@ -1928,7 +1959,7 @@ project │ └── const-agg [as=j:5, outer=(5)] │ └── j:5 └── filters - └── sum:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── sum:7 = 1 [outer=(7), immutable, constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] # Don't simplify left join norm expect-not=SimplifyRightJoin @@ -2092,6 +2123,7 @@ SELECT * FROM a FULL JOIN (SELECT k+1 AS k FROM a) AS a2 ON a.k=a2.k full-join (hash) ├── columns: k:1 i:2 f:3 s:4 j:5 k:11 ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) + ├── immutable ├── fd: (1)-->(2-5) ├── scan a │ ├── columns: a.k:1!null i:2 f:3!null s:4 j:5 @@ -2099,11 +2131,12 @@ full-join (hash) │ └── fd: (1)-->(2-5) ├── project │ ├── columns: k:11!null + │ ├── immutable │ ├── scan a │ │ ├── columns: a.k:6!null │ │ └── key: (6) │ └── projections - │ └── a.k:6 + 1 [as=k:11, outer=(6)] + │ └── a.k:6 + 1 [as=k:11, outer=(6), immutable] └── filters └── a.k:1 = k:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] @@ -2591,6 +2624,7 @@ inner-join (hash) │ └── i:2 > 0 [outer=(2), constraints=(/2: [/1 - ]; tight)] ├── project │ ├── columns: z:8!null x:6!null y:7!null + │ ├── immutable │ ├── key: (6) │ ├── fd: (6)-->(7), (7)-->(8) │ ├── select @@ -2604,7 +2638,7 @@ inner-join (hash) │ │ └── filters │ │ └── y:7 > 10 [outer=(7), constraints=(/7: [/11 - ]; tight)] │ └── projections - │ └── y:7 + 1 [as=z:8, outer=(7)] + │ └── y:7 + 1 [as=z:8, outer=(7), immutable] └── filters ├── f:3 >= z:8::FLOAT8 [outer=(3,8), immutable, constraints=(/3: (/NULL - ])] ├── f:3 >= z:8::FLOAT8 [outer=(3,8), immutable, constraints=(/3: (/NULL - ])] @@ -2640,15 +2674,18 @@ SELECT * FROM xy JOIN uv ON x+y=u ---- project ├── columns: x:1!null y:2 u:3!null v:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2), (1,2)-->(3,4), (3)-->(4) └── inner-join (hash) ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + ├── immutable ├── key: (1) ├── fd: (1)-->(2), (1,2)-->(5), (3)-->(4), (3)==(5), (5)==(3) ├── project │ ├── columns: column5:5 x:1!null y:2 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2), (1,2)-->(5) │ ├── scan xy @@ -2656,7 +2693,7 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── x:1 + y:2 [as=column5:5, outer=(1,2)] + │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] ├── scan uv │ ├── columns: u:3!null v:4 │ ├── key: (3) @@ -2669,15 +2706,18 @@ SELECT * FROM xy JOIN uv ON u=x+y ---- project ├── columns: x:1!null y:2 u:3!null v:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2), (1,2)-->(3,4), (3)-->(4) └── inner-join (hash) ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + ├── immutable ├── key: (1) ├── fd: (1)-->(2), (1,2)-->(5), (3)-->(4), (3)==(5), (5)==(3) ├── project │ ├── columns: column5:5 x:1!null y:2 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2), (1,2)-->(5) │ ├── scan xy @@ -2685,7 +2725,7 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── x:1 + y:2 [as=column5:5, outer=(1,2)] + │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] ├── scan uv │ ├── columns: u:3!null v:4 │ ├── key: (3) @@ -2698,11 +2738,13 @@ SELECT * FROM xy JOIN uv ON x=u+v ---- project ├── columns: x:1!null y:2 u:3!null v:4 + ├── immutable ├── key: (3) ├── fd: (1)-->(2), (3)-->(4), (3,4)-->(1,2) └── inner-join (hash) ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) + ├── immutable ├── key: (3) ├── fd: (1)-->(2), (3)-->(4), (3,4)-->(5), (1)==(5), (5)==(1) ├── scan xy @@ -2711,6 +2753,7 @@ project │ └── fd: (1)-->(2) ├── project │ ├── columns: column5:5 u:3!null v:4 + │ ├── immutable │ ├── key: (3) │ ├── fd: (3)-->(4), (3,4)-->(5) │ ├── scan uv @@ -2718,7 +2761,7 @@ project │ │ ├── key: (3) │ │ └── fd: (3)-->(4) │ └── projections - │ └── u:3 + v:4 [as=column5:5, outer=(3,4)] + │ └── u:3 + v:4 [as=column5:5, outer=(3,4), immutable] └── filters └── x:1 = column5:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] @@ -2727,11 +2770,13 @@ SELECT * FROM xy JOIN uv ON u+v=x ---- project ├── columns: x:1!null y:2 u:3!null v:4 + ├── immutable ├── key: (3) ├── fd: (1)-->(2), (3)-->(4), (3,4)-->(1,2) └── inner-join (hash) ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) + ├── immutable ├── key: (3) ├── fd: (1)-->(2), (3)-->(4), (3,4)-->(5), (1)==(5), (5)==(1) ├── scan xy @@ -2740,6 +2785,7 @@ project │ └── fd: (1)-->(2) ├── project │ ├── columns: column5:5 u:3!null v:4 + │ ├── immutable │ ├── key: (3) │ ├── fd: (3)-->(4), (3,4)-->(5) │ ├── scan uv @@ -2747,7 +2793,7 @@ project │ │ ├── key: (3) │ │ └── fd: (3)-->(4) │ └── projections - │ └── u:3 + v:4 [as=column5:5, outer=(3,4)] + │ └── u:3 + v:4 [as=column5:5, outer=(3,4), immutable] └── filters └── x:1 = column5:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] @@ -2756,14 +2802,17 @@ SELECT * FROM xy JOIN uv ON x+y=u+v ---- project ├── columns: x:1!null y:2 u:3!null v:4 + ├── immutable ├── key: (1,3) ├── fd: (1)-->(2), (3)-->(4) └── inner-join (hash) ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null column6:6!null + ├── immutable ├── key: (1,3) ├── fd: (1)-->(2), (1,2)-->(5), (3)-->(4), (3,4)-->(6), (5)==(6), (6)==(5) ├── project │ ├── columns: column5:5 x:1!null y:2 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2), (1,2)-->(5) │ ├── scan xy @@ -2771,9 +2820,10 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── x:1 + y:2 [as=column5:5, outer=(1,2)] + │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] ├── project │ ├── columns: column6:6 u:3!null v:4 + │ ├── immutable │ ├── key: (3) │ ├── fd: (3)-->(4), (3,4)-->(6) │ ├── scan uv @@ -2781,7 +2831,7 @@ project │ │ ├── key: (3) │ │ └── fd: (3)-->(4) │ └── projections - │ └── u:3 + v:4 [as=column6:6, outer=(3,4)] + │ └── u:3 + v:4 [as=column6:6, outer=(3,4), immutable] └── filters └── column5:5 = column6:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] @@ -2791,15 +2841,18 @@ SELECT * FROM xy JOIN uv ON x+y=u AND x=u+v AND x*y+1=u*v+2 ---- project ├── columns: x:1!null y:2 u:3!null v:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2), (1,2)-->(3,4), (3)-->(4), (3,4)-->(1,2) └── inner-join (hash) ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null column6:6!null column7:7!null column8:8!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) + ├── immutable ├── key: (1) ├── fd: (1)-->(2), (1,2)-->(5,7), (3)-->(4), (3,4)-->(6,8), (3)==(5), (5)==(3), (1)==(6), (6)==(1), (7)==(8), (8)==(7) ├── project │ ├── columns: column7:7 column5:5 x:1!null y:2 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2), (1,2)-->(5,7) │ ├── scan xy @@ -2807,10 +2860,11 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ ├── (x:1 * y:2) + 1 [as=column7:7, outer=(1,2)] - │ └── x:1 + y:2 [as=column5:5, outer=(1,2)] + │ ├── (x:1 * y:2) + 1 [as=column7:7, outer=(1,2), immutable] + │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] ├── project │ ├── columns: column8:8 column6:6 u:3!null v:4 + │ ├── immutable │ ├── key: (3) │ ├── fd: (3)-->(4), (3,4)-->(6,8) │ ├── scan uv @@ -2818,8 +2872,8 @@ project │ │ ├── key: (3) │ │ └── fd: (3)-->(4) │ └── projections - │ ├── (u:3 * v:4) + 2 [as=column8:8, outer=(3,4)] - │ └── u:3 + v:4 [as=column6:6, outer=(3,4)] + │ ├── (u:3 * v:4) + 2 [as=column8:8, outer=(3,4), immutable] + │ └── u:3 + v:4 [as=column6:6, outer=(3,4), immutable] └── filters ├── column5:5 = u:3 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] ├── x:1 = column6:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] @@ -2831,15 +2885,18 @@ SELECT * FROM xy JOIN uv ON x+y=u AND x+u=v ---- project ├── columns: x:1!null y:2 u:3!null v:4!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2), (1,2)-->(3,4), (3)-->(4) └── inner-join (hash) ├── columns: x:1!null y:2 u:3!null v:4!null column5:5!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + ├── immutable ├── key: (1) ├── fd: (1)-->(2), (1,2)-->(5), (3)-->(4), (3)==(5), (5)==(3) ├── project │ ├── columns: column5:5 x:1!null y:2 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2), (1,2)-->(5) │ ├── scan xy @@ -2847,13 +2904,13 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── x:1 + y:2 [as=column5:5, outer=(1,2)] + │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] ├── scan uv │ ├── columns: u:3!null v:4 │ ├── key: (3) │ └── fd: (3)-->(4) └── filters - ├── v:4 = (x:1 + u:3) [outer=(1,3,4), constraints=(/4: (/NULL - ])] + ├── v:4 = (x:1 + u:3) [outer=(1,3,4), immutable, constraints=(/4: (/NULL - ])] └── column5:5 = u:3 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] # Cases with non-extractable equality. @@ -2882,6 +2939,7 @@ SELECT * FROM xy FULL OUTER JOIN uv ON x+y=1 full-join (cross) ├── columns: x:1 y:2 u:3 v:4 ├── multiplicity: left-rows(one-or-more), right-rows(one-or-more) + ├── immutable ├── key: (1,3) ├── fd: (1)-->(2), (3)-->(4) ├── scan xy @@ -2893,7 +2951,7 @@ full-join (cross) │ ├── key: (3) │ └── fd: (3)-->(4) └── filters - └── (x:1 + y:2) = 1 [outer=(1,2)] + └── (x:1 + y:2) = 1 [outer=(1,2), immutable] norm expect-not=ExtractJoinEqualities SELECT * FROM xy FULL OUTER JOIN uv ON 1=u+v @@ -2901,6 +2959,7 @@ SELECT * FROM xy FULL OUTER JOIN uv ON 1=u+v full-join (cross) ├── columns: x:1 y:2 u:3 v:4 ├── multiplicity: left-rows(one-or-more), right-rows(one-or-more) + ├── immutable ├── key: (1,3) ├── fd: (1)-->(2), (3)-->(4) ├── scan xy @@ -2912,7 +2971,7 @@ full-join (cross) │ ├── key: (3) │ └── fd: (3)-->(4) └── filters - └── (u:3 + v:4) = 1 [outer=(3,4)] + └── (u:3 + v:4) = 1 [outer=(3,4), immutable] norm expect-not=ExtractJoinEqualities SELECT * FROM xy INNER JOIN uv ON (SELECT k FROM a WHERE i=x)=u diff --git a/pkg/sql/opt/norm/testdata/rules/limit b/pkg/sql/opt/norm/testdata/rules/limit index 13976acf9325..a8bb6f0c9dcc 100644 --- a/pkg/sql/opt/norm/testdata/rules/limit +++ b/pkg/sql/opt/norm/testdata/rules/limit @@ -169,6 +169,7 @@ SELECT k, f*2.0 AS r FROM a LIMIT 5 project ├── columns: k:1!null r:6 ├── cardinality: [0 - 5] + ├── immutable ├── key: (1) ├── fd: (1)-->(6) ├── limit @@ -183,7 +184,7 @@ project │ │ └── limit hint: 5.00 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3)] + └── f:3 * 2.0 [as=r:6, outer=(3), immutable] norm expect=PushLimitIntoProject SELECT k, f*2.0 AS r FROM a ORDER BY k LIMIT 5 @@ -191,6 +192,7 @@ SELECT k, f*2.0 AS r FROM a ORDER BY k LIMIT 5 project ├── columns: k:1!null r:6 ├── cardinality: [0 - 5] + ├── immutable ├── key: (1) ├── fd: (1)-->(6) ├── ordering: +1 @@ -209,7 +211,7 @@ project │ │ └── limit hint: 5.00 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3)] + └── f:3 * 2.0 [as=r:6, outer=(3), immutable] # Don't push the limit through project when the ordering is on a # synthesized column. @@ -220,17 +222,20 @@ limit ├── columns: k:1!null r:6 ├── internal-ordering: +6 ├── cardinality: [0 - 5] + ├── immutable ├── key: (1) ├── fd: (1)-->(6) ├── ordering: +6 ├── sort │ ├── columns: k:1!null r:6 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(6) │ ├── ordering: +6 │ ├── limit hint: 5.00 │ └── project │ ├── columns: r:6 k:1!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(6) │ ├── scan a @@ -238,7 +243,7 @@ limit │ │ ├── key: (1) │ │ └── fd: (1)-->(3) │ └── projections - │ └── f:3 * 2.0 [as=r:6, outer=(3)] + │ └── f:3 * 2.0 [as=r:6, outer=(3), immutable] └── 5 @@ -249,6 +254,7 @@ SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i) a ORDER BY f LIMIT project ├── columns: f:3 r:6 ├── cardinality: [0 - 5] + ├── immutable ├── ordering: +3 ├── limit │ ├── columns: i:2 f:3 @@ -270,7 +276,7 @@ project │ │ └── columns: i:2 f:3 │ └── 5 └── projections - └── f:3 + 1.1 [as=r:6, outer=(3)] + └── f:3 + 1.1 [as=r:6, outer=(3), immutable] # Don't push negative limit into Scan. norm @@ -297,6 +303,7 @@ SELECT k, f*2.0 AS r FROM a OFFSET 5 ---- project ├── columns: k:1!null r:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(6) ├── offset @@ -309,13 +316,14 @@ project │ │ └── fd: (1)-->(3) │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3)] + └── f:3 * 2.0 [as=r:6, outer=(3), immutable] norm expect=PushOffsetIntoProject SELECT k, f*2.0 AS r FROM a ORDER BY k OFFSET 5 ---- project ├── columns: k:1!null r:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(6) ├── ordering: +1 @@ -332,7 +340,7 @@ project │ │ └── ordering: +1 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3)] + └── f:3 * 2.0 [as=r:6, outer=(3), immutable] # Don't push the offset through project when the ordering is on a # synthesized column. @@ -342,16 +350,19 @@ SELECT k, f*2.0 AS r FROM a ORDER BY r OFFSET 5 offset ├── columns: k:1!null r:6 ├── internal-ordering: +6 + ├── immutable ├── key: (1) ├── fd: (1)-->(6) ├── ordering: +6 ├── sort │ ├── columns: k:1!null r:6 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(6) │ ├── ordering: +6 │ └── project │ ├── columns: r:6 k:1!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(6) │ ├── scan a @@ -359,7 +370,7 @@ offset │ │ ├── key: (1) │ │ └── fd: (1)-->(3) │ └── projections - │ └── f:3 * 2.0 [as=r:6, outer=(3)] + │ └── f:3 * 2.0 [as=r:6, outer=(3), immutable] └── 5 # Detect PushOffsetIntoProject and FilterUnusedOffsetCols dependency cycle. @@ -368,6 +379,7 @@ SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i) a ORDER BY f OFFSET ---- project ├── columns: f:3 r:6 + ├── immutable ├── ordering: +3 ├── offset │ ├── columns: i:2 f:3 @@ -386,7 +398,7 @@ project │ │ └── columns: i:2 f:3 │ └── 5 └── projections - └── f:3 + 1.1 [as=r:6, outer=(3)] + └── f:3 + 1.1 [as=r:6, outer=(3), immutable] # -------------------------------------------------- # PushLimitIntoProject + PushOffsetIntoProject @@ -397,6 +409,7 @@ SELECT k, f*2.0 AS r FROM a OFFSET 5 LIMIT 10 project ├── columns: k:1!null r:6 ├── cardinality: [0 - 10] + ├── immutable ├── key: (1) ├── fd: (1)-->(6) ├── offset @@ -417,7 +430,7 @@ project │ │ └── 15 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3)] + └── f:3 * 2.0 [as=r:6, outer=(3), immutable] norm expect=(PushLimitIntoProject,PushOffsetIntoProject) SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i) a ORDER BY f OFFSET 5 LIMIT 10 @@ -425,6 +438,7 @@ SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i) a ORDER BY f OFFSET project ├── columns: f:3 r:6 ├── cardinality: [0 - 10] + ├── immutable ├── ordering: +3 ├── offset │ ├── columns: i:2 f:3 @@ -453,7 +467,7 @@ project │ │ └── 15 │ └── 5 └── projections - └── f:3 + 1.1 [as=r:6, outer=(3)] + └── f:3 + 1.1 [as=r:6, outer=(3), immutable] # -------------------------------------------------- # PushLimitIntoOffset diff --git a/pkg/sql/opt/norm/testdata/rules/numeric b/pkg/sql/opt/norm/testdata/rules/numeric index c95cd3ba267e..c4a57aeca72b 100644 --- a/pkg/sql/opt/norm/testdata/rules/numeric +++ b/pkg/sql/opt/norm/testdata/rules/numeric @@ -16,15 +16,16 @@ FROM a ---- project ├── columns: r:6 s:7 t:8 u:9 v:10 w:11 + ├── immutable ├── scan a │ └── columns: i:2 f:3 d:4 └── projections - ├── i:2 + i:2 [as=r:6, outer=(2)] - ├── i:2 + i:2 [as=s:7, outer=(2)] - ├── f:3 + f:3 [as=t:8, outer=(3)] - ├── f:3 + f:3 [as=u:9, outer=(3)] - ├── d:4 + d:4 [as=v:10, outer=(4)] - └── d:4 + d:4 [as=w:11, outer=(4)] + ├── i:2 + i:2 [as=r:6, outer=(2), immutable] + ├── i:2 + i:2 [as=s:7, outer=(2), immutable] + ├── f:3 + f:3 [as=t:8, outer=(3), immutable] + ├── f:3 + f:3 [as=u:9, outer=(3), immutable] + ├── d:4 + d:4 [as=v:10, outer=(4), immutable] + └── d:4 + d:4 [as=w:11, outer=(4), immutable] # Regression test for #35113. @@ -64,12 +65,13 @@ FROM a ---- project ├── columns: r:6 s:7 t:8 + ├── immutable ├── scan a │ └── columns: i:2 f:3 d:4 └── projections - ├── i:2 + i:2 [as=r:6, outer=(2)] - ├── f:3 + f:3 [as=s:7, outer=(3)] - └── d:4 + d:4 [as=t:8, outer=(4)] + ├── i:2 + i:2 [as=r:6, outer=(2), immutable] + ├── f:3 + f:3 [as=s:7, outer=(3), immutable] + └── d:4 + d:4 [as=t:8, outer=(4), immutable] # Regression test for #35113. norm expect=FoldMinusZero @@ -108,15 +110,16 @@ FROM a ---- project ├── columns: r:6 s:7 t:8 u:9 v:10 w:11 + ├── immutable ├── scan a │ └── columns: i:2 f:3 d:4 └── projections - ├── i:2 + i:2 [as=r:6, outer=(2)] - ├── i:2 + i:2 [as=s:7, outer=(2)] - ├── f:3 + f:3 [as=t:8, outer=(3)] - ├── f:3 + f:3 [as=u:9, outer=(3)] - ├── d:4 + d:4 [as=v:10, outer=(4)] - └── d:4 + d:4 [as=w:11, outer=(4)] + ├── i:2 + i:2 [as=r:6, outer=(2), immutable] + ├── i:2 + i:2 [as=s:7, outer=(2), immutable] + ├── f:3 + f:3 [as=t:8, outer=(3), immutable] + ├── f:3 + f:3 [as=u:9, outer=(3), immutable] + ├── d:4 + d:4 [as=v:10, outer=(4), immutable] + └── d:4 + d:4 [as=w:11, outer=(4), immutable] # Regression test for #35113. norm expect=FoldMultOne @@ -197,12 +200,13 @@ FROM a ---- project ├── columns: r:6 s:7 t:8 + ├── immutable ├── scan a │ └── columns: i:2 f:3 d:4 a.t:5 └── projections - ├── f:3 - f:3 [as=r:6, outer=(3)] - ├── i:2 - d:4 [as=s:7, outer=(2,4)] - └── a.t:5 - a.t:5 [as=t:8, outer=(5)] + ├── f:3 - f:3 [as=r:6, outer=(3), immutable] + ├── i:2 - d:4 [as=s:7, outer=(2,4), immutable] + └── a.t:5 - a.t:5 [as=t:8, outer=(5), immutable] # -------------------------------------------------- # EliminateUnaryMinus diff --git a/pkg/sql/opt/norm/testdata/rules/ordering b/pkg/sql/opt/norm/testdata/rules/ordering index 0240a6d35021..8ad1113dfab5 100644 --- a/pkg/sql/opt/norm/testdata/rules/ordering +++ b/pkg/sql/opt/norm/testdata/rules/ordering @@ -209,20 +209,23 @@ EXPLAIN SELECT b, b+1 AS plus, c FROM abcde ORDER BY b, plus, c ---- explain ├── columns: tree:7 field:8 description:9 + ├── immutable └── sort ├── columns: b:2 plus:6 c:3 + ├── immutable ├── lax-key: (2,3) ├── fd: (2)-->(6) ├── ordering: +2,+3 └── project ├── columns: plus:6 b:2 c:3 + ├── immutable ├── lax-key: (2,3) ├── fd: (2)-->(6) ├── scan abcde │ ├── columns: b:2 c:3 │ └── lax-key: (2,3) └── projections - └── b:2 + 1 [as=plus:6, outer=(2)] + └── b:2 + 1 [as=plus:6, outer=(2), immutable] # Regression: Explain a statement having constant column, but with no ordering. norm diff --git a/pkg/sql/opt/norm/testdata/rules/project b/pkg/sql/opt/norm/testdata/rules/project index f180b75d1886..6724fe951e74 100644 --- a/pkg/sql/opt/norm/testdata/rules/project +++ b/pkg/sql/opt/norm/testdata/rules/project @@ -86,11 +86,12 @@ SELECT 1+b.x FROM b LEFT JOIN a ON b.x = a.x ---- project ├── columns: "?column?":7!null + ├── immutable ├── scan b │ ├── columns: b.x:1!null │ └── key: (1) └── projections - └── b.x:1 + 1 [as="?column?":7, outer=(1)] + └── b.x:1 + 1 [as="?column?":7, outer=(1), immutable] # Case with no references to the left side. norm expect=EliminateJoinUnderProjectLeft @@ -124,6 +125,7 @@ SELECT b.x, b.z, 1+a.x FROM b LEFT JOIN a ON b.x = a.x ---- project ├── columns: x:1!null z:2 "?column?":7 + ├── immutable ├── key: (1) ├── fd: (1)-->(2,7) ├── left-join (hash) @@ -141,7 +143,7 @@ project │ └── filters │ └── b.x:1 = a.x:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] └── projections - └── a.x:3 + 1 [as="?column?":7, outer=(3)] + └── a.x:3 + 1 [as="?column?":7, outer=(3), immutable] # No-op case because r2 is nullable, and therefore rows may not match despite # the fact that it is a foreign key. @@ -306,6 +308,7 @@ SELECT y+1 AS r FROM (SELECT a.y FROM a, b WHERE a.x=b.x) a ---- project ├── columns: r:7 + ├── immutable ├── inner-join (hash) │ ├── columns: a.x:1!null y:2 b.x:5!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) @@ -321,7 +324,7 @@ project │ └── filters │ └── a.x:1 = b.x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── projections - └── y:2 + 1 [as=r:7, outer=(2)] + └── y:2 + 1 [as=r:7, outer=(2), immutable] # Outer and inner projections have synthesized columns. norm expect=MergeProjects @@ -329,11 +332,12 @@ SELECT y1, f+1 FROM (SELECT y+1 AS y1, f FROM a) ---- project ├── columns: y1:5 "?column?":6 + ├── immutable ├── scan a │ └── columns: y:2 f:3 └── projections - ├── f:3 + 1.0 [as="?column?":6, outer=(3)] - └── y:2 + 1 [as=y1:5, outer=(2)] + ├── f:3 + 1.0 [as="?column?":6, outer=(3), immutable] + └── y:2 + 1 [as=y1:5, outer=(2), immutable] # Multiple synthesized columns in both outer and inner projections. norm expect=MergeProjects @@ -341,15 +345,16 @@ SELECT y1, f+1, x2, s||'foo' FROM (SELECT y+1 AS y1, f, s, x*2 AS x2 FROM a) ---- project ├── columns: y1:5 "?column?":7 x2:6!null "?column?":8 + ├── immutable ├── scan a │ ├── columns: x:1!null y:2 f:3 s:4 │ ├── key: (1) │ └── fd: (1)-->(2-4) └── projections - ├── f:3 + 1.0 [as="?column?":7, outer=(3)] - ├── s:4 || 'foo' [as="?column?":8, outer=(4)] - ├── y:2 + 1 [as=y1:5, outer=(2)] - └── x:1 * 2 [as=x2:6, outer=(1)] + ├── f:3 + 1.0 [as="?column?":7, outer=(3), immutable] + ├── s:4 || 'foo' [as="?column?":8, outer=(4), immutable] + ├── y:2 + 1 [as=y1:5, outer=(2), immutable] + └── x:1 * 2 [as=x2:6, outer=(1), immutable] # Outer project selects subset of inner columns. norm expect=MergeProjects @@ -357,10 +362,11 @@ SELECT y1 FROM (SELECT y+1 AS y1, f*2 AS f2 FROM a) ---- project ├── columns: y1:5 + ├── immutable ├── scan a │ └── columns: y:2 └── projections - └── y:2 + 1 [as=y1:5, outer=(2)] + └── y:2 + 1 [as=y1:5, outer=(2), immutable] # Don't merge, since outer depends on inner. norm expect-not=MergeProjects @@ -368,14 +374,16 @@ SELECT y1*2, y1/2 FROM (SELECT y+1 AS y1 FROM a) ---- project ├── columns: "?column?":6 "?column?":7 + ├── immutable ├── project │ ├── columns: y1:5 + │ ├── immutable │ ├── scan a │ │ └── columns: y:2 │ └── projections - │ └── y:2 + 1 [as=y1:5, outer=(2)] + │ └── y:2 + 1 [as=y1:5, outer=(2), immutable] └── projections - ├── y1:5 * 2 [as="?column?":6, outer=(5)] + ├── y1:5 * 2 [as="?column?":6, outer=(5), immutable] └── y1:5 / 2 [as="?column?":7, outer=(5)] # Discard all inner columns. @@ -459,7 +467,7 @@ project │ ├── fd: ()-->(1) │ └── ($1::INT8,) └── projections - ├── column1:1 + 1 [as="?column?":3, outer=(1)] + ├── column1:1 + 1 [as="?column?":3, outer=(1), immutable] └── 3 [as="?column?":4] # -------------------------------------------------- @@ -525,16 +533,19 @@ SELECT (SELECT (tup).@1 * x FROM b) FROM (VALUES ((1,2)), ((3,4))) AS v(tup) project ├── columns: "?column?":5 ├── cardinality: [1 - ] + ├── immutable ├── ensure-distinct-on │ ├── columns: "?column?":4 rownum:8!null │ ├── grouping columns: rownum:8!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── cardinality: [1 - ] + │ ├── immutable │ ├── key: (8) │ ├── fd: (8)-->(4) │ ├── left-join-apply │ │ ├── columns: "?column?":4 column1_1:6!null rownum:8!null │ │ ├── cardinality: [2 - ] + │ │ ├── immutable │ │ ├── fd: (8)-->(6) │ │ ├── ordinality │ │ │ ├── columns: column1_1:6!null rownum:8!null @@ -549,11 +560,12 @@ project │ │ ├── project │ │ │ ├── columns: "?column?":4 │ │ │ ├── outer: (6) + │ │ │ ├── immutable │ │ │ ├── scan b │ │ │ │ ├── columns: x:2!null │ │ │ │ └── key: (2) │ │ │ └── projections - │ │ │ └── x:2 * column1_1:6 [as="?column?":4, outer=(2,6)] + │ │ │ └── x:2 * column1_1:6 [as="?column?":4, outer=(2,6), immutable] │ │ └── filters (true) │ └── aggregations │ └── const-agg [as="?column?":4, outer=(4)] @@ -838,6 +850,7 @@ SELECT x*1, x+1 FROM (VALUES (1), (2)) f(x) project ├── columns: "?column?":2!null "?column?":3!null ├── cardinality: [2 - 2] + ├── immutable ├── fd: (2)-->(3) ├── values │ ├── columns: "?column?":2!null @@ -845,7 +858,7 @@ project │ ├── (1,) │ └── (2,) └── projections - └── "?column?":2 + 1 [as="?column?":3, outer=(2)] + └── "?column?":2 + 1 [as="?column?":3, outer=(2), immutable] # Case with a subquery reference to a remapped column. norm expect=PushColumnRemappingIntoValues diff --git a/pkg/sql/opt/norm/testdata/rules/project_set b/pkg/sql/opt/norm/testdata/rules/project_set index 805bce7ba816..970727a0c8d7 100644 --- a/pkg/sql/opt/norm/testdata/rules/project_set +++ b/pkg/sql/opt/norm/testdata/rules/project_set @@ -192,8 +192,10 @@ SELECT unnest(ARRAY[x,y]), unnest(ARRAY[1,x*100]) FROM xy ---- project ├── columns: unnest:4 unnest:5 + ├── immutable └── inner-join-apply ├── columns: x:1!null y:2 unnest:4 unnest:5 + ├── immutable ├── fd: (1)-->(2) ├── scan xy │ ├── columns: x:1!null y:2 @@ -203,6 +205,7 @@ project │ ├── columns: unnest:4 unnest:5 │ ├── outer: (1,2) │ ├── cardinality: [2 - 2] + │ ├── immutable │ ├── (x:1, 1) │ └── (y:2, x:1 * 100) └── filters (true) diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index 51fd55beee7a..ac78d128c501 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -53,11 +53,12 @@ SELECT k1*2 FROM (SELECT k+1 AS k1, i+1 FROM a) a ---- project ├── columns: "?column?":7!null + ├── immutable ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── (k:1 + 1) * 2 [as="?column?":7, outer=(1)] + └── (k:1 + 1) * 2 [as="?column?":7, outer=(1), immutable] # Use column values within computed column. norm expect=PruneProjectCols @@ -94,7 +95,7 @@ project │ └── projections │ └── length(s:4) [as=l:5, outer=(4), immutable] └── projections - └── l:5 * 2 [as="?column?":6, outer=(5)] + └── l:5 * 2 [as="?column?":6, outer=(5), immutable] # Compute column based on another computed column. norm expect=PruneProjectCols @@ -117,7 +118,7 @@ project │ └── projections │ └── length(s:4) [as=l:5, outer=(4), immutable] └── projections - └── l:5 * l:5 [as=r:6, outer=(5)] + └── l:5 * l:5 [as=r:6, outer=(5), immutable] # -------------------------------------------------- # PruneScanCols @@ -137,6 +138,7 @@ SELECT k, k+1 AS r, i+1 AS s FROM a ---- project ├── columns: k:1!null r:5!null s:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(5,6) ├── scan a @@ -144,8 +146,8 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - ├── k:1 + 1 [as=r:5, outer=(1)] - └── i:2 + 1 [as=s:6, outer=(2)] + ├── k:1 + 1 [as=r:5, outer=(1), immutable] + └── i:2 + 1 [as=s:6, outer=(2), immutable] # Use columns only in computed columns. norm expect=PruneScanCols @@ -153,12 +155,13 @@ SELECT k+i AS r FROM a ---- project ├── columns: r:5 + ├── immutable ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - └── k:1 + i:2 [as=r:5, outer=(1,2)] + └── k:1 + i:2 [as=r:5, outer=(1,2), immutable] # Use no scan columns. norm expect=PruneScanCols @@ -232,8 +235,10 @@ SELECT i-1 AS r, k*k AS t FROM a WHERE k+1<5 AND s||'o'='foo' ---- project ├── columns: r:5 t:6!null + ├── immutable ├── select │ ├── columns: k:1!null i:2 s:4 + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2,4) │ ├── scan a @@ -242,10 +247,10 @@ project │ │ └── fd: (1)-->(2,4) │ └── filters │ ├── k:1 < 4 [outer=(1), constraints=(/1: (/NULL - /3]; tight)] - │ └── (s:4 || 'o') = 'foo' [outer=(4)] + │ └── (s:4 || 'o') = 'foo' [outer=(4), immutable] └── projections - ├── i:2 - 1 [as=r:5, outer=(2)] - └── k:1 * k:1 [as=t:6, outer=(1)] + ├── i:2 - 1 [as=r:5, outer=(2), immutable] + └── k:1 * k:1 [as=t:6, outer=(1), immutable] # Select nested in select. norm expect=PruneSelectCols @@ -290,8 +295,10 @@ SELECT f, f+1.1 AS r FROM (SELECT f, k FROM a GROUP BY f, k HAVING sum(k)=100) a ---- project ├── columns: f:3 r:6 + ├── immutable ├── select │ ├── columns: k:1!null f:3 sum:5!null + │ ├── immutable │ ├── key: (1) │ ├── fd: ()-->(5), (1)-->(3) │ ├── group-by @@ -309,9 +316,9 @@ project │ │ └── const-agg [as=f:3, outer=(3)] │ │ └── f:3 │ └── filters - │ └── sum:5 = 100 [outer=(5), constraints=(/5: [/100 - /100]; tight), fd=()-->(5)] + │ └── sum:5 = 100 [outer=(5), immutable, constraints=(/5: [/100 - /100]; tight), fd=()-->(5)] └── projections - └── f:3 + 1.1 [as=r:6, outer=(3)] + └── f:3 + 1.1 [as=r:6, outer=(3), immutable] # -------------------------------------------------- # PruneLimitCols @@ -416,6 +423,7 @@ SELECT f, f*2.0 AS r FROM (SELECT f, s FROM a GROUP BY f, s LIMIT 5) a project ├── columns: f:3 r:5 ├── cardinality: [0 - 5] + ├── immutable ├── limit │ ├── columns: f:3 s:4 │ ├── cardinality: [0 - 5] @@ -430,7 +438,7 @@ project │ │ └── limit hint: 6.02 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:5, outer=(3)] + └── f:3 * 2.0 [as=r:5, outer=(3), immutable] # -------------------------------------------------- # PruneOffsetCols @@ -657,6 +665,7 @@ SELECT f, f*2.0 AS r FROM (SELECT f, s FROM a GROUP BY f, s OFFSET 5 LIMIT 5) a project ├── columns: f:3 r:5 ├── cardinality: [0 - 5] + ├── immutable ├── offset │ ├── columns: f:3 s:4 │ ├── cardinality: [0 - 5] @@ -676,7 +685,7 @@ project │ │ └── 10 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:5, outer=(3)] + └── f:3 * 2.0 [as=r:5, outer=(3), immutable] # -------------------------------------------------- # PruneJoinLeftCols @@ -733,6 +742,7 @@ SELECT a.k+1 AS r, xy.* FROM a FULL JOIN xy ON True ---- project ├── columns: r:7 x:5 y:6 + ├── immutable ├── fd: (5)-->(6) ├── full-join (cross) │ ├── columns: k:1 x:5 y:6 @@ -748,7 +758,7 @@ project │ │ └── fd: (5)-->(6) │ └── filters (true) └── projections - └── k:1 + 1 [as=r:7, outer=(1)] + └── k:1 + 1 [as=r:7, outer=(1), immutable] # No columns needed from left side of join. norm expect=PruneJoinLeftCols @@ -770,18 +780,22 @@ SELECT a.k+1 AS r, a.i/2 AS s, xy.* FROM a INNER JOIN xy ON a.k*a.k=xy.x AND a.s ---- project ├── columns: r:8!null s:9 x:5!null y:6 + ├── immutable ├── fd: (5)-->(6) ├── inner-join (hash) │ ├── columns: k:1!null i:2 x:5!null y:6 column7:7!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2,7), (5)-->(6), (5)==(7), (7)==(5) │ ├── project │ │ ├── columns: column7:7!null k:1!null i:2 + │ │ ├── immutable │ │ ├── key: (1) │ │ ├── fd: (1)-->(2,7) │ │ ├── select │ │ │ ├── columns: k:1!null i:2 a.s:4 + │ │ │ ├── immutable │ │ │ ├── key: (1) │ │ │ ├── fd: (1)-->(2,4) │ │ │ ├── scan a @@ -789,9 +803,9 @@ project │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2,4) │ │ │ └── filters - │ │ │ └── (a.s:4 || 'o') = 'foo' [outer=(4)] + │ │ │ └── (a.s:4 || 'o') = 'foo' [outer=(4), immutable] │ │ └── projections - │ │ └── k:1 * k:1 [as=column7:7, outer=(1)] + │ │ └── k:1 * k:1 [as=column7:7, outer=(1), immutable] │ ├── scan xy │ │ ├── columns: x:5!null y:6 │ │ ├── key: (5) @@ -799,7 +813,7 @@ project │ └── filters │ └── column7:7 = x:5 [outer=(5,7), constraints=(/5: (/NULL - ]; /7: (/NULL - ]), fd=(5)==(7), (7)==(5)] └── projections - ├── k:1 + 1 [as=r:8, outer=(1)] + ├── k:1 + 1 [as=r:8, outer=(1), immutable] └── i:2 / 2 [as=s:9, outer=(2)] # Join that is nested in another join. @@ -849,20 +863,24 @@ WHERE (SELECT k+1 AS r FROM xy WHERE y=k) = 1 ---- project ├── columns: k:1!null i:2 + ├── immutable ├── key: (1) ├── fd: (1)-->(2) └── select ├── columns: k:1!null i:2 r:7!null + ├── immutable ├── key: (1) ├── fd: ()-->(7), (1)-->(2) ├── ensure-distinct-on │ ├── columns: k:1!null i:2 r:7 │ ├── grouping columns: k:1!null │ ├── error: "more than one row returned by a subquery used as an expression" + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2,7) │ ├── left-join-apply │ │ ├── columns: k:1!null i:2 y:6 r:7 + │ │ ├── immutable │ │ ├── fd: (1)-->(2) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 @@ -871,11 +889,12 @@ project │ │ ├── project │ │ │ ├── columns: r:7 y:6 │ │ │ ├── outer: (1) + │ │ │ ├── immutable │ │ │ ├── fd: ()-->(7) │ │ │ ├── scan xy │ │ │ │ └── columns: y:6 │ │ │ └── projections - │ │ │ └── k:1 + 1 [as=r:7, outer=(1)] + │ │ │ └── k:1 + 1 [as=r:7, outer=(1), immutable] │ │ └── filters │ │ └── y:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] │ └── aggregations @@ -1012,6 +1031,7 @@ SELECT xy.*, a.k+1 AS r FROM xy FULL JOIN a ON True ---- project ├── columns: x:1 y:2 r:7 + ├── immutable ├── fd: (1)-->(2) ├── full-join (cross) │ ├── columns: x:1 y:2 k:3 @@ -1027,7 +1047,7 @@ project │ │ └── key: (3) │ └── filters (true) └── projections - └── k:3 + 1 [as=r:7, outer=(3)] + └── k:3 + 1 [as=r:7, outer=(3), immutable] # No columns needed from right side of join. norm expect=PruneJoinRightCols @@ -1049,10 +1069,12 @@ SELECT xy.*, a.k+1 AS r, a.i/2 AS s FROM xy INNER JOIN a ON xy.x=a.k*a.k AND a.s ---- project ├── columns: x:1!null y:2 r:8!null s:9 + ├── immutable ├── fd: (1)-->(2) ├── inner-join (hash) │ ├── columns: x:1!null y:2 k:3!null i:4 column7:7!null │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) + │ ├── immutable │ ├── key: (3) │ ├── fd: (1)-->(2), (3)-->(4,7), (1)==(7), (7)==(1) │ ├── scan xy @@ -1061,10 +1083,12 @@ project │ │ └── fd: (1)-->(2) │ ├── project │ │ ├── columns: column7:7!null k:3!null i:4 + │ │ ├── immutable │ │ ├── key: (3) │ │ ├── fd: (3)-->(4,7) │ │ ├── select │ │ │ ├── columns: k:3!null i:4 a.s:6 + │ │ │ ├── immutable │ │ │ ├── key: (3) │ │ │ ├── fd: (3)-->(4,6) │ │ │ ├── scan a @@ -1072,13 +1096,13 @@ project │ │ │ │ ├── key: (3) │ │ │ │ └── fd: (3)-->(4,6) │ │ │ └── filters - │ │ │ └── (a.s:6 || 'o') = 'foo' [outer=(6)] + │ │ │ └── (a.s:6 || 'o') = 'foo' [outer=(6), immutable] │ │ └── projections - │ │ └── k:3 * k:3 [as=column7:7, outer=(3)] + │ │ └── k:3 * k:3 [as=column7:7, outer=(3), immutable] │ └── filters │ └── x:1 = column7:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── projections - ├── k:3 + 1 [as=r:8, outer=(3)] + ├── k:3 + 1 [as=r:8, outer=(3), immutable] └── i:4 / 2 [as=s:9, outer=(4)] # Join that is nested in another join. @@ -1144,6 +1168,7 @@ SELECT a.k, xy.x, a.k+xy.x AS r FROM a LEFT JOIN xy ON a.k=xy.x ---- project ├── columns: k:1!null x:5 r:7 + ├── immutable ├── key: (1) ├── fd: (1)-->(5), (1,5)-->(7) ├── left-join (hash) @@ -1160,7 +1185,7 @@ project │ └── filters │ └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── projections - └── k:1 + x:5 [as=r:7, outer=(1,5)] + └── k:1 + x:5 [as=r:7, outer=(1,5), immutable] # -------------------------------------------------- # PruneAggCols @@ -1344,6 +1369,7 @@ SELECT s, i+1 AS r FROM a GROUP BY i, s, s||'foo' ---- project ├── columns: s:4 r:6 + ├── immutable ├── distinct-on │ ├── columns: i:2 s:4 │ ├── grouping columns: i:2 s:4 @@ -1351,7 +1377,7 @@ project │ └── scan a │ └── columns: i:2 s:4 └── projections - └── i:2 + 1 [as=r:6, outer=(2)] + └── i:2 + 1 [as=r:6, outer=(2), immutable] # Groupby a groupby. norm expect=PruneGroupByCols @@ -1400,14 +1426,16 @@ SELECT icnt FROM (SELECT count(i+1) AS icnt, count(k+1) FROM a); scalar-group-by ├── columns: icnt:6!null ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(6) ├── project │ ├── columns: column5:5 + │ ├── immutable │ ├── scan a │ │ └── columns: i:2 │ └── projections - │ └── i:2 + 1 [as=column5:5, outer=(2)] + │ └── i:2 + 1 [as=column5:5, outer=(2), immutable] └── aggregations └── count [as=count:6, outer=(5)] └── column5:5 @@ -1499,9 +1527,11 @@ SELECT k FROM (SELECT k, min(s) FROM a GROUP BY k HAVING sum(i) > 5) ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── select ├── columns: k:1!null sum:6!null + ├── immutable ├── key: (1) ├── fd: (1)-->(6) ├── group-by @@ -1517,7 +1547,7 @@ project │ └── sum [as=sum:6, outer=(2)] │ └── i:2 └── filters - └── sum:6 > 5 [outer=(6), constraints=(/6: (/5 - ]; tight)] + └── sum:6 > 5 [outer=(6), immutable, constraints=(/6: (/5 - ]; tight)] # -------------------------------------------------- # PruneOrdinalityCols @@ -1822,16 +1852,19 @@ delete a ├── columns: k:5!null column9:9!null ├── internal-ordering: +9 ├── cardinality: [0 - 10] + ├── immutable ├── key: (5) ├── fd: (5)-->(9) ├── sort │ ├── columns: k:5!null column9:9!null + │ ├── immutable │ ├── key: (5) │ ├── fd: (5)-->(9) │ ├── ordering: +9 │ ├── limit hint: 10.00 │ └── project │ ├── columns: column9:9!null k:5!null + │ ├── immutable │ ├── key: (5) │ ├── fd: (5)-->(9) │ ├── select @@ -1845,7 +1878,7 @@ delete a │ │ └── filters │ │ └── i:6 > 0 [outer=(6), constraints=(/6: [/1 - ]; tight)] │ └── projections - │ └── i:6 * 2 [as=column9:9, outer=(6)] + │ └── i:6 * 2 [as=column9:9, outer=(6), immutable] └── 10 # Prune when a secondary index is present on the table. @@ -1931,6 +1964,7 @@ update "family" ├── volatile, side-effects, mutations └── project ├── columns: a_new:11!null a:6!null b:7 c:8 d:9 e:10 + ├── immutable ├── key: (6) ├── fd: (6)-->(7-11) ├── select @@ -1944,7 +1978,7 @@ update "family" │ └── filters │ └── a:6 > 100 [outer=(6), constraints=(/6: [/101 - ]; tight)] └── projections - └── a:6 + 1 [as=a_new:11, outer=(6)] + └── a:6 + 1 [as=a_new:11, outer=(6), immutable] # Do not prune columns that must be returned. norm expect=(PruneMutationFetchCols, PruneMutationReturnCols) @@ -1963,6 +1997,7 @@ project ├── fd: (1)-->(2) └── project ├── columns: c_new:11 a:6!null b:7 c:8 d:9 + ├── immutable ├── key: (6) ├── fd: (6)-->(7-9), (8)-->(11) ├── scan "family" @@ -1970,7 +2005,7 @@ project │ ├── key: (6) │ └── fd: (6)-->(7-9) └── projections - └── c:8 + 1 [as=c_new:11, outer=(8)] + └── c:8 + 1 [as=c_new:11, outer=(8), immutable] # Prune unused upsert columns. norm expect=PruneMutationInputCols @@ -1992,6 +2027,7 @@ upsert a └── project ├── columns: upsert_i:15 column1:5!null column2:6!null column7:7 column8:8 k:9 i:10 f:11 s:12 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(5-12,15) ├── left-join (cross) @@ -2019,7 +2055,7 @@ upsert a │ │ └── k:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] │ └── filters (true) └── projections - └── CASE WHEN k:9 IS NULL THEN column7:7 ELSE i:10 + 1 END [as=upsert_i:15, outer=(7,9,10)] + └── CASE WHEN k:9 IS NULL THEN column7:7 ELSE i:10 + 1 END [as=upsert_i:15, outer=(7,9,10), immutable] # Prune update columns replaced by upsert columns. # TODO(andyk): Need to also prune output columns. @@ -2049,6 +2085,7 @@ upsert a └── project ├── columns: upsert_k:14 upsert_i:15 upsert_f:16 upsert_s:17 column1:5!null column2:6!null column7:7 column8:8 k:9 i:10 f:11 s:12 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(5-12,14-17) ├── left-join (cross) @@ -2077,7 +2114,7 @@ upsert a │ └── filters (true) └── projections ├── CASE WHEN k:9 IS NULL THEN column1:5 ELSE k:9 END [as=upsert_k:14, outer=(5,9)] - ├── CASE WHEN k:9 IS NULL THEN column7:7 ELSE i:10 + 1 END [as=upsert_i:15, outer=(7,9,10)] + ├── CASE WHEN k:9 IS NULL THEN column7:7 ELSE i:10 + 1 END [as=upsert_i:15, outer=(7,9,10), immutable] ├── CASE WHEN k:9 IS NULL THEN column8:8 ELSE f:11 END [as=upsert_f:16, outer=(8,9,11)] └── CASE WHEN k:9 IS NULL THEN column2:6 ELSE s:12 END [as=upsert_s:17, outer=(6,9,12)] @@ -2326,6 +2363,7 @@ project ├── fd: (8)-->(1-7) └── project ├── columns: a_new:17 a:9 b:10 c:11 d:12 e:13 f:14 g:15 rowid:16!null + ├── immutable ├── key: (16) ├── fd: (16)-->(9-15), (9)~~>(10-16), (9)-->(17) ├── scan returning_test @@ -2333,7 +2371,7 @@ project │ ├── key: (16) │ └── fd: (16)-->(9-15), (9)~~>(10-16) └── projections - └── a:9 + 1 [as=a_new:17, outer=(9)] + └── a:9 + 1 [as=a_new:17, outer=(9), immutable] # Fetch all the columns in the (d, e, f, g) family as d is being set. @@ -2355,6 +2393,7 @@ project ├── fd: (8)-->(1,4), (1)~~>(4,8) └── project ├── columns: d_new:17 a:9 d:12 e:13 f:14 g:15 rowid:16!null + ├── immutable ├── key: (16) ├── fd: (16)-->(9,12-15), (9)~~>(12-16), (9,12)-->(17) ├── scan returning_test @@ -2362,7 +2401,7 @@ project │ ├── key: (16) │ └── fd: (16)-->(9,12-15), (9)~~>(12-16) └── projections - └── a:9 + d:12 [as=d_new:17, outer=(9,12)] + └── a:9 + d:12 [as=d_new:17, outer=(9,12), immutable] # Fetch only whats being updated (not the (d, e, f, g) family). norm @@ -2381,6 +2420,7 @@ project ├── fd: (8)-->(1) └── project ├── columns: a_new:17 a:9 rowid:16!null + ├── immutable ├── key: (16) ├── fd: (16)-->(9,17), (9)~~>(16,17) ├── scan returning_test @@ -2388,7 +2428,7 @@ project │ ├── key: (16) │ └── fd: (16)-->(9,12), (9)~~>(12,16) └── projections - └── a:9 + d:12 [as=a_new:17, outer=(9,12)] + └── a:9 + d:12 [as=a_new:17, outer=(9,12), immutable] # We only fetch the minimal set of columns which is (a, b, c, rowid). norm @@ -2410,6 +2450,7 @@ project ├── fd: (8)-->(1-3), (2)~~>(1,3,8) └── project ├── columns: a_new:17 a:9 b:10 c:11 rowid:16!null + ├── immutable ├── key: (16) ├── fd: (16)-->(9-11), (9)~~>(10,11,16), (9,10)-->(17) ├── scan returning_test @@ -2417,7 +2458,7 @@ project │ ├── key: (16) │ └── fd: (16)-->(9-11), (9)~~>(10,11,16) └── projections - └── a:9 + b:10 [as=a_new:17, outer=(9,10)] + └── a:9 + b:10 [as=a_new:17, outer=(9,10), immutable] # We apply the PruneMutationReturnCols rule multiple times, to get @@ -2442,6 +2483,7 @@ with &1 │ ├── fd: (8)-->(1-3) │ └── project │ ├── columns: a_new:17 returning_test.a:9 returning_test.b:10 returning_test.c:11 rowid:16!null + │ ├── immutable │ ├── key: (16) │ ├── fd: (16)-->(9-11), (9)~~>(10,11,16), (9)-->(17) │ ├── scan returning_test @@ -2449,7 +2491,7 @@ with &1 │ │ ├── key: (16) │ │ └── fd: (16)-->(9-11), (9)~~>(10,11,16) │ └── projections - │ └── returning_test.a:9 + 1 [as=a_new:17, outer=(9)] + │ └── returning_test.a:9 + 1 [as=a_new:17, outer=(9), immutable] └── project ├── columns: a:21 ├── with-scan &1 @@ -2482,6 +2524,7 @@ with &1 │ ├── fd: (8)-->(1-3) │ └── project │ ├── columns: a_new:17 returning_test.a:9 returning_test.b:10 returning_test.c:11 rowid:16!null + │ ├── immutable │ ├── key: (16) │ ├── fd: (16)-->(9-11), (9)~~>(10,11,16), (9)-->(17) │ ├── scan returning_test @@ -2489,7 +2532,7 @@ with &1 │ │ ├── key: (16) │ │ └── fd: (16)-->(9-11), (9)~~>(10,11,16) │ └── projections - │ └── returning_test.a:9 + 1 [as=a_new:17, outer=(9)] + │ └── returning_test.a:9 + 1 [as=a_new:17, outer=(9), immutable] └── project ├── columns: a:21!null ├── select @@ -2528,6 +2571,7 @@ with &2 │ ├── fd: (18)-->(11-13) │ └── project │ ├── columns: a_new:27 returning_test.a:19 returning_test.b:20 returning_test.c:21 rowid:26!null + │ ├── immutable │ ├── key: (26) │ ├── fd: (26)-->(19-21), (19)~~>(20,21,26), (19)-->(27) │ ├── scan returning_test @@ -2535,7 +2579,7 @@ with &2 │ │ ├── key: (26) │ │ └── fd: (26)-->(19-21), (19)~~>(20,21,26) │ └── projections - │ └── returning_test.a:19 + 1 [as=a_new:27, outer=(19)] + │ └── returning_test.a:19 + 1 [as=a_new:27, outer=(19), immutable] └── inner-join (cross) ├── columns: a:9 b:10 a:31!null b:32 ├── fd: (9)~~>(10) @@ -2633,7 +2677,7 @@ project │ │ └── a:14 = 1 [outer=(14), constraints=(/14: [/1 - /1]; tight), fd=()-->(14)] │ └── filters (true) └── projections - ├── CASE WHEN rowid:21 IS NULL THEN column1:9 ELSE column1:9 + a:14 END [as=upsert_a:23, outer=(9,14,21)] + ├── CASE WHEN rowid:21 IS NULL THEN column1:9 ELSE column1:9 + a:14 END [as=upsert_a:23, outer=(9,14,21), immutable] ├── CASE WHEN rowid:21 IS NULL THEN column2:10 ELSE b:15 END [as=upsert_b:24, outer=(10,15,21)] ├── CASE WHEN rowid:21 IS NULL THEN column3:11 ELSE c:16 END [as=upsert_c:25, outer=(11,16,21)] └── CASE WHEN rowid:21 IS NULL THEN column13:13 ELSE rowid:21 END [as=upsert_rowid:30, outer=(13,21)] @@ -2654,6 +2698,7 @@ project ├── fd: (8)-->(1,2,4), (1)-->(2,4,8) └── select ├── columns: a:9!null b:10 d:12 rowid:16!null + ├── immutable ├── key: (16) ├── fd: (16)-->(9,10,12), (9)-->(10,12,16) ├── scan returning_test @@ -2661,7 +2706,7 @@ project │ ├── key: (16) │ └── fd: (16)-->(9,10,12), (9)~~>(10,12,16) └── filters - └── a:9 < (b:10 + d:12) [outer=(9,10,12), constraints=(/9: (/NULL - ])] + └── a:9 < (b:10 + d:12) [outer=(9,10,12), immutable, constraints=(/9: (/NULL - ])] norm UPSERT INTO returning_test (a, b, c) VALUES (1, 2, 'c') RETURNING a, b, c, d @@ -2811,6 +2856,7 @@ SELECT a, b, c FROM abcde WHERE EXISTS (SELECT * FROM family WHERE abcde.a=famil ---- semi-join (hash) ├── columns: a:1!null b:2 c:3 + ├── immutable ├── key: (1) ├── fd: (1)-->(2,3), (2,3)~~>(1) ├── scan abcde @@ -2823,7 +2869,7 @@ semi-join (hash) │ └── fd: (6)-->(7,8) └── filters ├── abcde.a:1 = "family".a:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── abcde.b:2 > ("family".b:7 + "family".c:8) [outer=(2,7,8), constraints=(/2: (/NULL - ])] + └── abcde.b:2 > ("family".b:7 + "family".c:8) [outer=(2,7,8), immutable, constraints=(/2: (/NULL - ])] norm expect=PruneSemiAntiJoinRightCols SELECT a, b, c FROM abcde WHERE NOT EXISTS (SELECT * FROM family WHERE abcde.a=family.a) diff --git a/pkg/sql/opt/norm/testdata/rules/reject_nulls b/pkg/sql/opt/norm/testdata/rules/reject_nulls index de7cbf3b88cf..a32bbef2785d 100644 --- a/pkg/sql/opt/norm/testdata/rules/reject_nulls +++ b/pkg/sql/opt/norm/testdata/rules/reject_nulls @@ -256,9 +256,11 @@ HAVING sum(DISTINCT y)=1 ---- project ├── columns: sum:7!null + ├── immutable ├── fd: ()-->(7) └── select ├── columns: k:1!null sum:7!null + ├── immutable ├── key: (1) ├── fd: ()-->(7) ├── group-by @@ -283,7 +285,7 @@ project │ └── sum │ └── y:6 └── filters - └── sum:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── sum:7 = 1 [outer=(7), immutable, constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] # Single max aggregate function without grouping columns. norm expect=RejectNullsGroupBy @@ -651,18 +653,22 @@ HAVING string_agg(s || 'bar', ',')='foo' ---- project ├── columns: string_agg:9!null + ├── immutable ├── fd: ()-->(9) └── select ├── columns: k:3 string_agg:9!null + ├── immutable ├── key: (3) ├── fd: ()-->(9) ├── group-by │ ├── columns: k:3 string_agg:9 │ ├── grouping columns: k:3 + │ ├── immutable │ ├── key: (3) │ ├── fd: (3)-->(9) │ ├── project │ │ ├── columns: column7:7 column8:8!null k:3 + │ │ ├── immutable │ │ ├── fd: ()-->(8), (3)-->(7) │ │ ├── left-join (cross) │ │ │ ├── columns: k:3 s:6 @@ -675,7 +681,7 @@ project │ │ │ │ └── fd: (3)-->(6) │ │ │ └── filters (true) │ │ └── projections - │ │ ├── s:6 || 'bar' [as=column7:7, outer=(6)] + │ │ ├── s:6 || 'bar' [as=column7:7, outer=(6), immutable] │ │ └── ',' [as=column8:8] │ └── aggregations │ └── string-agg [as=string_agg:9, outer=(7,8)] @@ -716,15 +722,18 @@ exprnorm select ├── columns: sum:6!null ├── cardinality: [0 - 1] + ├── immutable ├── key: () ├── fd: ()-->(6) ├── scalar-group-by │ ├── columns: sum:6 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () │ ├── fd: ()-->(6) │ ├── inner-join-apply │ │ ├── columns: x:1!null u:3!null z:5 + │ │ ├── immutable │ │ ├── key: (1,3) │ │ ├── fd: (1,3)-->(5) │ │ ├── scan xy @@ -733,6 +742,7 @@ select │ │ ├── left-join-apply │ │ │ ├── columns: u:3!null z:5 │ │ │ ├── outer: (1) + │ │ │ ├── immutable │ │ │ ├── key: (3) │ │ │ ├── fd: (3)-->(5) │ │ │ ├── scan uv @@ -742,6 +752,7 @@ select │ │ │ │ ├── columns: z:5 │ │ │ │ ├── outer: (1,3) │ │ │ │ ├── cardinality: [1 - 1] + │ │ │ │ ├── immutable │ │ │ │ ├── key: () │ │ │ │ ├── fd: ()-->(5) │ │ │ │ └── (x:1 + u:3,) diff --git a/pkg/sql/opt/norm/testdata/rules/scalar b/pkg/sql/opt/norm/testdata/rules/scalar index 724008efdfc5..67b7f53cf36b 100644 --- a/pkg/sql/opt/norm/testdata/rules/scalar +++ b/pkg/sql/opt/norm/testdata/rules/scalar @@ -28,20 +28,21 @@ FROM a ---- project ├── columns: r:7 s:8 t:9!null u:10!null v:11 w:12 x:13 y:14 z:15 + ├── immutable ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - ├── k:1 = (i:2 + 1) [as=r:7, outer=(1,2)] - ├── i:2 != (2 - k:1) [as=s:8, outer=(1,2)] - ├── k:1 IS NOT DISTINCT FROM (i:2 + 1) [as=t:9, outer=(1,2)] - ├── k:1 IS DISTINCT FROM (i:2 - 1) [as=u:10, outer=(1,2)] - ├── k:1 + (i:2 * 2) [as=v:11, outer=(1,2)] - ├── k:1 * (i:2 + 2) [as=w:12, outer=(1,2)] - ├── k:1 & (i:2 ^ 2) [as=x:13, outer=(1,2)] - ├── k:1 | (i:2 ^ 2) [as=y:14, outer=(1,2)] - └── k:1 # (i:2 * i:2) [as=z:15, outer=(1,2)] + ├── k:1 = (i:2 + 1) [as=r:7, outer=(1,2), immutable] + ├── i:2 != (2 - k:1) [as=s:8, outer=(1,2), immutable] + ├── k:1 IS NOT DISTINCT FROM (i:2 + 1) [as=t:9, outer=(1,2), immutable] + ├── k:1 IS DISTINCT FROM (i:2 - 1) [as=u:10, outer=(1,2), immutable] + ├── k:1 + (i:2 * 2) [as=v:11, outer=(1,2), immutable] + ├── k:1 * (i:2 + 2) [as=w:12, outer=(1,2), immutable] + ├── k:1 & (i:2 ^ 2) [as=x:13, outer=(1,2), immutable] + ├── k:1 | (i:2 ^ 2) [as=y:14, outer=(1,2), immutable] + └── k:1 # (i:2 * i:2) [as=z:15, outer=(1,2), immutable] # -------------------------------------------------- # CommuteConst @@ -62,20 +63,21 @@ FROM a ---- project ├── columns: r:7 s:8 t:9!null u:10!null v:11 w:12 x:13 y:14 z:15!null + ├── immutable ├── scan a │ ├── columns: k:1!null i:2 f:3 │ ├── key: (1) │ └── fd: (1)-->(2,3) └── projections - ├── (i:2 + k:1) = 4 [as=r:7, outer=(1,2)] - ├── (i:2 * 2) != 3 [as=s:8, outer=(2)] - ├── (1 - k:1) IS NOT DISTINCT FROM 5 [as=t:9, outer=(1)] + ├── (i:2 + k:1) = 4 [as=r:7, outer=(1,2), immutable] + ├── (i:2 * 2) != 3 [as=s:8, outer=(2), immutable] + ├── (1 - k:1) IS NOT DISTINCT FROM 5 [as=t:9, outer=(1), immutable] ├── k:1 IS DISTINCT FROM 11 [as=u:10, outer=(1)] - ├── f:3 + 1.0 [as=v:11, outer=(3)] - ├── (i:2 * i:2) * 15 [as=w:12, outer=(2)] - ├── (i:2 + i:2) & 10000 [as=x:13, outer=(2)] - ├── (i:2 + i:2) | 4 [as=y:14, outer=(2)] - └── (k:1 ^ 2) # -2 [as=z:15, outer=(1)] + ├── f:3 + 1.0 [as=v:11, outer=(3), immutable] + ├── (i:2 * i:2) * 15 [as=w:12, outer=(2), immutable] + ├── (i:2 + i:2) & 10000 [as=x:13, outer=(2), immutable] + ├── (i:2 + i:2) | 4 [as=y:14, outer=(2), immutable] + └── (k:1 ^ 2) # -2 [as=z:15, outer=(1), immutable] # -------------------------------------------------- # EliminateCoalesce @@ -119,10 +121,11 @@ SELECT COALESCE(NULL, NULL, s, s || 'foo') FROM a ---- project ├── columns: coalesce:7 + ├── immutable ├── scan a │ └── columns: s:4 └── projections - └── COALESCE(s:4, s:4 || 'foo') [as=coalesce:7, outer=(4)] + └── COALESCE(s:4, s:4 || 'foo') [as=coalesce:7, outer=(4), immutable] # Trailing null can't be removed. norm @@ -247,6 +250,7 @@ SELECT values ├── columns: "?column?":1 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(1) └── (true IN (NULL, NULL, ('201.249.149.90/18' & '97a7:3650:3dd8:d4e9:35fe:6cfb:a714:1c17/61') << 'e22f:2067:2ed2:7b07:b167:206f:f17b:5b7d/82'),) @@ -581,6 +585,7 @@ SELECT * FROM a WHERE j->'a' = '"b"'::JSON ---- select ├── columns: k:1!null i:2 f:3 s:4 j:5 arr:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-6) ├── scan a @@ -588,13 +593,14 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters - └── j:5 @> '{"a": "b"}' [outer=(5)] + └── j:5 @> '{"a": "b"}' [outer=(5), immutable] norm expect=NormalizeJSONFieldAccess SELECT * FROM a WHERE j->'a'->'x' = '"b"'::JSON ---- select ├── columns: k:1!null i:2 f:3 s:4 j:5 arr:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-6) ├── scan a @@ -602,7 +608,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters - └── j:5 @> '{"a": {"x": "b"}}' [outer=(5)] + └── j:5 @> '{"a": {"x": "b"}}' [outer=(5), immutable] # The transformation is not valid in this case. norm expect-not=NormalizeJSONFieldAccess @@ -610,6 +616,7 @@ SELECT * FROM a WHERE j->2 = '"b"'::JSON ---- select ├── columns: k:1!null i:2 f:3 s:4 j:5 arr:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-6) ├── scan a @@ -617,7 +624,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters - └── (j:5->2) = '"b"' [outer=(5)] + └── (j:5->2) = '"b"' [outer=(5), immutable] # The transformation is not valid in this case, since j->'a' could be an array. norm expect-not=NormalizeJSONFieldAccess @@ -625,6 +632,7 @@ SELECT * FROM a WHERE j->'a' @> '"b"'::JSON ---- select ├── columns: k:1!null i:2 f:3 s:4 j:5 arr:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-6) ├── scan a @@ -632,7 +640,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters - └── (j:5->'a') @> '"b"' [outer=(5)] + └── (j:5->'a') @> '"b"' [outer=(5), immutable] # The transformation is not valid in this case, since containment doesn't imply # equality for non-scalars. @@ -641,11 +649,12 @@ SELECT j->'a' = '["b"]'::JSON, j->'a' = '{"b": "c"}'::JSON FROM a ---- project ├── columns: "?column?":7 "?column?":8 + ├── immutable ├── scan a │ └── columns: j:5 └── projections - ├── (j:5->'a') = '["b"]' [as="?column?":7, outer=(5)] - └── (j:5->'a') = '{"b": "c"}' [as="?column?":8, outer=(5)] + ├── (j:5->'a') = '["b"]' [as="?column?":7, outer=(5), immutable] + └── (j:5->'a') = '{"b": "c"}' [as="?column?":8, outer=(5), immutable] # -------------------------------------------------- # NormalizeJSONContains @@ -656,6 +665,7 @@ SELECT * FROM a WHERE j->'a' @> '{"x": "b"}'::JSON ---- select ├── columns: k:1!null i:2 f:3 s:4 j:5 arr:6 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-6) ├── scan a @@ -663,7 +673,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-6) └── filters - └── j:5 @> '{"a": {"x": "b"}}' [outer=(5)] + └── j:5 @> '{"a": {"x": "b"}}' [outer=(5), immutable] # -------------------------------------------------- # SimplifyCaseWhenConstValue @@ -1072,9 +1082,11 @@ SELECT k FROM e WHERE tz > '2017-11-12 07:35:01+00:00'::TIMESTAMP ---- project ├── columns: k:1!null + ├── stable ├── key: (1) └── select ├── columns: k:1!null tz:4!null + ├── stable ├── key: (1) ├── fd: (1)-->(4) ├── scan e @@ -1082,7 +1094,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(4) └── filters - └── tz:4 > '2017-11-12 07:35:01+00:00' [outer=(4), constraints=(/4: (/NULL - ])] + └── tz:4 > '2017-11-12 07:35:01+00:00' [outer=(4), stable, constraints=(/4: (/NULL - ])] norm expect=UnifyComparisonTypes SELECT k FROM e WHERE tz > '2017-11-12 07:35:01+00:00'::TIMESTAMP @@ -1127,9 +1139,11 @@ SELECT k FROM e WHERE d > '2018-07-01' AND d < '2018-07-01'::DATE + '1w1s'::INTE ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── select ├── columns: k:1!null d:5!null + ├── immutable ├── key: (1) ├── fd: (1)-->(5) ├── scan e @@ -1138,7 +1152,7 @@ project │ └── fd: (1)-->(5) └── filters ├── d:5 > '2018-07-01' [outer=(5), constraints=(/5: [/'2018-07-02' - ]; tight)] - └── d:5 < '2018-07-08 00:00:01+00:00' [outer=(5), constraints=(/5: (/NULL - ])] + └── d:5 < '2018-07-08 00:00:01+00:00' [outer=(5), immutable, constraints=(/5: (/NULL - ])] # NULL value. norm @@ -1190,12 +1204,13 @@ SELECT k FROM a WHERE k IN (VALUES ((SELECT k*i FROM a)), (2), (3)) ---- select ├── columns: k:1!null + ├── immutable ├── key: (1) ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── filters - └── in [outer=(1), subquery] + └── in [outer=(1), immutable, subquery] ├── k:1 └── tuple ├── subquery @@ -1203,16 +1218,18 @@ select │ ├── columns: "?column?":13 │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── cardinality: [0 - 1] + │ ├── immutable │ ├── key: () │ ├── fd: ()-->(13) │ └── project │ ├── columns: "?column?":13 + │ ├── immutable │ ├── scan a │ │ ├── columns: k:7!null i:8 │ │ ├── key: (7) │ │ └── fd: (7)-->(8) │ └── projections - │ └── k:7 * i:8 [as="?column?":13, outer=(7,8)] + │ └── k:7 * i:8 [as="?column?":13, outer=(7,8), immutable] ├── 2 └── 3 @@ -1245,9 +1262,11 @@ SELECT k FROM a WHERE (k, i) IN (SELECT b, a FROM (VALUES (1, 1), (2, 2), (3, 3) ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── semi-join (hash) ├── columns: k:1!null column10:10 + ├── immutable ├── key: (1) ├── fd: (1)-->(10) ├── project @@ -1272,7 +1291,7 @@ project │ └── projections │ └── (column2:8, column1:7) [as=column9:9, outer=(7,8)] └── filters - └── column10:10 = column9:9 [outer=(9,10), constraints=(/9: (/NULL - ]; /10: (/NULL - ]), fd=(9)==(10), (10)==(9)] + └── column10:10 = column9:9 [outer=(9,10), immutable, constraints=(/9: (/NULL - ]; /10: (/NULL - ]), fd=(9)==(10), (10)==(9)] # -------------------------------------------------- # SimplifyEqualsAnyTuple diff --git a/pkg/sql/opt/norm/testdata/rules/select b/pkg/sql/opt/norm/testdata/rules/select index bc00ba6b0167..a113435cc086 100644 --- a/pkg/sql/opt/norm/testdata/rules/select +++ b/pkg/sql/opt/norm/testdata/rules/select @@ -270,9 +270,11 @@ SELECT k FROM e WHERE d > '2018-07-01' AND d < '2018-07-01'::DATE + '1w1s'::INTE ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── select ├── columns: k:1!null d:5!null + ├── immutable ├── key: (1) ├── fd: (1)-->(5) ├── scan e @@ -281,7 +283,7 @@ project │ └── fd: (1)-->(5) └── filters ├── d:5 > '2018-07-01' [outer=(5), constraints=(/5: [/'2018-07-02' - ]; tight)] - └── d:5 < '2018-07-08 00:00:01+00:00' [outer=(5), constraints=(/5: (/NULL - ])] + └── d:5 < '2018-07-08 00:00:01+00:00' [outer=(5), immutable, constraints=(/5: (/NULL - ])] # Ranges can be merged with other filters to create new ranges. norm expect=ConsolidateSelectFilters disable=InlineConstVar @@ -533,6 +535,7 @@ SELECT * FROM (SELECT i, i+1 AS r, f FROM a) a WHERE f=10.0 ---- project ├── columns: i:2 r:6 f:3!null + ├── immutable ├── fd: ()-->(3), (2)-->(6) ├── select │ ├── columns: i:2 f:3!null @@ -542,7 +545,7 @@ project │ └── filters │ └── f:3 = 10.0 [outer=(3), constraints=(/3: [/10.0 - /10.0]; tight), fd=()-->(3)] └── projections - └── i:2 + 1 [as=r:6, outer=(2)] + └── i:2 + 1 [as=r:6, outer=(2), immutable] # Don't push down select if it depends on computed column that can't be inlined. norm expect-not=PushSelectIntoProject @@ -550,6 +553,7 @@ SELECT * FROM (SELECT i, i/2 div, f FROM a) a WHERE div=2 ---- select ├── columns: i:2 div:6!null f:3 + ├── immutable ├── fd: ()-->(6) ├── project │ ├── columns: div:6 i:2 f:3 @@ -559,7 +563,7 @@ select │ └── projections │ └── i:2 / 2 [as=div:6, outer=(2)] └── filters - └── div:6 = 2 [outer=(6), constraints=(/6: [/2 - /2]; tight), fd=()-->(6)] + └── div:6 = 2 [outer=(6), immutable, constraints=(/6: [/2 - /2]; tight), fd=()-->(6)] # Push down some conjuncts, but not others. norm expect=PushSelectIntoProject @@ -567,6 +571,7 @@ SELECT * FROM (SELECT i, i/2 div, f FROM a) a WHERE 10.0=f AND 2=div AND i=1 ---- select ├── columns: i:2!null div:6!null f:3!null + ├── immutable ├── fd: ()-->(2,3,6) ├── project │ ├── columns: div:6!null i:2!null f:3!null @@ -582,7 +587,7 @@ select │ └── projections │ └── i:2 / 2 [as=div:6, outer=(2)] └── filters - └── div:6 = 2 [outer=(6), constraints=(/6: [/2 - /2]; tight), fd=()-->(6)] + └── div:6 = 2 [outer=(6), immutable, constraints=(/6: [/2 - /2]; tight), fd=()-->(6)] # Detect PushSelectIntoProject and FilterUnusedSelectCols dependency cycle. norm @@ -590,6 +595,7 @@ SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i HAVING sum(f)=10.0) ---- project ├── columns: f:3 r:7 + ├── immutable ├── select │ ├── columns: i:2 f:3 sum:6!null │ ├── key: (2,3) @@ -607,7 +613,7 @@ project │ └── filters │ └── sum:6 = 10.0 [outer=(6), constraints=(/6: [/10.0 - /10.0]; tight), fd=()-->(6)] └── projections - └── f:3 + 1.1 [as=r:7, outer=(3)] + └── f:3 + 1.1 [as=r:7, outer=(3), immutable] # -------------------------------------- # PushSelectCondLeftIntoJoinLeftAndRight @@ -1395,9 +1401,11 @@ SELECT k FROM b WHERE i+k IS NOT NULL ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── select ├── columns: k:1!null i:2 + ├── immutable ├── key: (1) ├── fd: (1)-->(2) ├── scan b @@ -1405,7 +1413,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── filters - └── (i:2 + k:1) IS NOT NULL [outer=(1,2)] + └── (i:2 + k:1) IS NOT NULL [outer=(1,2), immutable] # -------------------------------------------------- # DetectSelectContradiction diff --git a/pkg/sql/opt/norm/testdata/rules/with b/pkg/sql/opt/norm/testdata/rules/with index 378ec83f101a..9b61b82b4fad 100644 --- a/pkg/sql/opt/norm/testdata/rules/with +++ b/pkg/sql/opt/norm/testdata/rules/with @@ -137,6 +137,7 @@ WITH foo AS (SELECT 1), bar AS (SELECT 2) SELECT (SELECT * FROM foo) + (SELECT * values ├── columns: "?column?":5 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(5) └── tuple @@ -162,6 +163,7 @@ WITH foo AS (SELECT 1), bar AS (SELECT 2) SELECT (SELECT * FROM foo) + (SELECT * with &2 (bar) ├── columns: "?column?":6 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(6) ├── values @@ -173,6 +175,7 @@ with &2 (bar) └── values ├── columns: "?column?":6 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(6) └── tuple @@ -647,6 +650,7 @@ with &2 (cte) ├── left columns: c:7(int) ├── right columns: "?column?":9(int) ├── cardinality: [1 - 2] + ├── immutable ├── stats: [rows=2, distinct(10)=2, null(10)=0] ├── cost: 0.1 ├── key: (10) @@ -663,6 +667,7 @@ with &2 (cte) └── project ├── columns: "?column?":9(int!null) ├── cardinality: [1 - 1] + ├── immutable ├── stats: [rows=1, distinct(9)=1, null(9)=0] ├── cost: 0.04 ├── key: () @@ -679,7 +684,7 @@ with &2 (cte) │ ├── fd: ()-->(8) │ └── prune: (8) └── projections - └── plus [as="?column?":9, type=int, outer=(8)] + └── plus [as="?column?":9, type=int, outer=(8), immutable] ├── variable: c:8 [type=int] └── const: 1 [type=int] @@ -800,6 +805,7 @@ with &2 (t) ├── columns: sum:6 ├── materialized ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(6) ├── recursive-c-t-e @@ -808,6 +814,7 @@ with &2 (t) │ ├── initial columns: column1:1 │ ├── recursive columns: "?column?":4 │ ├── cardinality: [1 - ] + │ ├── immutable │ ├── values │ │ ├── columns: column1:1!null │ │ ├── cardinality: [1 - 1] @@ -816,6 +823,7 @@ with &2 (t) │ │ └── (1,) │ └── project │ ├── columns: "?column?":4!null + │ ├── immutable │ ├── select │ │ ├── columns: n:3!null │ │ ├── with-scan &1 (t) @@ -826,7 +834,7 @@ with &2 (t) │ │ └── filters │ │ └── n:3 < 100 [outer=(3), constraints=(/3: (/NULL - /99]; tight)] │ └── projections - │ └── n:3 + 1 [as="?column?":4, outer=(3)] + │ └── n:3 + 1 [as="?column?":4, outer=(3), immutable] └── scalar-group-by ├── columns: sum:6 ├── cardinality: [1 - 1] @@ -848,17 +856,20 @@ WITH RECURSIVE t(n) AS NOT MATERIALIZED (VALUES (1) UNION ALL SELECT n+1 FROM t scalar-group-by ├── columns: sum:6 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(6) ├── project │ ├── columns: n:5 │ ├── cardinality: [1 - ] + │ ├── immutable │ ├── recursive-c-t-e │ │ ├── columns: n:2 │ │ ├── working table binding: &1 │ │ ├── initial columns: column1:1 │ │ ├── recursive columns: "?column?":4 │ │ ├── cardinality: [1 - ] + │ │ ├── immutable │ │ ├── values │ │ │ ├── columns: column1:1!null │ │ │ ├── cardinality: [1 - 1] @@ -867,6 +878,7 @@ scalar-group-by │ │ │ └── (1,) │ │ └── project │ │ ├── columns: "?column?":4!null + │ │ ├── immutable │ │ ├── select │ │ │ ├── columns: n:3!null │ │ │ ├── with-scan &1 (t) @@ -877,7 +889,7 @@ scalar-group-by │ │ │ └── filters │ │ │ └── n:3 < 100 [outer=(3), constraints=(/3: (/NULL - /99]; tight)] │ │ └── projections - │ │ └── n:3 + 1 [as="?column?":4, outer=(3)] + │ │ └── n:3 + 1 [as="?column?":4, outer=(3), immutable] │ └── projections │ └── n:2 [as=n:5, outer=(2)] └── aggregations diff --git a/pkg/sql/opt/optgen/exprgen/testdata/join b/pkg/sql/opt/optgen/exprgen/testdata/join index b9c2696c26cc..cc7ef5cfb13d 100644 --- a/pkg/sql/opt/optgen/exprgen/testdata/join +++ b/pkg/sql/opt/optgen/exprgen/testdata/join @@ -115,6 +115,7 @@ expr ---- inner-join-apply ├── columns: t.public.abc.a:1(int) t.public.abc.b:2(int) t.public.abc.c:3(int) t.public.def.d:5(int) t.public.def.e:6(int) t.public.def.f:7(int) + ├── immutable ├── stats: [rows=333333.333] ├── cost: 5611.39451 ├── prune: (7) @@ -129,6 +130,7 @@ inner-join-apply ├── select │ ├── columns: t.public.def.d:5(int) t.public.def.e:6(int) t.public.def.f:7(int) │ ├── outer: (1) + │ ├── immutable │ ├── stats: [rows=333.333333, distinct(1)=1, null(1)=0] │ ├── cost: 1080.03 │ ├── prune: (7) @@ -138,7 +140,7 @@ inner-join-apply │ │ ├── cost: 1070.02 │ │ └── prune: (5-7) │ └── filters - │ └── eq [type=bool, outer=(1,5,6), constraints=(/1: (/NULL - ])] + │ └── eq [type=bool, outer=(1,5,6), immutable, constraints=(/1: (/NULL - ])] │ ├── variable: t.public.abc.a:1 [type=int] │ └── plus [type=int] │ ├── variable: t.public.def.d:5 [type=int] diff --git a/pkg/sql/opt/optgen/exprgen/testdata/values b/pkg/sql/opt/optgen/exprgen/testdata/values index dbb56ff93c9b..de5f30986491 100644 --- a/pkg/sql/opt/optgen/exprgen/testdata/values +++ b/pkg/sql/opt/optgen/exprgen/testdata/values @@ -33,6 +33,7 @@ expr project ├── columns: y:2(int!null) x:1(int!null) ├── cardinality: [1 - 1] + ├── immutable ├── stats: [rows=1] ├── cost: 0.05 ├── key: () @@ -49,6 +50,6 @@ project │ └── tuple [type=tuple{int}] │ └── const: 1 [type=int] └── projections - └── plus [as=y:2, type=int, outer=(1)] + └── plus [as=y:2, type=int, outer=(1), immutable] ├── variable: x:1 [type=int] └── const: 10 [type=int] diff --git a/pkg/sql/opt/xform/testdata/coster/join b/pkg/sql/opt/xform/testdata/coster/join index 02549a112260..2e37722218d4 100644 --- a/pkg/sql/opt/xform/testdata/coster/join +++ b/pkg/sql/opt/xform/testdata/coster/join @@ -11,12 +11,14 @@ SELECT k, x FROM a INNER JOIN b ON k=x WHERE d=1.0 ---- project ├── columns: k:1!null x:5!null + ├── immutable ├── stats: [rows=99] ├── cost: 2124.725 ├── fd: (1)==(5), (5)==(1) └── inner-join (hash) ├── columns: k:1!null d:4!null x:5!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + ├── immutable ├── stats: [rows=99, distinct(1)=10, null(1)=0, distinct(5)=10, null(5)=0] ├── cost: 2123.725 ├── fd: ()-->(4), (1)==(5), (5)==(1) @@ -26,6 +28,7 @@ project │ └── cost: 1040.02 ├── select │ ├── columns: k:1!null d:4!null + │ ├── immutable │ ├── stats: [rows=10, distinct(1)=10, null(1)=0, distinct(4)=1, null(4)=0] │ ├── cost: 1070.03 │ ├── key: (1) @@ -37,7 +40,7 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(4) │ └── filters - │ └── d:4 = 1.0 [outer=(4), constraints=(/4: [/1.0 - /1.0]; tight), fd=()-->(4)] + │ └── d:4 = 1.0 [outer=(4), immutable, constraints=(/4: [/1.0 - /1.0]; tight), fd=()-->(4)] └── filters └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] diff --git a/pkg/sql/opt/xform/testdata/coster/project b/pkg/sql/opt/xform/testdata/coster/project index 572b477cfab0..d3866dbe0eb2 100644 --- a/pkg/sql/opt/xform/testdata/coster/project +++ b/pkg/sql/opt/xform/testdata/coster/project @@ -7,6 +7,7 @@ SELECT k, i, s || 'foo' FROM a ---- project ├── columns: k:1!null i:2 "?column?":5 + ├── immutable ├── stats: [rows=1000] ├── cost: 1090.03 ├── key: (1) @@ -18,13 +19,14 @@ project │ ├── key: (1) │ └── fd: (1)-->(2,3) └── projections - └── s:3 || 'foo' [as="?column?":5, outer=(3)] + └── s:3 || 'foo' [as="?column?":5, outer=(3), immutable] opt SELECT k, k+2, i*d FROM a ---- project ├── columns: k:1!null "?column?":5!null "?column?":6 + ├── immutable ├── stats: [rows=1000] ├── cost: 1100.03 ├── key: (1) @@ -36,5 +38,5 @@ project │ ├── key: (1) │ └── fd: (1)-->(2,4) └── projections - ├── k:1 + 2 [as="?column?":5, outer=(1)] - └── i:2 * d:4 [as="?column?":6, outer=(2,4)] + ├── k:1 + 2 [as="?column?":5, outer=(1), immutable] + └── i:2 * d:4 [as="?column?":6, outer=(2,4), immutable] diff --git a/pkg/sql/opt/xform/testdata/coster/scan b/pkg/sql/opt/xform/testdata/coster/scan index 3492d8f04382..12e5e1174630 100644 --- a/pkg/sql/opt/xform/testdata/coster/scan +++ b/pkg/sql/opt/xform/testdata/coster/scan @@ -104,6 +104,7 @@ SELECT id FROM speed_test@primary WHERE id BETWEEN 1 AND 1000 AND ((id % 16) = 0 select ├── columns: id:1!null ├── cardinality: [0 - 1000] + ├── immutable ├── stats: [rows=333.333333, distinct(1)=333.333333, null(1)=0] ├── cost: 1030.02 ├── key: (1) @@ -116,7 +117,7 @@ select │ ├── cost: 1020.01 │ └── key: (1) └── filters - └── (id:1 % 16) = 0 [outer=(1)] + └── (id:1 % 16) = 0 [outer=(1), immutable] opt SELECT id FROM speed_test@primary WHERE id BETWEEN 1 AND 2000 AND ((id % 16) = 0) @@ -124,6 +125,7 @@ SELECT id FROM speed_test@primary WHERE id BETWEEN 1 AND 2000 AND ((id % 16) = 0 select ├── columns: id:1!null ├── cardinality: [0 - 2000] + ├── immutable ├── stats: [rows=333.333333, distinct(1)=333.333333, null(1)=0] ├── cost: 1030.02 ├── key: (1) @@ -136,4 +138,4 @@ select │ ├── cost: 1020.01 │ └── key: (1) └── filters - └── (id:1 % 16) = 0 [outer=(1)] + └── (id:1 % 16) = 0 [outer=(1), immutable] diff --git a/pkg/sql/opt/xform/testdata/external/customer b/pkg/sql/opt/xform/testdata/external/customer index 3ad492ee5fd2..3de6b33116b1 100644 --- a/pkg/sql/opt/xform/testdata/external/customer +++ b/pkg/sql/opt/xform/testdata/external/customer @@ -232,12 +232,14 @@ LIMIT 50 project ├── columns: score:9!null expires_at:15!null [hidden: updated_at_inverse:14!null] ├── cardinality: [0 - 50] + ├── immutable ├── fd: ()-->(15) ├── ordering: -9,-14 opt(15) [actual: -9,-14] └── scan leaderboard_record@test_idx,rev ├── columns: id:1!null leaderboard_id:2!null score:9!null updated_at_inverse:14!null expires_at:15!null ├── constraint: /2/15/9/14/1/3: [/'\x74657374'/0 - /'\x74657374'/0/100/500/'\x736f6d655f6964') ├── limit: 50(rev) + ├── immutable ├── key: (1) ├── fd: ()-->(2,15), (1)-->(9,14) └── ordering: -9,-14 opt(2,15) [actual: -9,-14] @@ -569,5 +571,5 @@ project │ │ └── filters (true) │ └── filters (true) └── projections - ├── value:4->>'secondary_id' [as=secondary_id:6, outer=(4)] + ├── value:4->>'secondary_id' [as=secondary_id:6, outer=(4), immutable] └── data:3 || jsonb_build_object('primary_id', primary_id:1) [as="?column?":7, outer=(1,3), stable] diff --git a/pkg/sql/opt/xform/testdata/external/hibernate b/pkg/sql/opt/xform/testdata/external/hibernate index 9ba9b07f084c..49161434f34e 100644 --- a/pkg/sql/opt/xform/testdata/external/hibernate +++ b/pkg/sql/opt/xform/testdata/external/hibernate @@ -1988,18 +1988,22 @@ WHERE ---- project ├── columns: customer1_1_0_:1!null ordernum2_1_0_:2!null orderdat3_1_0_:3!null formula101_0_:18 customer1_2_1_:4 ordernum2_2_1_:5 producti3_2_1_:6 customer1_2_2_:4 ordernum2_2_2_:5 producti3_2_2_:6 quantity4_2_2_:7 + ├── immutable ├── key: (6) ├── fd: ()-->(1-3), (6)-->(4,5,7,18) ├── group-by │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 sum:17 │ ├── grouping columns: lineitems1_.productid:6 + │ ├── immutable │ ├── key: (6) │ ├── fd: ()-->(1-3), (6)-->(1-5,7,17) │ ├── right-join (hash) │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 li.customerid:8 li.ordernumber:9 column16:16 + │ │ ├── immutable │ │ ├── fd: ()-->(1-3), (6)-->(4,5,7) │ │ ├── project │ │ │ ├── columns: column16:16 li.customerid:8!null li.ordernumber:9!null + │ │ │ ├── immutable │ │ │ ├── inner-join (hash) │ │ │ │ ├── columns: li.customerid:8!null li.ordernumber:9!null li.productid:10!null li.quantity:11 p.productid:12!null cost:14 │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) @@ -2016,7 +2020,7 @@ project │ │ │ │ └── filters │ │ │ │ └── li.productid:10 = p.productid:12 [outer=(10,12), constraints=(/10: (/NULL - ]; /12: (/NULL - ]), fd=(10)==(12), (12)==(10)] │ │ │ └── projections - │ │ │ └── li.quantity:11 * cost:14 [as=column16:16, outer=(11,14)] + │ │ │ └── li.quantity:11 * cost:14 [as=column16:16, outer=(11,14), immutable] │ │ ├── left-join (merge) │ │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 │ │ │ ├── left ordering: +1,+2 @@ -2110,25 +2114,30 @@ FROM ---- project ├── columns: customer1_0_0_:1!null customer1_1_1_:4 ordernum2_1_1_:5 customer1_2_2_:7 ordernum2_2_2_:8 producti3_2_2_:9 producti1_3_3_:11 name2_0_0_:2!null address3_0_0_:3!null orderdat3_1_1_:6 formula103_1_:30 customer1_1_0__:4 ordernum2_1_0__:5 ordernum2_0__:5 quantity4_2_2_:10 customer1_2_1__:7 ordernum2_2_1__:8 producti3_2_1__:9 descript2_3_3_:12 cost3_3_3_:13 numberav4_3_3_:14 formula104_3_:31 + ├── immutable ├── key: (1,4,5,7-9) ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(2,3,6,10-14,30,31) ├── group-by │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 sum:24 sum:29 │ ├── grouping columns: customer0_.customerid:1!null orders1_.customerid:4 orders1_.ordernumber:5 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 + │ ├── immutable │ ├── key: (1,4,5,7-9) │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(2,3,6,10-14,24,29) │ ├── left-join (hash) │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 sum:24 li.productid:27 li.quantity:28 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) + │ │ ├── immutable │ │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(2,3,6,10-14,24) │ │ ├── group-by │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 sum:24 │ │ │ ├── grouping columns: customer0_.customerid:1!null orders1_.customerid:4 orders1_.ordernumber:5 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 + │ │ │ ├── immutable │ │ │ ├── key: (1,4,5,7-9) │ │ │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(2,3,6,10-14,24) │ │ │ ├── left-join (hash) │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 li.customerid:15 li.ordernumber:16 column23:23 │ │ │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) + │ │ │ │ ├── immutable │ │ │ │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(11-14) │ │ │ │ ├── left-join (hash) │ │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 @@ -2172,6 +2181,7 @@ project │ │ │ │ │ └── lineitems2_.productid:9 = product3_.productid:11 [outer=(9,11), constraints=(/9: (/NULL - ]; /11: (/NULL - ]), fd=(9)==(11), (11)==(9)] │ │ │ │ ├── project │ │ │ │ │ ├── columns: column23:23 li.customerid:15!null li.ordernumber:16!null + │ │ │ │ │ ├── immutable │ │ │ │ │ ├── inner-join (hash) │ │ │ │ │ │ ├── columns: li.customerid:15!null li.ordernumber:16!null li.productid:17!null li.quantity:18 p.productid:19!null p.cost:21 │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) @@ -2188,7 +2198,7 @@ project │ │ │ │ │ │ └── filters │ │ │ │ │ │ └── li.productid:17 = p.productid:19 [outer=(17,19), constraints=(/17: (/NULL - ]; /19: (/NULL - ]), fd=(17)==(19), (19)==(17)] │ │ │ │ │ └── projections - │ │ │ │ │ └── li.quantity:18 * p.cost:21 [as=column23:23, outer=(18,21)] + │ │ │ │ │ └── li.quantity:18 * p.cost:21 [as=column23:23, outer=(18,21), immutable] │ │ │ │ └── filters │ │ │ │ ├── li.customerid:15 = orders1_.customerid:4 [outer=(4,15), constraints=(/4: (/NULL - ]; /15: (/NULL - ]), fd=(4)==(15), (15)==(4)] │ │ │ │ └── li.ordernumber:16 = orders1_.ordernumber:5 [outer=(5,16), constraints=(/5: (/NULL - ]; /16: (/NULL - ]), fd=(5)==(16), (16)==(5)] @@ -2274,18 +2284,22 @@ WHERE ---- project ├── columns: customer1_1_0_:1!null ordernum2_1_0_:2!null orderdat3_1_0_:3!null formula105_0_:18 customer1_2_1_:4 ordernum2_2_1_:5 producti3_2_1_:6 customer1_2_2_:4 ordernum2_2_2_:5 producti3_2_2_:6 quantity4_2_2_:7 + ├── immutable ├── key: (6) ├── fd: ()-->(1-3), (6)-->(4,5,7,18) ├── group-by │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 sum:17 │ ├── grouping columns: lineitems1_.productid:6 + │ ├── immutable │ ├── key: (6) │ ├── fd: ()-->(1-3), (6)-->(1-5,7,17) │ ├── right-join (hash) │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 li.customerid:8 li.ordernumber:9 column16:16 + │ │ ├── immutable │ │ ├── fd: ()-->(1-3), (6)-->(4,5,7) │ │ ├── project │ │ │ ├── columns: column16:16 li.customerid:8!null li.ordernumber:9!null + │ │ │ ├── immutable │ │ │ ├── inner-join (hash) │ │ │ │ ├── columns: li.customerid:8!null li.ordernumber:9!null li.productid:10!null li.quantity:11 p.productid:12!null cost:14 │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) @@ -2302,7 +2316,7 @@ project │ │ │ │ └── filters │ │ │ │ └── li.productid:10 = p.productid:12 [outer=(10,12), constraints=(/10: (/NULL - ]; /12: (/NULL - ]), fd=(10)==(12), (12)==(10)] │ │ │ └── projections - │ │ │ └── li.quantity:11 * cost:14 [as=column16:16, outer=(11,14)] + │ │ │ └── li.quantity:11 * cost:14 [as=column16:16, outer=(11,14), immutable] │ │ ├── left-join (merge) │ │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 │ │ │ ├── left ordering: +1,+2 @@ -2363,16 +2377,19 @@ FROM ---- project ├── columns: customer1_10_:1!null ordernum2_10_:2!null orderdat3_10_:3!null formula273_:14 + ├── immutable ├── key: (1,2) ├── fd: (1,2)-->(3,14) ├── group-by │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null sum:13 │ ├── grouping columns: order0_.customerid:1!null order0_.ordernumber:2!null + │ ├── immutable │ ├── key: (1,2) │ ├── fd: (1,2)-->(3,13) │ ├── left-join (hash) │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null li.customerid:4 li.ordernumber:5 column12:12 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) + │ │ ├── immutable │ │ ├── fd: (1,2)-->(3) │ │ ├── scan order0_ │ │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null @@ -2380,6 +2397,7 @@ project │ │ │ └── fd: (1,2)-->(3) │ │ ├── project │ │ │ ├── columns: column12:12 li.customerid:4!null li.ordernumber:5!null + │ │ │ ├── immutable │ │ │ ├── inner-join (hash) │ │ │ │ ├── columns: li.customerid:4!null li.ordernumber:5!null li.productid:6!null quantity:7 p.productid:8!null cost:10 │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) @@ -2396,7 +2414,7 @@ project │ │ │ │ └── filters │ │ │ │ └── li.productid:6 = p.productid:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] │ │ │ └── projections - │ │ │ └── quantity:7 * cost:10 [as=column12:12, outer=(7,10)] + │ │ │ └── quantity:7 * cost:10 [as=column12:12, outer=(7,10), immutable] │ │ └── filters │ │ ├── li.customerid:4 = order0_.customerid:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ └── li.ordernumber:5 = order0_.ordernumber:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] diff --git a/pkg/sql/opt/xform/testdata/external/tpcc b/pkg/sql/opt/xform/testdata/external/tpcc index 619d2740fda0..20a5476544f1 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc +++ b/pkg/sql/opt/xform/testdata/external/tpcc @@ -39,6 +39,7 @@ project └── project ├── columns: d_next_o_id_new:23 d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 ├── cardinality: [0 - 1] + ├── immutable ├── key: () ├── fd: ()-->(12-23) ├── scan district @@ -48,7 +49,7 @@ project │ ├── key: () │ └── fd: ()-->(12-22) └── projections - └── d_next_o_id:22 + 1 [as=d_next_o_id_new:23, outer=(22)] + └── d_next_o_id:22 + 1 [as=d_next_o_id_new:23, outer=(22), immutable] opt format=hide-qual SELECT w_tax FROM warehouse WHERE w_id = 10 @@ -585,7 +586,7 @@ project │ └── projections │ ├── crdb_internal.round_decimal_values(customer.c_balance:38 - 3860.61, 2) [as=c_balance:47, outer=(38), immutable] │ ├── crdb_internal.round_decimal_values(customer.c_ytd_payment:39 + 3860.61, 2) [as=c_ytd_payment:48, outer=(39), immutable] - │ ├── c_payment_cnt:40 + 1 [as=c_payment_cnt_new:45, outer=(40)] + │ ├── c_payment_cnt:40 + 1 [as=c_payment_cnt_new:45, outer=(40), immutable] │ └── CASE c_credit:35 WHEN 'BC' THEN left((((((c_id:22::STRING || c_d_id:23::STRING) || c_w_id:24::STRING) || '5') || '10') || '3860.61') || c_data:42, 500) ELSE c_data:42 END [as=c_data_new:46, outer=(22-24,35,42), immutable] └── projections └── CASE c_credit:14 WHEN 'BC' THEN left(c_data:21, 200) ELSE '' END [as=case:49, outer=(14,21), immutable] @@ -906,7 +907,7 @@ update customer │ └── fd: ()-->(24), (22,23)-->(25-42) └── projections ├── crdb_internal.round_decimal_values(customer.c_balance:38 + CASE c_d_id:23 WHEN 6 THEN 57214.780000 WHEN 8 THEN 67755.430000 WHEN 1 THEN 51177.840000 WHEN 2 THEN 73840.700000 WHEN 4 THEN 45906.990000 WHEN 9 THEN 32523.760000 WHEN 10 THEN 20240.200000 WHEN 3 THEN 75299.790000 WHEN 5 THEN 56543.340000 WHEN 7 THEN 67157.940000 END, 2) [as=c_balance:45, outer=(23,38), immutable] - └── c_delivery_cnt:41 + 1 [as=c_delivery_cnt_new:43, outer=(41)] + └── c_delivery_cnt:41 + 1 [as=c_delivery_cnt_new:43, outer=(41), immutable] opt format=hide-qual DELETE FROM new_order @@ -1071,12 +1072,14 @@ WHERE w_ytd != sum_d_ytd scalar-group-by ├── columns: count:22!null ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(22) ├── inner-join (merge) │ ├── columns: w_id:1!null w_ytd:9!null d_w_id:11!null sum:21!null │ ├── left ordering: +1 │ ├── right ordering: +11 + │ ├── immutable │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) │ ├── scan warehouse @@ -1097,7 +1100,7 @@ scalar-group-by │ │ └── sum [as=sum:21, outer=(19)] │ │ └── d_ytd:19 │ └── filters - │ └── w_ytd:9 != sum:21 [outer=(9,21), constraints=(/9: (/NULL - ]; /21: (/NULL - ])] + │ └── w_ytd:9 != sum:21 [outer=(9,21), immutable, constraints=(/9: (/NULL - ]; /21: (/NULL - ])] └── aggregations └── count-rows [as=count_rows:22] @@ -1165,10 +1168,12 @@ WHERE nod != -1 scalar-group-by ├── columns: count:8!null ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(8) ├── select │ ├── columns: no_d_id:2!null no_w_id:3!null max:4!null min:5!null count_rows:6!null + │ ├── immutable │ ├── key: (2,3) │ ├── fd: (2,3)-->(4-6) │ ├── group-by @@ -1188,7 +1193,7 @@ scalar-group-by │ │ │ └── no_o_id:1 │ │ └── count-rows [as=count_rows:6] │ └── filters - │ └── ((max:4 - min:5) - count_rows:6) != -1 [outer=(4-6)] + │ └── ((max:4 - min:5) - count_rows:6) != -1 [outer=(4-6), immutable] └── aggregations └── count-rows [as=count_rows:8] diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats index 5c7ee92c9925..cc9552e7230a 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats @@ -42,6 +42,7 @@ project └── project ├── columns: d_next_o_id_new:23 d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 ├── cardinality: [0 - 1] + ├── immutable ├── key: () ├── fd: ()-->(12-23) ├── scan district @@ -51,7 +52,7 @@ project │ ├── key: () │ └── fd: ()-->(12-22) └── projections - └── d_next_o_id:22 + 1 [as=d_next_o_id_new:23, outer=(22)] + └── d_next_o_id:22 + 1 [as=d_next_o_id_new:23, outer=(22), immutable] opt format=hide-qual SELECT w_tax FROM warehouse WHERE w_id = 10 @@ -588,7 +589,7 @@ project │ └── projections │ ├── crdb_internal.round_decimal_values(customer.c_balance:38 - 3860.61, 2) [as=c_balance:47, outer=(38), immutable] │ ├── crdb_internal.round_decimal_values(customer.c_ytd_payment:39 + 3860.61, 2) [as=c_ytd_payment:48, outer=(39), immutable] - │ ├── c_payment_cnt:40 + 1 [as=c_payment_cnt_new:45, outer=(40)] + │ ├── c_payment_cnt:40 + 1 [as=c_payment_cnt_new:45, outer=(40), immutable] │ └── CASE c_credit:35 WHEN 'BC' THEN left((((((c_id:22::STRING || c_d_id:23::STRING) || c_w_id:24::STRING) || '5') || '10') || '3860.61') || c_data:42, 500) ELSE c_data:42 END [as=c_data_new:46, outer=(22-24,35,42), immutable] └── projections └── CASE c_credit:14 WHEN 'BC' THEN left(c_data:21, 200) ELSE '' END [as=case:49, outer=(14,21), immutable] @@ -909,7 +910,7 @@ update customer │ └── fd: ()-->(24), (22,23)-->(25-42) └── projections ├── crdb_internal.round_decimal_values(customer.c_balance:38 + CASE c_d_id:23 WHEN 6 THEN 57214.780000 WHEN 8 THEN 67755.430000 WHEN 1 THEN 51177.840000 WHEN 2 THEN 73840.700000 WHEN 4 THEN 45906.990000 WHEN 9 THEN 32523.760000 WHEN 10 THEN 20240.200000 WHEN 3 THEN 75299.790000 WHEN 5 THEN 56543.340000 WHEN 7 THEN 67157.940000 END, 2) [as=c_balance:45, outer=(23,38), immutable] - └── c_delivery_cnt:41 + 1 [as=c_delivery_cnt_new:43, outer=(41)] + └── c_delivery_cnt:41 + 1 [as=c_delivery_cnt_new:43, outer=(41), immutable] opt format=hide-qual DELETE FROM new_order @@ -1073,12 +1074,14 @@ WHERE w_ytd != sum_d_ytd scalar-group-by ├── columns: count:22!null ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(22) ├── inner-join (merge) │ ├── columns: w_id:1!null w_ytd:9!null d_w_id:11!null sum:21!null │ ├── left ordering: +1 │ ├── right ordering: +11 + │ ├── immutable │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) │ ├── scan warehouse @@ -1099,7 +1102,7 @@ scalar-group-by │ │ └── sum [as=sum:21, outer=(19)] │ │ └── d_ytd:19 │ └── filters - │ └── w_ytd:9 != sum:21 [outer=(9,21), constraints=(/9: (/NULL - ]; /21: (/NULL - ])] + │ └── w_ytd:9 != sum:21 [outer=(9,21), immutable, constraints=(/9: (/NULL - ]; /21: (/NULL - ])] └── aggregations └── count-rows [as=count_rows:22] @@ -1167,10 +1170,12 @@ WHERE nod != -1 scalar-group-by ├── columns: count:8!null ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(8) ├── select │ ├── columns: no_d_id:2!null no_w_id:3!null max:4!null min:5!null count_rows:6!null + │ ├── immutable │ ├── key: (2,3) │ ├── fd: (2,3)-->(4-6) │ ├── group-by @@ -1190,7 +1195,7 @@ scalar-group-by │ │ │ └── no_o_id:1 │ │ └── count-rows [as=count_rows:6] │ └── filters - │ └── ((max:4 - min:5) - count_rows:6) != -1 [outer=(4-6)] + │ └── ((max:4 - min:5) - count_rows:6) != -1 [outer=(4-6), immutable] └── aggregations └── count-rows [as=count_rows:8] diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats index 5e33f14a1419..50ac4ef9384f 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats @@ -36,6 +36,7 @@ project └── project ├── columns: d_next_o_id_new:23 d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 ├── cardinality: [0 - 1] + ├── immutable ├── key: () ├── fd: ()-->(12-23) ├── scan district @@ -45,7 +46,7 @@ project │ ├── key: () │ └── fd: ()-->(12-22) └── projections - └── d_next_o_id:22 + 1 [as=d_next_o_id_new:23, outer=(22)] + └── d_next_o_id:22 + 1 [as=d_next_o_id_new:23, outer=(22), immutable] opt format=hide-qual SELECT w_tax FROM warehouse WHERE w_id = 10 @@ -582,7 +583,7 @@ project │ └── projections │ ├── crdb_internal.round_decimal_values(customer.c_balance:38 - 3860.61, 2) [as=c_balance:47, outer=(38), immutable] │ ├── crdb_internal.round_decimal_values(customer.c_ytd_payment:39 + 3860.61, 2) [as=c_ytd_payment:48, outer=(39), immutable] - │ ├── c_payment_cnt:40 + 1 [as=c_payment_cnt_new:45, outer=(40)] + │ ├── c_payment_cnt:40 + 1 [as=c_payment_cnt_new:45, outer=(40), immutable] │ └── CASE c_credit:35 WHEN 'BC' THEN left((((((c_id:22::STRING || c_d_id:23::STRING) || c_w_id:24::STRING) || '5') || '10') || '3860.61') || c_data:42, 500) ELSE c_data:42 END [as=c_data_new:46, outer=(22-24,35,42), immutable] └── projections └── CASE c_credit:14 WHEN 'BC' THEN left(c_data:21, 200) ELSE '' END [as=case:49, outer=(14,21), immutable] @@ -907,7 +908,7 @@ update customer │ └── fd: ()-->(24), (22,23)-->(25-42) └── projections ├── crdb_internal.round_decimal_values(customer.c_balance:38 + CASE c_d_id:23 WHEN 6 THEN 57214.780000 WHEN 8 THEN 67755.430000 WHEN 1 THEN 51177.840000 WHEN 2 THEN 73840.700000 WHEN 4 THEN 45906.990000 WHEN 9 THEN 32523.760000 WHEN 10 THEN 20240.200000 WHEN 3 THEN 75299.790000 WHEN 5 THEN 56543.340000 WHEN 7 THEN 67157.940000 END, 2) [as=c_balance:45, outer=(23,38), immutable] - └── c_delivery_cnt:41 + 1 [as=c_delivery_cnt_new:43, outer=(41)] + └── c_delivery_cnt:41 + 1 [as=c_delivery_cnt_new:43, outer=(41), immutable] opt format=hide-qual DELETE FROM new_order @@ -1071,12 +1072,14 @@ WHERE w_ytd != sum_d_ytd scalar-group-by ├── columns: count:22!null ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(22) ├── inner-join (lookup warehouse) │ ├── columns: w_id:1!null w_ytd:9!null d_w_id:11!null sum:21!null │ ├── key columns: [11] = [1] │ ├── lookup columns are key + │ ├── immutable │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) │ ├── group-by @@ -1092,7 +1095,7 @@ scalar-group-by │ │ └── sum [as=sum:21, outer=(19)] │ │ └── d_ytd:19 │ └── filters - │ └── w_ytd:9 != sum:21 [outer=(9,21), constraints=(/9: (/NULL - ]; /21: (/NULL - ])] + │ └── w_ytd:9 != sum:21 [outer=(9,21), immutable, constraints=(/9: (/NULL - ]; /21: (/NULL - ])] └── aggregations └── count-rows [as=count_rows:22] @@ -1160,10 +1163,12 @@ WHERE nod != -1 scalar-group-by ├── columns: count:8!null ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(8) ├── select │ ├── columns: no_d_id:2!null no_w_id:3!null max:4!null min:5!null count_rows:6!null + │ ├── immutable │ ├── key: (2,3) │ ├── fd: (2,3)-->(4-6) │ ├── group-by @@ -1183,7 +1188,7 @@ scalar-group-by │ │ │ └── no_o_id:1 │ │ └── count-rows [as=count_rows:6] │ └── filters - │ └── ((max:4 - min:5) - count_rows:6) != -1 [outer=(4-6)] + │ └── ((max:4 - min:5) - count_rows:6) != -1 [outer=(4-6), immutable] └── aggregations └── count-rows [as=count_rows:8] diff --git a/pkg/sql/opt/xform/testdata/external/tpch b/pkg/sql/opt/xform/testdata/external/tpch index 5ed71ce79f0f..a985dbf3f03a 100644 --- a/pkg/sql/opt/xform/testdata/external/tpch +++ b/pkg/sql/opt/xform/testdata/external/tpch @@ -42,16 +42,19 @@ ORDER BY ---- sort ├── columns: l_returnflag:9!null l_linestatus:10!null sum_qty:17!null sum_base_price:18!null sum_disc_price:20!null sum_charge:22!null avg_qty:23!null avg_price:24!null avg_disc:25!null count_order:26!null + ├── immutable ├── key: (9,10) ├── fd: (9,10)-->(17,18,20,22-26) ├── ordering: +9,+10 └── group-by ├── columns: l_returnflag:9!null l_linestatus:10!null sum:17!null sum:18!null sum:20!null sum:22!null avg:23!null avg:24!null avg:25!null count_rows:26!null ├── grouping columns: l_returnflag:9!null l_linestatus:10!null + ├── immutable ├── key: (9,10) ├── fd: (9,10)-->(17,18,20,22-26) ├── project │ ├── columns: column19:19!null column21:21!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_returnflag:9!null l_linestatus:10!null + │ ├── immutable │ ├── select │ │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_tax:8!null l_returnflag:9!null l_linestatus:10!null l_shipdate:11!null │ │ ├── scan lineitem @@ -59,8 +62,8 @@ sort │ │ └── filters │ │ └── l_shipdate:11 <= '1998-09-02' [outer=(11), constraints=(/11: (/NULL - /'1998-09-02']; tight)] │ └── projections - │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column19:19, outer=(6,7)] - │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column21:21, outer=(6-8)] + │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column19:19, outer=(6,7), immutable] + │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column21:21, outer=(6-8), immutable] └── aggregations ├── sum [as=sum:17, outer=(5)] │ └── l_quantity:5 @@ -342,11 +345,13 @@ limit ├── columns: l_orderkey:18!null revenue:35!null o_orderdate:13!null o_shippriority:16!null ├── internal-ordering: -35,+13 ├── cardinality: [0 - 10] + ├── immutable ├── key: (18) ├── fd: (18)-->(13,16,35) ├── ordering: -35,+13 ├── sort │ ├── columns: o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null sum:35!null + │ ├── immutable │ ├── key: (18) │ ├── fd: (18)-->(13,16,35) │ ├── ordering: -35,+13 @@ -354,10 +359,12 @@ limit │ └── group-by │ ├── columns: o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null sum:35!null │ ├── grouping columns: l_orderkey:18!null + │ ├── immutable │ ├── key: (18) │ ├── fd: (18)-->(13,16,35) │ ├── project │ │ ├── columns: column34:34!null o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null + │ │ ├── immutable │ │ ├── fd: (18)-->(13,16) │ │ ├── inner-join (lookup lineitem) │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_shipdate:28!null @@ -393,7 +400,7 @@ limit │ │ │ └── filters │ │ │ └── l_shipdate:28 > '1995-03-15' [outer=(28), constraints=(/28: [/'1995-03-16' - ]; tight)] │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column34:34, outer=(23,24)] + │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column34:34, outer=(23,24), immutable] │ └── aggregations │ ├── sum [as=sum:35, outer=(34)] │ │ └── column34:34 @@ -516,16 +523,19 @@ ORDER BY ---- sort ├── columns: n_name:42!null revenue:49!null + ├── immutable ├── key: (42) ├── fd: (42)-->(49) ├── ordering: -49 └── group-by ├── columns: n_name:42!null sum:49!null ├── grouping columns: n_name:42!null + ├── immutable ├── key: (42) ├── fd: (42)-->(49) ├── project │ ├── columns: column48:48!null n_name:42!null + │ ├── immutable │ ├── inner-join (hash) │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null s_suppkey:34!null s_nationkey:37!null n_nationkey:41!null n_name:42!null n_regionkey:43!null r_regionkey:45!null r_name:46!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) @@ -588,7 +598,7 @@ sort │ │ ├── c_custkey:1 = o_custkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ │ └── c_nationkey:4 = s_nationkey:37 [outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ]), fd=(4)==(37), (37)==(4)] │ └── projections - │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column48:48, outer=(23,24)] + │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column48:48, outer=(23,24), immutable] └── aggregations └── sum [as=sum:49, outer=(48)] └── column48:48 @@ -622,10 +632,12 @@ WHERE scalar-group-by ├── columns: revenue:18 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(18) ├── project │ ├── columns: column17:17!null + │ ├── immutable │ ├── select │ │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null │ │ ├── index-join lineitem @@ -639,7 +651,7 @@ scalar-group-by │ │ ├── (l_discount:7 >= 0.05) AND (l_discount:7 <= 0.07) [outer=(7), constraints=(/7: [/0.05 - /0.07]; tight)] │ │ └── l_quantity:5 < 24.0 [outer=(5), constraints=(/5: (/NULL - /23.999999999999996]; tight)] │ └── projections - │ └── l_extendedprice:6 * l_discount:7 [as=column17:17, outer=(6,7)] + │ └── l_extendedprice:6 * l_discount:7 [as=column17:17, outer=(6,7), immutable] └── aggregations └── sum [as=sum:18, outer=(17)] └── column17:17 @@ -759,7 +771,7 @@ sort │ │ └── s_nationkey:4 = n1.n_nationkey:41 [outer=(4,41), constraints=(/4: (/NULL - ]; /41: (/NULL - ]), fd=(4)==(41), (41)==(4)] │ └── projections │ ├── extract('year', l_shipdate:18) [as=l_year:49, outer=(18), immutable] - │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=volume:50, outer=(13,14)] + │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=volume:50, outer=(13,14), immutable] └── aggregations └── sum [as=sum:51, outer=(50)] └── volume:50 @@ -934,7 +946,7 @@ sort │ │ │ │ └── p_partkey:1 = l_partkey:18 [outer=(1,18), constraints=(/1: (/NULL - ]; /18: (/NULL - ]), fd=(1)==(18), (18)==(1)] │ │ │ └── projections │ │ │ ├── extract('year', o_orderdate:37) [as=o_year:61, outer=(37), immutable] - │ │ │ └── l_extendedprice:22 * (1.0 - l_discount:23) [as=volume:62, outer=(22,23)] + │ │ │ └── l_extendedprice:22 * (1.0 - l_discount:23) [as=volume:62, outer=(22,23), immutable] │ │ └── projections │ │ └── CASE WHEN n2.n_name:55 = 'BRAZIL' THEN volume:62 ELSE 0.0 END [as=column63:63, outer=(55,62)] │ └── aggregations @@ -1060,7 +1072,7 @@ sort │ │ └── p_name:2 LIKE '%green%' [outer=(2), constraints=(/2: (/NULL - ])] │ └── projections │ ├── extract('year', o_orderdate:42) [as=o_year:51, outer=(42), immutable] - │ └── (l_extendedprice:22 * (1.0 - l_discount:23)) - (ps_supplycost:36 * l_quantity:21) [as=amount:52, outer=(21-23,36)] + │ └── (l_extendedprice:22 * (1.0 - l_discount:23)) - (ps_supplycost:36 * l_quantity:21) [as=amount:52, outer=(21-23,36), immutable] └── aggregations └── sum [as=sum:53, outer=(52)] └── amount:52 @@ -1117,11 +1129,13 @@ limit ├── columns: c_custkey:1!null c_name:2!null revenue:39!null c_acctbal:6!null n_name:35!null c_address:3!null c_phone:5!null c_comment:8!null ├── internal-ordering: -39 ├── cardinality: [0 - 20] + ├── immutable ├── key: (1) ├── fd: (1)-->(2,3,5,6,8,35,39) ├── ordering: -39 ├── sort │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null sum:39!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2,3,5,6,8,35,39) │ ├── ordering: -39 @@ -1129,10 +1143,12 @@ limit │ └── group-by │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null sum:39!null │ ├── grouping columns: c_custkey:1!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2,3,5,6,8,35,39) │ ├── project │ │ ├── columns: column38:38!null c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null + │ │ ├── immutable │ │ ├── fd: (1)-->(2,3,5,6,8,35) │ │ ├── inner-join (hash) │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null n_nationkey:34!null n_name:35!null @@ -1170,7 +1186,7 @@ limit │ │ │ └── filters │ │ │ └── c_nationkey:4 = n_nationkey:34 [outer=(4,34), constraints=(/4: (/NULL - ]; /34: (/NULL - ]), fd=(4)==(34), (34)==(4)] │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column38:38, outer=(23,24)] + │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column38:38, outer=(23,24), immutable] │ └── aggregations │ ├── sum [as=sum:39, outer=(38)] │ │ └── column38:38 @@ -1334,7 +1350,7 @@ sort │ └── sum [as=sum:36, outer=(35)] │ └── column35:35 └── projections - └── sum:36 * 0.0001 [as="?column?":37, outer=(36)] + └── sum:36 * 0.0001 [as="?column?":37, outer=(36), immutable] # -------------------------------------------------- # Q12 @@ -1527,10 +1543,12 @@ project ├── scalar-group-by │ ├── columns: sum:27 sum:29 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () │ ├── fd: ()-->(27,29) │ ├── project │ │ ├── columns: column26:26!null column28:28!null + │ │ ├── immutable │ │ ├── inner-join (hash) │ │ │ ├── columns: l_partkey:2!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null p_partkey:17!null p_type:21!null │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) @@ -1549,8 +1567,8 @@ project │ │ │ └── filters │ │ │ └── l_partkey:2 = p_partkey:17 [outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] │ │ └── projections - │ │ ├── CASE WHEN p_type:21 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column26:26, outer=(6,7,21)] - │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, outer=(6,7)] + │ │ ├── CASE WHEN p_type:21 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column26:26, outer=(6,7,21), immutable] + │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, outer=(6,7), immutable] │ └── aggregations │ ├── sum [as=sum:27, outer=(26)] │ │ └── column26:26 @@ -1607,6 +1625,7 @@ ORDER BY ---- project ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null total_revenue:25!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2,3,5,25) ├── ordering: +1 @@ -1614,6 +1633,7 @@ project ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null l_suppkey:10!null sum:25!null ├── left ordering: +1 ├── right ordering: +10 + ├── immutable ├── key: (10) ├── fd: (1)-->(2,3,5), (10)-->(25), (1)==(10), (10)==(1) ├── ordering: +(1|10) [actual: +1] @@ -1624,20 +1644,24 @@ project │ └── ordering: +1 ├── sort │ ├── columns: l_suppkey:10!null sum:25!null + │ ├── immutable │ ├── key: (10) │ ├── fd: (10)-->(25) │ ├── ordering: +10 │ └── select │ ├── columns: l_suppkey:10!null sum:25!null + │ ├── immutable │ ├── key: (10) │ ├── fd: (10)-->(25) │ ├── group-by │ │ ├── columns: l_suppkey:10!null sum:25!null │ │ ├── grouping columns: l_suppkey:10!null + │ │ ├── immutable │ │ ├── key: (10) │ │ ├── fd: (10)-->(25) │ │ ├── project │ │ │ ├── columns: column24:24!null l_suppkey:10!null + │ │ │ ├── immutable │ │ │ ├── index-join lineitem │ │ │ │ ├── columns: l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null │ │ │ │ └── scan lineitem@l_sd @@ -1646,26 +1670,29 @@ project │ │ │ │ ├── key: (8,11) │ │ │ │ └── fd: (8,11)-->(18) │ │ │ └── projections - │ │ │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=column24:24, outer=(13,14)] + │ │ │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=column24:24, outer=(13,14), immutable] │ │ └── aggregations │ │ └── sum [as=sum:25, outer=(24)] │ │ └── column24:24 │ └── filters - │ └── eq [outer=(25), subquery, constraints=(/25: (/NULL - ])] + │ └── eq [outer=(25), immutable, subquery, constraints=(/25: (/NULL - ])] │ ├── sum:25 │ └── subquery │ └── scalar-group-by │ ├── columns: max:44 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () │ ├── fd: ()-->(44) │ ├── group-by │ │ ├── columns: l_suppkey:28!null sum:43!null │ │ ├── grouping columns: l_suppkey:28!null + │ │ ├── immutable │ │ ├── key: (28) │ │ ├── fd: (28)-->(43) │ │ ├── project │ │ │ ├── columns: column42:42!null l_suppkey:28!null + │ │ │ ├── immutable │ │ │ ├── index-join lineitem │ │ │ │ ├── columns: l_suppkey:28!null l_extendedprice:31!null l_discount:32!null l_shipdate:36!null │ │ │ │ └── scan lineitem@l_sd @@ -1674,7 +1701,7 @@ project │ │ │ │ ├── key: (26,29) │ │ │ │ └── fd: (26,29)-->(36) │ │ │ └── projections - │ │ │ └── l_extendedprice:31 * (1.0 - l_discount:32) [as=column42:42, outer=(31,32)] + │ │ │ └── l_extendedprice:31 * (1.0 - l_discount:32) [as=column42:42, outer=(31,32), immutable] │ │ └── aggregations │ │ └── sum [as=sum:43, outer=(42)] │ │ └── column42:42 @@ -1820,25 +1847,30 @@ WHERE project ├── columns: avg_yearly:45 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(45) ├── scalar-group-by │ ├── columns: sum:44 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () │ ├── fd: ()-->(44) │ ├── inner-join (lookup lineitem) │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null p_partkey:17!null "?column?":43!null │ │ ├── key columns: [1 4] = [1 4] │ │ ├── lookup columns are key + │ │ ├── immutable │ │ ├── fd: (17)-->(43), (2)==(17), (17)==(2) │ │ ├── inner-join (lookup lineitem@l_pk) │ │ │ ├── columns: l_orderkey:1!null l_partkey:2!null l_linenumber:4!null p_partkey:17!null "?column?":43 │ │ │ ├── key columns: [17] = [2] + │ │ │ ├── immutable │ │ │ ├── key: (1,4) │ │ │ ├── fd: (17)-->(43), (1,4)-->(2), (2)==(17), (17)==(2) │ │ │ ├── project │ │ │ │ ├── columns: "?column?":43 p_partkey:17!null + │ │ │ │ ├── immutable │ │ │ │ ├── key: (17) │ │ │ │ ├── fd: (17)-->(43) │ │ │ │ ├── group-by @@ -1878,7 +1910,7 @@ project │ │ │ │ │ └── avg [as=avg:42, outer=(30)] │ │ │ │ │ └── l_quantity:30 │ │ │ │ └── projections - │ │ │ │ └── avg:42 * 0.2 [as="?column?":43, outer=(42)] + │ │ │ │ └── avg:42 * 0.2 [as="?column?":43, outer=(42), immutable] │ │ │ └── filters (true) │ │ └── filters │ │ └── l_quantity:5 < "?column?":43 [outer=(5,43), constraints=(/5: (/NULL - ]; /43: (/NULL - ])] @@ -2067,10 +2099,12 @@ WHERE scalar-group-by ├── columns: revenue:27 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(27) ├── project │ ├── columns: column26:26!null + │ ├── immutable │ ├── inner-join (hash) │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipinstruct:14!null l_shipmode:15!null p_partkey:17!null p_brand:20!null p_size:22!null p_container:23!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) @@ -2097,7 +2131,7 @@ scalar-group-by │ │ ├── p_partkey:17 = l_partkey:2 [outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] │ │ └── ((((((p_brand:20 = 'Brand#12') AND (p_container:23 IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity:5 >= 1.0)) AND (l_quantity:5 <= 11.0)) AND (p_size:22 <= 5)) OR (((((p_brand:20 = 'Brand#23') AND (p_container:23 IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity:5 >= 10.0)) AND (l_quantity:5 <= 20.0)) AND (p_size:22 <= 10))) OR (((((p_brand:20 = 'Brand#34') AND (p_container:23 IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity:5 >= 20.0)) AND (l_quantity:5 <= 30.0)) AND (p_size:22 <= 15)) [outer=(5,20,22,23), constraints=(/5: [/1.0 - /30.0]; /20: [/'Brand#12' - /'Brand#12'] [/'Brand#23' - /'Brand#23'] [/'Brand#34' - /'Brand#34']; /22: (/NULL - /15]; /23: [/'LG BOX' - /'LG BOX'] [/'LG CASE' - /'LG CASE'] [/'LG PACK' - /'LG PACK'] [/'LG PKG' - /'LG PKG'] [/'MED BAG' - /'MED BAG'] [/'MED BOX' - /'MED BOX'] [/'MED PACK' - /'MED PACK'] [/'MED PKG' - /'MED PKG'] [/'SM BOX' - /'SM BOX'] [/'SM CASE' - /'SM CASE'] [/'SM PACK' - /'SM PACK'] [/'SM PKG' - /'SM PKG'])] │ └── projections - │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column26:26, outer=(6,7)] + │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column26:26, outer=(6,7), immutable] └── aggregations └── sum [as=sum:27, outer=(26)] └── column26:26 @@ -2157,16 +2191,20 @@ ORDER BY ---- sort ├── columns: s_name:2!null s_address:3!null + ├── immutable ├── ordering: +2 └── project ├── columns: s_name:2!null s_address:3!null + ├── immutable └── inner-join (hash) ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null n_nationkey:8!null n_name:9!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + ├── immutable ├── key: (1) ├── fd: ()-->(9), (1)-->(2-4), (4)==(8), (8)==(4) ├── semi-join (hash) │ ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── scan supplier @@ -2175,18 +2213,22 @@ sort │ │ └── fd: (1)-->(2-4) │ ├── project │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null + │ │ ├── immutable │ │ ├── key: (12,13) │ │ └── project │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null p_partkey:17!null + │ │ ├── immutable │ │ ├── key: (13,17) │ │ ├── fd: (12)==(17), (17)==(12) │ │ └── inner-join (hash) │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null p_partkey:17!null p_name:18!null sum:42 │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + │ │ ├── immutable │ │ ├── key: (13,17) │ │ ├── fd: (12,13)-->(14,42), (17)-->(18), (12)==(17), (17)==(12) │ │ ├── select │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null sum:42 + │ │ │ ├── immutable │ │ │ ├── key: (12,13) │ │ │ ├── fd: (12,13)-->(14,42) │ │ │ ├── group-by @@ -2217,7 +2259,7 @@ sort │ │ │ │ └── const-agg [as=ps_availqty:14, outer=(14)] │ │ │ │ └── ps_availqty:14 │ │ │ └── filters - │ │ │ └── ps_availqty:14 > (sum:42 * 0.5) [outer=(14,42), constraints=(/14: (/NULL - ])] + │ │ │ └── ps_availqty:14 > (sum:42 * 0.5) [outer=(14,42), immutable, constraints=(/14: (/NULL - ])] │ │ ├── select │ │ │ ├── columns: p_partkey:17!null p_name:18!null │ │ │ ├── key: (17) diff --git a/pkg/sql/opt/xform/testdata/external/tpch-no-stats b/pkg/sql/opt/xform/testdata/external/tpch-no-stats index afc1dfbeb09c..b248d7168fb7 100644 --- a/pkg/sql/opt/xform/testdata/external/tpch-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpch-no-stats @@ -40,14 +40,17 @@ ORDER BY group-by ├── columns: l_returnflag:9!null l_linestatus:10!null sum_qty:17!null sum_base_price:18!null sum_disc_price:20!null sum_charge:22!null avg_qty:23!null avg_price:24!null avg_disc:25!null count_order:26!null ├── grouping columns: l_returnflag:9!null l_linestatus:10!null + ├── immutable ├── key: (9,10) ├── fd: (9,10)-->(17,18,20,22-26) ├── ordering: +9,+10 ├── sort │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_returnflag:9!null l_linestatus:10!null column19:19!null column21:21!null + │ ├── immutable │ ├── ordering: +9,+10 │ └── project │ ├── columns: column19:19!null column21:21!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_returnflag:9!null l_linestatus:10!null + │ ├── immutable │ ├── select │ │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_tax:8!null l_returnflag:9!null l_linestatus:10!null l_shipdate:11!null │ │ ├── scan lineitem @@ -55,8 +58,8 @@ group-by │ │ └── filters │ │ └── l_shipdate:11 <= '1998-09-02' [outer=(11), constraints=(/11: (/NULL - /'1998-09-02']; tight)] │ └── projections - │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column19:19, outer=(6,7)] - │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column21:21, outer=(6-8)] + │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column19:19, outer=(6,7), immutable] + │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column21:21, outer=(6-8), immutable] └── aggregations ├── sum [as=sum:17, outer=(5)] │ └── l_quantity:5 @@ -339,11 +342,13 @@ limit ├── columns: l_orderkey:18!null revenue:35!null o_orderdate:13!null o_shippriority:16!null ├── internal-ordering: -35,+13 ├── cardinality: [0 - 10] + ├── immutable ├── key: (18) ├── fd: (18)-->(13,16,35) ├── ordering: -35,+13 ├── sort │ ├── columns: o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null sum:35!null + │ ├── immutable │ ├── key: (18) │ ├── fd: (18)-->(13,16,35) │ ├── ordering: -35,+13 @@ -351,10 +356,12 @@ limit │ └── group-by │ ├── columns: o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null sum:35!null │ ├── grouping columns: l_orderkey:18!null + │ ├── immutable │ ├── key: (18) │ ├── fd: (18)-->(13,16,35) │ ├── project │ │ ├── columns: column34:34!null o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null + │ │ ├── immutable │ │ ├── fd: (18)-->(13,16) │ │ ├── inner-join (hash) │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_shipdate:28!null @@ -393,7 +400,7 @@ limit │ │ │ └── filters │ │ │ └── l_orderkey:18 = o_orderkey:9 [outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ]), fd=(9)==(18), (18)==(9)] │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column34:34, outer=(23,24)] + │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column34:34, outer=(23,24), immutable] │ └── aggregations │ ├── sum [as=sum:35, outer=(34)] │ │ └── column34:34 @@ -520,16 +527,19 @@ ORDER BY ---- sort ├── columns: n_name:42!null revenue:49!null + ├── immutable ├── key: (42) ├── fd: (42)-->(49) ├── ordering: -49 └── group-by ├── columns: n_name:42!null sum:49!null ├── grouping columns: n_name:42!null + ├── immutable ├── key: (42) ├── fd: (42)-->(49) ├── project │ ├── columns: column48:48!null n_name:42!null + │ ├── immutable │ ├── inner-join (hash) │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null s_suppkey:34!null s_nationkey:37!null n_nationkey:41!null n_name:42!null n_regionkey:43!null r_regionkey:45!null r_name:46!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) @@ -600,7 +610,7 @@ sort │ │ ├── c_custkey:1 = o_custkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ │ └── c_nationkey:4 = s_nationkey:37 [outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ]), fd=(4)==(37), (37)==(4)] │ └── projections - │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column48:48, outer=(23,24)] + │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column48:48, outer=(23,24), immutable] └── aggregations └── sum [as=sum:49, outer=(48)] └── column48:48 @@ -634,10 +644,12 @@ WHERE scalar-group-by ├── columns: revenue:18 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(18) ├── project │ ├── columns: column17:17!null + │ ├── immutable │ ├── select │ │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null │ │ ├── scan lineitem @@ -647,7 +659,7 @@ scalar-group-by │ │ ├── (l_shipdate:11 >= '1994-01-01') AND (l_shipdate:11 < '1995-01-01') [outer=(11), constraints=(/11: [/'1994-01-01' - /'1994-12-31']; tight)] │ │ └── l_quantity:5 < 24.0 [outer=(5), constraints=(/5: (/NULL - /23.999999999999996]; tight)] │ └── projections - │ └── l_extendedprice:6 * l_discount:7 [as=column17:17, outer=(6,7)] + │ └── l_extendedprice:6 * l_discount:7 [as=column17:17, outer=(6,7), immutable] └── aggregations └── sum [as=sum:18, outer=(17)] └── column17:17 @@ -780,7 +792,7 @@ group-by │ │ └── s_nationkey:4 = n1.n_nationkey:41 [outer=(4,41), constraints=(/4: (/NULL - ]; /41: (/NULL - ]), fd=(4)==(41), (41)==(4)] │ └── projections │ ├── extract('year', l_shipdate:18) [as=l_year:49, outer=(18), immutable] - │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=volume:50, outer=(13,14)] + │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=volume:50, outer=(13,14), immutable] └── aggregations └── sum [as=sum:51, outer=(50)] └── volume:50 @@ -943,7 +955,7 @@ sort │ │ │ │ └── p_type:5 = 'ECONOMY ANODIZED STEEL' [outer=(5), constraints=(/5: [/'ECONOMY ANODIZED STEEL' - /'ECONOMY ANODIZED STEEL']; tight), fd=()-->(5)] │ │ │ └── projections │ │ │ ├── extract('year', o_orderdate:37) [as=o_year:61, outer=(37), immutable] - │ │ │ └── l_extendedprice:22 * (1.0 - l_discount:23) [as=volume:62, outer=(22,23)] + │ │ │ └── l_extendedprice:22 * (1.0 - l_discount:23) [as=volume:62, outer=(22,23), immutable] │ │ └── projections │ │ └── CASE WHEN n2.n_name:55 = 'BRAZIL' THEN volume:62 ELSE 0.0 END [as=column63:63, outer=(55,62)] │ └── aggregations @@ -1069,7 +1081,7 @@ sort │ │ └── p_name:2 LIKE '%green%' [outer=(2), constraints=(/2: (/NULL - ])] │ └── projections │ ├── extract('year', o_orderdate:42) [as=o_year:51, outer=(42), immutable] - │ └── (l_extendedprice:22 * (1.0 - l_discount:23)) - (ps_supplycost:36 * l_quantity:21) [as=amount:52, outer=(21-23,36)] + │ └── (l_extendedprice:22 * (1.0 - l_discount:23)) - (ps_supplycost:36 * l_quantity:21) [as=amount:52, outer=(21-23,36), immutable] └── aggregations └── sum [as=sum:53, outer=(52)] └── amount:52 @@ -1126,11 +1138,13 @@ limit ├── columns: c_custkey:1!null c_name:2!null revenue:39!null c_acctbal:6!null n_name:35!null c_address:3!null c_phone:5!null c_comment:8!null ├── internal-ordering: -39 ├── cardinality: [0 - 20] + ├── immutable ├── key: (1) ├── fd: (1)-->(2,3,5,6,8,35,39) ├── ordering: -39 ├── sort │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null sum:39!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2,3,5,6,8,35,39) │ ├── ordering: -39 @@ -1138,10 +1152,12 @@ limit │ └── group-by │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null sum:39!null │ ├── grouping columns: c_custkey:1!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2,3,5,6,8,35,39) │ ├── project │ │ ├── columns: column38:38!null c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null + │ │ ├── immutable │ │ ├── fd: (1)-->(2,3,5,6,8,35) │ │ ├── inner-join (lookup nation) │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null n_nationkey:34!null n_name:35!null @@ -1170,7 +1186,7 @@ limit │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column38:38, outer=(23,24)] + │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column38:38, outer=(23,24), immutable] │ └── aggregations │ ├── sum [as=sum:39, outer=(38)] │ │ └── column38:38 @@ -1328,7 +1344,7 @@ sort │ └── sum [as=sum:36, outer=(35)] │ └── column35:35 └── projections - └── sum:36 * 0.0001 [as="?column?":37, outer=(36)] + └── sum:36 * 0.0001 [as="?column?":37, outer=(36), immutable] # -------------------------------------------------- # Q12 @@ -1518,10 +1534,12 @@ project ├── scalar-group-by │ ├── columns: sum:27 sum:29 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () │ ├── fd: ()-->(27,29) │ ├── project │ │ ├── columns: column26:26!null column28:28!null + │ │ ├── immutable │ │ ├── inner-join (hash) │ │ │ ├── columns: l_partkey:2!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null p_partkey:17!null p_type:21!null │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) @@ -1539,8 +1557,8 @@ project │ │ │ └── filters │ │ │ └── l_partkey:2 = p_partkey:17 [outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] │ │ └── projections - │ │ ├── CASE WHEN p_type:21 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column26:26, outer=(6,7,21)] - │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, outer=(6,7)] + │ │ ├── CASE WHEN p_type:21 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column26:26, outer=(6,7,21), immutable] + │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, outer=(6,7), immutable] │ └── aggregations │ ├── sum [as=sum:27, outer=(26)] │ │ └── column26:26 @@ -1597,30 +1615,36 @@ ORDER BY ---- sort ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null total_revenue:25!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2,3,5,25) ├── ordering: +1 └── project ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null sum:25!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2,3,5,25) └── inner-join (lookup supplier) ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null l_suppkey:10!null sum:25!null ├── key columns: [10] = [1] ├── lookup columns are key + ├── immutable ├── key: (10) ├── fd: (1)-->(2,3,5), (10)-->(25), (1)==(10), (10)==(1) ├── select │ ├── columns: l_suppkey:10!null sum:25!null + │ ├── immutable │ ├── key: (10) │ ├── fd: (10)-->(25) │ ├── group-by │ │ ├── columns: l_suppkey:10!null sum:25!null │ │ ├── grouping columns: l_suppkey:10!null + │ │ ├── immutable │ │ ├── key: (10) │ │ ├── fd: (10)-->(25) │ │ ├── project │ │ │ ├── columns: column24:24!null l_suppkey:10!null + │ │ │ ├── immutable │ │ │ ├── select │ │ │ │ ├── columns: l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null │ │ │ │ ├── scan lineitem @@ -1628,26 +1652,29 @@ sort │ │ │ │ └── filters │ │ │ │ └── (l_shipdate:18 >= '1996-01-01') AND (l_shipdate:18 < '1996-04-01') [outer=(18), constraints=(/18: [/'1996-01-01' - /'1996-03-31']; tight)] │ │ │ └── projections - │ │ │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=column24:24, outer=(13,14)] + │ │ │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=column24:24, outer=(13,14), immutable] │ │ └── aggregations │ │ └── sum [as=sum:25, outer=(24)] │ │ └── column24:24 │ └── filters - │ └── eq [outer=(25), subquery, constraints=(/25: (/NULL - ])] + │ └── eq [outer=(25), immutable, subquery, constraints=(/25: (/NULL - ])] │ ├── sum:25 │ └── subquery │ └── scalar-group-by │ ├── columns: max:44 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () │ ├── fd: ()-->(44) │ ├── group-by │ │ ├── columns: l_suppkey:28!null sum:43!null │ │ ├── grouping columns: l_suppkey:28!null + │ │ ├── immutable │ │ ├── key: (28) │ │ ├── fd: (28)-->(43) │ │ ├── project │ │ │ ├── columns: column42:42!null l_suppkey:28!null + │ │ │ ├── immutable │ │ │ ├── select │ │ │ │ ├── columns: l_suppkey:28!null l_extendedprice:31!null l_discount:32!null l_shipdate:36!null │ │ │ │ ├── scan lineitem @@ -1655,7 +1682,7 @@ sort │ │ │ │ └── filters │ │ │ │ └── (l_shipdate:36 >= '1996-01-01') AND (l_shipdate:36 < '1996-04-01') [outer=(36), constraints=(/36: [/'1996-01-01' - /'1996-03-31']; tight)] │ │ │ └── projections - │ │ │ └── l_extendedprice:31 * (1.0 - l_discount:32) [as=column42:42, outer=(31,32)] + │ │ │ └── l_extendedprice:31 * (1.0 - l_discount:32) [as=column42:42, outer=(31,32), immutable] │ │ └── aggregations │ │ └── sum [as=sum:43, outer=(42)] │ │ └── column42:42 @@ -1795,25 +1822,30 @@ WHERE project ├── columns: avg_yearly:45 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(45) ├── scalar-group-by │ ├── columns: sum:44 │ ├── cardinality: [1 - 1] + │ ├── immutable │ ├── key: () │ ├── fd: ()-->(44) │ ├── inner-join (lookup lineitem) │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null p_partkey:17!null "?column?":43!null │ │ ├── key columns: [1 4] = [1 4] │ │ ├── lookup columns are key + │ │ ├── immutable │ │ ├── fd: (17)-->(43), (2)==(17), (17)==(2) │ │ ├── inner-join (lookup lineitem@l_pk) │ │ │ ├── columns: l_orderkey:1!null l_partkey:2!null l_linenumber:4!null p_partkey:17!null "?column?":43 │ │ │ ├── key columns: [17] = [2] + │ │ │ ├── immutable │ │ │ ├── key: (1,4) │ │ │ ├── fd: (17)-->(43), (1,4)-->(2), (2)==(17), (17)==(2) │ │ │ ├── project │ │ │ │ ├── columns: "?column?":43 p_partkey:17!null + │ │ │ │ ├── immutable │ │ │ │ ├── key: (17) │ │ │ │ ├── fd: (17)-->(43) │ │ │ │ ├── group-by @@ -1853,7 +1885,7 @@ project │ │ │ │ │ └── avg [as=avg:42, outer=(30)] │ │ │ │ │ └── l_quantity:30 │ │ │ │ └── projections - │ │ │ │ └── avg:42 * 0.2 [as="?column?":43, outer=(42)] + │ │ │ │ └── avg:42 * 0.2 [as="?column?":43, outer=(42), immutable] │ │ │ └── filters (true) │ │ └── filters │ │ └── l_quantity:5 < "?column?":43 [outer=(5,43), constraints=(/5: (/NULL - ]; /43: (/NULL - ])] @@ -2048,10 +2080,12 @@ WHERE scalar-group-by ├── columns: revenue:27 ├── cardinality: [1 - 1] + ├── immutable ├── key: () ├── fd: ()-->(27) ├── project │ ├── columns: column26:26!null + │ ├── immutable │ ├── inner-join (lookup part) │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipinstruct:14!null l_shipmode:15!null p_partkey:17!null p_brand:20!null p_size:22!null p_container:23!null │ │ ├── key columns: [2] = [17] @@ -2069,7 +2103,7 @@ scalar-group-by │ │ ├── ((((((p_brand:20 = 'Brand#12') AND (p_container:23 IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity:5 >= 1.0)) AND (l_quantity:5 <= 11.0)) AND (p_size:22 <= 5)) OR (((((p_brand:20 = 'Brand#23') AND (p_container:23 IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity:5 >= 10.0)) AND (l_quantity:5 <= 20.0)) AND (p_size:22 <= 10))) OR (((((p_brand:20 = 'Brand#34') AND (p_container:23 IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity:5 >= 20.0)) AND (l_quantity:5 <= 30.0)) AND (p_size:22 <= 15)) [outer=(5,20,22,23), constraints=(/5: [/1.0 - /30.0]; /20: [/'Brand#12' - /'Brand#12'] [/'Brand#23' - /'Brand#23'] [/'Brand#34' - /'Brand#34']; /22: (/NULL - /15]; /23: [/'LG BOX' - /'LG BOX'] [/'LG CASE' - /'LG CASE'] [/'LG PACK' - /'LG PACK'] [/'LG PKG' - /'LG PKG'] [/'MED BAG' - /'MED BAG'] [/'MED BOX' - /'MED BOX'] [/'MED PACK' - /'MED PACK'] [/'MED PKG' - /'MED PKG'] [/'SM BOX' - /'SM BOX'] [/'SM CASE' - /'SM CASE'] [/'SM PACK' - /'SM PACK'] [/'SM PKG' - /'SM PKG'])] │ │ └── p_size:22 >= 1 [outer=(22), constraints=(/22: [/1 - ]; tight)] │ └── projections - │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column26:26, outer=(6,7)] + │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column26:26, outer=(6,7), immutable] └── aggregations └── sum [as=sum:27, outer=(26)] └── column26:26 @@ -2129,37 +2163,46 @@ ORDER BY ---- sort ├── columns: s_name:2!null s_address:3!null + ├── immutable ├── ordering: +2 └── project ├── columns: s_name:2!null s_address:3!null + ├── immutable └── inner-join (lookup nation) ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null n_nationkey:8!null n_name:9!null ├── key columns: [4] = [8] ├── lookup columns are key + ├── immutable ├── key: (1) ├── fd: ()-->(9), (1)-->(2-4), (4)==(8), (8)==(4) ├── project │ ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null + │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ └── inner-join (lookup supplier) │ ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null ps_suppkey:13!null │ ├── key columns: [13] = [1] │ ├── lookup columns are key + │ ├── immutable │ ├── key: (13) │ ├── fd: (1)-->(2-4), (1)==(13), (13)==(1) │ ├── distinct-on │ │ ├── columns: ps_suppkey:13!null │ │ ├── grouping columns: ps_suppkey:13!null + │ │ ├── immutable │ │ ├── key: (13) │ │ └── semi-join (hash) │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null + │ │ ├── immutable │ │ ├── key: (12,13) │ │ ├── project │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null + │ │ │ ├── immutable │ │ │ ├── key: (12,13) │ │ │ └── select │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null sum:42 + │ │ │ ├── immutable │ │ │ ├── key: (12,13) │ │ │ ├── fd: (12,13)-->(14,42) │ │ │ ├── group-by @@ -2190,7 +2233,7 @@ sort │ │ │ │ └── const-agg [as=ps_availqty:14, outer=(14)] │ │ │ │ └── ps_availqty:14 │ │ │ └── filters - │ │ │ └── ps_availqty:14 > (sum:42 * 0.5) [outer=(14,42), constraints=(/14: (/NULL - ])] + │ │ │ └── ps_availqty:14 > (sum:42 * 0.5) [outer=(14,42), immutable, constraints=(/14: (/NULL - ])] │ │ ├── select │ │ │ ├── columns: p_partkey:17!null p_name:18!null │ │ │ ├── key: (17) diff --git a/pkg/sql/opt/xform/testdata/external/trading b/pkg/sql/opt/xform/testdata/external/trading index 1d0e0eff7284..52ab6fbdbafc 100644 --- a/pkg/sql/opt/xform/testdata/external/trading +++ b/pkg/sql/opt/xform/testdata/external/trading @@ -549,6 +549,7 @@ FROM CardsView WHERE Version > 1584421773604892000.0000000000 ---- project ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:9!null sellprice:10!null desiredinventory:12!null actualinventory:13!null version:15!null discount:11!null maxinventory:14!null + ├── immutable ├── stats: [rows=1] ├── key: (15) ├── fd: (1)-->(2-6,9-15), (2,4,5)~~>(1,3,6), (15)-->(1-6,9-14) @@ -556,11 +557,13 @@ project ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null dealerid:7!null cardid:8!null buyprice:9!null sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null version:15!null ├── key columns: [8] = [1] ├── lookup columns are key + ├── immutable ├── stats: [rows=1, distinct(1)=0.0201621393, null(1)=0, distinct(8)=0.0201621393, null(8)=0] ├── key: (8) ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1) ├── index-join cardsinfo │ ├── columns: dealerid:7!null cardid:8!null buyprice:9!null sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null version:15!null + │ ├── immutable │ ├── stats: [rows=0.0201621426, distinct(7)=0.0201621426, null(7)=0, distinct(8)=0.0201621393, null(8)=0, distinct(9)=0.02016214, null(9)=0, distinct(10)=0.02016214, null(10)=0, distinct(11)=0.02016214, null(11)=0, distinct(12)=0.02016214, null(12)=0, distinct(13)=0.02016214, null(13)=0, distinct(14)=0.02016214, null(14)=0, distinct(15)=0.0201621426, null(15)=0, distinct(7,15)=0.0201621426, null(7,15)=0] │ │ histogram(15)= 0 0 0.020162 0 │ │ <--- 1584421773604892000.0000000000 ---------- 1584421778604892000 @@ -735,6 +738,7 @@ LIMIT 50 project ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:9!null sellprice:10!null desiredinventory:12!null actualinventory:13!null version:15!null discount:11!null maxinventory:14!null twodaysales:25 ├── cardinality: [0 - 50] + ├── immutable ├── stats: [rows=50] ├── key: (15,25) ├── fd: (1)-->(2-6,9-15), (2,4,5)~~>(1,3,6), (15)-->(1-6,9-14) @@ -743,12 +747,14 @@ project │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:24 │ ├── internal-ordering: +2,+4,+5 │ ├── cardinality: [0 - 50] + │ ├── immutable │ ├── stats: [rows=50] │ ├── key: (8) │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,24), (15)-->(8-14), (1)==(8), (8)==(1) │ ├── ordering: +2,+4,+5 │ ├── sort │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:24 + │ │ ├── immutable │ │ ├── stats: [rows=19000, distinct(8)=19000, null(8)=0] │ │ ├── key: (8) │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,24), (15)-->(8-14), (1)==(8), (8)==(1) @@ -757,11 +763,13 @@ project │ │ └── group-by │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:24 │ │ ├── grouping columns: cardsinfo.cardid:8!null + │ │ ├── immutable │ │ ├── stats: [rows=19000, distinct(8)=19000, null(8)=0] │ │ ├── key: (8) │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,24), (15)-->(8-14), (1)==(8), (8)==(1) │ │ ├── right-join (hash) │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null transactiondetails.dealerid:16 isbuy:17 transactiondate:18 transactiondetails.cardid:19 quantity:20 + │ │ │ ├── immutable │ │ │ ├── stats: [rows=5523583.18, distinct(8)=19000, null(8)=0, distinct(19)=19000, null(19)=0] │ │ │ ├── key: (8,18-20) │ │ │ ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1), (8,18-20)-->(16,17) @@ -775,11 +783,13 @@ project │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null │ │ │ │ ├── left ordering: +1 │ │ │ │ ├── right ordering: +8 + │ │ │ │ ├── immutable │ │ │ │ ├── stats: [rows=29618.4611, distinct(1)=19000, null(1)=0, distinct(2)=11668.1409, null(2)=0, distinct(5)=829, null(5)=0, distinct(6)=5572.85686, null(6)=0, distinct(7)=1, null(7)=0, distinct(8)=19000, null(8)=0, distinct(9)=21037.9959, null(9)=0, distinct(10)=21037.9959, null(10)=0, distinct(11)=21037.9959, null(11)=0, distinct(12)=21037.9959, null(12)=0, distinct(13)=21037.9959, null(13)=0, distinct(14)=21037.9959, null(14)=0, distinct(15)=23225.5851, null(15)=0] │ │ │ │ ├── key: (8) │ │ │ │ ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1) │ │ │ │ ├── select │ │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null + │ │ │ │ │ ├── immutable │ │ │ │ │ ├── stats: [rows=19000, distinct(1)=19000, null(1)=0, distinct(2)=13000, null(2)=0, distinct(5)=829, null(5)=0, distinct(6)=5601.15328, null(6)=0] │ │ │ │ │ ├── key: (1) │ │ │ │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6) @@ -791,7 +801,7 @@ project │ │ │ │ │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6) │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ └── filters - │ │ │ │ │ └── (name:2, setname:4, number:5) > ('Shock', '7E', 248) [outer=(2,4,5), constraints=(/2/4/5: [/'Shock'/'7E'/249 - ]; tight)] + │ │ │ │ │ └── (name:2, setname:4, number:5) > ('Shock', '7E', 248) [outer=(2,4,5), immutable, constraints=(/2/4/5: [/'Shock'/'7E'/249 - ]; tight)] │ │ │ │ ├── scan cardsinfo │ │ │ │ │ ├── columns: cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null │ │ │ │ │ ├── constraint: /7/8: [/1 - /1] @@ -942,9 +952,9 @@ sort │ │ └── filters │ │ └── id:16 = transactiondetails.cardid:4 [outer=(4,16), constraints=(/4: (/NULL - ]; /16: (/NULL - ]), fd=(4)==(16), (16)==(4)] │ └── projections - │ ├── transactiondetails.sellprice:6 * quantity:5 [as=column31:31, outer=(5,6)] - │ ├── transactiondetails.buyprice:7 * quantity:5 [as=column33:33, outer=(5,7)] - │ ├── quantity:5 * (transactiondetails.sellprice:6 - transactiondetails.buyprice:7) [as=column35:35, outer=(5-7)] + │ ├── transactiondetails.sellprice:6 * quantity:5 [as=column31:31, outer=(5,6), immutable] + │ ├── transactiondetails.buyprice:7 * quantity:5 [as=column33:33, outer=(5,7), immutable] + │ ├── quantity:5 * (transactiondetails.sellprice:6 - transactiondetails.buyprice:7) [as=column35:35, outer=(5-7), immutable] │ └── extract('day', transactiondate:3) [as=column37:37, outer=(3), stable] └── aggregations ├── sum [as=sum:32, outer=(31)] diff --git a/pkg/sql/opt/xform/testdata/external/trading-mutation b/pkg/sql/opt/xform/testdata/external/trading-mutation index 40ebb10a4454..b48c839d8c85 100644 --- a/pkg/sql/opt/xform/testdata/external/trading-mutation +++ b/pkg/sql/opt/xform/testdata/external/trading-mutation @@ -557,6 +557,7 @@ FROM CardsView WHERE Version > 1584421773604892000.0000000000 ---- project ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:9!null sellprice:10!null desiredinventory:12!null actualinventory:13!null version:15!null discount:11!null maxinventory:14!null + ├── immutable ├── stats: [rows=1] ├── key: (15) ├── fd: (1)-->(2-6,9-15), (2,4,5)~~>(1,3,6), (15)-->(1-6,9-14) @@ -564,11 +565,13 @@ project ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null dealerid:7!null cardid:8!null buyprice:9!null sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null version:15!null ├── key columns: [8] = [1] ├── lookup columns are key + ├── immutable ├── stats: [rows=1, distinct(1)=6.35833333e-05, null(1)=0, distinct(8)=6.35833333e-05, null(8)=0] ├── key: (8) ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1) ├── index-join cardsinfo │ ├── columns: dealerid:7!null cardid:8!null buyprice:9!null sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null version:15!null + │ ├── immutable │ ├── stats: [rows=6.35833333e-05, distinct(7)=6.35833333e-05, null(7)=0, distinct(8)=6.35833333e-05, null(8)=0, distinct(9)=6.35833333e-05, null(9)=0, distinct(10)=6.35833333e-05, null(10)=0, distinct(11)=6.35833333e-05, null(11)=0, distinct(12)=6.35833333e-05, null(12)=0, distinct(13)=6.35833333e-05, null(13)=0, distinct(14)=6.35833333e-05, null(14)=0, distinct(15)=6.35833333e-05, null(15)=0, distinct(7,15)=6.35833333e-05, null(7,15)=0] │ │ histogram(15)= │ ├── key: (8) @@ -739,6 +742,7 @@ LIMIT 50 project ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:9!null sellprice:10!null desiredinventory:12!null actualinventory:13!null version:15!null discount:11!null maxinventory:14!null twodaysales:31 ├── cardinality: [0 - 50] + ├── immutable ├── stats: [rows=50] ├── key: (15,31) ├── fd: (1)-->(2-6,9-15), (2,4,5)~~>(1,3,6), (15)-->(1-6,9-14) @@ -747,12 +751,14 @@ project │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:30 │ ├── internal-ordering: +2,+4,+5 │ ├── cardinality: [0 - 50] + │ ├── immutable │ ├── stats: [rows=50] │ ├── key: (8) │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,30), (15)-->(8-14), (1)==(8), (8)==(1) │ ├── ordering: +2,+4,+5 │ ├── sort │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:30 + │ │ ├── immutable │ │ ├── stats: [rows=19000, distinct(8)=19000, null(8)=0] │ │ ├── key: (8) │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,30), (15)-->(8-14), (1)==(8), (8)==(1) @@ -761,11 +767,13 @@ project │ │ └── group-by │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:30 │ │ ├── grouping columns: cardsinfo.cardid:8!null + │ │ ├── immutable │ │ ├── stats: [rows=19000, distinct(8)=19000, null(8)=0] │ │ ├── key: (8) │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,30), (15)-->(8-14), (1)==(8), (8)==(1) │ │ ├── right-join (hash) │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null transactiondetails.dealerid:20 isbuy:21 transactiondate:22 transactiondetails.cardid:23 quantity:24 + │ │ │ ├── immutable │ │ │ ├── stats: [rows=5523583.18, distinct(8)=19000, null(8)=0, distinct(23)=19000, null(23)=0] │ │ │ ├── key: (8,22-24) │ │ │ ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1), (8,22-24)-->(20,21) @@ -779,11 +787,13 @@ project │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null │ │ │ │ ├── left ordering: +1 │ │ │ │ ├── right ordering: +8 + │ │ │ │ ├── immutable │ │ │ │ ├── stats: [rows=29618.4611, distinct(1)=19000, null(1)=0, distinct(2)=11668.1409, null(2)=0, distinct(5)=829, null(5)=0, distinct(6)=5572.85686, null(6)=0, distinct(7)=1, null(7)=0, distinct(8)=19000, null(8)=0, distinct(9)=21037.9959, null(9)=0, distinct(10)=21037.9959, null(10)=0, distinct(11)=21037.9959, null(11)=0, distinct(12)=21037.9959, null(12)=0, distinct(13)=21037.9959, null(13)=0, distinct(14)=21037.9959, null(14)=0, distinct(15)=23225.5851, null(15)=0] │ │ │ │ ├── key: (8) │ │ │ │ ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1) │ │ │ │ ├── select │ │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null + │ │ │ │ │ ├── immutable │ │ │ │ │ ├── stats: [rows=19000, distinct(1)=19000, null(1)=0, distinct(2)=13000, null(2)=0, distinct(5)=829, null(5)=0, distinct(6)=5601.15328, null(6)=0] │ │ │ │ │ ├── key: (1) │ │ │ │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6) @@ -795,7 +805,7 @@ project │ │ │ │ │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6) │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ └── filters - │ │ │ │ │ └── (name:2, setname:4, number:5) > ('Shock', '7E', 248) [outer=(2,4,5), constraints=(/2/4/5: [/'Shock'/'7E'/249 - ]; tight)] + │ │ │ │ │ └── (name:2, setname:4, number:5) > ('Shock', '7E', 248) [outer=(2,4,5), immutable, constraints=(/2/4/5: [/'Shock'/'7E'/249 - ]; tight)] │ │ │ │ ├── scan cardsinfo │ │ │ │ │ ├── columns: cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null │ │ │ │ │ ├── constraint: /7/8: [/1 - /1] @@ -946,9 +956,9 @@ sort │ │ └── filters │ │ └── id:20 = transactiondetails.cardid:4 [outer=(4,20), constraints=(/4: (/NULL - ]; /20: (/NULL - ]), fd=(4)==(20), (20)==(4)] │ └── projections - │ ├── transactiondetails.sellprice:6 * quantity:5 [as=column39:39, outer=(5,6)] - │ ├── transactiondetails.buyprice:7 * quantity:5 [as=column41:41, outer=(5,7)] - │ ├── quantity:5 * (transactiondetails.sellprice:6 - transactiondetails.buyprice:7) [as=column43:43, outer=(5-7)] + │ ├── transactiondetails.sellprice:6 * quantity:5 [as=column39:39, outer=(5,6), immutable] + │ ├── transactiondetails.buyprice:7 * quantity:5 [as=column41:41, outer=(5,7), immutable] + │ ├── quantity:5 * (transactiondetails.sellprice:6 - transactiondetails.buyprice:7) [as=column43:43, outer=(5-7), immutable] │ └── extract('day', transactiondate:3) [as=column45:45, outer=(3), stable] └── aggregations ├── sum [as=sum:40, outer=(39)] diff --git a/pkg/sql/opt/xform/testdata/physprops/ordering b/pkg/sql/opt/xform/testdata/physprops/ordering index dcd4ddebf5f9..2c50e2b95161 100644 --- a/pkg/sql/opt/xform/testdata/physprops/ordering +++ b/pkg/sql/opt/xform/testdata/physprops/ordering @@ -130,6 +130,7 @@ SELECT x+1 AS r, y FROM a ORDER BY x, y DESC ---- project ├── columns: r:5!null y:2!null [hidden: x:1!null] + ├── immutable ├── key: (1,2) ├── fd: (1)-->(5) ├── ordering: +1,-2 @@ -138,7 +139,7 @@ project │ ├── key: (1,2) │ └── ordering: +1,-2 └── projections - └── x:1 + 1 [as=r:5, outer=(1)] + └── x:1 + 1 [as=r:5, outer=(1), immutable] # Pass through ordering to scan operator that can't support it. opt @@ -146,11 +147,13 @@ SELECT y, x, z+1 AS r FROM a ORDER BY x, y ---- sort (segmented) ├── columns: y:2!null x:1!null r:5 + ├── immutable ├── key: (1,2) ├── fd: (1,2)-->(5) ├── ordering: +1,+2 └── project ├── columns: r:5 x:1!null y:2!null + ├── immutable ├── key: (1,2) ├── fd: (1,2)-->(5) ├── ordering: +1 @@ -160,7 +163,7 @@ sort (segmented) │ ├── fd: (1,2)-->(3) │ └── ordering: +1 └── projections - └── z:3 + 1 [as=r:5, outer=(3)] + └── z:3 + 1 [as=r:5, outer=(3), immutable] # Ordering cannot be passed through because it includes computed column. opt @@ -168,11 +171,13 @@ SELECT x, y+1 AS computed, y FROM a ORDER BY x, computed ---- sort (segmented) ├── columns: x:1!null computed:5!null y:2!null + ├── immutable ├── key: (1,2) ├── fd: (1,2)-->(5) ├── ordering: +1,+5 └── project ├── columns: computed:5!null x:1!null y:2!null + ├── immutable ├── key: (1,2) ├── fd: (1,2)-->(5) ├── ordering: +1 @@ -181,7 +186,7 @@ sort (segmented) │ ├── key: (1,2) │ └── ordering: +1 └── projections - └── y:2 + 1.0 [as=computed:5, outer=(2)] + └── y:2 + 1.0 [as=computed:5, outer=(2), immutable] # Ordering on an expression that gets constant-folded to a simple variable. # Example from #43360: a boolean (possibly a placeholder) indicates the sort @@ -243,6 +248,7 @@ SELECT y, x-1 AS z FROM a WHERE x>y ORDER BY x, y DESC ---- project ├── columns: y:2!null z:5!null [hidden: x:1!null] + ├── immutable ├── key: (1,2) ├── fd: (1)-->(5) ├── ordering: +1,-2 @@ -257,7 +263,7 @@ project │ └── filters │ └── x:1 > y:2 [outer=(1,2)] └── projections - └── x:1 - 1 [as=z:5, outer=(1)] + └── x:1 - 1 [as=z:5, outer=(1), immutable] memo SELECT y, x-1 AS z FROM a WHERE x>y ORDER BY x, y DESC @@ -990,10 +996,12 @@ limit ├── columns: a:1!null b:2!null c:3!null ├── internal-ordering: +1,+2 ├── cardinality: [0 - 10] + ├── immutable ├── key: (1-3) ├── ordering: +1,+2 ├── select │ ├── columns: a:1!null b:2!null c:3!null + │ ├── immutable │ ├── key: (1-3) │ ├── ordering: +1,+2 │ ├── limit hint: 10.00 @@ -1003,7 +1011,7 @@ limit │ │ ├── ordering: +1,+2 │ │ └── limit hint: 30.00 │ └── filters - │ └── c:3 < (a:1 + b:2) [outer=(1-3)] + │ └── c:3 < (a:1 + b:2) [outer=(1-3), immutable] └── 10 opt @@ -1043,15 +1051,18 @@ SELECT * FROM (SELECT * FROM abc WHERE a+b>c ORDER BY a, b LIMIT 10) ORDER BY b sort ├── columns: a:1!null b:2!null c:3!null ├── cardinality: [0 - 10] + ├── immutable ├── key: (1-3) ├── ordering: +2 └── limit ├── columns: a:1!null b:2!null c:3!null ├── internal-ordering: +1,+2 ├── cardinality: [0 - 10] + ├── immutable ├── key: (1-3) ├── select │ ├── columns: a:1!null b:2!null c:3!null + │ ├── immutable │ ├── key: (1-3) │ ├── ordering: +1,+2 │ ├── limit hint: 10.00 @@ -1061,7 +1072,7 @@ sort │ │ ├── ordering: +1,+2 │ │ └── limit hint: 30.00 │ └── filters - │ └── c:3 < (a:1 + b:2) [outer=(1-3)] + │ └── c:3 < (a:1 + b:2) [outer=(1-3), immutable] └── 10 opt @@ -1101,10 +1112,12 @@ limit ├── columns: a:1!null b:2!null c:3!null ├── internal-ordering: +1,+2 ├── cardinality: [0 - 10] + ├── immutable ├── key: (1-3) ├── ordering: +1 ├── select │ ├── columns: a:1!null b:2!null c:3!null + │ ├── immutable │ ├── key: (1-3) │ ├── ordering: +1,+2 │ ├── limit hint: 10.00 @@ -1114,7 +1127,7 @@ limit │ │ ├── ordering: +1,+2 │ │ └── limit hint: 30.00 │ └── filters - │ └── c:3 < (a:1 + b:2) [outer=(1-3)] + │ └── c:3 < (a:1 + b:2) [outer=(1-3), immutable] └── 10 opt @@ -1150,10 +1163,12 @@ limit ├── columns: a:1!null b:2!null c:3!null ├── internal-ordering: +1,+2 ├── cardinality: [0 - 10] + ├── immutable ├── key: (1-3) ├── ordering: +1,+2,+3 ├── select │ ├── columns: a:1!null b:2!null c:3!null + │ ├── immutable │ ├── key: (1-3) │ ├── ordering: +1,+2,+3 │ ├── limit hint: 10.00 @@ -1163,7 +1178,7 @@ limit │ │ ├── ordering: +1,+2,+3 │ │ └── limit hint: 30.00 │ └── filters - │ └── c:3 < (a:1 + b:2) [outer=(1-3)] + │ └── c:3 < (a:1 + b:2) [outer=(1-3), immutable] └── 10 opt @@ -1665,6 +1680,7 @@ sort │ └── project │ ├── columns: b_new:11 abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null │ ├── cardinality: [0 - 10] + │ ├── immutable │ ├── key: (10) │ ├── fd: (10)-->(6-9), (7)-->(11) │ ├── scan abcd@cd @@ -1673,7 +1689,7 @@ sort │ │ ├── key: (10) │ │ └── fd: (10)-->(6-9) │ └── projections - │ └── abcd.b:7 + 1 [as=b_new:11, outer=(7)] + │ └── abcd.b:7 + 1 [as=b_new:11, outer=(7), immutable] └── with-scan &1 ├── columns: a:12 b:13 c:14 d:15 ├── mapping: @@ -1718,6 +1734,7 @@ sort │ └── project │ ├── columns: b_new:11 abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null │ ├── cardinality: [0 - 10] + │ ├── immutable │ ├── key: (10) │ ├── fd: (10)-->(6-9), (7)-->(11) │ ├── scan abcd@cd @@ -1726,7 +1743,7 @@ sort │ │ ├── key: (10) │ │ └── fd: (10)-->(6-9) │ └── projections - │ └── abcd.b:7 + 1 [as=b_new:11, outer=(7)] + │ └── abcd.b:7 + 1 [as=b_new:11, outer=(7), immutable] └── select ├── columns: a:12 b:13!null c:14!null d:15 ├── cardinality: [0 - 10] diff --git a/pkg/sql/opt/xform/testdata/physprops/presentation b/pkg/sql/opt/xform/testdata/physprops/presentation index d8269dd86c88..966100a42bbf 100644 --- a/pkg/sql/opt/xform/testdata/physprops/presentation +++ b/pkg/sql/opt/xform/testdata/physprops/presentation @@ -36,6 +36,7 @@ SELECT 1+a.y AS plus, a.x FROM a ---- project ├── columns: plus:3 x:1!null + ├── immutable ├── key: (1) ├── fd: (1)-->(3) ├── scan a @@ -43,7 +44,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - └── y:2 + 1 [as=plus:3, outer=(2)] + └── y:2 + 1 [as=plus:3, outer=(2), immutable] # Join operator. opt diff --git a/pkg/sql/opt/xform/testdata/rules/computed b/pkg/sql/opt/xform/testdata/rules/computed index 09f83ebde9fe..1f013f657cf1 100644 --- a/pkg/sql/opt/xform/testdata/rules/computed +++ b/pkg/sql/opt/xform/testdata/rules/computed @@ -81,8 +81,10 @@ SELECT k_int FROM t_mult WHERE (k_int, k_int_2) > (1, 2) ---- project ├── columns: k_int:1!null + ├── immutable └── select ├── columns: k_int:1!null k_int_2:2 + ├── immutable ├── scan t_mult │ ├── columns: k_int:1 k_int_2:2 │ └── computed column expressions @@ -93,7 +95,7 @@ project │ └── c_mult_2:5 │ └── k_int:1 + 1 └── filters - └── (k_int:1, k_int_2:2) > (1, 2) [outer=(1,2), constraints=(/1/2: [/1/3 - ]; tight)] + └── (k_int:1, k_int_2:2) > (1, 2) [outer=(1,2), immutable, constraints=(/1/2: [/1/3 - ]; tight)] # Don't constrain when filter has multiple spans. opt diff --git a/pkg/sql/opt/xform/testdata/rules/groupby b/pkg/sql/opt/xform/testdata/rules/groupby index bb686abee482..1a90e794f4b8 100644 --- a/pkg/sql/opt/xform/testdata/rules/groupby +++ b/pkg/sql/opt/xform/testdata/rules/groupby @@ -766,6 +766,7 @@ SELECT v + 1, min(w), v FROM kuvw WHERE v = 5 AND w IS NOT NULL GROUP BY v project ├── columns: "?column?":6!null min:5!null v:3!null ├── cardinality: [0 - 1] + ├── immutable ├── key: () ├── fd: ()-->(3,5,6) ├── project @@ -782,7 +783,7 @@ project │ └── projections │ └── w:4 [as=min:5, outer=(4)] └── projections - └── v:3 + 1 [as="?column?":6, outer=(3)] + └── v:3 + 1 [as="?column?":6, outer=(3), immutable] # Add const_agg function, as well as max function. opt expect=ReplaceMaxWithLimit @@ -791,6 +792,7 @@ SELECT v + 1, max(w), v FROM kuvw WHERE v = 5 GROUP BY v project ├── columns: "?column?":6!null max:5 v:3!null ├── cardinality: [0 - 1] + ├── immutable ├── key: () ├── fd: ()-->(3,5,6) ├── project @@ -807,7 +809,7 @@ project │ └── projections │ └── w:4 [as=max:5, outer=(4)] └── projections - └── v:3 + 1 [as="?column?":6, outer=(3)] + └── v:3 + 1 [as="?column?":6, outer=(3), immutable] # Use multiple grouping columns with min function. opt expect=ReplaceMinWithLimit diff --git a/pkg/sql/opt/xform/testdata/rules/join b/pkg/sql/opt/xform/testdata/rules/join index 29828aafb4bc..ba78e5089a7d 100644 --- a/pkg/sql/opt/xform/testdata/rules/join +++ b/pkg/sql/opt/xform/testdata/rules/join @@ -2271,6 +2271,7 @@ SELECT b,a FROM t5 WHERE b @> '{"a":1}' ---- index-join t5 ├── columns: b:2 a:1!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2) └── scan t5@b_idx @@ -2283,6 +2284,7 @@ SELECT b,a FROM t5 WHERE b @> '{"a":[[{"b":{"c":[{"d":"e"}]}}]]}' ---- index-join t5 ├── columns: b:2 a:1!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2) └── scan t5@b_idx @@ -2298,6 +2300,7 @@ inner-join (lookup t5) ├── columns: b:2 a:1!null ├── key columns: [1] = [1] ├── lookup columns are key + ├── immutable ├── key: (1) ├── fd: (1)-->(2) ├── inner-join (zigzag t5@b_idx t5@b_idx) @@ -2307,7 +2310,7 @@ inner-join (lookup t5) │ ├── right fixed columns: [2] = ['{"c": 2}'] │ └── filters (true) └── filters - └── b:2 @> '{"a": 1, "c": 2}' [outer=(2)] + └── b:2 @> '{"a": 1, "c": 2}' [outer=(2), immutable] memo SELECT a FROM t5 WHERE b @> '{"a":1, "c":2}' @@ -2352,6 +2355,7 @@ inner-join (lookup t5) ├── columns: b:2 a:1!null ├── key columns: [1] = [1] ├── lookup columns are key + ├── immutable ├── key: (1) ├── fd: (1)-->(2) ├── inner-join (zigzag t5@b_idx t5@b_idx) @@ -2361,7 +2365,7 @@ inner-join (lookup t5) │ ├── right fixed columns: [2] = ['{"a": [{"d": 3}]}'] │ └── filters (true) └── filters - └── b:2 @> '{"a": [{"b": "c", "d": 3}, 5]}' [outer=(2)] + └── b:2 @> '{"a": [{"b": "c", "d": 3}, 5]}' [outer=(2), immutable] # Regression test for issue where zero-column expressions could exist multiple # times in the tree, causing collisions. @@ -2878,7 +2882,7 @@ select │ ├── volatile, side-effects │ └── key: (1) └── filters - └── b:2 @> '{"a": 1, "c": 2}' [outer=(2)] + └── b:2 @> '{"a": 1, "c": 2}' [outer=(2), immutable] # -------------------------------------------------- # AssociateJoin diff --git a/pkg/sql/opt/xform/testdata/rules/select b/pkg/sql/opt/xform/testdata/rules/select index 8156dc5440f5..49ab4cd8743d 100644 --- a/pkg/sql/opt/xform/testdata/rules/select +++ b/pkg/sql/opt/xform/testdata/rules/select @@ -494,6 +494,7 @@ SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k+u = 1 select ├── columns: k:1!null u:2 v:3!null j:4 ├── cardinality: [0 - 10] + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)-->(1,2,4) ├── index-join b @@ -508,7 +509,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(3), (3)-->(1) └── filters - └── (k:1 + u:2) = 1 [outer=(1,2)] + └── (k:1 + u:2) = 1 [outer=(1,2), immutable] memo SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k+u = 1 @@ -550,6 +551,7 @@ SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k+u = 1 AND k > 5 select ├── columns: k:1!null u:2 v:3!null j:4 ├── cardinality: [0 - 10] + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)-->(1,2,4) ├── index-join b @@ -571,7 +573,7 @@ select │ └── filters │ └── k:1 > 5 [outer=(1), constraints=(/1: [/6 - ]; tight)] └── filters - └── (k:1 + u:2) = 1 [outer=(1,2)] + └── (k:1 + u:2) = 1 [outer=(1,2), immutable] memo SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k+u = 1 AND k > 5 @@ -625,6 +627,7 @@ SELECT * FROM b WHERE (u, k, v) > (1, 2, 3) AND (u, k, v) < (8, 9, 10) ---- select ├── columns: k:1!null u:2!null v:3 j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)~~>(1,2,4) ├── index-join b @@ -637,8 +640,8 @@ select │ ├── key: (1) │ └── fd: (1)-->(2) └── filters - ├── (u:2, k:1, v:3) > (1, 2, 3) [outer=(1-3), constraints=(/2/1/3: [/1/2/4 - ]; tight)] - └── (u:2, k:1, v:3) < (8, 9, 10) [outer=(1-3), constraints=(/2/1/3: (/NULL - /8/9/9]; tight)] + ├── (u:2, k:1, v:3) > (1, 2, 3) [outer=(1-3), immutable, constraints=(/2/1/3: [/1/2/4 - ]; tight)] + └── (u:2, k:1, v:3) < (8, 9, 10) [outer=(1-3), immutable, constraints=(/2/1/3: (/NULL - /8/9/9]; tight)] memo SELECT * FROM b WHERE (u, k, v) > (1, 2, 3) AND (u, k, v) < (8, 9, 10) @@ -734,7 +737,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(3), (3)-->(1) └── filters - └── (k:1 + u:2) = 1 [outer=(1,2)] + └── (k:1 + u:2) = 1 [outer=(1,2), immutable] # -------------------------------------------------- # GenerateInvertedIndexScans @@ -746,9 +749,11 @@ SELECT k FROM b WHERE j @> '{"a": "b"}' ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── index-join b ├── columns: k:1!null j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(4) └── scan b@inv_idx @@ -789,11 +794,13 @@ SELECT k FROM b WHERE j @> '{"a": "b", "c": "d"}' ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── inner-join (lookup b) ├── columns: k:1!null j:4 ├── key columns: [1] = [1] ├── lookup columns are key + ├── immutable ├── key: (1) ├── fd: (1)-->(4) ├── inner-join (zigzag b@inv_idx b@inv_idx) @@ -803,7 +810,7 @@ project │ ├── right fixed columns: [4] = ['{"c": "d"}'] │ └── filters (true) └── filters - └── j:4 @> '{"a": "b", "c": "d"}' [outer=(4)] + └── j:4 @> '{"a": "b", "c": "d"}' [outer=(4), immutable] # Query requiring an index join with no remaining filter. opt @@ -811,10 +818,12 @@ SELECT u, k FROM b WHERE j @> '{"a": "b"}' ---- project ├── columns: u:2 k:1!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2) └── index-join b ├── columns: k:1!null u:2 j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2,4) └── scan b@inv_idx @@ -827,6 +836,7 @@ SELECT j, k FROM b WHERE j @> '{"a": "b"}' ---- index-join b ├── columns: j:4 k:1!null + ├── immutable ├── key: (1) ├── fd: (1)-->(4) └── scan b@inv_idx @@ -839,6 +849,7 @@ SELECT * FROM b WHERE j @> '{"a": "b"}' ---- index-join b ├── columns: k:1!null u:2 v:3 j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)~~>(1,2,4) └── scan b@inv_idx @@ -855,6 +866,7 @@ inner-join (lookup b) ├── columns: j:4 k:1!null ├── key columns: [1] = [1] ├── lookup columns are key + ├── immutable ├── key: (1) ├── fd: (1)-->(4) ├── inner-join (zigzag b@inv_idx b@inv_idx) @@ -864,7 +876,7 @@ inner-join (lookup b) │ ├── right fixed columns: [4] = ['{"c": "d"}'] │ └── filters (true) └── filters - └── j:4 @> '{"a": "b", "c": "d"}' [outer=(4)] + └── j:4 @> '{"a": "b", "c": "d"}' [outer=(4), immutable] opt SELECT * FROM b WHERE j @> '{"a": {"b": "c", "d": "e"}, "f": "g"}' @@ -873,6 +885,7 @@ inner-join (lookup b) ├── columns: k:1!null u:2 v:3 j:4 ├── key columns: [1] = [1] ├── lookup columns are key + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)~~>(1,2,4) ├── inner-join (zigzag b@inv_idx b@inv_idx) @@ -882,13 +895,14 @@ inner-join (lookup b) │ ├── right fixed columns: [4] = ['{"a": {"d": "e"}}'] │ └── filters (true) └── filters - └── j:4 @> '{"a": {"b": "c", "d": "e"}, "f": "g"}' [outer=(4)] + └── j:4 @> '{"a": {"b": "c", "d": "e"}, "f": "g"}' [outer=(4), immutable] opt SELECT * FROM b WHERE j @> '{}' ---- select ├── columns: k:1!null u:2 v:3 j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)~~>(1,2,4) ├── scan b @@ -896,13 +910,14 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-4), (3)~~>(1,2,4) └── filters - └── j:4 @> '{}' [outer=(4)] + └── j:4 @> '{}' [outer=(4), immutable] opt SELECT * FROM b WHERE j @> '[]' ---- select ├── columns: k:1!null u:2 v:3 j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)~~>(1,2,4) ├── scan b @@ -910,13 +925,14 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-4), (3)~~>(1,2,4) └── filters - └── j:4 @> '[]' [outer=(4)] + └── j:4 @> '[]' [outer=(4), immutable] opt SELECT * FROM b WHERE j @> '2' ---- index-join b ├── columns: k:1!null u:2 v:3 j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)~~>(1,2,4) └── scan b@inv_idx @@ -931,6 +947,7 @@ SELECT * FROM b WHERE j @> '[{}]' ---- select ├── columns: k:1!null u:2 v:3 j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)~~>(1,2,4) ├── scan b @@ -938,13 +955,14 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-4), (3)~~>(1,2,4) └── filters - └── j:4 @> '[{}]' [outer=(4)] + └── j:4 @> '[{}]' [outer=(4), immutable] opt SELECT * FROM b WHERE j @> '{"a": {}}' ---- select ├── columns: k:1!null u:2 v:3 j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)~~>(1,2,4) ├── scan b @@ -952,13 +970,14 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-4), (3)~~>(1,2,4) └── filters - └── j:4 @> '{"a": {}}' [outer=(4)] + └── j:4 @> '{"a": {}}' [outer=(4), immutable] opt SELECT * FROM b WHERE j @> '{"a": []}' ---- select ├── columns: k:1!null u:2 v:3 j:4 + ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3)~~>(1,2,4) ├── scan b @@ -966,7 +985,7 @@ select │ ├── key: (1) │ └── fd: (1)-->(2-4), (3)~~>(1,2,4) └── filters - └── j:4 @> '{"a": []}' [outer=(4)] + └── j:4 @> '{"a": []}' [outer=(4), immutable] # GenerateInvertedIndexScans propagates row-level locking information. opt @@ -994,9 +1013,11 @@ SELECT k FROM c WHERE a @> ARRAY[1] ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── index-join c ├── columns: k:1!null a:2 + ├── immutable ├── key: (1) ├── fd: (1)-->(2) └── scan c@inv_idx @@ -1009,11 +1030,13 @@ SELECT k FROM c WHERE a @> ARRAY[1,3,1,5] ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── inner-join (lookup c) ├── columns: k:1!null a:2 ├── key columns: [1] = [1] ├── lookup columns are key + ├── immutable ├── key: (1) ├── fd: (1)-->(2) ├── inner-join (zigzag c@inv_idx c@inv_idx) @@ -1023,16 +1046,18 @@ project │ ├── right fixed columns: [2] = [ARRAY[3]] │ └── filters (true) └── filters - └── a:2 @> ARRAY[1,3,1,5] [outer=(2)] + └── a:2 @> ARRAY[1,3,1,5] [outer=(2), immutable] opt SELECT k FROM c WHERE a @> ARRAY[]::INT[] ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── select ├── columns: k:1!null a:2 + ├── immutable ├── key: (1) ├── fd: (1)-->(2) ├── scan c @@ -1040,7 +1065,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) └── filters - └── a:2 @> ARRAY[] [outer=(2)] + └── a:2 @> ARRAY[] [outer=(2), immutable] opt SELECT k FROM c WHERE a IS NULL @@ -1328,16 +1353,19 @@ SELECT k FROM b WHERE k = 1 OR j @> '{"foo": "bar"}' ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── distinct-on ├── columns: k:1!null j:4 ├── grouping columns: k:1!null + ├── immutable ├── key: (1) ├── fd: (1)-->(4) ├── union-all │ ├── columns: k:1!null j:4 │ ├── left columns: k:1!null j:4 │ ├── right columns: k:5 j:8 + │ ├── immutable │ ├── scan b │ │ ├── columns: k:1!null j:4 │ │ ├── constraint: /1: [/1 - /1] @@ -1346,6 +1374,7 @@ project │ │ └── fd: ()-->(1,4) │ └── index-join b │ ├── columns: k:5!null j:8 + │ ├── immutable │ ├── key: (5) │ ├── fd: (5)-->(8) │ └── scan b@inv_idx @@ -1362,16 +1391,19 @@ SELECT k FROM c WHERE k = 1 OR a @> ARRAY[2] ---- project ├── columns: k:1!null + ├── immutable ├── key: (1) └── distinct-on ├── columns: k:1!null a:2 ├── grouping columns: k:1!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2) ├── union-all │ ├── columns: k:1!null a:2 │ ├── left columns: k:1!null a:2 │ ├── right columns: k:4 a:5 + │ ├── immutable │ ├── scan c │ │ ├── columns: k:1!null a:2 │ │ ├── constraint: /1: [/1 - /1] @@ -1380,6 +1412,7 @@ project │ │ └── fd: ()-->(1,2) │ └── index-join c │ ├── columns: k:4!null a:5 + │ ├── immutable │ ├── key: (4) │ ├── fd: (4)-->(5) │ └── scan c@inv_idx @@ -2299,15 +2332,18 @@ SELECT u, j FROM b WHERE u = 1 OR j @> '{"foo": "bar"}' ---- project ├── columns: u:2 j:4 + ├── immutable └── distinct-on ├── columns: k:1!null u:2 j:4 ├── grouping columns: k:1!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2,4) ├── union-all │ ├── columns: k:1!null u:2 j:4 │ ├── left columns: k:1!null u:2 j:4 │ ├── right columns: k:5 u:6 j:8 + │ ├── immutable │ ├── index-join b │ │ ├── columns: k:1!null u:2!null j:4 │ │ ├── key: (1) @@ -2319,6 +2355,7 @@ project │ │ └── fd: ()-->(2) │ └── index-join b │ ├── columns: k:5!null u:6 j:8 + │ ├── immutable │ ├── key: (5) │ ├── fd: (5)-->(6,8) │ └── scan b@inv_idx @@ -2337,15 +2374,18 @@ SELECT u, a FROM c WHERE u = 1 OR a @> ARRAY[2] ---- project ├── columns: u:3 a:2 + ├── immutable └── distinct-on ├── columns: k:1!null a:2 u:3 ├── grouping columns: k:1!null + ├── immutable ├── key: (1) ├── fd: (1)-->(2,3) ├── union-all │ ├── columns: k:1!null a:2 u:3 │ ├── left columns: k:1!null a:2 u:3 │ ├── right columns: k:4 a:5 u:6 + │ ├── immutable │ ├── index-join c │ │ ├── columns: k:1!null a:2 u:3!null │ │ ├── key: (1) @@ -2357,6 +2397,7 @@ project │ │ └── fd: ()-->(3) │ └── index-join c │ ├── columns: k:4!null a:5 u:6 + │ ├── immutable │ ├── key: (4) │ ├── fd: (4)-->(5,6) │ └── scan c@inv_idx From a1fecbc7b11457a7665f20d1181b990ebe1d8afa Mon Sep 17 00:00:00 2001 From: arulajmani Date: Mon, 8 Jun 2020 13:29:20 -0400 Subject: [PATCH 20/46] cmd/generate-binary: move some decimal encoding tests to auto-gen script `sql/pgwire/testdata/encodings.json` is autogenerated using `cmd/generate-binary/main.go`. Previously, a few tests were missing from this file, which would have been lost the next time new tests were added and `encodings.json` was autogenerated. This PR fixes that by moving the tests to the auto-gen script. Release note (none) --- pkg/cmd/generate-binary/main.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pkg/cmd/generate-binary/main.go b/pkg/cmd/generate-binary/main.go index 0f19b64d4e34..2b40ab3f0998 100644 --- a/pkg/cmd/generate-binary/main.go +++ b/pkg/cmd/generate-binary/main.go @@ -17,7 +17,7 @@ // The target postgres server must accept plaintext (non-ssl) connections from // the postgres:postgres account. A suitable server can be started with: // -// `docker run -p 127.0.0.1:5432:5432 postgres` +// `docker run -p 127.0.0.1:5432:5432 postgres:11` // // The output of this file generates pkg/sql/pgwire/testdata/encodings.json. package main @@ -197,6 +197,9 @@ var inputs = map[string][]string{ "2.2289971159100284", "3409589268520956934250.234098732045120934701239846", "42", + "42.0", + "420000", + "420000.0", }, "'%s'::float8": { From 759e144a5cd2dd046f0b74cb9659e3de0f0aa3bb Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Thu, 4 Jun 2020 00:00:01 -0700 Subject: [PATCH 21/46] opt: add rule to fold limits Previously, there was no rule to fold a Limit on top of a Limit when the outer limit value is smaller than the inner limit value. This patch adds a rule to fold two Limits together when the outer Limit has a smaller limit value than the inner Limit and the inner ordering implies the outer ordering. Release note (sql change): The optimizer can now fold two Limit operators together. --- pkg/sql/opt/norm/general_funcs.go | 15 +- pkg/sql/opt/norm/rules/limit.opt | 19 +++ pkg/sql/opt/norm/testdata/rules/limit | 189 ++++++++++++++++++++++---- 3 files changed, 197 insertions(+), 26 deletions(-) diff --git a/pkg/sql/opt/norm/general_funcs.go b/pkg/sql/opt/norm/general_funcs.go index b709940c9609..ffd0ef69ec03 100644 --- a/pkg/sql/opt/norm/general_funcs.go +++ b/pkg/sql/opt/norm/general_funcs.go @@ -570,12 +570,15 @@ func (c *CustomFuncs) OrdinalityOrdering(private *memo.OrdinalityPrivate) physic } // IsSameOrdering evaluates whether the two orderings are equal. -func (c *CustomFuncs) IsSameOrdering( - first physical.OrderingChoice, other physical.OrderingChoice, -) bool { +func (c *CustomFuncs) IsSameOrdering(first, other physical.OrderingChoice) bool { return first.Equals(&other) } +// OrderingImplies returns true if the first OrderingChoice implies the second. +func (c *CustomFuncs) OrderingImplies(first, second physical.OrderingChoice) bool { + return first.Implies(&second) +} + // ----------------------------------------------------------------------- // // Filter functions @@ -950,3 +953,9 @@ func (c *CustomFuncs) CanAddConstInts(first tree.Datum, second tree.Datum) bool func (c *CustomFuncs) IntConst(d *tree.DInt) opt.ScalarExpr { return c.f.ConstructConst(d, types.Int) } + +// IsGreaterThan returns true if the first datum compares as greater than the +// second. +func (c *CustomFuncs) IsGreaterThan(first, second tree.Datum) bool { + return first.Compare(c.f.evalCtx, second) == 1 +} diff --git a/pkg/sql/opt/norm/rules/limit.opt b/pkg/sql/opt/norm/rules/limit.opt index 7a09989239f0..73ec3f86bc00 100644 --- a/pkg/sql/opt/norm/rules/limit.opt +++ b/pkg/sql/opt/norm/rules/limit.opt @@ -202,3 +202,22 @@ $input $limitExpr $ordering ) + +# FoldLimits replaces a Limit on top of a Limit with a single Limit operator +# when the outer limit value is smaller than or equal to the inner limit value +# and the inner ordering implies the outer ordering. Note: the case when the +# outer limit value is larger than the inner is handled by EliminateLimit. +[FoldLimits, Normalize] +(Limit + (Limit + $innerInput:* + $innerLimitExpr:(Const $innerLimit:*) + $innerOrdering:* + ) + $outerLimitExpr:(Const $outerLimit:*) & + ^(IsGreaterThan $outerLimit $innerLimit) + $outerOrdering:* & + (OrderingImplies $innerOrdering $outerOrdering) +) +=> +(Limit $innerInput $outerLimitExpr $innerOrdering) diff --git a/pkg/sql/opt/norm/testdata/rules/limit b/pkg/sql/opt/norm/testdata/rules/limit index 13976acf9325..6596ffb0f281 100644 --- a/pkg/sql/opt/norm/testdata/rules/limit +++ b/pkg/sql/opt/norm/testdata/rules/limit @@ -51,8 +51,9 @@ limit │ └── limit hint: 100.00 └── 100 -# Don't eliminate the outer limit if it's less than the inner. -norm +# Don't eliminate the outer limit if it's less than the inner (the limit is +# instead removed by FoldLimits). +norm expect-not=EliminateLimit SELECT * FROM (SELECT * FROM a LIMIT 100) LIMIT 99 ---- limit @@ -60,22 +61,15 @@ limit ├── cardinality: [0 - 99] ├── key: (1) ├── fd: (1)-->(2-5) - ├── limit + ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ ├── cardinality: [0 - 100] │ ├── key: (1) │ ├── fd: (1)-->(2-5) - │ ├── limit hint: 99.00 - │ ├── scan a - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-5) - │ │ └── limit hint: 100.00 - │ └── 100 + │ └── limit hint: 99.00 └── 99 # High limits (> max uint32), can't eliminate in this case. -norm +norm expect-not=EliminateLimit SELECT * FROM (SELECT * FROM a LIMIT 5000000000) LIMIT 5100000000 ---- limit @@ -95,8 +89,9 @@ limit │ └── 5000000000 └── 5100000000 -# Don't eliminate in case of negative limit. -norm +# Don't eliminate in case of negative limit (the limit is instead removed by +# FoldLimits). +norm expect-not=EliminateLimit SELECT * FROM (SELECT * FROM a LIMIT 0) LIMIT -1 ---- limit @@ -1109,18 +1104,11 @@ limit │ │ ├── cardinality: [0 - 10] │ │ ├── key: (1) │ │ ├── fd: (1)-->(2) - │ │ ├── limit + │ │ ├── scan ab │ │ │ ├── columns: a:1!null b:2 - │ │ │ ├── cardinality: [0 - 20] │ │ │ ├── key: (1) │ │ │ ├── fd: (1)-->(2) - │ │ │ ├── limit hint: 10.00 - │ │ │ ├── scan ab - │ │ │ │ ├── columns: a:1!null b:2 - │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2) - │ │ │ │ └── limit hint: 20.00 - │ │ │ └── 20 + │ │ │ └── limit hint: 10.00 │ │ └── 10 │ ├── scan uv │ │ ├── columns: u:3!null v:4 @@ -1245,3 +1233,158 @@ limit │ └── filters │ └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] └── 10 + +# ---------- +# FoldLimits +# ---------- + +# Basic case with no orderings. +norm expect=FoldLimits +SELECT * FROM (SELECT * FROM ab LIMIT 10) LIMIT 5 +---- +limit + ├── columns: a:1!null b:2 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2) + ├── scan ab + │ ├── columns: a:1!null b:2 + │ ├── key: (1) + │ ├── fd: (1)-->(2) + │ └── limit hint: 5.00 + └── 5 + +# Case where the inner limit has an ordering and the outer limit is unordered. +norm expect=FoldLimits +SELECT * FROM (SELECT * FROM ab ORDER BY a LIMIT 10) LIMIT 5 +---- +limit + ├── columns: a:1!null b:2 + ├── internal-ordering: +1 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2) + ├── scan ab + │ ├── columns: a:1!null b:2 + │ ├── key: (1) + │ ├── fd: (1)-->(2) + │ ├── ordering: +1 + │ └── limit hint: 5.00 + └── 5 + +# Case where the inner limit ordering implies the outer ordering. +norm expect=FoldLimits +SELECT * FROM (SELECT * FROM a ORDER BY i, f LIMIT 10) ORDER BY i LIMIT 5 +---- +limit + ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── internal-ordering: +2,+3 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2-5) + ├── ordering: +2 + ├── sort + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── key: (1) + │ ├── fd: (1)-->(2-5) + │ ├── ordering: +2,+3 + │ ├── limit hint: 5.00 + │ └── scan a + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── key: (1) + │ └── fd: (1)-->(2-5) + └── 5 + +# No-op case where the outer limit is larger than the inner limit. (The limit is +# instead removed by EliminateLimit). +norm expect-not=FoldLimits +SELECT * FROM (SELECT * FROM ab LIMIT 5) LIMIT 10 +---- +limit + ├── columns: a:1!null b:2 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2) + ├── scan ab + │ ├── columns: a:1!null b:2 + │ ├── key: (1) + │ ├── fd: (1)-->(2) + │ └── limit hint: 5.00 + └── 5 + +# No-op case where the inner limit ordering does not imply the outer limit +# ordering. +norm expect-not=FoldLimits +SELECT * FROM (SELECT * FROM ab ORDER BY b LIMIT 10) ORDER BY a LIMIT 5 +---- +limit + ├── columns: a:1!null b:2 + ├── internal-ordering: +1 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2) + ├── ordering: +1 + ├── sort + │ ├── columns: a:1!null b:2 + │ ├── cardinality: [0 - 10] + │ ├── key: (1) + │ ├── fd: (1)-->(2) + │ ├── ordering: +1 + │ ├── limit hint: 5.00 + │ └── limit + │ ├── columns: a:1!null b:2 + │ ├── internal-ordering: +2 + │ ├── cardinality: [0 - 10] + │ ├── key: (1) + │ ├── fd: (1)-->(2) + │ ├── sort + │ │ ├── columns: a:1!null b:2 + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2) + │ │ ├── ordering: +2 + │ │ ├── limit hint: 10.00 + │ │ └── scan ab + │ │ ├── columns: a:1!null b:2 + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2) + │ └── 10 + └── 5 + +# No-op case where the outer ordering implies the inner, but the inner doesn't +# imply the outer. +norm expect-not=FoldLimits +SELECT * FROM (SELECT * FROM a ORDER BY i LIMIT 10) ORDER BY i, f LIMIT 5 +---- +limit + ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── internal-ordering: +2,+3 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2-5) + ├── ordering: +2,+3 + ├── sort (segmented) + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── cardinality: [0 - 10] + │ ├── key: (1) + │ ├── fd: (1)-->(2-5) + │ ├── ordering: +2,+3 + │ ├── limit hint: 5.00 + │ └── limit + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── internal-ordering: +2 + │ ├── cardinality: [0 - 10] + │ ├── key: (1) + │ ├── fd: (1)-->(2-5) + │ ├── ordering: +2 + │ ├── sort + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2-5) + │ │ ├── ordering: +2 + │ │ ├── limit hint: 10.00 + │ │ └── scan a + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5) + │ └── 10 + └── 5 From 12b774baffb6e1d04c1c7c8954faeef089d84c32 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Mon, 8 Jun 2020 15:12:17 -0400 Subject: [PATCH 22/46] vendor: bump Pebble to feb930 feb930 db: close tableCache on open error 660b76 internal/record: bump LogWriter pending queue size a9b799 db: remove table loading goroutine d18729 db: add a per-tableCacheShard table closing goroutine 9687c6 internal/manifest: add Level type Release note: None --- Gopkg.lock | 5 +++-- vendor | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 20132c3f683d..139af6ca6fbc 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -458,7 +458,7 @@ [[projects]] branch = "master" - digest = "1:01f450f2b42cdd2e3eac4a0ff01e0eccc287b5e6c590dbb592b3a320cd29cce0" + digest = "1:085203aa725af444d832f6f0e38511e7a2a7bf7d630f618257e467fb043ea109" name = "github.com/cockroachdb/pebble" packages = [ ".", @@ -484,7 +484,7 @@ "vfs", ] pruneopts = "UT" - revision = "4887c526300055e1c30635c53fd16b3fe9d9e132" + revision = "feb93032c41f991845506cd423f1771618edf2b0" [[projects]] branch = "master" @@ -2146,6 +2146,7 @@ "github.com/apache/arrow/go/arrow/memory", "github.com/armon/circbuf", "github.com/aws/aws-sdk-go/aws", + "github.com/aws/aws-sdk-go/aws/awserr", "github.com/aws/aws-sdk-go/aws/credentials", "github.com/aws/aws-sdk-go/aws/session", "github.com/aws/aws-sdk-go/service/s3", diff --git a/vendor b/vendor index 2096d93a4d7f..a6597287a886 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit 2096d93a4d7fed85e90e897e487366bff4813d00 +Subproject commit a6597287a88652dd0b769822ea047c0b58462cdf From 5138754c056392413a7228abf3759355142630d0 Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Mon, 8 Jun 2020 15:20:59 -0400 Subject: [PATCH 23/46] importccl: add basic tests for CSV export of enum types Work for #49971. This PR adds some basic tests for CSV export of enum types. Release note: None --- pkg/ccl/importccl/exportcsv_test.go | 44 +++++++++++++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/pkg/ccl/importccl/exportcsv_test.go b/pkg/ccl/importccl/exportcsv_test.go index 8ebcc9033ed7..bd38c5966ec3 100644 --- a/pkg/ccl/importccl/exportcsv_test.go +++ b/pkg/ccl/importccl/exportcsv_test.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/workload/bank" "github.com/cockroachdb/cockroach/pkg/workload/workloadsql" "github.com/gogo/protobuf/proto" + "github.com/stretchr/testify/require" ) func setupExportableBank(t *testing.T, nodes, rows int) (*sqlutils.SQLRunner, string, func()) { @@ -203,6 +204,49 @@ func TestExportOrder(t *testing.T) { } } +func TestExportUserDefinedTypes(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + baseDir, cleanup := testutils.TempDir(t) + defer cleanup() + tc := testcluster.StartTestCluster( + t, 1, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ExternalIODir: baseDir}}) + defer tc.Stopper().Stop(ctx) + conn := tc.Conns[0] + sqlDB := sqlutils.MakeSQLRunner(conn) + // Set up some initial state for the tests. + sqlDB.Exec(t, ` +SET experimental_enable_enums = true; +CREATE TYPE greeting AS ENUM ('hello', 'hi'); +CREATE TABLE greeting_table (x greeting, y greeting); +INSERT INTO greeting_table VALUES ('hello', 'hello'), ('hi', 'hi'); +`) + tests := []struct { + stmt string + expected string + }{ + { + stmt: "EXPORT INTO CSV 'nodelocal://0/test/' FROM (SELECT 'hello':::greeting, 'hi':::greeting)", + expected: "hello,hi\n", + }, + { + stmt: "EXPORT INTO CSV 'nodelocal://0/test/' FROM TABLE greeting_table", + expected: "hello,hello\nhi,hi\n", + }, + { + stmt: "EXPORT INTO CSV 'nodelocal://0/test/' FROM (SELECT x, y, enum_first(x) FROM greeting_table)", + expected: "hello,hello,hello\nhi,hi,hello\n", + }, + } + for _, test := range tests { + sqlDB.Exec(t, test.stmt) + // Read the dumped file. + contents, err := ioutil.ReadFile(filepath.Join(baseDir, "test", "n1.0.csv")) + require.NoError(t, err) + require.Equal(t, test.expected, string(contents)) + } +} + func TestExportOrderCompressed(t *testing.T) { defer leaktest.AfterTest(t)() dir, cleanupDir := testutils.TempDir(t) From 7137028cca89e777fc3a2a2ab2c90364e7590194 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Mon, 8 Jun 2020 12:36:48 -0700 Subject: [PATCH 24/46] build: add build that simply compiles CRDB on supported platforms Abstract away the process of building from `./pkg/cmd/publish-*-artifacts`, and use this in `./pkg/cmd/compile-builds`. This is intended to become a CI job for TeamCity. Release note: None --- Makefile | 1 + build/teamcity-compile-builds.sh | 9 ++ pkg/cmd/compile-builds/main.go | 36 +++++ pkg/cmd/publish-artifacts/main.go | 96 +++--------- pkg/cmd/publish-provisional-artifacts/main.go | 120 ++++----------- .../main_test.go | 3 +- pkg/release/release.go | 144 ++++++++++++++++++ pkg/testutils/lint/lint_test.go | 2 +- 8 files changed, 244 insertions(+), 167 deletions(-) create mode 100644 build/teamcity-compile-builds.sh create mode 100644 pkg/cmd/compile-builds/main.go create mode 100644 pkg/release/release.go diff --git a/Makefile b/Makefile index e992478db232..b272f01d1de4 100644 --- a/Makefile +++ b/Makefile @@ -1671,6 +1671,7 @@ bins = \ bin/benchmark \ bin/cockroach-oss \ bin/cockroach-short \ + bin/compile-builds \ bin/docgen \ bin/execgen \ bin/fuzz \ diff --git a/build/teamcity-compile-builds.sh b/build/teamcity-compile-builds.sh new file mode 100644 index 000000000000..4c66ac36197e --- /dev/null +++ b/build/teamcity-compile-builds.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +set -euxo pipefail + +export BUILDER_HIDE_GOPATH_SRC=1 + +build/builder.sh go install ./pkg/cmd/compile-builds +build/builder.sh env \ + compile-builds diff --git a/pkg/cmd/compile-builds/main.go b/pkg/cmd/compile-builds/main.go new file mode 100644 index 000000000000..39954478ce4c --- /dev/null +++ b/pkg/cmd/compile-builds/main.go @@ -0,0 +1,36 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// compile-builds attempts to compile all CRDB builds we support. + +package main + +import ( + "go/build" + "log" + + "github.com/cockroachdb/cockroach/pkg/release" +) + +func main() { + pkg, err := build.Import("github.com/cockroachdb/cockroach", "", build.FindOnly) + if err != nil { + log.Fatalf("unable to locate CRDB directory: %s", err) + } + + for _, target := range release.SupportedTargets { + if err := release.MakeRelease( + target, + pkg.Dir, + ); err != nil { + log.Fatal(err) + } + } +} diff --git a/pkg/cmd/publish-artifacts/main.go b/pkg/cmd/publish-artifacts/main.go index c87d976fc046..a1e12909d297 100644 --- a/pkg/cmd/publish-artifacts/main.go +++ b/pkg/cmd/publish-artifacts/main.go @@ -13,7 +13,6 @@ package main import ( "archive/tar" "archive/zip" - "bufio" "bytes" "compress/gzip" "flag" @@ -31,6 +30,7 @@ import ( "github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/aws/session" "github.com/aws/aws-sdk-go/service/s3" + "github.com/cockroachdb/cockroach/pkg/release" "github.com/cockroachdb/cockroach/pkg/util/version" "github.com/kr/pretty" ) @@ -56,20 +56,6 @@ var testableS3 = func() (s3putter, error) { return s3.New(sess), nil } -var libsRe = func() *regexp.Regexp { - libs := strings.Join([]string{ - regexp.QuoteMeta("linux-vdso.so."), - regexp.QuoteMeta("librt.so."), - regexp.QuoteMeta("libpthread.so."), - regexp.QuoteMeta("libdl.so."), - regexp.QuoteMeta("libm.so."), - regexp.QuoteMeta("libc.so."), - regexp.QuoteMeta("libresolv.so."), - strings.Replace(regexp.QuoteMeta("ld-linux-ARCH.so."), "ARCH", ".*", -1), - }, "|") - return regexp.MustCompile(libs) -}() - var osVersionRe = regexp.MustCompile(`\d+(\.\d+)*-`) var isRelease = flag.Bool("release", false, "build in release mode instead of bleeding-edge mode") @@ -157,17 +143,7 @@ func main() { }) } - for _, target := range []struct { - buildType string - suffix string - }{ - // TODO(tamird): consider shifting this information into the builder - // image; it's conceivable that we'll want to target multiple versions - // of a given triple. - {buildType: "darwin", suffix: ".darwin-10.9-amd64"}, - {buildType: "linux-gnu", suffix: ".linux-2.6.32-gnu-amd64"}, - {buildType: "windows", suffix: ".windows-6.2-amd64.exe"}, - } { + for _, target := range release.SupportedTargets { for i, extraArgs := range []struct { goflags string suffix string @@ -188,9 +164,9 @@ func main() { o.VersionStr = versionStr o.BucketName = bucketName o.Branch = branch - o.BuildType = target.buildType + o.BuildType = target.BuildType o.GoFlags = extraArgs.goflags - o.Suffix = extraArgs.suffix + target.suffix + o.Suffix = extraArgs.suffix + target.Suffix o.Tags = extraArgs.tags log.Printf("building %s", pretty.Sprint(o)) @@ -256,58 +232,30 @@ func buildArchive(svc s3putter, o opts) { } func buildOneCockroach(svc s3putter, o opts) { + log.Printf("building cockroach %s", pretty.Sprint(o)) defer func() { log.Printf("done building cockroach: %s", pretty.Sprint(o)) }() - { - args := []string{o.BuildType} - args = append(args, fmt.Sprintf("%s=%s", "GOFLAGS", o.GoFlags)) - args = append(args, fmt.Sprintf("%s=%s", "SUFFIX", o.Suffix)) - args = append(args, fmt.Sprintf("%s=%s", "TAGS", o.Tags)) - args = append(args, fmt.Sprintf("%s=%s", "BUILDCHANNEL", "official-binary")) - if *isRelease { - args = append(args, fmt.Sprintf("%s=%s", "BUILD_TAGGED_RELEASE", "true")) - } - cmd := exec.Command("mkrelease", args...) - cmd.Dir = o.PkgDir - cmd.Stdout = os.Stdout - cmd.Stderr = os.Stderr - log.Printf("%s %s", cmd.Env, cmd.Args) - if err := cmd.Run(); err != nil { - log.Fatalf("%s: %s", cmd.Args, err) - } + opts := []release.MakeReleaseOption{ + release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "GOFLAGS", o.GoFlags)), + release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "SUFFIX", o.Suffix)), + release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "TAGS", o.Tags)), + release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "BUILDCHANNEL", "official-binary")), + } + if *isRelease { + opts = append(opts, release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "BUILD_TAGGED_RELEASE", "true"))) } - if strings.Contains(o.BuildType, "linux") { - binaryName := "./cockroach" + o.Suffix - - cmd := exec.Command(binaryName, "version") - cmd.Dir = o.PkgDir - cmd.Env = append(cmd.Env, "MALLOC_CONF=prof:true") - cmd.Stdout = os.Stdout - cmd.Stderr = os.Stderr - log.Printf("%s %s", cmd.Env, cmd.Args) - if err := cmd.Run(); err != nil { - log.Fatalf("%s %s: %s", cmd.Env, cmd.Args, err) - } - - cmd = exec.Command("ldd", binaryName) - cmd.Dir = o.PkgDir - log.Printf("%s %s", cmd.Env, cmd.Args) - out, err := cmd.Output() - if err != nil { - log.Fatalf("%s: out=%q err=%s", cmd.Args, out, err) - } - scanner := bufio.NewScanner(bytes.NewReader(out)) - for scanner.Scan() { - if line := scanner.Text(); !libsRe.MatchString(line) { - log.Fatalf("%s is not properly statically linked:\n%s", binaryName, out) - } - } - if err := scanner.Err(); err != nil { - log.Fatal(err) - } + if err := release.MakeRelease( + release.SupportedTarget{ + BuildType: o.BuildType, + Suffix: o.Suffix, + }, + o.PkgDir, + opts..., + ); err != nil { + log.Fatal(err) } o.Base = "cockroach" + o.Suffix diff --git a/pkg/cmd/publish-provisional-artifacts/main.go b/pkg/cmd/publish-provisional-artifacts/main.go index 586dce191840..b92b62783b48 100644 --- a/pkg/cmd/publish-provisional-artifacts/main.go +++ b/pkg/cmd/publish-provisional-artifacts/main.go @@ -13,7 +13,6 @@ package main import ( "archive/tar" "archive/zip" - "bufio" "bytes" "compress/gzip" "flag" @@ -31,8 +30,8 @@ import ( "github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/aws/session" "github.com/aws/aws-sdk-go/service/s3" + "github.com/cockroachdb/cockroach/pkg/release" "github.com/cockroachdb/cockroach/pkg/util/version" - "github.com/cockroachdb/errors" "github.com/kr/pretty" ) @@ -49,8 +48,6 @@ type s3I interface { PutObject(*s3.PutObjectInput) (*s3.PutObjectOutput, error) } -type execRunner func(*exec.Cmd) ([]byte, error) - func makeS3() (s3I, error) { sess, err := session.NewSession(&aws.Config{ Region: aws.String("us-east-1"), @@ -61,20 +58,6 @@ func makeS3() (s3I, error) { return s3.New(sess), nil } -var libsRe = func() *regexp.Regexp { - libs := strings.Join([]string{ - regexp.QuoteMeta("linux-vdso.so."), - regexp.QuoteMeta("librt.so."), - regexp.QuoteMeta("libpthread.so."), - regexp.QuoteMeta("libdl.so."), - regexp.QuoteMeta("libm.so."), - regexp.QuoteMeta("libc.so."), - regexp.QuoteMeta("libresolv.so."), - strings.Replace(regexp.QuoteMeta("ld-linux-ARCH.so."), "ARCH", ".*", -1), - }, "|") - return regexp.MustCompile(libs) -}() - var osVersionRe = regexp.MustCompile(`\d+(\.\d+)*-`) var isReleaseF = flag.Bool("release", false, "build in release mode instead of bleeding-edge mode") @@ -106,16 +89,7 @@ func main() { if err != nil { log.Fatalf("Creating AWS S3 session: %s", err) } - execFn := func(c *exec.Cmd) ([]byte, error) { - if c.Stdout != nil { - return nil, errors.New("exec: Stdout already set") - } - var stdout bytes.Buffer - c.Stdout = io.MultiWriter(&stdout, os.Stdout) - err := c.Run() - return stdout.Bytes(), err - } - + execFn := release.DefaultExecFn branch, ok := os.LookupEnv(teamcityBuildBranchKey) if !ok { log.Fatalf("VCS branch environment variable %s is not set", teamcityBuildBranchKey) @@ -149,7 +123,7 @@ type runFlags struct { pkgDir string } -func run(svc s3I, execFn execRunner, flags runFlags) { +func run(svc s3I, execFn release.ExecFn, flags runFlags) { // TODO(dan): non-release builds currently aren't broken into the two // phases. Instead, the provisional phase does them both. if !flags.isRelease { @@ -198,17 +172,7 @@ func run(svc s3I, execFn execRunner, flags runFlags) { log.Printf("Using S3 bucket: %s", bucketName) var cockroachBuildOpts []opts - for _, target := range []struct { - buildType string - suffix string - }{ - // TODO(tamird): consider shifting this information into the builder - // image; it's conceivable that we'll want to target multiple versions - // of a given triple. - {buildType: "darwin", suffix: ".darwin-10.9-amd64"}, - {buildType: "linux-gnu", suffix: ".linux-2.6.32-gnu-amd64"}, - {buildType: "windows", suffix: ".windows-6.2-amd64.exe"}, - } { + for _, target := range release.SupportedTargets { for i, extraArgs := range []struct { goflags string suffix string @@ -229,9 +193,9 @@ func run(svc s3I, execFn execRunner, flags runFlags) { o.Branch = flags.branch o.VersionStr = versionStr o.BucketName = bucketName - o.BuildType = target.buildType + o.BuildType = target.BuildType o.GoFlags = extraArgs.goflags - o.Suffix = extraArgs.suffix + target.suffix + o.Suffix = extraArgs.suffix + target.Suffix o.Tags = extraArgs.tags o.Base = "cockroach" + o.Suffix @@ -252,7 +216,7 @@ func run(svc s3I, execFn execRunner, flags runFlags) { if flags.doProvisional { for _, o := range cockroachBuildOpts { - buildCockroach(svc, execFn, flags, o) + buildCockroach(execFn, flags, o) absolutePath := filepath.Join(o.PkgDir, o.Base) binary, err := os.Open(absolutePath) @@ -285,7 +249,7 @@ func run(svc s3I, execFn execRunner, flags runFlags) { } } -func buildAndPutArchive(svc s3I, execFn execRunner, o opts) { +func buildAndPutArchive(svc s3I, execFn release.ExecFn, o opts) { log.Printf("building archive %s", pretty.Sprint(o)) defer func() { log.Printf("done building archive: %s", pretty.Sprint(o)) @@ -325,59 +289,33 @@ func buildAndPutArchive(svc s3I, execFn execRunner, o opts) { } } -func buildCockroach(svc s3I, execFn execRunner, flags runFlags, o opts) { +func buildCockroach(execFn release.ExecFn, flags runFlags, o opts) { log.Printf("building cockroach %s", pretty.Sprint(o)) defer func() { log.Printf("done building cockroach: %s", pretty.Sprint(o)) }() - { - args := []string{o.BuildType} - args = append(args, fmt.Sprintf("%s=%s", "GOFLAGS", o.GoFlags)) - args = append(args, fmt.Sprintf("%s=%s", "SUFFIX", o.Suffix)) - args = append(args, fmt.Sprintf("%s=%s", "TAGS", o.Tags)) - args = append(args, fmt.Sprintf("%s=%s", "BUILDCHANNEL", "official-binary")) - if flags.isRelease { - args = append(args, fmt.Sprintf("%s=%s", "BUILDINFO_TAG", o.VersionStr)) - args = append(args, fmt.Sprintf("%s=%s", "BUILD_TAGGED_RELEASE", "true")) - } - cmd := exec.Command("mkrelease", args...) - cmd.Dir = o.PkgDir - cmd.Stderr = os.Stderr - log.Printf("%s %s", cmd.Env, cmd.Args) - if out, err := execFn(cmd); err != nil { - log.Fatalf("%s %s: %s\n\n%s", cmd.Env, cmd.Args, err, out) - } + opts := []release.MakeReleaseOption{ + release.WithMakeReleaseOptionExecFn(execFn), + release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "GOFLAGS", o.GoFlags)), + release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "SUFFIX", o.Suffix)), + release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "TAGS", o.Tags)), + release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "BUILDCHANNEL", "official-binary")), } - - if strings.Contains(o.BuildType, "linux") { - binaryName := "./cockroach" + o.Suffix - - cmd := exec.Command(binaryName, "version") - cmd.Dir = o.PkgDir - cmd.Env = append(cmd.Env, "MALLOC_CONF=prof:true") - cmd.Stderr = os.Stderr - log.Printf("%s %s", cmd.Env, cmd.Args) - if out, err := execFn(cmd); err != nil { - log.Fatalf("%s %s: %s\n\n%s", cmd.Env, cmd.Args, err, out) - } - - cmd = exec.Command("ldd", binaryName) - cmd.Dir = o.PkgDir - log.Printf("%s %s", cmd.Env, cmd.Args) - out, err := execFn(cmd) - if err != nil { - log.Fatalf("%s: out=%q err=%s", cmd.Args, out, err) - } - scanner := bufio.NewScanner(bytes.NewReader(out)) - for scanner.Scan() { - if line := scanner.Text(); !libsRe.MatchString(line) { - log.Fatalf("%s is not properly statically linked:\n%s", binaryName, out) - } - } - if err := scanner.Err(); err != nil { - log.Fatal(err) - } + if flags.isRelease { + opts = append(opts, release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "BUILDINFO_TAG", o.VersionStr))) + opts = append(opts, release.WithMakeReleaseOptionBuildArg(fmt.Sprintf("%s=%s", "BUILD_TAGGED_RELEASE", "true"))) + } + + if err := release.MakeRelease( + release.SupportedTarget{ + BuildType: o.BuildType, + Suffix: o.Suffix, + }, + o.PkgDir, + opts..., + ); err != nil { + log.Fatal(err) } } diff --git a/pkg/cmd/publish-provisional-artifacts/main_test.go b/pkg/cmd/publish-provisional-artifacts/main_test.go index c9b74fe55e1f..25a1f19e0602 100644 --- a/pkg/cmd/publish-provisional-artifacts/main_test.go +++ b/pkg/cmd/publish-provisional-artifacts/main_test.go @@ -21,6 +21,7 @@ import ( "unicode/utf8" "github.com/aws/aws-sdk-go/service/s3" + "github.com/cockroachdb/cockroach/pkg/release" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" @@ -227,7 +228,7 @@ func TestBless(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { var s3 mockS3 - var execFn execRunner // bless shouldn't exec anything + var execFn release.ExecFn // bless shouldn't exec anything run(&s3, execFn, test.flags) require.Equal(t, test.expectedGets, s3.gets) require.Equal(t, test.expectedPuts, s3.puts) diff --git a/pkg/release/release.go b/pkg/release/release.go new file mode 100644 index 000000000000..f0e75313983c --- /dev/null +++ b/pkg/release/release.go @@ -0,0 +1,144 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Package release contains utilities for assisting with the release process. +// This is intended for use for the release commands. +package release + +import ( + "bufio" + "bytes" + "io" + "log" + "os" + "os/exec" + "regexp" + "strings" + + "github.com/cockroachdb/errors" +) + +// linuxStaticLibsRe returns the regexp of all static libraries. +var linuxStaticLibsRe = func() *regexp.Regexp { + libs := strings.Join([]string{ + regexp.QuoteMeta("linux-vdso.so."), + regexp.QuoteMeta("librt.so."), + regexp.QuoteMeta("libpthread.so."), + regexp.QuoteMeta("libdl.so."), + regexp.QuoteMeta("libm.so."), + regexp.QuoteMeta("libc.so."), + regexp.QuoteMeta("libresolv.so."), + strings.Replace(regexp.QuoteMeta("ld-linux-ARCH.so."), "ARCH", ".*", -1), + }, "|") + return regexp.MustCompile(libs) +}() + +// SupportedTarget contains metadata about a supported target. +type SupportedTarget struct { + BuildType string + Suffix string +} + +// SupportedTargets contains the supported targets that we build. +var SupportedTargets = []SupportedTarget{ + {BuildType: "darwin", Suffix: ".darwin-10.9-amd64"}, + {BuildType: "linux-gnu", Suffix: ".linux-2.6.32-gnu-amd64"}, + {BuildType: "windows", Suffix: ".windows-6.2-amd64.exe"}, +} + +// makeReleaseAndVerifyOptions are options for MakeRelease. +type makeReleaseAndVerifyOptions struct { + args []string + execFn ExecFn +} + +// ExecFn is a mockable wrapper that executes the given command. +type ExecFn func(*exec.Cmd) ([]byte, error) + +// DefaultExecFn is the default exec function. +var DefaultExecFn ExecFn = func(c *exec.Cmd) ([]byte, error) { + if c.Stdout != nil { + return nil, errors.New("exec: Stdout already set") + } + var stdout bytes.Buffer + c.Stdout = io.MultiWriter(&stdout, os.Stdout) + err := c.Run() + return stdout.Bytes(), err +} + +// MakeReleaseOption as an option for the MakeRelease function. +type MakeReleaseOption func(makeReleaseAndVerifyOptions) makeReleaseAndVerifyOptions + +// WithMakeReleaseOptionBuildArg adds a build argument to release. +func WithMakeReleaseOptionBuildArg(arg string) MakeReleaseOption { + return func(m makeReleaseAndVerifyOptions) makeReleaseAndVerifyOptions { + m.args = append(m.args, arg) + return m + } +} + +// WithMakeReleaseOptionExecFn changes the exec function of the given execFn. +func WithMakeReleaseOptionExecFn(r ExecFn) MakeReleaseOption { + return func(m makeReleaseAndVerifyOptions) makeReleaseAndVerifyOptions { + m.execFn = r + return m + } +} + +// MakeRelease makes the release binary. +func MakeRelease(b SupportedTarget, pkgDir string, opts ...MakeReleaseOption) error { + params := makeReleaseAndVerifyOptions{ + execFn: DefaultExecFn, + } + for _, opt := range opts { + params = opt(params) + } + + { + args := append([]string{b.BuildType}, params.args...) + cmd := exec.Command("mkrelease", args...) + cmd.Dir = pkgDir + cmd.Stderr = os.Stderr + log.Printf("%s %s", cmd.Env, cmd.Args) + if out, err := params.execFn(cmd); err != nil { + return errors.Newf("%s %s: %s\n\n%s", cmd.Env, cmd.Args, err, out) + } + } + if strings.Contains(b.BuildType, "linux") { + binaryName := "./cockroach" + b.Suffix + + cmd := exec.Command(binaryName, "version") + cmd.Dir = pkgDir + cmd.Env = append(cmd.Env, "MALLOC_CONF=prof:true") + cmd.Stderr = os.Stderr + log.Printf("%s %s", cmd.Env, cmd.Args) + if out, err := params.execFn(cmd); err != nil { + return errors.Newf("%s %s: %s\n\n%s", cmd.Env, cmd.Args, err, out) + } + + cmd = exec.Command("ldd", binaryName) + cmd.Dir = pkgDir + log.Printf("%s %s", cmd.Env, cmd.Args) + out, err := params.execFn(cmd) + if err != nil { + log.Fatalf("%s: out=%q err=%s", cmd.Args, out, err) + } + scanner := bufio.NewScanner(bytes.NewReader(out)) + for scanner.Scan() { + if line := scanner.Text(); !linuxStaticLibsRe.MatchString(line) { + return errors.Newf("%s is not properly statically linked:\n%s", binaryName, out) + } + } + if err := scanner.Err(); err != nil { + return err + } + } + return nil +} diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 49d0519e3c25..c2e1b43ab3c7 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1290,7 +1290,7 @@ func TestLint(t *testing.T) { stream.GrepNot(`cockroach/pkg/testutils/lint: log$`), stream.GrepNot(`cockroach/pkg/util/sysutil: syscall$`), stream.GrepNot(`cockroach/pkg/util/log: github\.com/pkg/errors$`), - stream.GrepNot(`cockroach/pkg/(base|security|util/(log|randutil|stop)): log$`), + stream.GrepNot(`cockroach/pkg/(base|release|security|util/(log|randutil|stop)): log$`), stream.GrepNot(`cockroach/pkg/(server/serverpb|ts/tspb): github\.com/golang/protobuf/proto$`), stream.GrepNot(`cockroach/pkg/util/uuid: github\.com/satori/go\.uuid$`), From ef7ab9bc810fd4be7755ebf8880e36ace9453f4b Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Sun, 26 Apr 2020 22:08:44 +0200 Subject: [PATCH 25/46] util/log: make sensitive data redactable MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit See "log redaction" RFC from #48076 for details. General concept: - logs stored on disk contain markers around data items potentially containing PII or confidential data (“unsafe data”). - `debug zip`, `debug merge-log`, and the `Logs` and `LogFiles` RPCs edit the log entries upon request to replace all the unsafe data by `‹×›`. See also the release notes below for a summary of UX. Example redacted output: ``` I200426 20:06:45.558765 1 cli/start.go:671 ⋮ GEOS initialized at ‹×› I200426 20:06:45.558772 1 cli/start.go:676 ⋮ starting cockroach node I200426 20:06:45.562212 75 storage/rocksdb.go:606 ⋮ opening rocksdb instance at ‹×› I200426 20:06:45.574608 75 server/server.go:750 ⋮ [n?] monitoring forward clock jumps based on server.clock.forward_jump_check_enabled I200426 20:06:45.574782 75 storage/rocksdb.go:606 ⋮ opening rocksdb instance at ‹×› I200426 20:06:45.581451 75 server/config.go:576 ⋮ [n?] ‹×› storage engine‹×› initialized I200426 20:06:45.581459 75 server/config.go:579 ⋮ [n?] ‹×› I200426 20:06:45.581466 75 server/config.go:579 ⋮ [n?] ‹×› I200426 20:06:45.602204 75 server/server.go:1270 ⋮ [n?] **** add additional nodes by specifying --join=‹×› I200426 20:06:45.602218 75 server/init.go:166 ⋮ [n?] no stores bootstrapped and --join flag specified, awaiting init command or join with an already initialized node. I200426 20:06:45.602234 75 server/init.go:186 ⋮ [n?] **** cluster ‹×› has been created I200426 20:06:47.676567 117 gossip/gossip.go:1538 ⋮ [n1] node has connected to cluster via gossip I200426 20:06:47.676620 117 kv/kvserver/stores.go:255 ⋮ [n1] wrote ‹×› node addresses to persistent storage I200508 12:34:37.238123 1346 kv/kvserver/replica_consistency.go:255 ⋮ [n1,consistencyChecker,s2,r‹×›] triggering stats recomputation to resolve delta of ‹×› ``` This output was produced from the following raw log: ``` I200426 20:06:45.558765 1 cli/start.go:671 ⋮ GEOS initialized at ‹/data/home/kena/src/go/src/github.com/cockroachdb/cockroach/lib/libgeos_c.so› I200426 20:06:45.558772 1 cli/start.go:676 ⋮ starting cockroach node I200426 20:06:45.562212 75 storage/rocksdb.go:606 ⋮ opening rocksdb instance at ‹"/data/home/kena/src/go/src/github.com/cockroachdb/cockroach/cockroach-data/cockroach-temp302681397"› I200426 20:06:45.574608 75 server/server.go:750 ⋮ [n?] monitoring forward clock jumps based on server.clock.forward_jump_check_enabled I200426 20:06:45.574782 75 storage/rocksdb.go:606 ⋮ opening rocksdb instance at ‹"/data/home/kena/src/go/src/github.com/cockroachdb/cockroach/cockroach-data"› I200426 20:06:45.581451 75 server/config.go:576 ⋮ [n?] ‹1› storage engine‹› initialized I200426 20:06:45.581459 75 server/config.go:579 ⋮ [n?] ‹RocksDB cache size: 128 MiB› I200426 20:06:45.581466 75 server/config.go:579 ⋮ [n?] ‹store 0: RocksDB, max size 0 B, max open file limit 1878853› I200426 20:06:45.602204 75 server/server.go:1270 ⋮ [n?] **** add additional nodes by specifying --join=‹kenax:26257› I200426 20:06:45.602218 75 server/init.go:166 ⋮ [n?] no stores bootstrapped and --join flag specified, awaiting init command or join with an already initialized node. I200426 20:06:45.602234 75 server/init.go:186 ⋮ [n?] **** cluster ‹2a7d34b5-a2aa-469b-8efd-ef7deb62ca73› has been created I200426 20:06:47.676567 117 gossip/gossip.go:1538 ⋮ [n1] node has connected to cluster via gossip I200426 20:06:47.676620 117 kv/kvserver/stores.go:255 ⋮ [n1] wrote ‹0› node addresses to persistent storage I200508 12:34:37.238123 1346 kv/kvserver/replica_consistency.go:255 ⋮ [n1,consistencyChecker,s2,r‹4/1:/System{/tsd-tse}›] triggering stats recomputation to resolve delta of ‹{ContainsEstimates:1438 LastUpdateNanos:1588941276231756380 IntentAge:0 GCBytesAge:0 LiveBytes:-35159 LiveCount:-692 KeyBytes:-33660 KeyCount:-692 ValBytes:-1499 ValCount:-692 IntentBytes:0 IntentCount:0 SysBytes:0 SysCount:0}› ``` Release note (cli change): Certain kinds of advanced troubleshooting information, for example internal details from the Go runtime, are now output to a separate log file in the configured log directory. This file is named `-stderr.xxx.log`. Release note (cli change): The server CLI commands (`start`, `start-single-node`, `demo`) now support a new flag `--redactable-logs`, which introduces markers in generated log entries to facilitate redaction of sensitive information by `cockroach debug zip` and other CockroachDB APIs. This flag is currently not enabled by default, but is recommended for all deployments and will be enabled by default in a later version. This feature is experimental. Note: the flag is *advertised* (in `--help`) by `cockroach start` and `cockroach start-single-node`, where log files are expected as a matter of course; it is also supported, yet *hidden* (from `--help`), by every other CLI command because every command can also produce log entries when passed `--log-dir` or `--logtostderr`. Release note (cli change): The commands `cockroach debug zip` and `cockroach debug merge-logs` support a new flag `--redact-logs`. When specified, it causes log messages that potentially contain confidential data or PII to be redacted away. This feature is experimental. Note 1: At the time of this writing, this redaction is extremely aggressive and may hinder the ability of Cockroach Labs to investigate issues from log files, especially if `--redactable-logs` is not enabled server-side. Note 2: Other potentially-confidential data may still be retrieved by `cockroach debug zip`; the new flag only applies to retrieved log files inside the output zip file. --- Makefile | 15 +- c-deps/libroach/protos/util/log/log.pb.cc | 131 +- c-deps/libroach/protos/util/log/log.pb.h | 122 +- docs/generated/redact_safe.md | 29 + pkg/base/node_id.go | 11 +- pkg/ccl/changefeedccl/changefeed_test.go | 9 +- pkg/ccl/changefeedccl/nemeses_test.go | 3 +- pkg/cli/cliflags/flags.go | 10 + pkg/cli/context.go | 5 + pkg/cli/debug.go | 34 +- pkg/cli/debug_merge_logs.go | 36 +- pkg/cli/debug_merge_logs_test.go | 57 +- pkg/cli/flags.go | 6 +- .../test_missing_log_output.tcl | 4 + .../test_sql_mem_monitor.tcl | 20 +- pkg/cli/start.go | 13 +- pkg/cli/testdata/merge_logs/4/npe-repanic.log | 2 +- pkg/cli/testdata/merge_logs/5/redactable.log | 2 + .../results/5.redact-off-redactable-off | 2 + .../results/5.redact-off-redactable-on | 2 + .../results/5.redact-on-redactable-off | 2 + .../results/5.redact-on-redactable-on | 2 + pkg/cli/zip.go | 27 +- pkg/cmd/roachtest/log.go | 4 +- pkg/gossip/gossip.go | 15 +- pkg/gossip/status.go | 76 +- .../concurrency/concurrency_manager_test.go | 10 +- .../protectedts/ptstorage/storage_test.go | 6 +- pkg/kv/kvserver/raft.go | 3 + pkg/kv/kvserver/raft_snapshot_queue.go | 4 +- pkg/kv/kvserver/replica.go | 53 +- pkg/kv/kvserver/replica_learner_test.go | 2 +- pkg/kv/kvserver/replica_raftstorage.go | 15 +- pkg/kv/kvserver/replica_test.go | 4 +- pkg/kv/txn_test.go | 6 +- pkg/roachpb/data.go | 52 +- pkg/roachpb/metadata.go | 123 +- pkg/roachpb/metadata_test.go | 5 +- pkg/server/debug/logspy.go | 13 +- pkg/server/serverpb/status.pb.go | 1086 ++++++++++------- pkg/server/serverpb/status.pb.gw.go | 18 + pkg/server/serverpb/status.proto | 24 + pkg/server/status.go | 68 +- pkg/server/status/runtime.go | 7 +- pkg/server/status_test.go | 142 ++- pkg/sql/flowinfra/inbound.go | 5 +- pkg/sql/pgwire/auth_test.go | 15 +- pkg/testutils/lint/lint_test.go | 10 + .../lint/passes/fmtsafe/functions.go | 45 +- .../lint/testdata/errcheck_excludes.txt | 2 + .../cluster/containers/nodeLogs/index.tsx | 2 +- pkg/util/hlc/timestamp.go | 3 + pkg/util/log/ambient_context_test.go | 8 +- pkg/util/log/clog.go | 107 +- pkg/util/log/clog_test.go | 39 +- pkg/util/log/crash_reporting.go | 9 +- pkg/util/log/exit_override.go | 15 +- pkg/util/log/file.go | 16 +- pkg/util/log/flags.go | 115 +- pkg/util/log/log.go | 4 +- pkg/util/log/log.pb.go | 194 ++- pkg/util/log/log.proto | 20 + pkg/util/log/log_bridge.go | 28 +- pkg/util/log/log_buffer.go | 2 +- pkg/util/log/log_entry.go | 211 +++- pkg/util/log/log_gc_test.go | 8 - pkg/util/log/logflags/logflags.go | 6 +- pkg/util/log/loggers.go | 21 + pkg/util/log/redact.go | 168 +++ pkg/util/log/redact_test.go | 181 +++ pkg/util/log/secondary_log.go | 43 +- pkg/util/log/secondary_log_test.go | 16 +- pkg/util/log/structured.go | 54 +- pkg/util/log/sync_buffer.go | 48 +- pkg/util/log/test_log_scope.go | 56 +- pkg/util/log/testshout/shout_test.go | 3 - pkg/util/log/trace.go | 162 ++- pkg/util/log/trace_test.go | 9 +- pkg/util/redact/api.go | 121 ++ pkg/util/redact/doc.go | 23 + pkg/util/redact/make_format.go | 81 ++ pkg/util/redact/make_format_test.go | 53 + pkg/util/redact/markers.go | 90 ++ pkg/util/redact/markers_internal_constants.go | 36 + pkg/util/redact/markers_internal_escape.go | 126 ++ pkg/util/redact/markers_internal_print.go | 133 ++ pkg/util/redact/markers_internal_printer.go | 77 ++ pkg/util/redact/markers_internal_types.go | 27 + pkg/util/redact/markers_print.go | 73 ++ pkg/util/redact/markers_test.go | 314 +++++ pkg/util/redact/wrappers.go | 59 + pkg/util/syncutil/atomic.go | 26 + 92 files changed, 4075 insertions(+), 1069 deletions(-) create mode 100644 docs/generated/redact_safe.md create mode 100644 pkg/cli/testdata/merge_logs/5/redactable.log create mode 100644 pkg/cli/testdata/merge_logs/results/5.redact-off-redactable-off create mode 100644 pkg/cli/testdata/merge_logs/results/5.redact-off-redactable-on create mode 100644 pkg/cli/testdata/merge_logs/results/5.redact-on-redactable-off create mode 100644 pkg/cli/testdata/merge_logs/results/5.redact-on-redactable-on create mode 100644 pkg/util/log/loggers.go create mode 100644 pkg/util/log/redact.go create mode 100644 pkg/util/log/redact_test.go create mode 100644 pkg/util/redact/api.go create mode 100644 pkg/util/redact/doc.go create mode 100644 pkg/util/redact/make_format.go create mode 100644 pkg/util/redact/make_format_test.go create mode 100644 pkg/util/redact/markers.go create mode 100644 pkg/util/redact/markers_internal_constants.go create mode 100644 pkg/util/redact/markers_internal_escape.go create mode 100644 pkg/util/redact/markers_internal_print.go create mode 100644 pkg/util/redact/markers_internal_printer.go create mode 100644 pkg/util/redact/markers_internal_types.go create mode 100644 pkg/util/redact/markers_print.go create mode 100644 pkg/util/redact/markers_test.go create mode 100644 pkg/util/redact/wrappers.go diff --git a/Makefile b/Makefile index 729bc98d371e..de8a3ed8ba56 100644 --- a/Makefile +++ b/Makefile @@ -825,7 +825,7 @@ SQLPARSER_TARGETS = \ PROTOBUF_TARGETS := bin/.go_protobuf_sources bin/.gw_protobuf_sources bin/.cpp_protobuf_sources bin/.cpp_ccl_protobuf_sources -DOCGEN_TARGETS := bin/.docgen_bnfs bin/.docgen_functions +DOCGEN_TARGETS := bin/.docgen_bnfs bin/.docgen_functions docs/generated/redact_safe.md EXECGEN_TARGETS = \ pkg/col/coldata/vec.eg.go \ @@ -1528,6 +1528,19 @@ bin/.docgen_functions: bin/docgen docgen functions docs/generated/sql --quiet touch $@ +.PHONY: docs/generated/redact_safe.md + +docs/generated/redact_safe.md: + @(echo "The following types are considered always safe for reporting:"; echo; \ + echo "File | Type"; echo "--|--") >$@.tmp + @git grep '^func \(.*\) SafeValue\(\)' | \ + grep -v '^pkg/util/redact' | \ + sed -E -e 's/^([^:]*):func \(([^ ]* )?(.*)\) SafeValue.*$$/\1 | \`\3\`/g' >>$@.tmp || rm -f $@.tmp + @git grep 'redact\.RegisterSafeType' | \ + grep -v '^pkg/util/redact' | \ + sed -E -e 's/^([^:]*):.*redact\.RegisterSafeType\((.*)\).*/\1 | \`\2\`/g' >>$@.tmp || rm -f $@.tmp + @mv -f $@.tmp $@ + settings-doc-gen := $(if $(filter buildshort,$(MAKECMDGOALS)),$(COCKROACHSHORT),$(COCKROACH)) $(SETTINGS_DOC_PAGE): $(settings-doc-gen) diff --git a/c-deps/libroach/protos/util/log/log.pb.cc b/c-deps/libroach/protos/util/log/log.pb.cc index 331a03b9b171..7c9786787966 100644 --- a/c-deps/libroach/protos/util/log/log.pb.cc +++ b/c-deps/libroach/protos/util/log/log.pb.cc @@ -121,6 +121,9 @@ const int Entry::kGoroutineFieldNumber; const int Entry::kFileFieldNumber; const int Entry::kLineFieldNumber; const int Entry::kMessageFieldNumber; +const int Entry::kTagsFieldNumber; +const int Entry::kCounterFieldNumber; +const int Entry::kRedactableFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 Entry::Entry() @@ -142,18 +145,23 @@ Entry::Entry(const Entry& from) if (from.message().size() > 0) { message_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.message_); } + tags_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + if (from.tags().size() > 0) { + tags_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.tags_); + } ::memcpy(&time_, &from.time_, - static_cast(reinterpret_cast(&severity_) - - reinterpret_cast(&time_)) + sizeof(severity_)); + static_cast(reinterpret_cast(&counter_) - + reinterpret_cast(&time_)) + sizeof(counter_)); // @@protoc_insertion_point(copy_constructor:cockroach.util.log.Entry) } void Entry::SharedCtor() { file_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); message_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + tags_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); ::memset(&time_, 0, static_cast( - reinterpret_cast(&severity_) - - reinterpret_cast(&time_)) + sizeof(severity_)); + reinterpret_cast(&counter_) - + reinterpret_cast(&time_)) + sizeof(counter_)); } Entry::~Entry() { @@ -164,6 +172,7 @@ Entry::~Entry() { void Entry::SharedDtor() { file_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); message_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + tags_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); } void Entry::SetCachedSize(int size) const { @@ -183,9 +192,10 @@ void Entry::Clear() { file_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); message_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + tags_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); ::memset(&time_, 0, static_cast( - reinterpret_cast(&severity_) - - reinterpret_cast(&time_)) + sizeof(severity_)); + reinterpret_cast(&counter_) - + reinterpret_cast(&time_)) + sizeof(counter_)); _internal_metadata_.Clear(); } @@ -294,6 +304,50 @@ bool Entry::MergePartialFromCodedStream( break; } + // string tags = 7; + case 7: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(58u /* 58 & 0xFF */)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadString( + input, this->mutable_tags())); + DO_(::google::protobuf::internal::WireFormatLite::VerifyUtf8String( + this->tags().data(), static_cast(this->tags().length()), + ::google::protobuf::internal::WireFormatLite::PARSE, + "cockroach.util.log.Entry.tags")); + } else { + goto handle_unusual; + } + break; + } + + // uint64 counter = 8; + case 8: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(64u /* 64 & 0xFF */)) { + + DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< + ::google::protobuf::uint64, ::google::protobuf::internal::WireFormatLite::TYPE_UINT64>( + input, &counter_))); + } else { + goto handle_unusual; + } + break; + } + + // bool redactable = 9; + case 9: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(72u /* 72 & 0xFF */)) { + + DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< + bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( + input, &redactable_))); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -361,6 +415,26 @@ void Entry::SerializeWithCachedSizes( ::google::protobuf::internal::WireFormatLite::WriteInt64(6, this->goroutine(), output); } + // string tags = 7; + if (this->tags().size() > 0) { + ::google::protobuf::internal::WireFormatLite::VerifyUtf8String( + this->tags().data(), static_cast(this->tags().length()), + ::google::protobuf::internal::WireFormatLite::SERIALIZE, + "cockroach.util.log.Entry.tags"); + ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( + 7, this->tags(), output); + } + + // uint64 counter = 8; + if (this->counter() != 0) { + ::google::protobuf::internal::WireFormatLite::WriteUInt64(8, this->counter(), output); + } + + // bool redactable = 9; + if (this->redactable() != 0) { + ::google::protobuf::internal::WireFormatLite::WriteBool(9, this->redactable(), output); + } + output->WriteRaw((::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).data(), static_cast((::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).size())); // @@protoc_insertion_point(serialize_end:cockroach.util.log.Entry) @@ -386,6 +460,13 @@ size_t Entry::ByteSizeLong() const { this->message()); } + // string tags = 7; + if (this->tags().size() > 0) { + total_size += 1 + + ::google::protobuf::internal::WireFormatLite::StringSize( + this->tags()); + } + // int64 time = 2; if (this->time() != 0) { total_size += 1 + @@ -400,6 +481,17 @@ size_t Entry::ByteSizeLong() const { this->line()); } + // .cockroach.util.log.Severity severity = 1; + if (this->severity() != 0) { + total_size += 1 + + ::google::protobuf::internal::WireFormatLite::EnumSize(this->severity()); + } + + // bool redactable = 9; + if (this->redactable() != 0) { + total_size += 1 + 1; + } + // int64 goroutine = 6; if (this->goroutine() != 0) { total_size += 1 + @@ -407,10 +499,11 @@ size_t Entry::ByteSizeLong() const { this->goroutine()); } - // .cockroach.util.log.Severity severity = 1; - if (this->severity() != 0) { + // uint64 counter = 8; + if (this->counter() != 0) { total_size += 1 + - ::google::protobuf::internal::WireFormatLite::EnumSize(this->severity()); + ::google::protobuf::internal::WireFormatLite::UInt64Size( + this->counter()); } int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); @@ -438,17 +531,27 @@ void Entry::MergeFrom(const Entry& from) { message_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.message_); } + if (from.tags().size() > 0) { + + tags_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.tags_); + } if (from.time() != 0) { set_time(from.time()); } if (from.line() != 0) { set_line(from.line()); } + if (from.severity() != 0) { + set_severity(from.severity()); + } + if (from.redactable() != 0) { + set_redactable(from.redactable()); + } if (from.goroutine() != 0) { set_goroutine(from.goroutine()); } - if (from.severity() != 0) { - set_severity(from.severity()); + if (from.counter() != 0) { + set_counter(from.counter()); } } @@ -473,10 +576,14 @@ void Entry::InternalSwap(Entry* other) { GetArenaNoVirtual()); message_.Swap(&other->message_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), GetArenaNoVirtual()); + tags_.Swap(&other->tags_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); swap(time_, other->time_); swap(line_, other->line_); - swap(goroutine_, other->goroutine_); swap(severity_, other->severity_); + swap(redactable_, other->redactable_); + swap(goroutine_, other->goroutine_); + swap(counter_, other->counter_); _internal_metadata_.Swap(&other->_internal_metadata_); } diff --git a/c-deps/libroach/protos/util/log/log.pb.h b/c-deps/libroach/protos/util/log/log.pb.h index 60b212c5412f..7ea71bdca2b4 100644 --- a/c-deps/libroach/protos/util/log/log.pb.h +++ b/c-deps/libroach/protos/util/log/log.pb.h @@ -201,6 +201,20 @@ class Entry : public ::google::protobuf::MessageLite /* @@protoc_insertion_point ::std::string* release_message(); void set_allocated_message(::std::string* message); + // string tags = 7; + void clear_tags(); + static const int kTagsFieldNumber = 7; + const ::std::string& tags() const; + void set_tags(const ::std::string& value); + #if LANG_CXX11 + void set_tags(::std::string&& value); + #endif + void set_tags(const char* value); + void set_tags(const char* value, size_t size); + ::std::string* mutable_tags(); + ::std::string* release_tags(); + void set_allocated_tags(::std::string* tags); + // int64 time = 2; void clear_time(); static const int kTimeFieldNumber = 2; @@ -213,17 +227,29 @@ class Entry : public ::google::protobuf::MessageLite /* @@protoc_insertion_point ::google::protobuf::int64 line() const; void set_line(::google::protobuf::int64 value); + // .cockroach.util.log.Severity severity = 1; + void clear_severity(); + static const int kSeverityFieldNumber = 1; + ::cockroach::util::log::Severity severity() const; + void set_severity(::cockroach::util::log::Severity value); + + // bool redactable = 9; + void clear_redactable(); + static const int kRedactableFieldNumber = 9; + bool redactable() const; + void set_redactable(bool value); + // int64 goroutine = 6; void clear_goroutine(); static const int kGoroutineFieldNumber = 6; ::google::protobuf::int64 goroutine() const; void set_goroutine(::google::protobuf::int64 value); - // .cockroach.util.log.Severity severity = 1; - void clear_severity(); - static const int kSeverityFieldNumber = 1; - ::cockroach::util::log::Severity severity() const; - void set_severity(::cockroach::util::log::Severity value); + // uint64 counter = 8; + void clear_counter(); + static const int kCounterFieldNumber = 8; + ::google::protobuf::uint64 counter() const; + void set_counter(::google::protobuf::uint64 value); // @@protoc_insertion_point(class_scope:cockroach.util.log.Entry) private: @@ -231,10 +257,13 @@ class Entry : public ::google::protobuf::MessageLite /* @@protoc_insertion_point ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; ::google::protobuf::internal::ArenaStringPtr file_; ::google::protobuf::internal::ArenaStringPtr message_; + ::google::protobuf::internal::ArenaStringPtr tags_; ::google::protobuf::int64 time_; ::google::protobuf::int64 line_; - ::google::protobuf::int64 goroutine_; int severity_; + bool redactable_; + ::google::protobuf::int64 goroutine_; + ::google::protobuf::uint64 counter_; mutable ::google::protobuf::internal::CachedSize _cached_size_; friend struct ::protobuf_util_2flog_2flog_2eproto::TableStruct; }; @@ -698,6 +727,87 @@ inline void Entry::set_allocated_message(::std::string* message) { // @@protoc_insertion_point(field_set_allocated:cockroach.util.log.Entry.message) } +// string tags = 7; +inline void Entry::clear_tags() { + tags_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); +} +inline const ::std::string& Entry::tags() const { + // @@protoc_insertion_point(field_get:cockroach.util.log.Entry.tags) + return tags_.GetNoArena(); +} +inline void Entry::set_tags(const ::std::string& value) { + + tags_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:cockroach.util.log.Entry.tags) +} +#if LANG_CXX11 +inline void Entry::set_tags(::std::string&& value) { + + tags_.SetNoArena( + &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); + // @@protoc_insertion_point(field_set_rvalue:cockroach.util.log.Entry.tags) +} +#endif +inline void Entry::set_tags(const char* value) { + GOOGLE_DCHECK(value != NULL); + + tags_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:cockroach.util.log.Entry.tags) +} +inline void Entry::set_tags(const char* value, size_t size) { + + tags_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), + ::std::string(reinterpret_cast(value), size)); + // @@protoc_insertion_point(field_set_pointer:cockroach.util.log.Entry.tags) +} +inline ::std::string* Entry::mutable_tags() { + + // @@protoc_insertion_point(field_mutable:cockroach.util.log.Entry.tags) + return tags_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); +} +inline ::std::string* Entry::release_tags() { + // @@protoc_insertion_point(field_release:cockroach.util.log.Entry.tags) + + return tags_.ReleaseNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); +} +inline void Entry::set_allocated_tags(::std::string* tags) { + if (tags != NULL) { + + } else { + + } + tags_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), tags); + // @@protoc_insertion_point(field_set_allocated:cockroach.util.log.Entry.tags) +} + +// uint64 counter = 8; +inline void Entry::clear_counter() { + counter_ = GOOGLE_ULONGLONG(0); +} +inline ::google::protobuf::uint64 Entry::counter() const { + // @@protoc_insertion_point(field_get:cockroach.util.log.Entry.counter) + return counter_; +} +inline void Entry::set_counter(::google::protobuf::uint64 value) { + + counter_ = value; + // @@protoc_insertion_point(field_set:cockroach.util.log.Entry.counter) +} + +// bool redactable = 9; +inline void Entry::clear_redactable() { + redactable_ = false; +} +inline bool Entry::redactable() const { + // @@protoc_insertion_point(field_get:cockroach.util.log.Entry.redactable) + return redactable_; +} +inline void Entry::set_redactable(bool value) { + + redactable_ = value; + // @@protoc_insertion_point(field_set:cockroach.util.log.Entry.redactable) +} + // ------------------------------------------------------------------- // FileDetails diff --git a/docs/generated/redact_safe.md b/docs/generated/redact_safe.md new file mode 100644 index 000000000000..956cabe1bef8 --- /dev/null +++ b/docs/generated/redact_safe.md @@ -0,0 +1,29 @@ +The following types are considered always safe for reporting: + +File | Type +--|-- +pkg/kv/kvserver/raft.go | `SnapshotRequest_Type` +pkg/roachpb/data.go | `ReplicaChangeType` +pkg/roachpb/metadata.go | `NodeID` +pkg/roachpb/metadata.go | `StoreID` +pkg/roachpb/metadata.go | `RangeID` +pkg/roachpb/metadata.go | `ReplicaID` +pkg/roachpb/metadata.go | `ReplicaType` +pkg/util/hlc/timestamp.go | `Timestamp` +pkg/util/log/redact.go | `reflect.TypeOf(true)` +pkg/util/log/redact.go | `reflect.TypeOf(123)` +pkg/util/log/redact.go | `reflect.TypeOf(int8(0))` +pkg/util/log/redact.go | `reflect.TypeOf(int16(0))` +pkg/util/log/redact.go | `reflect.TypeOf(int32(0))` +pkg/util/log/redact.go | `reflect.TypeOf(int64(0))` +pkg/util/log/redact.go | `reflect.TypeOf(uint8(0))` +pkg/util/log/redact.go | `reflect.TypeOf(uint16(0))` +pkg/util/log/redact.go | `reflect.TypeOf(uint32(0))` +pkg/util/log/redact.go | `reflect.TypeOf(uint64(0))` +pkg/util/log/redact.go | `reflect.TypeOf(float32(0))` +pkg/util/log/redact.go | `reflect.TypeOf(float64(0))` +pkg/util/log/redact.go | `reflect.TypeOf(complex64(0))` +pkg/util/log/redact.go | `reflect.TypeOf(complex128(0))` +pkg/util/log/redact.go | `reflect.TypeOf(os.Interrupt)` +pkg/util/log/redact.go | `reflect.TypeOf(time.Time{})` +pkg/util/log/redact.go | `reflect.TypeOf(time.Duration(0))` diff --git a/pkg/base/node_id.go b/pkg/base/node_id.go index 97cc4a91c15e..8132600b9bbb 100644 --- a/pkg/base/node_id.go +++ b/pkg/base/node_id.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/redact" ) // NodeIDContainer is used to share a single roachpb.NodeID instance between @@ -34,11 +35,17 @@ type NodeIDContainer struct { // String returns the node ID, or "?" if it is unset. func (n *NodeIDContainer) String() string { + return redact.StringWithoutMarkers(n) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (n *NodeIDContainer) SafeFormat(w redact.SafePrinter, _ rune) { val := n.Get() if val == 0 { - return "?" + w.SafeRune('?') + } else { + w.Print(val) } - return strconv.Itoa(int(val)) } // Get returns the current node ID; 0 if it is unset. diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index e859590915a3..85ac5745498a 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -595,7 +595,8 @@ func TestChangefeedSchemaChangeNoBackfill(t *testing.T) { t.Run(`sinkless`, sinklessTest(testFn)) t.Run(`enterprise`, enterpriseTest(testFn)) log.Flush() - entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, regexp.MustCompile("cdc ux violation")) + entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, regexp.MustCompile("cdc ux violation"), + log.WithFlattenedSensitiveData) if err != nil { t.Fatal(err) } @@ -777,7 +778,8 @@ func TestChangefeedSchemaChangeAllowBackfill(t *testing.T) { t.Run(`sinkless`, sinklessTest(testFn)) t.Run(`enterprise`, enterpriseTest(testFn)) log.Flush() - entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, regexp.MustCompile("cdc ux violation")) + entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, + regexp.MustCompile("cdc ux violation"), log.WithFlattenedSensitiveData) if err != nil { t.Fatal(err) } @@ -877,7 +879,8 @@ func TestChangefeedAfterSchemaChangeBackfill(t *testing.T) { t.Run(`sinkless`, sinklessTest(testFn)) t.Run(`enterprise`, enterpriseTest(testFn)) log.Flush() - entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, regexp.MustCompile("cdc ux violation")) + entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, + regexp.MustCompile("cdc ux violation"), log.WithFlattenedSensitiveData) if err != nil { t.Fatal(err) } diff --git a/pkg/ccl/changefeedccl/nemeses_test.go b/pkg/ccl/changefeedccl/nemeses_test.go index 1bd80bb817cd..7e6613699450 100644 --- a/pkg/ccl/changefeedccl/nemeses_test.go +++ b/pkg/ccl/changefeedccl/nemeses_test.go @@ -45,7 +45,8 @@ func TestChangefeedNemeses(t *testing.T) { t.Run(`enterprise`, enterpriseTest(testFn)) t.Run(`cloudstorage`, cloudStorageTest(testFn)) log.Flush() - entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, regexp.MustCompile("cdc ux violation")) + entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, + regexp.MustCompile("cdc ux violation"), log.WithFlattenedSensitiveData) if err != nil { t.Fatal(err) } diff --git a/pkg/cli/cliflags/flags.go b/pkg/cli/cliflags/flags.go index 1b2fbf157de5..0f9e1f3ea27b 100644 --- a/pkg/cli/cliflags/flags.go +++ b/pkg/cli/cliflags/flags.go @@ -1172,4 +1172,14 @@ List of nodes to exclude. Can be specified as a comma-delimited list of node IDs or ranges of node IDs, for example: 5,10-20,23. The default is to not exclude any node.`, } + + ZipRedactLogs = FlagInfo{ + Name: "redact-logs", + Description: ` +Redact text that may contain confidential data or PII from retrieved +log entries. Note that this flag only operates on log entries; +other items retrieved by the zip command may still consider +confidential data or PII. +`, + } ) diff --git a/pkg/cli/context.go b/pkg/cli/context.go index c658ca993721..c56c2567c47f 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -93,6 +93,7 @@ func initCLIDefaults() { sqlCtx.echo = false zipCtx.nodes = nodeSelection{} + zipCtx.redactLogs = false dumpCtx.dumpMode = dumpBoth dumpCtx.asOf = "" @@ -274,6 +275,10 @@ var sqlCtx = struct { // Defaults set by InitCLIDefaults() above. var zipCtx struct { nodes nodeSelection + + // redactLogs indicates whether log files should be redacted + // server-side during retrieval. + redactLogs bool } // dumpCtx captures the command-line parameters of the `dump` command. diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 14a89906fa27..6ca6eadf6b03 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -1187,26 +1187,35 @@ the captured parts will be printed. RunE: runDebugMergeLogs, } +// TODO(knz): this struct belongs elsewhere. +// See: https://github.com/cockroachdb/cockroach/issues/49509 var debugMergeLogsOpts = struct { - from time.Time - to time.Time - filter *regexp.Regexp - program *regexp.Regexp - file *regexp.Regexp - prefix string + from time.Time + to time.Time + filter *regexp.Regexp + program *regexp.Regexp + file *regexp.Regexp + prefix string + keepRedactable bool + redactInput bool }{ - program: regexp.MustCompile("^cockroach.*$"), - file: regexp.MustCompile(log.FilePattern), + program: regexp.MustCompile("^cockroach.*$"), + file: regexp.MustCompile(log.FilePattern), + keepRedactable: true, + redactInput: false, } func runDebugMergeLogs(cmd *cobra.Command, args []string) error { o := debugMergeLogsOpts + + inputEditMode := log.SelectEditMode(o.redactInput, o.keepRedactable) + s, err := newMergedStreamFromPatterns(context.Background(), - args, o.file, o.program, o.from, o.to) + args, o.file, o.program, o.from, o.to, inputEditMode) if err != nil { return err } - return writeLogStream(s, cmd.OutOrStdout(), o.filter, o.prefix) + return writeLogStream(s, cmd.OutOrStdout(), o.filter, o.prefix, o.keepRedactable) } // DebugCmdsForRocksDB lists debug commands that access rocksdb through the engine @@ -1303,6 +1312,7 @@ func init() { f = debugMergeLogsCommand.Flags() f.Var(flagutil.Time(&debugMergeLogsOpts.from), "from", "time before which messages should be filtered") + // TODO(knz): the "to" should be named "until" - it's a time boundary, not a space boundary. f.Var(flagutil.Time(&debugMergeLogsOpts.to), "to", "time after which messages should be filtered") f.Var(flagutil.Regexp(&debugMergeLogsOpts.filter), "filter", @@ -1314,4 +1324,8 @@ func init() { "if no such group exists, program-filter is ignored") f.StringVar(&debugMergeLogsOpts.prefix, "prefix", "${host}> ", "expansion template (see regexp.Expand) used as prefix to merged log messages evaluated on file-pattern") + f.BoolVar(&debugMergeLogsOpts.keepRedactable, "redactable-output", debugMergeLogsOpts.keepRedactable, + "keep the output log file redactable") + f.BoolVar(&debugMergeLogsOpts.redactInput, "redact", debugMergeLogsOpts.redactInput, + "redact the input files to remove sensitive information") } diff --git a/pkg/cli/debug_merge_logs.go b/pkg/cli/debug_merge_logs.go index 970fdfe37491..dbf28786dbe1 100644 --- a/pkg/cli/debug_merge_logs.go +++ b/pkg/cli/debug_merge_logs.go @@ -36,7 +36,9 @@ type logStream interface { // writeLogStream pops messages off of s and writes them to out prepending // prefix per message and filtering messages which match filter. -func writeLogStream(s logStream, out io.Writer, filter *regexp.Regexp, prefix string) error { +func writeLogStream( + s logStream, out io.Writer, filter *regexp.Regexp, prefix string, keepRedactable bool, +) error { const chanSize = 1 << 16 // 64k const maxWriteBufSize = 1 << 18 // 256kB @@ -61,6 +63,9 @@ func writeLogStream(s logStream, out io.Writer, filter *regexp.Regexp, prefix st if _, err = w.Write(prefixBytes); err != nil { return err } + if !keepRedactable { + ei.Redactable = false + } return ei.Format(w) } @@ -166,6 +171,7 @@ func newMergedStreamFromPatterns( patterns []string, filePattern, programFilter *regexp.Regexp, from, to time.Time, + editMode log.EditSensitiveData, ) (logStream, error) { paths, err := expandPatterns(patterns) if err != nil { @@ -176,7 +182,7 @@ func newMergedStreamFromPatterns( if err != nil { return nil, err } - return newMergedStream(ctx, files, from, to) + return newMergedStream(ctx, files, from, to, editMode) } func groupIndex(re *regexp.Regexp, groupName string) int { @@ -189,7 +195,7 @@ func groupIndex(re *regexp.Regexp, groupName string) int { } func newMergedStream( - ctx context.Context, files []fileInfo, from, to time.Time, + ctx context.Context, files []fileInfo, from, to time.Time, editMode log.EditSensitiveData, ) (*mergedStream, error) { // TODO(ajwerner): think about clock movement and PID const maxConcurrentFiles = 256 // should be far less than the FD limit @@ -200,7 +206,7 @@ func newMergedStream( return func() error { sem <- struct{}{} defer func() { <-sem }() - s, err := newFileLogStream(files[i], from, to) + s, err := newFileLogStream(files[i], from, to, editMode) if s != nil { res[i] = s } @@ -431,6 +437,7 @@ type fileLogStream struct { f *os.File d *log.EntryDecoder read bool + editMode log.EditSensitiveData e log.Entry err error @@ -442,11 +449,14 @@ type fileLogStream struct { // encountered during the initial peek, that error is returned. The underlying // file is always closed before returning from this constructor so the initial // peek does not consume resources. -func newFileLogStream(fi fileInfo, from, to time.Time) (logStream, error) { +func newFileLogStream( + fi fileInfo, from, to time.Time, editMode log.EditSensitiveData, +) (logStream, error) { s := &fileLogStream{ - fi: fi, - from: from, - to: to, + fi: fi, + from: from, + to: to, + editMode: editMode, } if _, ok := s.peek(); !ok { if err := s.error(); err != io.EOF { @@ -469,10 +479,10 @@ func (s *fileLogStream) open() bool { if s.f, s.err = os.Open(s.fi.path); s.err != nil { return false } - if s.err = seekToFirstAfterFrom(s.f, s.from); s.err != nil { + if s.err = seekToFirstAfterFrom(s.f, s.from, s.editMode); s.err != nil { return false } - s.d = log.NewEntryDecoder(bufio.NewReaderSize(s.f, readBufSize)) + s.d = log.NewEntryDecoder(bufio.NewReaderSize(s.f, readBufSize), s.editMode) return true } @@ -527,7 +537,7 @@ func (s *fileLogStream) error() error { return s.err } // seekToFirstAfterFrom uses binary search to seek to an offset after all // entries which occur before from. -func seekToFirstAfterFrom(f *os.File, from time.Time) (err error) { +func seekToFirstAfterFrom(f *os.File, from time.Time, editMode log.EditSensitiveData) (err error) { if from.IsZero() { return nil } @@ -546,7 +556,7 @@ func seekToFirstAfterFrom(f *os.File, from time.Time) (err error) { panic(err) } var e log.Entry - err := log.NewEntryDecoder(f).Decode(&e) + err := log.NewEntryDecoder(f, editMode).Decode(&e) if err != nil { if err == io.EOF { return true @@ -559,7 +569,7 @@ func seekToFirstAfterFrom(f *os.File, from time.Time) (err error) { return err } var e log.Entry - if err := log.NewEntryDecoder(f).Decode(&e); err != nil { + if err := log.NewEntryDecoder(f, editMode).Decode(&e); err != nil { return err } _, err = f.Seek(int64(offset), io.SeekStart) diff --git a/pkg/cli/debug_merge_logs_test.go b/pkg/cli/debug_merge_logs_test.go index d4fe488201f6..6c3bc28c479d 100644 --- a/pkg/cli/debug_merge_logs_test.go +++ b/pkg/cli/debug_merge_logs_test.go @@ -30,32 +30,34 @@ type testCase struct { var cases = []testCase{ { - name: "1.all", - args: []string{"testdata/merge_logs/1/*/*"}, + name: "1.all", + args: []string{"testdata/merge_logs/1/*/*"}, + flags: []string{"--redact=false", "--redactable-output=false"}, }, { name: "1.filter-program", args: []string{"testdata/merge_logs/1/*/*"}, - flags: []string{"--program-filter", "not-cockroach"}, + flags: []string{"--redact=false", "--redactable-output=false", "--program-filter", "not-cockroach"}, }, { name: "1.seek-past-end-of-file", args: []string{"testdata/merge_logs/1/*/*"}, - flags: []string{"--from", "181130 22:15:07.525317"}, + flags: []string{"--redact=false", "--redactable-output=false", "--from", "181130 22:15:07.525317"}, }, { name: "1.filter-message", args: []string{"testdata/merge_logs/1/*/*"}, - flags: []string{"--filter", "gossip"}, + flags: []string{"--redact=false", "--redactable-output=false", "--filter", "gossip"}, }, { - name: "2.multiple-files-from-node", - args: []string{"testdata/merge_logs/2/*/*"}, + name: "2.multiple-files-from-node", + args: []string{"testdata/merge_logs/2/*/*"}, + flags: []string{"--redact=false", "--redactable-output=false"}, }, { name: "2.skip-file", args: []string{"testdata/merge_logs/2/*/*"}, - flags: []string{"--from", "181130 22:15:07.525316"}, + flags: []string{"--redact=false", "--redactable-output=false", "--from", "181130 22:15:07.525316"}, }, { name: "2.remove-duplicates", @@ -67,36 +69,36 @@ var cases = []testCase{ "testdata/merge_logs/2/2.logs/cockroach.test-0002.ubuntu.2018-11-30T22_06_47Z.003959.log", "testdata/merge_logs/2/2.logs/roachprod.log", }, - flags: []string{"--from", "181130 22:15:07.525316"}, + flags: []string{"--redact=false", "--redactable-output=false", "--from", "181130 22:15:07.525316"}, }, { name: "3.non-standard", args: []string{"testdata/merge_logs/3/*/*"}, - flags: []string{"--file-pattern", ".*", "--prefix", ""}, + flags: []string{"--redact=false", "--redactable-output=false", "--file-pattern", ".*", "--prefix", ""}, }, { // Prints only lines that match the filter (if no submatches). name: "4.filter", args: []string{"testdata/merge_logs/4/*"}, - flags: []string{"--file-pattern", ".*", "--filter", "3:0"}, + flags: []string{"--redact=false", "--redactable-output=false", "--file-pattern", ".*", "--filter", "3:0"}, }, { // Prints only the submatch. name: "4.filter-submatch", args: []string{"testdata/merge_logs/4/*"}, - flags: []string{"--file-pattern", ".*", "--filter", "(3:)0"}, + flags: []string{"--redact=false", "--redactable-output=false", "--file-pattern", ".*", "--filter", "(3:)0"}, }, { // Prints only the submatches. name: "4.filter-submatch-double", args: []string{"testdata/merge_logs/4/*"}, - flags: []string{"--file-pattern", ".*", "--filter", "(3):(0)"}, + flags: []string{"--redact=false", "--redactable-output=false", "--file-pattern", ".*", "--filter", "(3):(0)"}, }, { // Simple grep for a panic line only. name: "4.filter-npe", args: []string{"testdata/merge_logs/4/npe.log"}, - flags: []string{"--file-pattern", ".*", "--filter", `(panic: .*)`}, + flags: []string{"--redact=false", "--redactable-output=false", "--file-pattern", ".*", "--filter", `(panic: .*)`}, }, { // Grep for a panic and a few lines more. This is often not so useful @@ -104,7 +106,7 @@ var cases = []testCase{ // source of the panic is harder to find. name: "4.filter-npe-with-context", args: []string{"testdata/merge_logs/4/npe.log"}, - flags: []string{"--file-pattern", ".*", "--filter", `(?m)(panic:.(?:.*\n){0,5})`}, + flags: []string{"--redact=false", "--redactable-output=false", "--file-pattern", ".*", "--filter", `(?m)(panic:.(?:.*\n){0,5})`}, }, { // This regexp attempts to find the source of the panic, essentially by @@ -121,8 +123,29 @@ var cases = []testCase{ // usually alternate with panic(). name: "4.filter-npe-origin-stack-only", args: []string{"testdata/merge_logs/4/npe-repanic.log"}, // (?:panic\(.*)* - flags: []string{"--file-pattern", ".*", "--filter", `(?m)^(panic\(.*\n.*\n.*\n.*\n[^p].*)`}, - }} + flags: []string{"--redact=false", "--redactable-output=false", "--file-pattern", ".*", "--filter", `(?m)^(panic\(.*\n.*\n.*\n.*\n[^p].*)`}, + }, + { + name: "5.redact-off-redactable-off", + args: []string{"testdata/merge_logs/5/redactable.log"}, + flags: []string{"--redact=false", "--redactable-output=false", "--file-pattern", ".*"}, + }, + { + name: "5.redact-off-redactable-on", + args: []string{"testdata/merge_logs/5/redactable.log"}, + flags: []string{"--redact=false", "--redactable-output=true", "--file-pattern", ".*"}, + }, + { + name: "5.redact-on-redactable-off", + args: []string{"testdata/merge_logs/5/redactable.log"}, + flags: []string{"--redact=true", "--redactable-output=false", "--file-pattern", ".*"}, + }, + { + name: "5.redact-on-redactable-on", + args: []string{"testdata/merge_logs/5/redactable.log"}, + flags: []string{"--redact=true", "--redactable-output=true", "--file-pattern", ".*"}, + }, +} func (c testCase) run(t *testing.T) { outBuf := bytes.Buffer{} diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index 8f55a8dda0fc..dc942d5c8fe9 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -273,9 +273,8 @@ func init() { flag.Hidden = true } switch flag.Name { - case logflags.NoRedirectStderrName: - flag.Hidden = true - case logflags.ShowLogsName: + case logflags.ShowLogsName, // test-only flag + logflags.RedactableLogsName: // support-only flag flag.Hidden = true case logflags.LogToStderrName: // The actual default value for --logtostderr is overridden in @@ -562,6 +561,7 @@ func init() { f := debugZipCmd.Flags() VarFlag(f, &zipCtx.nodes.inclusive, cliflags.ZipNodes) VarFlag(f, &zipCtx.nodes.exclusive, cliflags.ZipExcludeNodes) + BoolFlag(f, &zipCtx.redactLogs, cliflags.ZipRedactLogs, zipCtx.redactLogs) } // Decommission command. diff --git a/pkg/cli/interactive_tests/test_missing_log_output.tcl b/pkg/cli/interactive_tests/test_missing_log_output.tcl index c4c92687a28a..e53817e51ac5 100644 --- a/pkg/cli/interactive_tests/test_missing_log_output.tcl +++ b/pkg/cli/interactive_tests/test_missing_log_output.tcl @@ -97,6 +97,10 @@ send "cat logs/db/logs/cockroach.log\r" eexpect "a SQL panic has occurred" eexpect "helloworld" eexpect "a panic has occurred" +eexpect ":/# " +send "cat logs/db/logs/cockroach-stderr.log\r" +eexpect "panic" +eexpect "helloworld" eexpect "goroutine" eexpect ":/# " diff --git a/pkg/cli/interactive_tests/test_sql_mem_monitor.tcl b/pkg/cli/interactive_tests/test_sql_mem_monitor.tcl index 09a4f116dacd..9efb75daa489 100644 --- a/pkg/cli/interactive_tests/test_sql_mem_monitor.tcl +++ b/pkg/cli/interactive_tests/test_sql_mem_monitor.tcl @@ -43,10 +43,14 @@ eexpect ":/# " send "ulimit -v [ expr {3*$vmem/2} ]\r" eexpect ":/# " -# Start a server with this limit set. The server will now run in the foreground. -send "$argv start-single-node --insecure --max-sql-memory=25% --no-redirect-stderr -s=path=logs/db \r" -eexpect "restarted pre-existing node" -sleep 1 +# Start a server with this limit set. +send "$argv start-single-node --insecure --max-sql-memory=25% -s=path=logs/db --background --pid-file=server_pid\r" +eexpect ":/# " +send "$argv sql --insecure -e 'select 1'\r" +eexpect "1 row" +eexpect ":/# " +send "tail -F logs/db/logs/cockroach-stderr.log\r" +eexpect "stderr capture started" # Spawn a client. spawn $argv sql @@ -68,7 +72,8 @@ eexpect root@ # Disable query distribution to force in-memory computation. send "set distsql=off;\r" eexpect SET -send "with a as (select * from generate_series(1,10000)) select * from a as a, a as b, a as c, a as d limit 10;\r" +send "with a as (select * from generate_series(1,10000000)) select * from a as a, a as b, a as c, a as d limit 10;\r" +eexpect "connection lost" # Check that the query crashed the server set spawn_id $shell_spawn_id @@ -83,18 +88,19 @@ expect { "signal SIGSEGV" {} timeout { handle_timeout "memory allocation error" } } +# Stop the tail command. +interrupt eexpect ":/# " # Check that the client got a bad connection error set spawn_id $client_spawn_id -eexpect "bad connection" eexpect root@ end_test start_test "Ensure that memory monitoring prevents crashes" # Re-launch a server with relatively lower limit for SQL memory set spawn_id $shell_spawn_id -send "$argv start-single-node --insecure --max-sql-memory=1000K --no-redirect-stderr -s=path=logs/db \r" +send "$argv start-single-node --insecure --max-sql-memory=1000K -s=path=logs/db \r" eexpect "restarted pre-existing node" sleep 2 diff --git a/pkg/cli/start.go b/pkg/cli/start.go index ee150905ba0d..9014060bf531 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -1223,11 +1223,6 @@ func setupAndInitializeLoggingAndProfiling( return nil, err } - // NB: this message is a crutch until #33458 is addressed. Without it, - // the calls to log.Shout below can be the first use of logging, hitting - // the bug described in the issue. - log.Infof(ctx, "logging to directory %s", logDir) - // Start the log file GC daemon to remove files that make the log // directory too large. log.StartGCDaemon(ctx) @@ -1243,6 +1238,14 @@ func setupAndInitializeLoggingAndProfiling( }() } + // Initialize the redirection of stderr and log redaction. Note, + // this function must be called even if there is no log directory + // configured, to verify whether the combination of requested flags + // is valid. + if _, err := log.SetupRedactionAndStderrRedirects(); err != nil { + return nil, err + } + // We want to be careful to still produce useful debug dumps if the // server configuration has disabled logging to files. outputDirectory := "." diff --git a/pkg/cli/testdata/merge_logs/4/npe-repanic.log b/pkg/cli/testdata/merge_logs/4/npe-repanic.log index ef61d1c33611..ed451f5d640f 100644 --- a/pkg/cli/testdata/merge_logs/4/npe-repanic.log +++ b/pkg/cli/testdata/merge_logs/4/npe-repanic.log @@ -1,4 +1,4 @@ -I190412 10:06:00.490104 183717 ccl/partitionccl/partition_test.go:204 i was created via the following code: +I190412 10:06:00.490104 183717 ccl/partitionccl/partition_test.go:204 i was created via the following code: func main() { defer cleanup() defer cleanup() diff --git a/pkg/cli/testdata/merge_logs/5/redactable.log b/pkg/cli/testdata/merge_logs/5/redactable.log new file mode 100644 index 000000000000..9053f9a06a61 --- /dev/null +++ b/pkg/cli/testdata/merge_logs/5/redactable.log @@ -0,0 +1,2 @@ +I190412 10:06:00.490104 183717 server/server.go:1423 ⋮ safe ‹unsafe› +I190412 10:06:00.490104 183717 server/server.go:1424 unknownsafe diff --git a/pkg/cli/testdata/merge_logs/results/5.redact-off-redactable-off b/pkg/cli/testdata/merge_logs/results/5.redact-off-redactable-off new file mode 100644 index 000000000000..8412e23e7b8f --- /dev/null +++ b/pkg/cli/testdata/merge_logs/results/5.redact-off-redactable-off @@ -0,0 +1,2 @@ +> I190412 10:06:00.490104 183717 server/server.go:1423 safe unsafe +> I190412 10:06:00.490104 183717 server/server.go:1424 unknownsafe diff --git a/pkg/cli/testdata/merge_logs/results/5.redact-off-redactable-on b/pkg/cli/testdata/merge_logs/results/5.redact-off-redactable-on new file mode 100644 index 000000000000..6ae8be145614 --- /dev/null +++ b/pkg/cli/testdata/merge_logs/results/5.redact-off-redactable-on @@ -0,0 +1,2 @@ +> I190412 10:06:00.490104 183717 server/server.go:1423 ⋮ safe ‹unsafe› +> I190412 10:06:00.490104 183717 server/server.go:1424 ⋮ ‹unknownsafe› diff --git a/pkg/cli/testdata/merge_logs/results/5.redact-on-redactable-off b/pkg/cli/testdata/merge_logs/results/5.redact-on-redactable-off new file mode 100644 index 000000000000..9a4d3ec83be5 --- /dev/null +++ b/pkg/cli/testdata/merge_logs/results/5.redact-on-redactable-off @@ -0,0 +1,2 @@ +> I190412 10:06:00.490104 183717 server/server.go:1423 safe ‹×› +> I190412 10:06:00.490104 183717 server/server.go:1424 ‹×› diff --git a/pkg/cli/testdata/merge_logs/results/5.redact-on-redactable-on b/pkg/cli/testdata/merge_logs/results/5.redact-on-redactable-on new file mode 100644 index 000000000000..5547f1c32955 --- /dev/null +++ b/pkg/cli/testdata/merge_logs/results/5.redact-on-redactable-on @@ -0,0 +1,2 @@ +> I190412 10:06:00.490104 183717 server/server.go:1423 ⋮ safe ‹×› +> I190412 10:06:00.490104 183717 server/server.go:1424 ⋮ ‹×› diff --git a/pkg/cli/zip.go b/pkg/cli/zip.go index 40b6b677e503..459ab201416e 100644 --- a/pkg/cli/zip.go +++ b/pkg/cli/zip.go @@ -574,7 +574,9 @@ func runDebugZip(cmd *cobra.Command, args []string) (retErr error) { if err := runZipRequestWithTimeout(baseCtx, fmt.Sprintf("requesting log file %s", file.Name), timeout, func(ctx context.Context) error { entries, err = status.LogFile( - ctx, &serverpb.LogFileRequest{NodeId: id, File: file.Name}) + ctx, &serverpb.LogFileRequest{ + NodeId: id, File: file.Name, Redact: zipCtx.redactLogs, KeepRedactable: true, + }) return err }); err != nil { if err := z.createError(name, err); err != nil { @@ -586,11 +588,34 @@ func runDebugZip(cmd *cobra.Command, args []string) (retErr error) { if err != nil { return err } + warnRedactLeak := false for _, e := range entries.Entries { + // If the user requests redaction, and some non-redactable + // data was found in the log, *despite KeepRedactable + // being set*, this means that this zip client is talking + // to a node that doesn't yet know how to redact. This + // also means that node may be leaking sensitive data. + // + // In that case, we do the redaction work ourselves in the + // most conservative way possible. (It's not great that + // possibly confidential data flew over the network, but + // at least it stops here.) + if zipCtx.redactLogs && !e.Redactable { + e.Message = "REDACTEDBYZIP" + // We're also going to print a warning at the end. + warnRedactLeak = true + } if err := e.Format(logOut); err != nil { return err } } + if warnRedactLeak { + // Defer the warning, so that it does not get "drowned" as + // part of the main zip output. + defer func(fileName string) { + fmt.Fprintf(stderr, "WARNING: server-side redaction failed for %s, completed client-side (--redact-logs=true)\n", fileName) + }(file.Name) + } } } diff --git a/pkg/cmd/roachtest/log.go b/pkg/cmd/roachtest/log.go index e1c5698c26c9..973034502e5e 100644 --- a/pkg/cmd/roachtest/log.go +++ b/pkg/cmd/roachtest/log.go @@ -250,7 +250,7 @@ func (l *logger) Printf(f string, args ...interface{}) { // which stack frame is reported as the file:line in the message. depth=1 is // equivalent to PrintfCtx. E.g. pass 2 to ignore the caller's frame. func (l *logger) PrintfCtxDepth(ctx context.Context, depth int, f string, args ...interface{}) { - msg := crdblog.MakeMessage(ctx, f, args) + msg := crdblog.FormatWithContextTags(ctx, f, args...) if err := l.stdoutL.Output(depth+1, msg); err != nil { // Changing our interface to return an Error from a logging method seems too // onerous. Let's yell to the default logger and if that fails, oh well. @@ -264,7 +264,7 @@ func (l *logger) ErrorfCtx(ctx context.Context, f string, args ...interface{}) { } func (l *logger) ErrorfCtxDepth(ctx context.Context, depth int, f string, args ...interface{}) { - msg := crdblog.MakeMessage(ctx, f, args) + msg := crdblog.FormatWithContextTags(ctx, f, args...) if err := l.stderrL.Output(depth+1, msg); err != nil { // Changing our interface to return an Error from a logging method seems too // onerous. Let's yell to the default logger and if that fails, oh well. diff --git a/pkg/gossip/gossip.go b/pkg/gossip/gossip.go index ca6423effe58..e4f257b21c30 100644 --- a/pkg/gossip/gossip.go +++ b/pkg/gossip/gossip.go @@ -72,6 +72,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/redact" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -275,7 +276,7 @@ type Gossip struct { localityTierMap map[string]struct{} - lastConnectivity string + lastConnectivity redact.RedactableString defaultZoneConfig *zonepb.ZoneConfig } @@ -561,21 +562,23 @@ func (g *Gossip) GetNodeDescriptor(nodeID roachpb.NodeID) (*roachpb.NodeDescript func (g *Gossip) LogStatus() { g.mu.RLock() n := len(g.nodeDescs) - status := "ok" + status := redact.SafeString("ok") if g.mu.is.getInfo(KeySentinel) == nil { - status = "stalled" + status = redact.SafeString("stalled") } g.mu.RUnlock() - var connectivity string - if s := g.Connectivity().String(); s != g.lastConnectivity { + var connectivity redact.RedactableString + if s := redact.Sprint(g.Connectivity()); s != g.lastConnectivity { g.lastConnectivity = s connectivity = s } ctx := g.AnnotateCtx(context.TODO()) log.Infof(ctx, "gossip status (%s, %d node%s)\n%s%s%s", - status, n, util.Pluralize(int64(n)), g.clientStatus(), g.server.status(), connectivity) + status, n, util.Pluralize(int64(n)), + g.clientStatus(), g.server.status(), + connectivity) } func (g *Gossip) clientStatus() ClientStatus { diff --git a/pkg/gossip/status.go b/pkg/gossip/status.go index 5c0ae33338b0..eb34cf8d8b6e 100644 --- a/pkg/gossip/status.go +++ b/pkg/gossip/status.go @@ -11,11 +11,10 @@ package gossip import ( - "bytes" - "fmt" "time" "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/redact" ) // Metrics contains gossip metrics used per node and server. @@ -38,7 +37,7 @@ func makeMetrics() Metrics { } func (m Metrics) String() string { - return m.Snapshot().String() + return redact.StringWithoutMarkers(m.Snapshot()) } // Snapshot returns a snapshot of the metrics. @@ -53,54 +52,81 @@ func (m Metrics) Snapshot() MetricSnap { } func (m MetricSnap) String() string { - s := fmt.Sprintf("infos %d/%d sent/received, bytes %dB/%dB sent/received", - m.InfosSent, m.InfosReceived, m.BytesSent, m.BytesReceived) + return redact.StringWithoutMarkers(m) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (m MetricSnap) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("infos %d/%d sent/received, bytes %dB/%dB sent/received", + m.InfosSent, m.InfosReceived, + m.BytesSent, m.BytesReceived) if m.ConnsRefused > 0 { - s += fmt.Sprintf(", refused %d conns", m.ConnsRefused) + w.Printf(", refused %d conns", m.ConnsRefused) } - return s } func (c OutgoingConnStatus) String() string { - return fmt.Sprintf("%d: %s (%s: %s)", - c.NodeID, c.Address, roundSecs(time.Duration(c.AgeNanos)), c.MetricSnap) + return redact.StringWithoutMarkers(c) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (c OutgoingConnStatus) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("%d: %s (%s: %s)", + c.NodeID, c.Address, + roundSecs(time.Duration(c.AgeNanos)), c.MetricSnap) } func (c ClientStatus) String() string { - var buf bytes.Buffer - fmt.Fprintf(&buf, "gossip client (%d/%d cur/max conns)\n", len(c.ConnStatus), c.MaxConns) + return redact.StringWithoutMarkers(c) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (c ClientStatus) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("gossip client (%d/%d cur/max conns)\n", + len(c.ConnStatus), c.MaxConns) for _, conn := range c.ConnStatus { - fmt.Fprintf(&buf, " %s\n", conn) + w.Printf(" %s\n", conn) } - return buf.String() } func (c ConnStatus) String() string { - return fmt.Sprintf("%d: %s (%s)", c.NodeID, c.Address, roundSecs(time.Duration(c.AgeNanos))) + return redact.StringWithoutMarkers(c) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (c ConnStatus) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("%d: %s (%s)", c.NodeID, c.Address, + roundSecs(time.Duration(c.AgeNanos))) } func (s ServerStatus) String() string { - var buf bytes.Buffer - fmt.Fprintf(&buf, "gossip server (%d/%d cur/max conns, %s)\n", + return redact.StringWithoutMarkers(s) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (s ServerStatus) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("gossip server (%d/%d cur/max conns, %s)\n", len(s.ConnStatus), s.MaxConns, s.MetricSnap) for _, conn := range s.ConnStatus { - fmt.Fprintf(&buf, " %s\n", conn) + w.Printf(" %s\n", conn) } - return buf.String() } func (c Connectivity) String() string { - var buf bytes.Buffer - fmt.Fprintf(&buf, "gossip connectivity\n") + return redact.StringWithoutMarkers(c) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (c Connectivity) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("gossip connectivity\n") if c.SentinelNodeID != 0 { - fmt.Fprintf(&buf, " n%d [sentinel];\n", c.SentinelNodeID) + w.Printf(" n%d [sentinel];\n", c.SentinelNodeID) } if len(c.ClientConns) > 0 { - fmt.Fprintf(&buf, " ") + w.SafeRune(' ') for _, conn := range c.ClientConns { - fmt.Fprintf(&buf, " n%d -> n%d;", conn.SourceID, conn.TargetID) + w.Printf(" n%d -> n%d;", conn.SourceID, conn.TargetID) } - fmt.Fprintf(&buf, "\n") + w.SafeRune('\n') } - return buf.String() } diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index aba3ca1b4ebc..df078d108ec5 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -16,6 +16,7 @@ import ( "fmt" "io/ioutil" "reflect" + "regexp" "runtime" "sort" "strconv" @@ -925,11 +926,18 @@ func (m *monitor) collectRecordings() string { if log.g.opSeq != 0 { seq = strconv.Itoa(log.g.opSeq) } - fmt.Fprintf(&buf, "[%s] %s: %s", seq, log.g.opName, log.value) + logValue := stripFileLinePrefix(log.value) + fmt.Fprintf(&buf, "[%s] %s: %s", seq, log.g.opName, logValue) } return buf.String() } +func stripFileLinePrefix(s string) string { + return reFileLinePrefix.ReplaceAllString(s, "") +} + +var reFileLinePrefix = regexp.MustCompile(`^[^:]+:\d+ `) + func (m *monitor) hasNewEvents(g *monitoredGoroutine) bool { events := 0 rec := g.collect() diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go index 795824d31b70..a71549a51188 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go @@ -465,7 +465,8 @@ func TestCorruptData(t *testing.T) { return err })) log.Flush() - entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 100, msg) + entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 100, msg, + log.WithFlattenedSensitiveData) require.NoError(t, err) require.Len(t, entries, 1) for _, e := range entries { @@ -516,7 +517,8 @@ func TestCorruptData(t *testing.T) { })) log.Flush() - entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 100, msg) + entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 100, msg, + log.WithFlattenedSensitiveData) require.NoError(t, err) require.Len(t, entries, 1) for _, e := range entries { diff --git a/pkg/kv/kvserver/raft.go b/pkg/kv/kvserver/raft.go index 6b29eed5ac75..73571545a4da 100644 --- a/pkg/kv/kvserver/raft.go +++ b/pkg/kv/kvserver/raft.go @@ -293,3 +293,6 @@ func traceProposals(r *Replica, ids []kvserverbase.CmdIDKey, event string) { log.Eventf(ctx, "%v", event) } } + +// SafeValue implements the redact.SafeValue interface. +func (SnapshotRequest_Type) SafeValue() {} diff --git a/pkg/kv/kvserver/raft_snapshot_queue.go b/pkg/kv/kvserver/raft_snapshot_queue.go index 3bba46fecf00..2911dab29e89 100644 --- a/pkg/kv/kvserver/raft_snapshot_queue.go +++ b/pkg/kv/kvserver/raft_snapshot_queue.go @@ -123,7 +123,9 @@ func (rq *raftSnapshotQueue) processRaftSnapshot( // bail for now and try again later. err := errors.Errorf( "skipping snapshot; replica is likely a learner in the process of being added: %s", repDesc) - log.Infof(ctx, "%v", err) + // TODO(knz): print the error instead when the error package + // knows how to expose redactable strings. + log.Infof(ctx, "skipping snapshot; replica is likely a learner in the process of being added: %s", repDesc) // TODO(dan): This is super brittle and non-obvious. In the common case, // this check avoids duplicate work, but in rare cases, we send the // learner snap at an index before the one raft wanted here. The raft diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 2539ed91fbb4..4052d271d310 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -13,7 +13,6 @@ package kvserver import ( "context" "fmt" - "strings" "sync/atomic" "time" "unsafe" @@ -46,6 +45,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/quotapool" + "github.com/cockroachdb/cockroach/pkg/util/redact" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -133,30 +133,41 @@ type atomicDescString struct { // store atomically updates d.strPtr with the string representation of desc. func (d *atomicDescString) store(replicaID roachpb.ReplicaID, desc *roachpb.RangeDescriptor) { - var buf strings.Builder - fmt.Fprintf(&buf, "%d/", desc.RangeID) - if replicaID == 0 { - fmt.Fprintf(&buf, "?:") - } else { - fmt.Fprintf(&buf, "%d:", replicaID) - } + str := redact.Sprintfn(func(w redact.SafePrinter) { + w.Printf("%d/", desc.RangeID) + if replicaID == 0 { + w.SafeString("?:") + } else { + w.Printf("%d:", replicaID) + } - if !desc.IsInitialized() { - buf.WriteString("{-}") - } else { - const maxRangeChars = 30 - rngStr := keys.PrettyPrintRange(roachpb.Key(desc.StartKey), roachpb.Key(desc.EndKey), maxRangeChars) - buf.WriteString(rngStr) - } + if !desc.IsInitialized() { + w.SafeString("{-}") + } else { + const maxRangeChars = 30 + rngStr := keys.PrettyPrintRange(roachpb.Key(desc.StartKey), roachpb.Key(desc.EndKey), maxRangeChars) + w.UnsafeString(rngStr) + } + }) - str := buf.String() atomic.StorePointer(&d.strPtr, unsafe.Pointer(&str)) } // String returns the string representation of the range; since we are not // using a lock, the copy might be inconsistent. func (d *atomicDescString) String() string { - return *(*string)(atomic.LoadPointer(&d.strPtr)) + return d.get().StripMarkers() +} + +// SafeFormat renders the string safely. +func (d *atomicDescString) SafeFormat(w redact.SafePrinter, _ rune) { + w.Print(d.get()) +} + +// Get returns the string representation of the range; since we are not +// using a lock, the copy might be inconsistent. +func (d *atomicDescString) get() redact.RedactableString { + return *(*redact.RedactableString)(atomic.LoadPointer(&d.strPtr)) } // atomicConnectionClass stores an rpc.ConnectionClass atomically. @@ -581,7 +592,13 @@ var _ kv.Sender = &Replica{} // require a lock and its output may not be atomic with other ongoing work in // the replica. This is done to prevent deadlocks in logging sites. func (r *Replica) String() string { - return fmt.Sprintf("[n%d,s%d,r%s]", r.store.Ident.NodeID, r.store.Ident.StoreID, &r.rangeStr) + return redact.StringWithoutMarkers(r) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (r *Replica) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("[n%d,s%d,r%s]", + r.store.Ident.NodeID, r.store.Ident.StoreID, r.rangeStr.get()) } // ReplicaID returns the ID for the Replica. It may be zero if the replica does diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index de572196dbca..fb2b3997cc27 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -629,7 +629,7 @@ func TestLearnerReplicateQueueRace(t *testing.T) { } formattedTrace := trace.String() expectedMessages := []string{ - `could not promote .*n3,s3.* to voter, rolling back: change replicas of r\d+ failed: descriptor changed`, + `could not promote .*n3,s3.* to voter, rolling back:.*change replicas of r\d+ failed: descriptor changed`, `learner to roll back not found`, } return testutils.MatchInOrder(formattedTrace, expectedMessages...) diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index d615bb6bc228..324fe2c7f5e2 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/redact" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -462,7 +463,12 @@ type OutgoingSnapshot struct { } func (s *OutgoingSnapshot) String() string { - return fmt.Sprintf("%s snapshot %s at applied index %d", s.snapType, s.SnapUUID.Short(), s.State.RaftAppliedIndex) + return redact.StringWithoutMarkers(s) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (s *OutgoingSnapshot) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("%s snapshot %s at applied index %d", s.snapType, s.SnapUUID.Short(), s.State.RaftAppliedIndex) } // Close releases the resources associated with the snapshot. @@ -496,7 +502,12 @@ type IncomingSnapshot struct { } func (s *IncomingSnapshot) String() string { - return fmt.Sprintf("%s snapshot %s at applied index %d", s.snapType, s.SnapUUID.Short(), s.State.RaftAppliedIndex) + return redact.StringWithoutMarkers(s) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (s *IncomingSnapshot) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("%s snapshot %s at applied index %d", s.snapType, s.SnapUUID.Short(), s.State.RaftAppliedIndex) } // snapshot creates an OutgoingSnapshot containing a rocksdb snapshot for the diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 68b73212f409..02e34dc4f5ae 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -12086,7 +12086,7 @@ func TestProposalNotAcknowledgedOrReproposedAfterApplication(t *testing.T) { stopper.Quiesce(ctx) entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, - regexp.MustCompile("net/trace")) + regexp.MustCompile("net/trace"), log.WithFlattenedSensitiveData) if err != nil { t.Fatal(err) } @@ -12188,7 +12188,7 @@ func TestLaterReproposalsDoNotReuseContext(t *testing.T) { // Check and see if the trace package logged an error. log.Flush() entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 1, - regexp.MustCompile("net/trace")) + regexp.MustCompile("net/trace"), log.WithFlattenedSensitiveData) if err != nil { t.Fatal(err) } diff --git a/pkg/kv/txn_test.go b/pkg/kv/txn_test.go index 189b609f6c1d..87f688f3c745 100644 --- a/pkg/kv/txn_test.go +++ b/pkg/kv/txn_test.go @@ -62,7 +62,11 @@ func TestTxnSnowballTrace(t *testing.T) { found, err := regexp.MatchString( // The (?s) makes "." match \n. This makes the test resilient to other log // lines being interspersed. - "(?s).*event:inside txn\n.*event:client.Txn did AutoCommit. err: \n.*\n.*event:txn complete.*", + `(?s)`+ + `.*event:[^:]*:\d+ inside txn\n`+ + `.*event:[^:]*:\d+ client\.Txn did AutoCommit\. err: \n`+ + `.*\n`+ + `.*event:[^:]*:\d+ txn complete.*`, dump) if err != nil { t.Fatal(err) diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index ff09c219d246..e01ac2424cae 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/interval" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/redact" "github.com/cockroachdb/cockroach/pkg/util/timetz" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -1603,6 +1604,11 @@ func confChangeImpl( var _ fmt.Stringer = &ChangeReplicasTrigger{} func (crt ChangeReplicasTrigger) String() string { + return redact.StringWithoutMarkers(crt) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (crt ChangeReplicasTrigger) SafeFormat(w redact.SafePrinter, _ rune) { var nextReplicaID ReplicaID var afterReplicas []ReplicaDescriptor added, removed := crt.Added(), crt.Removed() @@ -1617,10 +1623,9 @@ func (crt ChangeReplicasTrigger) String() string { nextReplicaID = crt.DeprecatedNextReplicaID afterReplicas = crt.DeprecatedUpdatedReplicas } - var chgS strings.Builder cc, err := crt.ConfChange(nil) if err != nil { - fmt.Fprintf(&chgS, "", err) + w.Printf("", err) } else { ccv2 := cc.AsV2() if ccv2.LeaveJoint() { @@ -1628,24 +1633,48 @@ func (crt ChangeReplicasTrigger) String() string { // // TODO(tbg): could list the replicas that will actually leave the // voter set. - fmt.Fprintf(&chgS, "LEAVE_JOINT") + w.SafeString("LEAVE_JOINT") } else if _, ok := ccv2.EnterJoint(); ok { - fmt.Fprintf(&chgS, "ENTER_JOINT(%s) ", raftpb.ConfChangesToString(ccv2.Changes)) + w.Printf("ENTER_JOINT(%s) ", confChangesToRedactableString(ccv2.Changes)) } else { - fmt.Fprintf(&chgS, "SIMPLE(%s) ", raftpb.ConfChangesToString(ccv2.Changes)) + w.Printf("SIMPLE(%s) ", confChangesToRedactableString(ccv2.Changes)) } } if len(added) > 0 { - fmt.Fprintf(&chgS, "%s%s", ADD_REPLICA, added) + w.Printf("%s%s", ADD_REPLICA, added) } if len(removed) > 0 { if len(added) > 0 { - chgS.WriteString(", ") + w.SafeString(", ") } - fmt.Fprintf(&chgS, "%s%s", REMOVE_REPLICA, removed) + w.Printf("%s%s", REMOVE_REPLICA, removed) } - fmt.Fprintf(&chgS, ": after=%s next=%d", afterReplicas, nextReplicaID) - return chgS.String() + w.Printf(": after=%s next=%d", afterReplicas, nextReplicaID) +} + +// confChangesToRedactableString produces a safe representation for +// the configuration changes. +func confChangesToRedactableString(ccs []raftpb.ConfChangeSingle) redact.RedactableString { + return redact.Sprintfn(func(w redact.SafePrinter) { + for i, cc := range ccs { + if i > 0 { + w.SafeRune(' ') + } + switch cc.Type { + case raftpb.ConfChangeAddNode: + w.SafeRune('v') + case raftpb.ConfChangeAddLearnerNode: + w.SafeRune('l') + case raftpb.ConfChangeRemoveNode: + w.SafeRune('r') + case raftpb.ConfChangeUpdateNode: + w.SafeRune('u') + default: + w.SafeString("unknown") + } + w.Print(cc.NodeID) + } + }) } func (crt ChangeReplicasTrigger) legacy() (ReplicaDescriptor, bool) { @@ -2247,3 +2276,6 @@ func init() { enginepb.FormatBytesAsKey = func(k []byte) string { return Key(k).String() } enginepb.FormatBytesAsValue = func(v []byte) string { return Value{RawBytes: v}.PrettyPrint() } } + +// SafeValue implements the redact.SafeValue interface. +func (ReplicaChangeType) SafeValue() {} diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index e0d82e30d46b..70a34a1cf24e 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -11,7 +11,6 @@ package roachpb import ( - "bytes" "fmt" "sort" "strconv" @@ -19,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" + "github.com/cockroachdb/cockroach/pkg/util/redact" "github.com/cockroachdb/errors" ) @@ -32,6 +32,9 @@ func (n NodeID) String() string { return strconv.FormatInt(int64(n), 10) } +// SafeValue implements the redact.SafeValue interface. +func (n NodeID) SafeValue() {} + // StoreID is a custom type for a cockroach store ID. type StoreID int32 @@ -48,6 +51,9 @@ func (n StoreID) String() string { return strconv.FormatInt(int64(n), 10) } +// SafeValue implements the redact.SafeValue interface. +func (n StoreID) SafeValue() {} + // A RangeID is a unique ID associated to a Raft consensus group. type RangeID int64 @@ -56,6 +62,9 @@ func (r RangeID) String() string { return strconv.FormatInt(int64(r), 10) } +// SafeValue implements the redact.SafeValue interface. +func (r RangeID) SafeValue() {} + // RangeIDSlice implements sort.Interface. type RangeIDSlice []RangeID @@ -71,6 +80,9 @@ func (r ReplicaID) String() string { return strconv.FormatInt(int64(r), 10) } +// SafeValue implements the redact.SafeValue interface. +func (r ReplicaID) SafeValue() {} + // Equals returns whether the Attributes lists are equivalent. Attributes lists // are treated as sets, meaning that ordering and duplicates are ignored. func (a Attributes) Equals(b Attributes) bool { @@ -281,100 +293,60 @@ func (r *RangeDescriptor) Validate() error { } func (r RangeDescriptor) String() string { - var buf bytes.Buffer - fmt.Fprintf(&buf, "r%d:", r.RangeID) + return redact.StringWithoutMarkers(r) +} +// SafeFormat implements the redact.SafeFormatter interface. +func (r RangeDescriptor) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("r%d:", r.RangeID) if !r.IsInitialized() { - buf.WriteString("{-}") + w.SafeString("{-}") } else { - buf.WriteString(r.RSpan().String()) + w.Print(r.RSpan()) } - buf.WriteString(" [") + w.SafeString(" [") if allReplicas := r.Replicas().All(); len(allReplicas) > 0 { for i, rep := range allReplicas { if i > 0 { - buf.WriteString(", ") + w.SafeString(", ") } - buf.WriteString(rep.String()) + w.Print(rep) } } else { - buf.WriteString("") + w.SafeString("") } - fmt.Fprintf(&buf, ", next=%d, gen=%d", r.NextReplicaID, r.Generation) + w.Printf(", next=%d, gen=%d", r.NextReplicaID, r.Generation) if s := r.GetStickyBit(); !s.IsEmpty() { - fmt.Fprintf(&buf, ", sticky=%s", s) + w.Printf(", sticky=%s", s) } - buf.WriteString("]") - - return buf.String() + w.SafeString("]") } -// SafeMessage implements the SafeMessager interface. -// -// This method should be kept in sync with the String() method, except for the Start/End keys, which are customer data. -func (r RangeDescriptor) SafeMessage() string { - var buf bytes.Buffer - fmt.Fprintf(&buf, "r%d:", r.RangeID) - if !r.IsInitialized() { - buf.WriteString("{-}") - } - buf.WriteString(" [") - - if allReplicas := r.Replicas().All(); len(allReplicas) > 0 { - for i, rep := range allReplicas { - if i > 0 { - buf.WriteString(", ") - } - buf.WriteString(rep.SafeMessage()) - } - } else { - buf.WriteString("") - } - fmt.Fprintf(&buf, ", next=%d, gen=%d", r.NextReplicaID, r.Generation) - if s := r.GetStickyBit(); !s.IsEmpty() { - fmt.Fprintf(&buf, ", sticky=%s", s) - } - buf.WriteString("]") - - return buf.String() +func (r ReplicationTarget) String() string { + return redact.StringWithoutMarkers(r) } -func (r ReplicationTarget) String() string { - return fmt.Sprintf("n%d,s%d", r.NodeID, r.StoreID) +// SafeFormat implements the redact.SafeFormatter interface. +func (r ReplicationTarget) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("n%d,s%d", r.NodeID, r.StoreID) } func (r ReplicaDescriptor) String() string { - var buf bytes.Buffer - fmt.Fprintf(&buf, "(n%d,s%d):", r.NodeID, r.StoreID) - if r.ReplicaID == 0 { - buf.WriteString("?") - } else { - fmt.Fprintf(&buf, "%d", r.ReplicaID) - } - if typ := r.GetType(); typ != VOTER_FULL { - buf.WriteString(typ.String()) - } - return buf.String() + return redact.StringWithoutMarkers(r) } -// SafeMessage implements the SafeMessager interface. -// -// This method should be kept in sync with the String() method, while there is no customer data in the ReplicaDescriptor -// today, we maintain this method for future compatibility, since its used from other places -// such as RangeDescriptor#SafeMessage() -func (r ReplicaDescriptor) SafeMessage() string { - var buf bytes.Buffer - fmt.Fprintf(&buf, "(n%d,s%d):", r.NodeID, r.StoreID) +// SafeFormat implements the redact.SafeFormatter interface. +func (r ReplicaDescriptor) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("(n%d,s%d):", r.NodeID, r.StoreID) if r.ReplicaID == 0 { - buf.WriteString("?") + w.SafeRune('?') } else { - fmt.Fprintf(&buf, "%d", r.ReplicaID) + w.Print(r.ReplicaID) } if typ := r.GetType(); typ != VOTER_FULL { - buf.WriteString(typ.String()) + w.Print(typ) } - return buf.String() } // Validate performs some basic validation of the contents of a replica descriptor. @@ -399,6 +371,9 @@ func (r ReplicaDescriptor) GetType() ReplicaType { return *r.Type } +// SafeValue implements the redact.SafeValue interface. +func (r ReplicaType) SafeValue() {} + // PercentilesFromData derives percentiles from a slice of data points. // Sorts the input data if it isn't already sorted. func PercentilesFromData(data []float64) Percentiles { @@ -431,13 +406,23 @@ func percentileFromSortedData(data []float64, percent float64) float64 { // String returns a string representation of the Percentiles. func (p Percentiles) String() string { - return fmt.Sprintf("p10=%.2f p25=%.2f p50=%.2f p75=%.2f p90=%.2f pMax=%.2f", + return redact.StringWithoutMarkers(p) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (p Percentiles) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("p10=%.2f p25=%.2f p50=%.2f p75=%.2f p90=%.2f pMax=%.2f", p.P10, p.P25, p.P50, p.P75, p.P90, p.PMax) } // String returns a string representation of the StoreCapacity. func (sc StoreCapacity) String() string { - return fmt.Sprintf("disk (capacity=%s, available=%s, used=%s, logicalBytes=%s), "+ + return redact.StringWithoutMarkers(sc) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (sc StoreCapacity) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("disk (capacity=%s, available=%s, used=%s, logicalBytes=%s), "+ "ranges=%d, leases=%d, queries=%.2f, writes=%.2f, "+ "bytesPerReplica={%s}, writesPerReplica={%s}", humanizeutil.IBytes(sc.Capacity), humanizeutil.IBytes(sc.Available), diff --git a/pkg/roachpb/metadata_test.go b/pkg/roachpb/metadata_test.go index 4a7f52702cfb..e7357ddcb292 100644 --- a/pkg/roachpb/metadata_test.go +++ b/pkg/roachpb/metadata_test.go @@ -16,6 +16,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/util/redact" "github.com/cockroachdb/errors" ) @@ -89,9 +90,9 @@ func TestRangeDescriptorSafeMessage(t *testing.T) { }, } - const expStr = `r1: [(n1,s1):?, (n2,s2):?, (n3,s3):?, next=0, gen=0]` + const expStr = `r1:‹{c-g}› [(n1,s1):?, (n2,s2):?, (n3,s3):?, next=0, gen=0]` - if str := desc.SafeMessage(); str != expStr { + if str := redact.Sprint(desc); str != expStr { t.Errorf( "expected meta: %s\n"+ "got: %s", diff --git a/pkg/server/debug/logspy.go b/pkg/server/debug/logspy.go index 4434224f4c78..b0c1bd67f9b3 100644 --- a/pkg/server/debug/logspy.go +++ b/pkg/server/debug/logspy.go @@ -13,7 +13,6 @@ package debug import ( "context" "encoding/json" - "fmt" "io" "net/http" "net/url" @@ -22,7 +21,6 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -165,10 +163,9 @@ func (spy *logSpy) run(ctx context.Context, w io.Writer, opts logSpyOptions) (er defer func() { if err == nil { if dropped := atomic.LoadInt32(&countDropped); dropped > 0 { - f, l, _ := caller.Lookup(0) entry := log.MakeEntry( - log.Severity_WARNING, timeutil.Now().UnixNano(), f, l, - fmt.Sprintf("%d messages were dropped", dropped)) + ctx, log.Severity_WARNING, nil /* LogCounter */, 0 /* depth */, false, /* redactable */ + "%d messages were dropped", log.Safe(dropped)) err = entry.Format(w) // modify return value } } @@ -181,16 +178,16 @@ func (spy *logSpy) run(ctx context.Context, w io.Writer, opts logSpyOptions) (er entries := make(chan log.Entry, logSpyChanCap) { - f, l, _ := caller.Lookup(0) entry := log.MakeEntry( - log.Severity_INFO, timeutil.Now().UnixNano(), f, l, - fmt.Sprintf("intercepting logs with options %+v", opts)) + ctx, log.Severity_INFO, nil /* LogCounter */, 0 /* depth */, false, /* redactable */ + "intercepting logs with options %+v", opts) entries <- entry } spy.setIntercept(ctx, func(entry log.Entry) { if re := opts.Grep.re; re != nil { switch { + case re.MatchString(entry.Tags): case re.MatchString(entry.Message): case re.MatchString(entry.File): case opts.Grep.i != 0 && opts.Grep.i == entry.Goroutine: diff --git a/pkg/server/serverpb/status.pb.go b/pkg/server/serverpb/status.pb.go index ac311fcb22a1..af5149b1cd89 100644 --- a/pkg/server/serverpb/status.pb.go +++ b/pkg/server/serverpb/status.pb.go @@ -65,7 +65,7 @@ func (x StacksType) String() string { return proto.EnumName(StacksType_name, int32(x)) } func (StacksType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{0} + return fileDescriptor_status_731fb2d638c68f09, []int{0} } // Represents the type of file. @@ -92,7 +92,7 @@ func (x FileType) String() string { return proto.EnumName(FileType_name, int32(x)) } func (FileType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{1} + return fileDescriptor_status_731fb2d638c68f09, []int{1} } // We use an enum to allow reporting of client certs and potential others (eg: @@ -129,7 +129,7 @@ func (x CertificateDetails_CertificateType) String() string { return proto.EnumName(CertificateDetails_CertificateType_name, int32(x)) } func (CertificateDetails_CertificateType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{1, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{1, 0} } type ProfileRequest_Type int32 @@ -149,7 +149,7 @@ func (x ProfileRequest_Type) String() string { return proto.EnumName(ProfileRequest_Type_name, int32(x)) } func (ProfileRequest_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{36, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{36, 0} } // Enum for phase of execution. @@ -173,7 +173,7 @@ func (x ActiveQuery_Phase) String() string { return proto.EnumName(ActiveQuery_Phase_name, int32(x)) } func (ActiveQuery_Phase) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{44, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{44, 0} } type CertificatesRequest struct { @@ -186,7 +186,7 @@ func (m *CertificatesRequest) Reset() { *m = CertificatesRequest{} } func (m *CertificatesRequest) String() string { return proto.CompactTextString(m) } func (*CertificatesRequest) ProtoMessage() {} func (*CertificatesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{0} + return fileDescriptor_status_731fb2d638c68f09, []int{0} } func (m *CertificatesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -225,7 +225,7 @@ func (m *CertificateDetails) Reset() { *m = CertificateDetails{} } func (m *CertificateDetails) String() string { return proto.CompactTextString(m) } func (*CertificateDetails) ProtoMessage() {} func (*CertificateDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{1} + return fileDescriptor_status_731fb2d638c68f09, []int{1} } func (m *CertificateDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -266,7 +266,7 @@ func (m *CertificateDetails_Fields) Reset() { *m = CertificateDetails_Fi func (m *CertificateDetails_Fields) String() string { return proto.CompactTextString(m) } func (*CertificateDetails_Fields) ProtoMessage() {} func (*CertificateDetails_Fields) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{1, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{1, 0} } func (m *CertificateDetails_Fields) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -299,7 +299,7 @@ func (m *CertificatesResponse) Reset() { *m = CertificatesResponse{} } func (m *CertificatesResponse) String() string { return proto.CompactTextString(m) } func (*CertificatesResponse) ProtoMessage() {} func (*CertificatesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{2} + return fileDescriptor_status_731fb2d638c68f09, []int{2} } func (m *CertificatesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -336,7 +336,7 @@ func (m *DetailsRequest) Reset() { *m = DetailsRequest{} } func (m *DetailsRequest) String() string { return proto.CompactTextString(m) } func (*DetailsRequest) ProtoMessage() {} func (*DetailsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{3} + return fileDescriptor_status_731fb2d638c68f09, []int{3} } func (m *DetailsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -373,7 +373,7 @@ func (m *SystemInfo) Reset() { *m = SystemInfo{} } func (m *SystemInfo) String() string { return proto.CompactTextString(m) } func (*SystemInfo) ProtoMessage() {} func (*SystemInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{4} + return fileDescriptor_status_731fb2d638c68f09, []int{4} } func (m *SystemInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -410,7 +410,7 @@ func (m *DetailsResponse) Reset() { *m = DetailsResponse{} } func (m *DetailsResponse) String() string { return proto.CompactTextString(m) } func (*DetailsResponse) ProtoMessage() {} func (*DetailsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{5} + return fileDescriptor_status_731fb2d638c68f09, []int{5} } func (m *DetailsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -442,7 +442,7 @@ func (m *NodesRequest) Reset() { *m = NodesRequest{} } func (m *NodesRequest) String() string { return proto.CompactTextString(m) } func (*NodesRequest) ProtoMessage() {} func (*NodesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{6} + return fileDescriptor_status_731fb2d638c68f09, []int{6} } func (m *NodesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -476,7 +476,7 @@ func (m *NodesResponse) Reset() { *m = NodesResponse{} } func (m *NodesResponse) String() string { return proto.CompactTextString(m) } func (*NodesResponse) ProtoMessage() {} func (*NodesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{7} + return fileDescriptor_status_731fb2d638c68f09, []int{7} } func (m *NodesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -511,7 +511,7 @@ func (m *NodeRequest) Reset() { *m = NodeRequest{} } func (m *NodeRequest) String() string { return proto.CompactTextString(m) } func (*NodeRequest) ProtoMessage() {} func (*NodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{8} + return fileDescriptor_status_731fb2d638c68f09, []int{8} } func (m *NodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -556,7 +556,7 @@ func (m *RaftState) Reset() { *m = RaftState{} } func (m *RaftState) String() string { return proto.CompactTextString(m) } func (*RaftState) ProtoMessage() {} func (*RaftState) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{9} + return fileDescriptor_status_731fb2d638c68f09, []int{9} } func (m *RaftState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -593,7 +593,7 @@ func (m *RaftState_Progress) Reset() { *m = RaftState_Progress{} } func (m *RaftState_Progress) String() string { return proto.CompactTextString(m) } func (*RaftState_Progress) ProtoMessage() {} func (*RaftState_Progress) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{9, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{9, 0} } func (m *RaftState_Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -638,7 +638,7 @@ func (m *RangeProblems) Reset() { *m = RangeProblems{} } func (m *RangeProblems) String() string { return proto.CompactTextString(m) } func (*RangeProblems) ProtoMessage() {} func (*RangeProblems) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{10} + return fileDescriptor_status_731fb2d638c68f09, []int{10} } func (m *RangeProblems) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -674,7 +674,7 @@ func (m *RangeStatistics) Reset() { *m = RangeStatistics{} } func (m *RangeStatistics) String() string { return proto.CompactTextString(m) } func (*RangeStatistics) ProtoMessage() {} func (*RangeStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{11} + return fileDescriptor_status_731fb2d638c68f09, []int{11} } func (m *RangeStatistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -708,7 +708,7 @@ func (m *PrettySpan) Reset() { *m = PrettySpan{} } func (m *PrettySpan) String() string { return proto.CompactTextString(m) } func (*PrettySpan) ProtoMessage() {} func (*PrettySpan) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{12} + return fileDescriptor_status_731fb2d638c68f09, []int{12} } func (m *PrettySpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +754,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{13} + return fileDescriptor_status_731fb2d638c68f09, []int{13} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -790,7 +790,7 @@ func (m *RangesRequest) Reset() { *m = RangesRequest{} } func (m *RangesRequest) String() string { return proto.CompactTextString(m) } func (*RangesRequest) ProtoMessage() {} func (*RangesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{14} + return fileDescriptor_status_731fb2d638c68f09, []int{14} } func (m *RangesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -823,7 +823,7 @@ func (m *RangesResponse) Reset() { *m = RangesResponse{} } func (m *RangesResponse) String() string { return proto.CompactTextString(m) } func (*RangesResponse) ProtoMessage() {} func (*RangesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{15} + return fileDescriptor_status_731fb2d638c68f09, []int{15} } func (m *RangesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -858,7 +858,7 @@ func (m *GossipRequest) Reset() { *m = GossipRequest{} } func (m *GossipRequest) String() string { return proto.CompactTextString(m) } func (*GossipRequest) ProtoMessage() {} func (*GossipRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{16} + return fileDescriptor_status_731fb2d638c68f09, []int{16} } func (m *GossipRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -893,7 +893,7 @@ func (m *EngineStatsInfo) Reset() { *m = EngineStatsInfo{} } func (m *EngineStatsInfo) String() string { return proto.CompactTextString(m) } func (*EngineStatsInfo) ProtoMessage() {} func (*EngineStatsInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{17} + return fileDescriptor_status_731fb2d638c68f09, []int{17} } func (m *EngineStatsInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -928,7 +928,7 @@ func (m *EngineStatsRequest) Reset() { *m = EngineStatsRequest{} } func (m *EngineStatsRequest) String() string { return proto.CompactTextString(m) } func (*EngineStatsRequest) ProtoMessage() {} func (*EngineStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{18} + return fileDescriptor_status_731fb2d638c68f09, []int{18} } func (m *EngineStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -961,7 +961,7 @@ func (m *EngineStatsResponse) Reset() { *m = EngineStatsResponse{} } func (m *EngineStatsResponse) String() string { return proto.CompactTextString(m) } func (*EngineStatsResponse) ProtoMessage() {} func (*EngineStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{19} + return fileDescriptor_status_731fb2d638c68f09, []int{19} } func (m *EngineStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -995,7 +995,7 @@ func (m *TraceEvent) Reset() { *m = TraceEvent{} } func (m *TraceEvent) String() string { return proto.CompactTextString(m) } func (*TraceEvent) ProtoMessage() {} func (*TraceEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{20} + return fileDescriptor_status_731fb2d638c68f09, []int{20} } func (m *TraceEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1029,7 +1029,7 @@ func (m *AllocatorDryRun) Reset() { *m = AllocatorDryRun{} } func (m *AllocatorDryRun) String() string { return proto.CompactTextString(m) } func (*AllocatorDryRun) ProtoMessage() {} func (*AllocatorDryRun) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{21} + return fileDescriptor_status_731fb2d638c68f09, []int{21} } func (m *AllocatorDryRun) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1062,7 +1062,7 @@ func (m *AllocatorRangeRequest) Reset() { *m = AllocatorRangeRequest{} } func (m *AllocatorRangeRequest) String() string { return proto.CompactTextString(m) } func (*AllocatorRangeRequest) ProtoMessage() {} func (*AllocatorRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{22} + return fileDescriptor_status_731fb2d638c68f09, []int{22} } func (m *AllocatorRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1098,7 +1098,7 @@ func (m *AllocatorRangeResponse) Reset() { *m = AllocatorRangeResponse{} func (m *AllocatorRangeResponse) String() string { return proto.CompactTextString(m) } func (*AllocatorRangeResponse) ProtoMessage() {} func (*AllocatorRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{23} + return fileDescriptor_status_731fb2d638c68f09, []int{23} } func (m *AllocatorRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1132,7 +1132,7 @@ func (m *AllocatorRequest) Reset() { *m = AllocatorRequest{} } func (m *AllocatorRequest) String() string { return proto.CompactTextString(m) } func (*AllocatorRequest) ProtoMessage() {} func (*AllocatorRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{24} + return fileDescriptor_status_731fb2d638c68f09, []int{24} } func (m *AllocatorRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1165,7 +1165,7 @@ func (m *AllocatorResponse) Reset() { *m = AllocatorResponse{} } func (m *AllocatorResponse) String() string { return proto.CompactTextString(m) } func (*AllocatorResponse) ProtoMessage() {} func (*AllocatorResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{25} + return fileDescriptor_status_731fb2d638c68f09, []int{25} } func (m *AllocatorResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1198,7 +1198,7 @@ func (m *JSONResponse) Reset() { *m = JSONResponse{} } func (m *JSONResponse) String() string { return proto.CompactTextString(m) } func (*JSONResponse) ProtoMessage() {} func (*JSONResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{26} + return fileDescriptor_status_731fb2d638c68f09, []int{26} } func (m *JSONResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1232,13 +1232,25 @@ type LogsRequest struct { EndTime string `protobuf:"bytes,4,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` Max string `protobuf:"bytes,5,opt,name=max,proto3" json:"max,omitempty"` Pattern string `protobuf:"bytes,6,opt,name=pattern,proto3" json:"pattern,omitempty"` + // redact, if true, requests redaction of sensitive data away + // from the retrieved log entries. + // Only admin users can send a request with redact = false. + Redact bool `protobuf:"varint,7,opt,name=redact,proto3" json:"redact,omitempty"` + // keep_redactable, if true, requests that retrieved entries preserve + // the redaction markers if any were present in the log files. + // If false, redaction markers are stripped away. + // Note that redact = false && redactable = false implies + // "flat" entries with all sensitive information enclosed and + // no markers; this is suitable for backward-compatibility with + // RPC clients from prior the introduction of redactable logs. + KeepRedactable bool `protobuf:"varint,8,opt,name=keep_redactable,json=keepRedactable,proto3" json:"keep_redactable,omitempty"` } func (m *LogsRequest) Reset() { *m = LogsRequest{} } func (m *LogsRequest) String() string { return proto.CompactTextString(m) } func (*LogsRequest) ProtoMessage() {} func (*LogsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{27} + return fileDescriptor_status_731fb2d638c68f09, []int{27} } func (m *LogsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1271,7 +1283,7 @@ func (m *LogEntriesResponse) Reset() { *m = LogEntriesResponse{} } func (m *LogEntriesResponse) String() string { return proto.CompactTextString(m) } func (*LogEntriesResponse) ProtoMessage() {} func (*LogEntriesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{28} + return fileDescriptor_status_731fb2d638c68f09, []int{28} } func (m *LogEntriesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1306,7 +1318,7 @@ func (m *LogFilesListRequest) Reset() { *m = LogFilesListRequest{} } func (m *LogFilesListRequest) String() string { return proto.CompactTextString(m) } func (*LogFilesListRequest) ProtoMessage() {} func (*LogFilesListRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{29} + return fileDescriptor_status_731fb2d638c68f09, []int{29} } func (m *LogFilesListRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1339,7 +1351,7 @@ func (m *LogFilesListResponse) Reset() { *m = LogFilesListResponse{} } func (m *LogFilesListResponse) String() string { return proto.CompactTextString(m) } func (*LogFilesListResponse) ProtoMessage() {} func (*LogFilesListResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{30} + return fileDescriptor_status_731fb2d638c68f09, []int{30} } func (m *LogFilesListResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1369,13 +1381,25 @@ type LogFileRequest struct { // forwarding is necessary. NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` File string `protobuf:"bytes,2,opt,name=file,proto3" json:"file,omitempty"` + // redact, if true, requests redaction of sensitive data away + // from the retrieved log entries. + // Only admin users can send a request with redact = false. + Redact bool `protobuf:"varint,3,opt,name=redact,proto3" json:"redact,omitempty"` + // keep_redactable, if true, requests that retrieved entries preserve + // the redaction markers if any were present in the log files. + // If false, redaction markers are stripped away. + // Note that redact = false && redactable = false implies + // "flat" entries with all sensitive information enclosed and + // no markers; this is suitable for backward-compatibility with + // RPC clients from prior the introduction of redactable logs. + KeepRedactable bool `protobuf:"varint,4,opt,name=keep_redactable,json=keepRedactable,proto3" json:"keep_redactable,omitempty"` } func (m *LogFileRequest) Reset() { *m = LogFileRequest{} } func (m *LogFileRequest) String() string { return proto.CompactTextString(m) } func (*LogFileRequest) ProtoMessage() {} func (*LogFileRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{31} + return fileDescriptor_status_731fb2d638c68f09, []int{31} } func (m *LogFileRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1411,7 +1435,7 @@ func (m *StacksRequest) Reset() { *m = StacksRequest{} } func (m *StacksRequest) String() string { return proto.CompactTextString(m) } func (*StacksRequest) ProtoMessage() {} func (*StacksRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{32} + return fileDescriptor_status_731fb2d638c68f09, []int{32} } func (m *StacksRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1447,7 +1471,7 @@ func (m *File) Reset() { *m = File{} } func (m *File) String() string { return proto.CompactTextString(m) } func (*File) ProtoMessage() {} func (*File) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{33} + return fileDescriptor_status_731fb2d638c68f09, []int{33} } func (m *File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1491,7 +1515,7 @@ func (m *GetFilesRequest) Reset() { *m = GetFilesRequest{} } func (m *GetFilesRequest) String() string { return proto.CompactTextString(m) } func (*GetFilesRequest) ProtoMessage() {} func (*GetFilesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{34} + return fileDescriptor_status_731fb2d638c68f09, []int{34} } func (m *GetFilesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1524,7 +1548,7 @@ func (m *GetFilesResponse) Reset() { *m = GetFilesResponse{} } func (m *GetFilesResponse) String() string { return proto.CompactTextString(m) } func (*GetFilesResponse) ProtoMessage() {} func (*GetFilesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{35} + return fileDescriptor_status_731fb2d638c68f09, []int{35} } func (m *GetFilesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1561,7 +1585,7 @@ func (m *ProfileRequest) Reset() { *m = ProfileRequest{} } func (m *ProfileRequest) String() string { return proto.CompactTextString(m) } func (*ProfileRequest) ProtoMessage() {} func (*ProfileRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{36} + return fileDescriptor_status_731fb2d638c68f09, []int{36} } func (m *ProfileRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1596,7 +1620,7 @@ func (m *MetricsRequest) Reset() { *m = MetricsRequest{} } func (m *MetricsRequest) String() string { return proto.CompactTextString(m) } func (*MetricsRequest) ProtoMessage() {} func (*MetricsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{37} + return fileDescriptor_status_731fb2d638c68f09, []int{37} } func (m *MetricsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1630,7 +1654,7 @@ func (m *RaftRangeNode) Reset() { *m = RaftRangeNode{} } func (m *RaftRangeNode) String() string { return proto.CompactTextString(m) } func (*RaftRangeNode) ProtoMessage() {} func (*RaftRangeNode) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{38} + return fileDescriptor_status_731fb2d638c68f09, []int{38} } func (m *RaftRangeNode) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1663,7 +1687,7 @@ func (m *RaftRangeError) Reset() { *m = RaftRangeError{} } func (m *RaftRangeError) String() string { return proto.CompactTextString(m) } func (*RaftRangeError) ProtoMessage() {} func (*RaftRangeError) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{39} + return fileDescriptor_status_731fb2d638c68f09, []int{39} } func (m *RaftRangeError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1698,7 +1722,7 @@ func (m *RaftRangeStatus) Reset() { *m = RaftRangeStatus{} } func (m *RaftRangeStatus) String() string { return proto.CompactTextString(m) } func (*RaftRangeStatus) ProtoMessage() {} func (*RaftRangeStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{40} + return fileDescriptor_status_731fb2d638c68f09, []int{40} } func (m *RaftRangeStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1731,7 +1755,7 @@ func (m *RaftDebugRequest) Reset() { *m = RaftDebugRequest{} } func (m *RaftDebugRequest) String() string { return proto.CompactTextString(m) } func (*RaftDebugRequest) ProtoMessage() {} func (*RaftDebugRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{41} + return fileDescriptor_status_731fb2d638c68f09, []int{41} } func (m *RaftDebugRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1765,7 +1789,7 @@ func (m *RaftDebugResponse) Reset() { *m = RaftDebugResponse{} } func (m *RaftDebugResponse) String() string { return proto.CompactTextString(m) } func (*RaftDebugResponse) ProtoMessage() {} func (*RaftDebugResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{42} + return fileDescriptor_status_731fb2d638c68f09, []int{42} } func (m *RaftDebugResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1804,7 +1828,7 @@ func (m *TxnInfo) Reset() { *m = TxnInfo{} } func (m *TxnInfo) String() string { return proto.CompactTextString(m) } func (*TxnInfo) ProtoMessage() {} func (*TxnInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{43} + return fileDescriptor_status_731fb2d638c68f09, []int{43} } func (m *TxnInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1850,7 +1874,7 @@ func (m *ActiveQuery) Reset() { *m = ActiveQuery{} } func (m *ActiveQuery) String() string { return proto.CompactTextString(m) } func (*ActiveQuery) ProtoMessage() {} func (*ActiveQuery) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{44} + return fileDescriptor_status_731fb2d638c68f09, []int{44} } func (m *ActiveQuery) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1885,7 +1909,7 @@ func (m *ListSessionsRequest) Reset() { *m = ListSessionsRequest{} } func (m *ListSessionsRequest) String() string { return proto.CompactTextString(m) } func (*ListSessionsRequest) ProtoMessage() {} func (*ListSessionsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{45} + return fileDescriptor_status_731fb2d638c68f09, []int{45} } func (m *ListSessionsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1944,7 +1968,7 @@ func (m *Session) Reset() { *m = Session{} } func (m *Session) String() string { return proto.CompactTextString(m) } func (*Session) ProtoMessage() {} func (*Session) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{46} + return fileDescriptor_status_731fb2d638c68f09, []int{46} } func (m *Session) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1981,7 +2005,7 @@ func (m *ListSessionsError) Reset() { *m = ListSessionsError{} } func (m *ListSessionsError) String() string { return proto.CompactTextString(m) } func (*ListSessionsError) ProtoMessage() {} func (*ListSessionsError) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{47} + return fileDescriptor_status_731fb2d638c68f09, []int{47} } func (m *ListSessionsError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2018,7 +2042,7 @@ func (m *ListSessionsResponse) Reset() { *m = ListSessionsResponse{} } func (m *ListSessionsResponse) String() string { return proto.CompactTextString(m) } func (*ListSessionsResponse) ProtoMessage() {} func (*ListSessionsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{48} + return fileDescriptor_status_731fb2d638c68f09, []int{48} } func (m *ListSessionsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2063,7 +2087,7 @@ func (m *CancelQueryRequest) Reset() { *m = CancelQueryRequest{} } func (m *CancelQueryRequest) String() string { return proto.CompactTextString(m) } func (*CancelQueryRequest) ProtoMessage() {} func (*CancelQueryRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{49} + return fileDescriptor_status_731fb2d638c68f09, []int{49} } func (m *CancelQueryRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2100,7 +2124,7 @@ func (m *CancelQueryResponse) Reset() { *m = CancelQueryResponse{} } func (m *CancelQueryResponse) String() string { return proto.CompactTextString(m) } func (*CancelQueryResponse) ProtoMessage() {} func (*CancelQueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{50} + return fileDescriptor_status_731fb2d638c68f09, []int{50} } func (m *CancelQueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2140,7 +2164,7 @@ func (m *CancelSessionRequest) Reset() { *m = CancelSessionRequest{} } func (m *CancelSessionRequest) String() string { return proto.CompactTextString(m) } func (*CancelSessionRequest) ProtoMessage() {} func (*CancelSessionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{51} + return fileDescriptor_status_731fb2d638c68f09, []int{51} } func (m *CancelSessionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2174,7 +2198,7 @@ func (m *CancelSessionResponse) Reset() { *m = CancelSessionResponse{} } func (m *CancelSessionResponse) String() string { return proto.CompactTextString(m) } func (*CancelSessionResponse) ProtoMessage() {} func (*CancelSessionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{52} + return fileDescriptor_status_731fb2d638c68f09, []int{52} } func (m *CancelSessionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2209,7 +2233,7 @@ func (m *SpanStatsRequest) Reset() { *m = SpanStatsRequest{} } func (m *SpanStatsRequest) String() string { return proto.CompactTextString(m) } func (*SpanStatsRequest) ProtoMessage() {} func (*SpanStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{53} + return fileDescriptor_status_731fb2d638c68f09, []int{53} } func (m *SpanStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2244,7 +2268,7 @@ func (m *SpanStatsResponse) Reset() { *m = SpanStatsResponse{} } func (m *SpanStatsResponse) String() string { return proto.CompactTextString(m) } func (*SpanStatsResponse) ProtoMessage() {} func (*SpanStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{54} + return fileDescriptor_status_731fb2d638c68f09, []int{54} } func (m *SpanStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2278,7 +2302,7 @@ func (m *ProblemRangesRequest) Reset() { *m = ProblemRangesRequest{} } func (m *ProblemRangesRequest) String() string { return proto.CompactTextString(m) } func (*ProblemRangesRequest) ProtoMessage() {} func (*ProblemRangesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{55} + return fileDescriptor_status_731fb2d638c68f09, []int{55} } func (m *ProblemRangesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2313,7 +2337,7 @@ func (m *ProblemRangesResponse) Reset() { *m = ProblemRangesResponse{} } func (m *ProblemRangesResponse) String() string { return proto.CompactTextString(m) } func (*ProblemRangesResponse) ProtoMessage() {} func (*ProblemRangesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{56} + return fileDescriptor_status_731fb2d638c68f09, []int{56} } func (m *ProblemRangesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2354,7 +2378,7 @@ func (m *ProblemRangesResponse_NodeProblems) Reset() { *m = ProblemRange func (m *ProblemRangesResponse_NodeProblems) String() string { return proto.CompactTextString(m) } func (*ProblemRangesResponse_NodeProblems) ProtoMessage() {} func (*ProblemRangesResponse_NodeProblems) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{56, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{56, 0} } func (m *ProblemRangesResponse_NodeProblems) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2388,7 +2412,7 @@ func (m *HotRangesRequest) Reset() { *m = HotRangesRequest{} } func (m *HotRangesRequest) String() string { return proto.CompactTextString(m) } func (*HotRangesRequest) ProtoMessage() {} func (*HotRangesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{57} + return fileDescriptor_status_731fb2d638c68f09, []int{57} } func (m *HotRangesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2423,7 +2447,7 @@ func (m *HotRangesResponse) Reset() { *m = HotRangesResponse{} } func (m *HotRangesResponse) String() string { return proto.CompactTextString(m) } func (*HotRangesResponse) ProtoMessage() {} func (*HotRangesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{58} + return fileDescriptor_status_731fb2d638c68f09, []int{58} } func (m *HotRangesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2457,7 +2481,7 @@ func (m *HotRangesResponse_HotRange) Reset() { *m = HotRangesResponse_Ho func (m *HotRangesResponse_HotRange) String() string { return proto.CompactTextString(m) } func (*HotRangesResponse_HotRange) ProtoMessage() {} func (*HotRangesResponse_HotRange) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{58, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{58, 0} } func (m *HotRangesResponse_HotRange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2491,7 +2515,7 @@ func (m *HotRangesResponse_StoreResponse) Reset() { *m = HotRangesRespon func (m *HotRangesResponse_StoreResponse) String() string { return proto.CompactTextString(m) } func (*HotRangesResponse_StoreResponse) ProtoMessage() {} func (*HotRangesResponse_StoreResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{58, 1} + return fileDescriptor_status_731fb2d638c68f09, []int{58, 1} } func (m *HotRangesResponse_StoreResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2525,7 +2549,7 @@ func (m *HotRangesResponse_NodeResponse) Reset() { *m = HotRangesRespons func (m *HotRangesResponse_NodeResponse) String() string { return proto.CompactTextString(m) } func (*HotRangesResponse_NodeResponse) ProtoMessage() {} func (*HotRangesResponse_NodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{58, 2} + return fileDescriptor_status_731fb2d638c68f09, []int{58, 2} } func (m *HotRangesResponse_NodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2558,7 +2582,7 @@ func (m *RangeRequest) Reset() { *m = RangeRequest{} } func (m *RangeRequest) String() string { return proto.CompactTextString(m) } func (*RangeRequest) ProtoMessage() {} func (*RangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{59} + return fileDescriptor_status_731fb2d638c68f09, []int{59} } func (m *RangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2594,7 +2618,7 @@ func (m *RangeResponse) Reset() { *m = RangeResponse{} } func (m *RangeResponse) String() string { return proto.CompactTextString(m) } func (*RangeResponse) ProtoMessage() {} func (*RangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{60} + return fileDescriptor_status_731fb2d638c68f09, []int{60} } func (m *RangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2629,7 +2653,7 @@ func (m *RangeResponse_NodeResponse) Reset() { *m = RangeResponse_NodeRe func (m *RangeResponse_NodeResponse) String() string { return proto.CompactTextString(m) } func (*RangeResponse_NodeResponse) ProtoMessage() {} func (*RangeResponse_NodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{60, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{60, 0} } func (m *RangeResponse_NodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2665,7 +2689,7 @@ func (m *DiagnosticsRequest) Reset() { *m = DiagnosticsRequest{} } func (m *DiagnosticsRequest) String() string { return proto.CompactTextString(m) } func (*DiagnosticsRequest) ProtoMessage() {} func (*DiagnosticsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{61} + return fileDescriptor_status_731fb2d638c68f09, []int{61} } func (m *DiagnosticsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2700,7 +2724,7 @@ func (m *StoresRequest) Reset() { *m = StoresRequest{} } func (m *StoresRequest) String() string { return proto.CompactTextString(m) } func (*StoresRequest) ProtoMessage() {} func (*StoresRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{62} + return fileDescriptor_status_731fb2d638c68f09, []int{62} } func (m *StoresRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2743,7 +2767,7 @@ func (m *StoreDetails) Reset() { *m = StoreDetails{} } func (m *StoreDetails) String() string { return proto.CompactTextString(m) } func (*StoreDetails) ProtoMessage() {} func (*StoreDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{63} + return fileDescriptor_status_731fb2d638c68f09, []int{63} } func (m *StoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2776,7 +2800,7 @@ func (m *StoresResponse) Reset() { *m = StoresResponse{} } func (m *StoresResponse) String() string { return proto.CompactTextString(m) } func (*StoresResponse) ProtoMessage() {} func (*StoresResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{64} + return fileDescriptor_status_731fb2d638c68f09, []int{64} } func (m *StoresResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2809,7 +2833,7 @@ func (m *StatementsRequest) Reset() { *m = StatementsRequest{} } func (m *StatementsRequest) String() string { return proto.CompactTextString(m) } func (*StatementsRequest) ProtoMessage() {} func (*StatementsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{65} + return fileDescriptor_status_731fb2d638c68f09, []int{65} } func (m *StatementsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2847,7 +2871,7 @@ func (m *StatementsResponse) Reset() { *m = StatementsResponse{} } func (m *StatementsResponse) String() string { return proto.CompactTextString(m) } func (*StatementsResponse) ProtoMessage() {} func (*StatementsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{66} + return fileDescriptor_status_731fb2d638c68f09, []int{66} } func (m *StatementsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2885,7 +2909,7 @@ func (m *StatementsResponse_ExtendedStatementStatisticsKey) String() string { } func (*StatementsResponse_ExtendedStatementStatisticsKey) ProtoMessage() {} func (*StatementsResponse_ExtendedStatementStatisticsKey) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{66, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{66, 0} } func (m *StatementsResponse_ExtendedStatementStatisticsKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2923,7 +2947,7 @@ func (m *StatementsResponse_CollectedStatementStatistics) String() string { } func (*StatementsResponse_CollectedStatementStatistics) ProtoMessage() {} func (*StatementsResponse_CollectedStatementStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{66, 1} + return fileDescriptor_status_731fb2d638c68f09, []int{66, 1} } func (m *StatementsResponse_CollectedStatementStatistics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2960,7 +2984,7 @@ func (m *StatementDiagnosticsReport) Reset() { *m = StatementDiagnostics func (m *StatementDiagnosticsReport) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsReport) ProtoMessage() {} func (*StatementDiagnosticsReport) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{67} + return fileDescriptor_status_731fb2d638c68f09, []int{67} } func (m *StatementDiagnosticsReport) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2995,7 +3019,7 @@ func (m *CreateStatementDiagnosticsReportRequest) Reset() { func (m *CreateStatementDiagnosticsReportRequest) String() string { return proto.CompactTextString(m) } func (*CreateStatementDiagnosticsReportRequest) ProtoMessage() {} func (*CreateStatementDiagnosticsReportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{68} + return fileDescriptor_status_731fb2d638c68f09, []int{68} } func (m *CreateStatementDiagnosticsReportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3030,7 +3054,7 @@ func (m *CreateStatementDiagnosticsReportResponse) Reset() { func (m *CreateStatementDiagnosticsReportResponse) String() string { return proto.CompactTextString(m) } func (*CreateStatementDiagnosticsReportResponse) ProtoMessage() {} func (*CreateStatementDiagnosticsReportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{69} + return fileDescriptor_status_731fb2d638c68f09, []int{69} } func (m *CreateStatementDiagnosticsReportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3062,7 +3086,7 @@ func (m *StatementDiagnosticsReportsRequest) Reset() { *m = StatementDia func (m *StatementDiagnosticsReportsRequest) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsReportsRequest) ProtoMessage() {} func (*StatementDiagnosticsReportsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{70} + return fileDescriptor_status_731fb2d638c68f09, []int{70} } func (m *StatementDiagnosticsReportsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3095,7 +3119,7 @@ func (m *StatementDiagnosticsReportsResponse) Reset() { *m = StatementDi func (m *StatementDiagnosticsReportsResponse) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsReportsResponse) ProtoMessage() {} func (*StatementDiagnosticsReportsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{71} + return fileDescriptor_status_731fb2d638c68f09, []int{71} } func (m *StatementDiagnosticsReportsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3131,7 +3155,7 @@ func (m *StatementDiagnostics) Reset() { *m = StatementDiagnostics{} } func (m *StatementDiagnostics) String() string { return proto.CompactTextString(m) } func (*StatementDiagnostics) ProtoMessage() {} func (*StatementDiagnostics) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{72} + return fileDescriptor_status_731fb2d638c68f09, []int{72} } func (m *StatementDiagnostics) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3164,7 +3188,7 @@ func (m *StatementDiagnosticsRequest) Reset() { *m = StatementDiagnostic func (m *StatementDiagnosticsRequest) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsRequest) ProtoMessage() {} func (*StatementDiagnosticsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{73} + return fileDescriptor_status_731fb2d638c68f09, []int{73} } func (m *StatementDiagnosticsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3197,7 +3221,7 @@ func (m *StatementDiagnosticsResponse) Reset() { *m = StatementDiagnosti func (m *StatementDiagnosticsResponse) String() string { return proto.CompactTextString(m) } func (*StatementDiagnosticsResponse) ProtoMessage() {} func (*StatementDiagnosticsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{74} + return fileDescriptor_status_731fb2d638c68f09, []int{74} } func (m *StatementDiagnosticsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3230,7 +3254,7 @@ func (m *JobRegistryStatusRequest) Reset() { *m = JobRegistryStatusReque func (m *JobRegistryStatusRequest) String() string { return proto.CompactTextString(m) } func (*JobRegistryStatusRequest) ProtoMessage() {} func (*JobRegistryStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{75} + return fileDescriptor_status_731fb2d638c68f09, []int{75} } func (m *JobRegistryStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3264,7 +3288,7 @@ func (m *JobRegistryStatusResponse) Reset() { *m = JobRegistryStatusResp func (m *JobRegistryStatusResponse) String() string { return proto.CompactTextString(m) } func (*JobRegistryStatusResponse) ProtoMessage() {} func (*JobRegistryStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{76} + return fileDescriptor_status_731fb2d638c68f09, []int{76} } func (m *JobRegistryStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3297,7 +3321,7 @@ func (m *JobRegistryStatusResponse_Job) Reset() { *m = JobRegistryStatus func (m *JobRegistryStatusResponse_Job) String() string { return proto.CompactTextString(m) } func (*JobRegistryStatusResponse_Job) ProtoMessage() {} func (*JobRegistryStatusResponse_Job) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{76, 0} + return fileDescriptor_status_731fb2d638c68f09, []int{76, 0} } func (m *JobRegistryStatusResponse_Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3330,7 +3354,7 @@ func (m *JobStatusRequest) Reset() { *m = JobStatusRequest{} } func (m *JobStatusRequest) String() string { return proto.CompactTextString(m) } func (*JobStatusRequest) ProtoMessage() {} func (*JobStatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{77} + return fileDescriptor_status_731fb2d638c68f09, []int{77} } func (m *JobStatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3363,7 +3387,7 @@ func (m *JobStatusResponse) Reset() { *m = JobStatusResponse{} } func (m *JobStatusResponse) String() string { return proto.CompactTextString(m) } func (*JobStatusResponse) ProtoMessage() {} func (*JobStatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_status_fee345378c170d2b, []int{78} + return fileDescriptor_status_731fb2d638c68f09, []int{78} } func (m *JobStatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5932,6 +5956,26 @@ func (m *LogsRequest) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintStatus(dAtA, i, uint64(len(m.Pattern))) i += copy(dAtA[i:], m.Pattern) } + if m.Redact { + dAtA[i] = 0x38 + i++ + if m.Redact { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.KeepRedactable { + dAtA[i] = 0x40 + i++ + if m.KeepRedactable { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -6046,6 +6090,26 @@ func (m *LogFileRequest) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintStatus(dAtA, i, uint64(len(m.File))) i += copy(dAtA[i:], m.File) } + if m.Redact { + dAtA[i] = 0x18 + i++ + if m.Redact { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.KeepRedactable { + dAtA[i] = 0x20 + i++ + if m.KeepRedactable { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -8770,6 +8834,12 @@ func (m *LogsRequest) Size() (n int) { if l > 0 { n += 1 + l + sovStatus(uint64(l)) } + if m.Redact { + n += 2 + } + if m.KeepRedactable { + n += 2 + } return n } @@ -8830,6 +8900,12 @@ func (m *LogFileRequest) Size() (n int) { if l > 0 { n += 1 + l + sovStatus(uint64(l)) } + if m.Redact { + n += 2 + } + if m.KeepRedactable { + n += 2 + } return n } @@ -13924,6 +14000,46 @@ func (m *LogsRequest) Unmarshal(dAtA []byte) error { } m.Pattern = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Redact", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Redact = bool(v != 0) + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeepRedactable", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.KeepRedactable = bool(v != 0) default: iNdEx = preIndex skippy, err := skipStatus(dAtA[iNdEx:]) @@ -14273,6 +14389,46 @@ func (m *LogFileRequest) Unmarshal(dAtA []byte) error { } m.File = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Redact", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Redact = bool(v != 0) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeepRedactable", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.KeepRedactable = bool(v != 0) default: iNdEx = preIndex skippy, err := skipStatus(dAtA[iNdEx:]) @@ -21311,374 +21467,376 @@ var ( ) func init() { - proto.RegisterFile("server/serverpb/status.proto", fileDescriptor_status_fee345378c170d2b) -} - -var fileDescriptor_status_fee345378c170d2b = []byte{ - // 5831 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x7c, 0xdb, 0x6f, 0x1c, 0xc9, - 0x75, 0xb7, 0x7a, 0x6e, 0x9c, 0x39, 0xc3, 0xcb, 0xb0, 0x44, 0x52, 0xa3, 0x91, 0x96, 0x23, 0xb7, - 0x76, 0x75, 0xdb, 0xdd, 0x99, 0x5d, 0xed, 0xca, 0x2b, 0xef, 0xe7, 0x5d, 0x9b, 0x37, 0x49, 0x94, - 0xb8, 0xba, 0x34, 0xa9, 0xcf, 0x1f, 0xd6, 0xfe, 0xb6, 0xbf, 0x9e, 0xe9, 0xe2, 0xa8, 0xc5, 0x61, - 0xf7, 0xa8, 0xbb, 0x87, 0x1f, 0xc7, 0x1b, 0xd9, 0xce, 0xe6, 0xe6, 0x38, 0x8e, 0x6f, 0x71, 0x02, - 0x3f, 0x24, 0x40, 0xe0, 0x87, 0x38, 0x2f, 0x09, 0x1c, 0xe4, 0x25, 0x09, 0x90, 0x04, 0xb9, 0x20, - 0x31, 0x10, 0x20, 0x30, 0x90, 0x3c, 0x18, 0x09, 0x40, 0x27, 0x74, 0x1e, 0x02, 0xe4, 0x3f, 0x30, - 0x90, 0x20, 0xa8, 0x53, 0xd5, 0x3d, 0xd5, 0x33, 0xc3, 0x9e, 0x21, 0xb9, 0x5a, 0xe4, 0x61, 0x57, - 0xd3, 0x55, 0x75, 0x4e, 0xfd, 0xea, 0xd4, 0xa9, 0x53, 0xa7, 0x4e, 0x9d, 0x22, 0x9c, 0xf5, 0xa8, - 0xbb, 0x43, 0xdd, 0x2a, 0xff, 0xa7, 0x55, 0xab, 0x7a, 0xbe, 0xe1, 0xb7, 0xbd, 0x4a, 0xcb, 0x75, - 0x7c, 0x87, 0x9c, 0xae, 0x3b, 0xf5, 0x2d, 0xd7, 0x31, 0xea, 0x8f, 0x2a, 0xbc, 0x41, 0x25, 0x68, - 0x57, 0x2a, 0xd4, 0xda, 0x56, 0xd3, 0xac, 0x5a, 0xf6, 0xa6, 0xc3, 0x1b, 0x97, 0x4e, 0x36, 0x1c, - 0xcf, 0xb3, 0x5a, 0x55, 0xfe, 0x8f, 0x28, 0x9c, 0x7b, 0xec, 0xd4, 0xbc, 0x2a, 0xfb, 0x5f, 0xab, - 0x86, 0xff, 0x88, 0xf2, 0x53, 0xc8, 0xb5, 0x55, 0xab, 0x1a, 0xad, 0x96, 0xce, 0xfa, 0x0c, 0x2a, - 0x48, 0x50, 0x61, 0x1a, 0xbe, 0x11, 0x30, 0x09, 0xca, 0xb6, 0xa9, 0x6f, 0x48, 0xe5, 0x17, 0x04, - 0x78, 0xd3, 0x32, 0x1a, 0xb6, 0xe3, 0xf9, 0x56, 0x9d, 0xf5, 0x22, 0x7d, 0x89, 0x76, 0xe7, 0x83, - 0x41, 0xe2, 0xd8, 0xc4, 0x3f, 0x3d, 0x63, 0x2d, 0x3d, 0xe7, 0xf9, 0x8e, 0x6b, 0x34, 0x68, 0x95, - 0xda, 0x0d, 0xcb, 0xa6, 0xad, 0x9a, 0xf8, 0x21, 0xaa, 0xcf, 0xf4, 0x55, 0x6f, 0xef, 0xd4, 0xeb, - 0xa2, 0x72, 0xbe, 0xaf, 0xd2, 0x75, 0xea, 0x5b, 0x9e, 0x59, 0x13, 0xf5, 0x97, 0xb7, 0x76, 0xaa, - 0x5b, 0x3b, 0x02, 0x45, 0xf0, 0xa3, 0x55, 0xab, 0x36, 0xa9, 0xe1, 0x51, 0x3d, 0x02, 0x43, 0x3d, - 0xa0, 0x29, 0x6b, 0x14, 0x60, 0x79, 0xe1, 0x20, 0x76, 0xd6, 0x0e, 0xb5, 0xa9, 0x17, 0x8a, 0xb2, - 0xed, 0x5b, 0xcd, 0x6a, 0xd3, 0x69, 0xb0, 0xff, 0x44, 0x59, 0x09, 0xcb, 0xda, 0xb6, 0x4b, 0x3d, - 0xa7, 0xb9, 0x43, 0x4d, 0xdd, 0x30, 0x4d, 0x37, 0x18, 0x22, 0xf5, 0xeb, 0x66, 0xd5, 0x35, 0x36, - 0x7d, 0xfc, 0x1f, 0x1b, 0x85, 0xb1, 0xe9, 0x8b, 0xca, 0x99, 0x86, 0xd3, 0x70, 0xf0, 0x67, 0x95, - 0xfd, 0x12, 0xa5, 0x67, 0x1b, 0x8e, 0xd3, 0x68, 0xd2, 0xaa, 0xd1, 0xb2, 0xaa, 0x86, 0x6d, 0x3b, - 0xbe, 0xe1, 0x5b, 0x8e, 0x1d, 0x00, 0x28, 0x8b, 0x5a, 0xfc, 0xaa, 0xb5, 0x37, 0xab, 0xbe, 0xb5, - 0x4d, 0x3d, 0xdf, 0xd8, 0x16, 0xda, 0xa1, 0x56, 0xe0, 0xe4, 0x12, 0x75, 0x7d, 0x6b, 0xd3, 0xaa, - 0x1b, 0x3e, 0xf5, 0x34, 0xfa, 0xa4, 0x4d, 0x3d, 0x9f, 0x9c, 0x82, 0x31, 0xdb, 0x31, 0xa9, 0x6e, - 0x99, 0x45, 0xe5, 0x9c, 0x72, 0x29, 0xa7, 0x65, 0xd8, 0xe7, 0xaa, 0xa9, 0xfe, 0x67, 0x0a, 0x88, - 0x44, 0xb0, 0x4c, 0x7d, 0xc3, 0x6a, 0x7a, 0xe4, 0x01, 0xa4, 0xfc, 0x4e, 0x8b, 0x62, 0xe3, 0xc9, - 0xab, 0x6f, 0x55, 0x0e, 0xd4, 0xda, 0x4a, 0x3f, 0xb1, 0x5c, 0xb4, 0xd1, 0x69, 0x51, 0x0d, 0x59, - 0x91, 0xf3, 0x30, 0x41, 0x5d, 0xd7, 0x71, 0xf5, 0x6d, 0xea, 0x79, 0x46, 0x83, 0x16, 0x13, 0x08, - 0x64, 0x1c, 0x0b, 0xdf, 0xe1, 0x65, 0x84, 0x40, 0x8a, 0x69, 0x63, 0x31, 0x79, 0x4e, 0xb9, 0x34, - 0xae, 0xe1, 0x6f, 0xa2, 0x41, 0x66, 0xd3, 0xa2, 0x4d, 0xd3, 0x2b, 0xa6, 0xce, 0x25, 0x2f, 0xe5, - 0xaf, 0xbe, 0x7e, 0x38, 0x34, 0x37, 0x90, 0x76, 0x31, 0xf5, 0x83, 0xbd, 0xf2, 0x09, 0x4d, 0x70, - 0x2a, 0xfd, 0x61, 0x02, 0x32, 0xbc, 0x82, 0xcc, 0x41, 0xc6, 0xf2, 0xbc, 0x36, 0x75, 0x03, 0xc9, - 0xf0, 0x2f, 0x52, 0x84, 0x31, 0xaf, 0x5d, 0x7b, 0x4c, 0xeb, 0xbe, 0x40, 0x1a, 0x7c, 0x92, 0xe7, - 0x00, 0x76, 0x8c, 0xa6, 0x65, 0xea, 0x9b, 0xae, 0xb3, 0x8d, 0x50, 0x93, 0x5a, 0x0e, 0x4b, 0x6e, - 0xb8, 0xce, 0x36, 0x29, 0x43, 0x9e, 0x57, 0xb7, 0x6d, 0xdf, 0x6a, 0x16, 0x53, 0x58, 0xcf, 0x29, - 0x1e, 0xb2, 0x12, 0x72, 0x16, 0x72, 0x4c, 0x47, 0xa8, 0xe7, 0x51, 0xaf, 0x98, 0x3e, 0x97, 0xbc, - 0x94, 0xd3, 0xba, 0x05, 0xa4, 0x0a, 0x27, 0x3d, 0xab, 0x61, 0x1b, 0x7e, 0xdb, 0xa5, 0xba, 0xd1, - 0x6c, 0x38, 0xae, 0xe5, 0x3f, 0xda, 0x2e, 0x66, 0x10, 0x03, 0x09, 0xab, 0x16, 0x82, 0x1a, 0x06, - 0xa7, 0xd5, 0xae, 0x35, 0xad, 0xba, 0xbe, 0x45, 0x3b, 0xc5, 0x31, 0x6c, 0x97, 0xe3, 0x25, 0x77, - 0x68, 0x87, 0x9c, 0x81, 0xdc, 0x16, 0xed, 0xe8, 0x6d, 0x94, 0x79, 0x16, 0x7b, 0xcb, 0x6e, 0xd1, - 0xce, 0x43, 0x94, 0xf7, 0x4b, 0x40, 0xe8, 0xae, 0x4f, 0x6d, 0x93, 0x9a, 0x7a, 0xb7, 0x55, 0x0e, - 0x5b, 0x15, 0x82, 0x9a, 0x3b, 0xa2, 0xb5, 0xfa, 0x00, 0xa6, 0x7a, 0xe6, 0x96, 0x64, 0x20, 0xb1, - 0xb4, 0x50, 0x38, 0x41, 0xb2, 0x90, 0xba, 0x7b, 0x6f, 0x79, 0xa5, 0xa0, 0x90, 0x09, 0xc8, 0x2d, - 0xad, 0xad, 0xae, 0xdc, 0xdd, 0xd0, 0x97, 0x16, 0x0a, 0x09, 0x02, 0x90, 0xe1, 0x9f, 0x85, 0x24, - 0xc9, 0x41, 0xfa, 0xe1, 0x2a, 0x2b, 0x4e, 0x31, 0xba, 0x87, 0xab, 0x85, 0xb4, 0xea, 0xc0, 0x4c, - 0x54, 0x5f, 0xbd, 0x96, 0x63, 0x7b, 0x94, 0x7c, 0x06, 0xc6, 0xeb, 0x52, 0x79, 0x51, 0xc1, 0xa9, - 0x7f, 0xf9, 0x50, 0x53, 0x2f, 0xe6, 0x3c, 0xc2, 0x48, 0xad, 0xc2, 0xa4, 0xa8, 0x1e, 0xb6, 0x36, - 0x6e, 0xa7, 0xb2, 0x89, 0x42, 0x52, 0xbd, 0x0b, 0xb0, 0xde, 0xf1, 0x7c, 0xba, 0xbd, 0x6a, 0x6f, - 0x3a, 0x6c, 0x72, 0x3d, 0xfc, 0xd2, 0x99, 0x9d, 0x16, 0x04, 0xe0, 0x45, 0x1a, 0x6c, 0x51, 0xd7, - 0xa6, 0x4d, 0xde, 0x80, 0xab, 0x0e, 0xf0, 0x22, 0xd6, 0x40, 0xfd, 0x6a, 0x12, 0xa6, 0x42, 0x04, - 0x62, 0xb4, 0xef, 0x46, 0x21, 0xa4, 0x17, 0x17, 0xf6, 0xf7, 0xca, 0x99, 0xbb, 0x0c, 0xc6, 0xf2, - 0x4f, 0xf7, 0xca, 0xaf, 0x35, 0x2c, 0xff, 0x51, 0xbb, 0x56, 0xa9, 0x3b, 0xdb, 0xd5, 0x50, 0x00, - 0x66, 0xad, 0xfb, 0xbb, 0xda, 0xda, 0x6a, 0x54, 0x85, 0x49, 0xaf, 0x70, 0xb2, 0x60, 0x14, 0xe4, - 0x6d, 0x18, 0x13, 0xca, 0x85, 0x60, 0xf2, 0x57, 0xe7, 0x25, 0x21, 0x32, 0xdb, 0x55, 0x79, 0x18, - 0xda, 0xae, 0x05, 0xd3, 0x74, 0x85, 0xd4, 0x02, 0x22, 0xf2, 0x26, 0x00, 0x6e, 0x4c, 0x7c, 0x3c, - 0x49, 0x64, 0x31, 0x2b, 0xb1, 0xc0, 0xca, 0x0a, 0x1b, 0x9a, 0xa0, 0xcc, 0x61, 0x09, 0x0a, 0x63, - 0x2d, 0x2a, 0xad, 0x14, 0x12, 0xbf, 0x10, 0x33, 0x89, 0x5d, 0x49, 0x0b, 0x66, 0xb2, 0x68, 0xd7, - 0x21, 0xef, 0x3d, 0x69, 0xea, 0xc1, 0x68, 0xd2, 0x23, 0x8d, 0x86, 0x30, 0x36, 0xfb, 0x7b, 0x65, - 0x58, 0x7f, 0xb0, 0xb6, 0xc0, 0x29, 0x35, 0xf0, 0x9e, 0x34, 0xc5, 0x6f, 0x75, 0x12, 0xc6, 0x99, - 0xc0, 0x02, 0x6d, 0x50, 0xbf, 0x9d, 0x84, 0x09, 0x51, 0x20, 0x26, 0xe7, 0x16, 0xa4, 0x99, 0x28, - 0x03, 0x1d, 0x7c, 0x69, 0x00, 0x7c, 0xbe, 0xdd, 0x04, 0xbb, 0x20, 0xce, 0xc0, 0x3a, 0x7e, 0x88, - 0x51, 0x70, 0x06, 0xe4, 0xcf, 0x14, 0x38, 0x19, 0xec, 0x28, 0x7a, 0xad, 0xa3, 0x07, 0x73, 0x9e, - 0x40, 0xc6, 0x6f, 0xc7, 0xc8, 0x25, 0x82, 0xa8, 0xb2, 0x26, 0x78, 0x2c, 0x76, 0x70, 0xae, 0xcd, - 0x15, 0xdb, 0x77, 0x3b, 0x8b, 0xf7, 0xc4, 0x48, 0x0b, 0x3d, 0xd5, 0xcb, 0x1f, 0xfc, 0xf8, 0x68, - 0x1a, 0x54, 0x68, 0xf6, 0xf4, 0x53, 0xda, 0x85, 0xd9, 0x81, 0x7d, 0x93, 0x02, 0x24, 0x99, 0xf1, - 0x41, 0xe5, 0xd5, 0xd8, 0x4f, 0xb2, 0x0a, 0xe9, 0x1d, 0xa3, 0xd9, 0xe6, 0x66, 0x7e, 0xf2, 0xea, - 0x6b, 0xd2, 0xe0, 0xb6, 0x76, 0x2a, 0xc1, 0x16, 0x5b, 0x11, 0xdb, 0xbc, 0xe8, 0x34, 0x60, 0xce, - 0x85, 0xa7, 0x71, 0x0e, 0x6f, 0x26, 0xae, 0x2b, 0xea, 0x05, 0xc8, 0xb3, 0x06, 0x43, 0xf7, 0xb3, - 0xef, 0xa7, 0x20, 0xa7, 0x19, 0x9b, 0x3e, 0xe3, 0xc0, 0xcc, 0x1b, 0xb8, 0xb4, 0xd5, 0xb4, 0xea, - 0x46, 0xd0, 0x32, 0xb5, 0x38, 0xb1, 0xbf, 0x57, 0xce, 0x69, 0xbc, 0x74, 0x75, 0x59, 0xcb, 0x89, - 0x06, 0xab, 0x26, 0xf9, 0x38, 0xc0, 0x23, 0xc3, 0x35, 0xd1, 0x7b, 0xa0, 0x62, 0xb1, 0x4c, 0x57, - 0xf8, 0xc6, 0x5d, 0xb9, 0x65, 0xb8, 0x26, 0x32, 0x0d, 0xb4, 0xfc, 0x51, 0x50, 0xc0, 0x36, 0xad, - 0x26, 0x35, 0x4c, 0x5c, 0x1b, 0x29, 0x0d, 0x7f, 0x93, 0x19, 0x48, 0x73, 0x36, 0x29, 0x84, 0xc7, - 0x3f, 0xd8, 0x9e, 0x62, 0xb4, 0x5a, 0x4d, 0x8b, 0x9a, 0xa8, 0xbd, 0x29, 0x2d, 0xf8, 0x24, 0x1b, - 0x90, 0x6d, 0xb9, 0x4e, 0x03, 0x15, 0x3b, 0x83, 0xea, 0x70, 0x35, 0x46, 0x1d, 0xc2, 0x11, 0x56, - 0xee, 0x0b, 0x22, 0xae, 0x02, 0x1c, 0x5a, 0xc8, 0x89, 0x5c, 0x84, 0x29, 0x86, 0x46, 0xf7, 0x5d, - 0xc3, 0xf6, 0x36, 0xa9, 0x4b, 0x29, 0xee, 0x0f, 0x29, 0x6d, 0x92, 0x15, 0x6f, 0x84, 0xa5, 0xa5, - 0x5f, 0x55, 0x20, 0x1b, 0xb0, 0x62, 0xd8, 0xb7, 0x0d, 0xbf, 0xfe, 0x88, 0x0b, 0x4c, 0xe3, 0x1f, - 0x6c, 0x94, 0x36, 0xdd, 0xe5, 0x9b, 0x61, 0x4a, 0xc3, 0xdf, 0xdd, 0x51, 0x26, 0xe5, 0x51, 0xce, - 0x41, 0xa6, 0x65, 0xb4, 0x3d, 0x6a, 0xe2, 0xe0, 0xb3, 0x9a, 0xf8, 0x22, 0x97, 0xa1, 0xd0, 0xa2, - 0xb6, 0x69, 0xd9, 0x0d, 0xdd, 0xb3, 0x8d, 0x96, 0xf7, 0xc8, 0xf1, 0x85, 0x18, 0xa6, 0x44, 0xf9, - 0xba, 0x28, 0x2e, 0x3d, 0x86, 0x89, 0xc8, 0xc8, 0x64, 0x05, 0x4b, 0x71, 0x05, 0x5b, 0x92, 0x15, - 0x2c, 0x7e, 0x6b, 0xe8, 0x17, 0x97, 0xac, 0x5a, 0xfb, 0x09, 0x98, 0xd0, 0x0c, 0xbb, 0x41, 0xef, - 0xbb, 0x4e, 0xad, 0x49, 0xb7, 0x3d, 0x72, 0x0e, 0xf2, 0x6d, 0xdb, 0xd8, 0x31, 0xac, 0xa6, 0x51, - 0x6b, 0x72, 0x27, 0x28, 0xab, 0xc9, 0x45, 0xe4, 0x1a, 0x9c, 0x62, 0x12, 0xa4, 0xae, 0x6e, 0x3b, - 0xbe, 0xce, 0x9d, 0xce, 0x47, 0x4e, 0xd3, 0xa4, 0x2e, 0xc2, 0xc9, 0x6a, 0x33, 0xbc, 0xfa, 0xae, - 0xe3, 0xaf, 0xb1, 0xca, 0x5b, 0x58, 0x47, 0x9e, 0x87, 0x49, 0xdb, 0xd1, 0x99, 0x46, 0xe9, 0xbc, - 0x1e, 0x05, 0x97, 0xd5, 0xc6, 0x6d, 0x87, 0x61, 0x5c, 0xc3, 0x32, 0x72, 0x09, 0xa6, 0xda, 0xb6, - 0x49, 0x5d, 0xa1, 0x99, 0x7e, 0x28, 0xc8, 0xde, 0x62, 0x72, 0x1a, 0xb2, 0xb6, 0xc3, 0xbb, 0x47, - 0x49, 0x66, 0xb5, 0x31, 0xdb, 0xc1, 0x0e, 0xc9, 0x75, 0x28, 0x3e, 0x69, 0x5b, 0xd4, 0xab, 0x53, - 0xdb, 0xd7, 0xe9, 0x93, 0xb6, 0xd1, 0xf4, 0x74, 0xdf, 0xaa, 0x6f, 0x59, 0x76, 0x03, 0x7d, 0x89, - 0xac, 0x36, 0x17, 0xd6, 0xaf, 0x60, 0xf5, 0x06, 0xaf, 0x25, 0x2f, 0x02, 0xe1, 0x08, 0x9d, 0x86, - 0xee, 0x3b, 0x8e, 0xde, 0x34, 0xdc, 0x06, 0xd7, 0x9b, 0xac, 0x36, 0xc5, 0x6a, 0xd6, 0x9c, 0xc6, - 0x86, 0xe3, 0xac, 0xb1, 0x62, 0x72, 0x01, 0x26, 0x9d, 0x9d, 0x08, 0xd4, 0x2c, 0x36, 0xec, 0x29, - 0x55, 0xb7, 0x60, 0x0a, 0x65, 0xcc, 0xa6, 0xc1, 0xc2, 0x93, 0x04, 0xf3, 0x3d, 0x9e, 0xb4, 0xa9, - 0x6b, 0x51, 0x4f, 0x6f, 0x51, 0x57, 0xf7, 0x68, 0xdd, 0xb1, 0xf9, 0x22, 0x55, 0xb4, 0x82, 0xa8, - 0xb9, 0x4f, 0xdd, 0x75, 0x2c, 0x27, 0x57, 0x60, 0xfa, 0xff, 0xbb, 0x96, 0x1f, 0x6d, 0x9c, 0xc0, - 0xc6, 0x53, 0xbc, 0x22, 0x6c, 0xab, 0xde, 0x02, 0xb8, 0xef, 0x52, 0xdf, 0xef, 0xac, 0xb7, 0x0c, - 0x9b, 0x39, 0x40, 0x9e, 0x6f, 0xb8, 0xbe, 0x1e, 0x28, 0x50, 0x4e, 0xcb, 0x62, 0x01, 0xf3, 0x8e, - 0x4e, 0xc1, 0x18, 0xb5, 0xd1, 0xf7, 0x11, 0x5b, 0x75, 0x86, 0xda, 0xcc, 0xe1, 0x79, 0x33, 0xf5, - 0xef, 0xbf, 0x5d, 0x56, 0xd4, 0xaf, 0x66, 0x99, 0x39, 0xb1, 0x1b, 0x14, 0x37, 0xa0, 0x4f, 0x41, - 0xca, 0x6b, 0x19, 0x36, 0x32, 0x89, 0xdf, 0xc7, 0xba, 0xdd, 0x8b, 0x35, 0x89, 0x84, 0x64, 0x15, - 0x00, 0x45, 0x2b, 0x5b, 0x98, 0xe7, 0x47, 0x51, 0xdc, 0xc0, 0xe8, 0xb8, 0xa1, 0x69, 0xbb, 0x21, - 0x1b, 0x98, 0xfc, 0xd5, 0x2b, 0x43, 0xed, 0x6b, 0x38, 0x8c, 0x60, 0x4f, 0xe2, 0x8b, 0x75, 0x1b, - 0x26, 0x3d, 0xa7, 0xed, 0xd6, 0x69, 0xb8, 0x1b, 0xa5, 0xd1, 0x03, 0xb9, 0xb9, 0xbf, 0x57, 0x1e, - 0x5f, 0xc7, 0x9a, 0xe3, 0xf9, 0x21, 0xe3, 0x5e, 0x97, 0x89, 0x49, 0x9e, 0xc0, 0x94, 0xe8, 0x8e, - 0x21, 0xc3, 0xfe, 0x32, 0xd8, 0xdf, 0xea, 0xfe, 0x5e, 0x79, 0x82, 0xf7, 0xb7, 0xce, 0x6a, 0xb0, - 0xc3, 0xd7, 0x0f, 0xd5, 0xa1, 0xa0, 0xd3, 0x26, 0x3c, 0x89, 0x8d, 0xd9, 0x7f, 0xf0, 0x18, 0x1b, - 0x70, 0xf0, 0x58, 0x82, 0x09, 0xb1, 0x8a, 0x2d, 0x06, 0xac, 0x83, 0x9e, 0x72, 0xfe, 0x6a, 0x51, - 0x12, 0x6b, 0xd0, 0x0d, 0xae, 0xaf, 0xc0, 0xb7, 0x44, 0xa2, 0x5b, 0x9c, 0x86, 0xdc, 0x46, 0x23, - 0x8e, 0x36, 0xa4, 0x98, 0xc3, 0x69, 0xb9, 0x14, 0x3b, 0xb9, 0x92, 0xcd, 0x91, 0x4c, 0x37, 0xb7, - 0x41, 0x62, 0x7e, 0xbd, 0x22, 0xf4, 0xcd, 0xef, 0x40, 0x46, 0xdd, 0x85, 0x25, 0xcf, 0xaf, 0x47, - 0x3e, 0x07, 0x13, 0x4d, 0x66, 0xbf, 0xa9, 0xa7, 0x37, 0x9d, 0xba, 0xd1, 0x2c, 0xe6, 0x91, 0xdf, - 0xab, 0x43, 0xf5, 0x65, 0x8d, 0x51, 0xbd, 0x63, 0xd8, 0x46, 0x83, 0xba, 0x92, 0xda, 0x8c, 0x0b, - 0x6e, 0x6b, 0x8c, 0x19, 0x79, 0x0f, 0x26, 0x03, 0xee, 0x8d, 0xa6, 0x53, 0x33, 0x9a, 0xc5, 0xf1, - 0xe3, 0xb1, 0x0f, 0xc0, 0xde, 0x44, 0x6e, 0xe4, 0x21, 0x8c, 0xcb, 0x27, 0xfa, 0xe2, 0x04, 0x72, - 0x7f, 0x69, 0x38, 0x77, 0x46, 0x14, 0x71, 0xc1, 0xf2, 0xcd, 0x6e, 0x11, 0x3b, 0x81, 0x85, 0xc6, - 0xaf, 0x38, 0x89, 0x06, 0xab, 0x5b, 0xc0, 0x76, 0xe9, 0xc0, 0x52, 0x4e, 0x71, 0xa3, 0x2a, 0x3e, - 0xd5, 0x5f, 0x51, 0xc4, 0x56, 0x31, 0xf4, 0xf0, 0x40, 0x0c, 0xc8, 0xb9, 0xac, 0xa5, 0x6e, 0x99, - 0x1e, 0x3a, 0x78, 0xc9, 0xc5, 0xe5, 0xfd, 0xbd, 0x72, 0x96, 0x2f, 0xc3, 0x65, 0xef, 0xd0, 0xda, - 0x2d, 0x08, 0xb5, 0x2c, 0xb2, 0x5d, 0x35, 0x3d, 0x75, 0x03, 0x26, 0x03, 0x30, 0xc2, 0x55, 0x5d, - 0x84, 0x0c, 0xd6, 0x06, 0xbe, 0xea, 0xf3, 0xc3, 0xb4, 0x46, 0x92, 0xbc, 0xa0, 0x54, 0x2f, 0xc1, - 0xc4, 0x4d, 0x8c, 0x37, 0x0d, 0xf5, 0xb5, 0xbe, 0x9b, 0x80, 0xa9, 0x15, 0x0c, 0xcf, 0x30, 0xb1, - 0x7a, 0x68, 0x22, 0xdf, 0x83, 0x6c, 0xb8, 0xb0, 0xf9, 0x51, 0x66, 0x69, 0x7f, 0xaf, 0x3c, 0x76, - 0xdc, 0x25, 0x3d, 0xe6, 0x89, 0xc5, 0xbc, 0x09, 0x73, 0x6c, 0x32, 0xa8, 0xeb, 0xe9, 0x86, 0x6d, - 0xf2, 0xd5, 0xda, 0x70, 0x8d, 0xed, 0xe0, 0x70, 0xf3, 0x8a, 0x3c, 0x62, 0xae, 0x0e, 0x95, 0x20, - 0x84, 0x54, 0xd9, 0xe0, 0x94, 0x0b, 0xb6, 0x79, 0x2b, 0xa4, 0xd3, 0x66, 0xfc, 0x01, 0xa5, 0xe4, - 0x26, 0xe4, 0x39, 0x99, 0x8e, 0x71, 0x90, 0x24, 0x3a, 0xb1, 0x17, 0xe2, 0x98, 0x73, 0x49, 0x60, - 0xc0, 0x03, 0x68, 0xf8, 0x5b, 0x7d, 0x19, 0x88, 0x24, 0xa3, 0xa1, 0x32, 0xfd, 0xbf, 0x70, 0x32, - 0xd2, 0x5c, 0x4c, 0x6c, 0x68, 0x0d, 0xf8, 0xbc, 0xc6, 0x59, 0x83, 0x9e, 0x19, 0x89, 0x58, 0x03, - 0xf5, 0xff, 0x01, 0x6c, 0xb8, 0x46, 0x9d, 0xae, 0xec, 0x30, 0x45, 0xbf, 0x0e, 0x29, 0xdf, 0xda, - 0xa6, 0x62, 0x3f, 0x2b, 0x55, 0x78, 0x70, 0xa9, 0x12, 0x04, 0x97, 0x2a, 0x1b, 0x41, 0x70, 0x69, - 0x31, 0xcb, 0x98, 0x7c, 0xe3, 0xc7, 0x65, 0x45, 0x43, 0x0a, 0xb6, 0x44, 0xa2, 0x61, 0x9c, 0xe0, - 0x53, 0xfd, 0xbe, 0x02, 0x53, 0x0b, 0x4d, 0x66, 0x6a, 0x7c, 0xc7, 0x5d, 0x76, 0x3b, 0x5a, 0xdb, - 0x66, 0x4a, 0x11, 0xac, 0x05, 0xec, 0x2b, 0xc9, 0x95, 0x42, 0x68, 0xf4, 0x91, 0x57, 0xc2, 0x98, - 0x58, 0x09, 0xe4, 0x2d, 0xc8, 0x50, 0x36, 0x20, 0x4f, 0x9c, 0xa4, 0xe2, 0x76, 0xe6, 0xee, 0xf0, - 0x35, 0x41, 0xa4, 0x5e, 0x85, 0xd9, 0x10, 0x31, 0xf2, 0x0e, 0x66, 0xe9, 0x74, 0x2f, 0xee, 0xb0, - 0x4b, 0xf5, 0x8f, 0x15, 0x98, 0xeb, 0x25, 0x1a, 0x7c, 0x98, 0x4f, 0x7e, 0x98, 0x87, 0xf9, 0x25, - 0x18, 0x33, 0xdd, 0x8e, 0xee, 0xb6, 0x6d, 0xa1, 0xef, 0x71, 0x9a, 0xd0, 0x33, 0x0d, 0x5a, 0xc6, - 0xc4, 0x7f, 0xd5, 0xaf, 0x29, 0x50, 0xe8, 0x62, 0xff, 0x1f, 0x60, 0xc8, 0xde, 0x85, 0x69, 0x09, - 0x8f, 0x10, 0xe3, 0x0a, 0x64, 0xc5, 0x50, 0x47, 0xd1, 0xfa, 0xde, 0xb1, 0x8e, 0xf1, 0xb1, 0x7a, - 0xaa, 0x0a, 0xe3, 0xb7, 0xd7, 0xef, 0xdd, 0x0d, 0xd9, 0x06, 0x11, 0x46, 0xa5, 0x1b, 0x61, 0x54, - 0xbf, 0xab, 0x40, 0x7e, 0xcd, 0x69, 0x0c, 0x37, 0xea, 0x33, 0x90, 0x6e, 0xd2, 0x1d, 0xda, 0x14, - 0x4a, 0xcf, 0x3f, 0xc8, 0x73, 0x00, 0xdc, 0xc1, 0xc4, 0xc5, 0xc4, 0x8f, 0x42, 0xdc, 0xe5, 0x64, - 0x0b, 0x88, 0x69, 0x11, 0x73, 0x31, 0xb1, 0x92, 0x9f, 0x06, 0x99, 0xcb, 0x89, 0x55, 0x05, 0x48, - 0x6e, 0x1b, 0xbb, 0xe8, 0x71, 0xe5, 0x34, 0xf6, 0x93, 0x2d, 0xac, 0x96, 0xe1, 0xfb, 0xd4, 0xb5, - 0x45, 0xc4, 0x2f, 0xf8, 0x54, 0xef, 0x01, 0x59, 0x73, 0x1a, 0xec, 0x34, 0x64, 0x49, 0x16, 0xff, - 0x13, 0xcc, 0x7f, 0xc5, 0x22, 0x21, 0xa4, 0xd3, 0xbd, 0xf1, 0x90, 0xa6, 0xd3, 0xa8, 0xc8, 0xa7, - 0xc3, 0xa0, 0xbd, 0x5a, 0x81, 0x93, 0x6b, 0x4e, 0xe3, 0x86, 0xd5, 0xa4, 0xde, 0x9a, 0xe5, 0xf9, - 0x43, 0x4d, 0xd3, 0x7d, 0x98, 0x89, 0xb6, 0x17, 0x10, 0xae, 0x43, 0x7a, 0x93, 0x15, 0x0a, 0x00, - 0x67, 0x07, 0x01, 0x60, 0x54, 0xb2, 0x35, 0x42, 0x02, 0xf5, 0x2d, 0x98, 0x14, 0x1c, 0x87, 0x4a, - 0x9e, 0x40, 0x8a, 0xd1, 0x08, 0xc1, 0xe3, 0x6f, 0xb5, 0x0e, 0x13, 0xeb, 0xbe, 0x51, 0xdf, 0x1a, - 0x3e, 0x6f, 0x9f, 0x10, 0xe1, 0x6c, 0x1e, 0x8b, 0x88, 0x0d, 0x40, 0x21, 0xc3, 0x6e, 0xd8, 0x5a, - 0x5d, 0x87, 0x14, 0x03, 0x88, 0xc7, 0x5f, 0x43, 0xd8, 0xca, 0x9c, 0x86, 0xbf, 0xd9, 0xc9, 0x82, - 0x01, 0xd1, 0x3d, 0xeb, 0xf3, 0x9c, 0x77, 0x52, 0xcb, 0xb2, 0x82, 0x75, 0xeb, 0xf3, 0x94, 0x94, - 0x20, 0x5b, 0x77, 0x6c, 0x1f, 0xcd, 0x12, 0x0f, 0x67, 0x87, 0xdf, 0xea, 0x6f, 0x2a, 0x30, 0x75, - 0x93, 0xfa, 0x28, 0xcb, 0xa1, 0xe0, 0xcf, 0x40, 0xae, 0x69, 0x79, 0xbe, 0xee, 0xd8, 0xcd, 0x8e, - 0x38, 0x5d, 0x66, 0x59, 0xc1, 0x3d, 0xbb, 0xd9, 0x21, 0x6f, 0x88, 0x91, 0xa5, 0x71, 0x64, 0xe7, - 0x63, 0x46, 0xc6, 0x3a, 0x93, 0xc2, 0xf1, 0x25, 0xc8, 0x0a, 0xcd, 0xe2, 0x01, 0x87, 0x9c, 0x16, - 0x7e, 0xab, 0xab, 0x50, 0xe8, 0xa2, 0x13, 0xb3, 0x7c, 0x2d, 0x3a, 0xcb, 0xe5, 0x21, 0x3d, 0x05, - 0x53, 0xfc, 0x45, 0x98, 0xbc, 0xef, 0x3a, 0x9b, 0xa3, 0x4c, 0xf1, 0x62, 0x64, 0x28, 0x95, 0xd8, - 0xd3, 0x95, 0xcc, 0xb1, 0x22, 0xcd, 0x56, 0x01, 0x52, 0x18, 0x96, 0xce, 0x42, 0xea, 0xd6, 0xca, - 0xc2, 0xfd, 0xc2, 0x09, 0xf5, 0x32, 0x4c, 0xbe, 0x43, 0x7d, 0xd7, 0xaa, 0x0f, 0xdf, 0x7b, 0x7f, - 0x0f, 0xbd, 0xbb, 0x4d, 0x1f, 0x0d, 0x14, 0x33, 0xbc, 0xcf, 0x34, 0x2e, 0xfb, 0x69, 0x48, 0xa3, - 0x01, 0x1c, 0xe9, 0x18, 0xd8, 0x73, 0x74, 0x43, 0x42, 0xf5, 0x0a, 0xf3, 0xff, 0x04, 0xdc, 0x15, - 0x76, 0x98, 0x91, 0xb7, 0x65, 0x25, 0xba, 0x2d, 0x7f, 0x29, 0xc1, 0x0e, 0xe0, 0xa2, 0xb1, 0xf0, - 0x82, 0x9f, 0xf5, 0xb6, 0x7c, 0x13, 0x32, 0x78, 0xc6, 0x0a, 0xb6, 0xe5, 0xcb, 0x43, 0x4e, 0xba, - 0xdd, 0x81, 0x04, 0x2e, 0x29, 0x27, 0x27, 0xcb, 0x41, 0x04, 0x36, 0x89, 0x7c, 0x2e, 0x8d, 0xc2, - 0x87, 0x49, 0x3b, 0x12, 0x7d, 0x55, 0xdb, 0x50, 0x60, 0xb5, 0xcb, 0xb4, 0xd6, 0x6e, 0x04, 0xba, - 0x10, 0xd9, 0xdc, 0x94, 0x67, 0xb2, 0xb9, 0xfd, 0x63, 0x02, 0xa6, 0xa5, 0x7e, 0xc5, 0x72, 0xfa, - 0x9a, 0xd2, 0xe3, 0xaa, 0x5f, 0x1f, 0x32, 0xa8, 0x08, 0x39, 0xef, 0x46, 0x04, 0xfd, 0x3e, 0xc9, - 0x06, 0xf9, 0xc1, 0x8f, 0x8f, 0x08, 0x54, 0xa0, 0xf8, 0xd0, 0x26, 0xab, 0x44, 0x21, 0x2f, 0xa1, - 0x93, 0x03, 0x77, 0x49, 0x1e, 0xb8, 0xfb, 0x74, 0x34, 0x70, 0x77, 0x65, 0x94, 0x8e, 0xfa, 0x03, - 0xc2, 0x7f, 0xa1, 0xc0, 0xd8, 0xc6, 0xae, 0x8d, 0x87, 0x8e, 0x07, 0x90, 0x10, 0x2a, 0x3c, 0xbe, - 0xb8, 0xc0, 0xc0, 0xfc, 0xd3, 0xa8, 0x6b, 0x93, 0xdf, 0xdd, 0xb6, 0x2d, 0xb3, 0xf2, 0xf0, 0xe1, - 0x2a, 0x9b, 0xf9, 0xc4, 0xea, 0xb2, 0x96, 0xb0, 0x4c, 0xf2, 0x26, 0x3a, 0xdc, 0xae, 0x2f, 0x40, - 0x8e, 0xe6, 0x1b, 0x73, 0x12, 0x72, 0x11, 0xa6, 0xfc, 0x5d, 0x5b, 0x37, 0xa9, 0x57, 0x77, 0xad, - 0x96, 0x6f, 0x39, 0xb6, 0x70, 0x0a, 0x26, 0xfd, 0x5d, 0x7b, 0xb9, 0x5b, 0xaa, 0xfe, 0x57, 0x02, - 0xf2, 0x0b, 0x75, 0xdf, 0xda, 0xa1, 0x0f, 0xda, 0xd4, 0xed, 0x90, 0xb9, 0x70, 0x1c, 0xb9, 0xc5, - 0x8c, 0x04, 0xa6, 0x00, 0x49, 0xef, 0x49, 0xe0, 0x74, 0xb0, 0x9f, 0x5d, 0x78, 0xc9, 0xc3, 0xc3, - 0x7b, 0x01, 0x26, 0x2d, 0x4f, 0x37, 0x2d, 0xcf, 0x77, 0xad, 0x5a, 0xbb, 0x1b, 0x5d, 0x9c, 0xb0, - 0xbc, 0xe5, 0x6e, 0x21, 0x59, 0x84, 0x74, 0xeb, 0x51, 0x10, 0x58, 0x9c, 0x1c, 0x78, 0xed, 0x11, - 0x3a, 0x5f, 0xdd, 0x31, 0x54, 0xee, 0x33, 0x1a, 0x8d, 0x93, 0xe2, 0x26, 0xd3, 0x8d, 0x6a, 0x2b, - 0x97, 0x12, 0x52, 0x6c, 0xfa, 0xb3, 0x90, 0x61, 0x52, 0xb2, 0x4c, 0x8c, 0xc7, 0x8c, 0x2f, 0x2e, - 0x1f, 0x6f, 0xe2, 0xd2, 0x4c, 0x19, 0x96, 0xb5, 0xb4, 0xbf, 0x6b, 0xaf, 0x9a, 0xea, 0x0b, 0x90, - 0x46, 0x20, 0x64, 0x02, 0x72, 0xf7, 0xb5, 0x95, 0xfb, 0x0b, 0xda, 0xea, 0xdd, 0x9b, 0x85, 0x13, - 0xec, 0x73, 0xe5, 0xff, 0xac, 0x2c, 0x3d, 0xdc, 0x60, 0x9f, 0x8a, 0xfa, 0x2a, 0x9c, 0x64, 0xae, - 0xcc, 0x3a, 0xf5, 0x3c, 0xcb, 0xb1, 0xc3, 0x1d, 0xa2, 0x04, 0xd9, 0xb6, 0x47, 0x5d, 0x69, 0xbf, - 0x0f, 0xbf, 0xd5, 0x6f, 0xa7, 0x61, 0x4c, 0xb4, 0x7f, 0xa6, 0xdb, 0x83, 0x8c, 0x21, 0x11, 0xc5, - 0xc0, 0x66, 0xb0, 0xde, 0xb4, 0xa8, 0xed, 0x87, 0x77, 0x61, 0x5c, 0xbf, 0x26, 0x78, 0xa9, 0xb8, - 0xda, 0x22, 0x97, 0xa1, 0x80, 0xd7, 0x0b, 0x75, 0x4c, 0x21, 0xd0, 0x91, 0x15, 0x77, 0x40, 0xa7, - 0xa4, 0xf2, 0xbb, 0x8c, 0xe3, 0x3a, 0x4c, 0x1a, 0x38, 0x89, 0xba, 0x08, 0xbc, 0xe2, 0xbd, 0x74, - 0x3e, 0x7a, 0xe2, 0x3d, 0x78, 0xd6, 0x83, 0xe0, 0x8d, 0x11, 0x16, 0x59, 0xd4, 0xeb, 0x2a, 0x69, - 0xe6, 0xf0, 0x4a, 0xfa, 0x1e, 0xe4, 0xb6, 0x76, 0xf4, 0x88, 0x82, 0x2c, 0x1e, 0x5d, 0x39, 0xc6, - 0xee, 0xec, 0x70, 0xf5, 0x18, 0xdb, 0xc2, 0x1f, 0x18, 0x4e, 0x6e, 0x1a, 0x9e, 0xaf, 0x4b, 0xa3, - 0xee, 0x60, 0xe8, 0x3a, 0xa7, 0x4d, 0xb1, 0x8a, 0xfe, 0x65, 0x99, 0x43, 0x10, 0xf2, 0xb2, 0x2c, - 0x43, 0xde, 0x60, 0xc7, 0x0e, 0xbd, 0xd6, 0xf1, 0x29, 0x0f, 0xd4, 0x25, 0x35, 0xc0, 0xa2, 0x45, - 0x56, 0x42, 0x2e, 0xc0, 0xd4, 0xb6, 0xb1, 0xab, 0xcb, 0x8d, 0xf2, 0xd8, 0x68, 0x62, 0xdb, 0xd8, - 0x5d, 0xe8, 0xb6, 0x5b, 0x00, 0x10, 0x38, 0xfc, 0x5d, 0x5b, 0x44, 0xd0, 0xd4, 0xb8, 0x33, 0x2c, - 0xb7, 0x7b, 0x5a, 0x8e, 0x53, 0x6d, 0xec, 0xda, 0xea, 0x2f, 0x2b, 0x30, 0x2d, 0xab, 0x32, 0xf7, - 0x07, 0x9e, 0xa5, 0x82, 0x1e, 0x1c, 0x02, 0xf8, 0x5d, 0x05, 0x66, 0xa2, 0xcb, 0x4a, 0x6c, 0x7a, - 0xcb, 0x90, 0xf5, 0x44, 0x99, 0xd8, 0xf5, 0xe2, 0x46, 0x29, 0xc8, 0x83, 0xc8, 0x68, 0x40, 0x49, - 0x6e, 0xf7, 0xec, 0x54, 0x71, 0x96, 0xa9, 0x4f, 0x24, 0xd1, 0xcd, 0x4a, 0x7d, 0x02, 0x64, 0xc9, - 0xb0, 0xeb, 0xb4, 0x89, 0x33, 0x3d, 0xd4, 0x45, 0xbd, 0x00, 0x59, 0xd4, 0x14, 0x7e, 0x69, 0xcb, - 0xcc, 0x74, 0x9e, 0x69, 0x17, 0x12, 0x33, 0xed, 0xc2, 0xca, 0x9e, 0xc5, 0x9b, 0xec, 0x31, 0x20, - 0x37, 0xe1, 0x64, 0xa4, 0x4b, 0x21, 0x1b, 0x76, 0x5c, 0xc0, 0x62, 0x6a, 0x8a, 0x0b, 0xa7, 0xf0, - 0x9b, 0x1d, 0x3b, 0x11, 0x6f, 0x70, 0xec, 0xc4, 0x0f, 0xb5, 0x03, 0x33, 0x9c, 0x91, 0x18, 0xe0, - 0x50, 0xf4, 0x2f, 0x01, 0x08, 0x21, 0x06, 0xf8, 0xc7, 0xf9, 0x6d, 0xa8, 0x60, 0xb0, 0xba, 0xac, - 0xe5, 0x44, 0x83, 0x21, 0x63, 0x58, 0x85, 0xd9, 0x9e, 0xae, 0x8f, 0x3c, 0x8a, 0x7f, 0x56, 0xa0, - 0xb0, 0xde, 0x32, 0xec, 0x48, 0x78, 0xec, 0x7c, 0xcf, 0x10, 0x16, 0xa1, 0xab, 0xb7, 0xe1, 0x70, - 0x34, 0xf9, 0x5e, 0x87, 0x8f, 0xe6, 0xda, 0x4f, 0xf7, 0xca, 0xaf, 0x1e, 0xce, 0x0d, 0xba, 0x43, - 0x3b, 0xd2, 0x75, 0xd0, 0xdd, 0xee, 0x75, 0x50, 0xf2, 0x38, 0x1c, 0xc5, 0x2d, 0x92, 0xfa, 0x47, - 0x0a, 0x4c, 0x4b, 0xa3, 0x13, 0x52, 0x5a, 0x83, 0xbc, 0xef, 0xf8, 0x46, 0x53, 0x0f, 0x62, 0x7a, - 0x7d, 0xd7, 0x49, 0xbd, 0xc1, 0xc5, 0x77, 0xfe, 0xf7, 0xd2, 0x12, 0xf2, 0x08, 0xd2, 0x22, 0x90, - 0x1e, 0x4b, 0x98, 0x19, 0xe2, 0x3e, 0x6c, 0xdd, 0x69, 0xdb, 0xdc, 0x61, 0x49, 0x6b, 0x80, 0x45, - 0x4b, 0xac, 0x84, 0xbc, 0x0e, 0x73, 0x46, 0xab, 0xe5, 0x3a, 0xbb, 0xd6, 0xb6, 0xe1, 0x53, 0xb6, - 0xf3, 0x6f, 0x09, 0x6b, 0xc4, 0x6f, 0xac, 0x67, 0xa4, 0xda, 0x65, 0xcb, 0xdb, 0x42, 0xa3, 0xa4, - 0xfe, 0x2f, 0x98, 0x11, 0x97, 0x13, 0xd1, 0x88, 0xf7, 0x28, 0x73, 0xa3, 0x7e, 0x67, 0x02, 0x66, - 0x7b, 0xa8, 0xfb, 0xa3, 0x63, 0xd9, 0x0f, 0xdb, 0x24, 0xfd, 0x8d, 0x02, 0x27, 0x83, 0x0b, 0x14, - 0x39, 0xbf, 0x22, 0x87, 0x76, 0xe2, 0x46, 0xfc, 0x89, 0xb2, 0x1f, 0x6b, 0x25, 0xbc, 0x9c, 0x19, - 0x9c, 0x67, 0xd1, 0x53, 0x7d, 0xf4, 0x3c, 0x8b, 0x56, 0x4f, 0x3f, 0xa5, 0xbf, 0xcf, 0xf1, 0xac, - 0x94, 0xf0, 0x46, 0xba, 0xef, 0x0e, 0x4b, 0x19, 0x70, 0x87, 0xf5, 0x73, 0x0a, 0xcc, 0x4a, 0x97, - 0xd4, 0x7a, 0x6f, 0xd8, 0xee, 0xde, 0xfe, 0x5e, 0xf9, 0xe4, 0xc3, 0x6e, 0x83, 0x63, 0x1f, 0x72, - 0x4e, 0xb6, 0x7b, 0x99, 0x99, 0x1e, 0xf9, 0x7d, 0x05, 0x2e, 0x48, 0x37, 0xdc, 0x7d, 0x17, 0xe4, - 0x12, 0xac, 0x24, 0xc2, 0xfa, 0xdc, 0xfe, 0x5e, 0xf9, 0x5c, 0xf7, 0xfa, 0x3b, 0x7a, 0x65, 0x7e, - 0x6c, 0x8c, 0xe7, 0xdc, 0x58, 0xce, 0xa6, 0x47, 0xbe, 0xac, 0x40, 0x31, 0x7a, 0x2b, 0x2f, 0x41, - 0x4c, 0x21, 0xc4, 0xfb, 0xfb, 0x7b, 0xe5, 0x99, 0xbb, 0xd2, 0x1d, 0xfd, 0xb1, 0x61, 0xcd, 0xd8, - 0x7d, 0xdc, 0x4c, 0x8f, 0xec, 0x02, 0x09, 0xee, 0xf3, 0x25, 0x0c, 0x69, 0xc4, 0x70, 0x67, 0x7f, - 0xaf, 0x3c, 0x75, 0x97, 0xdf, 0xee, 0x1f, 0xbb, 0xfb, 0x29, 0x5b, 0x66, 0x64, 0x7a, 0xe4, 0xeb, - 0x0a, 0x9c, 0xee, 0xc9, 0x2e, 0x90, 0x10, 0x64, 0x10, 0xc1, 0xfa, 0xfe, 0x5e, 0xf9, 0xd4, 0xc3, - 0x68, 0xa3, 0x63, 0x23, 0x39, 0xd5, 0x1e, 0xc4, 0xd0, 0xf4, 0xc8, 0xef, 0x28, 0xa0, 0x1e, 0x94, - 0xc1, 0x20, 0x41, 0x1b, 0x43, 0x68, 0xef, 0xee, 0xef, 0x95, 0xe7, 0x1f, 0x0c, 0xcc, 0x67, 0x38, - 0x36, 0xc2, 0xf9, 0x27, 0x31, 0x7c, 0x4d, 0x8f, 0x7c, 0x4b, 0x81, 0xb3, 0xfd, 0x09, 0x13, 0x12, - 0xc4, 0x6c, 0x57, 0x7a, 0x5a, 0x34, 0x7d, 0xe2, 0xf8, 0xd2, 0x73, 0x07, 0x31, 0x34, 0x3d, 0xf2, - 0x55, 0x05, 0x8a, 0xd1, 0x14, 0x0c, 0x09, 0x50, 0x0e, 0x01, 0x69, 0xfb, 0x7b, 0xe5, 0xb9, 0x7b, - 0x3b, 0x1f, 0xea, 0x6c, 0xce, 0x39, 0x3b, 0x83, 0x26, 0xb3, 0xf4, 0x81, 0x12, 0xee, 0x07, 0x43, - 0x33, 0xc7, 0xd6, 0xa3, 0xf1, 0x81, 0xb7, 0x0e, 0x6d, 0xb6, 0x65, 0xcb, 0x29, 0x85, 0x0c, 0x6e, - 0xa7, 0xb2, 0x4a, 0x21, 0xab, 0xbe, 0x01, 0x85, 0x5b, 0x8e, 0x7f, 0x84, 0x3d, 0xed, 0x2b, 0x63, - 0x30, 0x2d, 0x51, 0x7e, 0x04, 0xa9, 0x9b, 0x7f, 0xab, 0xc0, 0xec, 0x23, 0xc7, 0xe7, 0x33, 0x37, - 0x20, 0x63, 0x70, 0x29, 0x46, 0x34, 0x7d, 0x48, 0xbb, 0x25, 0xd1, 0xed, 0xec, 0xbe, 0xd8, 0xce, - 0xa6, 0x7b, 0xeb, 0x8f, 0xbc, 0x9f, 0x4d, 0x3f, 0xea, 0xed, 0xa9, 0xb4, 0x03, 0xd9, 0x80, 0x3d, - 0xf9, 0x24, 0xa4, 0x4c, 0xea, 0xd5, 0x85, 0xdb, 0xa3, 0x0e, 0xc8, 0xb0, 0xc0, 0x76, 0x41, 0xa0, - 0x24, 0xf4, 0xd9, 0x91, 0xea, 0x80, 0xac, 0xa1, 0xc4, 0xe0, 0xac, 0xa1, 0xd2, 0xdf, 0x29, 0x30, - 0x81, 0x77, 0xc8, 0xe1, 0x7c, 0x3d, 0xeb, 0x0b, 0xea, 0x77, 0x01, 0xba, 0x53, 0x26, 0xe6, 0xe9, - 0xda, 0x91, 0xe6, 0x29, 0x4c, 0x34, 0x0c, 0x5a, 0x94, 0x7e, 0x49, 0xe1, 0x6e, 0x41, 0x38, 0x98, - 0x91, 0xdc, 0x02, 0x0d, 0x32, 0x08, 0x2e, 0x40, 0xf3, 0xe6, 0xa1, 0xd0, 0x44, 0xa4, 0xa7, 0x09, - 0x4e, 0xa5, 0x2f, 0xc2, 0xdc, 0x60, 0x75, 0x1a, 0xb0, 0x9e, 0xef, 0x45, 0xd7, 0xf3, 0x27, 0x0e, - 0xd5, 0xbd, 0x3c, 0x5c, 0x39, 0xfc, 0x77, 0x19, 0xc6, 0x47, 0xbd, 0xaa, 0xfd, 0x5e, 0x5a, 0x24, - 0x6d, 0x7c, 0x24, 0x6b, 0x56, 0x0e, 0xaa, 0x27, 0x9e, 0x41, 0x50, 0xfd, 0x2f, 0x15, 0x98, 0x71, - 0xc5, 0x40, 0x22, 0x26, 0x81, 0xc7, 0xc6, 0x3f, 0x35, 0xec, 0x1a, 0xa1, 0x1b, 0x42, 0x0e, 0x98, - 0x1c, 0x60, 0x0e, 0x7a, 0xeb, 0x8f, 0x6e, 0x0e, 0xdc, 0xde, 0x9e, 0x4a, 0xdf, 0xec, 0x55, 0xe4, - 0x12, 0x64, 0x83, 0x56, 0xc1, 0xb9, 0xd1, 0x3d, 0x50, 0xc9, 0x07, 0x3d, 0x1c, 0xf9, 0x34, 0xa4, - 0x2d, 0x7b, 0xd3, 0x09, 0xae, 0x08, 0x0e, 0x75, 0x9b, 0x82, 0x84, 0xa5, 0xf7, 0x61, 0x6e, 0xb0, - 0x48, 0x06, 0xa8, 0xf4, 0x9d, 0xa8, 0x4a, 0x5f, 0x1b, 0x59, 0xe8, 0x07, 0xa8, 0xf3, 0xed, 0x54, - 0x36, 0x55, 0x48, 0xab, 0x2f, 0x03, 0x59, 0xee, 0x3e, 0xb5, 0x1a, 0x7a, 0x5f, 0x75, 0x49, 0xd8, - 0xb6, 0xe1, 0x2d, 0xff, 0x20, 0x01, 0xe3, 0xd8, 0x34, 0x78, 0xdf, 0xf3, 0xac, 0xad, 0xe0, 0x8b, - 0x30, 0x4d, 0xed, 0xba, 0xdb, 0xc1, 0x38, 0x77, 0x90, 0xbc, 0x85, 0x67, 0x74, 0xad, 0xd0, 0xad, - 0x10, 0xf7, 0x50, 0xe5, 0xe0, 0x38, 0xcc, 0x2f, 0x18, 0xf9, 0xa1, 0x94, 0x9f, 0x70, 0xf1, 0x0e, - 0xb2, 0xdb, 0x80, 0x9f, 0x5a, 0x53, 0x52, 0x03, 0x1e, 0x40, 0xbb, 0x04, 0x05, 0x11, 0x40, 0xdb, - 0xa2, 0x1d, 0xc1, 0x86, 0x67, 0x16, 0x8b, 0xb0, 0xe6, 0x1d, 0xda, 0xe1, 0xac, 0xa2, 0x2d, 0x39, - 0xbf, 0x4c, 0x4f, 0x4b, 0x7e, 0xfe, 0xfd, 0x0c, 0x4c, 0x06, 0xd2, 0x0d, 0x33, 0x12, 0x02, 0x43, - 0xca, 0x83, 0x57, 0x17, 0x63, 0xef, 0x91, 0xbb, 0xd2, 0x0e, 0x62, 0x4e, 0x9c, 0x58, 0xbd, 0x0e, - 0xd3, 0x98, 0xc2, 0xb9, 0x4d, 0xed, 0xc3, 0x45, 0x3c, 0xd4, 0x6f, 0xa5, 0x81, 0xc8, 0xa4, 0x02, - 0x57, 0x0b, 0xf3, 0x0f, 0x44, 0xa9, 0xc0, 0x76, 0x3b, 0xfe, 0x8e, 0xbb, 0x87, 0x45, 0x65, 0xc9, - 0x69, 0x36, 0x69, 0xdd, 0xa7, 0x66, 0x58, 0xd7, 0x97, 0x4f, 0x28, 0xf5, 0x41, 0x96, 0x00, 0x30, - 0x7a, 0xea, 0x52, 0x8f, 0x1e, 0xee, 0x0e, 0x22, 0xc7, 0xe8, 0x34, 0x46, 0x46, 0xde, 0x80, 0xa2, - 0x65, 0xfb, 0xd4, 0xb5, 0x8d, 0xa6, 0x6e, 0xb4, 0x5a, 0x18, 0x9f, 0xd6, 0x5b, 0x2e, 0xdd, 0xb4, - 0x76, 0x45, 0x98, 0x7a, 0x36, 0xa8, 0x5f, 0x68, 0xb5, 0xee, 0x1a, 0xdb, 0xf4, 0x3e, 0x56, 0x96, - 0xfe, 0x5a, 0x81, 0xf9, 0x15, 0xf1, 0x36, 0x69, 0x00, 0xde, 0x3b, 0xb4, 0x43, 0x6e, 0x40, 0x96, - 0xcd, 0x6f, 0x98, 0xe9, 0xd1, 0x13, 0x5e, 0x79, 0xd2, 0xac, 0x0c, 0x26, 0x0c, 0x72, 0x24, 0xb6, - 0x68, 0x67, 0xd9, 0xf0, 0x0d, 0x79, 0xa7, 0x48, 0x7c, 0xc8, 0x3b, 0x05, 0x1b, 0xc6, 0xd9, 0x38, - 0xb9, 0x13, 0xb3, 0x6b, 0x77, 0xf2, 0x57, 0xd7, 0x0e, 0x37, 0xa1, 0xf1, 0xf2, 0x11, 0xc3, 0x44, - 0x5b, 0xf6, 0x76, 0x90, 0x5a, 0x96, 0xe8, 0x8f, 0x3b, 0x0f, 0x96, 0x53, 0x34, 0xa5, 0xec, 0xcb, - 0x09, 0x28, 0x85, 0x8d, 0x22, 0xe6, 0xab, 0xe5, 0xb8, 0x3e, 0x99, 0x0c, 0xef, 0xb4, 0x92, 0x18, - 0x34, 0x3f, 0x0b, 0xb9, 0xba, 0xb3, 0xdd, 0x6a, 0x52, 0x9f, 0x9a, 0x22, 0x9b, 0xa1, 0x5b, 0x40, - 0x5e, 0x83, 0xd9, 0x50, 0xcd, 0xf4, 0x4d, 0xcb, 0x6e, 0x50, 0xb7, 0xe5, 0x5a, 0xb6, 0x2f, 0x22, - 0x90, 0x33, 0x61, 0xe5, 0x8d, 0x6e, 0x1d, 0x79, 0x1b, 0x8a, 0x5d, 0x22, 0xe9, 0xad, 0x2a, 0x9b, - 0x35, 0x7c, 0x7d, 0x87, 0x80, 0x15, 0x6d, 0xce, 0x1b, 0x00, 0x13, 0xef, 0xa9, 0xc7, 0x5d, 0xbe, - 0x0c, 0xa9, 0xa9, 0x1b, 0xbe, 0x78, 0x58, 0x34, 0x9a, 0x3e, 0xe7, 0x43, 0xca, 0x05, 0x5f, 0x7d, - 0x0f, 0x2e, 0x2e, 0xb9, 0xd4, 0xf0, 0xe9, 0xc1, 0xf2, 0x08, 0xd6, 0xfb, 0x81, 0x03, 0x55, 0x0e, - 0x1e, 0xa8, 0xda, 0x81, 0x4b, 0xc3, 0xf9, 0x0b, 0xa3, 0xf0, 0x0e, 0x64, 0x5c, 0x2c, 0x11, 0xfa, - 0x73, 0x6d, 0x14, 0xfd, 0xe9, 0x67, 0x27, 0x98, 0xa8, 0xcf, 0x83, 0x7a, 0x70, 0xab, 0xf0, 0xf1, - 0xd4, 0xcf, 0xc0, 0xf9, 0xd8, 0x56, 0x02, 0xdb, 0x43, 0x18, 0xe3, 0x6c, 0x03, 0x6b, 0x75, 0x34, - 0x70, 0xc1, 0x62, 0x15, 0xbc, 0xd4, 0x3f, 0x51, 0x60, 0x66, 0x50, 0xeb, 0x3e, 0x1d, 0x3c, 0x50, - 0xf8, 0x89, 0x18, 0x2d, 0xbb, 0x09, 0xe3, 0xf5, 0x60, 0xb5, 0x32, 0x2d, 0x39, 0x8c, 0xd5, 0xcb, - 0x87, 0x94, 0x0b, 0xf8, 0x68, 0xc6, 0x77, 0x8d, 0x7a, 0xf8, 0x34, 0x08, 0x3f, 0xd4, 0xcf, 0xc0, - 0x99, 0xc1, 0x23, 0xe5, 0xfa, 0x72, 0x3d, 0x46, 0xc7, 0xf9, 0xc0, 0x0e, 0xd0, 0x6e, 0xf5, 0x09, - 0x9c, 0x1d, 0xcc, 0x58, 0x4c, 0xc6, 0x03, 0xc8, 0x4b, 0xfc, 0x84, 0x15, 0xa8, 0x1e, 0x76, 0x42, - 0x64, 0x1e, 0xea, 0x6b, 0x50, 0xbc, 0xed, 0xd4, 0x34, 0xda, 0xb0, 0x3c, 0xdf, 0xed, 0x88, 0xbb, - 0xfb, 0x61, 0x3e, 0xca, 0x7f, 0x28, 0x70, 0x7a, 0x00, 0xd5, 0x47, 0xe0, 0xb2, 0x7f, 0x16, 0xc6, - 0xdd, 0xb6, 0x6d, 0x5b, 0x76, 0x43, 0x7f, 0xec, 0xd4, 0x82, 0x63, 0x52, 0x5c, 0x42, 0xc6, 0x81, - 0x38, 0xb1, 0x26, 0x2f, 0xb8, 0xdd, 0x76, 0x6a, 0x5e, 0x69, 0x16, 0x92, 0xb7, 0x9d, 0x5a, 0xaf, - 0x0a, 0xaa, 0x97, 0xa1, 0x70, 0xdb, 0xa9, 0x45, 0x45, 0x33, 0x0b, 0x99, 0xc7, 0x4e, 0xad, 0x3b, - 0xa3, 0xe9, 0xc7, 0x4e, 0x6d, 0xd5, 0x54, 0x57, 0x60, 0x5a, 0x6a, 0x2a, 0xe4, 0xf1, 0x0a, 0x24, - 0x1f, 0x3b, 0x35, 0xb1, 0xb6, 0xe7, 0x7b, 0x6c, 0x36, 0xfe, 0x55, 0x00, 0xfe, 0x17, 0x02, 0x10, - 0x10, 0x6b, 0x7a, 0xe5, 0x1a, 0x40, 0x37, 0xb9, 0x8d, 0xcc, 0x40, 0xe1, 0xe6, 0x3d, 0xed, 0xde, - 0xc3, 0x8d, 0xd5, 0xbb, 0x2b, 0xfa, 0xfa, 0xc6, 0xc2, 0xd2, 0x9d, 0xf5, 0xc2, 0x09, 0x32, 0x0d, - 0x13, 0x1b, 0xb7, 0xb4, 0x95, 0x85, 0xe5, 0xa0, 0x48, 0xb9, 0xf2, 0x3c, 0x64, 0x83, 0xcc, 0xb1, - 0x6e, 0x56, 0x15, 0x99, 0x04, 0x08, 0xc9, 0xd7, 0x0b, 0xca, 0xd5, 0xbf, 0x7a, 0x01, 0x32, 0xc2, - 0x9b, 0xfb, 0x8e, 0x02, 0xe3, 0xf2, 0x93, 0x5e, 0x52, 0x19, 0xed, 0xd1, 0x6e, 0x20, 0x86, 0x52, - 0x75, 0xe4, 0xf6, 0x5c, 0x16, 0xea, 0xc5, 0x0f, 0xfe, 0xe1, 0xdf, 0x7e, 0x2d, 0xf1, 0x31, 0x52, - 0xae, 0x0a, 0x4f, 0xb3, 0x2a, 0xbf, 0xf8, 0xad, 0xbe, 0x2f, 0x14, 0xe7, 0x29, 0xf9, 0x79, 0x05, - 0xc6, 0x02, 0x0f, 0x38, 0x2e, 0xbf, 0x25, 0xfa, 0x40, 0xb8, 0x74, 0x65, 0x94, 0xa6, 0x02, 0x8b, - 0x8a, 0x58, 0xce, 0x92, 0x52, 0x88, 0xc5, 0xe4, 0x2d, 0x24, 0x18, 0x2e, 0xa4, 0xf1, 0x3d, 0x27, - 0xb9, 0x38, 0xfc, 0xc5, 0x27, 0x47, 0x70, 0x69, 0xd4, 0xa7, 0xa1, 0xea, 0x1c, 0xf6, 0x5f, 0x20, - 0x93, 0x61, 0xff, 0xfc, 0xe9, 0xe9, 0x17, 0x20, 0x85, 0x19, 0x6d, 0x17, 0x86, 0x70, 0x0a, 0x7a, - 0x3c, 0xd4, 0x2b, 0x57, 0xf5, 0x1c, 0xf6, 0x5a, 0x22, 0xc5, 0x68, 0xaf, 0xd2, 0x98, 0x9f, 0xf2, - 0x67, 0x99, 0x98, 0xc5, 0x44, 0x5e, 0x1c, 0x2d, 0xd7, 0xe9, 0x60, 0x24, 0x07, 0x26, 0x46, 0xa9, - 0xb3, 0x88, 0x64, 0x8a, 0x4c, 0x84, 0x48, 0x5c, 0x63, 0xd3, 0x27, 0x5f, 0x52, 0x20, 0xc3, 0x23, - 0x0b, 0x64, 0xe8, 0x93, 0x9c, 0x50, 0xea, 0x97, 0x47, 0x68, 0x29, 0xba, 0xfd, 0x18, 0x76, 0x7b, - 0x86, 0x9c, 0x96, 0xba, 0x65, 0x0d, 0x24, 0x09, 0x78, 0x90, 0xe1, 0xef, 0x2a, 0x62, 0x11, 0x44, - 0x9e, 0x5e, 0x94, 0xe4, 0x5c, 0x5a, 0xf1, 0x47, 0x40, 0xd8, 0xc1, 0x55, 0x48, 0xbd, 0xbf, 0x53, - 0xf1, 0xf7, 0x42, 0xba, 0x9d, 0x7e, 0x5b, 0x81, 0xbc, 0xf4, 0x20, 0x80, 0xbc, 0x3c, 0xda, 0xc3, - 0x81, 0xa0, 0xff, 0xca, 0xa8, 0xcd, 0x85, 0x18, 0x2e, 0x20, 0xa2, 0x73, 0x64, 0x3e, 0x44, 0xc4, - 0x6f, 0x30, 0xd1, 0x53, 0x94, 0x60, 0x7d, 0x53, 0x81, 0x5c, 0x98, 0xb1, 0x1d, 0xab, 0x0e, 0xbd, - 0x79, 0xea, 0xb1, 0xea, 0xd0, 0x97, 0x44, 0xae, 0x5e, 0x46, 0x40, 0xe7, 0xc9, 0xc7, 0x42, 0x40, - 0x46, 0xd0, 0x06, 0x55, 0x54, 0xc2, 0xf4, 0x5d, 0x05, 0x26, 0xa3, 0x19, 0xfd, 0xe4, 0x95, 0x91, - 0xfa, 0x92, 0xc2, 0x50, 0xa5, 0x57, 0x0f, 0x41, 0x21, 0x20, 0xbe, 0x88, 0x10, 0x5f, 0x20, 0xe7, - 0x07, 0x40, 0x44, 0x25, 0xaa, 0xbe, 0x1f, 0x04, 0x94, 0x9e, 0x92, 0xaf, 0x28, 0x30, 0x2e, 0xe7, - 0x34, 0xc4, 0x1a, 0xd7, 0x01, 0xa9, 0x4d, 0xb1, 0xc6, 0x75, 0x50, 0xce, 0x86, 0x7a, 0x1a, 0xe1, - 0x9d, 0x24, 0xd3, 0x21, 0xbc, 0x30, 0x11, 0xe3, 0xd7, 0x45, 0xce, 0x09, 0x3e, 0x05, 0xfb, 0xe8, - 0x10, 0x95, 0x11, 0xd1, 0x69, 0x72, 0x2a, 0x44, 0x84, 0x0f, 0xdb, 0x74, 0x19, 0x57, 0x5e, 0x4a, - 0xb1, 0x88, 0x55, 0xfa, 0xfe, 0xec, 0x8f, 0x58, 0xa5, 0x1f, 0x90, 0xb9, 0x31, 0x68, 0xfb, 0xc1, - 0x56, 0x3c, 0xa9, 0x48, 0xd2, 0xb0, 0xdf, 0x52, 0x60, 0x22, 0x92, 0x36, 0x41, 0xaa, 0x43, 0xbb, - 0x8a, 0xe6, 0x76, 0x94, 0x5e, 0x19, 0x9d, 0xe0, 0xc0, 0x15, 0x20, 0xd0, 0x09, 0x71, 0x49, 0xf8, - 0xbe, 0xa4, 0x40, 0x2e, 0x4c, 0x56, 0x88, 0x5d, 0x95, 0xbd, 0x09, 0x1b, 0xb1, 0xab, 0xb2, 0x2f, - 0xff, 0x41, 0x2d, 0x22, 0x26, 0xa2, 0x76, 0x8d, 0xb4, 0xd7, 0x32, 0xec, 0x37, 0x95, 0x2b, 0xe4, - 0x0b, 0xe8, 0x46, 0xd4, 0xb7, 0xe2, 0xcd, 0x74, 0x24, 0xeb, 0xbf, 0x14, 0xb7, 0x8b, 0xca, 0x4f, - 0x3f, 0x06, 0xd8, 0x4b, 0x0f, 0x19, 0x49, 0x22, 0xf8, 0x59, 0x05, 0xc6, 0x44, 0x76, 0x79, 0xac, - 0x87, 0x10, 0xcd, 0x40, 0x1f, 0x1d, 0x42, 0xbf, 0x7b, 0xd0, 0xe2, 0x9c, 0x7a, 0x30, 0x88, 0x94, - 0xf5, 0x58, 0x0c, 0xd1, 0xb4, 0xf6, 0xe3, 0x60, 0xd8, 0xe6, 0x9c, 0x24, 0x0c, 0xbf, 0xa0, 0x40, - 0x36, 0x78, 0x02, 0x40, 0xe2, 0xfc, 0x9f, 0x9e, 0x57, 0x0c, 0xa5, 0x17, 0x47, 0x6a, 0x2b, 0x90, - 0xf4, 0xbb, 0x0d, 0x18, 0xba, 0x8b, 0xee, 0x5f, 0xe3, 0xf2, 0xa3, 0x93, 0x78, 0xeb, 0xd2, 0xff, - 0x9a, 0x25, 0xde, 0xba, 0x0c, 0x78, 0xcd, 0xa2, 0x9e, 0x47, 0x4c, 0xcf, 0x91, 0x33, 0x92, 0x75, - 0x69, 0xf4, 0xc2, 0xfa, 0xba, 0x02, 0x63, 0x82, 0x3a, 0x76, 0x8a, 0xa2, 0xaf, 0x5b, 0x4a, 0x2f, - 0xc7, 0x37, 0xed, 0x79, 0xdb, 0xa3, 0x5e, 0x41, 0x28, 0xcf, 0x13, 0x35, 0x06, 0x4a, 0xf5, 0x7d, - 0x56, 0xf0, 0x94, 0xf9, 0x77, 0x6b, 0x4e, 0xc3, 0x8b, 0xf5, 0xef, 0xa4, 0x27, 0x4e, 0x87, 0x85, - 0x32, 0xc8, 0xe6, 0x36, 0x64, 0x89, 0x7c, 0x4b, 0xc1, 0xbf, 0xd8, 0xd0, 0xbd, 0x8d, 0x8d, 0xb5, - 0x6d, 0x83, 0x12, 0x8b, 0x62, 0x6d, 0xdb, 0xc0, 0x8b, 0x5e, 0x75, 0x1e, 0x51, 0x15, 0xc9, 0x9c, - 0xbc, 0x9a, 0x58, 0x3b, 0x91, 0xd3, 0xfe, 0x81, 0x02, 0xb9, 0xf0, 0x4a, 0x29, 0xd6, 0xa0, 0xf5, - 0xde, 0x08, 0xc7, 0x1a, 0xb4, 0xbe, 0x5b, 0x2a, 0xb5, 0x84, 0x40, 0x66, 0x08, 0x09, 0x81, 0x3c, - 0x72, 0x7c, 0x01, 0xe2, 0x29, 0xa4, 0xb9, 0x37, 0x71, 0x71, 0xf8, 0x2d, 0xc1, 0x70, 0x6f, 0x3f, - 0xea, 0x3b, 0x1c, 0xe0, 0x76, 0xca, 0x1e, 0xc3, 0x6f, 0x28, 0x90, 0x97, 0x63, 0x21, 0x71, 0x13, - 0xdf, 0x1f, 0x77, 0x18, 0x34, 0x29, 0x91, 0xbf, 0x0d, 0x27, 0xd1, 0xf0, 0xa0, 0xcc, 0x00, 0x1f, - 0x50, 0x22, 0x88, 0xee, 0x36, 0x19, 0x1e, 0x60, 0x1f, 0x62, 0xeb, 0xa5, 0x1b, 0x8e, 0x58, 0x97, - 0x3c, 0x1a, 0xad, 0x1f, 0x68, 0xed, 0x59, 0x03, 0x09, 0xc2, 0x2f, 0x2a, 0x78, 0x26, 0x0e, 0xa2, - 0xda, 0x2f, 0x8d, 0x18, 0x62, 0x1d, 0xbe, 0x82, 0xfa, 0x03, 0xb2, 0xea, 0x19, 0x84, 0x33, 0x4b, - 0x4e, 0xca, 0x9b, 0x4f, 0xd0, 0xf3, 0x0f, 0x15, 0x38, 0x37, 0x2c, 0xb2, 0x47, 0x16, 0xe3, 0xf6, - 0xfe, 0xd1, 0xc2, 0x8e, 0xa5, 0xa5, 0x63, 0xf1, 0x88, 0x9a, 0x48, 0xb5, 0x28, 0x0d, 0x65, 0xdb, - 0x67, 0xb3, 0x2c, 0x22, 0x71, 0x6c, 0x27, 0xff, 0x73, 0xe5, 0xa0, 0xb8, 0x13, 0x22, 0xf1, 0xc8, - 0x5b, 0x47, 0x8a, 0xf9, 0x85, 0xe2, 0x7f, 0xfb, 0xa8, 0xe4, 0x07, 0xee, 0x3d, 0x3d, 0x83, 0x20, - 0x7f, 0x7a, 0x50, 0x38, 0xf1, 0xe3, 0x87, 0xee, 0x9a, 0x43, 0x7e, 0xe3, 0xd0, 0x74, 0x02, 0xeb, - 0xeb, 0x88, 0xb5, 0x42, 0x5e, 0xea, 0xc3, 0x5a, 0x7d, 0xff, 0xa0, 0xa8, 0xe0, 0x53, 0xf2, 0x3d, - 0x05, 0x03, 0x47, 0xd1, 0x40, 0x15, 0x79, 0xed, 0x70, 0x61, 0x2d, 0x8e, 0xfc, 0xf5, 0xa3, 0xc4, - 0xc2, 0x06, 0x38, 0xc6, 0x8f, 0x9d, 0x9a, 0xee, 0x8a, 0xc6, 0x51, 0x6f, 0x23, 0x17, 0x86, 0xb8, - 0x62, 0xed, 0x74, 0x6f, 0xcc, 0x2c, 0xd6, 0x4e, 0xf7, 0x45, 0xcd, 0xd4, 0xe7, 0x10, 0xd1, 0x29, - 0x32, 0x2b, 0x23, 0xaa, 0xbe, 0xcf, 0xa3, 0x6e, 0x4f, 0x17, 0xaf, 0xfc, 0xe0, 0x5f, 0xe7, 0x4f, - 0xfc, 0x60, 0x7f, 0x5e, 0xf9, 0xe1, 0xfe, 0xbc, 0xf2, 0xa3, 0xfd, 0x79, 0xe5, 0x5f, 0xf6, 0xe7, - 0x95, 0x6f, 0xfc, 0x64, 0xfe, 0xc4, 0x0f, 0x7f, 0x32, 0x7f, 0xe2, 0x47, 0x3f, 0x99, 0x3f, 0xf1, - 0x6e, 0x36, 0x60, 0x5e, 0xcb, 0x60, 0xc0, 0xf7, 0xb5, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x50, - 0xa1, 0xe8, 0x97, 0xef, 0x53, 0x00, 0x00, + proto.RegisterFile("server/serverpb/status.proto", fileDescriptor_status_731fb2d638c68f09) +} + +var fileDescriptor_status_731fb2d638c68f09 = []byte{ + // 5866 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x7c, 0xdb, 0x8f, 0x1c, 0xc7, + 0x75, 0x37, 0x7b, 0x6e, 0x3b, 0x73, 0x66, 0x2f, 0xb3, 0xc5, 0xdd, 0xe5, 0x70, 0x48, 0xed, 0xd0, + 0x4d, 0x89, 0x22, 0x29, 0x69, 0x46, 0xa2, 0x44, 0x8b, 0xd6, 0x67, 0xc9, 0xde, 0x1b, 0xc9, 0x25, + 0x57, 0xbc, 0xf4, 0x2e, 0x3f, 0x07, 0xb2, 0xa3, 0x4e, 0xcf, 0x74, 0xed, 0xb0, 0xb9, 0xb3, 0xdd, + 0xc3, 0xee, 0x9e, 0xcd, 0x8e, 0x15, 0xda, 0x8e, 0x72, 0x73, 0x1c, 0xc7, 0xb7, 0x38, 0x81, 0x1f, + 0x12, 0x20, 0xf0, 0x43, 0x9c, 0x97, 0x04, 0x0e, 0xf2, 0x92, 0x04, 0x48, 0x82, 0x5c, 0x90, 0x18, + 0x08, 0x10, 0x18, 0x48, 0x1e, 0x8c, 0x04, 0x58, 0x27, 0xab, 0x3c, 0x04, 0xc8, 0x7f, 0x60, 0x20, + 0x41, 0x50, 0xa7, 0xaa, 0x7b, 0xaa, 0x67, 0x66, 0x7b, 0x66, 0x77, 0x45, 0x21, 0x0f, 0x12, 0xa7, + 0xab, 0xea, 0x9c, 0xfa, 0xd5, 0xa9, 0x53, 0xa7, 0x4e, 0x9d, 0x3a, 0xb5, 0x70, 0xd6, 0xa3, 0xee, + 0x0e, 0x75, 0xab, 0xfc, 0x9f, 0x56, 0xad, 0xea, 0xf9, 0x86, 0xdf, 0xf6, 0x2a, 0x2d, 0xd7, 0xf1, + 0x1d, 0x72, 0xba, 0xee, 0xd4, 0xb7, 0x5c, 0xc7, 0xa8, 0x3f, 0xac, 0xf0, 0x06, 0x95, 0xa0, 0x5d, + 0xa9, 0x50, 0x6b, 0x5b, 0x4d, 0xb3, 0x6a, 0xd9, 0x9b, 0x0e, 0x6f, 0x5c, 0x3a, 0xd9, 0x70, 0x3c, + 0xcf, 0x6a, 0x55, 0xf9, 0x3f, 0xa2, 0x70, 0xee, 0x91, 0x53, 0xf3, 0xaa, 0xec, 0x7f, 0xad, 0x1a, + 0xfe, 0x23, 0xca, 0x4f, 0x21, 0xd7, 0x56, 0xad, 0x6a, 0xb4, 0x5a, 0x3a, 0xeb, 0x33, 0xa8, 0x20, + 0x41, 0x85, 0x69, 0xf8, 0x46, 0xc0, 0x24, 0x28, 0xdb, 0xa6, 0xbe, 0x21, 0x95, 0x5f, 0x10, 0xe0, + 0x4d, 0xcb, 0x68, 0xd8, 0x8e, 0xe7, 0x5b, 0x75, 0xd6, 0x8b, 0xf4, 0x25, 0xda, 0x9d, 0x0f, 0x06, + 0x89, 0x63, 0x13, 0xff, 0xf4, 0x8c, 0xb5, 0xf4, 0x8c, 0xe7, 0x3b, 0xae, 0xd1, 0xa0, 0x55, 0x6a, + 0x37, 0x2c, 0x9b, 0xb6, 0x6a, 0xe2, 0x87, 0xa8, 0x3e, 0xd3, 0x57, 0xbd, 0xbd, 0x53, 0xaf, 0x8b, + 0xca, 0xf9, 0xbe, 0x4a, 0xd7, 0xa9, 0x6f, 0x79, 0x66, 0x4d, 0xd4, 0x5f, 0xda, 0xda, 0xa9, 0x6e, + 0xed, 0x08, 0x14, 0xc1, 0x8f, 0x56, 0xad, 0xda, 0xa4, 0x86, 0x47, 0xf5, 0x08, 0x0c, 0xf5, 0x80, + 0xa6, 0xac, 0x51, 0x80, 0xe5, 0xb9, 0x83, 0xd8, 0x59, 0x3b, 0xd4, 0xa6, 0x5e, 0x28, 0xca, 0xb6, + 0x6f, 0x35, 0xab, 0x4d, 0xa7, 0xc1, 0xfe, 0x13, 0x65, 0x25, 0x2c, 0x6b, 0xdb, 0x2e, 0xf5, 0x9c, + 0xe6, 0x0e, 0x35, 0x75, 0xc3, 0x34, 0xdd, 0x60, 0x88, 0xd4, 0xaf, 0x9b, 0x55, 0xd7, 0xd8, 0xf4, + 0xf1, 0x7f, 0x6c, 0x14, 0xc6, 0xa6, 0x2f, 0x2a, 0x67, 0x1a, 0x4e, 0xc3, 0xc1, 0x9f, 0x55, 0xf6, + 0x4b, 0x94, 0x9e, 0x6d, 0x38, 0x4e, 0xa3, 0x49, 0xab, 0x46, 0xcb, 0xaa, 0x1a, 0xb6, 0xed, 0xf8, + 0x86, 0x6f, 0x39, 0x76, 0x00, 0xa0, 0x2c, 0x6a, 0xf1, 0xab, 0xd6, 0xde, 0xac, 0xfa, 0xd6, 0x36, + 0xf5, 0x7c, 0x63, 0x5b, 0x68, 0x87, 0x5a, 0x81, 0x93, 0x4b, 0xd4, 0xf5, 0xad, 0x4d, 0xab, 0x6e, + 0xf8, 0xd4, 0xd3, 0xe8, 0xe3, 0x36, 0xf5, 0x7c, 0x72, 0x0a, 0xc6, 0x6c, 0xc7, 0xa4, 0xba, 0x65, + 0x16, 0x95, 0x73, 0xca, 0xc5, 0x9c, 0x96, 0x61, 0x9f, 0xab, 0xa6, 0xfa, 0xdf, 0x29, 0x20, 0x12, + 0xc1, 0x32, 0xf5, 0x0d, 0xab, 0xe9, 0x91, 0xfb, 0x90, 0xf2, 0x3b, 0x2d, 0x8a, 0x8d, 0x27, 0xaf, + 0xbc, 0x59, 0x39, 0x50, 0x6b, 0x2b, 0xfd, 0xc4, 0x72, 0xd1, 0x46, 0xa7, 0x45, 0x35, 0x64, 0x45, + 0xce, 0xc3, 0x04, 0x75, 0x5d, 0xc7, 0xd5, 0xb7, 0xa9, 0xe7, 0x19, 0x0d, 0x5a, 0x4c, 0x20, 0x90, + 0x71, 0x2c, 0x7c, 0x9b, 0x97, 0x11, 0x02, 0x29, 0xa6, 0x8d, 0xc5, 0xe4, 0x39, 0xe5, 0xe2, 0xb8, + 0x86, 0xbf, 0x89, 0x06, 0x99, 0x4d, 0x8b, 0x36, 0x4d, 0xaf, 0x98, 0x3a, 0x97, 0xbc, 0x98, 0xbf, + 0xf2, 0xda, 0xe1, 0xd0, 0x5c, 0x47, 0xda, 0xc5, 0xd4, 0x0f, 0xf6, 0xca, 0x27, 0x34, 0xc1, 0xa9, + 0xf4, 0xc7, 0x09, 0xc8, 0xf0, 0x0a, 0x32, 0x07, 0x19, 0xcb, 0xf3, 0xda, 0xd4, 0x0d, 0x24, 0xc3, + 0xbf, 0x48, 0x11, 0xc6, 0xbc, 0x76, 0xed, 0x11, 0xad, 0xfb, 0x02, 0x69, 0xf0, 0x49, 0x9e, 0x01, + 0xd8, 0x31, 0x9a, 0x96, 0xa9, 0x6f, 0xba, 0xce, 0x36, 0x42, 0x4d, 0x6a, 0x39, 0x2c, 0xb9, 0xee, + 0x3a, 0xdb, 0xa4, 0x0c, 0x79, 0x5e, 0xdd, 0xb6, 0x7d, 0xab, 0x59, 0x4c, 0x61, 0x3d, 0xa7, 0x78, + 0xc0, 0x4a, 0xc8, 0x59, 0xc8, 0x31, 0x1d, 0xa1, 0x9e, 0x47, 0xbd, 0x62, 0xfa, 0x5c, 0xf2, 0x62, + 0x4e, 0xeb, 0x16, 0x90, 0x2a, 0x9c, 0xf4, 0xac, 0x86, 0x6d, 0xf8, 0x6d, 0x97, 0xea, 0x46, 0xb3, + 0xe1, 0xb8, 0x96, 0xff, 0x70, 0xbb, 0x98, 0x41, 0x0c, 0x24, 0xac, 0x5a, 0x08, 0x6a, 0x18, 0x9c, + 0x56, 0xbb, 0xd6, 0xb4, 0xea, 0xfa, 0x16, 0xed, 0x14, 0xc7, 0xb0, 0x5d, 0x8e, 0x97, 0xdc, 0xa6, + 0x1d, 0x72, 0x06, 0x72, 0x5b, 0xb4, 0xa3, 0xb7, 0x51, 0xe6, 0x59, 0xec, 0x2d, 0xbb, 0x45, 0x3b, + 0x0f, 0x50, 0xde, 0x2f, 0x02, 0xa1, 0xbb, 0x3e, 0xb5, 0x4d, 0x6a, 0xea, 0xdd, 0x56, 0x39, 0x6c, + 0x55, 0x08, 0x6a, 0x6e, 0x8b, 0xd6, 0xea, 0x7d, 0x98, 0xea, 0x99, 0x5b, 0x92, 0x81, 0xc4, 0xd2, + 0x42, 0xe1, 0x04, 0xc9, 0x42, 0xea, 0xce, 0xdd, 0xe5, 0x95, 0x82, 0x42, 0x26, 0x20, 0xb7, 0xb4, + 0xb6, 0xba, 0x72, 0x67, 0x43, 0x5f, 0x5a, 0x28, 0x24, 0x08, 0x40, 0x86, 0x7f, 0x16, 0x92, 0x24, + 0x07, 0xe9, 0x07, 0xab, 0xac, 0x38, 0xc5, 0xe8, 0x1e, 0xac, 0x16, 0xd2, 0xaa, 0x03, 0x33, 0x51, + 0x7d, 0xf5, 0x5a, 0x8e, 0xed, 0x51, 0xf2, 0x19, 0x18, 0xaf, 0x4b, 0xe5, 0x45, 0x05, 0xa7, 0xfe, + 0xa5, 0x43, 0x4d, 0xbd, 0x98, 0xf3, 0x08, 0x23, 0xb5, 0x0a, 0x93, 0xa2, 0x7a, 0xd8, 0xda, 0xb8, + 0x95, 0xca, 0x26, 0x0a, 0x49, 0xf5, 0x0e, 0xc0, 0x7a, 0xc7, 0xf3, 0xe9, 0xf6, 0xaa, 0xbd, 0xe9, + 0xb0, 0xc9, 0xf5, 0xf0, 0x4b, 0x67, 0x76, 0x5a, 0x10, 0x80, 0x17, 0x69, 0xb0, 0x45, 0x5d, 0x9b, + 0x36, 0x79, 0x03, 0xae, 0x3a, 0xc0, 0x8b, 0x58, 0x03, 0xf5, 0xab, 0x49, 0x98, 0x0a, 0x11, 0x88, + 0xd1, 0xbe, 0x13, 0x85, 0x90, 0x5e, 0x5c, 0xd8, 0xdf, 0x2b, 0x67, 0xee, 0x30, 0x18, 0xcb, 0x3f, + 0xd9, 0x2b, 0xbf, 0xda, 0xb0, 0xfc, 0x87, 0xed, 0x5a, 0xa5, 0xee, 0x6c, 0x57, 0x43, 0x01, 0x98, + 0xb5, 0xee, 0xef, 0x6a, 0x6b, 0xab, 0x51, 0x15, 0x26, 0xbd, 0xc2, 0xc9, 0x82, 0x51, 0x90, 0xb7, + 0x60, 0x4c, 0x28, 0x17, 0x82, 0xc9, 0x5f, 0x99, 0x97, 0x84, 0xc8, 0x6c, 0x57, 0xe5, 0x41, 0x68, + 0xbb, 0x16, 0x4c, 0xd3, 0x15, 0x52, 0x0b, 0x88, 0xc8, 0x1b, 0x00, 0xb8, 0x31, 0xf1, 0xf1, 0x24, + 0x91, 0xc5, 0xac, 0xc4, 0x02, 0x2b, 0x2b, 0x6c, 0x68, 0x82, 0x32, 0x87, 0x25, 0x28, 0x8c, 0xb5, + 0xa8, 0xb4, 0x52, 0x48, 0xfc, 0x5c, 0xcc, 0x24, 0x76, 0x25, 0x2d, 0x98, 0xc9, 0xa2, 0x5d, 0x87, + 0xbc, 0xf7, 0xb8, 0xa9, 0x07, 0xa3, 0x49, 0x8f, 0x34, 0x1a, 0xc2, 0xd8, 0xec, 0xef, 0x95, 0x61, + 0xfd, 0xfe, 0xda, 0x02, 0xa7, 0xd4, 0xc0, 0x7b, 0xdc, 0x14, 0xbf, 0xd5, 0x49, 0x18, 0x67, 0x02, + 0x0b, 0xb4, 0x41, 0xfd, 0x76, 0x12, 0x26, 0x44, 0x81, 0x98, 0x9c, 0x9b, 0x90, 0x66, 0xa2, 0x0c, + 0x74, 0xf0, 0xc5, 0x01, 0xf0, 0xf9, 0x76, 0x13, 0xec, 0x82, 0x38, 0x03, 0xeb, 0xf8, 0x21, 0x46, + 0xc1, 0x19, 0x90, 0xbf, 0x50, 0xe0, 0x64, 0xb0, 0xa3, 0xe8, 0xb5, 0x8e, 0x1e, 0xcc, 0x79, 0x02, + 0x19, 0xbf, 0x15, 0x23, 0x97, 0x08, 0xa2, 0xca, 0x9a, 0xe0, 0xb1, 0xd8, 0xc1, 0xb9, 0x36, 0x57, + 0x6c, 0xdf, 0xed, 0x2c, 0xde, 0x15, 0x23, 0x2d, 0xf4, 0x54, 0x2f, 0xbf, 0xff, 0xe3, 0xa3, 0x69, + 0x50, 0xa1, 0xd9, 0xd3, 0x4f, 0x69, 0x17, 0x66, 0x07, 0xf6, 0x4d, 0x0a, 0x90, 0x64, 0xc6, 0x07, + 0x95, 0x57, 0x63, 0x3f, 0xc9, 0x2a, 0xa4, 0x77, 0x8c, 0x66, 0x9b, 0x9b, 0xf9, 0xc9, 0x2b, 0xaf, + 0x4a, 0x83, 0xdb, 0xda, 0xa9, 0x04, 0x5b, 0x6c, 0x45, 0x6c, 0xf3, 0xa2, 0xd3, 0x80, 0x39, 0x17, + 0x9e, 0xc6, 0x39, 0xbc, 0x91, 0xb8, 0xa6, 0xa8, 0x17, 0x20, 0xcf, 0x1a, 0x0c, 0xdd, 0xcf, 0xbe, + 0x9f, 0x82, 0x9c, 0x66, 0x6c, 0xfa, 0x8c, 0x03, 0x33, 0x6f, 0xe0, 0xd2, 0x56, 0xd3, 0xaa, 0x1b, + 0x41, 0xcb, 0xd4, 0xe2, 0xc4, 0xfe, 0x5e, 0x39, 0xa7, 0xf1, 0xd2, 0xd5, 0x65, 0x2d, 0x27, 0x1a, + 0xac, 0x9a, 0xe4, 0xe3, 0x00, 0x0f, 0x0d, 0xd7, 0x44, 0xef, 0x81, 0x8a, 0xc5, 0x32, 0x5d, 0xe1, + 0x1b, 0x77, 0xe5, 0xa6, 0xe1, 0x9a, 0xc8, 0x34, 0xd0, 0xf2, 0x87, 0x41, 0x01, 0xdb, 0xb4, 0x9a, + 0xd4, 0x30, 0x71, 0x6d, 0xa4, 0x34, 0xfc, 0x4d, 0x66, 0x20, 0xcd, 0xd9, 0xa4, 0x10, 0x1e, 0xff, + 0x60, 0x7b, 0x8a, 0xd1, 0x6a, 0x35, 0x2d, 0x6a, 0xa2, 0xf6, 0xa6, 0xb4, 0xe0, 0x93, 0x6c, 0x40, + 0xb6, 0xe5, 0x3a, 0x0d, 0x54, 0xec, 0x0c, 0xaa, 0xc3, 0x95, 0x18, 0x75, 0x08, 0x47, 0x58, 0xb9, + 0x27, 0x88, 0xb8, 0x0a, 0x70, 0x68, 0x21, 0x27, 0xf2, 0x3c, 0x4c, 0x31, 0x34, 0xba, 0xef, 0x1a, + 0xb6, 0xb7, 0x49, 0x5d, 0x4a, 0x71, 0x7f, 0x48, 0x69, 0x93, 0xac, 0x78, 0x23, 0x2c, 0x2d, 0xfd, + 0xba, 0x02, 0xd9, 0x80, 0x15, 0xc3, 0xbe, 0x6d, 0xf8, 0xf5, 0x87, 0x5c, 0x60, 0x1a, 0xff, 0x60, + 0xa3, 0xb4, 0xe9, 0x2e, 0xdf, 0x0c, 0x53, 0x1a, 0xfe, 0xee, 0x8e, 0x32, 0x29, 0x8f, 0x72, 0x0e, + 0x32, 0x2d, 0xa3, 0xed, 0x51, 0x13, 0x07, 0x9f, 0xd5, 0xc4, 0x17, 0xb9, 0x04, 0x85, 0x16, 0xb5, + 0x4d, 0xcb, 0x6e, 0xe8, 0x9e, 0x6d, 0xb4, 0xbc, 0x87, 0x8e, 0x2f, 0xc4, 0x30, 0x25, 0xca, 0xd7, + 0x45, 0x71, 0xe9, 0x11, 0x4c, 0x44, 0x46, 0x26, 0x2b, 0x58, 0x8a, 0x2b, 0xd8, 0x92, 0xac, 0x60, + 0xf1, 0x5b, 0x43, 0xbf, 0xb8, 0x64, 0xd5, 0xda, 0x4f, 0xc0, 0x84, 0x66, 0xd8, 0x0d, 0x7a, 0xcf, + 0x75, 0x6a, 0x4d, 0xba, 0xed, 0x91, 0x73, 0x90, 0x6f, 0xdb, 0xc6, 0x8e, 0x61, 0x35, 0x8d, 0x5a, + 0x93, 0x3b, 0x41, 0x59, 0x4d, 0x2e, 0x22, 0x57, 0xe1, 0x14, 0x93, 0x20, 0x75, 0x75, 0xdb, 0xf1, + 0x75, 0xee, 0x74, 0x3e, 0x74, 0x9a, 0x26, 0x75, 0x11, 0x4e, 0x56, 0x9b, 0xe1, 0xd5, 0x77, 0x1c, + 0x7f, 0x8d, 0x55, 0xde, 0xc4, 0x3a, 0xf2, 0x2c, 0x4c, 0xda, 0x8e, 0xce, 0x34, 0x4a, 0xe7, 0xf5, + 0x28, 0xb8, 0xac, 0x36, 0x6e, 0x3b, 0x0c, 0xe3, 0x1a, 0x96, 0x91, 0x8b, 0x30, 0xd5, 0xb6, 0x4d, + 0xea, 0x0a, 0xcd, 0xf4, 0x43, 0x41, 0xf6, 0x16, 0x93, 0xd3, 0x90, 0xb5, 0x1d, 0xde, 0x3d, 0x4a, + 0x32, 0xab, 0x8d, 0xd9, 0x0e, 0x76, 0x48, 0xae, 0x41, 0xf1, 0x71, 0xdb, 0xa2, 0x5e, 0x9d, 0xda, + 0xbe, 0x4e, 0x1f, 0xb7, 0x8d, 0xa6, 0xa7, 0xfb, 0x56, 0x7d, 0xcb, 0xb2, 0x1b, 0xe8, 0x4b, 0x64, + 0xb5, 0xb9, 0xb0, 0x7e, 0x05, 0xab, 0x37, 0x78, 0x2d, 0x79, 0x01, 0x08, 0x47, 0xe8, 0x34, 0x74, + 0xdf, 0x71, 0xf4, 0xa6, 0xe1, 0x36, 0xb8, 0xde, 0x64, 0xb5, 0x29, 0x56, 0xb3, 0xe6, 0x34, 0x36, + 0x1c, 0x67, 0x8d, 0x15, 0x93, 0x0b, 0x30, 0xe9, 0xec, 0x44, 0xa0, 0x66, 0xb1, 0x61, 0x4f, 0xa9, + 0xba, 0x05, 0x53, 0x28, 0x63, 0x36, 0x0d, 0x16, 0x9e, 0x24, 0x98, 0xef, 0xf1, 0xb8, 0x4d, 0x5d, + 0x8b, 0x7a, 0x7a, 0x8b, 0xba, 0xba, 0x47, 0xeb, 0x8e, 0xcd, 0x17, 0xa9, 0xa2, 0x15, 0x44, 0xcd, + 0x3d, 0xea, 0xae, 0x63, 0x39, 0xb9, 0x0c, 0xd3, 0x3f, 0xeb, 0x5a, 0x7e, 0xb4, 0x71, 0x02, 0x1b, + 0x4f, 0xf1, 0x8a, 0xb0, 0xad, 0x7a, 0x13, 0xe0, 0x9e, 0x4b, 0x7d, 0xbf, 0xb3, 0xde, 0x32, 0x6c, + 0xe6, 0x00, 0x79, 0xbe, 0xe1, 0xfa, 0x7a, 0xa0, 0x40, 0x39, 0x2d, 0x8b, 0x05, 0xcc, 0x3b, 0x3a, + 0x05, 0x63, 0xd4, 0x46, 0xdf, 0x47, 0x6c, 0xd5, 0x19, 0x6a, 0x33, 0x87, 0xe7, 0x8d, 0xd4, 0x7f, + 0xfe, 0x6e, 0x59, 0x51, 0xbf, 0x9a, 0x65, 0xe6, 0xc4, 0x6e, 0x50, 0xdc, 0x80, 0x3e, 0x05, 0x29, + 0xaf, 0x65, 0xd8, 0xc8, 0x24, 0x7e, 0x1f, 0xeb, 0x76, 0x2f, 0xd6, 0x24, 0x12, 0x92, 0x55, 0x00, + 0x14, 0xad, 0x6c, 0x61, 0x9e, 0x1d, 0x45, 0x71, 0x03, 0xa3, 0xe3, 0x86, 0xa6, 0xed, 0xba, 0x6c, + 0x60, 0xf2, 0x57, 0x2e, 0x0f, 0xb5, 0xaf, 0xe1, 0x30, 0x82, 0x3d, 0x89, 0x2f, 0xd6, 0x6d, 0x98, + 0xf4, 0x9c, 0xb6, 0x5b, 0xa7, 0xe1, 0x6e, 0x94, 0x46, 0x0f, 0xe4, 0xc6, 0xfe, 0x5e, 0x79, 0x7c, + 0x1d, 0x6b, 0x8e, 0xe7, 0x87, 0x8c, 0x7b, 0x5d, 0x26, 0x26, 0x79, 0x0c, 0x53, 0xa2, 0x3b, 0x86, + 0x0c, 0xfb, 0xcb, 0x60, 0x7f, 0xab, 0xfb, 0x7b, 0xe5, 0x09, 0xde, 0xdf, 0x3a, 0xab, 0xc1, 0x0e, + 0x5f, 0x3b, 0x54, 0x87, 0x82, 0x4e, 0x9b, 0xf0, 0x24, 0x36, 0x66, 0xff, 0xc1, 0x63, 0x6c, 0xc0, + 0xc1, 0x63, 0x09, 0x26, 0xc4, 0x2a, 0xb6, 0x18, 0xb0, 0x0e, 0x7a, 0xca, 0xf9, 0x2b, 0x45, 0x49, + 0xac, 0x41, 0x37, 0xb8, 0xbe, 0x02, 0xdf, 0x12, 0x89, 0x6e, 0x72, 0x1a, 0x72, 0x0b, 0x8d, 0x38, + 0xda, 0x90, 0x62, 0x0e, 0xa7, 0xe5, 0x62, 0xec, 0xe4, 0x4a, 0x36, 0x47, 0x32, 0xdd, 0xdc, 0x06, + 0x89, 0xf9, 0xf5, 0x8a, 0xd0, 0x37, 0xbf, 0x03, 0x19, 0x75, 0x17, 0x96, 0x3c, 0xbf, 0x1e, 0xf9, + 0x1c, 0x4c, 0x34, 0x99, 0xfd, 0xa6, 0x9e, 0xde, 0x74, 0xea, 0x46, 0xb3, 0x98, 0x47, 0x7e, 0xaf, + 0x0c, 0xd5, 0x97, 0x35, 0x46, 0xf5, 0xb6, 0x61, 0x1b, 0x0d, 0xea, 0x4a, 0x6a, 0x33, 0x2e, 0xb8, + 0xad, 0x31, 0x66, 0xe4, 0x5d, 0x98, 0x0c, 0xb8, 0x37, 0x9a, 0x4e, 0xcd, 0x68, 0x16, 0xc7, 0x8f, + 0xc7, 0x3e, 0x00, 0x7b, 0x03, 0xb9, 0x91, 0x07, 0x30, 0x2e, 0x9f, 0xe8, 0x8b, 0x13, 0xc8, 0xfd, + 0xc5, 0xe1, 0xdc, 0x19, 0x51, 0xc4, 0x05, 0xcb, 0x37, 0xbb, 0x45, 0xec, 0x04, 0x16, 0x1a, 0xbf, + 0xe2, 0x24, 0x1a, 0xac, 0x6e, 0x01, 0xdb, 0xa5, 0x03, 0x4b, 0x39, 0xc5, 0x8d, 0xaa, 0xf8, 0x54, + 0x7f, 0x4d, 0x11, 0x5b, 0xc5, 0xd0, 0xc3, 0x03, 0x31, 0x20, 0xe7, 0xb2, 0x96, 0xba, 0x65, 0x7a, + 0xe8, 0xe0, 0x25, 0x17, 0x97, 0xf7, 0xf7, 0xca, 0x59, 0xbe, 0x0c, 0x97, 0xbd, 0x43, 0x6b, 0xb7, + 0x20, 0xd4, 0xb2, 0xc8, 0x76, 0xd5, 0xf4, 0xd4, 0x0d, 0x98, 0x0c, 0xc0, 0x08, 0x57, 0x75, 0x11, + 0x32, 0x58, 0x1b, 0xf8, 0xaa, 0xcf, 0x0e, 0xd3, 0x1a, 0x49, 0xf2, 0x82, 0x52, 0xbd, 0x08, 0x13, + 0x37, 0x30, 0xde, 0x34, 0xd4, 0xd7, 0xfa, 0x6e, 0x02, 0xa6, 0x56, 0x30, 0x3c, 0xc3, 0xc4, 0xea, + 0xa1, 0x89, 0x7c, 0x17, 0xb2, 0xe1, 0xc2, 0xe6, 0x47, 0x99, 0xa5, 0xfd, 0xbd, 0xf2, 0xd8, 0x71, + 0x97, 0xf4, 0x98, 0x27, 0x16, 0xf3, 0x26, 0xcc, 0xb1, 0xc9, 0xa0, 0xae, 0xa7, 0x1b, 0xb6, 0xc9, + 0x57, 0x6b, 0xc3, 0x35, 0xb6, 0x83, 0xc3, 0xcd, 0xcb, 0xf2, 0x88, 0xb9, 0x3a, 0x54, 0x82, 0x10, + 0x52, 0x65, 0x83, 0x53, 0x2e, 0xd8, 0xe6, 0xcd, 0x90, 0x4e, 0x9b, 0xf1, 0x07, 0x94, 0x92, 0x1b, + 0x90, 0xe7, 0x64, 0x3a, 0xc6, 0x41, 0x92, 0xe8, 0xc4, 0x5e, 0x88, 0x63, 0xce, 0x25, 0x81, 0x01, + 0x0f, 0xa0, 0xe1, 0x6f, 0xf5, 0x25, 0x20, 0x92, 0x8c, 0x86, 0xca, 0xf4, 0xa7, 0xe1, 0x64, 0xa4, + 0xb9, 0x98, 0xd8, 0xd0, 0x1a, 0xf0, 0x79, 0x8d, 0xb3, 0x06, 0x3d, 0x33, 0x12, 0xb1, 0x06, 0xea, + 0xcf, 0x00, 0x6c, 0xb8, 0x46, 0x9d, 0xae, 0xec, 0x30, 0x45, 0xbf, 0x06, 0x29, 0xdf, 0xda, 0xa6, + 0x62, 0x3f, 0x2b, 0x55, 0x78, 0x70, 0xa9, 0x12, 0x04, 0x97, 0x2a, 0x1b, 0x41, 0x70, 0x69, 0x31, + 0xcb, 0x98, 0x7c, 0xe3, 0xc7, 0x65, 0x45, 0x43, 0x0a, 0xb6, 0x44, 0xa2, 0x61, 0x9c, 0xe0, 0x53, + 0xfd, 0xbe, 0x02, 0x53, 0x0b, 0x4d, 0x66, 0x6a, 0x7c, 0xc7, 0x5d, 0x76, 0x3b, 0x5a, 0xdb, 0x66, + 0x4a, 0x11, 0xac, 0x05, 0xec, 0x2b, 0xc9, 0x95, 0x42, 0x68, 0xf4, 0x91, 0x57, 0xc2, 0x98, 0x58, + 0x09, 0xe4, 0x4d, 0xc8, 0x50, 0x36, 0x20, 0x4f, 0x9c, 0xa4, 0xe2, 0x76, 0xe6, 0xee, 0xf0, 0x35, + 0x41, 0xa4, 0x5e, 0x81, 0xd9, 0x10, 0x31, 0xf2, 0x0e, 0x66, 0xe9, 0x74, 0x2f, 0xee, 0xb0, 0x4b, + 0xf5, 0x4f, 0x15, 0x98, 0xeb, 0x25, 0x1a, 0x7c, 0x98, 0x4f, 0x7e, 0x98, 0x87, 0xf9, 0x25, 0x18, + 0x33, 0xdd, 0x8e, 0xee, 0xb6, 0x6d, 0xa1, 0xef, 0x71, 0x9a, 0xd0, 0x33, 0x0d, 0x5a, 0xc6, 0xc4, + 0x7f, 0xd5, 0xaf, 0x29, 0x50, 0xe8, 0x62, 0xff, 0x3f, 0x60, 0xc8, 0xde, 0x81, 0x69, 0x09, 0x8f, + 0x10, 0xe3, 0x0a, 0x64, 0xc5, 0x50, 0x47, 0xd1, 0xfa, 0xde, 0xb1, 0x8e, 0xf1, 0xb1, 0x7a, 0xaa, + 0x0a, 0xe3, 0xb7, 0xd6, 0xef, 0xde, 0x09, 0xd9, 0x06, 0x11, 0x46, 0xa5, 0x1b, 0x61, 0x54, 0x3f, + 0x50, 0x20, 0xbf, 0xe6, 0x34, 0x86, 0x1b, 0xf5, 0x19, 0x48, 0x37, 0xe9, 0x0e, 0x6d, 0x0a, 0xa5, + 0xe7, 0x1f, 0xe4, 0x19, 0x00, 0xee, 0x60, 0xe2, 0x62, 0xe2, 0x47, 0x21, 0xee, 0x72, 0xb2, 0x05, + 0xc4, 0xb4, 0x88, 0xb9, 0x98, 0x58, 0xc9, 0x4f, 0x83, 0xcc, 0xe5, 0xc4, 0xaa, 0x02, 0x24, 0xb7, + 0x8d, 0x5d, 0xf4, 0xb8, 0x72, 0x1a, 0xfb, 0xc9, 0x16, 0x56, 0xcb, 0xf0, 0x7d, 0xea, 0xda, 0x22, + 0xe2, 0x17, 0x7c, 0xb2, 0x53, 0x95, 0x4b, 0x4d, 0xa3, 0xee, 0x0b, 0x57, 0x5c, 0x7c, 0xb1, 0x33, + 0xde, 0x16, 0xa5, 0x2d, 0x9d, 0x7f, 0xe2, 0x81, 0x45, 0xb8, 0xe0, 0xac, 0x58, 0x0b, 0x4b, 0xd5, + 0xbb, 0x40, 0xd6, 0x9c, 0x06, 0x3b, 0x4e, 0x59, 0xd2, 0x96, 0xf1, 0x09, 0xe6, 0x00, 0x63, 0x91, + 0x90, 0xf2, 0xe9, 0xde, 0x80, 0x4a, 0xd3, 0x69, 0x54, 0xe4, 0xe3, 0x65, 0xd0, 0x5e, 0xad, 0xc0, + 0xc9, 0x35, 0xa7, 0x71, 0xdd, 0x6a, 0x52, 0x6f, 0xcd, 0xf2, 0xfc, 0xa1, 0xb6, 0xed, 0x1e, 0xcc, + 0x44, 0xdb, 0x0b, 0x08, 0xd7, 0x20, 0xbd, 0xc9, 0x0a, 0x05, 0x80, 0xb3, 0x83, 0x00, 0x30, 0x2a, + 0xd9, 0x9c, 0x21, 0x81, 0xfa, 0x05, 0x98, 0x14, 0x1c, 0x87, 0x4e, 0x1d, 0x81, 0x14, 0xa3, 0x11, + 0x33, 0x87, 0xbf, 0x25, 0x91, 0x26, 0x87, 0x89, 0x34, 0x35, 0x50, 0xa4, 0x75, 0x98, 0x58, 0xf7, + 0x8d, 0xfa, 0xd6, 0x70, 0xcd, 0xf9, 0x84, 0x08, 0xa8, 0xf3, 0x68, 0x48, 0x6c, 0x08, 0x0c, 0x19, + 0x76, 0x03, 0xe7, 0xea, 0x3a, 0xa4, 0xd8, 0x08, 0xf1, 0x00, 0x6e, 0x08, 0x6b, 0x9d, 0xd3, 0xf0, + 0x37, 0x3b, 0xdb, 0xb0, 0x91, 0xe8, 0x9e, 0xf5, 0x79, 0xce, 0x3b, 0xa9, 0x65, 0x59, 0xc1, 0xba, + 0xf5, 0x79, 0x4a, 0x4a, 0x90, 0xad, 0x3b, 0xb6, 0x8f, 0x86, 0x91, 0x07, 0xd4, 0xc3, 0x6f, 0xf5, + 0xb7, 0x15, 0x98, 0xba, 0x41, 0x7d, 0x9c, 0x8c, 0xa1, 0xe0, 0xcf, 0x40, 0xae, 0x69, 0x79, 0xbe, + 0xee, 0xd8, 0xcd, 0x8e, 0x38, 0xdf, 0x66, 0x59, 0xc1, 0x5d, 0xbb, 0xd9, 0x21, 0xaf, 0x8b, 0x91, + 0xa5, 0x71, 0x64, 0xe7, 0x63, 0x46, 0xc6, 0x3a, 0x93, 0x2e, 0x04, 0x4a, 0x90, 0x15, 0xba, 0xcd, + 0x43, 0x1e, 0x39, 0x2d, 0xfc, 0x56, 0x57, 0xa1, 0xd0, 0x45, 0x27, 0xd4, 0xe4, 0x6a, 0x54, 0x4d, + 0xca, 0x43, 0x7a, 0x0a, 0x74, 0xe4, 0x8b, 0x30, 0x79, 0xcf, 0x75, 0x36, 0x47, 0xd1, 0x91, 0xc5, + 0xc8, 0x50, 0x2a, 0xb1, 0xe7, 0x3b, 0x99, 0x63, 0x45, 0x9a, 0xad, 0x02, 0xa4, 0x30, 0x30, 0x9e, + 0x85, 0xd4, 0xcd, 0x95, 0x85, 0x7b, 0x85, 0x13, 0xea, 0x25, 0x98, 0x7c, 0x9b, 0xfa, 0xae, 0x55, + 0x1f, 0xbe, 0xfb, 0xff, 0x01, 0xfa, 0x97, 0x9b, 0x3e, 0x9a, 0x48, 0x66, 0xfa, 0x9f, 0x6a, 0x64, + 0xf8, 0xd3, 0x90, 0x46, 0x13, 0x3c, 0xd2, 0x41, 0xb4, 0xe7, 0xf0, 0x88, 0x84, 0xea, 0x65, 0xe6, + 0x81, 0x0a, 0xb8, 0x2b, 0xec, 0x38, 0x25, 0x3b, 0x06, 0x4a, 0xd4, 0x31, 0xf8, 0x52, 0x02, 0xa6, + 0xc2, 0xc6, 0xc2, 0x0f, 0x7f, 0xda, 0x8e, 0xc1, 0x0d, 0xc8, 0xe0, 0x29, 0x2f, 0x70, 0x0c, 0x2e, + 0x0d, 0x39, 0x6b, 0x77, 0x07, 0x12, 0x38, 0xc5, 0x9c, 0x9c, 0x2c, 0x07, 0x31, 0xe0, 0x24, 0xf2, + 0xb9, 0x38, 0x0a, 0x1f, 0x26, 0xed, 0x48, 0xfc, 0x57, 0x6d, 0x43, 0x81, 0xd5, 0x2e, 0xd3, 0x5a, + 0xbb, 0x11, 0xe8, 0x42, 0x64, 0x7b, 0x55, 0x9e, 0xca, 0xf6, 0xfa, 0xcf, 0x09, 0x98, 0x96, 0xfa, + 0x15, 0xcb, 0xe9, 0x6b, 0x4a, 0xcf, 0x61, 0xe1, 0xda, 0x90, 0x41, 0x45, 0xc8, 0x79, 0x37, 0x22, + 0xec, 0xf8, 0x49, 0x36, 0xc8, 0xf7, 0x7f, 0x7c, 0x44, 0xa0, 0x02, 0xc5, 0x87, 0x36, 0x59, 0x25, + 0x0a, 0x79, 0x09, 0x9d, 0x1c, 0x3a, 0x4c, 0xf2, 0xd0, 0xe1, 0xa7, 0xa3, 0xa1, 0xc3, 0xcb, 0xa3, + 0x74, 0xd4, 0x1f, 0x92, 0xfe, 0x2b, 0x05, 0xc6, 0x36, 0x76, 0x6d, 0x3c, 0xf6, 0xdc, 0x87, 0x84, + 0x50, 0xe1, 0xf1, 0xc5, 0x05, 0x06, 0xe6, 0x5f, 0x46, 0x5d, 0x9b, 0xfc, 0xf6, 0xb8, 0x6d, 0x99, + 0x95, 0x07, 0x0f, 0x56, 0xd9, 0xcc, 0x27, 0x56, 0x97, 0xb5, 0x84, 0x65, 0x92, 0x37, 0xd0, 0xe5, + 0x77, 0x7d, 0x01, 0x72, 0x34, 0xef, 0x9c, 0x93, 0xb0, 0x0d, 0xcc, 0xdf, 0xb5, 0x75, 0x93, 0x7a, + 0x75, 0xd7, 0x6a, 0xf9, 0x96, 0x63, 0x0b, 0xb7, 0x64, 0xd2, 0xdf, 0xb5, 0x97, 0xbb, 0xa5, 0xea, + 0xff, 0x24, 0x20, 0xbf, 0x50, 0xf7, 0xad, 0x1d, 0x7a, 0xbf, 0x4d, 0xdd, 0x0e, 0x99, 0x0b, 0xc7, + 0x91, 0x5b, 0xcc, 0x48, 0x60, 0x0a, 0x90, 0xf4, 0x1e, 0x07, 0x6e, 0x0f, 0xfb, 0xd9, 0x85, 0x97, + 0x3c, 0x3c, 0xbc, 0xe7, 0x60, 0xd2, 0xf2, 0x74, 0xd3, 0xf2, 0x7c, 0xd7, 0xaa, 0xb5, 0xbb, 0xf1, + 0xcd, 0x09, 0xcb, 0x5b, 0xee, 0x16, 0x92, 0x45, 0x48, 0xb7, 0x1e, 0x06, 0xa1, 0xcd, 0xc9, 0x81, + 0x17, 0x2f, 0xa1, 0xfb, 0xd7, 0x1d, 0x43, 0xe5, 0x1e, 0xa3, 0xd1, 0x38, 0x29, 0x6e, 0x32, 0xdd, + 0xb8, 0xba, 0x72, 0x31, 0x21, 0x45, 0xc7, 0x3f, 0x0b, 0x19, 0x26, 0x25, 0xcb, 0x44, 0x8f, 0x6a, + 0x7c, 0x71, 0xf9, 0x78, 0x13, 0x97, 0x66, 0xca, 0xb0, 0xac, 0xa5, 0xfd, 0x5d, 0x7b, 0xd5, 0x54, + 0x9f, 0x83, 0x34, 0x02, 0x21, 0x13, 0x90, 0xbb, 0xa7, 0xad, 0xdc, 0x5b, 0xd0, 0x56, 0xef, 0xdc, + 0x28, 0x9c, 0x60, 0x9f, 0x2b, 0x3f, 0xb5, 0xb2, 0xf4, 0x60, 0x83, 0x7d, 0x2a, 0xea, 0x2b, 0x70, + 0x92, 0xf9, 0x42, 0xeb, 0xd4, 0xf3, 0x2c, 0xc7, 0x0e, 0x77, 0x88, 0x12, 0x64, 0xdb, 0x1e, 0x75, + 0xa5, 0xfd, 0x3e, 0xfc, 0x56, 0xbf, 0x9d, 0x86, 0x31, 0xd1, 0xfe, 0xa9, 0x6e, 0x0f, 0x32, 0x86, + 0x44, 0x14, 0x03, 0x9b, 0xc1, 0x7a, 0xd3, 0xa2, 0xb6, 0x1f, 0xde, 0xc6, 0x71, 0xfd, 0x9a, 0xe0, + 0xa5, 0xe2, 0x72, 0x8d, 0x5c, 0x82, 0x02, 0x5e, 0x70, 0xd4, 0x31, 0x89, 0x41, 0x47, 0x56, 0xdc, + 0x05, 0x9e, 0x92, 0xca, 0xef, 0x30, 0x8e, 0xeb, 0x30, 0x69, 0xe0, 0x24, 0xea, 0x22, 0xf4, 0x8b, + 0x37, 0xe3, 0xf9, 0xe8, 0x99, 0xfb, 0xe0, 0x59, 0x0f, 0xc2, 0x47, 0x46, 0x58, 0x64, 0x51, 0xaf, + 0xab, 0xa4, 0x99, 0xc3, 0x2b, 0xe9, 0xbb, 0x90, 0xdb, 0xda, 0xd1, 0x23, 0x0a, 0xb2, 0x78, 0x74, + 0xe5, 0x18, 0xbb, 0xbd, 0xc3, 0xd5, 0x63, 0x6c, 0x0b, 0x7f, 0x60, 0x40, 0xbb, 0x69, 0x78, 0xbe, + 0x2e, 0x8d, 0xba, 0x83, 0x9e, 0x7b, 0x4e, 0x9b, 0x62, 0x15, 0xfd, 0xcb, 0x32, 0x87, 0x20, 0xe4, + 0x65, 0x59, 0x86, 0xbc, 0xc1, 0x0e, 0x3e, 0x7a, 0xad, 0xe3, 0x53, 0x1e, 0x2a, 0x4c, 0x6a, 0x80, + 0x45, 0x8b, 0xac, 0x84, 0x5c, 0x80, 0xa9, 0x6d, 0x63, 0x57, 0x97, 0x1b, 0xe5, 0xb1, 0xd1, 0xc4, + 0xb6, 0xb1, 0xbb, 0xd0, 0x6d, 0xb7, 0x00, 0x20, 0x70, 0xf8, 0xbb, 0xb6, 0x88, 0xe1, 0xa9, 0x71, + 0xa7, 0x68, 0x6e, 0xf7, 0xb4, 0x1c, 0xa7, 0xda, 0xd8, 0xb5, 0xd5, 0x5f, 0x55, 0x60, 0x5a, 0x56, + 0x65, 0xee, 0x0f, 0x3c, 0x4d, 0x05, 0x3d, 0x38, 0x08, 0xf1, 0xfb, 0x0a, 0xcc, 0x44, 0x97, 0x95, + 0xd8, 0xf4, 0x96, 0x21, 0xeb, 0x89, 0x32, 0xb1, 0xeb, 0xc5, 0x8d, 0x52, 0x90, 0x07, 0xb1, 0xd9, + 0x80, 0x92, 0xdc, 0xea, 0xd9, 0xa9, 0xe2, 0x2c, 0x53, 0x9f, 0x48, 0xa2, 0x9b, 0x95, 0xfa, 0x18, + 0xc8, 0x92, 0x61, 0xd7, 0x69, 0x13, 0x67, 0x7a, 0xa8, 0x8b, 0x7a, 0x01, 0xb2, 0xa8, 0x29, 0xfc, + 0xda, 0x98, 0x99, 0xe9, 0x3c, 0xd3, 0x2e, 0x24, 0x66, 0xda, 0x85, 0x95, 0x3d, 0x8b, 0x37, 0xd9, + 0x63, 0x40, 0x6e, 0xc0, 0xc9, 0x48, 0x97, 0x42, 0x36, 0xec, 0xb8, 0x80, 0xc5, 0xd4, 0x14, 0x57, + 0x5e, 0xe1, 0x37, 0x3b, 0xf8, 0x22, 0xde, 0xe0, 0xe0, 0x8b, 0x1f, 0x6a, 0x07, 0x66, 0x38, 0x23, + 0x31, 0xc0, 0xa1, 0xe8, 0x5f, 0x04, 0x10, 0x42, 0x0c, 0xf0, 0x8f, 0xf3, 0xfb, 0x58, 0xc1, 0x60, + 0x75, 0x59, 0xcb, 0x89, 0x06, 0x43, 0xc6, 0xb0, 0x0a, 0xb3, 0x3d, 0x5d, 0x1f, 0x79, 0x14, 0xff, + 0xaa, 0x40, 0x61, 0xbd, 0x65, 0xd8, 0x91, 0x00, 0xdd, 0xf9, 0x9e, 0x21, 0x2c, 0x42, 0x57, 0x6f, + 0xc3, 0xe1, 0x68, 0xf2, 0xcd, 0x12, 0x1f, 0xcd, 0xd5, 0x9f, 0xec, 0x95, 0x5f, 0x39, 0x9c, 0x1b, + 0x74, 0x9b, 0x76, 0xa4, 0x0b, 0xa9, 0x3b, 0xdd, 0x0b, 0xa9, 0xe4, 0x71, 0x38, 0x8a, 0x7b, 0x2c, + 0xf5, 0x4f, 0x14, 0x98, 0x96, 0x46, 0x27, 0xa4, 0xb4, 0x06, 0x79, 0xdf, 0xf1, 0x8d, 0xa6, 0x1e, + 0x44, 0x15, 0xfb, 0x2e, 0xb4, 0x7a, 0xc3, 0x9b, 0x6f, 0xff, 0xff, 0xa5, 0x25, 0xe4, 0x11, 0x24, + 0x66, 0x20, 0x3d, 0x96, 0x30, 0x33, 0xc4, 0x7d, 0xd8, 0xba, 0xd3, 0xb6, 0xb9, 0xc3, 0x92, 0xd6, + 0x00, 0x8b, 0x96, 0x58, 0x09, 0x79, 0x0d, 0xe6, 0x8c, 0x56, 0xcb, 0x75, 0x76, 0xad, 0x6d, 0xc3, + 0xa7, 0x6c, 0xe7, 0xdf, 0x12, 0xd6, 0x88, 0xdf, 0x99, 0xcf, 0x48, 0xb5, 0xcb, 0x96, 0xb7, 0x85, + 0x46, 0x49, 0xfd, 0x7f, 0x30, 0x23, 0xae, 0x47, 0xa2, 0x31, 0xf7, 0x51, 0xe6, 0x46, 0xfd, 0xce, + 0x04, 0xcc, 0xf6, 0x50, 0xf7, 0xc7, 0xe7, 0xb2, 0x1f, 0xb6, 0x49, 0xfa, 0x3b, 0x05, 0x4e, 0x06, + 0x57, 0x38, 0x72, 0x86, 0x47, 0x0e, 0xed, 0xc4, 0xf5, 0xf8, 0x13, 0x65, 0x3f, 0xd6, 0x4a, 0x78, + 0x3d, 0x34, 0x38, 0xd3, 0xa3, 0xa7, 0xfa, 0xe8, 0x99, 0x1e, 0xad, 0x9e, 0x7e, 0x4a, 0xff, 0x98, + 0xe3, 0x79, 0x31, 0xe1, 0x9d, 0x78, 0xdf, 0x2d, 0x9a, 0x32, 0xe0, 0x16, 0xed, 0x17, 0x14, 0x98, + 0x95, 0xae, 0xc9, 0xf5, 0xde, 0xc0, 0xe1, 0xdd, 0xfd, 0xbd, 0xf2, 0xc9, 0x07, 0xdd, 0x06, 0xc7, + 0x3e, 0xe4, 0x9c, 0x6c, 0xf7, 0x32, 0x33, 0x3d, 0xf2, 0x87, 0x0a, 0x5c, 0x90, 0xee, 0xd8, 0xfb, + 0xae, 0xe8, 0x25, 0x58, 0x49, 0x84, 0xf5, 0xb9, 0xfd, 0xbd, 0xf2, 0xb9, 0xee, 0x05, 0x7c, 0xf4, + 0xd2, 0xfe, 0xd8, 0x18, 0xcf, 0xb9, 0xb1, 0x9c, 0x4d, 0x8f, 0x7c, 0x59, 0x81, 0x62, 0x34, 0x2f, + 0x40, 0x82, 0x98, 0x42, 0x88, 0xf7, 0xf6, 0xf7, 0xca, 0x33, 0x77, 0xa4, 0x2c, 0x81, 0x63, 0xc3, + 0x9a, 0xb1, 0xfb, 0xb8, 0x99, 0x1e, 0xd9, 0x05, 0x12, 0x64, 0x14, 0x48, 0x18, 0xd2, 0x88, 0xe1, + 0xf6, 0xfe, 0x5e, 0x79, 0xea, 0x0e, 0xcf, 0x2f, 0x38, 0x76, 0xf7, 0x53, 0xb6, 0xcc, 0xc8, 0xf4, + 0xc8, 0xd7, 0x15, 0x38, 0xdd, 0x93, 0xdf, 0x20, 0x21, 0xc8, 0x20, 0x82, 0xf5, 0xfd, 0xbd, 0xf2, + 0xa9, 0x07, 0xd1, 0x46, 0xc7, 0x46, 0x72, 0xaa, 0x3d, 0x88, 0xa1, 0xe9, 0x91, 0xdf, 0x53, 0x40, + 0x3d, 0x28, 0x87, 0x42, 0x82, 0x36, 0x86, 0xd0, 0xde, 0xd9, 0xdf, 0x2b, 0xcf, 0xdf, 0x1f, 0x98, + 0x51, 0x71, 0x6c, 0x84, 0xf3, 0x8f, 0x63, 0xf8, 0x9a, 0x1e, 0xf9, 0x96, 0x02, 0x67, 0xfb, 0x53, + 0x36, 0x24, 0x88, 0xd9, 0xae, 0xf4, 0xb4, 0x68, 0x02, 0xc7, 0xf1, 0xa5, 0xe7, 0x0e, 0x62, 0x68, + 0x7a, 0xe4, 0xab, 0x0a, 0x14, 0xa3, 0x49, 0x20, 0x12, 0xa0, 0x1c, 0x02, 0xd2, 0xf6, 0xf7, 0xca, + 0x73, 0x77, 0x77, 0x3e, 0xd4, 0xd9, 0x9c, 0x73, 0x76, 0x06, 0x4d, 0x66, 0xe9, 0x7d, 0x25, 0xdc, + 0x0f, 0x86, 0xe6, 0xae, 0xad, 0x47, 0xe3, 0x03, 0x6f, 0x1e, 0xda, 0x6c, 0xcb, 0x96, 0x53, 0x0a, + 0x19, 0xdc, 0x4a, 0x65, 0x95, 0x42, 0x56, 0x7d, 0x1d, 0x0a, 0x37, 0x1d, 0xff, 0x08, 0x7b, 0xda, + 0x57, 0xc6, 0x60, 0x5a, 0xa2, 0xfc, 0x08, 0x92, 0x47, 0xff, 0x5e, 0x81, 0xd9, 0x87, 0x8e, 0xcf, + 0x67, 0x6e, 0x40, 0xce, 0xe2, 0x52, 0x8c, 0x68, 0xfa, 0x90, 0x76, 0x4b, 0xa2, 0xdb, 0xd9, 0x3d, + 0xb1, 0x9d, 0x4d, 0xf7, 0xd6, 0x1f, 0x79, 0x3f, 0x9b, 0x7e, 0xd8, 0xdb, 0x53, 0x69, 0x07, 0xb2, + 0x01, 0x7b, 0xf2, 0x49, 0x48, 0x99, 0xd4, 0xab, 0x0b, 0xb7, 0x47, 0x1d, 0x90, 0xe3, 0x81, 0xed, + 0x82, 0x40, 0x49, 0xe8, 0xb3, 0x23, 0xd5, 0x01, 0x79, 0x4b, 0x89, 0xc1, 0x79, 0x4b, 0xa5, 0x7f, + 0x50, 0x60, 0x02, 0x6f, 0xb1, 0xc3, 0xf9, 0x7a, 0xda, 0x57, 0xe4, 0xef, 0x00, 0x74, 0xa7, 0x4c, + 0xcc, 0xd3, 0xd5, 0x23, 0xcd, 0x53, 0x98, 0xea, 0x18, 0xb4, 0x28, 0xfd, 0x8a, 0xc2, 0xdd, 0x82, + 0x70, 0x30, 0x23, 0xb9, 0x05, 0x1a, 0x64, 0x10, 0x5c, 0x80, 0xe6, 0x8d, 0x43, 0xa1, 0x89, 0x48, + 0x4f, 0x13, 0x9c, 0x4a, 0x5f, 0x84, 0xb9, 0xc1, 0xea, 0x34, 0x60, 0x3d, 0xdf, 0x8d, 0xae, 0xe7, + 0x4f, 0x1c, 0xaa, 0x7b, 0x79, 0xb8, 0x72, 0xf8, 0xef, 0x12, 0x8c, 0x8f, 0x7a, 0x59, 0xfc, 0xbd, + 0xb4, 0x48, 0x1b, 0xf9, 0x48, 0xd6, 0xac, 0x1c, 0x54, 0x4f, 0x3c, 0x85, 0xa0, 0xfa, 0x5f, 0x2b, + 0x30, 0xe3, 0x8a, 0x81, 0x44, 0x4c, 0x02, 0x8f, 0x8d, 0x7f, 0x6a, 0xd8, 0x35, 0x42, 0x37, 0x84, + 0x1c, 0x30, 0x39, 0xc0, 0x1c, 0xf4, 0xd6, 0x1f, 0xdd, 0x1c, 0xb8, 0xbd, 0x3d, 0x95, 0xbe, 0xd9, + 0xab, 0xc8, 0x25, 0xc8, 0x06, 0xad, 0x82, 0x73, 0xa3, 0x7b, 0xa0, 0x92, 0x0f, 0x7a, 0xba, 0xf2, + 0x69, 0x48, 0x5b, 0xf6, 0xa6, 0x13, 0x5c, 0x11, 0x1c, 0xea, 0x36, 0x05, 0x09, 0x4b, 0xef, 0xc1, + 0xdc, 0x60, 0x91, 0x0c, 0x50, 0xe9, 0xdb, 0x51, 0x95, 0xbe, 0x3a, 0xb2, 0xd0, 0x0f, 0x50, 0xe7, + 0x5b, 0xa9, 0x6c, 0xaa, 0x90, 0x56, 0x5f, 0x02, 0xb2, 0xdc, 0x7d, 0xec, 0x35, 0xf4, 0xbe, 0xea, + 0xa2, 0xb0, 0x6d, 0xc3, 0x5b, 0xfe, 0x51, 0x02, 0xc6, 0xb1, 0x69, 0xf0, 0xc2, 0xe8, 0x69, 0x5b, + 0xc1, 0x17, 0x60, 0x9a, 0xda, 0x75, 0xb7, 0x83, 0x71, 0xee, 0x20, 0x7d, 0x0c, 0xcf, 0xe8, 0x5a, + 0xa1, 0x5b, 0x21, 0xee, 0xa1, 0xca, 0xc1, 0x71, 0x98, 0x5f, 0x30, 0xf2, 0x43, 0x29, 0x3f, 0xe1, + 0xe2, 0x1d, 0x64, 0xb7, 0x01, 0x3f, 0xb5, 0xa6, 0xa4, 0x06, 0x3c, 0x80, 0x76, 0x11, 0x0a, 0x22, + 0x80, 0xb6, 0x45, 0x3b, 0x82, 0x0d, 0xcf, 0x6d, 0x16, 0x61, 0xcd, 0xdb, 0xb4, 0xc3, 0x59, 0x45, + 0x5b, 0x72, 0x7e, 0x99, 0x9e, 0x96, 0xfc, 0xfc, 0xfb, 0x19, 0x98, 0x0c, 0xa4, 0x1b, 0xe6, 0x44, + 0x04, 0x86, 0x94, 0x07, 0xaf, 0x9e, 0x8f, 0xbd, 0x47, 0xee, 0x4a, 0x3b, 0x88, 0x39, 0x71, 0x62, + 0xf5, 0x1a, 0x4c, 0x63, 0x12, 0xe9, 0x36, 0xb5, 0x0f, 0x17, 0xf1, 0x50, 0xbf, 0x95, 0x06, 0x22, + 0x93, 0x0a, 0x5c, 0x2d, 0xcc, 0x80, 0x10, 0xa5, 0x02, 0xdb, 0xad, 0xf8, 0x3b, 0xee, 0x1e, 0x16, + 0x95, 0x25, 0xa7, 0xd9, 0xa4, 0x75, 0x9f, 0x9a, 0x61, 0x5d, 0x5f, 0x46, 0xa3, 0xd4, 0x07, 0x59, + 0x02, 0xc0, 0xe8, 0xa9, 0x4b, 0x3d, 0x7a, 0xb8, 0x3b, 0x88, 0x1c, 0xa3, 0xd3, 0x18, 0x19, 0x79, + 0x1d, 0x8a, 0x96, 0xed, 0x53, 0xd7, 0x36, 0x9a, 0xba, 0xd1, 0x6a, 0x61, 0x7c, 0x5a, 0x6f, 0xb9, + 0x74, 0xd3, 0xda, 0x15, 0x61, 0xea, 0xd9, 0xa0, 0x7e, 0xa1, 0xd5, 0xba, 0x63, 0x6c, 0xd3, 0x7b, + 0x58, 0x59, 0xfa, 0x5b, 0x05, 0xe6, 0x57, 0xc4, 0xeb, 0xa8, 0x01, 0x78, 0x6f, 0xd3, 0x0e, 0xb9, + 0x0e, 0x59, 0x36, 0xbf, 0x61, 0xae, 0x49, 0x4f, 0x78, 0xe5, 0x71, 0xb3, 0x32, 0x98, 0x30, 0x48, + 0xb2, 0xd8, 0xa2, 0x9d, 0x65, 0xc3, 0x37, 0xe4, 0x9d, 0x22, 0xf1, 0x21, 0xef, 0x14, 0x6c, 0x18, + 0x67, 0xe3, 0xe4, 0x4e, 0xcc, 0xae, 0xdd, 0xc9, 0x5f, 0x59, 0x3b, 0xdc, 0x84, 0xc6, 0xcb, 0x47, + 0x0c, 0x13, 0x6d, 0xd9, 0x5b, 0x41, 0x72, 0x5b, 0xa2, 0x3f, 0xee, 0x3c, 0x58, 0x4e, 0xd1, 0xa4, + 0xb6, 0x2f, 0x27, 0xa0, 0x14, 0x36, 0x8a, 0x98, 0xaf, 0x96, 0xe3, 0xfa, 0x64, 0x32, 0xbc, 0xd3, + 0x4a, 0x62, 0xd0, 0xfc, 0x2c, 0xe4, 0xea, 0xce, 0x76, 0xab, 0x49, 0x7d, 0x6a, 0x8a, 0x6c, 0x86, + 0x6e, 0x01, 0x79, 0x15, 0x66, 0x43, 0x35, 0xd3, 0x37, 0x2d, 0xbb, 0x41, 0xdd, 0x96, 0x6b, 0xd9, + 0xbe, 0x88, 0x40, 0xce, 0x84, 0x95, 0xd7, 0xbb, 0x75, 0xe4, 0x2d, 0x28, 0x76, 0x89, 0xa4, 0xd7, + 0xb2, 0x6c, 0xd6, 0xf0, 0xfd, 0x1f, 0x02, 0x56, 0xb4, 0x39, 0x6f, 0x00, 0x4c, 0xbc, 0xa7, 0x1e, + 0x77, 0xf9, 0x32, 0xa4, 0xa6, 0x6e, 0xf8, 0xe2, 0x69, 0xd3, 0x68, 0xfa, 0x9c, 0x0f, 0x29, 0x17, + 0x7c, 0xf5, 0x5d, 0x78, 0x7e, 0xc9, 0xa5, 0x86, 0x4f, 0x0f, 0x96, 0x47, 0xb0, 0xde, 0x0f, 0x1c, + 0xa8, 0x72, 0xf0, 0x40, 0xd5, 0x0e, 0x5c, 0x1c, 0xce, 0x5f, 0x18, 0x85, 0xb7, 0x21, 0xe3, 0x62, + 0x89, 0xd0, 0x9f, 0xab, 0xa3, 0xe8, 0x4f, 0x3f, 0x3b, 0xc1, 0x44, 0x7d, 0x16, 0xd4, 0x83, 0x5b, + 0x85, 0xcf, 0xb7, 0x7e, 0x0e, 0xce, 0xc7, 0xb6, 0x12, 0xd8, 0x1e, 0xc0, 0x18, 0x67, 0x1b, 0x58, + 0xab, 0xa3, 0x81, 0x0b, 0x16, 0xab, 0xe0, 0xa5, 0xfe, 0x99, 0x02, 0x33, 0x83, 0x5a, 0xf7, 0xe9, + 0xe0, 0x81, 0xc2, 0x4f, 0xc4, 0x68, 0xd9, 0x0d, 0x18, 0xaf, 0x07, 0xab, 0x95, 0x69, 0xc9, 0x61, + 0xac, 0x5e, 0x3e, 0xa4, 0x5c, 0xc0, 0x67, 0x3b, 0xbe, 0x6b, 0xd4, 0xc3, 0xc7, 0x49, 0xf8, 0xa1, + 0x7e, 0x06, 0xce, 0x0c, 0x1e, 0x29, 0xd7, 0x97, 0x6b, 0x31, 0x3a, 0xce, 0x07, 0x76, 0x80, 0x76, + 0xab, 0x8f, 0xe1, 0xec, 0x60, 0xc6, 0x62, 0x32, 0xee, 0x43, 0x5e, 0xe2, 0x27, 0xac, 0x40, 0xf5, + 0xb0, 0x13, 0x22, 0xf3, 0x50, 0x5f, 0x85, 0xe2, 0x2d, 0xa7, 0xa6, 0xd1, 0x86, 0xe5, 0xf9, 0x6e, + 0x47, 0xdc, 0xdd, 0x0f, 0xf3, 0x51, 0xfe, 0x4b, 0x81, 0xd3, 0x03, 0xa8, 0x3e, 0x02, 0x97, 0xfd, + 0xb3, 0x30, 0xee, 0xb6, 0x6d, 0xdb, 0xb2, 0x1b, 0xfa, 0x23, 0xa7, 0x16, 0x1c, 0x93, 0xe2, 0x12, + 0x32, 0x0e, 0xc4, 0x89, 0x35, 0x79, 0xc1, 0xed, 0x96, 0x53, 0xf3, 0x4a, 0xb3, 0x90, 0xbc, 0xe5, + 0xd4, 0x7a, 0x55, 0x50, 0xbd, 0x04, 0x85, 0x5b, 0x4e, 0x2d, 0x2a, 0x9a, 0x59, 0xc8, 0x3c, 0x72, + 0x6a, 0xdd, 0x19, 0x4d, 0x3f, 0x72, 0x6a, 0xab, 0xa6, 0xba, 0x02, 0xd3, 0x52, 0x53, 0x21, 0x8f, + 0x97, 0x21, 0xf9, 0xc8, 0xa9, 0x89, 0xb5, 0x3d, 0xdf, 0x63, 0xb3, 0xf1, 0xef, 0x12, 0xf0, 0xbf, + 0x51, 0x80, 0x80, 0x58, 0xd3, 0xcb, 0x57, 0x01, 0xba, 0xc9, 0x6d, 0x64, 0x06, 0x0a, 0x37, 0xee, + 0x6a, 0x77, 0x1f, 0x6c, 0xac, 0xde, 0x59, 0xd1, 0xd7, 0x37, 0x16, 0x96, 0x6e, 0xaf, 0x17, 0x4e, + 0x90, 0x69, 0x98, 0xd8, 0xb8, 0xa9, 0xad, 0x2c, 0x2c, 0x07, 0x45, 0xca, 0xe5, 0x67, 0x21, 0x1b, + 0x64, 0x8e, 0x75, 0xb3, 0xaa, 0xc8, 0x24, 0x40, 0x48, 0xbe, 0x5e, 0x50, 0xae, 0xfc, 0xcd, 0x73, + 0x90, 0x11, 0xde, 0xdc, 0x77, 0x14, 0x18, 0x97, 0x1f, 0x15, 0x93, 0xca, 0x68, 0xcf, 0x86, 0x03, + 0x31, 0x94, 0xaa, 0x23, 0xb7, 0xe7, 0xb2, 0x50, 0x9f, 0x7f, 0xff, 0x9f, 0xfe, 0xe3, 0x37, 0x12, + 0x1f, 0x23, 0xe5, 0xaa, 0xf0, 0x34, 0xab, 0xf2, 0x9b, 0xe3, 0xea, 0x7b, 0x42, 0x71, 0x9e, 0x90, + 0x5f, 0x54, 0x60, 0x2c, 0xf0, 0x80, 0xe3, 0xf2, 0x5b, 0xa2, 0x4f, 0x94, 0x4b, 0x97, 0x47, 0x69, + 0x2a, 0xb0, 0xa8, 0x88, 0xe5, 0x2c, 0x29, 0x85, 0x58, 0x4c, 0xde, 0x42, 0x82, 0xe1, 0x42, 0x1a, + 0x5f, 0x94, 0x92, 0xe7, 0x87, 0xbf, 0x39, 0xe5, 0x08, 0x2e, 0x8e, 0xfa, 0x38, 0x55, 0x9d, 0xc3, + 0xfe, 0x0b, 0x64, 0x32, 0xec, 0x9f, 0x3f, 0x7e, 0xfd, 0x02, 0xa4, 0x30, 0xa3, 0xed, 0xc2, 0x10, + 0x4e, 0x41, 0x8f, 0x87, 0x7a, 0x67, 0xab, 0x9e, 0xc3, 0x5e, 0x4b, 0xa4, 0x18, 0xed, 0x55, 0x1a, + 0xf3, 0x13, 0xfe, 0x30, 0x14, 0xb3, 0x98, 0xc8, 0x0b, 0xa3, 0xe5, 0x3a, 0x1d, 0x8c, 0xe4, 0xc0, + 0xc4, 0x28, 0x75, 0x16, 0x91, 0x4c, 0x91, 0x89, 0x10, 0x89, 0x6b, 0x6c, 0xfa, 0xe4, 0x4b, 0x0a, + 0x64, 0x78, 0x64, 0x81, 0x0c, 0x7d, 0x14, 0x14, 0x4a, 0xfd, 0xd2, 0x08, 0x2d, 0x45, 0xb7, 0x1f, + 0xc3, 0x6e, 0xcf, 0x90, 0xd3, 0x52, 0xb7, 0xac, 0x81, 0x24, 0x01, 0x0f, 0x32, 0xfc, 0x65, 0x47, + 0x2c, 0x82, 0xc8, 0xe3, 0x8f, 0x92, 0x9c, 0x8c, 0x2b, 0xfe, 0x0c, 0x09, 0x3b, 0xb8, 0x0a, 0xa9, + 0xf7, 0x77, 0x2a, 0xfe, 0x62, 0x49, 0xb7, 0xd3, 0x6f, 0x2b, 0x90, 0x97, 0x9e, 0x24, 0x90, 0x97, + 0x46, 0x7b, 0xba, 0x10, 0xf4, 0x5f, 0x19, 0xb5, 0xb9, 0x10, 0xc3, 0x05, 0x44, 0x74, 0x8e, 0xcc, + 0x87, 0x88, 0xf8, 0x0d, 0x26, 0x7a, 0x8a, 0x12, 0xac, 0x6f, 0x2a, 0x90, 0x0b, 0x73, 0xc6, 0x63, + 0xd5, 0xa1, 0x37, 0x53, 0x3e, 0x56, 0x1d, 0xfa, 0xd2, 0xd8, 0xd5, 0x4b, 0x08, 0xe8, 0x3c, 0xf9, + 0x58, 0x08, 0xc8, 0x08, 0xda, 0xa0, 0x8a, 0x4a, 0x98, 0xbe, 0xab, 0xc0, 0x64, 0xf4, 0x4d, 0x01, + 0x79, 0x79, 0xa4, 0xbe, 0xa4, 0x30, 0x54, 0xe9, 0x95, 0x43, 0x50, 0x08, 0x88, 0x2f, 0x20, 0xc4, + 0xe7, 0xc8, 0xf9, 0x01, 0x10, 0x51, 0x89, 0xaa, 0xef, 0x05, 0x01, 0xa5, 0x27, 0xe4, 0x2b, 0x0a, + 0x8c, 0xcb, 0x39, 0x0d, 0xb1, 0xc6, 0x75, 0x40, 0x6a, 0x53, 0xac, 0x71, 0x1d, 0x94, 0xb3, 0xa1, + 0x9e, 0x46, 0x78, 0x27, 0xc9, 0x74, 0x08, 0x2f, 0x4c, 0xc4, 0xf8, 0x4d, 0x91, 0x73, 0x82, 0x8f, + 0xd1, 0x3e, 0x3a, 0x44, 0x65, 0x44, 0x74, 0x9a, 0x9c, 0x0a, 0x11, 0xe1, 0xd3, 0x3a, 0x5d, 0xc6, + 0x95, 0x97, 0x52, 0x2c, 0x62, 0x95, 0xbe, 0x3f, 0xfb, 0x23, 0x56, 0xe9, 0x07, 0x64, 0x6e, 0x0c, + 0xda, 0x7e, 0xb0, 0x15, 0x4f, 0x2a, 0x92, 0x34, 0xec, 0x77, 0x14, 0x98, 0x88, 0xa4, 0x4d, 0x90, + 0xea, 0xd0, 0xae, 0xa2, 0xb9, 0x1d, 0xa5, 0x97, 0x47, 0x27, 0x38, 0x70, 0x05, 0x08, 0x74, 0x42, + 0x5c, 0x12, 0xbe, 0x2f, 0x29, 0x90, 0x0b, 0x93, 0x15, 0x62, 0x57, 0x65, 0x6f, 0xc2, 0x46, 0xec, + 0xaa, 0xec, 0xcb, 0x7f, 0x50, 0x8b, 0x88, 0x89, 0xa8, 0x5d, 0x23, 0xed, 0xb5, 0x0c, 0xfb, 0x0d, + 0xe5, 0x32, 0xf9, 0x02, 0xba, 0x11, 0xf5, 0xad, 0x78, 0x33, 0x1d, 0xc9, 0xfa, 0x2f, 0xc5, 0xed, + 0xa2, 0xf2, 0xe3, 0x93, 0x01, 0xf6, 0xd2, 0x43, 0x46, 0x92, 0x08, 0x7e, 0x5e, 0x81, 0x31, 0x91, + 0x5d, 0x1e, 0xeb, 0x21, 0x44, 0x33, 0xd0, 0x47, 0x87, 0xd0, 0xef, 0x1e, 0xb4, 0x38, 0xa7, 0x1e, + 0x0c, 0x22, 0x65, 0x3d, 0x16, 0x43, 0x34, 0xad, 0xfd, 0x38, 0x18, 0xb6, 0x39, 0x27, 0x09, 0xc3, + 0x2f, 0x29, 0x90, 0x0d, 0x9e, 0x00, 0x90, 0x38, 0xff, 0xa7, 0xe7, 0x15, 0x43, 0xe9, 0x85, 0x91, + 0xda, 0x0a, 0x24, 0xfd, 0x6e, 0x03, 0x86, 0xee, 0xa2, 0xfb, 0xd7, 0xb8, 0xfc, 0x6a, 0x25, 0xde, + 0xba, 0xf4, 0x3f, 0x87, 0x89, 0xb7, 0x2e, 0x03, 0x9e, 0xc3, 0xa8, 0xe7, 0x11, 0xd3, 0x33, 0xe4, + 0x8c, 0x64, 0x5d, 0x1a, 0xbd, 0xb0, 0xbe, 0xae, 0xc0, 0x98, 0xa0, 0x8e, 0x9d, 0xa2, 0xe8, 0xf3, + 0x98, 0xd2, 0x4b, 0xf1, 0x4d, 0x7b, 0x1e, 0x07, 0xa9, 0x97, 0x11, 0xca, 0xb3, 0x44, 0x8d, 0x81, + 0x52, 0x7d, 0x8f, 0x15, 0x3c, 0x61, 0xfe, 0xdd, 0x9a, 0xd3, 0xf0, 0x62, 0xfd, 0x3b, 0xe9, 0x91, + 0xd5, 0x61, 0xa1, 0x0c, 0xb2, 0xb9, 0x0d, 0x59, 0x22, 0xdf, 0x52, 0xf0, 0x6f, 0x46, 0x74, 0x6f, + 0x63, 0x63, 0x6d, 0xdb, 0xa0, 0xc4, 0xa2, 0x58, 0xdb, 0x36, 0xf0, 0xa2, 0x57, 0x9d, 0x47, 0x54, + 0x45, 0x32, 0x27, 0xaf, 0x26, 0xd6, 0x4e, 0xe4, 0xb4, 0xbf, 0xaf, 0x40, 0x2e, 0xbc, 0x52, 0x8a, + 0x35, 0x68, 0xbd, 0x37, 0xc2, 0xb1, 0x06, 0xad, 0xef, 0x96, 0x4a, 0x2d, 0x21, 0x90, 0x19, 0x42, + 0x42, 0x20, 0x0f, 0x1d, 0x5f, 0x80, 0x78, 0x02, 0x69, 0xee, 0x4d, 0x3c, 0x3f, 0xfc, 0x96, 0x60, + 0xb8, 0xb7, 0x1f, 0xf5, 0x1d, 0x0e, 0x70, 0x3b, 0x65, 0x8f, 0xe1, 0xb7, 0x14, 0xc8, 0xcb, 0xb1, + 0x90, 0xb8, 0x89, 0xef, 0x8f, 0x3b, 0x0c, 0x9a, 0x94, 0xc8, 0x5f, 0xa7, 0x93, 0x68, 0x78, 0x50, + 0x66, 0x80, 0x0f, 0x28, 0x11, 0x44, 0x77, 0x9b, 0x0c, 0x0f, 0xb0, 0x0f, 0xb1, 0xf5, 0xd2, 0x0d, + 0x47, 0xac, 0x4b, 0x1e, 0x8d, 0xd6, 0x0f, 0xb4, 0xf6, 0xac, 0x81, 0x04, 0xe1, 0x97, 0x15, 0x3c, + 0x13, 0x07, 0x51, 0xed, 0x17, 0x47, 0x0c, 0xb1, 0x0e, 0x5f, 0x41, 0xfd, 0x01, 0x59, 0xf5, 0x0c, + 0xc2, 0x99, 0x25, 0x27, 0xe5, 0xcd, 0x27, 0xe8, 0xf9, 0x87, 0x0a, 0x9c, 0x1b, 0x16, 0xd9, 0x23, + 0x8b, 0x71, 0x7b, 0xff, 0x68, 0x61, 0xc7, 0xd2, 0xd2, 0xb1, 0x78, 0x44, 0x4d, 0xa4, 0x5a, 0x94, + 0x86, 0xb2, 0xed, 0xb3, 0x59, 0x16, 0x91, 0x38, 0xb6, 0x93, 0xff, 0xa5, 0x72, 0x50, 0xdc, 0x09, + 0x91, 0x78, 0xe4, 0xcd, 0x23, 0xc5, 0xfc, 0x42, 0xf1, 0xbf, 0x75, 0x54, 0xf2, 0x03, 0xf7, 0x9e, + 0x9e, 0x41, 0x90, 0x3f, 0x3f, 0x28, 0x9c, 0xf8, 0xf1, 0x43, 0x77, 0xcd, 0x21, 0xbf, 0x7e, 0x68, + 0x3a, 0x81, 0xf5, 0x35, 0xc4, 0x5a, 0x21, 0x2f, 0xf6, 0x61, 0xad, 0xbe, 0x77, 0x50, 0x54, 0xf0, + 0x09, 0xf9, 0x9e, 0x82, 0x81, 0xa3, 0x68, 0xa0, 0x8a, 0xbc, 0x7a, 0xb8, 0xb0, 0x16, 0x47, 0xfe, + 0xda, 0x51, 0x62, 0x61, 0x03, 0x1c, 0xe3, 0x47, 0x4e, 0x4d, 0x77, 0x45, 0xe3, 0xa8, 0xb7, 0x91, + 0x0b, 0x43, 0x5c, 0xb1, 0x76, 0xba, 0x37, 0x66, 0x16, 0x6b, 0xa7, 0xfb, 0xa2, 0x66, 0xea, 0x33, + 0x88, 0xe8, 0x14, 0x99, 0x95, 0x11, 0x55, 0xdf, 0xe3, 0x51, 0xb7, 0x27, 0x8b, 0x97, 0x7f, 0xf0, + 0xef, 0xf3, 0x27, 0x7e, 0xb0, 0x3f, 0xaf, 0xfc, 0x70, 0x7f, 0x5e, 0xf9, 0xd1, 0xfe, 0xbc, 0xf2, + 0x6f, 0xfb, 0xf3, 0xca, 0x37, 0x3e, 0x98, 0x3f, 0xf1, 0xc3, 0x0f, 0xe6, 0x4f, 0xfc, 0xe8, 0x83, + 0xf9, 0x13, 0xef, 0x64, 0x03, 0xe6, 0xb5, 0x0c, 0x06, 0x7c, 0x5f, 0xfd, 0xdf, 0x00, 0x00, 0x00, + 0xff, 0xff, 0x5b, 0x83, 0xbe, 0x44, 0x71, 0x54, 0x00, 0x00, } diff --git a/pkg/server/serverpb/status.pb.gw.go b/pkg/server/serverpb/status.pb.gw.go index 38e9da0050c9..66d381cbb4e9 100644 --- a/pkg/server/serverpb/status.pb.gw.go +++ b/pkg/server/serverpb/status.pb.gw.go @@ -1127,6 +1127,10 @@ func local_request_Status_LogFilesList_0(ctx context.Context, marshaler runtime. } +var ( + filter_Status_LogFile_0 = &utilities.DoubleArray{Encoding: map[string]int{"node_id": 0, "file": 1}, Base: []int{1, 1, 2, 0, 0}, Check: []int{0, 1, 1, 2, 3}} +) + func request_Status_LogFile_0(ctx context.Context, marshaler runtime.Marshaler, client StatusClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { var protoReq LogFileRequest var metadata runtime.ServerMetadata @@ -1160,6 +1164,13 @@ func request_Status_LogFile_0(ctx context.Context, marshaler runtime.Marshaler, return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "file", err) } + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_Status_LogFile_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + msg, err := client.LogFile(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) return msg, metadata, err @@ -1198,6 +1209,13 @@ func local_request_Status_LogFile_0(ctx context.Context, marshaler runtime.Marsh return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "file", err) } + if err := req.ParseForm(); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_Status_LogFile_0); err != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + msg, err := server.LogFile(ctx, &protoReq) return msg, metadata, err diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index a54f1539b128..42cdf68f9f38 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -299,6 +299,18 @@ message LogsRequest { string end_time = 4; string max = 5; string pattern = 6; + // redact, if true, requests redaction of sensitive data away + // from the retrieved log entries. + // Only admin users can send a request with redact = false. + bool redact = 7; + // keep_redactable, if true, requests that retrieved entries preserve + // the redaction markers if any were present in the log files. + // If false, redaction markers are stripped away. + // Note that redact = false && redactable = false implies + // "flat" entries with all sensitive information enclosed and + // no markers; this is suitable for backward-compatibility with + // RPC clients from prior the introduction of redactable logs. + bool keep_redactable = 8; } message LogEntriesResponse { @@ -322,6 +334,18 @@ message LogFileRequest { // forwarding is necessary. string node_id = 1; string file = 2; + // redact, if true, requests redaction of sensitive data away + // from the retrieved log entries. + // Only admin users can send a request with redact = false. + bool redact = 3; + // keep_redactable, if true, requests that retrieved entries preserve + // the redaction markers if any were present in the log files. + // If false, redaction markers are stripped away. + // Note that redact = false && redactable = false implies + // "flat" entries with all sensitive information enclosed and + // no markers; this is suitable for backward-compatibility with + // RPC clients from prior the introduction of redactable logs. + bool keep_redactable = 4; } enum StacksType { diff --git a/pkg/server/status.go b/pkg/server/status.go index af0d578a5901..5f3e61782a95 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -791,16 +791,17 @@ func (s *statusServer) LogFilesList( func (s *statusServer) LogFile( ctx context.Context, req *serverpb.LogFileRequest, ) (*serverpb.LogEntriesResponse, error) { - if _, err := s.admin.requireAdminUser(ctx); err != nil { - return nil, err - } - + ctx = propagateGatewayMetadata(ctx) + ctx = s.AnnotateCtx(ctx) if !debug.GatewayRemoteAllowed(ctx, s.st) { return nil, remoteDebuggingErr } - ctx = propagateGatewayMetadata(ctx) - ctx = s.AnnotateCtx(ctx) + _, isAdmin, err := s.admin.getUserAndRole(ctx) + if err != nil { + return nil, err + } + nodeID, local, err := s.parseNodeID(req.NodeId) if err != nil { return nil, grpcstatus.Errorf(codes.InvalidArgument, err.Error()) @@ -813,7 +814,18 @@ func (s *statusServer) LogFile( return status.LogFile(ctx, req) } + // Determine whether the request is valid for the current user. + if !isAdmin && !req.Redact { + return nil, errInsufficientPrivilege + } + + // Determine how to redact. + inputEditMode := log.SelectEditMode(req.Redact, req.KeepRedactable) + + // Ensure that the latest log entries are available in files. log.Flush() + + // Read the logs. reader, err := log.GetLogReader(req.File, true /* restricted */) if reader == nil || err != nil { return nil, fmt.Errorf("log file %s could not be opened: %s", req.File, err) @@ -822,7 +834,7 @@ func (s *statusServer) LogFile( var entry log.Entry var resp serverpb.LogEntriesResponse - decoder := log.NewEntryDecoder(reader) + decoder := log.NewEntryDecoder(reader, inputEditMode) for { if err := decoder.Decode(&entry); err != nil { if err == io.EOF { @@ -833,6 +845,13 @@ func (s *statusServer) LogFile( resp.Entries = append(resp.Entries, entry) } + // Erase the redactable bit if requested by client. + if !req.KeepRedactable { + for i := range resp.Entries { + resp.Entries[i].Redactable = false + } + } + return &resp, nil } @@ -870,15 +889,17 @@ func parseInt64WithDefault(s string, defaultValue int64) (int64, error) { func (s *statusServer) Logs( ctx context.Context, req *serverpb.LogsRequest, ) (*serverpb.LogEntriesResponse, error) { - if _, err := s.admin.requireAdminUser(ctx); err != nil { - return nil, err - } + ctx = propagateGatewayMetadata(ctx) + ctx = s.AnnotateCtx(ctx) if !debug.GatewayRemoteAllowed(ctx, s.st) { return nil, remoteDebuggingErr } - ctx = propagateGatewayMetadata(ctx) - ctx = s.AnnotateCtx(ctx) + _, isAdmin, err := s.admin.getUserAndRole(ctx) + if err != nil { + return nil, err + } + nodeID, local, err := s.parseNodeID(req.NodeId) if err != nil { return nil, grpcstatus.Errorf(codes.InvalidArgument, err.Error()) @@ -891,8 +912,15 @@ func (s *statusServer) Logs( return status.Logs(ctx, req) } - log.Flush() + // Determine whether the request is valid for the current user. + if !isAdmin && !req.Redact { + return nil, errInsufficientPrivilege + } + + // Determine how to redact. + inputEditMode := log.SelectEditMode(req.Redact, req.KeepRedactable) + // Select the time interval. startTimestamp, err := parseInt64WithDefault( req.StartTime, timeutil.Now().AddDate(0, 0, -1).UnixNano()) @@ -924,11 +952,23 @@ func (s *statusServer) Logs( } } - entries, err := log.FetchEntriesFromFiles(startTimestamp, endTimestamp, int(maxEntries), regex) + // Ensure that the latest log entries are available in files. + log.Flush() + + // Read the logs. + entries, err := log.FetchEntriesFromFiles( + startTimestamp, endTimestamp, int(maxEntries), regex, inputEditMode) if err != nil { return nil, err } + // Erase the redactable bit if requested by client. + if !req.KeepRedactable { + for i := range entries { + entries[i].Redactable = false + } + } + return &serverpb.LogEntriesResponse{Entries: entries}, nil } diff --git a/pkg/server/status/runtime.go b/pkg/server/status/runtime.go index 429c2d47aad2..3ce7f6b6de2a 100644 --- a/pkg/server/status/runtime.go +++ b/pkg/server/status/runtime.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" + "github.com/cockroachdb/cockroach/pkg/util/redact" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/dustin/go-humanize" "github.com/elastic/gosigar" @@ -496,7 +497,11 @@ func (rsr *RuntimeStatSampler) SampleEnvironment(ctx context.Context, ms GoMemSt staleMsg = "(stale)" } goTotal := ms.Sys - ms.HeapReleased - log.Infof(ctx, "%s", log.Safe(fmt.Sprintf("runtime stats: %s RSS, %d goroutines, %s/%s/%s GO alloc/idle/total%s, "+ + + // TODO(knz): make utility wrapper around humanize.IBytes that + // returns a safe value and collapse the entire log.Infof -> Safe -> + // Sprintf sequence as a flat Infof call. + log.Infof(ctx, "%s", redact.Safe(fmt.Sprintf("runtime stats: %s RSS, %d goroutines, %s/%s/%s GO alloc/idle/total%s, "+ "%s/%s CGO alloc/total, %.1f CGO/sec, %.1f/%.1f %%(u/s)time, %.1f %%gc (%dx), "+ "%s/%s (r/w)net", humanize.IBytes(mem.Resident), numGoroutine, diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go index a9c8ec23c70e..a70eb8f581ee 100644 --- a/pkg/server/status_test.go +++ b/pkg/server/status_test.go @@ -57,6 +57,7 @@ import ( "github.com/cockroachdb/errors" "github.com/gogo/protobuf/proto" "github.com/kr/pretty" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -522,7 +523,7 @@ func TestStatusLocalLogs(t *testing.T) { if err := getStatusJSONProto(ts, "logfiles/local", &wrapper); err != nil { t.Fatal(err) } - if a, e := len(wrapper.Files), 1; a != e { + if a, e := len(wrapper.Files), 2; a != e { t.Fatalf("expected %d log files; got %d", e, a) } @@ -630,6 +631,145 @@ func TestStatusLocalLogs(t *testing.T) { } } +// TestStatusLogRedaction checks that the log file retrieval RPCs +// honor the redaction flags. +func TestStatusLogRedaction(t *testing.T) { + defer leaktest.AfterTest(t)() + + testData := []struct { + redactableLogs bool // logging flag + redact bool // RPC request flag + keepRedactable bool // RPC request flag + expectedMessage string + expectedRedactable bool // redactable bit in result entries + }{ + // Note: all 2^3 combinations of (redactableLogs, redact, + // keepRedactable) must be tested below. + + // redact=false, keepredactable=false results in an unsafe "flat" + // format regardless of whether there were markers in the log + // file. + {false, false, false, `THISISSAFE THISISUNSAFE`, false}, + // keepredactable=true, if there were no markers to start with + // (redactableLogs=false), introduces markers around the entire + // message to indicate it's not known to be safe. + {false, false, true, `‹THISISSAFE THISISUNSAFE›`, true}, + // redact=true must be conservative and redact everything out if + // there were no markers to start with (redactableLogs=false). + {false, true, false, `‹×›`, false}, + {false, true, true, `‹×›`, false}, + // redact=false, keepredactable=false results in an unsafe "flat" + // format regardless of whether there were markers in the log + // file. + {true, false, false, `THISISSAFE THISISUNSAFE`, false}, + // keepredactable=true, redact=false, keeps whatever was in the + // log file. + {true, false, true, `THISISSAFE ‹THISISUNSAFE›`, true}, + // if there were markers in the log to start with, redact=true + // removes only the unsafe information. + {true, true, false, `THISISSAFE ‹×›`, false}, + // Whether or not to keep the redactable markers has no influence + // on the output of redaction, just on the presence of the + // "redactable" marker. In any case no information is leaked. + {true, true, true, `THISISSAFE ‹×›`, true}, + } + + testutils.RunTrueAndFalse(t, "redactableLogs", + func(t *testing.T, redactableLogs bool) { + s := log.ScopeWithoutShowLogs(t) + defer s.Close(t) + + // Apply the redactable log boolean for this test. + defer log.TestingSetRedactable(redactableLogs)() + + ts := startServer(t) + defer ts.Stopper().Stop(context.Background()) + + // Log something. + log.Infof(context.Background(), "THISISSAFE %s", "THISISUNSAFE") + + // Determine the log file name. + var wrapper serverpb.LogFilesListResponse + if err := getStatusJSONProto(ts, "logfiles/local", &wrapper); err != nil { + t.Fatal(err) + } + // We expect a main log and a stderr log. + if a, e := len(wrapper.Files), 2; a != e { + t.Fatalf("expected %d log files; got %d: %+v", e, a, wrapper.Files) + } + var file log.FileInfo + // Find the main log. + for _, f := range wrapper.Files { + if !strings.Contains("stderr", f.Name) { + file = f + break + } + } + + for _, tc := range testData { + if tc.redactableLogs != redactableLogs { + continue + } + t.Run(fmt.Sprintf("redact=%v,keepredactable=%v", tc.redact, tc.keepRedactable), + func(t *testing.T) { + // checkEntries asserts that the redaction results are + // those expected in tc. + checkEntries := func(entries []log.Entry) { + foundMessage := false + for _, entry := range entries { + if !strings.HasSuffix(entry.File, "status_test.go") { + continue + } + foundMessage = true + + assert.Equal(t, tc.expectedMessage, entry.Message) + } + if !foundMessage { + t.Fatalf("did not find expected message from test in log") + } + } + + // Retrieve the log entries with the configured flags using + // the LogFiles() RPC. + logFilesURL := fmt.Sprintf("logfiles/local/%s?redact=%v&keep_redactable=%v", + file.Name, tc.redact, tc.keepRedactable) + var wrapper serverpb.LogEntriesResponse + if err := getStatusJSONProto(ts, logFilesURL, &wrapper); err != nil { + t.Fatal(err) + } + checkEntries(wrapper.Entries) + + // If the test specifies redact=false, check that a non-admin + // user gets a privilege error. + if !tc.redact { + err := getStatusJSONProtoWithAdminOption(ts, logFilesURL, &wrapper, false /* isAdmin */) + if !testutils.IsError(err, "status: 403") { + t.Fatalf("expected privilege error, got %v", err) + } + } + + // Retrieve the log entries using the Logs() RPC. + logsURL := fmt.Sprintf("logs/local?redact=%v&keep_redactable=%v", + tc.redact, tc.keepRedactable) + var wrapper2 serverpb.LogEntriesResponse + if err := getStatusJSONProto(ts, logsURL, &wrapper2); err != nil { + t.Fatal(err) + } + checkEntries(wrapper2.Entries) + + // If the test specifies redact=false, check that a non-admin + // user gets a privilege error. + if !tc.redact { + err := getStatusJSONProtoWithAdminOption(ts, logsURL, &wrapper2, false /* isAdmin */) + if !testutils.IsError(err, "status: 403") { + t.Fatalf("expected privilege error, got %v", err) + } + } + }) + } + }) +} + // TestNodeStatusResponse verifies that node status returns the expected // results. func TestNodeStatusResponse(t *testing.T) { diff --git a/pkg/sql/flowinfra/inbound.go b/pkg/sql/flowinfra/inbound.go index baf386f38045..2e7ef7748b3e 100644 --- a/pkg/sql/flowinfra/inbound.go +++ b/pkg/sql/flowinfra/inbound.go @@ -194,7 +194,10 @@ func processProducerMessage( if err != nil { return processMessageResult{ err: errors.Wrapf(err, "%s", - log.MakeMessage(ctx, "decoding error", nil /* args */)), + // TODO(knz): Instead of pre-formatting the string here, use + // errors.WithContextTags() here and let the error formatter + // show the tags later. + log.FormatWithContextTags(ctx, "decoding error")), consumerClosed: false, } } diff --git a/pkg/sql/pgwire/auth_test.go b/pkg/sql/pgwire/auth_test.go index 30c3d19d3506..e400b8e4853f 100644 --- a/pkg/sql/pgwire/auth_test.go +++ b/pkg/sql/pgwire/auth_test.go @@ -258,7 +258,8 @@ func hbaRunTest(t *testing.T, insecure bool) { // this is currently broken for secondary loggers. // See: https://github.com/cockroachdb/cockroach/issues/45745 // So instead we need to do the filtering ourselves. - entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 10000, authLogFileRe) + entries, err := log.FetchEntriesFromFiles(0, math.MaxInt64, 10000, authLogFileRe, + log.WithFlattenedSensitiveData) if err != nil { t.Fatal(err) } @@ -274,13 +275,17 @@ func hbaRunTest(t *testing.T, insecure bool) { entry := &entries[i] t.Logf("found log entry: %+v", *entry) - // The message is going to contain a client address, with a random port number. + // The tag part is going to contain a client address, with a random port number. // To make the test deterministic, erase the random part. - msg := addrRe.ReplaceAllString(entry.Message, ",client=XXX") + tags := addrRe.ReplaceAllString(entry.Tags, ",client=XXX") + var maybeTags string + if len(tags) > 0 { + maybeTags = "[" + tags + "] " + } // Ditto with the duration. - msg = durationRe.ReplaceAllString(msg, "duration: XXX") + msg := durationRe.ReplaceAllString(entry.Message, "duration: XXX") - fmt.Fprintf(&buf, "%c: %s\n", entry.Severity.String()[0], msg) + fmt.Fprintf(&buf, "%c: %s%s\n", entry.Severity.String()[0], maybeTags, msg) } lastLogMsg := entries[0].Message if !re.MatchString(lastLogMsg) { diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 5c01227d382a..e070657439c4 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1710,6 +1710,10 @@ func TestLint(t *testing.T) { "WarningfDepth", "Wrapf", "WrapWithDepthf", + "redact.Fprint", + "redact.Fprintf", + "redact.Sprint", + "redact.Sprintf", }, ",") filters := []stream.Filter{ @@ -1769,6 +1773,12 @@ func TestLint(t *testing.T) { // because addStructured takes its positional argument as []interface{}, // instead of ...interface{}. stream.GrepNot(`pkg/util/log/structured\.go:\d+:\d+: addStructured\(\): format argument is not a constant expression`), + // The markers test file is passing test case formats + // to the printf function. + stream.GrepNot(`pkg/util/redact/markers_test\.go:\d+:\d+: TestRedactStream\(\): format argument is not a constant expression`), + // roachtest is not collecting redactable logs so we don't care + // about printf hygiene there as much. + stream.GrepNot(`pkg/cmd/roachtest/log\.go:.*format argument is not a constant expression`), } roachlint, err := exec.LookPath("roachvet") diff --git a/pkg/testutils/lint/passes/fmtsafe/functions.go b/pkg/testutils/lint/passes/fmtsafe/functions.go index 04cc59c12469..40a4909835cd 100644 --- a/pkg/testutils/lint/passes/fmtsafe/functions.go +++ b/pkg/testutils/lint/passes/fmtsafe/functions.go @@ -60,20 +60,29 @@ var requireConstFmt = map[string]bool{ "(*log.Logger).Panicf": true, "(*log.Logger).Printf": true, - "github.com/cockroachdb/cockroach/pkg/util/log.Shoutf": true, - "github.com/cockroachdb/cockroach/pkg/util/log.Infof": true, - "github.com/cockroachdb/cockroach/pkg/util/log.Warningf": true, - "github.com/cockroachdb/cockroach/pkg/util/log.Errorf": true, - "github.com/cockroachdb/cockroach/pkg/util/log.Eventf": true, - "github.com/cockroachdb/cockroach/pkg/util/log.VEventf": true, - "github.com/cockroachdb/cockroach/pkg/util/log.VErrEventf": true, - "github.com/cockroachdb/cockroach/pkg/util/log.InfofDepth": true, - "github.com/cockroachdb/cockroach/pkg/util/log.WarningfDepth": true, - "github.com/cockroachdb/cockroach/pkg/util/log.ErrorfDepth": true, - "github.com/cockroachdb/cockroach/pkg/util/log.FatalfDepth": true, - "github.com/cockroachdb/cockroach/pkg/util/log.VEventfDepth": true, - "github.com/cockroachdb/cockroach/pkg/util/log.VErrEventfDepth": true, - "github.com/cockroachdb/cockroach/pkg/util/log.ReportOrPanic": true, + "github.com/cockroachdb/cockroach/pkg/util/log.Shoutf": true, + "github.com/cockroachdb/cockroach/pkg/util/log.Infof": true, + "github.com/cockroachdb/cockroach/pkg/util/log.Warningf": true, + "github.com/cockroachdb/cockroach/pkg/util/log.Errorf": true, + "github.com/cockroachdb/cockroach/pkg/util/log.Eventf": true, + "github.com/cockroachdb/cockroach/pkg/util/log.vEventf": true, + "github.com/cockroachdb/cockroach/pkg/util/log.VEventf": true, + "github.com/cockroachdb/cockroach/pkg/util/log.VErrEventf": true, + "github.com/cockroachdb/cockroach/pkg/util/log.InfofDepth": true, + "github.com/cockroachdb/cockroach/pkg/util/log.WarningfDepth": true, + "github.com/cockroachdb/cockroach/pkg/util/log.ErrorfDepth": true, + "github.com/cockroachdb/cockroach/pkg/util/log.FatalfDepth": true, + "github.com/cockroachdb/cockroach/pkg/util/log.VEventfDepth": true, + "github.com/cockroachdb/cockroach/pkg/util/log.VErrEventfDepth": true, + "github.com/cockroachdb/cockroach/pkg/util/log.ReportOrPanic": true, + "github.com/cockroachdb/cockroach/pkg/util/log.MakeEntry": true, + "github.com/cockroachdb/cockroach/pkg/util/log.FormatWithContextTags": true, + "github.com/cockroachdb/cockroach/pkg/util/log.renderArgs": true, + + "(*github.com/cockroachdb/cockroach/pkg/util/log.loggerT).makeStartLine": true, + "(*github.com/cockroachdb/cockroach/pkg/util/log.SecondaryLogger).output": true, + "(*github.com/cockroachdb/cockroach/pkg/util/log.SecondaryLogger).Logf": true, + "(*github.com/cockroachdb/cockroach/pkg/util/log.SecondaryLogger).LogfDepth": true, "(github.com/cockroachdb/cockroach/pkg/rpc.breakerLogger).Debugf": true, "(github.com/cockroachdb/cockroach/pkg/rpc.breakerLogger).Infof": true, @@ -146,6 +155,12 @@ var requireConstFmt = map[string]bool{ "github.com/cockroachdb/errors.NewAssertionErrorWithWrappedErrf": true, "github.com/cockroachdb/errors.WithSafeDetails": true, + "github.com/cockroachdb/cockroach/pkg/util/redact.Sprintf": true, + "github.com/cockroachdb/cockroach/pkg/util/redact.Fprintf": true, + "(github.com/cockroachdb/cockroach/pkg/util/redact.SafePrinter).Printf": true, + "(github.com/cockroachdb/cockroach/pkg/util/redact.SafeWriter).Printf": true, + "(*github.com/cockroachdb/cockroach/pkg/util/redact.printer).Printf": true, + "github.com/cockroachdb/cockroach/pkg/roachpb.NewErrorf": true, "github.com/cockroachdb/cockroach/pkg/ccl/importccl.makeRowErr": true, @@ -159,6 +174,8 @@ var requireConstFmt = map[string]bool{ "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder.unimplementedWithIssueDetailf": true, + "(*github.com/cockroachdb/cockroach/pkg/sql/pgwire.authPipe).Logf": true, + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.Newf": true, "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.NewWithDepthf": true, "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror.DangerousStatementf": true, diff --git a/pkg/testutils/lint/testdata/errcheck_excludes.txt b/pkg/testutils/lint/testdata/errcheck_excludes.txt index 085af18a23b3..970c84af3b66 100644 --- a/pkg/testutils/lint/testdata/errcheck_excludes.txt +++ b/pkg/testutils/lint/testdata/errcheck_excludes.txt @@ -1,6 +1,8 @@ fmt.Fprint fmt.Fprintf fmt.Fprintln +github.com/cockroachdb/cockroach/pkg/util/redact.Fprint +github.com/cockroachdb/cockroach/pkg/util/redact.Fprintf (*bufio.Writer).Flush (*database/sql.DB).Close (*database/sql.Rows).Close diff --git a/pkg/ui/src/views/cluster/containers/nodeLogs/index.tsx b/pkg/ui/src/views/cluster/containers/nodeLogs/index.tsx index 7d7e113cf15a..d6c038944140 100644 --- a/pkg/ui/src/views/cluster/containers/nodeLogs/index.tsx +++ b/pkg/ui/src/views/cluster/containers/nodeLogs/index.tsx @@ -61,7 +61,7 @@ export class Logs extends React.Component { title: "Message", cell: (index: number) => (
-              { logEntries[index].message }
+              { (logEntries[index].tags ? "[" + logEntries[index].tags + "] " : "") + logEntries[index].message }
             
), }, diff --git a/pkg/util/hlc/timestamp.go b/pkg/util/hlc/timestamp.go index 9a00fafc91ea..0c415f55057e 100644 --- a/pkg/util/hlc/timestamp.go +++ b/pkg/util/hlc/timestamp.go @@ -92,6 +92,9 @@ func (t Timestamp) String() string { return *(*string)(unsafe.Pointer(&buf)) } +// SafeValue implements the redact.SafeValue interface. +func (Timestamp) SafeValue() {} + var ( timestampRegexp = regexp.MustCompile( `^(?P-)?(?P\d{1,19})(\.(?P\d{1,20}))?,(?P-?\d{1,10})$`) diff --git a/pkg/util/log/ambient_context_test.go b/pkg/util/log/ambient_context_test.go index a1a59c100d0e..698dab41f4b6 100644 --- a/pkg/util/log/ambient_context_test.go +++ b/pkg/util/log/ambient_context_test.go @@ -25,7 +25,7 @@ func TestAnnotateCtxTags(t *testing.T) { ac.AddLogTag("b", 2) ctx := ac.AnnotateCtx(context.Background()) - if exp, val := "[a1,b2] test", MakeMessage(ctx, "test", nil); val != exp { + if exp, val := "[a1,b2] test", FormatWithContextTags(ctx, "test"); val != exp { t.Errorf("expected '%s', got '%s'", exp, val) } @@ -34,7 +34,7 @@ func TestAnnotateCtxTags(t *testing.T) { ctx = logtags.AddTag(ctx, "aa", nil) ctx = ac.AnnotateCtx(ctx) - if exp, val := "[a1,aa,b2] test", MakeMessage(ctx, "test", nil); val != exp { + if exp, val := "[a1,aa,b2] test", FormatWithContextTags(ctx, "test"); val != exp { t.Errorf("expected '%s', got '%s'", exp, val) } } @@ -100,7 +100,7 @@ func TestAnnotateCtxNodeStoreReplica(t *testing.T) { ctx := n.AnnotateCtx(context.Background()) ctx = s.AnnotateCtx(ctx) ctx = r.AnnotateCtx(ctx) - if exp, val := "[n1,s2,r3] test", MakeMessage(ctx, "test", nil); val != exp { + if exp, val := "[n1,s2,r3] test", FormatWithContextTags(ctx, "test"); val != exp { t.Errorf("expected '%s', got '%s'", exp, val) } if tags := logtags.FromContext(ctx); tags != r.tags { @@ -115,7 +115,7 @@ func TestResetAndAnnotateCtx(t *testing.T) { ctx := context.Background() ctx = logtags.AddTag(ctx, "b", 2) ctx = ac.ResetAndAnnotateCtx(ctx) - if exp, val := "[a1] test", MakeMessage(ctx, "test", nil); val != exp { + if exp, val := "[a1] test", FormatWithContextTags(ctx, "test"); val != exp { t.Errorf("expected '%s', got '%s'", exp, val) } } diff --git a/pkg/util/log/clog.go b/pkg/util/log/clog.go index 4059c50b0be7..036150916bf3 100644 --- a/pkg/util/log/clog.go +++ b/pkg/util/log/clog.go @@ -22,21 +22,10 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/logtags" ) -// mainLog is the primary logger instance. -var mainLog loggerT - -// stderrLog is the logger where writes performed directly -// to the stderr file descriptor (such as that performed -// by the go runtime) *may* be redirected. -// NB: whether they are actually redirected is determined -// by stderrLog.redirectInternalStderrWrites(). -var stderrLog = &mainLog - // logging is the global state of the logging setup. var logging loggingT @@ -44,6 +33,9 @@ var logging loggingT // // TODO(knz): better separate global state and per-logger state. type loggingT struct { + // the --no-color flag. + noColor bool + // pool for entry formatting buffers. bufPool sync.Pool @@ -97,10 +89,10 @@ type loggerT struct { // stderr sink. stderrThreshold Severity - // noRedirectInternalStderrWrites, when UNset (set to false), causes - // this logger to capture writes to system-wide file descriptor 2 - // (the standard error stream) and os.Stderr and redirect them to this - // logger's output file. + // redirectInternalStderrWrites, when set, causes this logger to + // capture writes to system-wide file descriptor 2 (the standard + // error stream) and os.Stderr and redirect them to this logger's + // output file. // Users of the logging package should ensure that at most one // logger has this flag set to redirect the system-wide stderr. // @@ -109,17 +101,22 @@ type loggerT struct { // os.Stderr. This is because the Go runtime hardcodes stderr writes // as writes to file descriptor 2 and disregards the value of // os.Stderr entirely. - // - // Callers are encouraged to use the redirectInternalStderrWrites() - // accessor method for clarity. - // - // TODO(knz): The double negative is somewhat inconvenient. We could - // consider flipping the meaning for enhanced clarity. - noRedirectInternalStderrWrites bool + redirectInternalStderrWrites bool + + // whether or not to include redaction markers. + // This is atomic because tests using TestLogScope might + // override this asynchronously with log calls. + redactableLogs syncutil.AtomicBool // notify GC daemon that a new log file was created gcNotify chan struct{} + // logCounter supports the generation of a per-entry log entry + // counter. This is needed in audit logs to hinder malicious + // repudiation of log events by manually erasing log files or log + // entries. + logCounter EntryCounter + // mu protects the remaining elements of this structure and is // used to synchronize logging. mu struct { @@ -133,6 +130,18 @@ type loggerT struct { } } +// EntryCounter supports the generation of a per-entry log entry +// counter. This is needed in audit logs to hinder malicious +// repudiation of log events by manually erasing log files or log +// entries. +type EntryCounter struct { + // EnableMsgCount, if true, enables the production of entry + // counters. + EnableMsgCount bool + // msgCount is the current value of the counter. + msgCount uint64 +} + func init() { logging.bufPool.New = newBuffer logging.mu.fatalCh = make(chan struct{}) @@ -144,10 +153,9 @@ func init() { // commands set their default separately in cli/flags.go. mainLog.stderrThreshold = Severity_INFO mainLog.fileThreshold = Severity_INFO - // In addition, we want all writes performed directly to the - // internal stderr file descriptor (fd 2) to go to the stderr log - // file. - stderrLog.noRedirectInternalStderrWrites = false + // Don't capture stderr output until + // SetupRedactionAndStderrRedirects() has been called. + mainLog.redirectInternalStderrWrites = false } // FatalChan is closed when Fatal is called. This can be used to make @@ -180,9 +188,8 @@ func SetClusterID(clusterID string) { // Ensure that the clusterID is logged with the same format as for // new log files, even on the first log file. This ensures that grep // will always find it. - file, line, _ := caller.Lookup(1) - mainLog.outputLogEntry(Severity_INFO, file, line, - fmt.Sprintf("[config] clusterID: %s", clusterID)) + ctx := logtags.AddTag(context.Background(), "config", nil) + addStructured(ctx, Severity_INFO, 1, "clusterID: %s", []interface{}{clusterID}) // Perform the change proper. logging.mu.Lock() @@ -204,12 +211,6 @@ func (l *loggerT) ensureFile() error { return nil } -// redirectInternalStderrWrites is the positive accessor -// for the noRedirectInternalStderrWrites flag. -func (l *loggerT) redirectInternalStderrWrites() bool { - return !l.noRedirectInternalStderrWrites -} - // writeToFile writes to the file and applies the synchronization policy. // Assumes that l.mu is held by the caller. func (l *loggerT) writeToFile(data []byte) error { @@ -226,11 +227,7 @@ func (l *loggerT) writeToFile(data []byte) error { // outputLogEntry marshals a log entry proto into bytes, and writes // the data to the log files. If a trace location is set, stack traces // are added to the entry before marshaling. -func (l *loggerT) outputLogEntry(s Severity, file string, line int, msg string) { - // Set additional details in log entry. - now := timeutil.Now() - entry := MakeEntry(s, now.UnixNano(), file, line, msg) - +func (l *loggerT) outputLogEntry(entry Entry) { if f, ok := logging.interceptor.Load().(InterceptorFn); ok && f != nil { f(entry) return @@ -241,7 +238,7 @@ func (l *loggerT) outputLogEntry(s Severity, file string, line int, msg string) var stacks []byte var fatalTrigger chan struct{} - if s == Severity_FATAL { + if entry.Severity == Severity_FATAL { logging.signalFatalCh() switch traceback { @@ -301,7 +298,7 @@ func (l *loggerT) outputLogEntry(s Severity, file string, line int, msg string) }() } - if s >= l.stderrThreshold.get() { + if entry.Severity >= l.stderrThreshold.get() { if err := l.outputToStderr(entry, stacks); err != nil { // The external stderr log is unavailable. However, stderr was // chosen by the stderrThreshold configuration, so abandoning @@ -316,7 +313,7 @@ func (l *loggerT) outputLogEntry(s Severity, file string, line int, msg string) return // unreachable except in tests } } - if l.logDir.IsSet() && s >= l.fileThreshold.get() { + if l.logDir.IsSet() && entry.Severity >= l.fileThreshold.get() { if err := l.ensureFile(); err != nil { // We definitely do not like to lose log entries, so we stop // here. Note that exitLocked() shouts the error to both stderr @@ -340,7 +337,7 @@ func (l *loggerT) outputLogEntry(s Severity, file string, line int, msg string) putBuffer(buf) } // Flush and exit on fatal logging. - if s == Severity_FATAL { + if entry.Severity == Severity_FATAL { l.flushAndSync(true /*doSync*/) close(fatalTrigger) // Note: although it seems like the function is allowed to return @@ -370,7 +367,7 @@ func DumpStacks(ctx context.Context) { // // This function is a lightweight version of outputLogEntry() which // does not exit the process in case of error. -func (l *loggerT) printPanicToFile(depth int, r interface{}) { +func (l *loggerT) printPanicToFile(ctx context.Context, depth int, r interface{}) { if !l.logDir.IsSet() { // There's no log file. Can't do anything. return @@ -382,38 +379,38 @@ func (l *loggerT) printPanicToFile(depth int, r interface{}) { if err := l.ensureFile(); err != nil { // We're already exiting; no need to pile an error upon an // error. Simply report the logging error and continue. - l.reportErrorEverywhereLocked(err) + l.reportErrorEverywhereLocked(ctx, err) return } // Create a fully structured log entry. This ensures there a // timestamp in front of the panic object. - entry := makeEntryForPanicObject(depth+1, r) + entry := l.makeEntryForPanicObject(ctx, depth+1, r) buf := logging.processForFile(entry, debug.Stack()) defer putBuffer(buf) // Actually write the panic object to a file. if err := l.writeToFile(buf.Bytes()); err != nil { // Ditto; report the error but continue. We're terminating anyway. - l.reportErrorEverywhereLocked(err) + l.reportErrorEverywhereLocked(ctx, err) } } -func makeEntryForPanicObject(depth int, r interface{}) Entry { - file, line, _ := caller.Lookup(depth + 1) - return MakeEntry(Severity_ERROR, timeutil.Now().UnixNano(), file, line, fmt.Sprintf("panic: %v", r)) +func (l *loggerT) makeEntryForPanicObject(ctx context.Context, depth int, r interface{}) Entry { + return MakeEntry( + ctx, Severity_ERROR, &l.logCounter, depth+1, l.redactableLogs.Get(), "panic: %v", r) } // printPanicToExternalStderr is used by ReportPanic() in case we // understand that the Go runtime will not print the panic object to // the external stderr itself (e.g. because we've redirected it to a // file). -func (l *loggerT) printPanicToExternalStderr(depth int, r interface{}) { - entry := makeEntryForPanicObject(depth+1, r) +func (l *loggerT) printPanicToExternalStderr(ctx context.Context, depth int, r interface{}) { + entry := l.makeEntryForPanicObject(ctx, depth+1, r) if err := l.outputToStderr(entry, debug.Stack()); err != nil { l.mu.Lock() defer l.mu.Unlock() - l.reportErrorEverywhereLocked(err) + l.reportErrorEverywhereLocked(ctx, err) } } diff --git a/pkg/util/log/clog_test.go b/pkg/util/log/clog_test.go index 30f7f59e2995..0941f85c9369 100644 --- a/pkg/util/log/clog_test.go +++ b/pkg/util/log/clog_test.go @@ -90,8 +90,6 @@ func setFlags() { ResetExitFunc() mainLog.mu.Lock() defer mainLog.mu.Unlock() - // Make the internal stderr writes go to the stderr log file. - stderrLog.noRedirectInternalStderrWrites = false // Make all logged errors go to the external stderr, in addition to // the log file. mainLog.stderrThreshold = Severity_ERROR @@ -141,10 +139,16 @@ func TestStandardLog(t *testing.T) { // Verify that a log can be fetched in JSON format. func TestEntryDecoder(t *testing.T) { formatEntry := func(s Severity, now time.Time, gid int, file string, line int, msg string) string { - buf := logging.formatHeader(s, now, gid, file, line, nil) + entry := Entry{ + Severity: s, + Time: now.UnixNano(), + Goroutine: int64(gid), + File: file, + Line: int64(line), + Message: msg, + } + buf := logging.formatLogEntry(entry, nil /* stacks */, nil /* color profile */) defer putBuffer(buf) - buf.WriteString(msg) - buf.WriteString("\n") return buf.String() } @@ -174,7 +178,7 @@ func TestEntryDecoder(t *testing.T) { contents += formatEntry(Severity_INFO, t8, 7, "clog_test.go", 143, tooLongEntry) readAllEntries := func(contents string) []Entry { - decoder := NewEntryDecoder(strings.NewReader(contents)) + decoder := NewEntryDecoder(strings.NewReader(contents), WithFlattenedSensitiveData) var entries []Entry var entry Entry for { @@ -417,14 +421,20 @@ func TestListLogFiles(t *testing.T) { t.Fatalf("buffer wasn't created") } - expectedName := filepath.Base(sb.file.Name()) - results, err := ListLogFiles() if err != nil { t.Fatalf("error in ListLogFiles: %v", err) } - if len(results) != 1 || results[0].Name != expectedName { + expectedName := filepath.Base(sb.file.Name()) + foundExpected := false + for i := range results { + if results[i].Name == expectedName { + foundExpected = true + break + } + } + if !foundExpected { t.Fatalf("unexpected results: %q", results) } } @@ -616,7 +626,7 @@ func TestRedirectStderr(t *testing.T) { const stderrText = "hello stderr" fmt.Fprint(os.Stderr, stderrText) - contents, err := ioutil.ReadFile(mainLog.mu.file.(*syncBuffer).file.Name()) + contents, err := ioutil.ReadFile(stderrLog.mu.file.(*syncBuffer).file.Name()) if err != nil { t.Fatal(err) } @@ -680,8 +690,15 @@ func TestExitOnFullDisk(t *testing.T) { } func BenchmarkHeader(b *testing.B) { + entry := Entry{ + Severity: Severity_INFO, + Time: timeutil.Now().UnixNano(), + Goroutine: 200, + File: "file.go", + Line: 100, + } for i := 0; i < b.N; i++ { - buf := logging.formatHeader(Severity_INFO, timeutil.Now(), 200, "file.go", 100, nil) + buf := logging.formatLogEntryInternal(entry, nil /* profile */) putBuffer(buf) } } diff --git a/pkg/util/log/crash_reporting.go b/pkg/util/log/crash_reporting.go index fa2ca7e28f33..52de840866cc 100644 --- a/pkg/util/log/crash_reporting.go +++ b/pkg/util/log/crash_reporting.go @@ -100,9 +100,6 @@ func RecoverAndReportNonfatalPanic(ctx context.Context, sv *settings.Values) { } } -// Safe constructs a SafeMessager. -var Safe = errors.Safe - // ReportPanic reports a panic has occurred on the real stderr. func ReportPanic(ctx context.Context, sv *settings.Values, r interface{}, depth int) { // Announce the panic has occurred to all places. The purpose @@ -116,7 +113,7 @@ func ReportPanic(ctx context.Context, sv *settings.Values, r interface{}, depth // (The go runtime doesn't time stamp its output.) Shout(ctx, Severity_ERROR, "a panic has occurred!") - if stderrLog.redirectInternalStderrWrites() { + if stderrLog.redirectInternalStderrWrites { // If we decided that the internal stderr writes performed by the // Go runtime are going to our file, that's also where the panic // details will go automatically when the runtime processes the @@ -129,7 +126,7 @@ func ReportPanic(ctx context.Context, sv *settings.Values, r interface{}, depth // as an indication they also want to see panic details // there. Do it here. if LoggingToStderr(Severity_FATAL) { - stderrLog.printPanicToExternalStderr(depth+1, r) + stderrLog.printPanicToExternalStderr(ctx, depth+1, r) } } else { // If we are not redirecting internal stderr writes, then the @@ -138,7 +135,7 @@ func ReportPanic(ctx context.Context, sv *settings.Values, r interface{}, depth // // However, we actually want to persist these details. So print // them in the log file ourselves. - stderrLog.printPanicToFile(depth+1, r) + stderrLog.printPanicToFile(ctx, depth+1, r) } sendCrashReport(ctx, sv, PanicAsError(depth+1, r), ReportTypePanic) diff --git a/pkg/util/log/exit_override.go b/pkg/util/log/exit_override.go index 08d1c448c5a8..01b66753315a 100644 --- a/pkg/util/log/exit_override.go +++ b/pkg/util/log/exit_override.go @@ -11,12 +11,9 @@ package log import ( - "fmt" + "context" "io" "os" - - "github.com/cockroachdb/cockroach/pkg/util/caller" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) // SetExitFunc allows setting a function that will be called to exit @@ -59,7 +56,7 @@ func ResetExitFunc() { func (l *loggerT) exitLocked(err error) { l.mu.AssertHeld() - l.reportErrorEverywhereLocked(err) + l.reportErrorEverywhereLocked(context.Background(), err) logging.mu.Lock() f := logging.mu.exitOverride.f @@ -77,11 +74,11 @@ func (l *loggerT) exitLocked(err error) { // reportErrorEverywhereLocked writes the error details to both the // process' original stderr and the log file if configured. -func (l *loggerT) reportErrorEverywhereLocked(err error) { +func (l *loggerT) reportErrorEverywhereLocked(ctx context.Context, err error) { // Make a valid log entry for this error. - file, line, _ := caller.Lookup(1) - entry := MakeEntry(Severity_ERROR, timeutil.Now().UnixNano(), file, line, - fmt.Sprintf("logging error: %v", err)) + entry := MakeEntry( + ctx, Severity_ERROR, &l.logCounter, 1 /* depth */, l.redactableLogs.Get(), + "logging error: %v", err) // Format the entry for output below. Note how this formatting is // done just once here for both the stderr and file outputs below, diff --git a/pkg/util/log/file.go b/pkg/util/log/file.go index 6f43af62e833..52b2e34270f4 100644 --- a/pkg/util/log/file.go +++ b/pkg/util/log/file.go @@ -448,7 +448,10 @@ func selectFiles(logFiles []FileInfo, endTimestamp int64) []FileInfo { // 'pattern' if provided. The logs entries are returned in reverse // chronological order. func FetchEntriesFromFiles( - startTimestamp, endTimestamp int64, maxEntries int, pattern *regexp.Regexp, + startTimestamp, endTimestamp int64, + maxEntries int, + pattern *regexp.Regexp, + editMode EditSensitiveData, ) ([]Entry, error) { logFiles, err := ListLogFiles() if err != nil { @@ -464,7 +467,8 @@ func FetchEntriesFromFiles( startTimestamp, endTimestamp, maxEntries-len(entries), - pattern) + pattern, + editMode) if err != nil { return nil, err } @@ -489,7 +493,11 @@ func FetchEntriesFromFiles( // processed. If the number of entries returned exceeds 'maxEntries' then // processing of new entries is stopped immediately. func readAllEntriesFromFile( - file FileInfo, startTimestamp, endTimestamp int64, maxEntries int, pattern *regexp.Regexp, + file FileInfo, + startTimestamp, endTimestamp int64, + maxEntries int, + pattern *regexp.Regexp, + editMode EditSensitiveData, ) ([]Entry, bool, error) { reader, err := GetLogReader(file.Name, true /* restricted */) if reader == nil || err != nil { @@ -497,7 +505,7 @@ func readAllEntriesFromFile( } defer reader.Close() entries := []Entry{} - decoder := NewEntryDecoder(reader) + decoder := NewEntryDecoder(reader, editMode) entryBeforeStart := false for { entry := Entry{} diff --git a/pkg/util/log/flags.go b/pkg/util/log/flags.go index 3452cbc69a75..d154c9c82851 100644 --- a/pkg/util/log/flags.go +++ b/pkg/util/log/flags.go @@ -11,15 +11,19 @@ package log import ( + "context" "flag" "github.com/cockroachdb/cockroach/pkg/util/log/logflags" + "github.com/cockroachdb/errors" ) func init() { logflags.InitFlags( - &stderrLog.noRedirectInternalStderrWrites, - &mainLog.logDir, &showLogs, &noColor, + &mainLog.logDir, + &showLogs, + &logging.noColor, + &redactableLogsRequested, // NB: see doc on the variable definition. &logging.vmoduleConfig.mu.vmodule, &LogFileMaxSize, &LogFilesCombinedMaxSize, ) @@ -30,3 +34,110 @@ func init() { flag.Var(&mainLog.fileThreshold, logflags.LogFileVerbosityThresholdName, "minimum verbosity of messages written to the log file") } + +// SetupRedactionAndStderrRedirects should be called once after +// command-line flags have been parsed, and before the first log entry +// is emitted. +// +// The returned cleanup fn can be invoked by the caller to terminate +// the secondary logger. This may be useful in tests. However, for a +// long-running server process the cleanup function should likely not +// be called, to ensure that the file used to capture direct stderr +// writes remains open up until the process entirely terminates. This +// ensures that any Go runtime assertion failures on the way to +// termination can be properly captured. +func SetupRedactionAndStderrRedirects() (cleanup func(), err error) { + // The general goal of this function is to set up a secondary logger + // to capture internal Go writes to os.Stderr / fd 2 and redirect + // them to a separate (non-redactable) log file, This is, of course, + // only possible if there is a log directory to work with -- until + // we extend the log package to use e.g. network sinks. + // + // In case there is no log directory, we must be careful to not + // enable log redaction whatsoever. + // + // This is because otherwise, it is possible for some direct writer + // to fd 2, for example the Go runtime when processing an internal + // assertion error, to interleave its writes going to stderr + // together with a logger that wants to insert log redaction markers + // also on stderr. Because the log code cannot control this + // interleaving, it cannot guarantee that the direct fd 2 writes + // won't be positioned outside of log redaction markers and + // mistakenly considered as "safe for reporting". + + if mainLog.logDir.IsSet() { + // We have a log directory. We can enable stderr redirection. + + // Our own cancellable context to stop the secondary logger. + // + // Note: we don't want to take a cancellable context from the + // caller, because in the usual case we don't want to stop the + // logger when the remainder of the process stops. See the + // discussion on cancel at the top of the function. + ctx, cancel := context.WithCancel(context.Background()) + secLogger := NewSecondaryLogger(ctx, &mainLog.logDir, "stderr", + true /* enableGC */, true /* forceSyncWrites */, false /* enableMsgCount */) + + // This logger will capture direct stderr writes. + secLogger.logger.redirectInternalStderrWrites = true + // Stderr capture produces unsafe strings. This logger + // thus generally produces non-redactable entries. + secLogger.logger.redactableLogs.Set(false) + + // Force a log entry. This does two things: it forces + // the creation of a file and the redirection of fd 2 / os.Stderr. + // It also introduces a timestamp marker. + secLogger.Logf(ctx, "stderr capture started") + prevStderrLogger := stderrLog + stderrLog = &secLogger.logger + + // The cleanup fn is for use in tests. + cleanup := func() { + // Restore the apparent stderr logger used by Shout() and tests. + stderrLog = prevStderrLogger + + // Cancel the gc process for the secondary logger and close it. + // + // Note: this will close the file descriptor 2 used for direct + // writes to fd 2, as well as os.Stderr. We don't restore the + // stderr output with e.g. hijackStderr(OrigStderr). This is + // intentional. We can't risk redirecting that to the main + // logger's output file, or the terminal, if the main logger can + // also interleave redactable log entries. + cancel() + secLogger.Close() + } + + // Now that stderr is properly redirected, we can enable log file + // redaction as requested. It is safe because no interleaving + // is possible any more. + mainLog.redactableLogs.Set(redactableLogsRequested) + + return cleanup, nil + } + + // There is no log directory. + + // If redaction is requested and we have a change to produce some + // log entries on stderr, that's a configuration we cannot support + // safely. Reject it. + if redactableLogsRequested && mainLog.stderrThreshold.get() != Severity_NONE { + return nil, errors.New("cannot enable redactable logging without a logging directory") + } + + // Configuration valid. Assign it. + // (Note: This is a no-op, because either redactableLogsRequested is false, + // or it's true but stderrThreshold filters everything.) + mainLog.redactableLogs.Set(redactableLogsRequested) + return nil, nil +} + +// We use redactableLogsRequested instead of mainLog.redactableLogs +// directly when parsing command-line flags, to prevent the redactable +// flag from being set until SetupRedactionAndStderrRedirects() has +// been called. +// +// This ensures that we don't mistakenly start producing redaction +// markers until we have some confidence they won't be interleaved +// with arbitrary writes to the stderr file descriptor. +var redactableLogsRequested bool diff --git a/pkg/util/log/log.go b/pkg/util/log/log.go index 1f5eee29b45d..f403e6456351 100644 --- a/pkg/util/log/log.go +++ b/pkg/util/log/log.go @@ -65,7 +65,9 @@ func Shoutf(ctx context.Context, sev Severity, format string, args ...interface{ // however this is what the Shout() contract guarantees, so we do // it here. fmt.Fprintf(OrigStderr, "*\n* %s: %s\n*\n", sev.String(), - strings.Replace(MakeMessage(ctx, format, args), "\n", "\n* ", -1)) + strings.Replace( + FormatWithContextTags(ctx, format, args...), + "\n", "\n* ", -1)) } logDepth(ctx, 1, sev, format, args) } diff --git a/pkg/util/log/log.pb.go b/pkg/util/log/log.pb.go index 349ae72af696..96c5698938e4 100644 --- a/pkg/util/log/log.pb.go +++ b/pkg/util/log/log.pb.go @@ -61,7 +61,7 @@ func (x Severity) String() string { return proto.EnumName(Severity_name, int32(x)) } func (Severity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_log_127175c418f19ec4, []int{0} + return fileDescriptor_log_a411727cdcce1616, []int{0} } // Entry represents a cockroach structured log entry. @@ -73,13 +73,30 @@ type Entry struct { File string `protobuf:"bytes,3,opt,name=file,proto3" json:"file,omitempty"` Line int64 `protobuf:"varint,4,opt,name=line,proto3" json:"line,omitempty"` Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"` + // tags contains the context tags available in the context where the + // entry was created. + Tags string `protobuf:"bytes,7,opt,name=tags,proto3" json:"tags,omitempty"` + // counter is an entry counter, meant for use in audit logs as an + // instrument against log repudiation. + // See: https://en.wikipedia.org/wiki/Non-repudiation + // + // It is incremented for every use of the logger where the entry was + // produced. + Counter uint64 `protobuf:"varint,8,opt,name=counter,proto3" json:"counter,omitempty"` + // redactable is true if the message and tags fields include markers + // to delineate sensitive information. In that case, confidentiality + // can be obtained by only stripping away the data within this + // marker. If redactable is false or unknown, the message should be + // considered to only contain sensitive information, and should be + // stripped away completely for confidentiality. + Redactable bool `protobuf:"varint,9,opt,name=redactable,proto3" json:"redactable,omitempty"` } func (m *Entry) Reset() { *m = Entry{} } func (m *Entry) String() string { return proto.CompactTextString(m) } func (*Entry) ProtoMessage() {} func (*Entry) Descriptor() ([]byte, []int) { - return fileDescriptor_log_127175c418f19ec4, []int{0} + return fileDescriptor_log_a411727cdcce1616, []int{0} } func (m *Entry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -118,7 +135,7 @@ func (m *FileDetails) Reset() { *m = FileDetails{} } func (m *FileDetails) String() string { return proto.CompactTextString(m) } func (*FileDetails) ProtoMessage() {} func (*FileDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_log_127175c418f19ec4, []int{1} + return fileDescriptor_log_a411727cdcce1616, []int{1} } func (m *FileDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -154,7 +171,7 @@ func (m *FileInfo) Reset() { *m = FileInfo{} } func (m *FileInfo) String() string { return proto.CompactTextString(m) } func (*FileInfo) ProtoMessage() {} func (*FileInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_log_127175c418f19ec4, []int{2} + return fileDescriptor_log_a411727cdcce1616, []int{2} } func (m *FileInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -232,6 +249,27 @@ func (m *Entry) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintLog(dAtA, i, uint64(m.Goroutine)) } + if len(m.Tags) > 0 { + dAtA[i] = 0x3a + i++ + i = encodeVarintLog(dAtA, i, uint64(len(m.Tags))) + i += copy(dAtA[i:], m.Tags) + } + if m.Counter != 0 { + dAtA[i] = 0x40 + i++ + i = encodeVarintLog(dAtA, i, uint64(m.Counter)) + } + if m.Redactable { + dAtA[i] = 0x48 + i++ + if m.Redactable { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -358,6 +396,16 @@ func (m *Entry) Size() (n int) { if m.Goroutine != 0 { n += 1 + sovLog(uint64(m.Goroutine)) } + l = len(m.Tags) + if l > 0 { + n += 1 + l + sovLog(uint64(l)) + } + if m.Counter != 0 { + n += 1 + sovLog(uint64(m.Counter)) + } + if m.Redactable { + n += 2 + } return n } @@ -585,6 +633,74 @@ func (m *Entry) Unmarshal(dAtA []byte) error { break } } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tags", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLog + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthLog + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Tags = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Counter", wireType) + } + m.Counter = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLog + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Counter |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Redactable", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLog + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Redactable = bool(v != 0) default: iNdEx = preIndex skippy, err := skipLog(dAtA[iNdEx:]) @@ -1033,39 +1149,41 @@ var ( ErrIntOverflowLog = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/log/log.proto", fileDescriptor_log_127175c418f19ec4) } +func init() { proto.RegisterFile("util/log/log.proto", fileDescriptor_log_a411727cdcce1616) } -var fileDescriptor_log_127175c418f19ec4 = []byte{ - // 485 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x92, 0xc1, 0x8a, 0xd3, 0x40, - 0x18, 0xc7, 0x33, 0x9b, 0xa4, 0x4d, 0xa6, 0xb2, 0x84, 0xc1, 0x43, 0xd4, 0x35, 0x2d, 0x45, 0xa1, - 0x78, 0xc8, 0xc2, 0x7a, 0xf1, 0x26, 0x2d, 0x6d, 0xa5, 0xba, 0x4c, 0x65, 0xec, 0xb2, 0xa0, 0x87, - 0x92, 0x6d, 0x67, 0xb3, 0x83, 0x49, 0xa6, 0x24, 0x53, 0xa1, 0xbe, 0x83, 0xe0, 0x43, 0xf8, 0x0a, - 0xbe, 0x43, 0x8f, 0x7b, 0xdc, 0xd3, 0xa2, 0xe9, 0x8b, 0xc8, 0x37, 0xd9, 0x68, 0xc1, 0x3d, 0x04, - 0xfe, 0xf3, 0x9b, 0xff, 0x7c, 0xf3, 0xff, 0xbe, 0x09, 0x26, 0x6b, 0x25, 0x92, 0xe3, 0x44, 0xc6, - 0xf0, 0x85, 0xab, 0x5c, 0x2a, 0x49, 0xc8, 0x42, 0x2e, 0x3e, 0xe7, 0x32, 0x5a, 0x5c, 0x85, 0xb0, - 0x1b, 0x26, 0x32, 0x7e, 0xfc, 0x30, 0x96, 0xb1, 0xd4, 0xdb, 0xc7, 0xa0, 0x2a, 0x67, 0xf7, 0x27, - 0xc2, 0xf6, 0x28, 0x53, 0xf9, 0x86, 0xbc, 0xc2, 0x4e, 0xc1, 0xbf, 0xf0, 0x5c, 0xa8, 0x8d, 0x8f, - 0x3a, 0xa8, 0x77, 0x78, 0x72, 0x14, 0xfe, 0x5f, 0x26, 0xfc, 0x70, 0xe7, 0x61, 0x7f, 0xdd, 0x84, - 0x60, 0x4b, 0x89, 0x94, 0xfb, 0x07, 0x1d, 0xd4, 0x33, 0x99, 0xd6, 0xc0, 0x2e, 0x45, 0xc2, 0x7d, - 0xb3, 0x83, 0x7a, 0x2e, 0xd3, 0x1a, 0x58, 0x22, 0x32, 0xee, 0x5b, 0x95, 0x0f, 0x34, 0xf1, 0x71, - 0x33, 0xe5, 0x45, 0x11, 0xc5, 0xdc, 0xb7, 0xb5, 0xb5, 0x5e, 0x92, 0x23, 0xec, 0xc6, 0x32, 0x97, - 0x6b, 0x05, 0x47, 0x1a, 0xfa, 0xc8, 0x3f, 0xd0, 0xfd, 0x86, 0x70, 0x6b, 0x2c, 0x12, 0x3e, 0xe4, - 0x2a, 0x12, 0x49, 0x01, 0x75, 0x56, 0xb9, 0x8c, 0xf3, 0x28, 0xd5, 0xe1, 0x5d, 0x56, 0x2f, 0xe1, - 0xd6, 0x2b, 0x59, 0x28, 0x9d, 0xce, 0x65, 0x5a, 0x93, 0x27, 0xd8, 0x5d, 0x17, 0x3c, 0x9f, 0x67, - 0x51, 0x5a, 0x47, 0x74, 0x00, 0xd0, 0xa8, 0x8a, 0xae, 0xdb, 0xb1, 0xf7, 0xda, 0x79, 0x84, 0xcd, - 0x95, 0x58, 0x56, 0x31, 0x06, 0xcd, 0xf2, 0xb6, 0x6d, 0xbe, 0x9f, 0x0c, 0x19, 0xb0, 0xb7, 0x96, - 0x63, 0x79, 0x76, 0xf7, 0x07, 0xc2, 0x0e, 0xe4, 0x99, 0x64, 0x97, 0x12, 0x2a, 0xe8, 0xca, 0x55, - 0x12, 0xad, 0xc9, 0x53, 0x8c, 0x0b, 0xf1, 0x95, 0xcf, 0x2f, 0x36, 0x8a, 0x17, 0x77, 0xa3, 0x72, - 0x81, 0x0c, 0x00, 0x90, 0x67, 0xf8, 0x30, 0x95, 0xcb, 0x39, 0x5c, 0x36, 0xcf, 0xa2, 0x4c, 0x16, - 0x3a, 0x96, 0xc9, 0x1e, 0xa4, 0x72, 0x39, 0x13, 0x29, 0xa7, 0xc0, 0xc8, 0x6b, 0xdc, 0x5c, 0x56, - 0x0d, 0xeb, 0x21, 0xb6, 0x4e, 0xda, 0xf7, 0x3d, 0xd1, 0xde, 0x5c, 0x06, 0xd6, 0xf6, 0xb6, 0x6d, - 0xb0, 0xfa, 0xd4, 0x8b, 0x4f, 0xd8, 0xa9, 0x1f, 0x90, 0xb4, 0x70, 0xf3, 0x8c, 0xbe, 0xa3, 0xd3, - 0x73, 0xea, 0x19, 0xc4, 0xc1, 0xd6, 0x84, 0x8e, 0xa7, 0x1e, 0x02, 0x7c, 0xde, 0x67, 0x74, 0x42, - 0xdf, 0x78, 0x07, 0xc4, 0xc5, 0xf6, 0x88, 0xb1, 0x29, 0xf3, 0x4c, 0x90, 0xe3, 0xfe, 0xac, 0x7f, - 0xea, 0x59, 0x60, 0xa6, 0x53, 0x3a, 0xf2, 0x6c, 0x30, 0x0f, 0x47, 0xe3, 0xfe, 0xd9, 0xe9, 0xcc, - 0x6b, 0x0c, 0x9e, 0x6f, 0x7f, 0x07, 0xc6, 0xb6, 0x0c, 0xd0, 0x75, 0x19, 0xa0, 0x9b, 0x32, 0x40, - 0xbf, 0xca, 0x00, 0x7d, 0xdf, 0x05, 0xc6, 0xf5, 0x2e, 0x30, 0x6e, 0x76, 0x81, 0xf1, 0xd1, 0x4c, - 0x64, 0x7c, 0xd1, 0xd0, 0x7f, 0xde, 0xcb, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x15, 0x0b, 0x0c, - 0xe9, 0xb9, 0x02, 0x00, 0x00, +var fileDescriptor_log_a411727cdcce1616 = []byte{ + // 522 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x52, 0x41, 0x8b, 0xd3, 0x40, + 0x14, 0xce, 0x34, 0x49, 0x9b, 0xbc, 0xca, 0x12, 0x06, 0x0f, 0x51, 0xd7, 0xb4, 0x14, 0x85, 0xe2, + 0xa1, 0x0b, 0xeb, 0xc5, 0x9b, 0xb4, 0xb4, 0x95, 0xea, 0x92, 0xca, 0xd8, 0x65, 0x41, 0x0f, 0x25, + 0x6d, 0x67, 0xb3, 0xc1, 0x24, 0x53, 0x92, 0xa9, 0x50, 0xaf, 0x9e, 0x05, 0x7f, 0x84, 0x3f, 0xa6, + 0xc7, 0x3d, 0xee, 0x69, 0xd1, 0xf4, 0x8f, 0xc8, 0x9b, 0x6c, 0xb4, 0xe0, 0x1e, 0x02, 0xdf, 0x7c, + 0xf9, 0xde, 0xf7, 0xde, 0xfb, 0x66, 0x80, 0x6e, 0x64, 0x14, 0x9f, 0xc4, 0x22, 0xc4, 0xaf, 0xb7, + 0xce, 0x84, 0x14, 0x94, 0x2e, 0xc5, 0xf2, 0x73, 0x26, 0x82, 0xe5, 0x55, 0x0f, 0xff, 0xf6, 0x62, + 0x11, 0x3e, 0x7e, 0x18, 0x8a, 0x50, 0xa8, 0xdf, 0x27, 0x88, 0x4a, 0x65, 0xe7, 0x5b, 0x0d, 0xcc, + 0x51, 0x2a, 0xb3, 0x2d, 0x7d, 0x05, 0x56, 0xce, 0xbf, 0xf0, 0x2c, 0x92, 0x5b, 0x97, 0xb4, 0x49, + 0xf7, 0xe8, 0xf4, 0xb8, 0xf7, 0xbf, 0x4d, 0xef, 0xc3, 0x9d, 0x86, 0xfd, 0x55, 0x53, 0x0a, 0x86, + 0x8c, 0x12, 0xee, 0xd6, 0xda, 0xa4, 0xab, 0x33, 0x85, 0x91, 0xbb, 0x8c, 0x62, 0xee, 0xea, 0x6d, + 0xd2, 0xb5, 0x99, 0xc2, 0xc8, 0xc5, 0x51, 0xca, 0x5d, 0xa3, 0xd4, 0x21, 0xa6, 0x2e, 0x34, 0x12, + 0x9e, 0xe7, 0x41, 0xc8, 0x5d, 0x53, 0x49, 0xab, 0x23, 0x3d, 0x06, 0x3b, 0x14, 0x99, 0xd8, 0x48, + 0x2c, 0xa9, 0xab, 0x92, 0x7f, 0x84, 0xea, 0x19, 0x84, 0xb9, 0xdb, 0x28, 0xfd, 0x11, 0xa3, 0xd7, + 0x52, 0x6c, 0x52, 0xc9, 0x33, 0xd7, 0x6a, 0x93, 0xae, 0xc1, 0xaa, 0x23, 0xf5, 0x00, 0x32, 0xbe, + 0x0a, 0x96, 0x32, 0x58, 0xc4, 0xdc, 0xb5, 0xdb, 0xa4, 0x6b, 0xb1, 0x03, 0xa6, 0xf3, 0x9d, 0x40, + 0x73, 0x1c, 0xc5, 0x7c, 0xc8, 0x65, 0x10, 0xc5, 0xca, 0x69, 0x9d, 0x89, 0x30, 0x0b, 0x12, 0x15, + 0x85, 0xcd, 0xaa, 0x23, 0xf6, 0xbd, 0x12, 0xb9, 0x54, 0xbb, 0xda, 0x4c, 0x61, 0xfa, 0x04, 0xec, + 0x4d, 0xce, 0xb3, 0x79, 0x1a, 0x24, 0xd5, 0xc2, 0x16, 0x12, 0x7e, 0x50, 0x06, 0xa1, 0xc2, 0x31, + 0x0f, 0xc2, 0x79, 0x04, 0xfa, 0x3a, 0x5a, 0x95, 0x4b, 0x0d, 0x1a, 0xc5, 0x6d, 0x4b, 0x7f, 0x3f, + 0x19, 0x32, 0xe4, 0xde, 0x1a, 0x96, 0xe1, 0x98, 0x9d, 0x9f, 0x04, 0x2c, 0x9c, 0x67, 0x92, 0x5e, + 0x0a, 0x74, 0x50, 0xce, 0xe5, 0x24, 0x0a, 0xd3, 0xa7, 0x00, 0x79, 0xf4, 0x95, 0xcf, 0x17, 0x5b, + 0xc9, 0xf3, 0xbb, 0xe0, 0x6d, 0x64, 0x06, 0x48, 0xd0, 0x67, 0x70, 0x94, 0x88, 0xd5, 0x1c, 0x9b, + 0xcd, 0xd3, 0x20, 0x15, 0xb9, 0x1a, 0x4b, 0x67, 0x0f, 0x12, 0xb1, 0x9a, 0x45, 0x09, 0xf7, 0x91, + 0xa3, 0xaf, 0xa1, 0xb1, 0x2a, 0x17, 0x56, 0x57, 0xd2, 0x3c, 0x6d, 0xdd, 0x77, 0xe1, 0x07, 0xb9, + 0x0c, 0x8c, 0xdd, 0x6d, 0x4b, 0x63, 0x55, 0xd5, 0x8b, 0x4f, 0x60, 0x55, 0xcf, 0x81, 0x36, 0xa1, + 0x71, 0xee, 0xbf, 0xf3, 0xa7, 0x17, 0xbe, 0xa3, 0x51, 0x0b, 0x8c, 0x89, 0x3f, 0x9e, 0x3a, 0x04, + 0xe9, 0x8b, 0x3e, 0xf3, 0x27, 0xfe, 0x1b, 0xa7, 0x46, 0x6d, 0x30, 0x47, 0x8c, 0x4d, 0x99, 0xa3, + 0x23, 0x1c, 0xf7, 0x67, 0xfd, 0x33, 0xc7, 0x40, 0xb1, 0x3f, 0xf5, 0x47, 0x8e, 0x89, 0xe2, 0xe1, + 0x68, 0xdc, 0x3f, 0x3f, 0x9b, 0x39, 0xf5, 0xc1, 0xf3, 0xdd, 0x6f, 0x4f, 0xdb, 0x15, 0x1e, 0xb9, + 0x2e, 0x3c, 0x72, 0x53, 0x78, 0xe4, 0x57, 0xe1, 0x91, 0x1f, 0x7b, 0x4f, 0xbb, 0xde, 0x7b, 0xda, + 0xcd, 0xde, 0xd3, 0x3e, 0xea, 0xb1, 0x08, 0x17, 0x75, 0xf5, 0x8e, 0x5f, 0xfe, 0x09, 0x00, 0x00, + 0xff, 0xff, 0x1e, 0xd0, 0xea, 0xd5, 0x07, 0x03, 0x00, 0x00, } diff --git a/pkg/util/log/log.proto b/pkg/util/log/log.proto index 177449939da9..e579687f4738 100644 --- a/pkg/util/log/log.proto +++ b/pkg/util/log/log.proto @@ -39,6 +39,26 @@ message Entry { string file = 3; int64 line = 4; string message = 5; + + // tags contains the context tags available in the context where the + // entry was created. + string tags = 7; + + // counter is an entry counter, meant for use in audit logs as an + // instrument against log repudiation. + // See: https://en.wikipedia.org/wiki/Non-repudiation + // + // It is incremented for every use of the logger where the entry was + // produced. + uint64 counter = 8; + + // redactable is true if the message and tags fields include markers + // to delineate sensitive information. In that case, confidentiality + // can be obtained by only stripping away the data within this + // marker. If redactable is false or unknown, the message should be + // considered to only contain sensitive information, and should be + // stripped away completely for confidentiality. + bool redactable = 9; } // A FileDetails holds all of the particulars that can be parsed by the name of diff --git a/pkg/util/log/log_bridge.go b/pkg/util/log/log_bridge.go index f9298958cf69..09ae9c253cc1 100644 --- a/pkg/util/log/log_bridge.go +++ b/pkg/util/log/log_bridge.go @@ -12,6 +12,7 @@ package log import ( "bytes" + "context" "fmt" stdLog "log" "strconv" @@ -54,23 +55,26 @@ func init() { // Write parses the standard logging line and passes its components to the // logger for Severity(lb). func (lb logBridge) Write(b []byte) (n int, err error) { - var ( - file = "???" - line = 1 - text string - ) + entry := MakeEntry(context.Background(), + Severity(lb), &mainLog.logCounter, 0, /* depth */ + // Note: because the caller is using the stdLog interface, they are + // bypassing all the log marker logic. This means that the entire + // log message should be assumed to contain confidential + // information—it is thus not redactable. + false /* redactable */, "") + // Split "d.go:23: message" into "d.go", "23", and "message". if parts := bytes.SplitN(b, []byte{':'}, 3); len(parts) != 3 || len(parts[0]) < 1 || len(parts[2]) < 1 { - text = fmt.Sprintf("bad log format: %s", b) + entry.Message = fmt.Sprintf("bad log format: %s", b) } else { - file = string(parts[0]) - text = string(parts[2][1 : len(parts[2])-1]) // skip leading space and trailing newline - line, err = strconv.Atoi(string(parts[1])) + entry.File = string(parts[0]) + entry.Message = string(parts[2][1 : len(parts[2])-1]) // skip leading space and trailing newline + entry.Line, err = strconv.ParseInt(string(parts[1]), 10, 64) if err != nil { - text = fmt.Sprintf("bad line number: %s", b) - line = 1 + entry.Message = fmt.Sprintf("bad line number: %s", b) + entry.Line = 1 } } - mainLog.outputLogEntry(Severity(lb), file, line, text) + mainLog.outputLogEntry(entry) return len(b), nil } diff --git a/pkg/util/log/log_buffer.go b/pkg/util/log/log_buffer.go index 3edc8e1996ee..04efde3eed96 100644 --- a/pkg/util/log/log_buffer.go +++ b/pkg/util/log/log_buffer.go @@ -68,7 +68,7 @@ func (buf *buffer) nDigits(n, i, d int, pad byte) int { return n } -// someDigits formats a zero-prefixed variable-width integer at buf.tmp[i]. +// someDigits formats a variable-width integer at buf.tmp[i]. func (buf *buffer) someDigits(i, d int) int { // Print into the top, then copy down. We know there's space for at least // a 10-digit number. diff --git a/pkg/util/log/log_entry.go b/pkg/util/log/log_entry.go index ec9573efeb0d..3bc630841554 100644 --- a/pkg/util/log/log_entry.go +++ b/pkg/util/log/log_entry.go @@ -12,26 +12,26 @@ package log import ( "bufio" + "context" + "fmt" "io" "regexp" "strconv" "strings" + "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/util/caller" + "github.com/cockroachdb/cockroach/pkg/util/redact" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/ttycolor" "github.com/petermattis/goid" ) -// the --no-color flag. -var noColor bool - // formatLogEntry formats an Entry into a newly allocated *buffer. // The caller is responsible for calling putBuffer() afterwards. func (l *loggingT) formatLogEntry(entry Entry, stacks []byte, cp ttycolor.Profile) *buffer { - buf := l.formatHeader(entry.Severity, timeutil.Unix(0, entry.Time), - int(entry.Goroutine), entry.File, int(entry.Line), cp) - _, _ = buf.WriteString(entry.Message) + buf := l.formatLogEntryInternal(entry, cp) if buf.Bytes()[buf.Len()-1] != '\n' { _ = buf.WriteByte('\n') } @@ -41,13 +41,13 @@ func (l *loggingT) formatLogEntry(entry Entry, stacks []byte, cp ttycolor.Profil return buf } -// formatHeader formats a log header using the provided file name and -// line number. Log lines are colorized depending on severity. +// formatEntryInternal renders a log entry. +// Log lines are colorized depending on severity. // It uses a newly allocated *buffer. The caller is responsible // for calling putBuffer() afterwards. // // Log lines have this form: -// Lyymmdd hh:mm:ss.uuuuuu goid file:line msg... +// Lyymmdd hh:mm:ss.uuuuuu goid file:line [tags] counter msg... // where the fields are defined as follows: // L A single character, representing the log level (eg 'I' for INFO) // yy The year (zero padded; ie 2016 is '16') @@ -57,26 +57,26 @@ func (l *loggingT) formatLogEntry(entry Entry, stacks []byte, cp ttycolor.Profil // goid The goroutine id (omitted if zero for use by tests) // file The file name // line The line number +// tags The context tags +// counter The log entry counter, if non-zero // msg The user-supplied message -func (l *loggingT) formatHeader( - s Severity, now time.Time, gid int, file string, line int, cp ttycolor.Profile, -) *buffer { - if noColor { +func (l *loggingT) formatLogEntryInternal(entry Entry, cp ttycolor.Profile) *buffer { + if l.noColor { cp = nil } buf := getBuffer() - if line < 0 { - line = 0 // not a real line number, but acceptable to someDigits + if entry.Line < 0 { + entry.Line = 0 // not a real line number, but acceptable to someDigits } - if s > Severity_FATAL || s <= Severity_UNKNOWN { - s = Severity_INFO // for safety. + if entry.Severity > Severity_FATAL || entry.Severity <= Severity_UNKNOWN { + entry.Severity = Severity_INFO // for safety. } tmp := buf.tmp[:len(buf.tmp)] var n int var prefix []byte - switch s { + switch entry.Severity { case Severity_INFO: prefix = cp[ttycolor.Cyan] case Severity_WARNING: @@ -87,10 +87,11 @@ func (l *loggingT) formatHeader( n += copy(tmp, prefix) // Avoid Fprintf, for speed. The format is so simple that we can do it quickly by hand. // It's worth about 3X. Fprintf is hard. + now := timeutil.Unix(0, entry.Time) year, month, day := now.Date() hour, minute, second := now.Clock() // Lyymmdd hh:mm:ss.uuuuuu file:line - tmp[n] = severityChar[s-1] + tmp[n] = severityChar[entry.Severity-1] n++ if year < 2000 { year = 2000 @@ -113,23 +114,58 @@ func (l *loggingT) formatHeader( n += buf.nDigits(6, n, now.Nanosecond()/1000, '0') tmp[n] = ' ' n++ - if gid > 0 { - n += buf.someDigits(n, gid) + if entry.Goroutine > 0 { + n += buf.someDigits(n, int(entry.Goroutine)) tmp[n] = ' ' n++ } buf.Write(tmp[:n]) - buf.WriteString(file) + buf.WriteString(entry.File) tmp[0] = ':' - n = buf.someDigits(1, line) + n = buf.someDigits(1, int(entry.Line)) n++ - // Extra space between the header and the actual message for scannability. + // Reset the color to default. + n += copy(tmp[n:], cp[ttycolor.Reset]) tmp[n] = ' ' n++ - n += copy(tmp[n:], cp[ttycolor.Reset]) + // If redaction is enabled, indicate that the current entry has + // markers. This indicator is used in the log parser to determine + // which redaction strategy to adopt. + if entry.Redactable { + copy(tmp[n:], redactableIndicatorBytes) + n += len(redactableIndicatorBytes) + } + // Note: when the redactable indicator is not introduced + // there are two spaces next to each other. This is intended + // and should be preserved for backward-compatibility with + // 3rd party log parsers. tmp[n] = ' ' n++ buf.Write(tmp[:n]) + + // The remainder is variable-length and could exceed + // the static size of tmp. But we do have an upper bound. + buf.Grow(len(entry.Tags) + 14 + len(entry.Message)) + + // Display the tags if set. + if len(entry.Tags) != 0 { + buf.Write(cp[ttycolor.Blue]) + buf.WriteByte('[') + buf.WriteString(entry.Tags) + buf.WriteString("] ") + buf.Write(cp[ttycolor.Reset]) + } + + // Display the counter if set. + if entry.Counter > 0 { + n = buf.someDigits(0, int(entry.Counter)) + tmp[n] = ' ' + n++ + buf.Write(tmp[:n]) + } + + // Display the message. + buf.WriteString(entry.Message) return buf } @@ -144,14 +180,65 @@ func (l *loggingT) processForFile(entry Entry, stacks []byte) *buffer { } // MakeEntry creates an Entry. -func MakeEntry(s Severity, t int64, file string, line int, msg string) Entry { - return Entry{ - Severity: s, - Time: t, - Goroutine: goid.Get(), - File: file, - Line: int64(line), - Message: msg, +func MakeEntry( + ctx context.Context, + s Severity, + lc *EntryCounter, + depth int, + redactable bool, + format string, + args ...interface{}, +) (res Entry) { + res = Entry{ + Severity: s, + Time: timeutil.Now().UnixNano(), + Goroutine: goid.Get(), + Redactable: redactable, + } + + // Populate file/lineno. + file, line, _ := caller.Lookup(depth + 1) + res.File = file + res.Line = int64(line) + + // Optionally populate the counter. + if lc != nil && lc.EnableMsgCount { + // Add a counter. This is important for e.g. the SQL audit logs. + res.Counter = atomic.AddUint64(&lc.msgCount, 1) + } + + // Populate the tags. + var buf strings.Builder + if redactable { + redactTags(ctx, &buf) + } else { + formatTags(ctx, false /* brackets */, &buf) + } + res.Tags = buf.String() + + // Populate the message. + buf.Reset() + renderArgs(redactable, &buf, format, args...) + res.Message = buf.String() + + return +} + +func renderArgs(redactable bool, buf *strings.Builder, format string, args ...interface{}) { + if len(args) == 0 { + buf.WriteString(format) + } else if len(format) == 0 { + if redactable { + redact.Fprint(buf, args...) + } else { + fmt.Fprint(buf, args...) + } + } else { + if redactable { + redact.Fprintf(buf, format, args...) + } else { + fmt.Fprintf(buf, format, args...) + } } } @@ -167,7 +254,14 @@ func (e Entry) Format(w io.Writer) error { // preamble, because a capture group that handles multiline messages is very // slow when running on the large buffers passed to EntryDecoder.split. var entryRE = regexp.MustCompile( - `(?m)^([IWEF])(\d{6} \d{2}:\d{2}:\d{2}.\d{6}) (?:(\d+) )?([^:]+):(\d+)`) + `(?m)^` + + /* Severity */ `([IWEF])` + + /* Date and time */ `(\d{6} \d{2}:\d{2}:\d{2}.\d{6}) ` + + /* Goroutine ID */ `(?:(\d+) )?` + + /* File/Line */ `([^:]+):(\d+) ` + + /* Redactable flag */ `((?:` + redactableIndicator + `)?) ` + + /* Context tags */ `(?:\[([^]]+)\] )?`, +) // EntryDecoder reads successive encoded log entries from the input // buffer. Each entry is preceded by a single big-ending uint32 @@ -175,12 +269,17 @@ var entryRE = regexp.MustCompile( type EntryDecoder struct { re *regexp.Regexp scanner *bufio.Scanner + sensitiveEditor redactEditor truncatedLastEntry bool } // NewEntryDecoder creates a new instance of EntryDecoder. -func NewEntryDecoder(in io.Reader) *EntryDecoder { - d := &EntryDecoder{scanner: bufio.NewScanner(in), re: entryRE} +func NewEntryDecoder(in io.Reader, editMode EditSensitiveData) *EntryDecoder { + d := &EntryDecoder{ + re: entryRE, + scanner: bufio.NewScanner(in), + sensitiveEditor: getEditor(editMode), + } d.scanner.Split(d.split) return d } @@ -203,12 +302,18 @@ func (d *EntryDecoder) Decode(entry *Entry) error { if m == nil { continue } + + // Process the severity. entry.Severity = Severity(strings.IndexByte(severityChar, m[1][0]) + 1) + + // Process the timestamp. t, err := time.Parse(MessageTimeFormat, string(m[2])) if err != nil { return err } entry.Time = t.UnixNano() + + // Process the goroutine ID. if len(m[3]) > 0 { goroutine, err := strconv.Atoi(string(m[3])) if err != nil { @@ -216,17 +321,49 @@ func (d *EntryDecoder) Decode(entry *Entry) error { } entry.Goroutine = int64(goroutine) } + + // Process the file/line details. entry.File = string(m[4]) line, err := strconv.Atoi(string(m[5])) if err != nil { return err } entry.Line = int64(line) - entry.Message = strings.TrimSpace(string(b[len(m[0]):])) + + // Process the context tags. + redactable := len(m[6]) != 0 + if len(m[7]) != 0 { + r := redactablePackage{ + msg: m[7], + redactable: redactable, + } + r = d.sensitiveEditor(r) + entry.Tags = string(r.msg) + } + + // Process the log message itself + r := redactablePackage{ + msg: trimFinalNewLines(b[len(m[0]):]), + redactable: redactable, + } + r = d.sensitiveEditor(r) + entry.Message = string(r.msg) + entry.Redactable = r.redactable return nil } } +func trimFinalNewLines(s []byte) []byte { + for i := len(s) - 1; i >= 0; i-- { + if s[i] == '\n' { + s = s[:i] + } else { + break + } + } + return s +} + func (d *EntryDecoder) split(data []byte, atEOF bool) (advance int, token []byte, err error) { if atEOF && len(data) == 0 { return 0, nil, nil diff --git a/pkg/util/log/log_gc_test.go b/pkg/util/log/log_gc_test.go index e7e55a97482f..0ca764a7c3fc 100644 --- a/pkg/util/log/log_gc_test.go +++ b/pkg/util/log/log_gc_test.go @@ -75,14 +75,6 @@ func testLogGC( const newLogFiles = 20 - // Prevent direct writes to fd 2 from being sent to log - // files which would screw up the expected number of log file - // calculation below. - // - // TODO(knz): I suspect this is not needed. Investigate and perhaps - // remove. - stderrLog.noRedirectInternalStderrWrites = true - // Ensure the main log has at least one entry. This serves two // purposes. One is to serve as baseline for the file size. The // other is to ensure that the TestLogScope does not erase the diff --git a/pkg/util/log/logflags/logflags.go b/pkg/util/log/logflags/logflags.go index ef94b9d620d6..4c50397a78b2 100644 --- a/pkg/util/log/logflags/logflags.go +++ b/pkg/util/log/logflags/logflags.go @@ -60,9 +60,9 @@ var _ flag.Value = &atomicBool{} const ( LogToStderrName = "logtostderr" NoColorName = "no-color" + RedactableLogsName = "redactable-logs" VModuleName = "vmodule" LogDirName = "log-dir" - NoRedirectStderrName = "no-redirect-stderr" ShowLogsName = "show-logs" LogFileMaxSizeName = "log-file-max-size" LogFilesCombinedMaxSizeName = "log-dir-max-size" @@ -72,15 +72,15 @@ const ( // InitFlags creates logging flags which update the given variables. The passed mutex is // locked while the boolean variables are accessed during flag updates. func InitFlags( - noRedirectStderr *bool, logDir flag.Value, showLogs *bool, nocolor *bool, + redactableLogs *bool, vmodule flag.Value, logFileMaxSize, logFilesCombinedMaxSize *int64, ) { flag.BoolVar(nocolor, NoColorName, *nocolor, "disable standard error log colorization") - flag.BoolVar(noRedirectStderr, NoRedirectStderrName, *noRedirectStderr, "disable redirect of stderr to the log file") + flag.BoolVar(redactableLogs, RedactableLogsName, *redactableLogs, "make log outputs redactable for confidentiality") flag.Var(vmodule, VModuleName, "comma-separated list of pattern=N settings for file-filtered logging (significantly hurts performance)") flag.Var(logDir, LogDirName, "if non-empty, write log files in this directory") flag.BoolVar(showLogs, ShowLogsName, *showLogs, "print logs instead of saving them in files") diff --git a/pkg/util/log/loggers.go b/pkg/util/log/loggers.go new file mode 100644 index 000000000000..81895e671f34 --- /dev/null +++ b/pkg/util/log/loggers.go @@ -0,0 +1,21 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package log + +// mainLog is the primary logger instance. +var mainLog loggerT + +// stderrLog is the logger where writes performed directly +// to the stderr file descriptor (such as that performed +// by the go runtime) *may* be redirected. +// NB: whether they are actually redirected is determined +// by stderrLog.redirectInternalStderrWrites(). +var stderrLog = &mainLog diff --git a/pkg/util/log/redact.go b/pkg/util/log/redact.go new file mode 100644 index 000000000000..a6ae3297a8e7 --- /dev/null +++ b/pkg/util/log/redact.go @@ -0,0 +1,168 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package log + +import ( + "context" + "os" + "reflect" + "strings" + "time" + + "github.com/cockroachdb/cockroach/pkg/util/redact" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/logtags" +) + +// EditSensitiveData describes how the messages in log entries should +// be edited through the API. +type EditSensitiveData int + +const ( + invalidEditMode EditSensitiveData = iota + // WithMarkedSensitiveData is the "raw" log with sensitive data markers included. + WithMarkedSensitiveData + // WithFlattenedSensitiveData is the log with markers stripped. + WithFlattenedSensitiveData + // WithoutSensitiveData is the log with the sensitive data redacted. + WithoutSensitiveData +) + +// SelectEditMode returns an EditSensitiveData value that's suitable +// for use with NewDecoder depending on client-side desired +// "redact" and "keep redactable" flags. +// (See the documentation for the Logs and LogFile RPCs +// and that of the 'merge-logs' CLI command.) +func SelectEditMode(redact, keepRedactable bool) EditSensitiveData { + editMode := WithMarkedSensitiveData + if redact { + editMode = WithoutSensitiveData + } + if !keepRedactable && !redact { + editMode = WithFlattenedSensitiveData + } + return editMode +} + +type redactEditor func(redactablePackage) redactablePackage + +func getEditor(editMode EditSensitiveData) redactEditor { + switch editMode { + case WithMarkedSensitiveData: + return func(r redactablePackage) redactablePackage { + if !r.redactable { + r.msg = []byte(redact.EscapeBytes(r.msg)) + r.redactable = true + } + return r + } + case WithFlattenedSensitiveData: + return func(r redactablePackage) redactablePackage { + if r.redactable { + r.msg = redact.RedactableBytes(r.msg).StripMarkers() + r.redactable = false + } + return r + } + case WithoutSensitiveData: + return func(r redactablePackage) redactablePackage { + if r.redactable { + r.msg = []byte(redact.RedactableBytes(r.msg).Redact()) + } else { + r.msg = redact.RedactedMarker() + r.redactable = true + } + return r + } + case invalidEditMode: + fallthrough + default: + panic(errors.AssertionFailedf("unrecognized mode: %v", editMode)) + } +} + +// Safe constructs a SafeFormatter / SafeMessager. +// This is obsolete. Use redact.Safe directly. +// TODO(knz): Remove this. +var Safe = redact.Safe + +// SafeMessager aliases a type definition. +// This is obsolete. Use redact.SafeFormatter instead. +// TODO(knz): Remove this. +type SafeMessager = redact.SafeMessager + +func init() { + // We consider booleans and numeric values to be always safe for + // reporting. A log call can opt out by using redact.Unsafe() around + // a value that would be otherwise considered safe. + redact.RegisterSafeType(reflect.TypeOf(true)) // bool + redact.RegisterSafeType(reflect.TypeOf(123)) // int + redact.RegisterSafeType(reflect.TypeOf(int8(0))) + redact.RegisterSafeType(reflect.TypeOf(int16(0))) + redact.RegisterSafeType(reflect.TypeOf(int32(0))) + redact.RegisterSafeType(reflect.TypeOf(int64(0))) + redact.RegisterSafeType(reflect.TypeOf(uint8(0))) + redact.RegisterSafeType(reflect.TypeOf(uint16(0))) + redact.RegisterSafeType(reflect.TypeOf(uint32(0))) + redact.RegisterSafeType(reflect.TypeOf(uint64(0))) + redact.RegisterSafeType(reflect.TypeOf(float32(0))) + redact.RegisterSafeType(reflect.TypeOf(float64(0))) + redact.RegisterSafeType(reflect.TypeOf(complex64(0))) + redact.RegisterSafeType(reflect.TypeOf(complex128(0))) + // Signal names are also safe for reporting. + redact.RegisterSafeType(reflect.TypeOf(os.Interrupt)) + // Times and durations too. + redact.RegisterSafeType(reflect.TypeOf(time.Time{})) + redact.RegisterSafeType(reflect.TypeOf(time.Duration(0))) +} + +type redactablePackage struct { + msg []byte + redactable bool +} + +const redactableIndicator = "⋮" + +var redactableIndicatorBytes = []byte(redactableIndicator) + +func redactTags(ctx context.Context, buf *strings.Builder) { + tags := logtags.FromContext(ctx) + if tags == nil { + return + } + comma := "" + for _, t := range tags.Get() { + buf.WriteString(comma) + buf.WriteString(t.Key()) + if v := t.Value(); v != nil && v != "" { + if len(t.Key()) > 1 { + buf.WriteByte('=') + } + redact.Fprint(buf, v) + } + comma = "," + } +} + +// TestingSetRedactable sets the redactable flag for usage in a test. +// The caller is responsible for calling the cleanup function. This +// is exported for use in tests only -- it causes the logging +// configuration to be at risk of leaking unsafe information due to +// asynchronous direct writes to fd 2 / os.Stderr. +// +// See the discussion on SetupRedactionAndStderrRedirects() for +// details. +func TestingSetRedactable(redactableLogs bool) (cleanup func()) { + prev := mainLog.redactableLogs.Swap(redactableLogs) + return func() { + mainLog.redactableLogs.Set(prev) + } +} diff --git a/pkg/util/log/redact_test.go b/pkg/util/log/redact_test.go new file mode 100644 index 000000000000..37daad57a144 --- /dev/null +++ b/pkg/util/log/redact_test.go @@ -0,0 +1,181 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package log + +import ( + "context" + "fmt" + "io" + "path/filepath" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/redact" + "github.com/cockroachdb/logtags" + "github.com/stretchr/testify/assert" +) + +const startRedactable = "‹" +const endRedactable = "›" +const escapeMark = "?" + +// TestRedactedLogOutput ensures that the logging output emits markers +// when redactable logs are enabled, and no mark indicator when they +// are not. +func TestRedactedLogOutput(t *testing.T) { + s := ScopeWithoutShowLogs(t) + defer s.Close(t) + setFlags() + defer mainLog.swap(mainLog.newBuffers()) + + defer TestingSetRedactable(false)() + + Errorf(context.Background(), "test1 %v end", "hello") + if contains(redactableIndicator, t) { + t.Errorf("expected no marker indicator, got %q", contents()) + } + if !contains("test1 hello end", t) { + t.Errorf("expected no markers, got %q", contents()) + } + // Also verify that raw markers are preserved, when redactable + // markers are disabled. + mainLog.newBuffers() + Errorf(context.Background(), "test2 %v end", startRedactable+"hello"+endRedactable) + if !contains("test2 "+startRedactable+"hello"+endRedactable+" end", t) { + t.Errorf("expected unquoted markers, got %q", contents()) + } + + mainLog.newBuffers() + mainLog.redactableLogs.Set(true) + Errorf(context.Background(), "test3 %v end", "hello") + if !contains(redactableIndicator+" test3", t) { + t.Errorf("expected marker indicator, got %q", contents()) + } + if !contains("test3 "+startRedactable+"hello"+endRedactable+" end", t) { + t.Errorf("expected marked data, got %q", contents()) + } + // When redactable logs are enabled, the markers are always quoted. + mainLog.newBuffers() + const specialString = "x" + startRedactable + "hello" + endRedactable + "y" + Errorf(context.Background(), "test4 %v end", specialString) + if contains(specialString, t) { + t.Errorf("expected markers to be removed, got %q", contents()) + } + if !contains("test4 "+startRedactable+"x"+escapeMark+"hello"+escapeMark+"y"+endRedactable+" end", t) { + t.Errorf("expected escape mark, got %q", contents()) + } +} + +func quote(s string) string { + return startRedactable + s + endRedactable +} + +// TestRedactTags ensure that context tags can be redacted. +func TestRedactTags(t *testing.T) { + baseCtx := context.Background() + + testData := []struct { + ctx context.Context + expected string + }{ + {baseCtx, ""}, + {logtags.AddTag(baseCtx, "k", nil), "k"}, + {logtags.AddTag(baseCtx, "k", redact.Unsafe(123)), "k" + quote("123") + ""}, + {logtags.AddTag(baseCtx, "k", 123), "k123"}, + {logtags.AddTag(baseCtx, "k", redact.Safe(123)), "k123"}, + {logtags.AddTag(baseCtx, "k", startRedactable), "k" + quote(escapeMark) + ""}, + {logtags.AddTag(baseCtx, "kg", redact.Unsafe(123)), "kg=" + quote("123") + ""}, + {logtags.AddTag(baseCtx, "kg", 123), "kg=123"}, + {logtags.AddTag(baseCtx, "kg", redact.Safe(123)), "kg=123"}, + {logtags.AddTag(logtags.AddTag(baseCtx, "k", nil), "n", redact.Unsafe(55)), "k,n" + quote("55") + ""}, + {logtags.AddTag(logtags.AddTag(baseCtx, "k", nil), "n", 55), "k,n55"}, + {logtags.AddTag(logtags.AddTag(baseCtx, "k", nil), "n", redact.Safe(55)), "k,n55"}, + } + + for _, tc := range testData { + var buf strings.Builder + redactTags(tc.ctx, &buf) + assert.Equal(t, tc.expected, buf.String()) + } +} + +func TestRedactedDecodeFile(t *testing.T) { + testData := []struct { + redactableLogs bool + redactMode EditSensitiveData + expRedactable bool + expMessage string + }{ + {false, WithMarkedSensitiveData, true, "‹marker: this is safe, stray marks ??, this is not safe›"}, + {false, WithFlattenedSensitiveData, false, "marker: this is safe, stray marks ‹›, this is not safe"}, + {false, WithoutSensitiveData, true, "‹×›"}, + {true, WithMarkedSensitiveData, true, "marker: this is safe, stray marks ‹›, ‹this is not safe›"}, + {true, WithFlattenedSensitiveData, false, "marker: this is safe, stray marks , this is not safe"}, + {true, WithoutSensitiveData, true, "marker: this is safe, stray marks ‹×›, ‹×›"}, + } + + for _, tc := range testData { + // Use a closure to force scope boundaries. + t.Run(fmt.Sprintf("%v/%v", tc.redactableLogs, tc.redactMode), func(t *testing.T) { + // Initialize the logging system for this test. + // The log file go to a different directory in each sub-test. + s := ScopeWithoutShowLogs(t) + defer s.Close(t) + setFlags() + defer TestingSetRedactable(tc.redactableLogs)() + + // Force file re-initialization. + s.Rotate(t) + + // Emit the message of interest for this test. + Infof(context.Background(), "marker: this is safe, stray marks ‹›, %s", "this is not safe") + + // Retrieve the log writer and log location for this test. + info, ok := mainLog.mu.file.(*syncBuffer) + if !ok { + t.Fatalf("buffer wasn't created") + } + // Ensure our log message above made it to the file. + if err := info.Flush(); err != nil { + t.Fatal(err) + } + + // Prepare reading the entries from the file. + infoName := filepath.Base(info.file.Name()) + reader, err := GetLogReader(infoName, true /* restricted */) + if err != nil { + t.Fatal(err) + } + defer reader.Close() + decoder := NewEntryDecoder(reader, tc.redactMode) + + // Now verify we have what we want in the file. + foundMessage := false + var entry Entry + for { + if err := decoder.Decode(&entry); err != nil { + if err == io.EOF { + break + } + t.Fatal(err) + } + if strings.HasSuffix(entry.File, "redact_test.go") { + assert.Equal(t, tc.expRedactable, entry.Redactable) + assert.Equal(t, tc.expMessage, entry.Message) + foundMessage = true + } + } + if !foundMessage { + t.Error("expected marked message in log, found none") + } + }) + } +} diff --git a/pkg/util/log/secondary_log.go b/pkg/util/log/secondary_log.go index 337b6b86d00a..94a87e76989d 100644 --- a/pkg/util/log/secondary_log.go +++ b/pkg/util/log/secondary_log.go @@ -12,11 +12,7 @@ package log import ( "context" - "fmt" - "strings" - "sync/atomic" - "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -25,8 +21,6 @@ import ( // facility. type SecondaryLogger struct { logger loggerT - msgCount uint64 - enableMsgCount bool forceSyncWrites bool } @@ -66,16 +60,20 @@ func NewSecondaryLogger( } l := &SecondaryLogger{ logger: loggerT{ - logDir: DirName{name: dir}, - prefix: program + "-" + fileNamePrefix, - fileThreshold: Severity_INFO, - stderrThreshold: mainLog.stderrThreshold.get(), - noRedirectInternalStderrWrites: true, - gcNotify: make(chan struct{}, 1), + logDir: DirName{name: dir}, + prefix: program + "-" + fileNamePrefix, + fileThreshold: Severity_INFO, + stderrThreshold: mainLog.stderrThreshold.get(), + logCounter: EntryCounter{EnableMsgCount: enableMsgCount}, + gcNotify: make(chan struct{}, 1), + // Only one logger can have redirectInternalStderrWrites set to + // true; this is going to be either mainLog or stderrLog + // depending on configuration. + redirectInternalStderrWrites: false, }, forceSyncWrites: forceSyncWrites, - enableMsgCount: enableMsgCount, } + l.logger.redactableLogs.Set(mainLog.redactableLogs.Get()) l.logger.mu.syncWrites = forceSyncWrites || mainLog.mu.syncWrites // Ensure the registry knows about this logger. @@ -110,22 +108,9 @@ func (l *SecondaryLogger) Close() { func (l *SecondaryLogger) output( ctx context.Context, depth int, sev Severity, format string, args ...interface{}, ) { - file, line, _ := caller.Lookup(depth + 1) - var buf strings.Builder - formatTags(ctx, &buf) - - if l.enableMsgCount { - // Add a counter. This is important for the SQL audit logs. - counter := atomic.AddUint64(&l.msgCount, 1) - fmt.Fprintf(&buf, "%d ", counter) - } - - if format == "" { - fmt.Fprint(&buf, args...) - } else { - fmt.Fprintf(&buf, format, args...) - } - l.logger.outputLogEntry(Severity_INFO, file, line, buf.String()) + entry := MakeEntry( + ctx, sev, &l.logger.logCounter, depth+1, l.logger.redactableLogs.Get(), format, args...) + l.logger.outputLogEntry(entry) } // Logf logs an event on a secondary logger. diff --git a/pkg/util/log/secondary_log_test.go b/pkg/util/log/secondary_log_test.go index df8ce8d6a1a3..55bdcd364a4f 100644 --- a/pkg/util/log/secondary_log_test.go +++ b/pkg/util/log/secondary_log_test.go @@ -98,8 +98,8 @@ func TestRedirectStderrWithSecondaryLoggersActive(t *testing.T) { const stderrText = "hello stderr" fmt.Fprint(os.Stderr, stderrText) - // Check the main log file: we want our stderr text there. - contents, err := ioutil.ReadFile(mainLog.mu.file.(*syncBuffer).file.Name()) + // Check the stderr log file: we want our stderr text there. + contents, err := ioutil.ReadFile(stderrLog.mu.file.(*syncBuffer).file.Name()) if err != nil { t.Fatal(err) } @@ -133,14 +133,20 @@ func TestListLogFilesIncludeSecondaryLogs(t *testing.T) { l.Logf(ctx, "story time") Flush() - expectedName := filepath.Base(l.logger.mu.file.(*syncBuffer).file.Name()) - results, err := ListLogFiles() if err != nil { t.Fatalf("error in ListLogFiles: %v", err) } - if len(results) != 1 || results[0].Name != expectedName { + expectedName := filepath.Base(l.logger.mu.file.(*syncBuffer).file.Name()) + foundExpected := false + for i := range results { + if results[i].Name == expectedName { + foundExpected = true + break + } + } + if !foundExpected { t.Fatalf("unexpected results; expected file %q, got: %+v", expectedName, results) } } diff --git a/pkg/util/log/structured.go b/pkg/util/log/structured.go index 143a56330b7f..baba55f5ddbc 100644 --- a/pkg/util/log/structured.go +++ b/pkg/util/log/structured.go @@ -12,49 +12,30 @@ package log import ( "context" - "fmt" "strings" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/errors" - "github.com/cockroachdb/logtags" ) -// formatTags appends the tags to a strings.Builder. If there are no tags, -// returns false. -func formatTags(ctx context.Context, buf *strings.Builder) bool { - tags := logtags.FromContext(ctx) - if tags == nil { - return false - } - buf.WriteByte('[') - tags.FormatToString(buf) - buf.WriteString("] ") - return true -} - -// MakeMessage creates a structured log entry. -func MakeMessage(ctx context.Context, format string, args []interface{}) string { +// FormatWithContextTags formats the string and prepends the context +// tags. +// +// Redaction markers are *not* inserted. The resulting +// string is generally unsafe for reporting. +func FormatWithContextTags(ctx context.Context, format string, args ...interface{}) string { var buf strings.Builder - formatTags(ctx, &buf) - if len(args) == 0 { - buf.WriteString(format) - } else if len(format) == 0 { - fmt.Fprint(&buf, args...) - } else { - fmt.Fprintf(&buf, format, args...) - } + formatTags(ctx, true /* brackets */, &buf) + renderArgs(false, &buf, format, args...) return buf.String() } // addStructured creates a structured log entry to be written to the // specified facility of the logger. -func addStructured(ctx context.Context, s Severity, depth int, format string, args []interface{}) { - file, line, _ := caller.Lookup(depth + 1) - msg := MakeMessage(ctx, format, args) - - if s == Severity_FATAL { +func addStructured( + ctx context.Context, sev Severity, depth int, format string, args []interface{}, +) { + if sev == Severity_FATAL { // We load the ReportingSettings from the a global singleton in this // call path. See the singleton's comment for a rationale. if sv := settings.TODO(); sv != nil { @@ -62,8 +43,11 @@ func addStructured(ctx context.Context, s Severity, depth int, format string, ar sendCrashReport(ctx, sv, err, ReportTypePanic) } } - // MakeMessage already added the tags when forming msg, we don't want - // eventInternal to prepend them again. - eventInternal(ctx, (s >= Severity_ERROR), false /*withTags*/, "%s:%d %s", file, line, msg) - mainLog.outputLogEntry(s, file, line, msg) + + entry := MakeEntry( + ctx, sev, &mainLog.logCounter, depth+1, mainLog.redactableLogs.Get(), format, args...) + if sp, el, ok := getSpanOrEventLog(ctx); ok { + eventInternal(sp, el, (sev >= Severity_ERROR), entry) + } + mainLog.outputLogEntry(entry) } diff --git a/pkg/util/log/sync_buffer.go b/pkg/util/log/sync_buffer.go index 16aec151bea5..4d2ee027bcf1 100644 --- a/pkg/util/log/sync_buffer.go +++ b/pkg/util/log/sync_buffer.go @@ -12,15 +12,13 @@ package log import ( "bufio" - "fmt" + "context" "os" "sync/atomic" "time" "github.com/cockroachdb/cockroach/pkg/build" - "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/petermattis/goid" ) // syncBuffer joins a bufio.Writer to its underlying file, providing access to the @@ -97,7 +95,7 @@ func (sb *syncBuffer) rotateFile(now time.Time) error { // // This captures e.g. all writes performed by internal // assertions in the Go runtime. - if sb.logger.redirectInternalStderrWrites() { + if sb.logger.redirectInternalStderrWrites { // NB: any concurrent output to stderr may straddle the old and new // files. This doesn't apply to log messages as we won't reach this code // unless we're not logging to stderr. @@ -113,34 +111,27 @@ func (sb *syncBuffer) rotateFile(now time.Time) error { sb.Writer = bufio.NewWriterSize(sb.file, bufferSize) - messages := make([]string, 0, 6) + messages := make([]Entry, 0, 6) messages = append(messages, - fmt.Sprintf("[config] file created at: %s\n", now.Format("2006/01/02 15:04:05")), - fmt.Sprintf("[config] running on machine: %s\n", host), - fmt.Sprintf("[config] binary: %s\n", build.GetInfo().Short()), - fmt.Sprintf("[config] arguments: %s\n", os.Args), + sb.logger.makeStartLine("file created at: %s", Safe(now.Format("2006/01/02 15:04:05"))), + sb.logger.makeStartLine("running on machine: %s", host), + sb.logger.makeStartLine("binary: %s", Safe(build.GetInfo().Short())), + sb.logger.makeStartLine("arguments: %s", os.Args), ) logging.mu.Lock() if logging.mu.clusterID != "" { - messages = append(messages, fmt.Sprintf("[config] clusterID: %s\n", logging.mu.clusterID)) + messages = append(messages, sb.logger.makeStartLine("clusterID: %s", logging.mu.clusterID)) } logging.mu.Unlock() // Including a non-ascii character in the first 1024 bytes of the log helps // viewers that attempt to guess the character encoding. - messages = append(messages, fmt.Sprintf("line format: [IWEF]yymmdd hh:mm:ss.uuuuuu goid file:line msg utf8=\u2713\n")) - - f, l, _ := caller.Lookup(1) - for _, msg := range messages { - buf := logging.formatLogEntry(Entry{ - Severity: Severity_INFO, - Time: now.UnixNano(), - Goroutine: goid.Get(), - File: f, - Line: int64(l), - Message: msg, - }, nil, nil) + messages = append(messages, + sb.logger.makeStartLine("line format: [IWEF]yymmdd hh:mm:ss.uuuuuu goid file:line msg utf8=\u2713")) + + for _, entry := range messages { + buf := logging.formatLogEntry(entry, nil, nil) var n int n, err = sb.file.Write(buf.Bytes()) putBuffer(buf) @@ -156,3 +147,16 @@ func (sb *syncBuffer) rotateFile(now time.Time) error { } return nil } + +func (l *loggerT) makeStartLine(format string, args ...interface{}) Entry { + entry := MakeEntry( + context.Background(), + Severity_INFO, + nil, /* logCounter */ + 2, /* depth */ + l.redactableLogs.Get(), + format, + args...) + entry.Tags = "config" + return entry +} diff --git a/pkg/util/log/test_log_scope.go b/pkg/util/log/test_log_scope.go index 395cf1e1b999..696e75cedab4 100644 --- a/pkg/util/log/test_log_scope.go +++ b/pkg/util/log/test_log_scope.go @@ -81,23 +81,59 @@ func ScopeWithoutShowLogs(t tShim) *TestLogScope { // enableLogFileOutput turns on logging using the specified directory. // For unittesting only. func enableLogFileOutput(dir string, stderrSeverity Severity) (func(), error) { - mainLog.mu.Lock() - defer mainLog.mu.Unlock() - oldStderrThreshold := mainLog.stderrThreshold.get() - oldNoStderrRedirect := stderrLog.noRedirectInternalStderrWrites + oldStderrThreshold, err := func() (Severity, error) { + mainLog.mu.Lock() + defer mainLog.mu.Unlock() + + return mainLog.stderrThreshold.get(), mainLog.logDir.Set(dir) + }() + if err != nil { + return nil, err + } + var cancelStderr func() undo := func() { + if cancelStderr != nil { + cancelStderr() + _ = hijackStderr(OrigStderr) + } + mainLog.mu.Lock() defer mainLog.mu.Unlock() mainLog.stderrThreshold.set(oldStderrThreshold) - stderrLog.noRedirectInternalStderrWrites = oldNoStderrRedirect } + mainLog.stderrThreshold.set(stderrSeverity) - stderrLog.noRedirectInternalStderrWrites = true - // TODO(knz): if/when stderrLog becomes different from mainLog, - // ensure that the stderrLog file output gets re-opened - // and os.Stderr gets redirected at this point. - return undo, mainLog.logDir.Set(dir) + cancelStderr, err = SetupRedactionAndStderrRedirects() + return undo, err +} + +// Rotate closes the current log files so that the next log call will +// reopen them with current settings. This is useful when e.g. a test +// changes the logging configuration after opening a test log scope. +func (l *TestLogScope) Rotate(t tShim) { + // Ensure remaining logs are written. + Flush() + + func() { + mainLog.mu.Lock() + defer mainLog.mu.Unlock() + if err := mainLog.closeFileLocked(); err != nil { + t.Fatal(err) + } + }() + + secondaryLogRegistry.mu.Lock() + defer secondaryLogRegistry.mu.Unlock() + for _, l := range secondaryLogRegistry.mu.loggers { + func() { + l.logger.mu.Lock() + defer l.logger.mu.Unlock() + if err := l.logger.closeFileLocked(); err != nil { + t.Fatal(err) + } + }() + } } // Close cleans up a TestLogScope. The directory and its contents are diff --git a/pkg/util/log/testshout/shout_test.go b/pkg/util/log/testshout/shout_test.go index de59501e0452..c691c782a048 100644 --- a/pkg/util/log/testshout/shout_test.go +++ b/pkg/util/log/testshout/shout_test.go @@ -34,9 +34,6 @@ func Example_shout_before_log() { if err := flag.Set(logflags.LogToStderrName, "WARNING"); err != nil { panic(err) } - if err := flag.Set(logflags.NoRedirectStderrName, "false"); err != nil { - panic(err) - } log.Shout(context.Background(), log.Severity_INFO, "hello world") diff --git a/pkg/util/log/trace.go b/pkg/util/log/trace.go index 91d3f5814be9..71924833ffc0 100644 --- a/pkg/util/log/trace.go +++ b/pkg/util/log/trace.go @@ -12,11 +12,13 @@ package log import ( "context" - "fmt" + "strconv" "strings" + "github.com/cockroachdb/cockroach/pkg/util/redact" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/logtags" opentracing "github.com/opentracing/opentracing-go" otlog "github.com/opentracing/opentracing-go/log" "golang.org/x/net/trace" @@ -102,63 +104,132 @@ func getSpanOrEventLog(ctx context.Context) (opentracing.Span, *ctxEventLog, boo // eventInternal is the common code for logging an event. If no args are given, // the format is treated as a pre-formatted string. -func eventInternal(ctx context.Context, isErr, withTags bool, format string, args ...interface{}) { - if sp, el, ok := getSpanOrEventLog(ctx); ok { +// +// Note that when called from a logging function, this is taking the log +// message as input after introduction of redaction markers. This +// means the message may or may not contain markers already depending +// of the configuration of --redactable-logs. +// +// TODO(knz): change the interface to take a log.Entry +// as input instead. +func eventInternal(sp opentracing.Span, el *ctxEventLog, isErr bool, entry Entry) { + var msg string + if len(entry.Tags) == 0 && len(entry.File) == 0 && !entry.Redactable { + // Shortcut. + msg = entry.Message + } else { var buf strings.Builder - if withTags { - withTags = formatTags(ctx, &buf) + if len(entry.File) != 0 { + buf.WriteString(entry.File) + buf.WriteByte(':') + // TODO(knz): The "canonical" way to represent a file/line prefix + // is: :: msg + // with a colon between the line number and the message. + // However, some location filter deep inside SQL doesn't + // understand a colon after the line number. + buf.WriteString(strconv.FormatInt(entry.Line, 10)) + buf.WriteByte(' ') } + if len(entry.Tags) > 0 { + buf.WriteByte('[') + buf.WriteString(entry.Tags) + buf.WriteString("] ") + } + buf.WriteString(entry.Message) + msg = buf.String() - var msg string - if !withTags && len(args) == 0 { - // Fast path for pre-formatted messages. - msg = format - } else { - if len(args) == 0 { - buf.WriteString(format) - } else { - fmt.Fprintf(&buf, format, args...) - } - msg = buf.String() + if entry.Redactable { + // This is true when eventInternal is called from addStructured(), + // ie. a regular log call. In this case, the tags and message may contain + // redaction markers. We remove them here. + msg = redact.RedactableString(msg).StripMarkers() } + } - if sp != nil { - // TODO(radu): pass tags directly to sp.LogKV when LightStep supports - // that. - sp.LogFields(otlog.String(tracing.LogMessageField, msg)) - // if isErr { - // // TODO(radu): figure out a way to signal that this is an error. We - // // could use a different "error" key (provided it shows up in - // // LightStep). Things like NetTraceIntegrator would need to be modified - // // to understand the difference. We could also set a special Tag or - // // Baggage on the span. See #8827 for more discussion. - // } - } else { - el.Lock() - if el.eventLog != nil { - if isErr { - el.eventLog.Errorf("%s", msg) - } else { - el.eventLog.Printf("%s", msg) - } + if sp != nil { + // TODO(radu): pass tags directly to sp.LogKV when LightStep supports + // that. + sp.LogFields(otlog.String(tracing.LogMessageField, msg)) + // if isErr { + // // TODO(radu): figure out a way to signal that this is an error. We + // // could use a different "error" key (provided it shows up in + // // LightStep). Things like NetTraceIntegrator would need to be modified + // // to understand the difference. We could also set a special Tag or + // // Baggage on the span. See #8827 for more discussion. + // } + } else { + el.Lock() + if el.eventLog != nil { + if isErr { + el.eventLog.Errorf("%s", msg) + } else { + el.eventLog.Printf("%s", msg) } - el.Unlock() } + el.Unlock() } } +// formatTags appends the tags to a strings.Builder. If there are no tags, +// returns false. +func formatTags(ctx context.Context, brackets bool, buf *strings.Builder) bool { + tags := logtags.FromContext(ctx) + if tags == nil { + return false + } + if brackets { + buf.WriteByte('[') + } + tags.FormatToString(buf) + if brackets { + buf.WriteString("] ") + } + return true +} + // Event looks for an opentracing.Trace in the context and logs the given // message to it. If no Trace is found, it looks for an EventLog in the context // and logs the message to it. If neither is found, does nothing. func Event(ctx context.Context, msg string) { - eventInternal(ctx, false /* isErr */, true /* withTags */, msg) + sp, el, ok := getSpanOrEventLog(ctx) + if !ok { + // Nothing to log. Skip the work. + return + } + + // Format the tracing event and add it to the trace. + entry := MakeEntry(ctx, + Severity_INFO, /* unused for trace events */ + nil, /* logCounter, unused for trace events */ + 1, /* depth */ + // redactable is false because we want to flatten the data in traces + // -- we don't have infrastructure yet for trace redaction. + false, /* redactable */ + "") + entry.Message = msg + eventInternal(sp, el, false /* isErr */, entry) } // Eventf looks for an opentracing.Trace in the context and formats and logs // the given message to it. If no Trace is found, it looks for an EventLog in // the context and logs the message to it. If neither is found, does nothing. func Eventf(ctx context.Context, format string, args ...interface{}) { - eventInternal(ctx, false /* isErr */, true /* withTags */, format, args...) + sp, el, ok := getSpanOrEventLog(ctx) + if !ok { + // Nothing to log. Skip the work. + return + } + + // Format the tracing event and add it to the trace. + entry := MakeEntry(ctx, + Severity_INFO, /* unused for trace events */ + nil, /* logCounter, unused for trace events */ + 1, /* depth */ + // redactable is false because we want to flatten the data in traces + // -- we don't have infrastructure yet for trace redaction. + false, /* redactable */ + format, args...) + eventInternal(sp, el, false /* isErr */, entry) } func vEventf( @@ -172,7 +243,20 @@ func vEventf( } logDepth(ctx, 1+depth, sev, format, args) } else { - eventInternal(ctx, isErr, true /* withTags */, format, args...) + sp, el, ok := getSpanOrEventLog(ctx) + if !ok { + // Nothing to log. Skip the work. + return + } + entry := MakeEntry(ctx, + Severity_INFO, /* unused for trace events */ + nil, /* logCounter, unused for trace events */ + depth+1, + // redactable is false because we want to flatten the data in traces + // -- we don't have infrastructure yet for trace redaction. + false, /* redactable */ + format, args...) + eventInternal(sp, el, isErr, entry) } } diff --git a/pkg/util/log/trace_test.go b/pkg/util/log/trace_test.go index d3b65d826639..df42dbdfef05 100644 --- a/pkg/util/log/trace_test.go +++ b/pkg/util/log/trace_test.go @@ -207,8 +207,13 @@ func TestEventLogAndTrace(t *testing.T) { t.Fatal(err) } - elExpected := "[test1 test2 testerr(err) test6 finish]" - if evStr := fmt.Sprint(el.ev); evStr != elExpected { + elExpected := regexp.MustCompile(`^\[` + + `util/log/trace_test\.go:\d+ test1 ` + + `util/log/trace_test\.go:\d+ test2 ` + + `util/log/trace_test\.go:\d+ testerr\(err\) ` + + `util/log/trace_test\.go:\d+ test6 ` + + `finish\]$`) + if evStr := fmt.Sprint(el.ev); !elExpected.MatchString(evStr) { t.Errorf("expected events '%s', got '%s'", elExpected, evStr) } } diff --git a/pkg/util/redact/api.go b/pkg/util/redact/api.go new file mode 100644 index 000000000000..62052e9e4a26 --- /dev/null +++ b/pkg/util/redact/api.go @@ -0,0 +1,121 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import "fmt" + +// SafeFormatter is implemented by object types that want to separate +// safe and non-safe information. +type SafeFormatter interface { + // SafeFormat is like the Format method of fmt.Formatter, except + // that it operates using a SafePrinter instead of a fmt.State for + // output. + // + // The verb argument is the control character that defines + // the formatting mode in the surrounding Printf call. + // For example, if this method is called to format %03d, + // the verb is 'd'. + SafeFormat(s SafePrinter, verb rune) +} + +// SafeValue is a marker interface to be implemented by types that +// alias base Go types and whose natural representation via Printf is +// always safe for reporting. +// +// This is recognized by the SafePrinter interface as an alternative +// to SafeFormatter. +// +// It is provided to decorate "leaf" Go types, such as aliases to int. +// +// Typically, a linter enforces that a type can only implement this +// interface if it aliases a base go type. More complex types should +// implement SafeFormatter instead. +// +// It is advised to build an automatic process during builds to +// collect all the types that implement this interface, as well as all +// uses of this type, and produce a report. Changes to this report +// should receive maximal amount of scrutiny during code reviews. +type SafeValue interface { + SafeValue() +} + +// SafeMessager is an alternative to SafeFormatter used in previous +// versions of CockroachDB. +// NB: this interface is obsolete. Use SafeFormatter instead. +// TODO(knz): Remove this. +type SafeMessager interface { + SafeMessage() string +} + +// SafePrinter is a stateful helper that abstracts an output stream in +// the context of printf-like formatting, but with the ability to +// separate safe and unsafe bits of data. +// +// This package provides one implementation of this using marker +// delimiters for unsafe data, see markers.go. We would like to aim +// for alternate implementations to generate more structured formats. +type SafePrinter interface { + // SafePrinter inherits fmt.State to access format flags, however + // calls to fmt.State's underlying Write() as unsafe. + fmt.State + + // SafePrinter provides the SafeWriter interface. + SafeWriter +} + +// SafeWriter provides helper functions for use in implementations of +// SafeFormatter, to format mixes of safe and unsafe strings. +type SafeWriter interface { + // SafeString emits a safe string. + SafeString(SafeString) + + // SafeRune emits a safe rune. + SafeRune(SafeRune) + + // Print emits its arguments separated by spaces. + // For each argument it dynamically checks for the SafeFormatter or + // SafeValue interface and either use that, or mark the argument + // payload as unsafe. + Print(args ...interface{}) + + // For printf, a linter checks that the format string is + // a constant literal, so the implementation can assume it's always + // safe. + Printf(format string, arg ...interface{}) + + // UnsafeString writes an unsafe string. + UnsafeString(string) + + // UnsafeByte writes an unsafe byte. + UnsafeByte(byte) + + // UnsafeBytes writes an unsafe byte slice. + UnsafeBytes([]byte) + + // UnsafeRune writes an unsafe rune. + UnsafeRune(rune) +} + +// SafeString aliases string. This is not meant to be used directly; +// the type definition ensures that SafePrinter's SafeString method +// can only take constant string literals as arguments. Typically, a +// Go linter would ensure that ConstantString is never used to cast a +// value. +type SafeString string + +// SafeValue makes SafeString a SafeValue. +func (SafeString) SafeValue() {} + +// SafeRune aliases rune. See the explanation for SafeString. +type SafeRune rune + +// SafeValue makes SafeRune a SafeValue. +func (SafeRune) SafeValue() {} diff --git a/pkg/util/redact/doc.go b/pkg/util/redact/doc.go new file mode 100644 index 000000000000..d5367924f639 --- /dev/null +++ b/pkg/util/redact/doc.go @@ -0,0 +1,23 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Package redact provides facilities for separating “safe” and +// “unsafe” pieces of data when logging and constructing error object. +// +// An item is said to be “safe” if it is proven to not contain +// PII or otherwise confidential information that should not escape +// the boundaries of the current system, for example via telemetry +// or crash reporting. Conversely, data is considered “unsafe” +// until/unless it is known to be “safe”. +// +// TODO(knz): Move this package into a separate top-level repository +// for use from cockroachdb/errors and other projects in the Go +// community. +package redact diff --git a/pkg/util/redact/make_format.go b/pkg/util/redact/make_format.go new file mode 100644 index 000000000000..9e8e6e9d944a --- /dev/null +++ b/pkg/util/redact/make_format.go @@ -0,0 +1,81 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import ( + "fmt" + "io" + "strconv" + "strings" +) + +func reproducePrintf(w io.Writer, s fmt.State, verb rune, arg interface{}) { + justV, revFmt := MakeFormat(s, verb) + if justV { + // Common case, avoids generating then parsing the format again. + fmt.Fprint(w, arg) + } else { + fmt.Fprintf(w, revFmt, arg) + } +} + +// MakeFormat reproduces the format currently active in fmt.State and +// verb. This is provided because Go's standard fmt.State does not +// make the original format string available to us. +// +// If the return value justV is true, then the current state +// was found to be %v exactly; in that case the caller +// can avoid a full-blown Printf call and use just Print instead +// to take a shortcut. +func MakeFormat(s fmt.State, verb rune) (justV bool, format string) { + plus, minus, hash, sp, z := s.Flag('+'), s.Flag('-'), s.Flag('#'), s.Flag(' '), s.Flag('0') + w, wp := s.Width() + p, pp := s.Precision() + + if !plus && !minus && !hash && !sp && !z && !wp && !pp { + switch verb { + case 'v': + return true, "%v" + case 's': + return false, "%s" + case 'd': + return false, "%d" + } + // Other cases handled in the slow path below. + } + + var f strings.Builder + f.WriteByte('%') + if plus { + f.WriteByte('+') + } + if minus { + f.WriteByte('-') + } + if hash { + f.WriteByte('#') + } + if sp { + f.WriteByte(' ') + } + if z { + f.WriteByte('0') + } + if wp { + f.WriteString(strconv.Itoa(w)) + } + if pp { + f.WriteByte('.') + f.WriteString(strconv.Itoa(p)) + } + f.WriteRune(verb) + return false, f.String() +} diff --git a/pkg/util/redact/make_format_test.go b/pkg/util/redact/make_format_test.go new file mode 100644 index 000000000000..ee716739893d --- /dev/null +++ b/pkg/util/redact/make_format_test.go @@ -0,0 +1,53 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import ( + "fmt" + "testing" +) + +// TestMakeFormat verifies that the makeFormat() helper is able to +// reproduce the format given as input to fmt function. +func TestMakeFormat(t *testing.T) { + testData := []string{ + "%c", "%v", "%q", + "%3f", "%.3f", "%2.3f", + "%# v", "%012s", + "%+v", "%-12s", + } + + for _, test := range testData { + justV, revFmt := getFormat(test) + if (test == "%v") != justV { + t.Errorf("%q: expected justV %v, got %v", test, test == "%v", justV) + } + if revFmt != test { + t.Errorf("%q: got %q instead", test, revFmt) + } + } +} + +type formatTester struct { + fn func(fmt.State, rune) +} + +func (f formatTester) Format(s fmt.State, verb rune) { + f.fn(s, verb) +} + +func getFormat(testFmt string) (justV bool, revFmt string) { + f := formatTester{func(s fmt.State, verb rune) { + justV, revFmt = MakeFormat(s, verb) + }} + _ = fmt.Sprintf(testFmt, f) + return +} diff --git a/pkg/util/redact/markers.go b/pkg/util/redact/markers.go new file mode 100644 index 000000000000..66c55ffd811d --- /dev/null +++ b/pkg/util/redact/markers.go @@ -0,0 +1,90 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +// RedactableString is a string that contains a mix of safe and unsafe +// bits of data, but where it is known that unsafe bits are enclosed +// by redaction markers ‹ and ›, and occurrences of the markers +// inside the original data items have been escaped. +// +// Instances of RedactableString should not be constructed directly; +// instead use the facilities from print.go (Sprint, Sprintf) +// or the methods below. +type RedactableString string + +// StripMarkers removes the redaction markers from the +// RedactableString. This returns an unsafe string where all safe and +// unsafe bits are mixed together. +func (s RedactableString) StripMarkers() string { + return reStripMarkers.ReplaceAllString(string(s), "") +} + +// Redact replaces all occurrences of unsafe substrings by the +// “Redacted” marker, ‹×›. The result string is still safe. +func (s RedactableString) Redact() RedactableString { + return RedactableString(reStripSensitive.ReplaceAllString(string(s), redactedS)) +} + +// ToBytes converts the string to a byte slice. +func (s RedactableString) ToBytes() RedactableBytes { + return RedactableBytes([]byte(string(s))) +} + +// RedactableBytes is like RedactableString but is a byte slice. +// +// Instances of RedactableBytes should not be constructed directly; +// instead use the facilities from print.go (Sprint, Sprintf) +// or the methods below. +type RedactableBytes []byte + +// StripMarkers removes the redaction markers from the +// RedactableBytes. This returns an unsafe string where all safe and +// unsafe bits are mixed together. +func (s RedactableBytes) StripMarkers() []byte { + return reStripMarkers.ReplaceAll([]byte(s), nil) +} + +// Redact replaces all occurrences of unsafe substrings by the +// “Redacted” marker, ‹×›. +func (s RedactableBytes) Redact() RedactableBytes { + return RedactableBytes(reStripSensitive.ReplaceAll(s, redactedBytes)) +} + +// ToString converts the byte slice to a string. +func (s RedactableBytes) ToString() RedactableString { + return RedactableString(string([]byte(s))) +} + +// EscapeBytes escapes markers inside the given byte slice and encloses +// the entire byte slice between redaction markers. +func EscapeBytes(s []byte) RedactableBytes { + s = reStripMarkers.ReplaceAll(s, escapeBytes) + enclosed := make([]byte, 0, len(s)+len(startRedactableBytes)+len(endRedactableBytes)) + enclosed = append(enclosed, startRedactableBytes...) + enclosed = append(enclosed, s...) + enclosed = append(enclosed, endRedactableBytes...) + return RedactableBytes(enclosed) +} + +// StartMarker returns the start delimiter for an unsafe string. +func StartMarker() []byte { return startRedactableBytes } + +// EndMarker returns the end delimiter for an unsafe string. +func EndMarker() []byte { return endRedactableBytes } + +// RedactedMarker returns the special string used by Redact. +func RedactedMarker() []byte { return redactedBytes } + +// EscapeMarkers escapes the special delimiters from the provided +// byte slice. +func EscapeMarkers(s []byte) []byte { + return reStripMarkers.ReplaceAll(s, escapeBytes) +} diff --git a/pkg/util/redact/markers_internal_constants.go b/pkg/util/redact/markers_internal_constants.go new file mode 100644 index 000000000000..730e3bd7345d --- /dev/null +++ b/pkg/util/redact/markers_internal_constants.go @@ -0,0 +1,36 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import "regexp" + +const startRedactable = '‹' +const startRedactableS = string(startRedactable) + +var startRedactableBytes = []byte(startRedactableS) + +const endRedactable = '›' +const endRedactableS = string(endRedactable) + +var endRedactableBytes = []byte(endRedactableS) + +const escapeMark = '?' +const escapeMarkS = string(escapeMark) + +var escapeBytes = []byte(escapeMarkS) + +const redactedS = startRedactableS + "×" + endRedactableS + +var redactedBytes = []byte(redactedS) + +var reStripSensitive = regexp.MustCompile(startRedactableS + "[^" + startRedactableS + endRedactableS + "]*" + endRedactableS) + +var reStripMarkers = regexp.MustCompile("[" + startRedactableS + endRedactableS + "]") diff --git a/pkg/util/redact/markers_internal_escape.go b/pkg/util/redact/markers_internal_escape.go new file mode 100644 index 000000000000..0b95b20664a9 --- /dev/null +++ b/pkg/util/redact/markers_internal_escape.go @@ -0,0 +1,126 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import ( + "bytes" + "fmt" + "io" +) + +// escapeState abstracts on top of fmt.State and ensures that call +// calls to Write() enclose the writtne bytes between unsafe markers. +type escapeState struct { + fmt.State + w escapeWriter +} + +var _ fmt.State = (*escapeState)(nil) + +func makeEscapeState(s fmt.State, buf *bytes.Buffer) escapeState { + e := escapeState{State: s} + e.w = escapeWriter{w: buf, enclose: true} + return e +} + +// Write is part of the fmt.State interface and implements the io.Writer interface. +func (p *escapeState) Write(b []byte) (int, error) { + return p.w.Write(b) +} + +// escapeWriter abstracts on top of io.Writer and ensures that all +// calls to Write() escape markers. +// Also, final spaces and newlines are stripped if strip is set. +// Also, the overall result is enclosed inside redaction markers +// if enclose is true. +type escapeWriter struct { + w io.Writer + enclose bool + strip bool +} + +var _ io.Writer = (*escapeWriter)(nil) + +// Write implements the io.Writer interface. +func (p *escapeWriter) Write(b []byte) (int, error) { + st := escapeResult{0, nil} + + if p.strip { + // Trim final newlines/spaces, for convenience. + end := len(b) + for i := end - 1; i >= 0; i-- { + if b[i] == '\n' || b[i] == ' ' { + end = i + } else { + break + } + } + b = b[:end] + } + + // Here we could choose to omit the output + // entirely if there was nothing but empty space: + // if len(b) == 0 { return 0, nil } + + start := startRedactableBytes + ls := len(startRedactableS) + end := endRedactableBytes + le := len(endRedactableS) + escape := escapeBytes + + if p.enclose { + st = p.doWrite(start, st, false) + } + + // Now write the string. + k := 0 + for i := 0; i < len(b); i++ { + // Ensure that occurrences of the delimiter inside the string get + // escaped. + if i+ls <= len(b) && bytes.Equal(b[i:i+ls], start) { + st = p.doWrite(b[k:i], st, true) + st = p.doWrite(escape, st, false) + st.l += ls + k = i + ls + i += ls - 1 + } else if i+le <= len(b) && bytes.Equal(b[i:i+le], end) { + st = p.doWrite(b[k:i], st, true) + st = p.doWrite(escape, st, false) + st.l += le + k = i + le + i += le - 1 + } + } + st = p.doWrite(b[k:], st, true) + if p.enclose { + st = p.doWrite(end, st, false) + } + return st.l, st.err +} + +type escapeResult struct { + l int + err error +} + +func (p *escapeWriter) doWrite(b []byte, st escapeResult, count bool) escapeResult { + if st.err != nil { + // An error was encountered previously. + // No-op. + return st + } + sz, err := p.w.Write(b) + if count { + st.l += sz + } + st.err = err + return st +} diff --git a/pkg/util/redact/markers_internal_print.go b/pkg/util/redact/markers_internal_print.go new file mode 100644 index 000000000000..56ba054a61e6 --- /dev/null +++ b/pkg/util/redact/markers_internal_print.go @@ -0,0 +1,133 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import "fmt" + +// annotateArgs wraps the arguments to one of the print functions with +// an indirect formatter which either encloses the result of the +// display between redaction markers, or not. +func annotateArgs(args []interface{}) { + for i, arg := range args { + switch v := arg.(type) { + case RedactableString: + // Already formatted as redactable. Include as-is. + // We need an intermediate struct because we don't + // want the fmt machinery to re-format the string + // object by adding quotes, expanding the byte slice, etc. + args[i] = &passthrough{arg: []byte(v)} + case RedactableBytes: + args[i] = &passthrough{arg: v} + + case SafeFormatter: + // calls to Format() by fmt.Print will be redirected to + // v.SafeFormat(). This delegates the task of adding markers to + // the object itself. + args[i] = &redactFormatRedirect{v} + + case SafeValue: + // calls to Format() by fmt.Print will be redirected to a + // display of v without redaction markers. + // + // Note that we can't let the value be displayed as-is because + // we must prevent any marker inside the value from leaking into + // the result. (We want to avoid mismatched markers.) + args[i] = &escapeArg{arg: arg, enclose: false} + + case SafeMessager: + // Obsolete interface. + // TODO(knz): Remove this. + args[i] = &escapeArg{arg: v.SafeMessage(), enclose: false} + + default: + // calls to Format() by fmt.Print will be redirected to a + // display of v within redaction markers if the type is + // considered unsafe, without markers otherwise. In any case, + // occurrences of delimiters within are escaped. + args[i] = &escapeArg{arg: v, enclose: !isSafeValue(v)} + } + } +} + +// redactFormatRedirect wraps a SafeFormatter object and uses its +// SafeFormat method as-is to implement fmt.Formatter. +type redactFormatRedirect struct { + arg SafeFormatter +} + +// Format implements fmt.Formatter. +func (r *redactFormatRedirect) Format(s fmt.State, verb rune) { + defer func() { + if p := recover(); p != nil { + e := escapeWriter{w: s} + fmt.Fprintf(&e, "%%!%c(PANIC=SafeFormatter method: %v)", verb, p) + } + }() + p := &printer{} + p.escapeState = makeEscapeState(s, &p.buf) + r.arg.SafeFormat(p, verb) + _, _ = s.Write(p.buf.Bytes()) +} + +// passthrough passes a pre-formatted string through. +type passthrough struct{ arg []byte } + +// Format implements fmt.Formatter. +func (p *passthrough) Format(s fmt.State, _ rune) { + _, _ = s.Write(p.arg) +} + +// escapeArg wraps an arbitrary value and ensures that any occurrence +// of the redaction markers in its representation are escaped. +// +// The result of printing out the value is enclosed within markers or +// not depending on the value of the enclose bool. +type escapeArg struct { + arg interface{} + enclose bool +} + +func (r *escapeArg) Format(s fmt.State, verb rune) { + switch t := r.arg.(type) { + case fmt.Formatter: + // This is a special case from the default case below, which + // allows a shortcut through the layers of the fmt package. + p := &escapeState{ + State: s, + w: escapeWriter{ + w: s, + enclose: r.enclose, + strip: r.enclose, + }} + defer func() { + if recovered := recover(); recovered != nil { + fmt.Fprintf(p, "%%!%c(PANIC=Format method: %v)", verb, recovered) + } + }() + t.Format(p, verb) + + default: + // TODO(knz): It would be possible to implement struct formatting + // with conditional redaction based on field tag annotations here. + p := &escapeWriter{w: s, enclose: r.enclose, strip: r.enclose} + reproducePrintf(p, s, verb, r.arg) + } +} + +// printerfn is a helper struct for use by Sprintfn. +type printerfn struct { + fn func(SafePrinter) +} + +// SafeFormat implements the SafeFormatter interface. +func (p printerfn) SafeFormat(w SafePrinter, _ rune) { + p.fn(w) +} diff --git a/pkg/util/redact/markers_internal_printer.go b/pkg/util/redact/markers_internal_printer.go new file mode 100644 index 000000000000..e9dfbe30ea3d --- /dev/null +++ b/pkg/util/redact/markers_internal_printer.go @@ -0,0 +1,77 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import ( + "bytes" + "fmt" +) + +// printer implements SafePrinter. +// This is the machinery for the Print() functions offered +// by this package. +type printer struct { + escapeState + buf bytes.Buffer +} + +var _ fmt.State = (*printer)(nil) +var _ SafeWriter = (*printer)(nil) + +// Print is part of the SafeWriter interface. +func (b *printer) Print(args ...interface{}) { + _, _ = Fprint(&b.buf, args...) +} + +// Printf is part of the SafeWriter interface. +func (b *printer) Printf(format string, args ...interface{}) { + _, _ = Fprintf(&b.buf, format, args...) +} + +// SafeString is part of the SafeWriter interface. +func (b *printer) SafeString(s SafeString) { + w := escapeWriter{w: &b.buf, enclose: false} + _, _ = w.Write([]byte(s)) +} + +// SafeRune is part of the SafeWriter interface. +func (b *printer) SafeRune(s SafeRune) { + if s == startRedactable || s == endRedactable { + s = escapeMark + } + _, _ = b.buf.WriteRune(rune(s)) +} + +// UnsafeString is part of the SafeWriter interface. +func (b *printer) UnsafeString(s string) { + w := escapeWriter{w: &b.buf, enclose: true, strip: true} + _, _ = w.Write([]byte(s)) +} + +// UnsafeRune is part of the SafeWriter interface. +func (b *printer) UnsafeRune(s rune) { + _, _ = b.buf.WriteRune(startRedactable) + b.SafeRune(SafeRune(s)) + _, _ = b.buf.WriteRune(endRedactable) +} + +// UnsafeByte is part of the SafeWriter interface. +func (b *printer) UnsafeByte(s byte) { + _, _ = b.buf.WriteRune(startRedactable) + _ = b.buf.WriteByte(s) + _, _ = b.buf.WriteRune(endRedactable) +} + +// UnsafeBytes is part of the SafeWriter interface. +func (b *printer) UnsafeBytes(s []byte) { + w := escapeWriter{w: &b.buf, enclose: true, strip: true} + _, _ = w.Write(s) +} diff --git a/pkg/util/redact/markers_internal_types.go b/pkg/util/redact/markers_internal_types.go new file mode 100644 index 000000000000..ab04ec06636e --- /dev/null +++ b/pkg/util/redact/markers_internal_types.go @@ -0,0 +1,27 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import "reflect" + +// RegisterSafeType registers a data type to always be considered safe +// during the production of redactable strings. +func RegisterSafeType(t reflect.Type) { + safeTypeRegistry[t] = true +} + +// safeTypeRegistry registers Go data types that are to be always +// considered safe, even when they don't implement SafeValue. +var safeTypeRegistry = map[reflect.Type]bool{} + +func isSafeValue(a interface{}) bool { + return safeTypeRegistry[reflect.TypeOf(a)] +} diff --git a/pkg/util/redact/markers_print.go b/pkg/util/redact/markers_print.go new file mode 100644 index 000000000000..6b889241a731 --- /dev/null +++ b/pkg/util/redact/markers_print.go @@ -0,0 +1,73 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import ( + "fmt" + "io" +) + +// Sprint prints out the arguments and encloses unsafe bits +// between redaction markers. +// If either safe and unsafe bits of data contain the markers +// in their representation already, they are escaped first. +// If a RedactableString or RedactableBytes argument is passed, +// it is reproduced as-is without escaping. +func Sprint(args ...interface{}) RedactableString { + annotateArgs(args) + return RedactableString(fmt.Sprint(args...)) +} + +// Sprintf formats the arguments and encloses unsafe bits +// between redaction markers. +// If either safe and unsafe bits of data contain the markers +// in their representation already, they are escaped first. +// The format is always considered safe and the caller +// is responsible to ensure that the markers are not present +// in the format string. +func Sprintf(format string, args ...interface{}) RedactableString { + annotateArgs(args) + return RedactableString(fmt.Sprintf(format, args...)) +} + +// Sprintfn produces a RedactableString using the provided +// SafeFormat-alike function. +func Sprintfn(printer func(w SafePrinter)) RedactableString { + return Sprint(printerfn{printer}) +} + +// StringWithoutMarkers formats the provided SafeFormatter and strips +// the redaction markers from the result. This is provided for +// convenience to facilitate the implementation of String() methods +// alongside SafeFormat() to avoid code duplication. +// +// Note: if this function is ever found to be a performance +// bottleneck, one can consider using an alternate implementation of +// Sprint() which similarly calls the SafeFormat() methods but does +// not introduce markers and instead writes to a string buffer +// directly. +func StringWithoutMarkers(f SafeFormatter) string { + return Sprint(f).StripMarkers() +} + +// Fprint is like Sprint but outputs the redactable +// string to the provided Writer. +func Fprint(w io.Writer, args ...interface{}) (int, error) { + annotateArgs(args) + return fmt.Fprint(w, args...) +} + +// Fprintf is like Sprintf but outputs the redactable string to the +// provided Writer. +func Fprintf(w io.Writer, format string, args ...interface{}) (int, error) { + annotateArgs(args) + return fmt.Fprintf(w, format, args...) +} diff --git a/pkg/util/redact/markers_test.go b/pkg/util/redact/markers_test.go new file mode 100644 index 000000000000..fd26abd0d5ea --- /dev/null +++ b/pkg/util/redact/markers_test.go @@ -0,0 +1,314 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import ( + "bytes" + "fmt" + "reflect" + "strings" + "testing" +) + +type p = SafePrinter + +func TestPrinter(t *testing.T) { + testData := []struct { + fn func(p) + expected string + }{ + {func(w p) { w.SafeString("ab") }, `ab`}, + {func(w p) { w.SafeRune('☃') }, `☃`}, + {func(w p) { w.UnsafeString("rs") }, `‹rs›`}, + {func(w p) { w.UnsafeByte('t') }, `‹t›`}, + {func(w p) { w.UnsafeBytes([]byte("uv")) }, `‹uv›`}, + {func(w p) { w.UnsafeRune('🛑') }, `‹🛑›`}, + {func(w p) { w.Print("fg", safe("hi")) }, `‹fg› hi`}, + {func(w p) { w.Printf("jk %s %s", "lm", safe("no")) }, `jk ‹lm› no`}, + // Direct access to the fmt.State. + {func(w p) { _, _ = w.Write([]byte("pq")) }, `‹pq›`}, + // Safe strings and runes containing the delimiters get escaped. + {func(w p) { w.SafeString("a ‹ b › c") }, `a ? b ? c`}, + {func(w p) { w.SafeRune('‹') }, `?`}, + {func(w p) { w.SafeRune('›') }, `?`}, + {func(w p) { w.Print("a ‹ b › c", safe("d ‹ e › f")) }, + `‹a ? b ? c› d ? e ? f`}, + {func(w p) { w.Printf("f %s %s", "a ‹ b › c", safe("d ‹ e › f")) }, + `f ‹a ? b ? c› d ? e ? f`}, + // Space and newlines at the end of an unsafe string get removed, + // but not at the end of a safe string. + {func(w p) { w.SafeString("ab \n ") }, "ab \n "}, + {func(w p) { w.UnsafeString("cd \n ") }, `‹cd›`}, + {func(w p) { w.Print("ab ", safe("cd ")) }, "‹ab› cd "}, + {func(w p) { w.Printf("ab :%s: :%s: ", "cd ", safe("de ")) }, "ab :‹cd›: :de : "}, + // Spaces as runes get preserved. + {func(w p) { w.SafeRune(' ') }, ` `}, + {func(w p) { w.SafeRune('\n') }, "\n"}, + {func(w p) { w.UnsafeRune(' ') }, `‹ ›`}, + {func(w p) { w.UnsafeRune('\n') }, "‹\n›"}, + // The Safe() API turns anything into something safe. However, the contents + // still get escaped as needed. + {func(w p) { w.Print("ab ", Safe("c‹d›e ")) }, "‹ab› c?d?e "}, + {func(w p) { w.Printf("ab %03d ", Safe(12)) }, "ab 012 "}, + // Something that'd be otherwise safe, becomes unsafe with Unsafe(). + {func(w p) { w.Print(Unsafe(SafeString("abc"))) }, "‹abc›"}, + {func(w p) { w.Print(Unsafe(SafeRune('a'))) }, "‹97›"}, + {func(w p) { w.Print(Unsafe(Sprint("abc"))) }, "‹?abc?›"}, + {func(w p) { w.Print(Unsafe(Safe("abc"))) }, "‹abc›"}, + {func(w p) { w.Printf("%v", Unsafe(SafeString("abc"))) }, "‹abc›"}, + {func(w p) { w.Printf("%v", Unsafe(SafeRune('a'))) }, "‹97›"}, + {func(w p) { w.Printf("%v", Unsafe(Sprint("abc"))) }, "‹?abc?›"}, + {func(w p) { w.Printf("%v", Unsafe(Safe("abc"))) }, "‹abc›"}, + {func(w p) { w.Printf("%03d", Unsafe(12)) }, "‹012›"}, + // A string that's already redactable gets included as-is; + // in that case, the printf verb and flags are ignored. + {func(w p) { w.Print("ab ", Sprint(12, Safe(34))) }, "‹ab› ‹12› 34"}, + {func(w p) { w.Printf("ab %q", Sprint(12, Safe(34))) }, "ab ‹12› 34"}, + {func(w p) { w.Printf("ab %d", Sprint(12, Safe(34))) }, "ab ‹12› 34"}, + } + + var methods = []struct { + name string + fn func(interface{}) string + }{ + {"sprint", func(a interface{}) string { return string(Sprint(a)) }}, + {"sprintf", func(a interface{}) string { return string(Sprintf("%v", a)) }}, + {"fprint", func(a interface{}) string { var b strings.Builder; _, _ = Fprint(&b, a); return b.String() }}, + {"fprintf", func(a interface{}) string { var b strings.Builder; _, _ = Fprintf(&b, "%v", a); return b.String() }}, + } + + for _, m := range methods { + t.Run(m.name, func(t *testing.T) { + for i, tc := range testData { + res := m.fn(compose{fn: tc.fn}) + + if res != tc.expected { + t.Errorf("%d: expected:\n %s\n\ngot:\n%s", i, + strings.ReplaceAll(tc.expected, "\n", "\n "), + strings.ReplaceAll(res, "\n", "\n ")) + } + } + }) + } +} + +func TestCustomSafeTypes(t *testing.T) { + defer func(prev map[reflect.Type]bool) { safeTypeRegistry = prev }(safeTypeRegistry) + RegisterSafeType(reflect.TypeOf(int32(123))) + + actual := Sprint(123, int32(456)) + const expected = `‹123› 456` + if actual != expected { + t.Errorf("expected %q, got %q", expected, actual) + } + + // Unsafe can override. + actual = Sprint(123, Unsafe(int32(456))) + const expected2 = `‹123› ‹456›` + if actual != expected2 { + t.Errorf("expected %q, got %q", expected2, actual) + } +} + +func TestConversions(t *testing.T) { + const data = `‹123› 456` + s := RedactableString(data) + + bconv := s.ToBytes() + expected := []byte(data) + if !bytes.Equal(bconv, expected) { + t.Errorf("\nexpected: %+v,\n got: %+v", expected, bconv) + } + + sconv := bconv.ToString() + if s != sconv { + t.Errorf("expected %q, got %q", s, sconv) + } +} + +func TestFormatPropagation(t *testing.T) { + testData := []struct { + actual RedactableString + expected RedactableString + }{ + {Sprintf(":%10s:", safe("abc")), `: abc:`}, + {Sprintf(":%10s:", "abc"), `:‹ abc›:`}, + {Sprintf(":%+#03x:", safeint(123)), `:+0x7b:`}, + {Sprintf(":%+#03x:", 123), `:‹+0x7b›:`}, + } + + for _, tc := range testData { + if tc.actual != tc.expected { + t.Errorf("expected %q, got %q", tc.expected, tc.actual) + } + } +} + +type compose struct { + fn func(p) +} + +func (c compose) SafeFormat(w SafePrinter, _ rune) { + c.fn(w) +} + +type safe string + +func (safe) SafeValue() {} + +type safeint int + +func (safeint) SafeValue() {} + +func TestTransform(t *testing.T) { + testData := []struct { + actual string + expected string + }{ + {string(StartMarker()), `‹`}, + {string(EndMarker()), `›`}, + {string(RedactedMarker()), `‹×›`}, + {string(EscapeMarkers([]byte(`a ‹ b › c`))), `a ? b ? c`}, + {string(RedactableBytes([]byte(`a ‹ b › c`)).Redact()), `a ‹×› c`}, + {string(RedactableBytes([]byte(`a ‹ b › c`)).StripMarkers()), `a b c`}, + {string(RedactableString(`a ‹ b › c`).Redact()), `a ‹×› c`}, + {RedactableString(`a ‹ b › c`).StripMarkers(), `a b c`}, + } + + for _, tc := range testData { + if tc.actual != tc.expected { + t.Errorf("expected %q, got %q", tc.expected, tc.actual) + } + } +} + +// TestRedactStream verifies that the redaction logic is able to both +// add the redaction quotes and also respects the format parameters +// and verb. +func TestRedactStream(t *testing.T) { + testData := []struct { + f string + input interface{} + expected string + }{ + {"%v", "", "‹›"}, + {"%v", " ", "‹›"}, + {"%v", "abc ", "‹abc›"}, + {"%q", "abc ", `‹"abc "›`}, + {"%v", "abc\n ", "‹abc›"}, + {"%v", "abc \n\n", "‹abc›"}, + {"%v", " \n\nabc", "‹ \n\nabc›"}, + {"%v", "‹abc›", "‹?abc?›"}, + {"%v", 123, "‹123›"}, + {"%05d", 123, "‹00123›"}, + {"%v", Safe(123), "123"}, + {"%05d", Safe(123), "00123"}, + {"%#x", 17, "‹0x11›"}, + {"%+v", &complexObj{"‹›"}, "‹&{v:??}›"}, + {"%v", &safestringer{"as"}, "as"}, + {"%v", &stringer{"as"}, "‹as›"}, + {"%v", &safefmtformatter{"af"}, "af"}, + {"%v", &fmtformatter{"af"}, "‹af›"}, + {"%v", &safemsg{"az"}, "az"}, + // Printers that cause panics during rendering. + {"%v", &safepanicObj1{"s1-x‹y›z"}, `%!v(PANIC=String method: s1-x?y?z)`}, + {"%v", &safepanicObj2{"s2-x‹y›z"}, `%!v(PANIC=Format method: s2-x?y?z)`}, + {"%v", &panicObj1{"p1-x‹y›z"}, `‹%!v(PANIC=String method: p1-x?y?z)›`}, + {"%v", &panicObj2{"p2-x‹y›z"}, `‹%!v(PANIC=Format method: p2-x?y?z)›`}, + {"%v", &panicObj3{"p3-x‹y›z"}, `%!v(PANIC=SafeFormatter method: p3-x?y?z)`}, + } + + for i, tc := range testData { + var buf strings.Builder + n, _ := Fprintf(&buf, tc.f, tc.input) + result := buf.String() + if result != tc.expected { + t.Errorf("%d: expected %q, got %q", i, tc.expected, result) + } + if n != len(result) { + t.Errorf("%d: expected len %d, got %d", i, n, len(result)) + } + } +} + +type complexObj struct { + v string +} + +type stringer struct{ s string } + +var _ fmt.Stringer = (*stringer)(nil) + +func (s *stringer) String() string { return s.s } + +type safestringer struct{ s string } + +var _ SafeValue = (*safestringer)(nil) +var _ fmt.Stringer = (*safestringer)(nil) + +func (*safestringer) SafeValue() {} +func (s *safestringer) String() string { return s.s } + +type fmtformatter struct{ s string } + +var _ fmt.Formatter = (*fmtformatter)(nil) + +func (s *fmtformatter) Format(w fmt.State, _ rune) { fmt.Fprint(w, s.s) } + +type safefmtformatter struct{ s string } + +var _ SafeValue = (*safefmtformatter)(nil) +var _ fmt.Formatter = (*safefmtformatter)(nil) + +func (*safefmtformatter) SafeValue() {} +func (s *safefmtformatter) Format(w fmt.State, _ rune) { fmt.Fprint(w, s.s) } + +type panicObj1 struct{ s string } + +var _ fmt.Stringer = (*panicObj1)(nil) + +func (p *panicObj1) String() string { panic(p.s) } + +type panicObj2 struct{ s string } + +var _ fmt.Formatter = (*panicObj2)(nil) + +func (p *panicObj2) Format(fmt.State, rune) { panic(p.s) } + +type safepanicObj1 struct{ s string } + +var _ SafeValue = (*safepanicObj1)(nil) +var _ fmt.Stringer = (*safepanicObj1)(nil) + +func (*safepanicObj1) SafeValue() {} +func (p *safepanicObj1) String() string { panic(p.s) } + +type safepanicObj2 struct{ s string } + +var _ SafeValue = (*safepanicObj2)(nil) +var _ fmt.Formatter = (*safepanicObj2)(nil) + +func (*safepanicObj2) SafeValue() {} +func (p *safepanicObj2) Format(fmt.State, rune) { panic(p.s) } + +type panicObj3 struct{ s string } + +var _ SafeFormatter = (*panicObj3)(nil) + +func (p *panicObj3) SafeFormat(SafePrinter, rune) { panic(p.s) } + +type safemsg struct { + s string +} + +var _ SafeMessager = (*safemsg)(nil) + +func (p *safemsg) SafeMessage() string { return p.s } diff --git a/pkg/util/redact/wrappers.go b/pkg/util/redact/wrappers.go new file mode 100644 index 000000000000..35a08ea59812 --- /dev/null +++ b/pkg/util/redact/wrappers.go @@ -0,0 +1,59 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package redact + +import "fmt" + +// Unsafe turns any value that would otherwise be considered safe, +// into an unsafe value. +func Unsafe(a interface{}) interface{} { + return unsafeWrap{a} +} + +type unsafeWrap struct { + a interface{} +} + +func (w unsafeWrap) Format(s fmt.State, verb rune) { + reproducePrintf(s, s, verb, w.a) +} + +// Safe turns any value into an object that is considered as safe by +// the formatter. +// +// This is provided as an “escape hatch” for cases where the other +// interfaces and conventions fail. Increased usage of this mechanism +// should be taken as a signal that a new abstraction is missing. +// The implementation is also slow. +func Safe(a interface{}) SafeValue { + return safeWrapper{a} +} + +type safeWrapper struct { + a interface{} +} + +var _ SafeValue = safeWrapper{} +var _ fmt.Formatter = safeWrapper{} +var _ SafeMessager = safeWrapper{} + +// SafeValue implements the SafeValue interface. +func (w safeWrapper) SafeValue() {} + +// Format implements the fmt.Formatter interface. +func (w safeWrapper) Format(s fmt.State, verb rune) { + reproducePrintf(s, s, verb, w.a) +} + +// SafeMessage implements SafeMessager. +func (w safeWrapper) SafeMessage() string { + return fmt.Sprintf("%v", w.a) +} diff --git a/pkg/util/syncutil/atomic.go b/pkg/util/syncutil/atomic.go index 0ecf28ac877f..08cd9362f39d 100644 --- a/pkg/util/syncutil/atomic.go +++ b/pkg/util/syncutil/atomic.go @@ -30,3 +30,29 @@ func StoreFloat64(addr *AtomicFloat64, val float64) { func LoadFloat64(addr *AtomicFloat64) (val float64) { return math.Float64frombits(atomic.LoadUint64((*uint64)(addr))) } + +// AtomicBool mimics an atomic boolean. +type AtomicBool uint32 + +// Set atomically sets the boolean. +func (b *AtomicBool) Set(v bool) { + s := uint32(0) + if v { + s = 1 + } + atomic.StoreUint32((*uint32)(b), s) +} + +// Get atomically gets the boolean. +func (b *AtomicBool) Get() bool { + return atomic.LoadUint32((*uint32)(b)) != 0 +} + +// Swap atomically swaps the value. +func (b *AtomicBool) Swap(v bool) bool { + wanted := uint32(0) + if v { + wanted = 1 + } + return atomic.SwapUint32((*uint32)(b), wanted) != 0 +} From e895e441e5f95a89d56aaf55e51c86ab85e75dfd Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Mon, 8 Jun 2020 13:22:20 -0700 Subject: [PATCH 26/46] geo: fix GeoJSON and allow options in ST_AsGeoJSON There was a bug in GeoJSON where we used the high level name:"Feature" field, as opposed to the inner Geometry. This has been fixed. We also support more options for ST_AsGeoJSON now that these changes are in twpayne/go-geom. Release note (sql change): Implement ST_AsGeoJSON with options to show bbox and CRS information. --- Gopkg.lock | 6 +- docs/generated/sql/functions.md | 28 ++++- pkg/geo/encode.go | 82 ++++++++++++- pkg/geo/encode_test.go | 20 +++- pkg/geo/parse.go | 6 +- pkg/geo/parse_test.go | 4 +- .../logictest/testdata/logic_test/geospatial | 107 +++++++++++------ pkg/sql/sem/builtins/geo_builtins.go | 108 +++++++++++++++++- vendor | 2 +- 9 files changed, 301 insertions(+), 62 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 139af6ca6fbc..119a2271d642 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -1615,7 +1615,7 @@ version = "v1.4.0" [[projects]] - digest = "1:c8fafded0eb90c84e4ef05b9e1ff2a5c8fb2aaf5009ba11b91f6bed2ef680aae" + digest = "1:5b7e59f9330bbc88c3aa7b62d92d1d29bb6ccbc176b731986de70345f3522dd7" name = "github.com/twpayne/go-geom" packages = [ ".", @@ -1636,8 +1636,8 @@ "xy/orientation", ] pruneopts = "UT" - revision = "078e8ab21d838b07c627dc1d704f1f80688b1b90" - version = "v1.1.0" + revision = "62c03a64717d682dd89319176fd7247944f0c518" + version = "v1.2.1" [[projects]] digest = "1:43e0db2b113d1aee4bb68745598c135511976a44fb380ebd701cd0c14a77c303" diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 14153d12e99d..ba5955b6ef9a 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -720,9 +720,33 @@ has no relationship with the commit order of concurrent transactions.

st_asewkt(geometry: geometry) → string

Returns the EWKT representation of a given Geometry.

-st_asgeojson(geography: geography) → string

Returns the GeoJSON representation of a given Geography.

+st_asgeojson(geography: geography) → string

Returns the GeoJSON representation of a given Geography. Coordinates have a maximum of 9 decimal digits.

-st_asgeojson(geometry: geometry) → string

Returns the GeoJSON representation of a given Geometry.

+st_asgeojson(geography: geography, max_decimal_digits: int) → string

Returns the GeoJSON representation of a given Geography with max_decimal_digits output for each coordinate value.

+
+st_asgeojson(geography: geography, max_decimal_digits: int, options: int) → string

Returns the GeoJSON representation of a given Geography with max_decimal_digits output for each coordinate value.

+

Options is a flag that can be bitmasked. The options are:

+
    +
  • 0: no option (default for Geography)
  • +
  • 1: GeoJSON BBOX
  • +
  • 2: GeoJSON Short CRS (e.g EPSG:4326)
  • +
  • 4: GeoJSON Long CRS (e.g urn:ogc:def:crs:EPSG::4326)
  • +
  • 8: GeoJSON Short CRS if not EPSG:4326
  • +
+
+st_asgeojson(geometry: geometry) → string

Returns the GeoJSON representation of a given Geometry. Coordinates have a maximum of 9 decimal digits.

+
+st_asgeojson(geometry: geometry, max_decimal_digits: int) → string

Returns the GeoJSON representation of a given Geometry with max_decimal_digits output for each coordinate value.

+
+st_asgeojson(geometry: geometry, max_decimal_digits: int, options: int) → string

Returns the GeoJSON representation of a given Geometry with max_decimal_digits output for each coordinate value.

+

Options is a flag that can be bitmasked. The options are:

+
    +
  • 0: no option
  • +
  • 1: GeoJSON BBOX
  • +
  • 2: GeoJSON Short CRS (e.g EPSG:4326)
  • +
  • 4: GeoJSON Long CRS (e.g urn:ogc:def:crs:EPSG::4326)
  • +
  • 8: GeoJSON Short CRS if not EPSG:4326 (default for Geometry)
  • +
st_ashexewkb(geography: geography) → string

Returns the EWKB representation in hex of a given Geography.

diff --git a/pkg/geo/encode.go b/pkg/geo/encode.go index 229ffcfee1df..37edd8a010f4 100644 --- a/pkg/geo/encode.go +++ b/pkg/geo/encode.go @@ -17,6 +17,9 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/geo/geopb" + "github.com/cockroachdb/cockroach/pkg/geo/geoprojbase" + "github.com/cockroachdb/errors" + "github.com/twpayne/go-geom" "github.com/twpayne/go-geom/encoding/ewkb" "github.com/twpayne/go-geom/encoding/geojson" "github.com/twpayne/go-geom/encoding/kml" @@ -71,17 +74,86 @@ func EWKBToWKB(b geopb.EWKB, byteOrder binary.ByteOrder) (geopb.WKB, error) { return geopb.WKB(ret), err } +// EWKBToGeoJSONFlag maps to the ST_AsGeoJSON flags for PostGIS. +type EWKBToGeoJSONFlag int + +// These should be kept with ST_AsGeoJSON in PostGIS. +// 0: means no option +// 1: GeoJSON BBOX +// 2: GeoJSON Short CRS (e.g EPSG:4326) +// 4: GeoJSON Long CRS (e.g urn:ogc:def:crs:EPSG::4326) +// 8: GeoJSON Short CRS if not EPSG:4326 (default) +const ( + EWKBToGeoJSONFlagIncludeBBox EWKBToGeoJSONFlag = 1 << (iota) + EWKBToGeoJSONFlagShortCRS + EWKBToGeoJSONFlagLongCRS + EWKBToGeoJSONFlagShortCRSIfNot4326 + + EWKBToGeoJSONFlagZero = 0 +) + +// geomToGeoJSONCRS converts a geom to its CRS GeoJSON form. +func geomToGeoJSONCRS(t geom.T, long bool) (*geojson.CRS, error) { + projection, ok := geoprojbase.Projection(geopb.SRID(t.SRID())) + if !ok { + return nil, errors.Newf("unknown SRID: %d", t.SRID()) + } + var prop string + if long { + prop = fmt.Sprintf("urn:ogc:def:crs:%s::%d", projection.AuthName, projection.AuthSRID) + } else { + prop = fmt.Sprintf("%s:%d", projection.AuthName, projection.AuthSRID) + } + crs := &geojson.CRS{ + Type: "name", + Properties: map[string]interface{}{ + "name": prop, + }, + } + return crs, nil +} + // EWKBToGeoJSON transforms a given EWKB to GeoJSON. -func EWKBToGeoJSON(b geopb.EWKB) ([]byte, error) { +func EWKBToGeoJSON(b geopb.EWKB, maxDecimalDigits int, flag EWKBToGeoJSONFlag) ([]byte, error) { t, err := ewkb.Unmarshal([]byte(b)) if err != nil { return nil, err } - f := geojson.Feature{ - // TODO(otan): add features once we have spatial_ref_sys. - Geometry: t, + options := []geojson.EncodeGeometryOption{ + geojson.EncodeGeometryWithMaxDecimalDigits(maxDecimalDigits), + } + if flag&EWKBToGeoJSONFlagIncludeBBox != 0 { + options = append( + options, + geojson.EncodeGeometryWithBBox(), + ) } - return f.MarshalJSON() + // Take CRS flag in order of precedence. + if t.SRID() != 0 { + if flag&EWKBToGeoJSONFlagLongCRS != 0 { + crs, err := geomToGeoJSONCRS(t, true /* long */) + if err != nil { + return nil, err + } + options = append(options, geojson.EncodeGeometryWithCRS(crs)) + } else if flag&EWKBToGeoJSONFlagShortCRS != 0 { + crs, err := geomToGeoJSONCRS(t, false /* long */) + if err != nil { + return nil, err + } + options = append(options, geojson.EncodeGeometryWithCRS(crs)) + } else if flag&EWKBToGeoJSONFlagShortCRSIfNot4326 != 0 { + if t.SRID() != 4326 { + crs, err := geomToGeoJSONCRS(t, false /* long */) + if err != nil { + return nil, err + } + options = append(options, geojson.EncodeGeometryWithCRS(crs)) + } + } + } + + return geojson.Marshal(t, options...) } // EWKBToWKBHex transforms a given EWKB to WKBHex. diff --git a/pkg/geo/encode_test.go b/pkg/geo/encode_test.go index 06d2c79a381f..52534031b15b 100644 --- a/pkg/geo/encode_test.go +++ b/pkg/geo/encode_test.go @@ -80,17 +80,31 @@ func TestEWKBToWKB(t *testing.T) { func TestEWKBToGeoJSON(t *testing.T) { testCases := []struct { ewkt geopb.EWKT + flag EWKBToGeoJSONFlag expected string }{ - {"POINT(1.0 1.0)", `{"type":"Feature","geometry":{"type":"Point","coordinates":[1,1]},"properties":null}`}, - {"SRID=4;POINT(1.0 1.0)", `{"type":"Feature","geometry":{"type":"Point","coordinates":[1,1]},"properties":null}`}, + {"POINT(1.0 1.0)", EWKBToGeoJSONFlagZero, `{"type":"Point","coordinates":[1,1]}`}, + {"POINT(1.0 1.0)", EWKBToGeoJSONFlagIncludeBBox, `{"type":"Point","bbox":[1,1,1,1],"coordinates":[1,1]}`}, + {"POINT(1.0 1.0)", EWKBToGeoJSONFlagShortCRS | EWKBToGeoJSONFlagIncludeBBox, `{"type":"Point","bbox":[1,1,1,1],"coordinates":[1,1]}`}, + {"POINT(1.0 1.0)", EWKBToGeoJSONFlagShortCRS, `{"type":"Point","coordinates":[1,1]}`}, + {"POINT(1.0 1.0)", EWKBToGeoJSONFlagLongCRS, `{"type":"Point","coordinates":[1,1]}`}, + {"POINT(1.0 1.0)", EWKBToGeoJSONFlagShortCRSIfNot4326, `{"type":"Point","coordinates":[1,1]}`}, + {"POINT(1.1234567 1.9876543)", EWKBToGeoJSONFlagShortCRSIfNot4326, `{"type":"Point","coordinates":[1.123457,1.987654]}`}, + {"SRID=4326;POINT(1.0 1.0)", EWKBToGeoJSONFlagZero, `{"type":"Point","coordinates":[1,1]}`}, + {"SRID=4326;POINT(1.0 1.0)", EWKBToGeoJSONFlagIncludeBBox, `{"type":"Point","bbox":[1,1,1,1],"coordinates":[1,1]}`}, + {"SRID=4326;POINT(1.0 1.0)", EWKBToGeoJSONFlagLongCRS, `{"type":"Point","crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4326"}},"coordinates":[1,1]}`}, + {"SRID=4326;POINT(1.0 1.0)", EWKBToGeoJSONFlagShortCRS, `{"type":"Point","crs":{"type":"name","properties":{"name":"EPSG:4326"}},"coordinates":[1,1]}`}, + {"SRID=4004;POINT(1.0 1.0)", EWKBToGeoJSONFlagShortCRS, `{"type":"Point","crs":{"type":"name","properties":{"name":"EPSG:4004"}},"coordinates":[1,1]}`}, + {"SRID=4004;POINT(1.0 1.0)", EWKBToGeoJSONFlagShortCRS | EWKBToGeoJSONFlagIncludeBBox, `{"type":"Point","bbox":[1,1,1,1],"crs":{"type":"name","properties":{"name":"EPSG:4004"}},"coordinates":[1,1]}`}, + {"SRID=4326;POINT(1.0 1.0)", EWKBToGeoJSONFlagShortCRSIfNot4326, `{"type":"Point","coordinates":[1,1]}`}, + {"SRID=4004;POINT(1.0 1.0)", EWKBToGeoJSONFlagShortCRSIfNot4326, `{"type":"Point","crs":{"type":"name","properties":{"name":"EPSG:4004"}},"coordinates":[1,1]}`}, } for _, tc := range testCases { t.Run(string(tc.ewkt), func(t *testing.T) { so, err := parseEWKT(tc.ewkt, geopb.DefaultGeometrySRID, DefaultSRIDIsHint) require.NoError(t, err) - encoded, err := EWKBToGeoJSON(so.EWKB) + encoded, err := EWKBToGeoJSON(so.EWKB, 6, tc.flag) require.NoError(t, err) require.Equal(t, tc.expected, string(encoded)) }) diff --git a/pkg/geo/parse.go b/pkg/geo/parse.go index 4101c16faf39..3f05493c2f5f 100644 --- a/pkg/geo/parse.go +++ b/pkg/geo/parse.go @@ -99,12 +99,10 @@ func parseWKB(b []byte, defaultSRID geopb.SRID) (geopb.SpatialObject, error) { // parseGeoJSON takes given bytes assumed to be GeoJSON and transforms it into a SpatialObject. func parseGeoJSON(b []byte, defaultSRID geopb.SRID) (geopb.SpatialObject, error) { - var f geojson.Feature - if err := f.UnmarshalJSON(b); err != nil { + var t geom.T + if err := geojson.Unmarshal(b, &t); err != nil { return geopb.SpatialObject{}, err } - t := f.Geometry - // TODO(otan): check SRID from properties. if defaultSRID != 0 && t.SRID() == 0 { adjustGeomSRID(t, defaultSRID) } diff --git a/pkg/geo/parse_test.go b/pkg/geo/parse_test.go index 3427e6f90252..bd0d55c22766 100644 --- a/pkg/geo/parse_test.go +++ b/pkg/geo/parse_test.go @@ -225,7 +225,7 @@ func TestParseGeometry(t *testing.T) { "", }, { - `{ "type": "Feature", "geometry": { "type": "Point", "coordinates": [1.0, 1.0] }, "properties": { "name": "┳━┳ ヽ(ಠل͜ಠ)ノ" } }`, + `{ "type": "Point", "coordinates": [1.0, 1.0] }`, &Geometry{ SpatialObject: geopb.SpatialObject{ EWKB: []byte("\x01\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\xf0\x3f\x00\x00\x00\x00\x00\x00\xf0\x3f"), @@ -367,7 +367,7 @@ func TestParseGeography(t *testing.T) { "", }, { - `{ "type": "Feature", "geometry": { "type": "Point", "coordinates": [1.0, 1.0] }, "properties": { "name": "┳━┳ ヽ(ಠل͜ಠ)ノ" } }`, + `{ "type": "Point", "coordinates": [1.0, 1.0] }`, &Geography{ SpatialObject: geopb.SpatialObject{ EWKB: []byte("\x01\x01\x00\x00\x20\xe6\x10\x00\x00\x00\x00\x00\x00\x00\x00\xf0\x3f\x00\x00\x00\x00\x00\x00\xf0\x3f"), diff --git a/pkg/sql/logictest/testdata/logic_test/geospatial b/pkg/sql/logictest/testdata/logic_test/geospatial index 75b829a5cb78..f1542fdfcf00 100644 --- a/pkg/sql/logictest/testdata/logic_test/geospatial +++ b/pkg/sql/logictest/testdata/logic_test/geospatial @@ -196,8 +196,8 @@ INSERT INTO parse_test (geom, geog) VALUES (ST_GeomFromText('POINT(1.0 2.0)'), ST_GeogFromText('POINT(1.0 2.0)')), (ST_GeomFromText('SRID=4326;POINT(1.0 2.0)'), ST_GeogFromText('SRID=4326;POINT(1.0 2.0)')), (ST_GeometryFromText('SRID=4004;POINT(1.0 2.0)'), ST_GeographyFromText('POINT(1.0 2.0)')), - (ST_GeomFromGeoJSON('{"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null}'), ST_GeogFromGeoJSON('{"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null}')), - (ST_GeomFromGeoJSON('{"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null}'::jsonb), ST_GeogFromGeoJSON('{"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null}'::jsonb)), + (ST_GeomFromGeoJSON('{"type":"Point","coordinates":[1,2]}'), ST_GeogFromGeoJSON('{"type":"Point","coordinates":[1,2]}')), + (ST_GeomFromGeoJSON('{"type":"Point","coordinates":[1,2]}'::jsonb), ST_GeogFromGeoJSON('{"type":"Point","coordinates":[1,2]}'::jsonb)), (ST_GeomFromWKB(decode('0101000000000000000000F03F000000000000F03F', 'hex')), ST_GeogFromWKB(decode('0101000000000000000000F03F000000000000F03F', 'hex'))), (ST_GeomFromEWKB(decode('0101000000000000000000F03F000000000000F03F', 'hex')), ST_GeogFromEWKB(decode('0101000000000000000000F03F000000000000F03F', 'hex'))), (st_geomfromgeojson('null':::jsonb), st_geogfromgeojson('null':::jsonb)) @@ -209,7 +209,7 @@ SELECT ---- true true -query TTTTTTTT +query TTTTTTT SELECT ST_AsText(geom), ST_AsEWKT(geom), @@ -217,25 +217,41 @@ SELECT ST_AsBinary(geom, 'ndr'), ST_AsBinary(geom, 'xdr'), ST_AsEWKB(geom), - ST_AsKML(geom), - ST_AsGeoJSON(geom) + ST_AsKML(geom) +FROM parse_test ORDER BY id ASC +---- +POINT (1 2) POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 2) SRID=4004;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 164 15 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 2) POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 2) POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 1) POINT (1 1) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [0 0 0 0 1 63 240 0 0 0 0 0 0 63 240 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] +1,1 +POINT (1 1) POINT (1 1) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [0 0 0 0 1 63 240 0 0 0 0 0 0 63 240 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] +1,1 +NULL NULL NULL NULL NULL NULL NULL + + +query TTT +SELECT + ST_AsGeoJSON(geom), + ST_AsGeoJSON(geom, 6, 8), + ST_AsGeoJSON(geom, 6, 5) FROM parse_test ORDER BY id ASC ---- -POINT (1 2) POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 2) SRID=4004;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 164 15 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 2) POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 2) POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 1) POINT (1 1) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [0 0 0 0 1 63 240 0 0 0 0 0 0 63 240 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] -1,1 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,1]},"properties":null} -POINT (1 1) POINT (1 1) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [0 0 0 0 1 63 240 0 0 0 0 0 0 63 240 0 0 0 0 0 0] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] -1,1 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,1]},"properties":null} -NULL NULL NULL NULL NULL NULL NULL NULL +{"type":"Point","coordinates":[1,2]} {"type":"Point","coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"coordinates":[1,2]} +{"type":"Point","coordinates":[1,2]} {"type":"Point","coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4326"}},"coordinates":[1,2]} +{"type":"Point","crs":{"type":"name","properties":{"name":"EPSG:4004"}},"coordinates":[1,2]} {"type":"Point","crs":{"type":"name","properties":{"name":"EPSG:4004"}},"coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4004"}},"coordinates":[1,2]} +{"type":"Point","coordinates":[1,2]} {"type":"Point","coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"coordinates":[1,2]} +{"type":"Point","coordinates":[1,2]} {"type":"Point","coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"coordinates":[1,2]} +{"type":"Point","coordinates":[1,1]} {"type":"Point","coordinates":[1,1]} {"type":"Point","bbox":[1,1,1,1],"coordinates":[1,1]} +{"type":"Point","coordinates":[1,1]} {"type":"Point","coordinates":[1,1]} {"type":"Point","bbox":[1,1,1,1],"coordinates":[1,1]} +NULL NULL NULL query TTTT SELECT @@ -254,7 +270,7 @@ FROM parse_test ORDER BY id ASC 0101000000000000000000F03F000000000000F03F 0101000000000000000000F03F000000000000F03F 0101000000000000000000F03F000000000000F03F 00000000013FF00000000000003FF0000000000000 NULL NULL NULL NULL -query TTTTTTTT +query TTTTTTT SELECT ST_AsText(geog), ST_AsEWKT(geog), @@ -262,25 +278,40 @@ SELECT ST_AsBinary(geog, 'ndr'), ST_AsBinary(geog, 'xdr'), ST_AsEWKB(geog), - ST_AsKML(geog), - ST_AsGeoJSON(geog) + ST_AsKML(geog) +FROM parse_test ORDER BY id ASC +---- +POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] +1,2 +POINT (1 1) SRID=4326;POINT (1 1) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [0 0 0 0 1 63 240 0 0 0 0 0 0 63 240 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] +1,1 +POINT (1 1) SRID=4326;POINT (1 1) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [0 0 0 0 1 63 240 0 0 0 0 0 0 63 240 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] +1,1 +NULL NULL NULL NULL NULL NULL NULL + +query TTT +SELECT + ST_AsGeoJSON(geog), + ST_AsGeoJSON(geog, 6, 8), + ST_AsGeoJSON(geog, 6, 5) FROM parse_test ORDER BY id ASC ---- -POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 2) SRID=4326;POINT (1 2) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] [0 0 0 0 1 63 240 0 0 0 0 0 0 64 0 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 0 64] -1,2 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,2]},"properties":null} -POINT (1 1) SRID=4326;POINT (1 1) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [0 0 0 0 1 63 240 0 0 0 0 0 0 63 240 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] -1,1 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,1]},"properties":null} -POINT (1 1) SRID=4326;POINT (1 1) [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [1 1 0 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] [0 0 0 0 1 63 240 0 0 0 0 0 0 63 240 0 0 0 0 0 0] [1 1 0 0 32 230 16 0 0 0 0 0 0 0 0 240 63 0 0 0 0 0 0 240 63] -1,1 {"type":"Feature","geometry":{"type":"Point","coordinates":[1,1]},"properties":null} -NULL NULL NULL NULL NULL NULL NULL NULL +{"type":"Point","coordinates":[1,2]} {"type":"Point","coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4326"}},"coordinates":[1,2]} +{"type":"Point","coordinates":[1,2]} {"type":"Point","coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4326"}},"coordinates":[1,2]} +{"type":"Point","coordinates":[1,2]} {"type":"Point","coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4326"}},"coordinates":[1,2]} +{"type":"Point","coordinates":[1,2]} {"type":"Point","coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4326"}},"coordinates":[1,2]} +{"type":"Point","coordinates":[1,2]} {"type":"Point","coordinates":[1,2]} {"type":"Point","bbox":[1,2,1,2],"crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4326"}},"coordinates":[1,2]} +{"type":"Point","coordinates":[1,1]} {"type":"Point","coordinates":[1,1]} {"type":"Point","bbox":[1,1,1,1],"crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4326"}},"coordinates":[1,1]} +{"type":"Point","coordinates":[1,1]} {"type":"Point","coordinates":[1,1]} {"type":"Point","bbox":[1,1,1,1],"crs":{"type":"name","properties":{"name":"urn:ogc:def:crs:EPSG::4326"}},"coordinates":[1,1]} +NULL NULL NULL query TTTT SELECT diff --git a/pkg/sql/sem/builtins/geo_builtins.go b/pkg/sql/sem/builtins/geo_builtins.go index 6ed97ff883f1..33215914975a 100644 --- a/pkg/sql/sem/builtins/geo_builtins.go +++ b/pkg/sql/sem/builtins/geo_builtins.go @@ -47,6 +47,10 @@ const spheroidDistanceMessage = `"\n\nWhen operating on a spheroid, this functio `the closest two points using S2. The spheroid distance between these two points is calculated using GeographicLib. ` + `This follows observed PostGIS behavior.` +const ( + defaultGeoJSONDecimalDigits = 9 +) + // infoBuilder is used to build a detailed info string that is consistent between // geospatial data types. type infoBuilder struct { @@ -837,22 +841,118 @@ var geoBuiltins = map[string]builtinDefinition{ defProps(), geometryOverload1( func(_ *tree.EvalContext, g *tree.DGeometry) (tree.Datum, error) { - geojson, err := geo.EWKBToGeoJSON(g.Geometry.EWKB()) + geojson, err := geo.EWKBToGeoJSON(g.Geometry.EWKB(), defaultGeoJSONDecimalDigits, geo.EWKBToGeoJSONFlagShortCRSIfNot4326) return tree.NewDString(string(geojson)), err }, types.String, - infoBuilder{info: "Returns the GeoJSON representation of a given Geometry."}, + infoBuilder{ + info: fmt.Sprintf( + "Returns the GeoJSON representation of a given Geometry. Coordinates have a maximum of %d decimal digits.", + defaultGeoJSONDecimalDigits, + ), + }, tree.VolatilityImmutable, ), + tree.Overload{ + Types: tree.ArgTypes{ + {"geometry", types.Geometry}, + {"max_decimal_digits", types.Int}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(_ *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + g := args[0].(*tree.DGeometry) + maxDecimalDigits := int(tree.MustBeDInt(args[1])) + geojson, err := geo.EWKBToGeoJSON(g.Geometry.EWKB(), maxDecimalDigits, geo.EWKBToGeoJSONFlagShortCRSIfNot4326) + return tree.NewDString(string(geojson)), err + }, + Info: infoBuilder{ + info: `Returns the GeoJSON representation of a given Geometry with max_decimal_digits output for each coordinate value.`, + }.String(), + Volatility: tree.VolatilityImmutable, + }, + tree.Overload{ + Types: tree.ArgTypes{ + {"geometry", types.Geometry}, + {"max_decimal_digits", types.Int}, + {"options", types.Int}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(_ *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + g := args[0].(*tree.DGeometry) + maxDecimalDigits := int(tree.MustBeDInt(args[1])) + options := geo.EWKBToGeoJSONFlag(tree.MustBeDInt(args[2])) + geojson, err := geo.EWKBToGeoJSON(g.Geometry.EWKB(), maxDecimalDigits, options) + return tree.NewDString(string(geojson)), err + }, + Info: infoBuilder{ + info: `Returns the GeoJSON representation of a given Geometry with max_decimal_digits output for each coordinate value. + +Options is a flag that can be bitmasked. The options are: +* 0: no option +* 1: GeoJSON BBOX +* 2: GeoJSON Short CRS (e.g EPSG:4326) +* 4: GeoJSON Long CRS (e.g urn:ogc:def:crs:EPSG::4326) +* 8: GeoJSON Short CRS if not EPSG:4326 (default for Geometry) +`}.String(), + Volatility: tree.VolatilityImmutable, + }, geographyOverload1( func(_ *tree.EvalContext, g *tree.DGeography) (tree.Datum, error) { - geojson, err := geo.EWKBToGeoJSON(g.Geography.EWKB()) + geojson, err := geo.EWKBToGeoJSON(g.Geography.EWKB(), defaultGeoJSONDecimalDigits, geo.EWKBToGeoJSONFlagZero) return tree.NewDString(string(geojson)), err }, types.String, - infoBuilder{info: "Returns the GeoJSON representation of a given Geography."}, + infoBuilder{ + info: fmt.Sprintf( + "Returns the GeoJSON representation of a given Geography. Coordinates have a maximum of %d decimal digits.", + defaultGeoJSONDecimalDigits, + ), + }, tree.VolatilityImmutable, ), + tree.Overload{ + Types: tree.ArgTypes{ + {"geography", types.Geography}, + {"max_decimal_digits", types.Int}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(_ *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + g := args[0].(*tree.DGeography) + maxDecimalDigits := int(tree.MustBeDInt(args[1])) + geojson, err := geo.EWKBToGeoJSON(g.Geography.EWKB(), maxDecimalDigits, geo.EWKBToGeoJSONFlagZero) + return tree.NewDString(string(geojson)), err + }, + Info: infoBuilder{ + info: `Returns the GeoJSON representation of a given Geography with max_decimal_digits output for each coordinate value.`, + }.String(), + Volatility: tree.VolatilityImmutable, + }, + tree.Overload{ + Types: tree.ArgTypes{ + {"geography", types.Geography}, + {"max_decimal_digits", types.Int}, + {"options", types.Int}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(_ *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + g := args[0].(*tree.DGeography) + maxDecimalDigits := int(tree.MustBeDInt(args[1])) + options := geo.EWKBToGeoJSONFlag(tree.MustBeDInt(args[2])) + geojson, err := geo.EWKBToGeoJSON(g.Geography.EWKB(), maxDecimalDigits, options) + return tree.NewDString(string(geojson)), err + }, + Info: infoBuilder{ + info: `Returns the GeoJSON representation of a given Geography with max_decimal_digits output for each coordinate value. + +Options is a flag that can be bitmasked. The options are: +* 0: no option (default for Geography) +* 1: GeoJSON BBOX +* 2: GeoJSON Short CRS (e.g EPSG:4326) +* 4: GeoJSON Long CRS (e.g urn:ogc:def:crs:EPSG::4326) +* 8: GeoJSON Short CRS if not EPSG:4326 +`}.String(), + Volatility: tree.VolatilityImmutable, + }, ), // diff --git a/vendor b/vendor index a6597287a886..73bf2a7aadd3 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit a6597287a88652dd0b769822ea047c0b58462cdf +Subproject commit 73bf2a7aadd3f486a0745a427864877d0e3b5cde From 5cbdb0dcc42320f8a1e0b0183a33ceea0b00eb05 Mon Sep 17 00:00:00 2001 From: Jaewan Park Date: Tue, 9 Jun 2020 06:54:19 +0900 Subject: [PATCH 27/46] geo/geogfn: implement ST_Project Fixes #48402 Release note (sql change): This PR implement adds the ST_Project({geography,float8,float8}) --- docs/generated/sql/functions.md | 7 ++ pkg/geo/geogfn/unary_operators.go | 69 +++++++++++++++++++ pkg/geo/geogfn/unary_operators_test.go | 34 +++++++++ pkg/geo/geographiclib/geographiclib.go | 27 +++++++- pkg/geo/geographiclib/geographiclib_test.go | 29 ++++++++ .../logictest/testdata/logic_test/geospatial | 5 ++ pkg/sql/sem/builtins/geo_builtins.go | 50 ++++++++++++++ 7 files changed, 220 insertions(+), 1 deletion(-) diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 14153d12e99d..4f4e046e7cc8 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -1110,6 +1110,13 @@ given Geometry.

st_polygonfromwkb(wkb: bytes, srid: int) → geometry

Returns the Geometry from a WKB representation with an SRID. If the shape underneath is not Polygon, NULL is returned.

+st_project(geography: geography, distance: float, azimuth: float) → geography

Returns a point projected from a start point along a geodesic using a given distance and azimuth (bearing). +This is known as the direct geodesic problem.

+

The distance is given in meters. Negative values are supported.

+

The azimuth (also known as heading or bearing) is given in radians. It is measured clockwise from true north (azimuth zero). +East is azimuth π/2 (90 degrees); south is azimuth π (180 degrees); west is azimuth 3π/2 (270 degrees). +Negative azimuth values and values greater than 2π (360 degrees) are supported.

+
st_relate(geometry_a: geometry, geometry_b: geometry) → string

Returns the DE-9IM spatial relation between geometry_a and geometry_b.

This function utilizes the GEOS module.

diff --git a/pkg/geo/geogfn/unary_operators.go b/pkg/geo/geogfn/unary_operators.go index e5ce3ccf7c05..128c7f76a175 100644 --- a/pkg/geo/geogfn/unary_operators.go +++ b/pkg/geo/geogfn/unary_operators.go @@ -11,9 +11,12 @@ package geogfn import ( + "math" + "github.com/cockroachdb/cockroach/pkg/geo" "github.com/cockroachdb/cockroach/pkg/geo/geographiclib" "github.com/cockroachdb/errors" + "github.com/golang/geo/s1" "github.com/golang/geo/s2" "github.com/twpayne/go-geom" ) @@ -90,6 +93,43 @@ func Length(g *geo.Geography, useSphereOrSpheroid UseSphereOrSpheroid) (float64, return length(regions, useSphereOrSpheroid) } +// Project returns calculate a projected point given a source point, a distance and a azimuth. +func Project(point *geom.Point, distance float64, azimuth s1.Angle) (*geom.Point, error) { + spheroid := geographiclib.WGS84Spheroid + + // Normalize distance to be positive. + if distance < 0.0 { + distance = -distance + azimuth += math.Pi + } + + // Normalize azimuth + azimuth = azimuth.Normalized() + + // Check the distance validity. + if distance > (math.Pi * spheroid.Radius) { + return nil, errors.Newf("distance must not be greater than %f", math.Pi*spheroid.Radius) + } + + // Convert to ta geodetic point. + x := point.X() + y := point.Y() + + projected := spheroid.Project( + s2.LatLngFromDegrees(x, y), + distance, + azimuth, + ) + + return geom.NewPointFlat( + geom.XY, + []float64{ + float64(projected.Lng.Normalized()) * 180.0 / math.Pi, + normalizeLatitude(float64(projected.Lat)) * 180.0 / math.Pi, + }, + ), nil +} + // length returns the sum of the lengtsh and perimeters in the shapes of the Geography. // In OGC parlance, length returns both LineString lengths _and_ Polygon perimeters. func length(regions []s2.Region, useSphereOrSpheroid UseSphereOrSpheroid) (float64, error) { @@ -128,3 +168,32 @@ func length(regions []s2.Region, useSphereOrSpheroid UseSphereOrSpheroid) (float } return totalLength, nil } + +// normalizeLatitude convert a latitude to the range of -Pi/2, Pi/2. +func normalizeLatitude(lat float64) float64 { + if lat > 2.0*math.Pi { + lat = math.Remainder(lat, 2.0*math.Pi) + } + + if lat < -2.0*math.Pi { + lat = math.Remainder(lat, -2.0*math.Pi) + } + + if lat > math.Pi { + lat = math.Pi - lat + } + + if lat < -1.0*math.Pi { + lat = -1.0*math.Pi - lat + } + + if lat > math.Pi*2 { + lat = math.Pi - lat + } + + if lat < -1.0*math.Pi*2 { + lat = -1.0*math.Pi - lat + } + + return lat +} diff --git a/pkg/geo/geogfn/unary_operators_test.go b/pkg/geo/geogfn/unary_operators_test.go index 6149ad6762cb..a6918407ce32 100644 --- a/pkg/geo/geogfn/unary_operators_test.go +++ b/pkg/geo/geogfn/unary_operators_test.go @@ -15,7 +15,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/geo" + "github.com/golang/geo/s1" "github.com/stretchr/testify/require" + "github.com/twpayne/go-geom" ) type unaryOperatorExpectedResult struct { @@ -219,3 +221,35 @@ func TestLength(t *testing.T) { }) } } + +func TestProject(t *testing.T) { + var testCases = []struct { + desc string + point *geom.Point + distance float64 + azimuth float64 + projected *geom.Point + }{ + { + "POINT(0 0), 100000, radians(45)", + geom.NewPointFlat(geom.XY, []float64{0, 0}), + 100000, + 45 * math.Pi / 180.0, + geom.NewPointFlat(geom.XY, []float64{0.6352310291255374, 0.6394723347291977}), + }, + } + + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + projected, err := Project(tc.point, tc.distance, s1.Angle(tc.azimuth)) + require.NoError(t, err) + require.Equalf( + t, + tc.projected, + projected, + "expected %f, found %f", + &tc.projected, + projected) + }) + } +} diff --git a/pkg/geo/geographiclib/geographiclib.go b/pkg/geo/geographiclib/geographiclib.go index e1fc0a67eb34..c63ba65747c8 100644 --- a/pkg/geo/geographiclib/geographiclib.go +++ b/pkg/geo/geographiclib/geographiclib.go @@ -18,7 +18,12 @@ package geographiclib // #include "geographiclib.h" import "C" -import "github.com/golang/geo/s2" +import ( + "math" + + "github.com/golang/geo/s1" + "github.com/golang/geo/s2" +) var ( // WGS84Spheroid represents the default WGS84 ellipsoid. @@ -109,3 +114,23 @@ func (s *Spheroid) AreaAndPerimeter(points []s2.Point) (area float64, perimeter ) return float64(areaDouble), float64(perimeterDouble) } + +// Project returns computes the location of the projected point. +// +// Using the direct geodesic problem from GeographicLib (Karney 2013). +func (s *Spheroid) Project(point s2.LatLng, distance float64, azimuth s1.Angle) s2.LatLng { + var lat, lng C.double + + C.geod_direct( + &s.cRepr, + C.double(point.Lat.Degrees()), + C.double(point.Lng.Degrees()), + C.double(azimuth*180.0/math.Pi), + C.double(distance), + &lat, + &lng, + nil, + ) + + return s2.LatLngFromDegrees(float64(lat), float64(lng)) +} diff --git a/pkg/geo/geographiclib/geographiclib_test.go b/pkg/geo/geographiclib/geographiclib_test.go index 0d051b86365f..b383db8921e7 100644 --- a/pkg/geo/geographiclib/geographiclib_test.go +++ b/pkg/geo/geographiclib/geographiclib_test.go @@ -11,8 +11,10 @@ package geographiclib import ( + "math" "testing" + "github.com/golang/geo/s1" "github.com/golang/geo/s2" "github.com/stretchr/testify/require" ) @@ -101,3 +103,30 @@ func TestAreaAndPerimeter(t *testing.T) { }) } } + +func TestProject(t *testing.T) { + testCases := []struct { + desc string + spheroid Spheroid + point s2.LatLng + distance float64 + azimuth float64 + project s2.LatLng + }{ + { + desc: "{0,0} project to 100000, radians(45.0) on WGS84Spheroid", + spheroid: WGS84Spheroid, + point: s2.LatLng{Lat: 0, Lng: 0}, + distance: 100000, + azimuth: 45 * math.Pi / 180.0, + project: s2.LatLng{Lat: 0.011160897716439782, Lng: 0.011086872969072624}, + }, + } + + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + project := tc.spheroid.Project(tc.point, tc.distance, s1.Angle(tc.azimuth)) + require.Equal(t, tc.project, project) + }) + } +} diff --git a/pkg/sql/logictest/testdata/logic_test/geospatial b/pkg/sql/logictest/testdata/logic_test/geospatial index 75b829a5cb78..84923a4b83df 100644 --- a/pkg/sql/logictest/testdata/logic_test/geospatial +++ b/pkg/sql/logictest/testdata/logic_test/geospatial @@ -98,6 +98,11 @@ SELECT ST_AsText(p) FROM (VALUES POINT (1 2) POINT (3 4) +query T +SELECT ST_AsText(ST_Project('POINT(0 0)'::geography, 100000, radians(45.0))) +---- +POINT (0.6352310291255374 0.6394723347291977) + subtest cast_test query T diff --git a/pkg/sql/sem/builtins/geo_builtins.go b/pkg/sql/sem/builtins/geo_builtins.go index 6ed97ff883f1..f3e552952975 100644 --- a/pkg/sql/sem/builtins/geo_builtins.go +++ b/pkg/sql/sem/builtins/geo_builtins.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" + "github.com/golang/geo/s1" "github.com/twpayne/go-geom" "github.com/twpayne/go-geom/encoding/ewkb" ) @@ -854,6 +855,55 @@ var geoBuiltins = map[string]builtinDefinition{ tree.VolatilityImmutable, ), ), + "st_project": makeBuiltin( + defProps(), + tree.Overload{ + Types: tree.ArgTypes{ + {"geography", types.Geography}, + {"distance", types.Float}, + {"azimuth", types.Float}, + }, + ReturnType: tree.FixedReturnType(types.Geography), + Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + g := args[0].(*tree.DGeography) + distance := float64(*args[1].(*tree.DFloat)) + azimuth := float64(*args[2].(*tree.DFloat)) + + geomT, err := g.AsGeomT() + if err != nil { + return nil, err + } + + point, ok := geomT.(*geom.Point) + if !ok { + return nil, errors.Newf("ST_Project(geography) is only valid for point inputs") + } + + projected, err := geogfn.Project(point, distance, s1.Angle(azimuth)) + if err != nil { + return nil, err + } + + geog, err := geo.NewGeographyFromGeom(projected) + if err != nil { + return nil, err + } + + return &tree.DGeography{Geography: geog}, nil + }, + Info: infoBuilder{ + info: `Returns a point projected from a start point along a geodesic using a given distance and azimuth (bearing). +This is known as the direct geodesic problem. + +The distance is given in meters. Negative values are supported. + +The azimuth (also known as heading or bearing) is given in radians. It is measured clockwise from true north (azimuth zero). +East is azimuth π/2 (90 degrees); south is azimuth π (180 degrees); west is azimuth 3π/2 (270 degrees). +Negative azimuth values and values greater than 2π (360 degrees) are supported.`, + }.String(), + Volatility: tree.VolatilityImmutable, + }, + ), // // Unary functions. From 62bf6a2c21ea506002d35efbfb5e5b61139be500 Mon Sep 17 00:00:00 2001 From: richardjcai Date: Tue, 2 Jun 2020 23:39:33 -0400 Subject: [PATCH 28/46] sql: track fine-grained view dependencies Fixes #29021 Only create view dependencies on columns if the column is referenced in the view query. Release note (sql change): Views now only create a dependency on a table's column if the column is referenced in the view definition. Previously, all columns were added as a dependency meaning if a table was referenced in a view, all columns regardless of if the column was actually referenced would be added to the view's dependencies. --- .../logictest/testdata/logic_test/alter_table | 8 - .../testdata/logic_test/dependencies | 4 +- pkg/sql/logictest/testdata/logic_test/views | 30 +++ pkg/sql/opt/memo/expr_format.go | 11 +- pkg/sql/opt/optbuilder/builder.go | 15 ++ pkg/sql/opt/optbuilder/join.go | 4 + pkg/sql/opt/optbuilder/project.go | 6 +- pkg/sql/opt/optbuilder/scope.go | 8 +- pkg/sql/opt/optbuilder/select.go | 8 +- pkg/sql/opt/optbuilder/testdata/create_view | 188 ++++++++++++++++-- pkg/sql/opt/optbuilder/testdata/with | 4 +- pkg/sql/opt/view_dependencies.go | 30 ++- 12 files changed, 280 insertions(+), 36 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index 272473a8c93c..8ec588716509 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -321,14 +321,6 @@ ALTER TABLE t DROP COLUMN y statement error cannot drop column "e" because view "v" depends on it ALTER TABLE t DROP COLUMN e -# TODO(knz): this statement should succeed after #17269 is fixed. -statement error cannot drop column "d" because view "v" depends on it -ALTER TABLE t DROP COLUMN d - -# TODO(knz): remove the following once the test above succeeds. -statement ok -ALTER TABLE t DROP COLUMN d CASCADE - statement ok ALTER TABLE t DROP COLUMN e CASCADE diff --git a/pkg/sql/logictest/testdata/logic_test/dependencies b/pkg/sql/logictest/testdata/logic_test/dependencies index 699411f8a5de..bbf7cb6847cc 100644 --- a/pkg/sql/logictest/testdata/logic_test/dependencies +++ b/pkg/sql/logictest/testdata/logic_test/dependencies @@ -96,7 +96,7 @@ descriptor_id descriptor_name index_id dependedonby_id dependedonby_type de 53 test_kv NULL 54 fk NULL NULL NULL 53 test_kv NULL 55 fk NULL NULL NULL 53 test_kv NULL 56 fk NULL NULL NULL -53 test_kv NULL 59 view 0 NULL Columns: [1 2 3] +53 test_kv NULL 59 view 0 NULL Columns: [2] 53 test_kv 1 57 interleave 1 NULL SharedPrefixLen: 0 53 test_kv 1 58 interleave 2 NULL SharedPrefixLen: 0 59 test_v1 NULL 60 view 0 NULL Columns: [1] @@ -116,7 +116,7 @@ query ITIITITT colnames SELECT * FROM crdb_internal.forward_dependencies WHERE descriptor_name LIKE 'moretest_%' ORDER BY descriptor_id, index_id, dependedonby_type, dependedonby_id, dependedonby_index_id ---- descriptor_id descriptor_name index_id dependedonby_id dependedonby_type dependedonby_index_id dependedonby_name dependedonby_details -61 moretest_t NULL 62 view 0 NULL Columns: [1 2 3] +61 moretest_t NULL 62 view 0 NULL Columns: [2] # Check sequence dependencies. diff --git a/pkg/sql/logictest/testdata/logic_test/views b/pkg/sql/logictest/testdata/logic_test/views index 11a44d60b442..b846703e7087 100644 --- a/pkg/sql/logictest/testdata/logic_test/views +++ b/pkg/sql/logictest/testdata/logic_test/views @@ -688,3 +688,33 @@ statement ok CREATE OR REPLACE VIEW tview AS SELECT x AS x, x+1 AS x1, x+2 AS x2, x+3 AS x3 FROM t2 user root + +# Ensure a view that contains a table that is referenced multiple times with +# different column sets depends on the correct columns. +# Depended on columns should not be droppable. + +# Only column a should be depended on in this case. +statement ok +DROP TABLE ab CASCADE; +CREATE TABLE ab (a INT, b INT); +CREATE VIEW vab (x) AS SELECT ab.a FROM ab, ab AS ab2 + +statement ok +ALTER TABLE ab DROP COLUMN b + +statement error pq: cannot drop column "a" because view "vab" depends on it +ALTER TABLE ab DROP COLUMN a + +statement ok +CREATE TABLE abc (a INT, b INT, c INT); +CREATE VIEW vabc AS SELECT abc.a, abc2.b, abc3.c FROM abc, abc AS abc2, abc AS abc3 + +# All three columns a,b,c should not be droppable. +statement error pq: cannot drop column "a" because view "vabc" depends on it +ALTER TABLE abc DROP COLUMN a + +statement error pq: cannot drop column "b" because view "vabc" depends on it +ALTER TABLE abc DROP COLUMN b + +statement error pq: cannot drop column "c" because view "vabc" depends on it +ALTER TABLE abc DROP COLUMN c diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index d317beec2e8f..d9869798032b 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -552,8 +552,15 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) { if dep.SpecificIndex { fmt.Fprintf(f.Buffer, "@%s", dep.DataSource.(cat.Table).Index(dep.Index).Name()) } - if !dep.ColumnOrdinals.Empty() { - fmt.Fprintf(f.Buffer, " [columns: %s]", dep.ColumnOrdinals) + colNames, isTable := dep.GetColumnNames() + if len(colNames) > 0 { + fmt.Fprintf(f.Buffer, " [columns:") + for _, colName := range colNames { + fmt.Fprintf(f.Buffer, " %s", colName) + } + fmt.Fprintf(f.Buffer, "]") + } else if isTable { + fmt.Fprintf(f.Buffer, " [no columns]") } n.Child(f.Buffer.String()) } diff --git a/pkg/sql/opt/optbuilder/builder.go b/pkg/sql/opt/optbuilder/builder.go index 137066619bd4..8da993f8a41d 100644 --- a/pkg/sql/opt/optbuilder/builder.go +++ b/pkg/sql/opt/optbuilder/builder.go @@ -338,3 +338,18 @@ func (b *Builder) allocScope() *scope { r.builder = b return r } + +// trackReferencedColumnForViews is used to add a column to the view's +// dependencies. This should be called whenever a column reference is made in a +// view query. +func (b *Builder) trackReferencedColumnForViews(col *scopeColumn) { + if b.trackViewDeps { + for i := range b.viewDeps { + dep := b.viewDeps[i] + if ord, ok := dep.ColumnIDToOrd[col.id]; ok { + dep.ColumnOrdinals.Add(ord) + } + b.viewDeps[i] = dep + } + } +} diff --git a/pkg/sql/opt/optbuilder/join.go b/pkg/sql/opt/optbuilder/join.go index 1db24fca4c7e..901dfb1065a3 100644 --- a/pkg/sql/opt/optbuilder/join.go +++ b/pkg/sql/opt/optbuilder/join.go @@ -340,6 +340,8 @@ func (jb *usingJoinBuilder) buildUsingJoin(using *tree.UsingJoinCond) { jb.raiseUndefinedColError(name, "right") } + jb.b.trackReferencedColumnForViews(leftCol) + jb.b.trackReferencedColumnForViews(rightCol) jb.addEqualityCondition(leftCol, rightCol) } @@ -370,6 +372,8 @@ func (jb *usingJoinBuilder) buildNaturalJoin(natural tree.NaturalJoinCond) { rightCol := jb.findUsingColumn(jb.rightScope.cols, leftCol.name, "right table") if rightCol != nil { + jb.b.trackReferencedColumnForViews(leftCol) + jb.b.trackReferencedColumnForViews(rightCol) jb.addEqualityCondition(leftCol, rightCol) } } diff --git a/pkg/sql/opt/optbuilder/project.go b/pkg/sql/opt/optbuilder/project.go index 73e7e021d72e..4fc872d8d4cd 100644 --- a/pkg/sql/opt/optbuilder/project.go +++ b/pkg/sql/opt/optbuilder/project.go @@ -225,7 +225,9 @@ func (b *Builder) finishBuildScalar( } // Avoid synthesizing a new column if possible. - if col := outScope.findExistingCol(texpr, false /* allowSideEffects */); col != nil && col != outCol { + if col := outScope.findExistingCol( + texpr, false, /* allowSideEffects */ + ); col != nil && col != outCol { outCol.id = col.id outCol.scalar = scalar return scalar @@ -253,6 +255,8 @@ func (b *Builder) finishBuildScalar( func (b *Builder) finishBuildScalarRef( col *scopeColumn, inScope, outScope *scope, outCol *scopeColumn, colRefs *opt.ColSet, ) (out opt.ScalarExpr) { + + b.trackReferencedColumnForViews(col) // Update the sets of column references and outer columns if needed. if colRefs != nil { colRefs.Add(col.id) diff --git a/pkg/sql/opt/optbuilder/scope.go b/pkg/sql/opt/optbuilder/scope.go index c275c9282298..bd0340f7d076 100644 --- a/pkg/sql/opt/optbuilder/scope.go +++ b/pkg/sql/opt/optbuilder/scope.go @@ -574,8 +574,14 @@ func findExistingColInList( // findExistingCol finds the given expression among the bound variables in this // scope. Returns nil if the expression is not found (or an expression is found // but it has side-effects and allowSideEffects is false). +// If a column is found and we are tracking view dependencies, we add the column +// to the view dependencies since it means this column is being referenced. func (s *scope) findExistingCol(expr tree.TypedExpr, allowSideEffects bool) *scopeColumn { - return findExistingColInList(expr, s.cols, allowSideEffects) + col := findExistingColInList(expr, s.cols, allowSideEffects) + if col != nil { + s.builder.trackReferencedColumnForViews(col) + } + return col } // startAggFunc is called when the builder starts building an aggregate diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index eb1860b1710f..270374776dcb 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -530,8 +530,11 @@ func (b *Builder) buildScan( if b.trackViewDeps { dep := opt.ViewDep{DataSource: tab} - for i := 0; i < colCount; i++ { - dep.ColumnOrdinals.Add(getOrdinal(i)) + dep.ColumnIDToOrd = make(map[opt.ColumnID]int) + // We will track the ColumnID to Ord mapping so Ords can be added + // when a column is referenced. + for i, col := range outScope.cols { + dep.ColumnIDToOrd[col.id] = getOrdinal(i) } if private.Flags.ForceIndex { dep.SpecificIndex = true @@ -948,6 +951,7 @@ func (b *Builder) buildSelectClause( inScope *scope, ) (outScope *scope) { fromScope := b.buildFrom(sel.From, locking, inScope) + b.processWindowDefs(sel, fromScope) b.buildWhere(sel.Where, fromScope) diff --git a/pkg/sql/opt/optbuilder/testdata/create_view b/pkg/sql/opt/optbuilder/testdata/create_view index 6738aea2e07c..7ecd9e906a72 100644 --- a/pkg/sql/opt/optbuilder/testdata/create_view +++ b/pkg/sql/opt/optbuilder/testdata/create_view @@ -20,13 +20,13 @@ create-view t.public.v1 └── dependencies build -CREATE VIEW v1 AS SELECT a FROM ab +CREATE VIEW v1 AS SELECT a FROM ab ---- create-view t.public.v1 ├── SELECT a FROM t.public.ab ├── columns: a:1 └── dependencies - └── ab [columns: (0,1)] + └── ab [columns: a] # Test dependency on specific index. build @@ -36,7 +36,7 @@ create-view t.public.v1 ├── SELECT a FROM t.public.ab@idx ├── columns: a:1 └── dependencies - └── ab@idx [columns: (0,1)] + └── ab@idx [columns: a] build CREATE VIEW v1 AS SELECT a FROM ab@primary @@ -45,7 +45,7 @@ create-view t.public.v1 ├── SELECT a FROM t.public.ab@primary ├── columns: a:1 └── dependencies - └── ab@primary [columns: (0,1)] + └── ab@primary [columns: a] # Test dependency on view. exec-ddl @@ -53,13 +53,13 @@ CREATE VIEW av AS SELECT a FROM ab ---- build -CREATE VIEW v1 AS SELECT a FROM av +CREATE VIEW v1 AS SELECT a FROM av ---- create-view t.public.v1 ├── SELECT a FROM t.public.av ├── columns: a:1 └── dependencies - └── av [columns: (0)] + └── av build CREATE VIEW v1 AS SELECT av.a, ab.a FROM av, ab @@ -68,8 +68,8 @@ create-view t.public.v1 ├── SELECT av.a, ab.a FROM t.public.av, t.public.ab ├── columns: a:1 a:3 └── dependencies - ├── av [columns: (0)] - └── ab [columns: (0,1)] + ├── av + └── ab [columns: a] # Test that we don't report virtual table dependencies. build @@ -79,7 +79,7 @@ create-view t.public.v1 ├── SELECT a, table_schema FROM t.public.ab, "".information_schema.columns ├── columns: a:1 table_schema:5 └── dependencies - └── ab [columns: (0,1)] + └── ab [columns: a] # Test cases with specified column names. build @@ -89,9 +89,9 @@ create-view t.public.v2 ├── SELECT ab.a FROM t.public.ab, t.public.ab AS ab2, t.public.cd ├── columns: x:1 └── dependencies - ├── ab [columns: (0,1)] - ├── ab [columns: (0,1)] - └── cd [columns: (0,1)] + ├── ab [columns: a] + ├── ab [no columns] + └── cd [no columns] build CREATE VIEW v3 (x, y) AS SELECT a FROM ab @@ -116,7 +116,7 @@ create-view t.public.v5 ├── SELECT a FROM [53 AS t] ├── columns: a:1 └── dependencies - └── ab [columns: (0,1)] + └── ab [columns: a] # Verify that we only depend on the specified column. build @@ -126,7 +126,7 @@ create-view t.public.v6 ├── SELECT a FROM [53(1) AS t] ├── columns: a:1 └── dependencies - └── ab [columns: (0)] + └── ab [columns: a] # Verify dependency on sequence. build @@ -148,7 +148,7 @@ create-view t.public.v8 ├── WITH cd AS (SELECT a, b FROM t.public.ab) SELECT a + b FROM cd ├── columns: "?column?":5 └── dependencies - └── ab [columns: (0,1)] + └── ab [columns: a b] # Verify that we disallow mutation statements. build @@ -165,3 +165,161 @@ build CREATE VIEW v9 AS SELECT a,b FROM [DELETE FROM ab WHERE a>b RETURNING a, b] ---- error (42601): DELETE cannot be used inside a view definition + +# Regression 29021. + +# Dependencies should be tracked in the group by clause. +build +CREATE VIEW v10 AS SELECT a FROM ab GROUP BY a,b +---- +create-view t.public.v10 + ├── SELECT a FROM t.public.ab GROUP BY a, b + ├── columns: a:1 + └── dependencies + └── ab [columns: a b] + +# Dependencies should be tracked in the join on clause. +build +CREATE VIEW v10 as SELECT 1 FROM ab JOIN cd ON ab.a = cd.c +---- +create-view t.public.v10 + ├── SELECT 1 FROM t.public.ab JOIN t.public.cd ON ab.a = cd.c + ├── columns: "?column?":5 + └── dependencies + ├── ab [columns: a] + └── cd [columns: c] + +exec-ddl +CREATE TABLE ac (a INT, c INT) +---- + +# Dependencies should be tracked in a natural join clause. +build +CREATE VIEW v11 as SELECT 1 FROM ab NATURAL JOIN ac +---- +create-view t.public.v11 + ├── SELECT 1 FROM t.public.ab NATURAL JOIN t.public.ac + ├── columns: "?column?":6 + └── dependencies + ├── ab [columns: a] + └── ac [columns: a] + +# Dependencies should be tracked in a using join clause. +build +CREATE VIEW v12 as SELECT 1 FROM ab JOIN ac USING (a) +---- +create-view t.public.v12 + ├── SELECT 1 FROM t.public.ab JOIN t.public.ac USING (a) + ├── columns: "?column?":6 + └── dependencies + ├── ab [columns: a] + └── ac [columns: a] + +# Dependencies should be tracked in the where clause. +build +CREATE VIEW v13 AS SELECT a FROM ab WHERE b > 0 +---- +create-view t.public.v13 + ├── SELECT a FROM t.public.ab WHERE b > 0 + ├── columns: a:1 + └── dependencies + └── ab [columns: a b] + +# Dependencies should be tracked in aggregate / window functions. +build +CREATE VIEW v14 AS SELECT sum(a) FROM ab; +---- +create-view t.public.v14 + ├── SELECT sum(a) FROM t.public.ab + ├── columns: sum:3 + └── dependencies + └── ab [columns: a] + +# Dependencies should be tracked in partitions. +build +CREATE VIEW v15 AS SELECT sum(a) OVER (PARTITION by b) FROM ab; +---- +create-view t.public.v15 + ├── SELECT sum(a) OVER (PARTITION BY b) FROM t.public.ab + ├── columns: sum:3 + └── dependencies + └── ab [columns: a b] + +# Dependencies should be tracked in subqueries. +build +CREATE VIEW v16 AS SELECT a FROM (SELECT a,b FROM ab); +---- +create-view t.public.v16 + ├── SELECT a FROM (SELECT a, b FROM t.public.ab) + ├── columns: a:1 + └── dependencies + └── ab [columns: a b] + +# Dependencies should be tracked in the order by clause. +build +CREATE VIEW v16 AS SELECT a FROM ab ORDER BY b +---- +create-view t.public.v16 + ├── SELECT a FROM t.public.ab ORDER BY b + ├── columns: a:1 + └── dependencies + └── ab [columns: a b] + +exec-ddl +CREATE TABLE tf (f FLOAT) +---- + +# Dependencies should be tracked in ordered-set aggregate functions. +build +CREATE VIEW v17 AS SELECT percentile_cont(0.50) WITHIN GROUP (ORDER BY f) FROM tf +---- +create-view t.public.v17 + ├── SELECT percentile_cont(0.50) WITHIN GROUP (ORDER BY f) FROM t.public.tf + ├── columns: percentile_cont:4 + └── dependencies + └── tf [columns: f] + +# Dependencies should be tracked with multiple table statements. +build +CREATE VIEW v18 AS SELECT ab.a, ab2.b FROM ab, ab as ab2 +---- +create-view t.public.v18 + ├── SELECT ab.a, ab2.b FROM t.public.ab, t.public.ab AS ab2 + ├── columns: a:1 b:4 + └── dependencies + ├── ab [columns: a] + └── ab [columns: b] + +build +CREATE VIEW v19 AS SELECT 1 FROM (SELECT a FROM ab) t1 JOIN (SELECT b FROM AB) t2 on t1.a = t2.b +---- +create-view t.public.v19 + ├── SELECT 1 FROM (SELECT a FROM t.public.ab) AS t1 JOIN (SELECT b FROM t.public.ab) AS t2 ON t1.a = t2.b + ├── columns: "?column?":5 + └── dependencies + ├── ab [columns: a] + └── ab [columns: b] + +# Dependencies should be tracked if the column is used in a projection. +build +CREATE VIEW v20 AS SELECT a + b FROM ab +---- +create-view t.public.v20 + ├── SELECT a + b FROM t.public.ab + ├── columns: "?column?":3 + └── dependencies + └── ab [columns: a b] + +exec-ddl +CREATE TABLE abc (a INT, b INT, c INT) +---- + +# Dependencies should be tracked in an ORDER BY inside a partition. +build +CREATE VIEW v21 AS SELECT sum(a) OVER (PARTITION BY b ORDER BY c) FROM abc +---- +create-view t.public.v21 + ├── SELECT sum(a) OVER (PARTITION BY b ORDER BY c) FROM t.public.abc + ├── columns: sum:5 + └── dependencies + └── abc [columns: a b c] diff --git a/pkg/sql/opt/optbuilder/testdata/with b/pkg/sql/opt/optbuilder/testdata/with index aa378d742459..993def0ee9a9 100644 --- a/pkg/sql/opt/optbuilder/testdata/with +++ b/pkg/sql/opt/optbuilder/testdata/with @@ -93,8 +93,8 @@ create-view t.public.v1 ├── WITH t AS (SELECT a FROM t.public.y WHERE a < 3) SELECT 1 FROM t.public.x NATURAL JOIN t ├── columns: "?column?":7 └── dependencies - ├── y [columns: (0,1)] - └── x [columns: (0-2)] + ├── y [columns: a] + └── x [columns: a] build CREATE TABLE t1 AS diff --git a/pkg/sql/opt/view_dependencies.go b/pkg/sql/opt/view_dependencies.go index a3254de9201f..9003e9d57abd 100644 --- a/pkg/sql/opt/view_dependencies.go +++ b/pkg/sql/opt/view_dependencies.go @@ -11,6 +11,8 @@ package opt import ( + "sort" + "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/util" ) @@ -23,13 +25,35 @@ type ViewDep struct { DataSource cat.DataSource // ColumnOrdinals is the set of column ordinals that are referenced by the - // view for this table. In most cases, this consists of all "public" columns - // of the table; the only exception is when a table is referenced by table ID - // with a specific list of column IDs. + // view for this table. ColumnOrdinals util.FastIntSet + // ColumnIDToOrd maps a scopeColumn's ColumnID to its ColumnOrdinal. + // This helps us add only the columns that are actually referenced + // by the view's query into the view dependencies. We add a + // dependency on a column only when the column is referenced by the view + // and created as a scopeColumn. + ColumnIDToOrd map[ColumnID]int + // If an index is referenced specifically (via an index hint), SpecificIndex // is true and Index is the ordinal of that index. SpecificIndex bool Index cat.IndexOrdinal } + +// GetColumnNames returns a sorted list of the names of the column dependencies +// and a boolean to determine if the dependency was a table. +// We only track column dependencies on tables. +func (dep ViewDep) GetColumnNames() ([]string, bool) { + colNames := make([]string, 0) + if table, ok := dep.DataSource.(cat.Table); ok { + dep.ColumnOrdinals.ForEach(func(i int) { + name := table.Column(i).ColName() + colNames = append(colNames, name.String()) + }) + sort.Strings(colNames) + return colNames, ok + } + + return nil, false +} From 39ba63eaba4d91fb54fa9313c89c2e6c4db922ad Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Mon, 8 Jun 2020 14:50:18 -0400 Subject: [PATCH 29/46] importccl: Correctly handle errors and cancellations during import. Fixes #49977 Parallel importer could get stuck due to a race between emitted import batches and checking for context cancellation (either due to an unforeseen error, or due to explicit context cancallation). Fix the race condition, and add tests verifying correct behavior. Release notes (bug fix): correctly handle import cancellation and errors. --- pkg/ccl/importccl/read_import_base.go | 23 ++-- pkg/ccl/importccl/read_import_base_test.go | 145 +++++++++++++++++++++ 2 files changed, 158 insertions(+), 10 deletions(-) diff --git a/pkg/ccl/importccl/read_import_base.go b/pkg/ccl/importccl/read_import_base.go index 162614ee7ddc..07bb9059dd6a 100644 --- a/pkg/ccl/importccl/read_import_base.go +++ b/pkg/ccl/importccl/read_import_base.go @@ -395,7 +395,7 @@ type importFileContext struct { // handleCorruptRow reports an error encountered while processing a row // in an input file. func handleCorruptRow(ctx context.Context, fileCtx *importFileContext, err error) error { - log.Errorf(ctx, "%v", err) + log.Errorf(ctx, "%+v", err) if rowErr := (*importRowError)(nil); errors.As(err, &rowErr) && fileCtx.rejected != nil { fileCtx.rejected <- rowErr.row + "\n" @@ -545,7 +545,7 @@ func runParallelImport( } if producer.Err() == nil { - return importer.flush(ctx) + return importer.close(ctx) } return producer.Err() }) @@ -569,22 +569,25 @@ func (p *parallelImporter) add( return nil } -// Flush flushes currently accumulated data. +// close closes this importer, flushing remaining accumulated data if needed. +func (p *parallelImporter) close(ctx context.Context) error { + if len(p.b.data) > 0 { + return p.flush(ctx) + } + return nil +} + +// flush flushes currently accumulated data. func (p *parallelImporter) flush(ctx context.Context) error { select { case <-ctx.Done(): return ctx.Err() - default: - } - - // if the batch isn't empty, we need to flush it. - if len(p.b.data) > 0 { - p.recordCh <- p.b + case p.recordCh <- p.b: p.b = batch{ data: make([]interface{}, 0, cap(p.b.data)), } + return nil } - return nil } func (p *parallelImporter) importWorker( diff --git a/pkg/ccl/importccl/read_import_base_test.go b/pkg/ccl/importccl/read_import_base_test.go index 6687ec8d1be8..05123beef311 100644 --- a/pkg/ccl/importccl/read_import_base_test.go +++ b/pkg/ccl/importccl/read_import_base_test.go @@ -9,9 +9,19 @@ package importccl import ( + "context" + "math/rand" "testing" + "time" + "github.com/cockroachdb/cockroach/pkg/sql/row" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" ) func TestRejectedFilename(t *testing.T) { @@ -42,3 +52,138 @@ func TestRejectedFilename(t *testing.T) { } } } + +// nilDataProducer produces infinite stream of nulls. +// It implements importRowProducer. +type nilDataProducer struct{} + +func (p *nilDataProducer) Scan() bool { + return true +} + +func (p *nilDataProducer) Err() error { + return nil +} + +func (p *nilDataProducer) Skip() error { + return nil +} + +func (p *nilDataProducer) Row() (interface{}, error) { + return nil, nil +} + +func (p *nilDataProducer) Progress() float32 { + return 0.0 +} + +var _ importRowProducer = &nilDataProducer{} + +// errorReturningConsumer always returns an error. +// It implements importRowConsumer. +type errorReturningConsumer struct { + err error +} + +func (d *errorReturningConsumer) FillDatums( + _ interface{}, _ int64, c *row.DatumRowConverter, +) error { + return d.err +} + +var _ importRowConsumer = &errorReturningConsumer{} + +// nilDataConsumer consumes and emits infinite stream of null. +// it implements importRowConsumer. +type nilDataConsumer struct{} + +func (n *nilDataConsumer) FillDatums(_ interface{}, _ int64, c *row.DatumRowConverter) error { + c.Datums[0] = tree.DNull + return nil +} + +var _ importRowConsumer = &nilDataConsumer{} + +func TestParallelImportProducerHandlesConsumerErrors(t *testing.T) { + defer leaktest.AfterTest(t)() + + // Dummy descriptor for import + descr := sqlbase.TableDescriptor{ + Name: "test", + Columns: []sqlbase.ColumnDescriptor{ + {Name: "column", ID: 1, Type: types.Int, Nullable: true}, + }, + } + + // Flush datum converter frequently + defer row.TestingSetDatumRowConverterBatchSize(1)() + + // Create KV channel and arrange for it to be drained + kvCh := make(chan row.KVBatch) + defer close(kvCh) + go func() { + for range kvCh { + } + }() + + // Prepare import context, which flushes to kvCh frequently. + importCtx := ¶llelImportContext{ + numWorkers: 1, + batchSize: 2, + evalCtx: testEvalCtx, + tableDesc: &descr, + kvCh: kvCh, + } + + consumer := &errorReturningConsumer{errors.New("consumer aborted")} + + require.Equal(t, consumer.err, + runParallelImport(context.Background(), importCtx, + &importFileContext{}, &nilDataProducer{}, consumer)) +} + +func TestParallelImportProducerHandlesCancellation(t *testing.T) { + defer leaktest.AfterTest(t)() + + // Dummy descriptor for import + descr := sqlbase.TableDescriptor{ + Name: "test", + Columns: []sqlbase.ColumnDescriptor{ + {Name: "column", ID: 1, Type: types.Int, Nullable: true}, + }, + } + + // Flush datum converter frequently + defer row.TestingSetDatumRowConverterBatchSize(1)() + + // Create KV channel and arrange for it to be drained + kvCh := make(chan row.KVBatch) + defer close(kvCh) + go func() { + for range kvCh { + } + }() + + // Prepare import context, which flushes to kvCh frequently. + importCtx := ¶llelImportContext{ + numWorkers: 1, + batchSize: 2, + evalCtx: testEvalCtx, + tableDesc: &descr, + kvCh: kvCh, + } + + // Run a hundred imports, which will timeout shortly after they start. + require.NoError(t, ctxgroup.GroupWorkers(context.Background(), 100, + func(_ context.Context, _ int) error { + timeout := time.Millisecond * time.Duration(250+rand.Intn(250)) + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer func(f func()) { + f() + }(cancel) + require.Equal(t, context.DeadlineExceeded, + runParallelImport(ctx, importCtx, + &importFileContext{}, &nilDataProducer{}, &nilDataConsumer{})) + return nil + })) +} From 394d093a06764c9c151695ede2bcab0f7eaa5962 Mon Sep 17 00:00:00 2001 From: Jordan Lewis Date: Sat, 6 Jun 2020 23:04:20 -0400 Subject: [PATCH 30/46] *: s/whitelist/allowlist/, s/blacklist/blocklist/ Allow and block aren't hurtful terms, and they also happen to be clearer than white and black - they connote meaning instantly. c.f.: https://twitter.com/bradfitz/status/1269449722063773696?s=20 https://github.com/rails/rails/issues/33677 https://gitlab.com/gitlab-org/gitlab/-/issues/7554 https://github.com/lagom/lagom/issues/2532 https://bugs.chromium.org/p/chromium/issues/detail?id=981129 Release note: None --- docs/RFCS/20170628_web_session_login.md | 2 +- docs/RFCS/20190318_error_handling.md | 2 +- pkg/ccl/partitionccl/partition_test.go | 2 +- pkg/cli/cert.go | 2 +- pkg/cli/zip_test.go | 2 +- pkg/cmd/roachprod/install/cassandra_yaml.go | 2 +- pkg/cmd/roachtest/bank.go | 4 +- .../{blacklist_test.go => blocklist_test.go} | 30 ++++++------- pkg/cmd/roachtest/canary.go | 32 +++++++------- pkg/cmd/roachtest/django.go | 10 ++--- ...jango_blacklist.go => django_blocklist.go} | 20 ++++----- pkg/cmd/roachtest/gopg.go | 16 +++---- .../{gopg_blacklist.go => gopg_blocklist.go} | 18 ++++---- pkg/cmd/roachtest/hibernate.go | 8 ++-- ...te_blacklist.go => hibernate_blocklist.go} | 30 ++++++------- pkg/cmd/roachtest/java_helpers.go | 12 +++--- pkg/cmd/roachtest/libpq.go | 8 ++-- ...{libpq_blacklist.go => libpq_blocklist.go} | 16 +++---- pkg/cmd/roachtest/orm_helpers.go | 20 ++++----- pkg/cmd/roachtest/pgjdbc.go | 12 +++--- ...gjdbc_blacklist.go => pgjdbc_blocklist.go} | 30 ++++++------- pkg/cmd/roachtest/pgx.go | 14 +++---- .../{pgx_blacklist.go => pgx_blocklist.go} | 20 ++++----- pkg/cmd/roachtest/psycopg.go | 10 ++--- ...copg_blacklist.go => psycopg_blocklist.go} | 24 +++++------ pkg/cmd/roachtest/python_helpers.go | 2 +- pkg/cmd/roachtest/sqlalchemy.go | 10 ++--- ...y_blacklist.go => sqlalchemy_blocklist.go} | 20 ++++----- pkg/kv/kvserver/client_split_test.go | 12 +++--- pkg/kv/kvserver/replica_application_result.go | 26 ++++++------ pkg/kv/kvserver/replica_command.go | 4 +- pkg/kv/kvserver/replica_evaluate.go | 2 +- pkg/kv/kvserver/replica_proposal.go | 2 +- pkg/kv/kvserver/replica_read.go | 2 +- pkg/kv/kvserver/track_raft_protos.go | 10 ++--- pkg/server/authentication_test.go | 2 +- pkg/server/server_sql_test.go | 2 +- pkg/sql/create_role.go | 10 ++--- pkg/sql/distsql_physical_planner.go | 6 +-- pkg/sql/logictest/logic.go | 42 +++++++++---------- pkg/sql/opt/memo/expr.go | 2 +- pkg/sql/opt/norm/fold_constants_funcs.go | 10 ++--- pkg/sql/opt/optbuilder/builder.go | 2 +- pkg/sql/parser/scan.go | 4 +- pkg/sql/schema_changer.go | 10 ++--- pkg/sql/schema_changer_test.go | 4 +- pkg/sql/sem/builtins/builtins.go | 12 +++--- pkg/sql/sem/builtins/pg_builtins.go | 16 +++---- pkg/sql/sem/tree/expr.go | 6 +-- pkg/sql/sem/tree/function_definition.go | 6 +-- pkg/sql/telemetry_test.go | 30 ++++++------- pkg/sql/testdata/telemetry/error | 2 +- pkg/sql/testdata/telemetry/planning | 14 +++---- pkg/sql/testdata/telemetry/schema | 2 +- pkg/testutils/buildutil/build.go | 18 ++++---- pkg/workload/cli/cli.go | 6 +-- pkg/workload/dep_test.go | 4 +- 57 files changed, 323 insertions(+), 323 deletions(-) rename pkg/cmd/roachtest/{blacklist_test.go => blocklist_test.go} (79%) rename pkg/cmd/roachtest/{django_blacklist.go => django_blocklist.go} (94%) rename pkg/cmd/roachtest/{gopg_blacklist.go => gopg_blocklist.go} (93%) rename pkg/cmd/roachtest/{hibernate_blacklist.go => hibernate_blocklist.go} (99%) rename pkg/cmd/roachtest/{libpq_blacklist.go => libpq_blocklist.go} (92%) rename pkg/cmd/roachtest/{pgjdbc_blacklist.go => pgjdbc_blocklist.go} (99%) rename pkg/cmd/roachtest/{pgx_blacklist.go => pgx_blocklist.go} (94%) rename pkg/cmd/roachtest/{psycopg_blacklist.go => psycopg_blocklist.go} (99%) rename pkg/cmd/roachtest/{sqlalchemy_blacklist.go => sqlalchemy_blocklist.go} (97%) diff --git a/docs/RFCS/20170628_web_session_login.md b/docs/RFCS/20170628_web_session_login.md index 3ab394837318..e722dc1c2073 100644 --- a/docs/RFCS/20170628_web_session_login.md +++ b/docs/RFCS/20170628_web_session_login.md @@ -365,7 +365,7 @@ for incoming requests, but instead would simply need to verify the signature on the token. The major issue with JWT is that it does not provide a way to revoke login -sessions; to do this, we would need to store a blacklist of revoked session IDs, +sessions; to do this, we would need to store a blocklist of revoked session IDs, which removes much of the advantage of not having the sessions table in the first place. diff --git a/docs/RFCS/20190318_error_handling.md b/docs/RFCS/20190318_error_handling.md index f5a3bf06bbd9..83ff3b96548c 100644 --- a/docs/RFCS/20190318_error_handling.md +++ b/docs/RFCS/20190318_error_handling.md @@ -2096,7 +2096,7 @@ To achieve this, the library introduce *error domains*, which are computed attri its domain and preserves its message, structure, etc. - domains are preserved across the network. - a function `EnsureNotInDomain()` (described below) makes it possible to block - errors from one or more “blacklist” domains from escaping an API boundary, + errors from one or more “blocklist” domains from escaping an API boundary, or conditionally transmute them into appropriate substitute errors, in particular [barriers](#barriers-Error-barriers). diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index 84ca48091c0f..09dc05af611e 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -74,7 +74,7 @@ type partitioningTest struct { // scans are each a shorthand for an assertion of where data should live. // The map key is the used for the `WHERE` clause of a `SELECT *` and the - // value is a comma separated whitelist of nodes that are allowed to serve + // value is a comma separated allowlist of nodes that are allowed to serve // this query. Example: `map[string]string{`b = 1`: `n2`}` means that // `SELECT * FROM t WHERE b = 1` is required to be served entirely by node2. // diff --git a/pkg/cli/cert.go b/pkg/cli/cert.go index ef054f3ced2a..2997d3f83f7c 100644 --- a/pkg/cli/cert.go +++ b/pkg/cli/cert.go @@ -173,7 +173,7 @@ func runCreateClientCert(cmd *cobra.Command, args []string) error { var err error var username string // We intentionally allow the `node` user to have a cert. - if username, err = sql.NormalizeAndValidateUsernameNoBlacklist(args[0]); err != nil { + if username, err = sql.NormalizeAndValidateUsernameNoBlocklist(args[0]); err != nil { return errors.Wrap(err, "failed to generate client certificate and key") } diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 45f8b5fc4fde..3e6489e22c04 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -65,7 +65,7 @@ SELECT concat('crdb_internal.', table_name) as name FROM [ SELECT table_name FROM [ SHOW TABLES FROM crdb_internal ] ] WHERE table_name NOT IN ( - -- whitelisted tables that don't need to be in debug zip + -- allowlisted tables that don't need to be in debug zip 'backward_dependencies', 'builtin_functions', 'create_statements', diff --git a/pkg/cmd/roachprod/install/cassandra_yaml.go b/pkg/cmd/roachprod/install/cassandra_yaml.go index d14346f61cf6..1676bd8c87b6 100644 --- a/pkg/cmd/roachprod/install/cassandra_yaml.go +++ b/pkg/cmd/roachprod/install/cassandra_yaml.go @@ -88,7 +88,7 @@ num_tokens: 256 # May either be "true" or "false" to enable globally hinted_handoff_enabled: true -# When hinted_handoff_enabled is true, a black list of data centers that will not +# When hinted_handoff_enabled is true, a blocklist of data centers that will not # perform hinted handoff # hinted_handoff_disabled_datacenters: # - DC1 diff --git a/pkg/cmd/roachtest/bank.go b/pkg/cmd/roachtest/bank.go index a817446daadb..89ae1310de41 100644 --- a/pkg/cmd/roachtest/bank.go +++ b/pkg/cmd/roachtest/bank.go @@ -336,7 +336,7 @@ func isExpectedRelocateError(err error) bool { // for more failure modes not caught here. We decided to avoid adding // to this catchall and to fix the root causes instead. // We've also seen "breaker open" errors here. - whitelist := []string{ + allowlist := []string{ "descriptor changed", "unable to remove replica .* which is not present", "unable to add replica .* which is already present", @@ -344,7 +344,7 @@ func isExpectedRelocateError(err error) bool { "failed to apply snapshot: raft group deleted", "snapshot failed:", } - pattern := "(" + strings.Join(whitelist, "|") + ")" + pattern := "(" + strings.Join(allowlist, "|") + ")" return testutils.IsError(err, pattern) } diff --git a/pkg/cmd/roachtest/blacklist_test.go b/pkg/cmd/roachtest/blocklist_test.go similarity index 79% rename from pkg/cmd/roachtest/blacklist_test.go rename to pkg/cmd/roachtest/blocklist_test.go index 1fad6dec5796..8cf6e35e59ac 100644 --- a/pkg/cmd/roachtest/blacklist_test.go +++ b/pkg/cmd/roachtest/blocklist_test.go @@ -24,22 +24,22 @@ import ( ) const githubAPITokenEnv = "GITHUB_API_TOKEN" -const runBlacklistEnv = "RUN_BLACKLIST_TEST" +const runBlocklistEnv = "RUN_BLOCKLIST_TEST" -func TestBlacklists(t *testing.T) { - if _, ok := os.LookupEnv(runBlacklistEnv); !ok { - t.Skipf("Blackist test is only run if %s is set", runBlacklistEnv) +func TestBlocklists(t *testing.T) { + if _, ok := os.LookupEnv(runBlocklistEnv); !ok { + t.Skipf("Blocklist test is only run if %s is set", runBlocklistEnv) } - blacklists := map[string]blacklist{ - "hibernate": hibernateBlackList20_1, - "pgjdbc": pgjdbcBlackList20_1, - "psycopg": psycopgBlackList20_1, - "django": djangoBlacklist20_1, - "sqlAlchemy": sqlAlchemyBlacklist20_1, - "libpq": libPQBlacklist20_1, - "gopg": gopgBlackList20_1, - "pgx": pgxBlacklist20_1, + blocklists := map[string]blocklist{ + "hibernate": hibernateBlockList20_1, + "pgjdbc": pgjdbcBlockList20_1, + "psycopg": psycopgBlockList20_1, + "django": djangoBlocklist20_1, + "sqlAlchemy": sqlAlchemyBlocklist20_1, + "libpq": libPQBlocklist20_1, + "gopg": gopgBlockList20_1, + "pgx": pgxBlocklist20_1, } type reasonCount struct { reason string @@ -48,7 +48,7 @@ func TestBlacklists(t *testing.T) { } var failureMap = make(map[string]*reasonCount, 200) - for suite, bl := range blacklists { + for suite, bl := range blocklists { for _, reason := range bl { if _, ok := failureMap[reason]; !ok { failureMap[reason] = &reasonCount{ @@ -106,6 +106,6 @@ func TestBlacklists(t *testing.T) { } if anyClosed { - t.Fatal("Some closed issues appear in blacklists") + t.Fatal("Some closed issues appear in blocklists") } } diff --git a/pkg/cmd/roachtest/canary.go b/pkg/cmd/roachtest/canary.go index 8c0e823b7a09..350e0bb9cc5e 100644 --- a/pkg/cmd/roachtest/canary.go +++ b/pkg/cmd/roachtest/canary.go @@ -28,35 +28,35 @@ import ( // TODO(bram): There are more common elements between all the canary tests, // factor more of them into here. -// blacklist is a lists of known test errors and failures. -type blacklist map[string]string +// blocklist is a lists of known test errors and failures. +type blocklist map[string]string -// blacklistForVersion contains both a blacklist of known test errors and +// blocklistForVersion contains both a blocklist of known test errors and // failures but also an optional ignorelist for flaky tests. // When the test suite is run, the results are compared to this list. -// Any passed test that is not on this blacklist is reported as PASS - expected -// Any passed test that is on this blacklist is reported as PASS - unexpected -// Any failed test that is on this blacklist is reported as FAIL - expected -// Any failed test that is not on blackthis list is reported as FAIL - unexpected -// Any test on this blacklist that is not run is reported as FAIL - not run +// Any passed test that is not on this blocklist is reported as PASS - expected +// Any passed test that is on this blocklist is reported as PASS - unexpected +// Any failed test that is on this blocklist is reported as FAIL - expected +// Any failed test that is not on blocklist list is reported as FAIL - unexpected +// Any test on this blocklist that is not run is reported as FAIL - not run // Ant test in the ignorelist is reported as SKIP if it is run -type blacklistForVersion struct { +type blocklistForVersion struct { versionPrefix string - blacklistname string - blacklist blacklist + blocklistname string + blocklist blocklist ignorelistname string - ignorelist blacklist + ignorelist blocklist } -type blacklistsForVersion []blacklistForVersion +type blocklistsForVersion []blocklistForVersion -// getLists returns the appropriate blacklist and ignorelist based on the +// getLists returns the appropriate blocklist and ignorelist based on the // cockroach version. This check only looks to ensure that the prefix that // matches. -func (b blacklistsForVersion) getLists(version string) (string, blacklist, string, blacklist) { +func (b blocklistsForVersion) getLists(version string) (string, blocklist, string, blocklist) { for _, info := range b { if strings.HasPrefix(version, info.versionPrefix) { - return info.blacklistname, info.blacklist, info.ignorelistname, info.ignorelist + return info.blocklistname, info.blocklist, info.ignorelistname, info.ignorelist } } return "", nil, "", nil diff --git a/pkg/cmd/roachtest/django.go b/pkg/cmd/roachtest/django.go index 67895192558a..4d5d4746d260 100644 --- a/pkg/cmd/roachtest/django.go +++ b/pkg/cmd/roachtest/django.go @@ -165,15 +165,15 @@ func registerDjango(r *testRegistry) { t.Fatal(err) } - blacklistName, expectedFailureList, ignoredlistName, ignoredlist := djangoBlacklists.getLists(version) + blocklistName, expectedFailureList, ignoredlistName, ignoredlist := djangoBlocklists.getLists(version) if expectedFailureList == nil { - t.Fatalf("No django blacklist defined for cockroach version %s", version) + t.Fatalf("No django blocklist defined for cockroach version %s", version) } if ignoredlist == nil { t.Fatalf("No django ignorelist defined for cockroach version %s", version) } - c.l.Printf("Running cockroach version %s, using blacklist %s, using ignoredlist %s", - version, blacklistName, ignoredlistName) + c.l.Printf("Running cockroach version %s, using blocklist %s, using ignoredlist %s", + version, blocklistName, ignoredlistName) // TODO (rohany): move this to a file backed buffer if the output becomes // too large. @@ -197,7 +197,7 @@ func registerDjango(r *testRegistry) { results := newORMTestsResults() results.parsePythonUnitTestOutput(fullTestResults, expectedFailureList, ignoredlist) results.summarizeAll( - t, "django" /* ormName */, blacklistName, + t, "django" /* ormName */, blocklistName, expectedFailureList, version, djangoLatestTag, ) } diff --git a/pkg/cmd/roachtest/django_blacklist.go b/pkg/cmd/roachtest/django_blocklist.go similarity index 94% rename from pkg/cmd/roachtest/django_blacklist.go rename to pkg/cmd/roachtest/django_blocklist.go index 26790e01ad39..bd465f57d101 100644 --- a/pkg/cmd/roachtest/django_blacklist.go +++ b/pkg/cmd/roachtest/django_blocklist.go @@ -165,18 +165,18 @@ var enabledDjangoTests = []string{ "view_tests", } -var djangoBlacklists = blacklistsForVersion{ - {"v19.2", "djangoBlacklist19_2", djangoBlacklist19_2, "djangoIgnoreList19_2", djangoIgnoreList19_2}, - {"v20.1", "djangoBlacklist20_1", djangoBlacklist20_1, "djangoIgnoreList20_1", djangoIgnoreList20_1}, - {"v20.2", "djangoBlacklist20_2", djangoBlacklist20_2, "djangoIgnoreList20_2", djangoIgnoreList20_2}, +var djangoBlocklists = blocklistsForVersion{ + {"v19.2", "djangoBlocklist19_2", djangoBlocklist19_2, "djangoIgnoreList19_2", djangoIgnoreList19_2}, + {"v20.1", "djangoBlocklist20_1", djangoBlocklist20_1, "djangoIgnoreList20_1", djangoIgnoreList20_1}, + {"v20.2", "djangoBlocklist20_2", djangoBlocklist20_2, "djangoIgnoreList20_2", djangoIgnoreList20_2}, } // Maintain that this list is alphabetized. -var djangoBlacklist20_2 = blacklist{} +var djangoBlocklist20_2 = blocklist{} -var djangoBlacklist20_1 = blacklist{} +var djangoBlocklist20_1 = blocklist{} -var djangoBlacklist19_2 = blacklist{ +var djangoBlocklist19_2 = blocklist{ "admin_views.tests.AdminViewBasicTest.test_date_hierarchy_timezone_dst": "unknown", "admin_views.tests.SecureViewTests.test_secure_view_shows_login_if_not_logged_in": "unknown", "admin_views.tests.SecureViewTests.test_staff_member_required_decorator_works_with_argument": "unknown", @@ -218,8 +218,8 @@ var djangoBlacklist19_2 = blacklist{ //"postgres_tests.test_array.TestOtherTypesExactQuerying.test_exact_decimals": "23468", } -var djangoIgnoreList20_2 = blacklist{} +var djangoIgnoreList20_2 = blocklist{} -var djangoIgnoreList20_1 = blacklist{} +var djangoIgnoreList20_1 = blocklist{} -var djangoIgnoreList19_2 = blacklist{} +var djangoIgnoreList19_2 = blocklist{} diff --git a/pkg/cmd/roachtest/gopg.go b/pkg/cmd/roachtest/gopg.go index a2ab7c975bdf..f40c4312d9b4 100644 --- a/pkg/cmd/roachtest/gopg.go +++ b/pkg/cmd/roachtest/gopg.go @@ -92,21 +92,21 @@ func registerGopg(r *testRegistry) { t.Fatal(err) } - blacklistName, expectedFailures, ignorelistName, ignorelist := gopgBlacklists.getLists(version) + blocklistName, expectedFailures, ignorelistName, ignorelist := gopgBlocklists.getLists(version) if expectedFailures == nil { - t.Fatalf("No gopg blacklist defined for cockroach version %s", version) + t.Fatalf("No gopg blocklist defined for cockroach version %s", version) } if ignorelist == nil { t.Fatalf("No gopg ignorelist defined for cockroach version %s", version) } - c.l.Printf("Running cockroach version %s, using blacklist %s, using ignorelist %s", - version, blacklistName, ignorelistName) + c.l.Printf("Running cockroach version %s, using blocklist %s, using ignorelist %s", + version, blocklistName, ignorelistName) _ = c.RunE(ctx, node, fmt.Sprintf("mkdir -p %s", resultsDirPath)) t.Status("running gopg test suite") // go test provides colorful output which - when redirected - interferes - // with matching of the blacklisted tests, so we will strip off all color + // with matching of the blocklisted tests, so we will strip off all color // code escape sequences. const removeColorCodes = `sed -r "s/\x1B\[([0-9]{1,2}(;[0-9]{1,2})?)?[mGK]//g"` // Note that this is expected to return an error, since the test suite @@ -145,7 +145,7 @@ func registerGopg(r *testRegistry) { results.parseJUnitXML(t, expectedFailures, ignorelist, xmlResults) results.summarizeFailed( - t, "gopg", blacklistName, expectedFailures, version, latestTag, + t, "gopg", blocklistName, expectedFailures, version, latestTag, 0, /* notRunCount */ ) } @@ -166,7 +166,7 @@ func registerGopg(r *testRegistry) { // test suites from gopg ORM tests. TestGinkgo is a test harness that runs // several test suites described by gopg. func gormParseTestGinkgoOutput( - r *ormTestsResults, rawResults []byte, expectedFailures, ignorelist blacklist, + r *ormTestsResults, rawResults []byte, expectedFailures, ignorelist blocklist, ) (err error) { var ( totalRunCount, totalTestCount int @@ -265,7 +265,7 @@ func gormParseTestGinkgoOutput( } } - // Blacklist contains both the expected failures for "global" tests as well + // Blocklist contains both the expected failures for "global" tests as well // as TestGinkgo's tests. We need to figure the number of the latter ones. testGinkgoExpectedFailures := 0 for failure := range expectedFailures { diff --git a/pkg/cmd/roachtest/gopg_blacklist.go b/pkg/cmd/roachtest/gopg_blocklist.go similarity index 93% rename from pkg/cmd/roachtest/gopg_blacklist.go rename to pkg/cmd/roachtest/gopg_blocklist.go index aae504c63ca8..7fb30add732e 100644 --- a/pkg/cmd/roachtest/gopg_blacklist.go +++ b/pkg/cmd/roachtest/gopg_blocklist.go @@ -10,10 +10,10 @@ package main -var gopgBlacklists = blacklistsForVersion{ - {"v19.2", "gopgBlackList19_2", gopgBlackList19_2, "gopgIgnoreList19_2", gopgIgnoreList19_2}, - {"v20.1", "gopgBlackList20_1", gopgBlackList20_1, "gopgIgnoreList20_1", gopgIgnoreList20_1}, - {"v20.2", "gopgBlackList20_2", gopgBlackList20_2, "gopgIgnoreList20_2", gopgIgnoreList20_2}, +var gopgBlocklists = blocklistsForVersion{ + {"v19.2", "gopgBlockList19_2", gopgBlockList19_2, "gopgIgnoreList19_2", gopgIgnoreList19_2}, + {"v20.1", "gopgBlockList20_1", gopgBlockList20_1, "gopgIgnoreList20_1", gopgIgnoreList20_1}, + {"v20.2", "gopgBlockList20_2", gopgBlockList20_2, "gopgIgnoreList20_2", gopgIgnoreList20_2}, } // These are lists of known gopg test errors and failures. @@ -22,12 +22,12 @@ var gopgBlacklists = blacklistsForVersion{ // Any failed test that is not on this list is reported as FAIL - unexpected. // // Please keep these lists alphabetized for easy diffing. -// After a failed run, an updated version of this blacklist should be available +// After a failed run, an updated version of this blocklist should be available // in the test log. -var gopgBlackList20_2 = gopgBlackList20_1 +var gopgBlockList20_2 = gopgBlockList20_1 -var gopgBlackList20_1 = blacklist{ +var gopgBlockList20_1 = blocklist{ "pg | CopyFrom/CopyTo | copies corrupted data to a table": "41608", "pg | CopyFrom/CopyTo | copies data from a table and to a table": "41608", "pg | CountEstimate | works": "17511", @@ -47,7 +47,7 @@ var gopgBlackList20_1 = blacklist{ "v9.TestUnixSocket": "31113", } -var gopgBlackList19_2 = blacklist{ +var gopgBlockList19_2 = blocklist{ "pg | CopyFrom/CopyTo | copies corrupted data to a table": "5807", "pg | CopyFrom/CopyTo | copies data from a table and to a table": "5807", "pg | CountEstimate | works": "17511", @@ -97,7 +97,7 @@ var gopgIgnoreList20_2 = gopgIgnoreList20_1 var gopgIgnoreList20_1 = gopgIgnoreList19_2 -var gopgIgnoreList19_2 = blacklist{ +var gopgIgnoreList19_2 = blocklist{ // These "fetching" tests assume a particular order when ORDER BY clause is // omitted from the query by the ORM itself. "pg | ORM slice model | fetches Book relations": "41690", diff --git a/pkg/cmd/roachtest/hibernate.go b/pkg/cmd/roachtest/hibernate.go index 51a9bb3a6774..5a89a932eee9 100644 --- a/pkg/cmd/roachtest/hibernate.go +++ b/pkg/cmd/roachtest/hibernate.go @@ -109,11 +109,11 @@ func registerHibernate(r *testRegistry) { t.Fatal(err) } - blacklistName, expectedFailures, _, _ := hibernateBlacklists.getLists(version) + blocklistName, expectedFailures, _, _ := hibernateBlocklists.getLists(version) if expectedFailures == nil { - t.Fatalf("No hibernate blacklist defined for cockroach version %s", version) + t.Fatalf("No hibernate blocklist defined for cockroach version %s", version) } - c.l.Printf("Running cockroach version %s, using blacklist %s", version, blacklistName) + c.l.Printf("Running cockroach version %s, using blocklist %s", version, blocklistName) t.Status("running hibernate test suite, will take at least 3 hours") // When testing, it is helpful to run only a subset of the tests. To do so @@ -172,7 +172,7 @@ func registerHibernate(r *testRegistry) { parseAndSummarizeJavaORMTestsResults( ctx, t, c, node, "hibernate" /* ormName */, output, - blacklistName, expectedFailures, nil /* ignorelist */, version, latestTag, + blocklistName, expectedFailures, nil /* ignorelist */, version, latestTag, ) } diff --git a/pkg/cmd/roachtest/hibernate_blacklist.go b/pkg/cmd/roachtest/hibernate_blocklist.go similarity index 99% rename from pkg/cmd/roachtest/hibernate_blacklist.go rename to pkg/cmd/roachtest/hibernate_blocklist.go index 1a65feab8bce..656fe4791b14 100644 --- a/pkg/cmd/roachtest/hibernate_blacklist.go +++ b/pkg/cmd/roachtest/hibernate_blocklist.go @@ -10,26 +10,26 @@ package main -var hibernateBlacklists = blacklistsForVersion{ - {"v2.0", "hibernateBlackList2_0", hibernateBlackList2_0, "", nil}, - {"v2.1", "hibernateBlackList2_1", hibernateBlackList2_1, "", nil}, - {"v2.2", "hibernateBlackList19_1", hibernateBlackList19_1, "", nil}, - {"v19.1", "hibernateBlackList19_1", hibernateBlackList19_1, "", nil}, - {"v19.2", "hibernateBlackList19_2", hibernateBlackList19_2, "", nil}, - {"v20.1", "hibernateBlackList20_1", hibernateBlackList20_1, "", nil}, - {"v20.2", "hibernateBlackList20_2", hibernateBlackList20_2, "", nil}, +var hibernateBlocklists = blocklistsForVersion{ + {"v2.0", "hibernateBlockList2_0", hibernateBlockList2_0, "", nil}, + {"v2.1", "hibernateBlockList2_1", hibernateBlockList2_1, "", nil}, + {"v2.2", "hibernateBlockList19_1", hibernateBlockList19_1, "", nil}, + {"v19.1", "hibernateBlockList19_1", hibernateBlockList19_1, "", nil}, + {"v19.2", "hibernateBlockList19_2", hibernateBlockList19_2, "", nil}, + {"v20.1", "hibernateBlockList20_1", hibernateBlockList20_1, "", nil}, + {"v20.2", "hibernateBlockList20_2", hibernateBlockList20_2, "", nil}, } // Please keep these lists alphabetized for easy diffing. -// After a failed run, an updated version of this blacklist should be available +// After a failed run, an updated version of this blocklist should be available // in the test log. -var hibernateBlackList20_2 = blacklist{} +var hibernateBlockList20_2 = blocklist{} -var hibernateBlackList20_1 = blacklist{} +var hibernateBlockList20_1 = blocklist{} -var hibernateBlackList19_2 = blacklist{} +var hibernateBlockList19_2 = blocklist{} -var hibernateBlackList19_1 = blacklist{ +var hibernateBlockList19_1 = blocklist{ "org.hibernate.jpa.test.criteria.QueryBuilderTest.testDateTimeFunctions": "31708", "org.hibernate.jpa.test.indetifier.AssignedInitialValueTableGeneratorConfiguredTest.testTheFirstGeneratedIdIsEqualToTableGeneratorInitialValuePlusOne": "6583", "org.hibernate.jpa.test.indetifier.AssignedInitialValueTableGeneratorConfiguredTest.testTheGeneratedIdValuesAreCorrect": "6583", @@ -125,7 +125,7 @@ var hibernateBlackList19_1 = blacklist{ "org.hibernate.test.tool.schema.SchemaToolTransactionHandlingTest.testValidateInExistingJtaTransaction": "16769", } -var hibernateBlackList2_1 = blacklist{ +var hibernateBlockList2_1 = blocklist{ "org.hibernate.id.hhh12973.SequenceMismatchStrategyDefaultExceptionTest.test": "unknown", "org.hibernate.id.hhh12973.SequenceMismatchStrategyExceptionEnumTest.test": "unknown", "org.hibernate.id.hhh12973.SequenceMismatchStrategyFixWithSequenceGeneratorTest.test": "unknown", @@ -246,7 +246,7 @@ var hibernateBlackList2_1 = blacklist{ "org.hibernate.test.tool.schema.SchemaToolTransactionHandlingTest.testValidateInExistingJtaTransaction": "16769", } -var hibernateBlackList2_0 = blacklist{ +var hibernateBlockList2_0 = blocklist{ "org.hibernate.engine.spi.ExtraStateTest.shouldMaintainExtraStateWhenUsingIdentityIdGenerationStrategy": "unknown", "org.hibernate.event.EmbeddableCallbackTest.test": "unknown", "org.hibernate.id.CreateDeleteTest.createAndDeleteAnEntityInTheSameTransactionTest": "unknown", diff --git a/pkg/cmd/roachtest/java_helpers.go b/pkg/cmd/roachtest/java_helpers.go index c313ab26c12e..2e017942645a 100644 --- a/pkg/cmd/roachtest/java_helpers.go +++ b/pkg/cmd/roachtest/java_helpers.go @@ -114,7 +114,7 @@ func extractFailureFromJUnitXML(contents []byte) ([]string, []status, map[string // parseJUnitXML parses testOutputInJUnitXMLFormat and updates the receiver // accordingly. func (r *ormTestsResults) parseJUnitXML( - t *test, expectedFailures, ignorelist blacklist, testOutputInJUnitXMLFormat []byte, + t *test, expectedFailures, ignorelist blocklist, testOutputInJUnitXMLFormat []byte, ) { tests, statuses, issueHints, err := extractFailureFromJUnitXML(testOutputInJUnitXMLFormat) if err != nil { @@ -170,7 +170,7 @@ func (r *ormTestsResults) parseJUnitXML( // parseAndSummarizeJavaORMTestsResults parses the test output of running a // test suite for some Java ORM against cockroach and summarizes it. If an // unexpected result is observed (for example, a test unexpectedly failed or -// passed), a new blacklist is populated. +// passed), a new blocklist is populated. func parseAndSummarizeJavaORMTestsResults( ctx context.Context, t *test, @@ -178,9 +178,9 @@ func parseAndSummarizeJavaORMTestsResults( node nodeListOption, ormName string, testOutput []byte, - blacklistName string, - expectedFailures blacklist, - ignorelist blacklist, + blocklistName string, + expectedFailures blocklist, + ignorelist blocklist, version string, latestTag string, ) { @@ -214,6 +214,6 @@ func parseAndSummarizeJavaORMTestsResults( } results.summarizeAll( - t, ormName, blacklistName, expectedFailures, version, latestTag, + t, ormName, blocklistName, expectedFailures, version, latestTag, ) } diff --git a/pkg/cmd/roachtest/libpq.go b/pkg/cmd/roachtest/libpq.go index cc3776581bb4..c71c8b829616 100644 --- a/pkg/cmd/roachtest/libpq.go +++ b/pkg/cmd/roachtest/libpq.go @@ -81,11 +81,11 @@ func registerLibPQ(r *testRegistry) { _ = c.RunE(ctx, node, fmt.Sprintf("mkdir -p %s", resultsDir)) - blacklistName, expectedFailures, ignorelistName, ignoredFailures := libPQBlacklists.getLists(version) + blocklistName, expectedFailures, ignorelistName, ignoredFailures := libPQBlocklists.getLists(version) if expectedFailures == nil { - t.Fatalf("No lib/pq blacklist defined for cockroach version %s", version) + t.Fatalf("No lib/pq blocklist defined for cockroach version %s", version) } - c.l.Printf("Running cockroach version %s, using blacklist %s, using ignorelist %s", version, blacklistName, ignorelistName) + c.l.Printf("Running cockroach version %s, using blocklist %s, using ignorelist %s", version, blocklistName, ignorelistName) t.Status("running lib/pq test suite and collecting results") @@ -98,7 +98,7 @@ func registerLibPQ(r *testRegistry) { parseAndSummarizeJavaORMTestsResults( ctx, t, c, node, "lib/pq" /* ormName */, []byte(resultsPath), - blacklistName, expectedFailures, ignoredFailures, version, latestTag, + blocklistName, expectedFailures, ignoredFailures, version, latestTag, ) } diff --git a/pkg/cmd/roachtest/libpq_blacklist.go b/pkg/cmd/roachtest/libpq_blocklist.go similarity index 92% rename from pkg/cmd/roachtest/libpq_blacklist.go rename to pkg/cmd/roachtest/libpq_blocklist.go index 30004c14f3ae..de54eafca864 100644 --- a/pkg/cmd/roachtest/libpq_blacklist.go +++ b/pkg/cmd/roachtest/libpq_blocklist.go @@ -10,15 +10,15 @@ package main -var libPQBlacklists = blacklistsForVersion{ - {"v19.2", "libPQBlacklist19_2", libPQBlacklist19_2, "libPQIgnorelist19_2", libPQIgnorelist19_2}, - {"v20.1", "libPQBlacklist20_1", libPQBlacklist20_1, "libPQIgnorelist20_1", libPQIgnorelist20_1}, - {"v20.2", "libPQBlacklist20_2", libPQBlacklist20_2, "libPQIgnorelist20_2", libPQIgnorelist20_2}, +var libPQBlocklists = blocklistsForVersion{ + {"v19.2", "libPQBlocklist19_2", libPQBlocklist19_2, "libPQIgnorelist19_2", libPQIgnorelist19_2}, + {"v20.1", "libPQBlocklist20_1", libPQBlocklist20_1, "libPQIgnorelist20_1", libPQIgnorelist20_1}, + {"v20.2", "libPQBlocklist20_2", libPQBlocklist20_2, "libPQIgnorelist20_2", libPQIgnorelist20_2}, } -var libPQBlacklist20_2 = libPQBlacklist20_1 +var libPQBlocklist20_2 = libPQBlocklist20_1 -var libPQBlacklist20_1 = blacklist{ +var libPQBlocklist20_1 = blocklist{ "pq.TestBinaryByteSliceToInt": "41547", "pq.TestBinaryByteSlicetoUUID": "41547", "pq.TestByteaOutputFormats": "26947", @@ -53,7 +53,7 @@ var libPQBlacklist20_1 = blacklist{ "pq.TestStringWithNul": "26366", } -var libPQBlacklist19_2 = blacklist{ +var libPQBlocklist19_2 = blocklist{ "pq.TestBinaryByteSliceToInt": "41547", "pq.TestBinaryByteSlicetoUUID": "41547", "pq.TestBindError": "5807", @@ -105,7 +105,7 @@ var libPQIgnorelist20_2 = libPQIgnorelist20_1 var libPQIgnorelist20_1 = libPQIgnorelist19_2 -var libPQIgnorelist19_2 = blacklist{ +var libPQIgnorelist19_2 = blocklist{ // TestFormatTsBacked fails due to not returning an error for accepting a // timestamp format that postgres does not. "pq.TestFormatTsBackend": "41690", diff --git a/pkg/cmd/roachtest/orm_helpers.go b/pkg/cmd/roachtest/orm_helpers.go index e0975156d9da..4af7759738dd 100644 --- a/pkg/cmd/roachtest/orm_helpers.go +++ b/pkg/cmd/roachtest/orm_helpers.go @@ -114,9 +114,9 @@ func newORMTestsResults() *ormTestsResults { // summarizeAll summarizes the result of running an ORM or a driver test suite // against a cockroach node. If an unexpected result is observed (for example, -// a test unexpectedly failed or passed), a new blacklist is populated. +// a test unexpectedly failed or passed), a new blocklist is populated. func (r *ormTestsResults) summarizeAll( - t *test, ormName, blacklistName string, expectedFailures blacklist, version, latestTag string, + t *test, ormName, blocklistName string, expectedFailures blocklist, version, latestTag string, ) { // Collect all the tests that were not run. notRunCount := 0 @@ -142,7 +142,7 @@ func (r *ormTestsResults) summarizeAll( t.l.Printf("------------------------\n") r.summarizeFailed( - t, ormName, blacklistName, expectedFailures, version, latestTag, notRunCount, + t, ormName, blocklistName, expectedFailures, version, latestTag, notRunCount, ) } @@ -152,8 +152,8 @@ func (r *ormTestsResults) summarizeAll( // If a test suite outputs only the failures, then this method should be used. func (r *ormTestsResults) summarizeFailed( t *test, - ormName, blacklistName string, - expectedFailures blacklist, + ormName, blocklistName string, + expectedFailures blocklist, version, latestTag string, notRunCount int, ) { @@ -193,11 +193,11 @@ func (r *ormTestsResults) summarizeFailed( if r.failUnexpectedCount > 0 || r.passUnexpectedCount > 0 || notRunCount > 0 || r.unexpectedSkipCount > 0 { - // Create a new blacklist so we can easily update this test. + // Create a new blocklist so we can easily update this test. sort.Strings(r.currentFailures) var b strings.Builder - fmt.Fprintf(&b, "Here is new %s blacklist that can be used to update the test:\n\n", ormName) - fmt.Fprintf(&b, "var %s = blacklist{\n", blacklistName) + fmt.Fprintf(&b, "Here is new %s blocklist that can be used to update the test:\n\n", ormName) + fmt.Fprintf(&b, "var %s = blocklist{\n", blocklistName) for _, test := range r.currentFailures { issue := expectedFailures[test] if len(issue) == 0 || issue == "unknown" { @@ -211,9 +211,9 @@ func (r *ormTestsResults) summarizeFailed( fmt.Fprintf(&b, "}\n\n") t.l.Printf("\n\n%s\n\n", b.String()) t.l.Printf("------------------------\n") - t.Fatalf("\n%s\nAn updated blacklist (%s) is available in the artifacts' %s log\n", + t.Fatalf("\n%s\nAn updated blocklist (%s) is available in the artifacts' %s log\n", bResults.String(), - blacklistName, + blocklistName, ormName, ) } diff --git a/pkg/cmd/roachtest/pgjdbc.go b/pkg/cmd/roachtest/pgjdbc.go index e295e4950731..5782dc8adff6 100644 --- a/pkg/cmd/roachtest/pgjdbc.go +++ b/pkg/cmd/roachtest/pgjdbc.go @@ -117,14 +117,14 @@ func registerPgjdbc(r *testRegistry) { t.Fatal(err) } - blacklistName, expectedFailures, ignorelistName, ignorelist := pgjdbcBlacklists.getLists(version) + blocklistName, expectedFailures, ignorelistName, ignorelist := pgjdbcBlocklists.getLists(version) if expectedFailures == nil { - t.Fatalf("No pgjdbc blacklist defined for cockroach version %s", version) + t.Fatalf("No pgjdbc blocklist defined for cockroach version %s", version) } - status := fmt.Sprintf("Running cockroach version %s, using blacklist %s", version, blacklistName) + status := fmt.Sprintf("Running cockroach version %s, using blocklist %s", version, blocklistName) if ignorelist != nil { - status = fmt.Sprintf("Running cockroach version %s, using blacklist %s, using ignorelist %s", - version, blacklistName, ignorelistName) + status = fmt.Sprintf("Running cockroach version %s, using blocklist %s, using ignorelist %s", + version, blocklistName, ignorelistName) } c.l.Printf("%s", status) @@ -173,7 +173,7 @@ func registerPgjdbc(r *testRegistry) { parseAndSummarizeJavaORMTestsResults( ctx, t, c, node, "pgjdbc" /* ormName */, output, - blacklistName, expectedFailures, ignorelist, version, latestTag, + blocklistName, expectedFailures, ignorelist, version, latestTag, ) } diff --git a/pkg/cmd/roachtest/pgjdbc_blacklist.go b/pkg/cmd/roachtest/pgjdbc_blocklist.go similarity index 99% rename from pkg/cmd/roachtest/pgjdbc_blacklist.go rename to pkg/cmd/roachtest/pgjdbc_blocklist.go index af42084cf901..4f8edb00edd6 100644 --- a/pkg/cmd/roachtest/pgjdbc_blacklist.go +++ b/pkg/cmd/roachtest/pgjdbc_blocklist.go @@ -10,21 +10,21 @@ package main -var pgjdbcBlacklists = blacklistsForVersion{ - {"v2.1", "pgjdbcBlackList2_1", pgjdbcBlackList2_1, "", nil}, - {"v2.2", "pgjdbcBlackList19_1", pgjdbcBlackList19_1, "", pgjdbcIgnoreList19_1}, - {"v19.1", "pgjdbcBlackList19_1", pgjdbcBlackList19_1, "", pgjdbcIgnoreList19_1}, - {"v19.2", "pgjdbcBlackList19_2", pgjdbcBlackList19_2, "pgjdbcIgnoreList19_2", pgjdbcIgnoreList19_2}, - {"v20.1", "pgjdbcBlackList20_1", pgjdbcBlackList20_1, "pgjdbcIgnoreList20_1", pgjdbcIgnoreList20_1}, - {"v20.2", "pgjdbcBlackList20_2", pgjdbcBlackList20_2, "pgjdbcIgnoreList20_2", pgjdbcIgnoreList20_2}, +var pgjdbcBlocklists = blocklistsForVersion{ + {"v2.1", "pgjdbcBlockList2_1", pgjdbcBlockList2_1, "", nil}, + {"v2.2", "pgjdbcBlockList19_1", pgjdbcBlockList19_1, "", pgjdbcIgnoreList19_1}, + {"v19.1", "pgjdbcBlockList19_1", pgjdbcBlockList19_1, "", pgjdbcIgnoreList19_1}, + {"v19.2", "pgjdbcBlockList19_2", pgjdbcBlockList19_2, "pgjdbcIgnoreList19_2", pgjdbcIgnoreList19_2}, + {"v20.1", "pgjdbcBlockList20_1", pgjdbcBlockList20_1, "pgjdbcIgnoreList20_1", pgjdbcIgnoreList20_1}, + {"v20.2", "pgjdbcBlockList20_2", pgjdbcBlockList20_2, "pgjdbcIgnoreList20_2", pgjdbcIgnoreList20_2}, } // Please keep these lists alphabetized for easy diffing. -// After a failed run, an updated version of this blacklist should be available +// After a failed run, an updated version of this blocklist should be available // in the test log. -var pgjdbcBlackList20_2 = pgjdbcBlackList20_1 +var pgjdbcBlockList20_2 = pgjdbcBlockList20_1 -var pgjdbcBlackList20_1 = blacklist{ +var pgjdbcBlockList20_1 = blocklist{ "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testDeepInternalsBatchedQueryDecorator": "26508", "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testUnspecifiedParameterType": "26508", "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testVaryingTypeCounts": "26508", @@ -1149,7 +1149,7 @@ var pgjdbcBlackList20_1 = blacklist{ "org.postgresql.test.xa.XADataSourceTest.testWrapperEquals": "22329", } -var pgjdbcBlackList19_1 = blacklist{ +var pgjdbcBlockList19_1 = blocklist{ "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testDeepInternalsBatchedQueryDecorator": "26508", "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testUnspecifiedParameterType": "26508", "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testVaryingTypeCounts": "26508", @@ -2666,7 +2666,7 @@ var pgjdbcBlackList19_1 = blacklist{ "org.postgresql.test.xa.XADataSourceTest.testWrapperEquals": "22329", } -var pgjdbcBlackList19_2 = blacklist{ +var pgjdbcBlockList19_2 = blocklist{ "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testDeepInternalsBatchedQueryDecorator": "26508", "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testUnspecifiedParameterType": "26508", "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testVaryingTypeCounts": "26508", @@ -4138,7 +4138,7 @@ var pgjdbcBlackList19_2 = blacklist{ "org.postgresql.test.xa.XADataSourceTest.testWrapperEquals": "22329", } -var pgjdbcBlackList2_1 = blacklist{ +var pgjdbcBlockList2_1 = blocklist{ "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testDeepInternalsBatchedQueryDecorator": "26508", "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testUnspecifiedParameterType": "26508", "org.postgresql.jdbc.DeepBatchedInsertStatementTest.testVaryingTypeCounts": "26508", @@ -5551,14 +5551,14 @@ var pgjdbcIgnoreList20_2 = pgjdbcIgnoreList20_1 var pgjdbcIgnoreList20_1 = pgjdbcIgnoreList19_2 -var pgjdbcIgnoreList19_2 = blacklist{ +var pgjdbcIgnoreList19_2 = blocklist{ "org.postgresql.replication.ReplicationTestSuite.org.postgresql.replication.ReplicationTestSuite": "expected fail - no replication", "org.postgresql.test.core.LogServerMessagePropertyTest.testWithDefaults": "expected fail - checks error message", "org.postgresql.test.core.LogServerMessagePropertyTest.testWithExplicitlyEnabled": "expected fail - checks error message", "org.postgresql.test.core.LogServerMessagePropertyTest.testWithLogServerErrorDetailDisabled": "expected fail - checks error message", } -var pgjdbcIgnoreList19_1 = blacklist{ +var pgjdbcIgnoreList19_1 = blocklist{ "org.postgresql.replication.ReplicationTestSuite.org.postgresql.replication.ReplicationTestSuite": "expected fail - no replication", "org.postgresql.test.core.LogServerMessagePropertyTest.testWithDefaults": "expected fail - checks error message", "org.postgresql.test.core.LogServerMessagePropertyTest.testWithExplicitlyEnabled": "expected fail - checks error message", diff --git a/pkg/cmd/roachtest/pgx.go b/pkg/cmd/roachtest/pgx.go index 231362d16624..c6f88dc75605 100644 --- a/pkg/cmd/roachtest/pgx.go +++ b/pkg/cmd/roachtest/pgx.go @@ -66,15 +66,15 @@ func registerPgx(r *testRegistry) { t.Fatal(err) } - t.Status("checking blacklist") - blacklistName, expectedFailures, ignorelistName, ignorelist := pgxBlacklists.getLists(version) + t.Status("checking blocklist") + blocklistName, expectedFailures, ignorelistName, ignorelist := pgxBlocklists.getLists(version) if expectedFailures == nil { - t.Fatalf("No pgx blacklist defined for cockroach version %s", version) + t.Fatalf("No pgx blocklist defined for cockroach version %s", version) } - status := fmt.Sprintf("Running cockroach version %s, using blacklist %s", version, blacklistName) + status := fmt.Sprintf("Running cockroach version %s, using blocklist %s", version, blocklistName) if ignorelist != nil { - status = fmt.Sprintf("Running cockroach version %s, using blacklist %s, using ignorelist %s", - version, blacklistName, ignorelistName) + status = fmt.Sprintf("Running cockroach version %s, using blocklist %s, using ignorelist %s", + version, blocklistName, ignorelistName) } c.l.Printf("%s", status) @@ -109,7 +109,7 @@ func registerPgx(r *testRegistry) { results := newORMTestsResults() results.parseJUnitXML(t, expectedFailures, ignorelist, xmlResults) results.summarizeAll( - t, "pgx", blacklistName, expectedFailures, version, latestTag, + t, "pgx", blocklistName, expectedFailures, version, latestTag, ) } diff --git a/pkg/cmd/roachtest/pgx_blacklist.go b/pkg/cmd/roachtest/pgx_blocklist.go similarity index 94% rename from pkg/cmd/roachtest/pgx_blacklist.go rename to pkg/cmd/roachtest/pgx_blocklist.go index 6d7b33325542..3f3830a19029 100644 --- a/pkg/cmd/roachtest/pgx_blacklist.go +++ b/pkg/cmd/roachtest/pgx_blocklist.go @@ -10,18 +10,18 @@ package main -var pgxBlacklists = blacklistsForVersion{ - {"v19.2", "pgxBlacklist19_2", pgxBlacklist19_2, "pgxIgnorelist19_2", pgxIgnorelist19_2}, - {"v20.1", "pgxBlacklist20_1", pgxBlacklist20_1, "pgxIgnorelist20_1", pgxIgnorelist20_1}, - {"v20.2", "pgxBlacklist20_2", pgxBlacklist20_2, "pgxIgnorelist20_2", pgxIgnorelist20_2}, +var pgxBlocklists = blocklistsForVersion{ + {"v19.2", "pgxBlocklist19_2", pgxBlocklist19_2, "pgxIgnorelist19_2", pgxIgnorelist19_2}, + {"v20.1", "pgxBlocklist20_1", pgxBlocklist20_1, "pgxIgnorelist20_1", pgxIgnorelist20_1}, + {"v20.2", "pgxBlocklist20_2", pgxBlocklist20_2, "pgxIgnorelist20_2", pgxIgnorelist20_2}, } // Please keep these lists alphabetized for easy diffing. -// After a failed run, an updated version of this blacklist should be available +// After a failed run, an updated version of this blocklist should be available // in the test log. -var pgxBlacklist20_2 = pgxBlacklist20_1 +var pgxBlocklist20_2 = pgxBlocklist20_1 -var pgxBlacklist20_1 = blacklist{ +var pgxBlocklist20_1 = blocklist{ "v4.Example_CustomType": "27796", "v4.TestConnBeginBatchDeferredError": "31632", "v4.TestConnCopyFromFailServerSideMidway": "19603", @@ -65,12 +65,12 @@ var pgxBlacklist20_1 = blacklist{ var pgxIgnorelist20_2 = pgxIgnorelist20_1 -var pgxIgnorelist20_1 = blacklist{ +var pgxIgnorelist20_1 = blocklist{ "v4.TestBeginIsoLevels": "We don't support isolation levels", "v4.TestQueryEncodeError": "This test checks the exact error message", } -var pgxBlacklist19_2 = blacklist{ +var pgxBlocklist19_2 = blocklist{ "v4.Example_CustomType": "27796", "v4.TestConnBeginBatchDeferredError": "31632", "v4.TestConnCopyFromCopyFromSourceErrorEnd": "5807", @@ -129,7 +129,7 @@ var pgxBlacklist19_2 = blacklist{ "v4.TestUnregisteredTypeUsableAsStringArgumentAndBaseResult": "27796", } -var pgxIgnorelist19_2 = blacklist{ +var pgxIgnorelist19_2 = blocklist{ "v4.TestBeginIsoLevels": "We don't support isolation levels", "v4.TestQueryEncodeError": "This test checks the exact error message", } diff --git a/pkg/cmd/roachtest/psycopg.go b/pkg/cmd/roachtest/psycopg.go index 8bb2271d9ebc..7384a7fcf7b8 100644 --- a/pkg/cmd/roachtest/psycopg.go +++ b/pkg/cmd/roachtest/psycopg.go @@ -90,15 +90,15 @@ func registerPsycopg(r *testRegistry) { t.Fatal(err) } - blacklistName, expectedFailures, ignoredlistName, ignoredlist := psycopgBlacklists.getLists(version) + blocklistName, expectedFailures, ignoredlistName, ignoredlist := psycopgBlocklists.getLists(version) if expectedFailures == nil { - t.Fatalf("No psycopg blacklist defined for cockroach version %s", version) + t.Fatalf("No psycopg blocklist defined for cockroach version %s", version) } if ignoredlist == nil { t.Fatalf("No psycopg ignorelist defined for cockroach version %s", version) } - c.l.Printf("Running cockroach version %s, using blacklist %s, using ignoredlist %s", - version, blacklistName, ignoredlistName) + c.l.Printf("Running cockroach version %s, using blocklist %s, using ignoredlist %s", + version, blocklistName, ignoredlistName) t.Status("running psycopg test suite") // Note that this is expected to return an error, since the test suite @@ -119,7 +119,7 @@ func registerPsycopg(r *testRegistry) { results := newORMTestsResults() results.parsePythonUnitTestOutput(rawResults, expectedFailures, ignoredlist) results.summarizeAll( - t, "psycopg" /* ormName */, blacklistName, expectedFailures, + t, "psycopg" /* ormName */, blocklistName, expectedFailures, version, latestTag, ) } diff --git a/pkg/cmd/roachtest/psycopg_blacklist.go b/pkg/cmd/roachtest/psycopg_blocklist.go similarity index 99% rename from pkg/cmd/roachtest/psycopg_blacklist.go rename to pkg/cmd/roachtest/psycopg_blocklist.go index 56dcea06ec8d..0417167c3432 100644 --- a/pkg/cmd/roachtest/psycopg_blacklist.go +++ b/pkg/cmd/roachtest/psycopg_blocklist.go @@ -10,12 +10,12 @@ package main -var psycopgBlacklists = blacklistsForVersion{ - {"v2.2", "psycopgBlackList19_1", psycopgBlackList19_1, "psycopgIgnoreList19_1", psycopgIgnoreList19_1}, - {"v19.1", "psycopgBlackList19_1", psycopgBlackList19_1, "psycopgIgnoreList19_1", psycopgIgnoreList19_1}, - {"v19.2", "psycopgBlackList19_2", psycopgBlackList19_2, "psycopgIgnoreList19_2", psycopgIgnoreList19_2}, - {"v20.1", "psycopgBlackList20_1", psycopgBlackList20_1, "psycopgIgnoreList20_1", psycopgIgnoreList20_1}, - {"v20.2", "psycopgBlackList20_2", psycopgBlackList20_2, "psycopgIgnoreList20_2", psycopgIgnoreList20_2}, +var psycopgBlocklists = blocklistsForVersion{ + {"v2.2", "psycopgBlockList19_1", psycopgBlockList19_1, "psycopgIgnoreList19_1", psycopgIgnoreList19_1}, + {"v19.1", "psycopgBlockList19_1", psycopgBlockList19_1, "psycopgIgnoreList19_1", psycopgIgnoreList19_1}, + {"v19.2", "psycopgBlockList19_2", psycopgBlockList19_2, "psycopgIgnoreList19_2", psycopgIgnoreList19_2}, + {"v20.1", "psycopgBlockList20_1", psycopgBlockList20_1, "psycopgIgnoreList20_1", psycopgIgnoreList20_1}, + {"v20.2", "psycopgBlockList20_2", psycopgBlockList20_2, "psycopgIgnoreList20_2", psycopgIgnoreList20_2}, } // These are lists of known psycopg test errors and failures. @@ -27,11 +27,11 @@ var psycopgBlacklists = blacklistsForVersion{ // Any test on this list that is not run is reported as FAIL - not run // // Please keep these lists alphabetized for easy diffing. -// After a failed run, an updated version of this blacklist should be available +// After a failed run, an updated version of this blocklist should be available // in the test log. -var psycopgBlackList20_2 = psycopgBlackList20_1 +var psycopgBlockList20_2 = psycopgBlockList20_1 -var psycopgBlackList20_1 = blacklist{ +var psycopgBlockList20_1 = blocklist{ "tests.test_async.AsyncTests.test_async_callproc": "44701", "tests.test_async.AsyncTests.test_error": "44706", "tests.test_async.AsyncTests.test_flush_on_write": "44709", @@ -240,7 +240,7 @@ var psycopgBlackList20_1 = blacklist{ "tests.test_with.WithCursorTestCase.test_named_with_noop": "30352", } -var psycopgBlackList19_2 = blacklist{ +var psycopgBlockList19_2 = blocklist{ "tests.test_async.AsyncTests.test_async_after_async": "5807", "tests.test_async.AsyncTests.test_async_callproc": "5807", "tests.test_async.AsyncTests.test_async_connection_error_message": "5807", @@ -527,7 +527,7 @@ var psycopgBlackList19_2 = blacklist{ "tests.test_with.WithCursorTestCase.test_named_with_noop": "30352", } -var psycopgBlackList19_1 = blacklist{ +var psycopgBlockList19_1 = blocklist{ "tests.test_async.AsyncTests.test_async_after_async": "5807", "tests.test_async.AsyncTests.test_async_callproc": "5807", "tests.test_async.AsyncTests.test_async_connection_error_message": "5807", @@ -825,7 +825,7 @@ var psycopgIgnoreList20_1 = psycopgIgnoreList19_2 var psycopgIgnoreList19_2 = psycopgIgnoreList19_1 -var psycopgIgnoreList19_1 = blacklist{ +var psycopgIgnoreList19_1 = blocklist{ "tests.test_green.GreenTestCase.test_flush_on_write": "flakey", "tests.test_connection.TestConnectionInfo.test_backend_pid": "we return -1 for pg_backend_pid()", } diff --git a/pkg/cmd/roachtest/python_helpers.go b/pkg/cmd/roachtest/python_helpers.go index 53040104536b..98d19ce9d580 100644 --- a/pkg/cmd/roachtest/python_helpers.go +++ b/pkg/cmd/roachtest/python_helpers.go @@ -20,7 +20,7 @@ import ( var pythonUnitTestOutputRegex = regexp.MustCompile(`(?P.*) \((?P.*)\) \.\.\. (?P[^ ']*)(?: u?['"](?P.*)['"])?`) func (r *ormTestsResults) parsePythonUnitTestOutput( - input []byte, expectedFailures blacklist, ignoredList blacklist, + input []byte, expectedFailures blocklist, ignoredList blocklist, ) { scanner := bufio.NewScanner(bytes.NewReader(input)) for scanner.Scan() { diff --git a/pkg/cmd/roachtest/sqlalchemy.go b/pkg/cmd/roachtest/sqlalchemy.go index bee053fd7f70..101253b29ce9 100644 --- a/pkg/cmd/roachtest/sqlalchemy.go +++ b/pkg/cmd/roachtest/sqlalchemy.go @@ -155,12 +155,12 @@ func registerSQLAlchemy(r *testRegistry) { t.Fatal(err) } - blacklistName, expectedFailures, ignoredlistName, ignoredlist := sqlAlchemyBlacklists.getLists(version) + blocklistName, expectedFailures, ignoredlistName, ignoredlist := sqlAlchemyBlocklists.getLists(version) if expectedFailures == nil { - t.Fatalf("No sqlalchemy blacklist defined for cockroach version %s", version) + t.Fatalf("No sqlalchemy blocklist defined for cockroach version %s", version) } - c.l.Printf("Running cockroach version %s, using blacklist %s, using ignoredlist %s", - version, blacklistName, ignoredlistName) + c.l.Printf("Running cockroach version %s, using blocklist %s, using ignoredlist %s", + version, blocklistName, ignoredlistName) t.Status("running sqlalchemy test suite") // Note that this is expected to return an error, since the test suite @@ -222,7 +222,7 @@ func registerSQLAlchemy(r *testRegistry) { } results.summarizeAll( - t, "sqlalchemy" /* ormName */, blacklistName, expectedFailures, version, latestTag) + t, "sqlalchemy" /* ormName */, blocklistName, expectedFailures, version, latestTag) } r.Add(testSpec{ diff --git a/pkg/cmd/roachtest/sqlalchemy_blacklist.go b/pkg/cmd/roachtest/sqlalchemy_blocklist.go similarity index 97% rename from pkg/cmd/roachtest/sqlalchemy_blacklist.go rename to pkg/cmd/roachtest/sqlalchemy_blocklist.go index b7c21dcf9b21..12c87d484341 100644 --- a/pkg/cmd/roachtest/sqlalchemy_blacklist.go +++ b/pkg/cmd/roachtest/sqlalchemy_blocklist.go @@ -10,17 +10,17 @@ package main -var sqlAlchemyBlacklists = blacklistsForVersion{ - {"v2.1", "sqlAlchemyBlacklist", sqlAlchemyBlacklist, "sqlAlchemyIgnoreList", sqlAlchemyIgnoreList}, - {"v19.1", "sqlAlchemyBlacklist", sqlAlchemyBlacklist, "sqlAlchemyIgnoreList", sqlAlchemyIgnoreList}, - {"v19.2", "sqlAlchemyBlacklist", sqlAlchemyBlacklist, "sqlAlchemyIgnoreList", sqlAlchemyIgnoreList}, - {"v20.1", "sqlAlchemyBlacklist20_1", sqlAlchemyBlacklist20_1, "sqlAlchemyIgnoreList20_1", sqlAlchemyIgnoreList20_1}, - {"v20.2", "sqlAlchemyBlacklist20_2", sqlAlchemyBlacklist20_2, "sqlAlchemyIgnoreList20_2", sqlAlchemyIgnoreList20_2}, +var sqlAlchemyBlocklists = blocklistsForVersion{ + {"v2.1", "sqlAlchemyBlocklist", sqlAlchemyBlocklist, "sqlAlchemyIgnoreList", sqlAlchemyIgnoreList}, + {"v19.1", "sqlAlchemyBlocklist", sqlAlchemyBlocklist, "sqlAlchemyIgnoreList", sqlAlchemyIgnoreList}, + {"v19.2", "sqlAlchemyBlocklist", sqlAlchemyBlocklist, "sqlAlchemyIgnoreList", sqlAlchemyIgnoreList}, + {"v20.1", "sqlAlchemyBlocklist20_1", sqlAlchemyBlocklist20_1, "sqlAlchemyIgnoreList20_1", sqlAlchemyIgnoreList20_1}, + {"v20.2", "sqlAlchemyBlocklist20_2", sqlAlchemyBlocklist20_2, "sqlAlchemyIgnoreList20_2", sqlAlchemyIgnoreList20_2}, } -var sqlAlchemyBlacklist20_2 = sqlAlchemyBlacklist20_1 +var sqlAlchemyBlocklist20_2 = sqlAlchemyBlocklist20_1 -var sqlAlchemyBlacklist20_1 = blacklist{ +var sqlAlchemyBlocklist20_1 = blocklist{ "test/dialect/test_suite.py::ComponentReflectionTest_cockroachdb+psycopg2_9_5_0::test_autoincrement_col": "5807", "test/dialect/test_suite.py::ComponentReflectionTest_cockroachdb+psycopg2_9_5_0::test_deprecated_get_primary_keys": "5807", "test/dialect/test_suite.py::ComponentReflectionTest_cockroachdb+psycopg2_9_5_0::test_dialect_initialize": "5807", @@ -90,7 +90,7 @@ var sqlAlchemyBlacklist20_1 = blacklist{ "test/dialect/test_suite.py::TableDDLTest_cockroachdb+psycopg2_9_5_0::test_create_table_schema": "unknown", } -var sqlAlchemyBlacklist = blacklist{ +var sqlAlchemyBlocklist = blocklist{ "test/dialect/test_suite.py::ComponentReflectionTest_cockroachdb+psycopg2_9_5_0::test_autoincrement_col": "5807", "test/dialect/test_suite.py::ComponentReflectionTest_cockroachdb+psycopg2_9_5_0::test_deprecated_get_primary_keys": "5807", "test/dialect/test_suite.py::ComponentReflectionTest_cockroachdb+psycopg2_9_5_0::test_dialect_initialize": "5807", @@ -164,7 +164,7 @@ var sqlAlchemyIgnoreList20_2 = sqlAlchemyIgnoreList var sqlAlchemyIgnoreList20_1 = sqlAlchemyIgnoreList -var sqlAlchemyIgnoreList = blacklist{ +var sqlAlchemyIgnoreList = blocklist{ "test/dialect/test_suite.py::ExceptionTest_cockroachdb+psycopg2_9_5_0::test_integrity_error": "passes, but can't parse result", "test/dialect/test_suite.py::TableDDLTest_cockroachdb+psycopg2_9_5_0::test_create_table": "flaky", } diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index a69c3709b579..dd94107152a5 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -1705,9 +1705,9 @@ func TestStoreSplitTimestampCacheDifferentLeaseHolder(t *testing.T) { } return replica } - blacklistedLeaseHolder := leaseHolder(leftKey) - log.Infof(ctx, "blacklisting replica %+v for leases", blacklistedLeaseHolder) - noLeaseForDesc.Store(&blacklistedLeaseHolder) + blocklistedLeaseHolder := leaseHolder(leftKey) + log.Infof(ctx, "blocklisting replica %+v for leases", blocklistedLeaseHolder) + noLeaseForDesc.Store(&blocklistedLeaseHolder) // Pull the trigger. This actually also reads the RHS descriptor after the // split, so when this returns, we've got the leases set up already. @@ -1727,9 +1727,9 @@ func TestStoreSplitTimestampCacheDifferentLeaseHolder(t *testing.T) { } if currentLHSLeaseHolder := leaseHolder(leftKey); !reflect.DeepEqual( - currentLHSLeaseHolder, blacklistedLeaseHolder) { + currentLHSLeaseHolder, blocklistedLeaseHolder) { t.Fatalf("lease holder changed from %+v to %+v, should de-flake this test", - blacklistedLeaseHolder, currentLHSLeaseHolder) + blocklistedLeaseHolder, currentLHSLeaseHolder) } // This write (to the right-hand side of the split) should hit the @@ -1755,7 +1755,7 @@ func TestStoreSplitTimestampCacheDifferentLeaseHolder(t *testing.T) { // that it's the same ReplicaID, which is not required but should always // hold). if rhsLease := leaseHolder(rightKey); !reflect.DeepEqual( - rhsLease, blacklistedLeaseHolder, + rhsLease, blocklistedLeaseHolder, ) { t.Errorf("expected LHS and RHS to have same lease holder") } diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index 801c0a09ab6e..95ad607c0e02 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -46,7 +46,7 @@ import ( func isTrivial(r *kvserverpb.ReplicatedEvalResult) bool { // Check if there are any non-trivial State updates. if r.State != nil { - stateWhitelist := *r.State + stateAllowlist := *r.State // ReplicaState.Stats was previously non-nullable which caused nodes to // send a zero-value MVCCStats structure. If the proposal was generated by // an old node, we'll have decoded that zero-value structure setting @@ -54,23 +54,23 @@ func isTrivial(r *kvserverpb.ReplicatedEvalResult) bool { // field in ReplicatedEvalResult" assertion to fire if we didn't clear it. // TODO(ajwerner): eliminate this case that likely can no longer occur as of // at least 19.1. - if stateWhitelist.Stats != nil && (*stateWhitelist.Stats == enginepb.MVCCStats{}) { - stateWhitelist.Stats = nil + if stateAllowlist.Stats != nil && (*stateAllowlist.Stats == enginepb.MVCCStats{}) { + stateAllowlist.Stats = nil } - if stateWhitelist != (kvserverpb.ReplicaState{}) { + if stateAllowlist != (kvserverpb.ReplicaState{}) { return false } } - // Set whitelist to the value of r and clear the whitelisted fields. - // If whitelist is zero-valued after clearing the whitelisted fields then + // Set allowlist to the value of r and clear the allowlisted fields. + // If allowlist is zero-valued after clearing the allowlisted fields then // it is trivial. - whitelist := *r - whitelist.Delta = enginepb.MVCCStatsDelta{} - whitelist.Timestamp = hlc.Timestamp{} - whitelist.DeprecatedDelta = nil - whitelist.PrevLeaseProposal = nil - whitelist.State = nil - return whitelist.Equal(kvserverpb.ReplicatedEvalResult{}) + allowlist := *r + allowlist.Delta = enginepb.MVCCStatsDelta{} + allowlist.Timestamp = hlc.Timestamp{} + allowlist.DeprecatedDelta = nil + allowlist.PrevLeaseProposal = nil + allowlist.State = nil + return allowlist.Equal(kvserverpb.ReplicatedEvalResult{}) } // clearTrivialReplicatedEvalResultFields is used to zero out the fields of a diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 5eea9f930352..10aabe75219b 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -2037,12 +2037,12 @@ func (s *Store) AdminRelocateRange( rangeDesc = *newDesc canRetry := func(err error) bool { - whitelist := []string{ + allowlist := []string{ snapshotApplySemBusyMsg, IntersectingSnapshotMsg, } errStr := err.Error() - for _, substr := range whitelist { + for _, substr := range allowlist { if strings.Contains(errStr, substr) { return true } diff --git a/pkg/kv/kvserver/replica_evaluate.go b/pkg/kv/kvserver/replica_evaluate.go index 1058e5f8c061..20989bc074d3 100644 --- a/pkg/kv/kvserver/replica_evaluate.go +++ b/pkg/kv/kvserver/replica_evaluate.go @@ -187,7 +187,7 @@ func evaluateBatch( // has already been aborted. // - heartbeats don't check the abort span. If the txn is aborted, they'll // return an aborted proto in their otherwise successful response. - // TODO(nvanbenschoten): Let's remove heartbeats from this whitelist when + // TODO(nvanbenschoten): Let's remove heartbeats from this allowlist when // we rationalize the TODO in txnHeartbeater.heartbeat. if !ba.IsSingleAbortTxnRequest() && !ba.IsSingleHeartbeatTxnRequest() { if pErr := checkIfTxnAborted(ctx, rec, readWriter, *baHeader.Txn); pErr != nil { diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 9dea22aa6d01..1e326c2e06cf 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -751,7 +751,7 @@ func (r *Replica) evaluateProposal( } // Failed proposals can't have any Result except for what's - // whitelisted here. + // allowlisted here. res.Local = result.LocalResult{ EncounteredIntents: res.Local.DetachEncounteredIntents(), EndTxns: res.Local.DetachEndTxns(true /* alwaysOnly */), diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index 464d92401357..6d04e0d884ac 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -137,7 +137,7 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes( if pErr != nil { // Failed read-only batches can't have any Result except for what's - // whitelisted here. + // allowlisted here. res.Local = result.LocalResult{ EncounteredIntents: res.Local.DetachEncounteredIntents(), Metrics: res.Local.Metrics, diff --git a/pkg/kv/kvserver/track_raft_protos.go b/pkg/kv/kvserver/track_raft_protos.go index 6cb428fc0333..b77b1927b412 100644 --- a/pkg/kv/kvserver/track_raft_protos.go +++ b/pkg/kv/kvserver/track_raft_protos.go @@ -37,7 +37,7 @@ func TrackRaftProtos() func() []reflect.Type { applyRaftEntryFunc := funcName((*apply.Task).ApplyCommittedEntries) // We only need to track protos that could cause replica divergence // by being written to disk downstream of raft. - whitelist := []string{ + allowlist := []string{ // Some raft operations trigger gossip, but we don't require // strict consistency there. funcName((*gossip.Gossip).AddInfoProto), @@ -92,14 +92,14 @@ func TrackRaftProtos() func() []reflect.Type { for { f, more := frames.Next() - whitelisted := false - for _, s := range whitelist { + allowlisted := false + for _, s := range allowlist { if strings.Contains(f.Function, s) { - whitelisted = true + allowlisted = true break } } - if whitelisted { + if allowlisted { break } diff --git a/pkg/server/authentication_test.go b/pkg/server/authentication_test.go index ac540ccfef43..4e8867a4c40b 100644 --- a/pkg/server/authentication_test.go +++ b/pkg/server/authentication_test.go @@ -73,7 +73,7 @@ func (insecureCtx) HTTPRequestScheme() string { return "https" } -// Verify client certificate enforcement and user whitelisting. +// Verify client certificate enforcement and user allowlisting. func TestSSLEnforcement(t *testing.T) { defer leaktest.AfterTest(t)() s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ diff --git a/pkg/server/server_sql_test.go b/pkg/server/server_sql_test.go index e836eb1585c1..5ff9fa72f359 100644 --- a/pkg/server/server_sql_test.go +++ b/pkg/server/server_sql_test.go @@ -34,7 +34,7 @@ import ( // and must not rely on having a NodeID/NodeDescriptor/NodeLiveness/... // // In short, it should not rely on the test server through anything other than a -// `*kv.DB` and a small number of whitelisted RPCs. +// `*kv.DB` and a small number of allowlisted RPCs. func TestSQLServer(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() diff --git a/pkg/sql/create_role.go b/pkg/sql/create_role.go index 9bc1938ad81f..97c1cce6b3cc 100644 --- a/pkg/sql/create_role.go +++ b/pkg/sql/create_role.go @@ -235,7 +235,7 @@ const usernameHelp = "Usernames are case insensitive, must start with a letter, var usernameRE = regexp.MustCompile(`^[\p{Ll}0-9_][---\p{Ll}0-9_.]*$`) -var blacklistedUsernames = map[string]struct{}{ +var blocklistedUsernames = map[string]struct{}{ security.NodeUser: {}, } @@ -243,19 +243,19 @@ var blacklistedUsernames = map[string]struct{}{ // it validates according to the usernameRE regular expression. // It rejects reserved user names. func NormalizeAndValidateUsername(username string) (string, error) { - username, err := NormalizeAndValidateUsernameNoBlacklist(username) + username, err := NormalizeAndValidateUsernameNoBlocklist(username) if err != nil { return "", err } - if _, ok := blacklistedUsernames[username]; ok { + if _, ok := blocklistedUsernames[username]; ok { return "", pgerror.Newf(pgcode.ReservedName, "username %q reserved", username) } return username, nil } -// NormalizeAndValidateUsernameNoBlacklist case folds the specified username and verifies +// NormalizeAndValidateUsernameNoBlocklist case folds the specified username and verifies // it validates according to the usernameRE regular expression. -func NormalizeAndValidateUsernameNoBlacklist(username string) (string, error) { +func NormalizeAndValidateUsernameNoBlocklist(username string) (string, error) { username = tree.Name(username).Normalize() if !usernameRE.MatchString(username) { return "", errors.WithHint(pgerror.Newf(pgcode.InvalidName, "username %q invalid", username), usernameHelp) diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index cc84652464da..112c87988ee0 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -172,7 +172,7 @@ func (dsp *DistSQLPlanner) SetSpanResolver(spanResolver physicalplan.SpanResolve } // distSQLExprCheckVisitor is a tree.Visitor that checks if expressions -// contain things not supported by distSQL, like distSQL-blacklisted functions. +// contain things not supported by distSQL, like distSQL-blocklisted functions. type distSQLExprCheckVisitor struct { err error } @@ -185,7 +185,7 @@ func (v *distSQLExprCheckVisitor) VisitPre(expr tree.Expr) (recurse bool, newExp } switch t := expr.(type) { case *tree.FuncExpr: - if t.IsDistSQLBlacklist() { + if t.IsDistSQLBlocklist() { v.err = newQueryNotSupportedErrorf("function %s cannot be executed with distsql", t) return false, expr } @@ -206,7 +206,7 @@ func (v *distSQLExprCheckVisitor) VisitPre(expr tree.Expr) (recurse bool, newExp func (v *distSQLExprCheckVisitor) VisitPost(expr tree.Expr) tree.Expr { return expr } // checkExpr verifies that an expression doesn't contain things that are not yet -// supported by distSQL, like distSQL-blacklisted functions. +// supported by distSQL, like distSQL-blocklisted functions. func checkExpr(expr tree.Expr) error { if expr == nil { return nil diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index bc6124d539e7..7dcfbd1ab0e1 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -109,12 +109,12 @@ import ( // logicTestConfigs. If the directive is missing, the test is run in the // default configuration. // -// The directive also supports blacklists, i.e. running all specified -// configurations apart from a blacklisted configuration: +// The directive also supports blocklists, i.e. running all specified +// configurations apart from a blocklisted configuration: // // # LogicTest: default-configs !3node-tenant // -// If a blacklist is specified without an accompanying configuration, the +// If a blocklist is specified without an accompanying configuration, the // default config is assumed. i.e., the following directive is equivalent to the // one above: // @@ -1397,17 +1397,17 @@ CREATE DATABASE test; t.unsupported = 0 } -// applyBlacklistToConfigIdxs applies the given blacklist to config idxs, +// applyBlocklistToConfigIdxs applies the given blocklist to config idxs, // returning the result. -func applyBlacklistToConfigIdxs( - configIdxs []logicTestConfigIdx, blacklist map[string]struct{}, +func applyBlocklistToConfigIdxs( + configIdxs []logicTestConfigIdx, blocklist map[string]struct{}, ) []logicTestConfigIdx { - if len(blacklist) == 0 { + if len(blocklist) == 0 { return configIdxs } var newConfigIdxs []logicTestConfigIdx for _, idx := range configIdxs { - if _, ok := blacklist[logicTestConfigIdxToName[idx]]; ok { + if _, ok := blocklist[logicTestConfigIdxToName[idx]]; ok { continue } newConfigIdxs = append(newConfigIdxs, idx) @@ -1418,28 +1418,28 @@ func applyBlacklistToConfigIdxs( // processConfigs, given a list of configNames, returns the list of // corresponding logicTestConfigIdxs. func processConfigs(t *testing.T, path string, configNames []string) []logicTestConfigIdx { - const blacklistChar = '!' - blacklist := make(map[string]struct{}) - allConfigNamesAreBlacklistDirectives := true + const blocklistChar = '!' + blocklist := make(map[string]struct{}) + allConfigNamesAreBlocklistDirectives := true for _, configName := range configNames { - if configName[0] != blacklistChar { - allConfigNamesAreBlacklistDirectives = false + if configName[0] != blocklistChar { + allConfigNamesAreBlocklistDirectives = false continue } - blacklist[configName[1:]] = struct{}{} + blocklist[configName[1:]] = struct{}{} } var configs []logicTestConfigIdx - if len(blacklist) != 0 && allConfigNamesAreBlacklistDirectives { - // No configs specified, this blacklist applies to the default config. - return applyBlacklistToConfigIdxs(defaultConfig, blacklist) + if len(blocklist) != 0 && allConfigNamesAreBlocklistDirectives { + // No configs specified, this blocklist applies to the default config. + return applyBlocklistToConfigIdxs(defaultConfig, blocklist) } for _, configName := range configNames { - if configName[0] == blacklistChar { + if configName[0] == blocklistChar { continue } - if _, ok := blacklist[configName]; ok { + if _, ok := blocklist[configName]; ok { continue } @@ -1447,9 +1447,9 @@ func processConfigs(t *testing.T, path string, configNames []string) []logicTest if !ok { switch configName { case defaultConfigName: - configs = append(configs, applyBlacklistToConfigIdxs(defaultConfig, blacklist)...) + configs = append(configs, applyBlocklistToConfigIdxs(defaultConfig, blocklist)...) case fiveNodeDefaultConfigName: - configs = append(configs, applyBlacklistToConfigIdxs(fiveNodeDefaultConfig, blacklist)...) + configs = append(configs, applyBlocklistToConfigIdxs(fiveNodeDefaultConfig, blocklist)...) default: t.Fatalf("%s: unknown config name %s", path, configName) } diff --git a/pkg/sql/opt/memo/expr.go b/pkg/sql/opt/memo/expr.go index 88b98efbd42d..c2b14711cbd6 100644 --- a/pkg/sql/opt/memo/expr.go +++ b/pkg/sql/opt/memo/expr.go @@ -631,7 +631,7 @@ func (prj *ProjectExpr) initUnexportedFields(mem *Memo) { // // We only add the FD if composite types are not involved. // - // TODO(radu): add a whitelist of expressions/operators that are ok, like + // TODO(radu): add a allowlist of expressions/operators that are ok, like // arithmetic. composite := false for i, ok := from.Next(0); ok; i, ok = from.Next(i + 1) { diff --git a/pkg/sql/opt/norm/fold_constants_funcs.go b/pkg/sql/opt/norm/fold_constants_funcs.go index 7f3d6dac9a3c..bbf269d3f3c9 100644 --- a/pkg/sql/opt/norm/fold_constants_funcs.go +++ b/pkg/sql/opt/norm/fold_constants_funcs.go @@ -404,7 +404,7 @@ func (c *CustomFuncs) FoldColumnAccess(input opt.ScalarExpr, idx memo.TupleOrdin // FoldFunction evaluates a function expression with constant inputs. It // returns a constant expression as long as the function is contained in the -// FoldFunctionWhitelist, and the evaluation causes no error. +// FoldFunctionAllowlist, and the evaluation causes no error. func (c *CustomFuncs) FoldFunction( args memo.ScalarListExpr, private *memo.FunctionPrivate, ) opt.ScalarExpr { @@ -413,9 +413,9 @@ func (c *CustomFuncs) FoldFunction( if private.Properties.Class != tree.NormalClass { return nil } - // Functions that aren't immutable and also not in the whitelist cannot + // Functions that aren't immutable and also not in the allowlist cannot // be folded. - if _, ok := FoldFunctionWhitelist[private.Name]; !ok && private.Overload.Volatility > tree.VolatilityImmutable { + if _, ok := FoldFunctionAllowlist[private.Name]; !ok && private.Overload.Volatility > tree.VolatilityImmutable { return nil } @@ -442,9 +442,9 @@ func (c *CustomFuncs) FoldFunction( return c.f.ConstructConstVal(result, private.Typ) } -// FoldFunctionWhitelist contains non-immutable functions that are nevertheless +// FoldFunctionAllowlist contains non-immutable functions that are nevertheless // known to be safe for folding. -var FoldFunctionWhitelist = map[string]struct{}{ +var FoldFunctionAllowlist = map[string]struct{}{ // The SQL statement is generated in the optbuilder phase, so the remaining // function execution is immutable. "addgeometrycolumn": {}, diff --git a/pkg/sql/opt/optbuilder/builder.go b/pkg/sql/opt/optbuilder/builder.go index 137066619bd4..eb5d83bb4ce6 100644 --- a/pkg/sql/opt/optbuilder/builder.go +++ b/pkg/sql/opt/optbuilder/builder.go @@ -231,7 +231,7 @@ func (b *Builder) buildStmt( stmt tree.Statement, desiredTypes []*types.T, inScope *scope, ) (outScope *scope) { if b.insideViewDef { - // A black list of statements that can't be used from inside a view. + // A blocklist of statements that can't be used from inside a view. switch stmt := stmt.(type) { case *tree.Delete, *tree.Insert, *tree.Update, *tree.CreateTable, *tree.CreateView, *tree.Split, *tree.Unsplit, *tree.Relocate, diff --git a/pkg/sql/parser/scan.go b/pkg/sql/parser/scan.go index f91c0f8f1629..ceb9e5851547 100644 --- a/pkg/sql/parser/scan.go +++ b/pkg/sql/parser/scan.go @@ -568,12 +568,12 @@ func (s *scanner) scanIdent(lval *sqlSymType) { if lval.id != lex.IDENT { if isExperimental { if _, ok := lex.AllowedExperimental[kw]; !ok { - // If the parsed token is not on the whitelisted set of keywords, + // If the parsed token is not on the allowlisted set of keywords, // then it might have been intended to be parsed as something else. // In that case, re-tokenize the original string. lval.id = lex.GetKeywordID(lval.str) } else { - // It is a whitelisted keyword, so remember the shortened + // It is a allowlisted keyword, so remember the shortened // keyword for further processing. lval.str = kw } diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 9f668ddf5713..ed5fef1d3b24 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -118,8 +118,8 @@ func NewSchemaChangerForTesting( } // isPermanentSchemaChangeError returns true if the error results in -// a permanent failure of a schema change. This function is a whitelist -// instead of a blacklist: only known safe errors are confirmed to not be +// a permanent failure of a schema change. This function is a allowlist +// instead of a blocklist: only known safe errors are confirmed to not be // permanent errors. Anything unknown is assumed to be permanent. func isPermanentSchemaChangeError(err error) bool { if err == nil { @@ -1550,7 +1550,7 @@ func (*SchemaChangerTestingKnobs) ModuleTestingKnobs() {} // createSchemaChangeEvalCtx creates an extendedEvalContext() to be used for backfills. // // TODO(andrei): This EvalContext() will be broken for backfills trying to use -// functions marked with distsqlBlacklist. +// functions marked with distsqlBlocklist. // Also, the SessionTracing inside the context is unrelated to the one // used in the surrounding SQL session, so session tracing is unable // to capture schema change activity. @@ -1690,7 +1690,7 @@ func (r schemaChangeResumer) Resume( ) return nil case !isPermanentSchemaChangeError(scErr): - // Check if the error is on a whitelist of errors we should retry on, + // Check if the error is on a allowlist of errors we should retry on, // including the schema change not having the first mutation in line. default: // All other errors lead to a failed job. @@ -1808,7 +1808,7 @@ func (r schemaChangeResumer) OnFailOrCancel(ctx context.Context, phs interface{} // wrapping it in a retry error. return rollbackErr case !isPermanentSchemaChangeError(rollbackErr): - // Check if the error is on a whitelist of errors we should retry on, and + // Check if the error is on a allowlist of errors we should retry on, and // have the job registry retry. return jobs.NewRetryJobError(rollbackErr.Error()) default: diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index fd88e00ce3a2..fb98d800e072 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -6000,7 +6000,7 @@ CREATE UNIQUE INDEX i ON t.test(v); return nil } injectedError = true - // Any error not on the whitelist of retriable errors is considered permanent. + // Any error not on the allowlist of retriable errors is considered permanent. return errors.New("permanent error") }, }, @@ -6025,7 +6025,7 @@ CREATE UNIQUE INDEX i ON t.test(v); return nil } injectedError = true - // Any error not on the whitelist of retriable errors is considered permanent. + // Any error not on the allowlist of retriable errors is considered permanent. return errors.New("permanent error") }, }, diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index a5e67f046c8d..11566457770f 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -1735,7 +1735,7 @@ CockroachDB supports the following flags: "nextval": makeBuiltin( tree.FunctionProperties{ Category: categorySequences, - DistsqlBlacklist: true, + DistsqlBlocklist: true, Impure: true, }, tree.Overload{ @@ -1761,7 +1761,7 @@ CockroachDB supports the following flags: "currval": makeBuiltin( tree.FunctionProperties{ Category: categorySequences, - DistsqlBlacklist: true, + DistsqlBlocklist: true, Impure: true, }, tree.Overload{ @@ -1809,7 +1809,7 @@ CockroachDB supports the following flags: "setval": makeBuiltin( tree.FunctionProperties{ Category: categorySequences, - DistsqlBlacklist: true, + DistsqlBlocklist: true, Impure: true, }, tree.Overload{ @@ -3023,7 +3023,7 @@ may increase either contention or retry errors, or both.`, "current_schema": makeBuiltin( tree.FunctionProperties{ Category: categorySystemInfo, - DistsqlBlacklist: true, + DistsqlBlocklist: true, }, tree.Overload{ Types: tree.ArgTypes{}, @@ -3060,7 +3060,7 @@ may increase either contention or retry errors, or both.`, "current_schemas": makeBuiltin( tree.FunctionProperties{ Category: categorySystemInfo, - DistsqlBlacklist: true, + DistsqlBlocklist: true, }, tree.Overload{ Types: tree.ArgTypes{{"include_pg_catalog", types.Bool}}, @@ -3801,7 +3801,7 @@ may increase either contention or retry errors, or both.`, "crdb_internal.is_admin": makeBuiltin( tree.FunctionProperties{ Category: categorySystemInfo, - DistsqlBlacklist: true, + DistsqlBlocklist: true, }, tree.Overload{ Types: tree.ArgTypes{}, diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index 2d1273d027b9..96b2d625d9ce 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -389,7 +389,7 @@ func makePGPrivilegeInquiryDef( } return builtinDefinition{ props: tree.FunctionProperties{ - DistsqlBlacklist: true, + DistsqlBlocklist: true, }, overloads: variants, } @@ -645,7 +645,7 @@ var pgBuiltins = map[string]builtinDefinition{ // pg_get_constraintdef functions like SHOW CREATE CONSTRAINT would if we // supported that statement. - "pg_get_constraintdef": makeBuiltin(tree.FunctionProperties{DistsqlBlacklist: true}, + "pg_get_constraintdef": makeBuiltin(tree.FunctionProperties{DistsqlBlocklist: true}, makePGGetConstraintDef(tree.ArgTypes{ {"constraint_oid", types.Oid}, {"pretty_bool", types.Bool}}), makePGGetConstraintDef(tree.ArgTypes{{"constraint_oid", types.Oid}}), @@ -726,14 +726,14 @@ var pgBuiltins = map[string]builtinDefinition{ // pg_get_indexdef functions like SHOW CREATE INDEX would if we supported that // statement. - "pg_get_indexdef": makeBuiltin(tree.FunctionProperties{DistsqlBlacklist: true}, + "pg_get_indexdef": makeBuiltin(tree.FunctionProperties{DistsqlBlocklist: true}, makePGGetIndexDef(tree.ArgTypes{{"index_oid", types.Oid}}), makePGGetIndexDef(tree.ArgTypes{{"index_oid", types.Oid}, {"column_no", types.Int}, {"pretty_bool", types.Bool}}), ), // pg_get_viewdef functions like SHOW CREATE VIEW but returns the same format as // PostgreSQL leaving out the actual 'CREATE VIEW table_name AS' portion of the statement. - "pg_get_viewdef": makeBuiltin(tree.FunctionProperties{DistsqlBlacklist: true}, + "pg_get_viewdef": makeBuiltin(tree.FunctionProperties{DistsqlBlocklist: true}, makePGGetViewDef(tree.ArgTypes{{"view_oid", types.Oid}}), makePGGetViewDef(tree.ArgTypes{{"view_oid", types.Oid}, {"pretty_bool", types.Bool}}), ), @@ -766,7 +766,7 @@ var pgBuiltins = map[string]builtinDefinition{ }, ), - "pg_get_userbyid": makeBuiltin(tree.FunctionProperties{DistsqlBlacklist: true}, + "pg_get_userbyid": makeBuiltin(tree.FunctionProperties{DistsqlBlocklist: true}, tree.Overload{ Types: tree.ArgTypes{ {"role_oid", types.Oid}, @@ -791,7 +791,7 @@ var pgBuiltins = map[string]builtinDefinition{ }, ), - "pg_sequence_parameters": makeBuiltin(tree.FunctionProperties{DistsqlBlacklist: true}, + "pg_sequence_parameters": makeBuiltin(tree.FunctionProperties{DistsqlBlocklist: true}, // pg_sequence_parameters is an undocumented Postgres builtin that returns // information about a sequence given its OID. It's nevertheless used by // at least one UI tool, so we provide an implementation for compatibility. @@ -1717,7 +1717,7 @@ SELECT description "current_setting": makeBuiltin( tree.FunctionProperties{ Category: categorySystemInfo, - DistsqlBlacklist: true, + DistsqlBlocklist: true, }, tree.Overload{ Types: tree.ArgTypes{{"setting_name", types.String}}, @@ -1743,7 +1743,7 @@ SELECT description "set_config": makeBuiltin( tree.FunctionProperties{ Category: categorySystemInfo, - DistsqlBlacklist: true, + DistsqlBlocklist: true, Impure: true, }, tree.Overload{ diff --git a/pkg/sql/sem/tree/expr.go b/pkg/sql/sem/tree/expr.go index 769164a12281..1b12b32a9430 100644 --- a/pkg/sql/sem/tree/expr.go +++ b/pkg/sql/sem/tree/expr.go @@ -1401,9 +1401,9 @@ func (node *FuncExpr) IsImpure() bool { return node.fnProps != nil && node.fnProps.Impure } -// IsDistSQLBlacklist returns whether the function is not supported by DistSQL. -func (node *FuncExpr) IsDistSQLBlacklist() bool { - return node.fnProps != nil && node.fnProps.DistsqlBlacklist +// IsDistSQLBlocklist returns whether the function is not supported by DistSQL. +func (node *FuncExpr) IsDistSQLBlocklist() bool { + return node.fnProps != nil && node.fnProps.DistsqlBlocklist } // CanHandleNulls returns whether or not the function can handle null diff --git a/pkg/sql/sem/tree/function_definition.go b/pkg/sql/sem/tree/function_definition.go index 7e2203de6f4d..ad2eec54682f 100644 --- a/pkg/sql/sem/tree/function_definition.go +++ b/pkg/sql/sem/tree/function_definition.go @@ -65,15 +65,15 @@ type FunctionProperties struct { // as impure. Impure bool - // DistsqlBlacklist is set to true when a function depends on + // DistsqlBlocklist is set to true when a function depends on // members of the EvalContext that are not marshaled by DistSQL // (e.g. planner). Currently used for DistSQL to determine if // expressions can be evaluated on a different node without sending // over the EvalContext. // // TODO(andrei): Get rid of the planner from the EvalContext and then we can - // get rid of this blacklist. - DistsqlBlacklist bool + // get rid of this blocklist. + DistsqlBlocklist bool // Class is the kind of built-in function (normal/aggregate/window/etc.) Class FunctionClass diff --git a/pkg/sql/telemetry_test.go b/pkg/sql/telemetry_test.go index fd5ccb9d6fd9..de0171eab248 100644 --- a/pkg/sql/telemetry_test.go +++ b/pkg/sql/telemetry_test.go @@ -45,19 +45,19 @@ import ( // Executes SQL statements against the database. Outputs no results on // success. In case of error, outputs the error message. // -// - feature-whitelist +// - feature-allowlist // // The input for this command is not SQL, but a list of regular expressions. -// Tests that follow (until the next feature-whitelist command) will only -// output counters that match a regexp in this white list. +// Tests that follow (until the next feature-allowlist command) will only +// output counters that match a regexp in this allow list. // // - feature-usage, feature-counters // // Executes SQL statements and then outputs the feature counters from the -// white list that have been reported to the diagnostic server. The first +// allowlist that have been reported to the diagnostic server. The first // variant outputs only the names of the counters that changed; the second // variant outputs the counts as well. It is necessary to use -// feature-whitelist before these commands to avoid test flakes (e.g. because +// feature-allowlist before these commands to avoid test flakes (e.g. because // of counters that are changed by looking up descriptors) // // - schema @@ -103,7 +103,7 @@ func TestTelemetry(t *testing.T) { // issued multiple times. runner.Exec(t, "SET CLUSTER SETTING sql.query_cache.enabled = false") - var whitelist featureWhitelist + var allowlist featureAllowlist datadriven.RunTest(t, path, func(t *testing.T, td *datadriven.TestData) string { switch td.Cmd { case "exec": @@ -125,9 +125,9 @@ func TestTelemetry(t *testing.T) { } return buf.String() - case "feature-whitelist": + case "feature-allowlist": var err error - whitelist, err = makeWhitelist(strings.Split(td.Input, "\n")) + allowlist, err = makeAllowlist(strings.Split(td.Input, "\n")) if err != nil { td.Fatalf(t, "error parsing feature regex: %s", err) } @@ -150,8 +150,8 @@ func TestTelemetry(t *testing.T) { // Ignore zero values (shouldn't happen in practice) continue } - if !whitelist.Match(k) { - // Feature key not in whitelist. + if !allowlist.Match(k) { + // Feature key not in allowlist. continue } keys = append(keys, k) @@ -193,10 +193,10 @@ func TestTelemetry(t *testing.T) { }) } -type featureWhitelist []*regexp.Regexp +type featureAllowlist []*regexp.Regexp -func makeWhitelist(strings []string) (featureWhitelist, error) { - w := make(featureWhitelist, len(strings)) +func makeAllowlist(strings []string) (featureAllowlist, error) { + w := make(featureAllowlist, len(strings)) for i := range strings { var err error w[i], err = regexp.Compile("^" + strings[i] + "$") @@ -207,9 +207,9 @@ func makeWhitelist(strings []string) (featureWhitelist, error) { return w, nil } -func (w featureWhitelist) Match(feature string) bool { +func (w featureAllowlist) Match(feature string) bool { if w == nil { - // Unset whitelist matches all counters. + // Unset allowlist matches all counters. return true } for _, r := range w { diff --git a/pkg/sql/testdata/telemetry/error b/pkg/sql/testdata/telemetry/error index acc46d357955..1c7a035a9a5c 100644 --- a/pkg/sql/testdata/telemetry/error +++ b/pkg/sql/testdata/telemetry/error @@ -1,6 +1,6 @@ # This file contains telemetry tests for counters triggered by errors. -feature-whitelist +feature-allowlist othererror.* errorcodes.* unimplemented.* diff --git a/pkg/sql/testdata/telemetry/planning b/pkg/sql/testdata/telemetry/planning index ad25bcc04492..1c9674accacc 100644 --- a/pkg/sql/testdata/telemetry/planning +++ b/pkg/sql/testdata/telemetry/planning @@ -6,7 +6,7 @@ CREATE TABLE x (a INT PRIMARY KEY) # Tests for EXPLAIN counters. -feature-whitelist +feature-allowlist sql.plan.explain sql.plan.explain-analyze sql.plan.explain-opt @@ -45,7 +45,7 @@ sql.plan.explain-opt-verbose # Tests for hints. -feature-whitelist +feature-allowlist sql.plan.hints.* ---- @@ -84,7 +84,7 @@ sql.plan.hints.index.delete # Tests for tracking important setting changes. -feature-whitelist +feature-allowlist sql.plan.reorder-joins.* sql.plan.automatic-stats.* ---- @@ -125,7 +125,7 @@ RESET CLUSTER SETTING sql.stats.automatic_collection.enabled sql.plan.automatic-stats.enabled # Test telemetry for manual statistics creation. -feature-whitelist +feature-allowlist sql.plan.stats.created ---- @@ -135,7 +135,7 @@ CREATE STATISTICS stats FROM x sql.plan.stats.created # Test various planning counters. -feature-whitelist +feature-allowlist sql.plan.cte.* sql.plan.lateral-join sql.plan.subquery.* @@ -174,7 +174,7 @@ sql.plan.subquery.correlated # Test some sql.plan.ops counters, using some esoteric operators unlikely to be # executed in background activity). -feature-whitelist +feature-allowlist sql.plan.ops.cast.string::inet sql.plan.ops.bin.jsonb - string sql.plan.ops.array.* @@ -202,7 +202,7 @@ INSERT INTO x SELECT unnest(ARRAY[9, 10, 11, 12]) sql.plan.ops.array.cons # Test a few sql.plan.opt.node counters. -feature-whitelist +feature-allowlist sql.plan.opt.node.project-set sql.plan.opt.node.join.* ---- diff --git a/pkg/sql/testdata/telemetry/schema b/pkg/sql/testdata/telemetry/schema index 89f8e6ababc4..70f6bea8217a 100644 --- a/pkg/sql/testdata/telemetry/schema +++ b/pkg/sql/testdata/telemetry/schema @@ -36,7 +36,7 @@ table:_ ├── _: _ └── _: _ -feature-whitelist +feature-allowlist sql.schema.* ---- diff --git a/pkg/testutils/buildutil/build.go b/pkg/testutils/buildutil/build.go index 4270b0b22a51..299bd8e7aa1e 100644 --- a/pkg/testutils/buildutil/build.go +++ b/pkg/testutils/buildutil/build.go @@ -38,7 +38,7 @@ func short(in string) string { // indirectly) on forbidden packages. The forbidden packages are specified as // either exact matches or prefix matches. // A match is not reported if the package that includes the forbidden package -// is listed in the whitelist. +// is listed in the allowlist. // If GOPATH isn't set, it is an indication that the source is not available and // the test is skipped. func VerifyNoImports( @@ -46,7 +46,7 @@ func VerifyNoImports( pkgPath string, cgo bool, forbiddenPkgs, forbiddenPrefixes []string, - whitelist ...string, + allowlist ...string, ) { // Skip test if source is not available. @@ -68,14 +68,14 @@ func VerifyNoImports( for _, imp := range pkg.Imports { for _, forbidden := range forbiddenPkgs { if forbidden == imp { - whitelisted := false - for _, w := range whitelist { + allowlisted := false + for _, w := range allowlist { if path == w { - whitelisted = true + allowlisted = true break } } - if !whitelisted { + if !allowlisted { return errors.Errorf("%s imports %s, which is forbidden", short(path), short(imp)) } } @@ -129,10 +129,10 @@ func VerifyNoImports( } } -// VerifyTransitiveWhitelist checks that the entire set of transitive -// dependencies of the given package is in a whitelist. Vendored and stdlib +// VerifyTransitiveAllowlist checks that the entire set of transitive +// dependencies of the given package is in a allowlist. Vendored and stdlib // packages are always allowed. -func VerifyTransitiveWhitelist(t testing.TB, pkg string, allowedPkgs []string) { +func VerifyTransitiveAllowlist(t testing.TB, pkg string, allowedPkgs []string) { // Skip test if source is not available. if build.Default.GOPATH == "" { t.Skip("GOPATH isn't set") diff --git a/pkg/workload/cli/cli.go b/pkg/workload/cli/cli.go index b8a786e70a8d..8d276cf8ca46 100644 --- a/pkg/workload/cli/cli.go +++ b/pkg/workload/cli/cli.go @@ -28,18 +28,18 @@ func WorkloadCmd(userFacing bool) *cobra.Command { rootCmd.AddCommand(subCmdFn(userFacing)) } if userFacing { - whitelist := map[string]struct{}{ + allowlist := map[string]struct{}{ `workload`: {}, `init`: {}, `run`: {}, } for _, m := range workload.Registered() { - whitelist[m.Name] = struct{}{} + allowlist[m.Name] = struct{}{} } var addExperimental func(c *cobra.Command) addExperimental = func(c *cobra.Command) { c.Short = `[experimental] ` + c.Short - if _, ok := whitelist[c.Name()]; !ok { + if _, ok := allowlist[c.Name()]; !ok { c.Hidden = true } for _, sub := range c.Commands() { diff --git a/pkg/workload/dep_test.go b/pkg/workload/dep_test.go index 44449e8d7b0f..3016b803c338 100644 --- a/pkg/workload/dep_test.go +++ b/pkg/workload/dep_test.go @@ -17,12 +17,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) -func TestDepWhitelist(t *testing.T) { +func TestDepAllowlist(t *testing.T) { defer leaktest.AfterTest(t)() // We want workload to be lightweight. If you need to add a package to this // set of deps, run it by danhhz first. - buildutil.VerifyTransitiveWhitelist(t, "github.com/cockroachdb/cockroach/pkg/workload", + buildutil.VerifyTransitiveAllowlist(t, "github.com/cockroachdb/cockroach/pkg/workload", []string{ `github.com/cockroachdb/cockroach/pkg/col/coldata`, `github.com/cockroachdb/cockroach/pkg/col/typeconv`, From e7b0c76aabaf0cec0942facfb7c3d761aaec51af Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Mon, 8 Jun 2020 19:37:26 -0700 Subject: [PATCH 31/46] geo: handle maximum decimal digits for ST_AsText Release note (sql change): Introduce maxDecimalDigits arguments for ST_AsText and ST_AsEWKT, which allow rounding of the decimal digits output in the WKT representation. --- docs/generated/sql/functions.md | 16 ++- pkg/geo/encode.go | 8 +- pkg/geo/encode_test.go | 24 ++-- pkg/geo/geopb/geopb.go | 7 +- .../logictest/testdata/logic_test/geospatial | 135 +++++++++--------- pkg/sql/sem/builtins/geo_builtins.go | 117 +++++++++++++-- 6 files changed, 205 insertions(+), 102 deletions(-) diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 24f8cfb48726..88af3534afd5 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -716,9 +716,13 @@ has no relationship with the commit order of concurrent transactions.

st_asewkb(geometry: geometry) → bytes

Returns the EWKB representation of a given Geometry.

-st_asewkt(geography: geography) → string

Returns the EWKT representation of a given Geography.

+st_asewkt(geography: geography) → string

Returns the EWKT representation of a given Geography. A default of 15 decimal digits is used.

-st_asewkt(geometry: geometry) → string

Returns the EWKT representation of a given Geometry.

+st_asewkt(geography: geography, maximum_decimal_digits: int) → string

Returns the EWKT representation of a given Geography. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the .. Use -1 to print as many digits as possible.

+
+st_asewkt(geometry: geometry) → string

Returns the EWKT representation of a given Geometry. A maximum of 15 decimal digits is used.

+
+st_asewkt(geometry: geometry, maximum_decimal_digits: int) → string

Returns the WKT representation of a given Geometry. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the .. Use -1 to print as many digits as possible.

st_asgeojson(geography: geography) → string

Returns the GeoJSON representation of a given Geography. Coordinates have a maximum of 9 decimal digits.

@@ -764,9 +768,13 @@ has no relationship with the commit order of concurrent transactions.

st_askml(geometry: geometry) → string

Returns the KML representation of a given Geometry.

-st_astext(geography: geography) → string

Returns the WKT representation of a given Geography.

+st_astext(geography: geography) → string

Returns the WKT representation of a given Geography. A default of 15 decimal digits is used.

+
+st_astext(geography: geography, maximum_decimal_digits: int) → string

Returns the WKT representation of a given Geography. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the .. Use -1 to print as many digits as possible.

+
+st_astext(geometry: geometry) → string

Returns the WKT representation of a given Geometry. A maximum of 15 decimal digits is used.

-st_astext(geometry: geometry) → string

Returns the WKT representation of a given Geometry.

+st_astext(geometry: geometry, maximum_decimal_digits: int) → string

Returns the WKT representation of a given Geometry. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the .. Use -1 to print as many digits as possible.

st_buffer(geometry: geometry, distance: float) → geometry

Returns a Geometry that represents all points whose distance is less than or equal to the given distance from the given Geometry.

diff --git a/pkg/geo/encode.go b/pkg/geo/encode.go index 37edd8a010f4..4536bda8e71e 100644 --- a/pkg/geo/encode.go +++ b/pkg/geo/encode.go @@ -29,7 +29,7 @@ import ( ) // EWKBToWKT transforms a given EWKB to WKT. -func EWKBToWKT(b geopb.EWKB) (geopb.WKT, error) { +func EWKBToWKT(b geopb.EWKB, maxDecimalDigits int) (geopb.WKT, error) { // twpayne/go-geom doesn't seem to handle POINT EMPTY just yet. Add this hack in. // Remove after #49209 is resolved. if bytes.Equal(b, []byte{0x01, 0x01, 0x00, 0x00, 0x00}) { @@ -39,12 +39,12 @@ func EWKBToWKT(b geopb.EWKB) (geopb.WKT, error) { if err != nil { return "", err } - ret, err := wkt.Marshal(t) + ret, err := wkt.Marshal(t, wkt.EncodeOptionWithMaxDecimalDigits(maxDecimalDigits)) return geopb.WKT(ret), err } // EWKBToEWKT transforms a given EWKB to EWKT. -func EWKBToEWKT(b geopb.EWKB) (geopb.EWKT, error) { +func EWKBToEWKT(b geopb.EWKB, maxDecimalDigits int) (geopb.EWKT, error) { // twpayne/go-geom doesn't seem to handle POINT EMPTY just yet. Add this hack in. // Remove after #49209 is resolved. if bytes.Equal(b, []byte{0x01, 0x01, 0x00, 0x00, 0x00}) { @@ -54,7 +54,7 @@ func EWKBToEWKT(b geopb.EWKB) (geopb.EWKT, error) { if err != nil { return "", err } - ret, err := wkt.Marshal(t) + ret, err := wkt.Marshal(t, wkt.EncodeOptionWithMaxDecimalDigits(maxDecimalDigits)) if err != nil { return "", err } diff --git a/pkg/geo/encode_test.go b/pkg/geo/encode_test.go index 52534031b15b..662d2a5b19bb 100644 --- a/pkg/geo/encode_test.go +++ b/pkg/geo/encode_test.go @@ -19,18 +19,20 @@ import ( func TestEWKBToWKT(t *testing.T) { testCases := []struct { - ewkt geopb.EWKT - expected geopb.WKT + ewkt geopb.EWKT + maxDecimalDigits int + expected geopb.WKT }{ - {"POINT(1.0 1.0)", "POINT (1 1)"}, - {"SRID=4;POINT(1.0 1.0)", "POINT (1 1)"}, + {"POINT(1.01 1.01)", 15, "POINT (1.01 1.01)"}, + {"POINT(1.01 1.01)", 1, "POINT (1 1)"}, + {"SRID=4;POINT(1.0 1.0)", 15, "POINT (1 1)"}, } for _, tc := range testCases { t.Run(string(tc.ewkt), func(t *testing.T) { so, err := parseEWKT(tc.ewkt, geopb.DefaultGeometrySRID, DefaultSRIDIsHint) require.NoError(t, err) - encoded, err := EWKBToWKT(so.EWKB) + encoded, err := EWKBToWKT(so.EWKB, tc.maxDecimalDigits) require.NoError(t, err) require.Equal(t, tc.expected, encoded) }) @@ -39,18 +41,20 @@ func TestEWKBToWKT(t *testing.T) { func TestEWKBToEWKT(t *testing.T) { testCases := []struct { - ewkt geopb.EWKT - expected geopb.EWKT + ewkt geopb.EWKT + maxDecimalDigits int + expected geopb.EWKT }{ - {"POINT(1.0 1.0)", "POINT (1 1)"}, - {"SRID=4;POINT(1.0 1.0)", "SRID=4;POINT (1 1)"}, + {"POINT(1.01 1.01)", 15, "POINT (1.01 1.01)"}, + {"POINT(1.01 1.01)", 1, "POINT (1 1)"}, + {"SRID=4;POINT(1.0 1.0)", 15, "SRID=4;POINT (1 1)"}, } for _, tc := range testCases { t.Run(string(tc.ewkt), func(t *testing.T) { so, err := parseEWKT(tc.ewkt, geopb.DefaultGeometrySRID, DefaultSRIDIsHint) require.NoError(t, err) - encoded, err := EWKBToEWKT(so.EWKB) + encoded, err := EWKBToEWKT(so.EWKB, tc.maxDecimalDigits) require.NoError(t, err) require.Equal(t, tc.expected, encoded) }) diff --git a/pkg/geo/geopb/geopb.go b/pkg/geo/geopb/geopb.go index 0de9965b58e1..119eee414706 100644 --- a/pkg/geo/geopb/geopb.go +++ b/pkg/geo/geopb/geopb.go @@ -10,12 +10,9 @@ package geopb -import ( - "encoding/hex" - "strings" -) +import "fmt" // EWKBHex returns the EWKB-hex version of this data type func (b *SpatialObject) EWKBHex() string { - return strings.ToUpper(hex.EncodeToString(b.EWKB)) + return fmt.Sprintf("%X", b.EWKB) } diff --git a/pkg/sql/logictest/testdata/logic_test/geospatial b/pkg/sql/logictest/testdata/logic_test/geospatial index 61e295d40be4..885d1eeba8b9 100644 --- a/pkg/sql/logictest/testdata/logic_test/geospatial +++ b/pkg/sql/logictest/testdata/logic_test/geospatial @@ -101,7 +101,7 @@ POINT (3 4) query T SELECT ST_AsText(ST_Project('POINT(0 0)'::geography, 100000, radians(45.0))) ---- -POINT (0.6352310291255374 0.6394723347291977) +POINT (0.635231029125537 0.639472334729198) subtest cast_test @@ -509,16 +509,16 @@ SELECT FROM geom_operators_test a ORDER BY a.dsc ---- -Empty GeometryCollection POINT EMPTY POINT EMPTY -Empty LineString POINT EMPTY POINT EMPTY -Faraway point POINT (5 5) POINT (5 5) -Line going through left and right square POINT (0 0.5) POINT (-0.5 0.5) -NULL NULL NULL -Point middle of Left Square POINT (-0.5 0.5) POINT (-0.5 0.5) -Point middle of Right Square POINT (0.5 0.5) POINT (0.5 0.5) -Square (left) POINT (-0.5 0.5) POINT (-0.5 0.5) -Square (right) POINT (0.5 0.5) POINT (0.5 0.5) -Square overlapping left and right square POINT (0.4499999999999999 0.5) POINT (0.45 0.5) +Empty GeometryCollection POINT EMPTY POINT EMPTY +Empty LineString POINT EMPTY POINT EMPTY +Faraway point POINT (5 5) POINT (5 5) +Line going through left and right square POINT (0 0.5) POINT (-0.5 0.5) +NULL NULL NULL +Point middle of Left Square POINT (-0.5 0.5) POINT (-0.5 0.5) +Point middle of Right Square POINT (0.5 0.5) POINT (0.5 0.5) +Square (left) POINT (-0.5 0.5) POINT (-0.5 0.5) +Square (right) POINT (0.5 0.5) POINT (0.5 0.5) +Square overlapping left and right square POINT (0.45 0.5) POINT (0.45 0.5) # Functions which take in strings as input as well. query TT @@ -537,7 +537,7 @@ Point middle of Left Square POINT (-0.5 0.5) Point middle of Right Square POINT (0.5 0.5) Square (left) POINT (-0.5 0.5) Square (right) POINT (0.5 0.5) -Square overlapping left and right square POINT (0.4499999999999999 0.5) +Square overlapping left and right square POINT (0.45 0.5) # Binary operators query TTRR @@ -883,28 +883,25 @@ Square overlapping left and right square Square (left) Square overlapping left and right square Square (right) true false Square overlapping left and right square Square overlapping left and right square true false -# Buffer -- unfortunately due to floating point precision, these results can be off by small -# epsilon across operating systems. Until ST_AsEWKT with precision is implemented, we'll have to -# verify that it works by checking another statistic for now. -query TIII +query TTTT SELECT a.dsc, - ST_NPoints(ST_Buffer(a.geom, 10)), - ST_NPoints(ST_Buffer(a.geom, 10, 2)), - ST_NPoints(ST_Buffer(a.geom, 10, 'quad_segs=4 endcap=flat')) + ST_AsEWKT(ST_Buffer(a.geom, 10), 5), + ST_AsEWKT(ST_Buffer(a.geom, 10, 2), 5), + ST_AsEWKT(ST_Buffer(a.geom, 10, 'quad_segs=4 endcap=flat'), 5) FROM geom_operators_test a ORDER BY a.dsc ---- -Empty GeometryCollection 0 0 0 -Empty LineString 0 0 0 -Faraway point 33 9 0 -Line going through left and right square 35 11 5 -NULL NULL NULL NULL -Point middle of Left Square 33 9 0 -Point middle of Right Square 33 9 0 -Square (left) 37 13 21 -Square (right) 37 13 21 -Square overlapping left and right square 37 13 21 +Empty GeometryCollection POLYGON EMPTY POLYGON EMPTY POLYGON EMPTY +Empty LineString POLYGON EMPTY POLYGON EMPTY POLYGON EMPTY +Faraway point POLYGON ((15 5, 14.80785 3.0491, 14.2388 1.17317, 13.3147 -0.5557, 12.07107 -2.07107, 10.5557 -3.3147, 8.82683 -4.2388, 6.9509 -4.80785, 5 -5, 3.0491 -4.80785, 1.17317 -4.2388, -0.5557 -3.3147, -2.07107 -2.07107, -3.3147 -0.5557, -4.2388 1.17317, -4.80785 3.0491, -5 5, -4.80785 6.9509, -4.2388 8.82683, -3.3147 10.5557, -2.07107 12.07107, -0.5557 13.3147, 1.17317 14.2388, 3.0491 14.80785, 5 15, 6.9509 14.80785, 8.82683 14.2388, 10.5557 13.3147, 12.07107 12.07107, 13.3147 10.5557, 14.2388 8.82683, 14.80785 6.9509, 15 5)) POLYGON ((15 5, 12.07107 -2.07107, 5 -5, -2.07107 -2.07107, -5 5, -2.07107 12.07107, 5 15, 12.07107 12.07107, 15 5)) POLYGON EMPTY +Line going through left and right square POLYGON ((0.5 10.5, 2.4509 10.30785, 4.32683 9.7388, 6.0557 8.8147, 7.57107 7.57107, 8.8147 6.0557, 9.7388 4.32683, 10.30785 2.4509, 10.5 0.5, 10.30785 -1.4509, 9.7388 -3.32683, 8.8147 -5.0557, 7.57107 -6.57107, 6.0557 -7.8147, 4.32683 -8.7388, 2.4509 -9.30785, 0.5 -9.5, -0.5 -9.5, -2.4509 -9.30785, -4.32683 -8.7388, -6.0557 -7.8147, -7.57107 -6.57107, -8.8147 -5.0557, -9.7388 -3.32683, -10.30785 -1.4509, -10.5 0.5, -10.30785 2.4509, -9.7388 4.32683, -8.8147 6.0557, -7.57107 7.57107, -6.0557 8.8147, -4.32683 9.7388, -2.4509 10.30785, -0.5 10.5, 0.5 10.5)) POLYGON ((0.5 10.5, 7.57107 7.57107, 10.5 0.5, 7.57107 -6.57107, 0.5 -9.5, -0.5 -9.5, -7.57107 -6.57107, -10.5 0.5, -7.57107 7.57107, -0.5 10.5, 0.5 10.5)) POLYGON ((0.5 10.5, 0.5 -9.5, -0.5 -9.5, -0.5 10.5, 0.5 10.5)) +NULL NULL NULL NULL +Point middle of Left Square POLYGON ((9.5 0.5, 9.30785 -1.4509, 8.7388 -3.32683, 7.8147 -5.0557, 6.57107 -6.57107, 5.0557 -7.8147, 3.32683 -8.7388, 1.4509 -9.30785, -0.5 -9.5, -2.4509 -9.30785, -4.32683 -8.7388, -6.0557 -7.8147, -7.57107 -6.57107, -8.8147 -5.0557, -9.7388 -3.32683, -10.30785 -1.4509, -10.5 0.5, -10.30785 2.4509, -9.7388 4.32683, -8.8147 6.0557, -7.57107 7.57107, -6.0557 8.8147, -4.32683 9.7388, -2.4509 10.30785, -0.5 10.5, 1.4509 10.30785, 3.32683 9.7388, 5.0557 8.8147, 6.57107 7.57107, 7.8147 6.0557, 8.7388 4.32683, 9.30785 2.4509, 9.5 0.5)) POLYGON ((9.5 0.5, 6.57107 -6.57107, -0.5 -9.5, -7.57107 -6.57107, -10.5 0.5, -7.57107 7.57107, -0.5 10.5, 6.57107 7.57107, 9.5 0.5)) POLYGON EMPTY +Point middle of Right Square POLYGON ((10.5 0.5, 10.30785 -1.4509, 9.7388 -3.32683, 8.8147 -5.0557, 7.57107 -6.57107, 6.0557 -7.8147, 4.32683 -8.7388, 2.4509 -9.30785, 0.5 -9.5, -1.4509 -9.30785, -3.32683 -8.7388, -5.0557 -7.8147, -6.57107 -6.57107, -7.8147 -5.0557, -8.7388 -3.32683, -9.30785 -1.4509, -9.5 0.5, -9.30785 2.4509, -8.7388 4.32683, -7.8147 6.0557, -6.57107 7.57107, -5.0557 8.8147, -3.32683 9.7388, -1.4509 10.30785, 0.5 10.5, 2.4509 10.30785, 4.32683 9.7388, 6.0557 8.8147, 7.57107 7.57107, 8.8147 6.0557, 9.7388 4.32683, 10.30785 2.4509, 10.5 0.5)) POLYGON ((10.5 0.5, 7.57107 -6.57107, 0.5 -9.5, -6.57107 -6.57107, -9.5 0.5, -6.57107 7.57107, 0.5 10.5, 7.57107 7.57107, 10.5 0.5)) POLYGON EMPTY +Square (left) POLYGON ((-11 0, -11 1, -10.80785 2.9509, -10.2388 4.82683, -9.3147 6.5557, -8.07107 8.07107, -6.5557 9.3147, -4.82683 10.2388, -2.9509 10.80785, -1 11, 0 11, 1.9509 10.80785, 3.82683 10.2388, 5.5557 9.3147, 7.07107 8.07107, 8.3147 6.5557, 9.2388 4.82683, 9.80785 2.9509, 10 1, 10 0, 9.80785 -1.9509, 9.2388 -3.82683, 8.3147 -5.5557, 7.07107 -7.07107, 5.5557 -8.3147, 3.82683 -9.2388, 1.9509 -9.80785, 0 -10, -1 -10, -2.9509 -9.80785, -4.82683 -9.2388, -6.5557 -8.3147, -8.07107 -7.07107, -9.3147 -5.5557, -10.2388 -3.82683, -10.80785 -1.9509, -11 0)) POLYGON ((-11 0, -11 1, -8.07107 8.07107, -1 11, 0 11, 7.07107 8.07107, 10 1, 10 0, 7.07107 -7.07107, 0 -10, -1 -10, -8.07107 -7.07107, -11 0)) POLYGON ((-11 0, -11 1, -10.2388 4.82683, -8.07107 8.07107, -4.82683 10.2388, -1 11, 0 11, 3.82683 10.2388, 7.07107 8.07107, 9.2388 4.82683, 10 1, 10 0, 9.2388 -3.82683, 7.07107 -7.07107, 3.82683 -9.2388, 0 -10, -1 -10, -4.82683 -9.2388, -8.07107 -7.07107, -10.2388 -3.82683, -11 0)) +Square (right) POLYGON ((-10 0, -10 1, -9.80785 2.9509, -9.2388 4.82683, -8.3147 6.5557, -7.07107 8.07107, -5.5557 9.3147, -3.82683 10.2388, -1.9509 10.80785, 0 11, 1 11, 2.9509 10.80785, 4.82683 10.2388, 6.5557 9.3147, 8.07107 8.07107, 9.3147 6.5557, 10.2388 4.82683, 10.80785 2.9509, 11 1, 11 0, 10.80785 -1.9509, 10.2388 -3.82683, 9.3147 -5.5557, 8.07107 -7.07107, 6.5557 -8.3147, 4.82683 -9.2388, 2.9509 -9.80785, 1 -10, 0 -10, -1.9509 -9.80785, -3.82683 -9.2388, -5.5557 -8.3147, -7.07107 -7.07107, -8.3147 -5.5557, -9.2388 -3.82683, -9.80785 -1.9509, -10 0)) POLYGON ((-10 0, -10 1, -7.07107 8.07107, 0 11, 1 11, 8.07107 8.07107, 11 1, 11 0, 8.07107 -7.07107, 1 -10, 0 -10, -7.07107 -7.07107, -10 0)) POLYGON ((-10 0, -10 1, -9.2388 4.82683, -7.07107 8.07107, -3.82683 10.2388, 0 11, 1 11, 4.82683 10.2388, 8.07107 8.07107, 10.2388 4.82683, 11 1, 11 0, 10.2388 -3.82683, 8.07107 -7.07107, 4.82683 -9.2388, 1 -10, 0 -10, -3.82683 -9.2388, -7.07107 -7.07107, -9.2388 -3.82683, -10 0)) +Square overlapping left and right square POLYGON ((-10.1 0, -10.1 1, -9.90785 2.9509, -9.3388 4.82683, -8.4147 6.5557, -7.17107 8.07107, -5.6557 9.3147, -3.92683 10.2388, -2.0509 10.80785, -0.1 11, 1 11, 2.9509 10.80785, 4.82683 10.2388, 6.5557 9.3147, 8.07107 8.07107, 9.3147 6.5557, 10.2388 4.82683, 10.80785 2.9509, 11 1, 11 0, 10.80785 -1.9509, 10.2388 -3.82683, 9.3147 -5.5557, 8.07107 -7.07107, 6.5557 -8.3147, 4.82683 -9.2388, 2.9509 -9.80785, 1 -10, -0.1 -10, -2.0509 -9.80785, -3.92683 -9.2388, -5.6557 -8.3147, -7.17107 -7.07107, -8.4147 -5.5557, -9.3388 -3.82683, -9.90785 -1.9509, -10.1 0)) POLYGON ((-10.1 0, -10.1 1, -7.17107 8.07107, -0.1 11, 1 11, 8.07107 8.07107, 11 1, 11 0, 8.07107 -7.07107, 1 -10, -0.1 -10, -7.17107 -7.07107, -10.1 0)) POLYGON ((-10.1 0, -10.1 1, -9.3388 4.82683, -7.17107 8.07107, -3.92683 10.2388, -0.1 11, 1 11, 4.82683 10.2388, 8.07107 8.07107, 10.2388 4.82683, 11 1, 11 0, 10.2388 -3.82683, 8.07107 -7.07107, 4.82683 -9.2388, 1 -10, -0.1 -10, -3.92683 -9.2388, -7.17107 -7.07107, -9.3388 -3.82683, -10.1 0)) # Test raw string with ST_Buffer query I @@ -1541,16 +1538,16 @@ SELECT FROM geog_operators_test ORDER BY dsc ---- -Empty GeometryCollection GEOMETRYCOLLECTION EMPTY GEOMETRYCOLLECTION EMPTY -Empty LineString LINESTRING EMPTY LINESTRING EMPTY -Faraway point POINT (5 5) POINT (5 5) -Line going through left and right square LINESTRING (-0.5 0.5, -0.00000000000000009939611878359099 0.5000190382262164, 0.5 0.5) LINESTRING (-0.5 0.5, -0.25000000036247944 0.500014278647005, -0.00000000000000009939611878359099 0.5000190382262164, 0.2500000003624792 0.5000142786470051, 0.5 0.5) -NULL NULL NULL -Point middle of Left Square POINT (-0.5 0.5) POINT (-0.5 0.5) -Point middle of Right Square POINT (0.5 0.5) POINT (0.5 0.5) -Square (left) POLYGON ((-1 0, -0.5000000000000001 0, 0 0, 0 0.5, 0 1, -0.4999999999999998 1.0000380706528733, -1 1, -0.9999999999999998 0.5000000000000001, -1 0)) POLYGON ((-1 0, -0.7499999999999998 0, -0.5000000000000001 0, -0.2499999999999997 0, 0 0, 0 0.25, 0 0.5, 0 0.75, 0 1, -0.2499999985501929 1.0000285529443267, -0.4999999999999998 1.0000380706528733, -0.7500000014498067 1.0000285529443265, -1 1, -1 0.7499999999999998, -0.9999999999999998 0.5000000000000001, -0.9999999999999998 0.25, -1 0)) -Square (right) POLYGON ((0 0, 0.5 0, 1 0, 1 0.4999999999999999, 1 1, 0.5 1.0000380706528733, 0 1, 0 0.5000000000000001, 0 0)) POLYGON ((0 0, 0.25 0, 0.5 0, 0.75 0, 1 0, 0.9999999999999998 0.25, 1 0.4999999999999999, 0.9999999999999998 0.7499999999999999, 1 1, 0.750000001449807 1.0000285529443267, 0.5 1.0000380706528733, 0.2499999985501931 1.0000285529443267, 0 1, 0 0.7499999999999998, 0 0.5000000000000001, 0 0.2499999999999997, 0 0)) -Square overlapping left and right square POLYGON ((-0.1 0, 0.44999999999999996 0, 1 0, 1 0.4999999999999999, 1 1, 0.44999999999999996 1.0000460657968335, -0.1 1, -0.1 0.5000000000000001, -0.1 0)) POLYGON ((-0.1 0, 0.17500000000000007 0, 0.44999999999999996 0, 0.7249999999999999 0, 1 0, 0.9999999999999998 0.25, 1 0.4999999999999999, 0.9999999999999998 0.7499999999999999, 1 1, 0.7250000019297163 1.0000345492812595, 0.44999999999999996 1.0000460657968335, 0.17499999807028374 1.0000345492812592, -0.1 1, -0.1 0.75, -0.1 0.5000000000000001, -0.10000000000000002 0.2499999999999997, -0.1 0)) +Empty GeometryCollection GEOMETRYCOLLECTION EMPTY GEOMETRYCOLLECTION EMPTY +Empty LineString LINESTRING EMPTY LINESTRING EMPTY +Faraway point POINT (5 5) POINT (5 5) +Line going through left and right square LINESTRING (-0.5 0.5, -0 0.500019038226216, 0.5 0.5) LINESTRING (-0.5 0.5, -0.250000000362479 0.500014278647005, -0 0.500019038226216, 0.250000000362479 0.500014278647005, 0.5 0.5) +NULL NULL NULL +Point middle of Left Square POINT (-0.5 0.5) POINT (-0.5 0.5) +Point middle of Right Square POINT (0.5 0.5) POINT (0.5 0.5) +Square (left) POLYGON ((-1 0, -0.5 0, 0 0, 0 0.5, 0 1, -0.5 1.000038070652873, -1 1, -1 0.5, -1 0)) POLYGON ((-1 0, -0.75 0, -0.5 0, -0.25 0, 0 0, 0 0.25, 0 0.5, 0 0.75, 0 1, -0.249999998550193 1.000028552944327, -0.5 1.000038070652873, -0.750000001449807 1.000028552944326, -1 1, -1 0.75, -1 0.5, -1 0.25, -1 0)) +Square (right) POLYGON ((0 0, 0.5 0, 1 0, 1 0.5, 1 1, 0.5 1.000038070652873, 0 1, 0 0.5, 0 0)) POLYGON ((0 0, 0.25 0, 0.5 0, 0.75 0, 1 0, 1 0.25, 1 0.5, 1 0.75, 1 1, 0.750000001449807 1.000028552944327, 0.5 1.000038070652873, 0.249999998550193 1.000028552944327, 0 1, 0 0.75, 0 0.5, 0 0.25, 0 0)) +Square overlapping left and right square POLYGON ((-0.1 0, 0.45 0, 1 0, 1 0.5, 1 1, 0.45 1.000046065796834, -0.1 1, -0.1 0.5, -0.1 0)) POLYGON ((-0.1 0, 0.175 0, 0.45 0, 0.725 0, 1 0, 1 0.25, 1 0.5, 1 0.75, 1 1, 0.725000001929716 1.000034549281259, 0.45 1.000046065796834, 0.174999998070284 1.000034549281259, -0.1 1, -0.1 0.75, -0.1 0.5, -0.1 0.25, -0.1 0)) query T SELECT ST_AsText(ST_Segmentize('MULTIPOINT (0 0, 1 1)'::geography, -1)) @@ -1817,36 +1814,36 @@ JOIN (VALUES (0.0), (0.2), (0.5), (0.51), (1.0)) b(fraction) ON (1=1) JOIN (VALUES (true), (false)) c(repeat) ON (1=1) ORDER BY a.dsc, b.fraction, c.repeat ---- -Empty LineString 0.0 false POINT EMPTY POINT EMPTY POINT EMPTY -Empty LineString 0.0 true POINT EMPTY POINT EMPTY POINT EMPTY -Empty LineString 0.2 false POINT EMPTY POINT EMPTY POINT EMPTY -Empty LineString 0.2 true POINT EMPTY POINT EMPTY POINT EMPTY -Empty LineString 0.5 false POINT EMPTY POINT EMPTY POINT EMPTY -Empty LineString 0.5 true POINT EMPTY POINT EMPTY POINT EMPTY -Empty LineString 0.51 false POINT EMPTY POINT EMPTY POINT EMPTY -Empty LineString 0.51 true POINT EMPTY POINT EMPTY POINT EMPTY -Empty LineString 1.0 false POINT EMPTY POINT EMPTY POINT EMPTY -Empty LineString 1.0 true POINT EMPTY POINT EMPTY POINT EMPTY -LineString anticlockwise covering all the quadrants 0.0 false POINT (1 -1) POINT (1 -1) POINT (1 -1) -LineString anticlockwise covering all the quadrants 0.0 true POINT (1 -1) POINT (1 -1) POINT (1 -1) -LineString anticlockwise covering all the quadrants 0.2 false POINT (1.6529822128134706 0.9589466384404113) MULTIPOINT (1.6529822128134706 0.9589466384404113, 1.032455532033675 2, -1.0324555320336777 2, -1.65298221281347 0.9589466384404097, -1 -1) POINT (1.6529822128134706 0.9589466384404113) -LineString anticlockwise covering all the quadrants 0.2 true POINT (1.6529822128134706 0.9589466384404113) MULTIPOINT (1.6529822128134706 0.9589466384404113, 1.032455532033675 2, -1.0324555320336777 2, -1.65298221281347 0.9589466384404097, -1 -1) MULTIPOINT (1.6529822128134706 0.9589466384404113, 1.032455532033675 2, -1.0324555320336777 2, -1.65298221281347 0.9589466384404097, -1 -1) -LineString anticlockwise covering all the quadrants 0.5 false POINT (-0.0000000000000004440892098500626 2) MULTIPOINT (-0.0000000000000004440892098500626 2, -1 -1) POINT (-0.0000000000000004440892098500626 2) -LineString anticlockwise covering all the quadrants 0.5 true POINT (-0.0000000000000004440892098500626 2) MULTIPOINT (-0.0000000000000004440892098500626 2, -1 -1) MULTIPOINT (-0.0000000000000004440892098500626 2, -1 -1) -LineString anticlockwise covering all the quadrants 0.51 false POINT (-0.10324555320336826 2) POINT (-0.10324555320336826 2) POINT (-0.10324555320336826 2) -LineString anticlockwise covering all the quadrants 0.51 true POINT (-0.10324555320336826 2) POINT (-0.10324555320336826 2) POINT (-0.10324555320336826 2) -LineString anticlockwise covering all the quadrants 1.0 false POINT (-1 -1) POINT (-1 -1) POINT (-1 -1) -LineString anticlockwise covering all the quadrants 1.0 true POINT (-1 -1) POINT (-1 -1) POINT (-1 -1) -LineString clockwise covering all the quadrants with SRID 4004 0.0 false SRID=4004;POINT (1 -1) SRID=4004;POINT (1 -1) SRID=4004;POINT (1 -1) -LineString clockwise covering all the quadrants with SRID 4004 0.0 true SRID=4004;POINT (1 -1) SRID=4004;POINT (1 -1) SRID=4004;POINT (1 -1) -LineString clockwise covering all the quadrants with SRID 4004 0.2 false SRID=4004;POINT (-0.8324555320336762 -1) SRID=4004;MULTIPOINT (-0.8324555320336762 -1, -1.5264911064067355 0.579473319220206, -1.6649110640673515 2, 0.16754446796632472 2, 2 2) SRID=4004;POINT (-0.8324555320336762 -1) -LineString clockwise covering all the quadrants with SRID 4004 0.2 true SRID=4004;POINT (-0.8324555320336762 -1) SRID=4004;MULTIPOINT (-0.8324555320336762 -1, -1.5264911064067355 0.579473319220206, -1.6649110640673515 2, 0.16754446796632472 2, 2 2) SRID=4004;MULTIPOINT (-0.8324555320336762 -1, -1.5264911064067355 0.579473319220206, -1.6649110640673515 2, 0.16754446796632472 2, 2 2) -LineString clockwise covering all the quadrants with SRID 4004 0.5 false SRID=4004;POINT (-1.816227766016838 1.448683298050514) SRID=4004;MULTIPOINT (-1.816227766016838 1.448683298050514, 2 2) SRID=4004;POINT (-1.816227766016838 1.448683298050514) -LineString clockwise covering all the quadrants with SRID 4004 0.5 true SRID=4004;POINT (-1.816227766016838 1.448683298050514) SRID=4004;MULTIPOINT (-1.816227766016838 1.448683298050514, 2 2) SRID=4004;MULTIPOINT (-1.816227766016838 1.448683298050514, 2 2) -LineString clockwise covering all the quadrants with SRID 4004 0.51 false SRID=4004;POINT (-1.845201431977848 1.5356042959335445) SRID=4004;POINT (-1.845201431977848 1.5356042959335445) SRID=4004;POINT (-1.845201431977848 1.5356042959335445) -LineString clockwise covering all the quadrants with SRID 4004 0.51 true SRID=4004;POINT (-1.845201431977848 1.5356042959335445) SRID=4004;POINT (-1.845201431977848 1.5356042959335445) SRID=4004;POINT (-1.845201431977848 1.5356042959335445) -LineString clockwise covering all the quadrants with SRID 4004 1.0 false SRID=4004;POINT (2 2) SRID=4004;POINT (2 2) SRID=4004;POINT (2 2) -LineString clockwise covering all the quadrants with SRID 4004 1.0 true SRID=4004;POINT (2 2) SRID=4004;POINT (2 2) SRID=4004;POINT (2 2) +Empty LineString 0.0 false POINT EMPTY POINT EMPTY POINT EMPTY +Empty LineString 0.0 true POINT EMPTY POINT EMPTY POINT EMPTY +Empty LineString 0.2 false POINT EMPTY POINT EMPTY POINT EMPTY +Empty LineString 0.2 true POINT EMPTY POINT EMPTY POINT EMPTY +Empty LineString 0.5 false POINT EMPTY POINT EMPTY POINT EMPTY +Empty LineString 0.5 true POINT EMPTY POINT EMPTY POINT EMPTY +Empty LineString 0.51 false POINT EMPTY POINT EMPTY POINT EMPTY +Empty LineString 0.51 true POINT EMPTY POINT EMPTY POINT EMPTY +Empty LineString 1.0 false POINT EMPTY POINT EMPTY POINT EMPTY +Empty LineString 1.0 true POINT EMPTY POINT EMPTY POINT EMPTY +LineString anticlockwise covering all the quadrants 0.0 false POINT (1 -1) POINT (1 -1) POINT (1 -1) +LineString anticlockwise covering all the quadrants 0.0 true POINT (1 -1) POINT (1 -1) POINT (1 -1) +LineString anticlockwise covering all the quadrants 0.2 false POINT (1.652982212813471 0.958946638440411) MULTIPOINT (1.652982212813471 0.958946638440411, 1.032455532033675 2, -1.032455532033678 2, -1.65298221281347 0.95894663844041, -1 -1) POINT (1.652982212813471 0.958946638440411) +LineString anticlockwise covering all the quadrants 0.2 true POINT (1.652982212813471 0.958946638440411) MULTIPOINT (1.652982212813471 0.958946638440411, 1.032455532033675 2, -1.032455532033678 2, -1.65298221281347 0.95894663844041, -1 -1) MULTIPOINT (1.652982212813471 0.958946638440411, 1.032455532033675 2, -1.032455532033678 2, -1.65298221281347 0.95894663844041, -1 -1) +LineString anticlockwise covering all the quadrants 0.5 false POINT (-0 2) MULTIPOINT (-0 2, -1 -1) POINT (-0 2) +LineString anticlockwise covering all the quadrants 0.5 true POINT (-0 2) MULTIPOINT (-0 2, -1 -1) MULTIPOINT (-0 2, -1 -1) +LineString anticlockwise covering all the quadrants 0.51 false POINT (-0.103245553203368 2) POINT (-0.103245553203368 2) POINT (-0.103245553203368 2) +LineString anticlockwise covering all the quadrants 0.51 true POINT (-0.103245553203368 2) POINT (-0.103245553203368 2) POINT (-0.103245553203368 2) +LineString anticlockwise covering all the quadrants 1.0 false POINT (-1 -1) POINT (-1 -1) POINT (-1 -1) +LineString anticlockwise covering all the quadrants 1.0 true POINT (-1 -1) POINT (-1 -1) POINT (-1 -1) +LineString clockwise covering all the quadrants with SRID 4004 0.0 false SRID=4004;POINT (1 -1) SRID=4004;POINT (1 -1) SRID=4004;POINT (1 -1) +LineString clockwise covering all the quadrants with SRID 4004 0.0 true SRID=4004;POINT (1 -1) SRID=4004;POINT (1 -1) SRID=4004;POINT (1 -1) +LineString clockwise covering all the quadrants with SRID 4004 0.2 false SRID=4004;POINT (-0.832455532033676 -1) SRID=4004;MULTIPOINT (-0.832455532033676 -1, -1.526491106406735 0.579473319220206, -1.664911064067351 2, 0.167544467966325 2, 2 2) SRID=4004;POINT (-0.832455532033676 -1) +LineString clockwise covering all the quadrants with SRID 4004 0.2 true SRID=4004;POINT (-0.832455532033676 -1) SRID=4004;MULTIPOINT (-0.832455532033676 -1, -1.526491106406735 0.579473319220206, -1.664911064067351 2, 0.167544467966325 2, 2 2) SRID=4004;MULTIPOINT (-0.832455532033676 -1, -1.526491106406735 0.579473319220206, -1.664911064067351 2, 0.167544467966325 2, 2 2) +LineString clockwise covering all the quadrants with SRID 4004 0.5 false SRID=4004;POINT (-1.816227766016838 1.448683298050514) SRID=4004;MULTIPOINT (-1.816227766016838 1.448683298050514, 2 2) SRID=4004;POINT (-1.816227766016838 1.448683298050514) +LineString clockwise covering all the quadrants with SRID 4004 0.5 true SRID=4004;POINT (-1.816227766016838 1.448683298050514) SRID=4004;MULTIPOINT (-1.816227766016838 1.448683298050514, 2 2) SRID=4004;MULTIPOINT (-1.816227766016838 1.448683298050514, 2 2) +LineString clockwise covering all the quadrants with SRID 4004 0.51 false SRID=4004;POINT (-1.845201431977848 1.535604295933545) SRID=4004;POINT (-1.845201431977848 1.535604295933545) SRID=4004;POINT (-1.845201431977848 1.535604295933545) +LineString clockwise covering all the quadrants with SRID 4004 0.51 true SRID=4004;POINT (-1.845201431977848 1.535604295933545) SRID=4004;POINT (-1.845201431977848 1.535604295933545) SRID=4004;POINT (-1.845201431977848 1.535604295933545) +LineString clockwise covering all the quadrants with SRID 4004 1.0 false SRID=4004;POINT (2 2) SRID=4004;POINT (2 2) SRID=4004;POINT (2 2) +LineString clockwise covering all the quadrants with SRID 4004 1.0 true SRID=4004;POINT (2 2) SRID=4004;POINT (2 2) SRID=4004;POINT (2 2) statement error st_lineinterpolatepoint\(\): fraction -1.000000 should be within \[0 1\] range SELECT ST_LineInterpolatePoint('LINESTRING (0 0, 1 1)'::geometry, -1) diff --git a/pkg/sql/sem/builtins/geo_builtins.go b/pkg/sql/sem/builtins/geo_builtins.go index 8a28170471f2..3f2b13ca114d 100644 --- a/pkg/sql/sem/builtins/geo_builtins.go +++ b/pkg/sql/sem/builtins/geo_builtins.go @@ -50,6 +50,7 @@ const spheroidDistanceMessage = `"\n\nWhen operating on a spheroid, this functio const ( defaultGeoJSONDecimalDigits = 9 + defaultWKTDecimalDigits = 15 ) // infoBuilder is used to build a detailed info string that is consistent between @@ -595,43 +596,139 @@ var geoBuiltins = map[string]builtinDefinition{ defProps(), geometryOverload1( func(_ *tree.EvalContext, g *tree.DGeometry) (tree.Datum, error) { - wkt, err := geo.EWKBToWKT(g.Geometry.EWKB()) + wkt, err := geo.EWKBToWKT(g.Geometry.EWKB(), defaultWKTDecimalDigits) return tree.NewDString(string(wkt)), err }, types.String, - infoBuilder{info: "Returns the WKT representation of a given Geometry."}, + infoBuilder{ + info: fmt.Sprintf("Returns the WKT representation of a given Geometry. A maximum of %d decimal digits is used.", defaultWKTDecimalDigits), + }, tree.VolatilityImmutable, ), + tree.Overload{ + Types: tree.ArgTypes{ + {"geometry", types.Geometry}, + {"maximum_decimal_digits", types.Int}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + g := args[0].(*tree.DGeometry) + maxDecimalDigits := int(tree.MustBeDInt(args[1])) + + if maxDecimalDigits < -1 { + return nil, errors.Newf("maximum_decimal_digits must be >= -1") + } + + wkt, err := geo.EWKBToWKT(g.Geometry.EWKB(), maxDecimalDigits) + return tree.NewDString(string(wkt)), err + }, + Info: infoBuilder{ + info: "Returns the WKT representation of a given Geometry. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the `.`. Use -1 to print as many digits as possible.", + }.String(), + Volatility: tree.VolatilityImmutable, + }, geographyOverload1( func(_ *tree.EvalContext, g *tree.DGeography) (tree.Datum, error) { - wkt, err := geo.EWKBToWKT(g.Geography.EWKB()) + wkt, err := geo.EWKBToWKT(g.Geography.EWKB(), defaultWKTDecimalDigits) return tree.NewDString(string(wkt)), err }, types.String, - infoBuilder{info: "Returns the WKT representation of a given Geography."}, + infoBuilder{ + info: fmt.Sprintf("Returns the WKT representation of a given Geography. A default of %d decimal digits is used.", defaultWKTDecimalDigits), + }, tree.VolatilityImmutable, ), + tree.Overload{ + Types: tree.ArgTypes{ + {"geography", types.Geography}, + {"maximum_decimal_digits", types.Int}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + g := args[0].(*tree.DGeography) + maxDecimalDigits := int(tree.MustBeDInt(args[1])) + + if maxDecimalDigits < -1 { + return nil, errors.Newf("maximum_decimal_digits must be >= -1") + } + + wkt, err := geo.EWKBToWKT(g.Geography.EWKB(), maxDecimalDigits) + return tree.NewDString(string(wkt)), err + }, + Info: infoBuilder{ + info: "Returns the WKT representation of a given Geography. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the `.`. Use -1 to print as many digits as possible.", + }.String(), + Volatility: tree.VolatilityImmutable, + }, ), "st_asewkt": makeBuiltin( defProps(), geometryOverload1( func(_ *tree.EvalContext, g *tree.DGeometry) (tree.Datum, error) { - ewkt, err := geo.EWKBToEWKT(g.Geometry.EWKB()) + ewkt, err := geo.EWKBToEWKT(g.Geometry.EWKB(), defaultWKTDecimalDigits) return tree.NewDString(string(ewkt)), err }, types.String, - infoBuilder{info: "Returns the EWKT representation of a given Geometry."}, + infoBuilder{ + info: fmt.Sprintf("Returns the EWKT representation of a given Geometry. A maximum of %d decimal digits is used.", defaultWKTDecimalDigits), + }, tree.VolatilityImmutable, ), + tree.Overload{ + Types: tree.ArgTypes{ + {"geometry", types.Geometry}, + {"maximum_decimal_digits", types.Int}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + g := args[0].(*tree.DGeometry) + maxDecimalDigits := int(tree.MustBeDInt(args[1])) + + if maxDecimalDigits < -1 { + return nil, errors.Newf("maximum_decimal_digits must be >= -1") + } + + ewkt, err := geo.EWKBToEWKT(g.Geometry.EWKB(), maxDecimalDigits) + return tree.NewDString(string(ewkt)), err + }, + Info: infoBuilder{ + info: "Returns the WKT representation of a given Geometry. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the `.`. Use -1 to print as many digits as possible.", + }.String(), + Volatility: tree.VolatilityImmutable, + }, geographyOverload1( func(_ *tree.EvalContext, g *tree.DGeography) (tree.Datum, error) { - ewkt, err := geo.EWKBToEWKT(g.Geography.EWKB()) + ewkt, err := geo.EWKBToEWKT(g.Geography.EWKB(), defaultWKTDecimalDigits) return tree.NewDString(string(ewkt)), err }, types.String, - infoBuilder{info: "Returns the EWKT representation of a given Geography."}, + infoBuilder{ + info: fmt.Sprintf("Returns the EWKT representation of a given Geography. A default of %d decimal digits is used.", defaultWKTDecimalDigits), + }, tree.VolatilityImmutable, ), + tree.Overload{ + Types: tree.ArgTypes{ + {"geography", types.Geography}, + {"maximum_decimal_digits", types.Int}, + }, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + g := args[0].(*tree.DGeography) + maxDecimalDigits := int(tree.MustBeDInt(args[1])) + + if maxDecimalDigits < -1 { + return nil, errors.Newf("maximum_decimal_digits must be >= -1") + } + + ewkt, err := geo.EWKBToEWKT(g.Geography.EWKB(), maxDecimalDigits) + return tree.NewDString(string(ewkt)), err + }, + Info: infoBuilder{ + info: "Returns the EWKT representation of a given Geography. The maximum_decimal_digits parameter controls the maximum decimal digits to print after the `.`. Use -1 to print as many digits as possible.", + }.String(), + Volatility: tree.VolatilityImmutable, + }, ), "st_asbinary": makeBuiltin( defProps(), @@ -736,7 +833,7 @@ var geoBuiltins = map[string]builtinDefinition{ defProps(), geometryOverload1( func(_ *tree.EvalContext, g *tree.DGeometry) (tree.Datum, error) { - return tree.NewDString(strings.ToUpper(fmt.Sprintf("%x", g.EWKB()))), nil + return tree.NewDString(g.Geometry.EWKBHex()), nil }, types.String, infoBuilder{info: "Returns the EWKB representation in hex of a given Geometry."}, @@ -744,7 +841,7 @@ var geoBuiltins = map[string]builtinDefinition{ ), geographyOverload1( func(_ *tree.EvalContext, g *tree.DGeography) (tree.Datum, error) { - return tree.NewDString(strings.ToUpper(fmt.Sprintf("%x", g.EWKB()))), nil + return tree.NewDString(g.Geography.EWKBHex()), nil }, types.String, infoBuilder{info: "Returns the EWKB representation in hex of a given Geography."}, From 8677bc019174c1fd062036544c445a1cb5cf2c04 Mon Sep 17 00:00:00 2001 From: Andrii Vorobiov Date: Fri, 17 Apr 2020 13:12:18 +0300 Subject: [PATCH 32/46] ui: CSS modules for Statements page filters This change refactors following components to use CSS modules instead of styles defined as global: - Dropdown - Search - PageConfig Release note: None --- pkg/ui/.storybook/decorators/index.ts | 1 + .../.storybook/decorators/withBackground.tsx | 18 ++++++++ .../src/views/app/components/Search/index.tsx | 23 +++++++--- .../{search.styl => search.module.styl} | 11 +++-- .../app/components/Search/search.stories.tsx | 22 +++++++++ .../views/cluster/components/range/index.tsx | 2 +- .../views/cluster/components/range/range.styl | 14 ++++++ .../containers/network/filter/index.tsx | 4 +- .../reports/containers/network/sort/index.tsx | 9 +++- .../reports/containers/network/sort/sort.styl | 6 +-- .../{dropdown.styl => dropdown.module.styl} | 14 +++--- .../shared/components/dropdown/index.tsx | 44 +++++++++++++----- .../shared/components/pageconfig/index.tsx | 11 +++-- .../pageconfig/pageConfig.module.styl | 46 +++++++++++++++++++ .../statements/statementsPage.stories.tsx | 3 +- 15 files changed, 187 insertions(+), 41 deletions(-) create mode 100644 pkg/ui/.storybook/decorators/withBackground.tsx rename pkg/ui/src/views/app/components/Search/{search.styl => search.module.styl} (89%) create mode 100644 pkg/ui/src/views/app/components/Search/search.stories.tsx rename pkg/ui/src/views/shared/components/dropdown/{dropdown.styl => dropdown.module.styl} (93%) create mode 100644 pkg/ui/src/views/shared/components/pageconfig/pageConfig.module.styl diff --git a/pkg/ui/.storybook/decorators/index.ts b/pkg/ui/.storybook/decorators/index.ts index b6636d1c4f94..1ce42bbb90dd 100644 --- a/pkg/ui/.storybook/decorators/index.ts +++ b/pkg/ui/.storybook/decorators/index.ts @@ -9,3 +9,4 @@ // licenses/APL.txt. export * from "./withRouterProvider"; +export * from "./withBackground"; diff --git a/pkg/ui/.storybook/decorators/withBackground.tsx b/pkg/ui/.storybook/decorators/withBackground.tsx new file mode 100644 index 000000000000..d3099e2b5d41 --- /dev/null +++ b/pkg/ui/.storybook/decorators/withBackground.tsx @@ -0,0 +1,18 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React from "react"; +import {RenderFunction} from "storybook__react"; + +export const withBackgroundFactory = (backgroundColor = "#F5F7FA") => (storyFn: RenderFunction) => ( +
+ {storyFn()} +
+); diff --git a/pkg/ui/src/views/app/components/Search/index.tsx b/pkg/ui/src/views/app/components/Search/index.tsx index 8a8fb790664f..824d529361d8 100644 --- a/pkg/ui/src/views/app/components/Search/index.tsx +++ b/pkg/ui/src/views/app/components/Search/index.tsx @@ -13,7 +13,8 @@ import { InputProps } from "antd/lib/input"; import CancelIcon from "assets/cancel.svg"; import SearchIcon from "assets/search.svg"; import React from "react"; -import "./search.styl"; +import classNames from "classnames/bind"; +import styles from "./search.module.styl"; interface ISearchProps { onSubmit: (value: string) => void; @@ -29,6 +30,8 @@ interface ISearchState { type TSearchProps = ISearchProps & InputProps; +const cx = classNames.bind(styles); + export class Search extends React.Component { state = { value: this.props.defaultValue || "", @@ -62,16 +65,24 @@ export class Search extends React.Component { const { value, submitted } = this.state; if (value.length > 0) { if (submitted) { - return ; + return ( + + ); } - return ; + return ; } return null; } render() { const { value, submitted } = this.state; - const className = submitted ? "_submitted" : ""; + const className = submitted ? cx("_submitted") : ""; /* current antdesign (3.25.3) has Input.d.ts incompatible with latest @types/react @@ -82,13 +93,13 @@ export class Search extends React.Component { const MyInput = Input as any; return ( -
+ } + prefix={} suffix={this.renderSuffix()} value={value} {...this.props} diff --git a/pkg/ui/src/views/app/components/Search/search.styl b/pkg/ui/src/views/app/components/Search/search.module.styl similarity index 89% rename from pkg/ui/src/views/app/components/Search/search.styl rename to pkg/ui/src/views/app/components/Search/search.module.styl index 584835a5f115..045af88d05ff 100644 --- a/pkg/ui/src/views/app/components/Search/search.styl +++ b/pkg/ui/src/views/app/components/Search/search.module.styl @@ -14,12 +14,13 @@ ._search-form width 280px height 40px - .ant-input-affix-wrapper + :global(.ant-input-affix-wrapper) + height 40px &:hover - .ant-input:not(.ant-input-disabled) + :global(.ant-input:not(.ant-input-disabled)) border-color $adminui-blue-1-base border-right-width 2px !important - .ant-btn + :global(.ant-btn) margin 0 padding 0 width auto @@ -44,7 +45,7 @@ line-height 0px !important &:hover color $adminui-grey-2 - .ant-input + :global(.ant-input) font-size 14px font-family $font-family--base color $adminui-grey-1 @@ -60,6 +61,6 @@ padding-left 35px padding-right 60px ._submitted - .ant-input + :global(.ant-input) &:not(:first-child) padding-right 40px diff --git a/pkg/ui/src/views/app/components/Search/search.stories.tsx b/pkg/ui/src/views/app/components/Search/search.stories.tsx new file mode 100644 index 000000000000..496f90dfeba3 --- /dev/null +++ b/pkg/ui/src/views/app/components/Search/search.stories.tsx @@ -0,0 +1,22 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import React from "react"; +import { storiesOf } from "@storybook/react"; + +import { Search } from "./index"; + +storiesOf("Search", module) + .add("empty", () => ( + {}} value="" /> + )) + .add("with search text", () => ( + {}} value="select * from" /> + )); diff --git a/pkg/ui/src/views/cluster/components/range/index.tsx b/pkg/ui/src/views/cluster/components/range/index.tsx index 3efbe9e73d8e..58befcc9b4f5 100644 --- a/pkg/ui/src/views/cluster/components/range/index.tsx +++ b/pkg/ui/src/views/cluster/components/range/index.tsx @@ -164,7 +164,7 @@ class RangeSelect extends React.Component { onClick={this.handleOptionButtonOnClick(option)} ghost > - {this.props.selected.title !== "Custom" && option.value === "Custom" ? "--" : option.timeLabel} + {this.props.selected.title !== "Custom" && option.value === "Custom" ? "--" : option.timeLabel} {option.value === "Custom" ? "Custom date range" : option.value} ) diff --git a/pkg/ui/src/views/cluster/components/range/range.styl b/pkg/ui/src/views/cluster/components/range/range.styl index f21d721f9884..95fef6685da9 100644 --- a/pkg/ui/src/views/cluster/components/range/range.styl +++ b/pkg/ui/src/views/cluster/components/range/range.styl @@ -230,3 +230,17 @@ border 1px solid $colors--neutral-5 &:hover background $background-color + +.range__range-title + display flex + justify-content center + align-items center + background $colors--neutral-3 + width 34px + text-align center + border-radius 3px + color $colors--neutral-7 + font-size 12px + line-height 24px + letter-spacing 0.1px + font-family $font-family--bold diff --git a/pkg/ui/src/views/reports/containers/network/filter/index.tsx b/pkg/ui/src/views/reports/containers/network/filter/index.tsx index 744c16cb3526..ff46acbc9d68 100644 --- a/pkg/ui/src/views/reports/containers/network/filter/index.tsx +++ b/pkg/ui/src/views/reports/containers/network/filter/index.tsx @@ -20,6 +20,7 @@ interface IFilterProps { deselectFilterByKey: (key: string) => void; sort: NetworkSort[]; filter: NetworkFilter; + dropDownClassName?: string; } interface IFilterState { @@ -106,6 +107,7 @@ export class Filter extends React.Component { render() { const { opened, width } = this.state; + const { dropDownClassName } = this.props; const containerLeft = this.rangeContainer.current ? this.rangeContainer.current.getBoundingClientRect().left : 0; const left = width >= (containerLeft + 240) ? 0 : width - (containerLeft + 240); return ( @@ -114,7 +116,7 @@ export class Filter extends React.Component { title="Filter" options={[]} selected="" - className={classNames({ "dropdown__focused": opened })} + className={classNames({ "dropdown__focused": opened }, dropDownClassName)} content={
this.setState({ opened: !opened })}/> diff --git a/pkg/ui/src/views/reports/containers/network/sort/index.tsx b/pkg/ui/src/views/reports/containers/network/sort/index.tsx index 9bbd678e63fd..635bc0fd1d89 100644 --- a/pkg/ui/src/views/reports/containers/network/sort/index.tsx +++ b/pkg/ui/src/views/reports/containers/network/sort/index.tsx @@ -68,8 +68,15 @@ class Sort extends React.Component { options={this.getSortValues(sort)} selected={this.pageView()} onChange={this.navigateTo} + className="Sort-latency__dropdown--spacing" + /> + - Collapse Nodes
diff --git a/pkg/ui/src/views/reports/containers/network/sort/sort.styl b/pkg/ui/src/views/reports/containers/network/sort/sort.styl index 229a826906f6..f8d9c306024f 100644 --- a/pkg/ui/src/views/reports/containers/network/sort/sort.styl +++ b/pkg/ui/src/views/reports/containers/network/sort/sort.styl @@ -12,6 +12,6 @@ display flex align-items center padding 0 24px - .dropdown - margin-right 24px - \ No newline at end of file + +.Sort-latency__dropdown--spacing + margin-right 24px diff --git a/pkg/ui/src/views/shared/components/dropdown/dropdown.styl b/pkg/ui/src/views/shared/components/dropdown/dropdown.module.styl similarity index 93% rename from pkg/ui/src/views/shared/components/dropdown/dropdown.styl rename to pkg/ui/src/views/shared/components/dropdown/dropdown.module.styl index 19bd8ef79e86..dc28e544d7ff 100644 --- a/pkg/ui/src/views/shared/components/dropdown/dropdown.styl +++ b/pkg/ui/src/views/shared/components/dropdown/dropdown.module.styl @@ -28,14 +28,14 @@ $dropdown-hover-color = darken($background-color, 2.5%) display flex align-items center - .Select + :global(.Select) position initial - .Select-menu-outer + :global(.Select-menu-outer) top calc(100% + 8px) padding 8px 0 - .Select-option + :global(.Select-option) font-size 14px line-height 22px font-family $font-family--base @@ -45,7 +45,7 @@ $dropdown-hover-color = darken($background-color, 2.5%) color $colors--primary-blue-3 !important background-color transparent - .dropdown__title, .Select-value-label + .dropdown__title, :global(.Select-value-label) color $adminui-grey-1 !important font-family SourceSansPro-SemiBold font-size 14px @@ -53,7 +53,7 @@ $dropdown-hover-color = darken($background-color, 2.5%) letter-spacing 0.1px &:hover - .Select-arrow-zone + :global(.Select-arrow-zone) path fill $colors--neutral-5 @@ -74,7 +74,7 @@ $dropdown-hover-color = darken($background-color, 2.5%) border 1px solid $colors--primary-blue-3 box-shadow 0px 0px 3px 2px $colors--primary-blue-1 - .Select-arrow-zone + :global(.Select-arrow-zone) color $adminui-blue-1-base .caret-down display flex @@ -163,7 +163,7 @@ $dropdown-hover-color = darken($background-color, 2.5%) &:hover background-color $dropdown-hover-color .dropdown.full-size - .Select-menu-outer, .Select-menu + :global(.Select-menu-outer), :global(.Select-menu) max-height 450px // NOTE: react-select styles can be found in styl/shame.styl diff --git a/pkg/ui/src/views/shared/components/dropdown/index.tsx b/pkg/ui/src/views/shared/components/dropdown/index.tsx index 65658fc54d45..70d98d25efd3 100644 --- a/pkg/ui/src/views/shared/components/dropdown/index.tsx +++ b/pkg/ui/src/views/shared/components/dropdown/index.tsx @@ -8,12 +8,12 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -import classNames from "classnames"; +import classNames from "classnames/bind"; import Select from "react-select"; import React from "react"; import _ from "lodash"; -import "./dropdown.styl"; +import styles from "./dropdown.module.styl"; import {leftArrow, rightArrow} from "src/views/shared/components/icons"; import { trustIcon } from "src/util/trust"; @@ -45,7 +45,17 @@ interface DropdownOwnProps { type?: "primary" | "secondary"; } -export const arrowRenderer = ({ isOpen }: { isOpen: boolean }) => ; +const cx = classNames.bind(styles); + +export const arrowRenderer = ({ isOpen }: { isOpen: boolean }) => + + + ; /** * Dropdown component that uses the URL query string for state. @@ -85,20 +95,27 @@ export default class Dropdown extends React.Component { render() { const { selected, options, onChange, onArrowClick, disabledArrows, content, isTimeRange, type = "secondary" } = this.props; - const className = classNames( + const className = cx( "dropdown", `dropdown--type-${type}`, - isTimeRange ? "_range" : "", - { "dropdown--side-arrows": !_.isNil(onArrowClick), "dropdown__focused": this.state.is_focused }, + { + "_range": isTimeRange, + "dropdown--side-arrows": !_.isNil(onArrowClick), + "dropdown__focused": this.state.is_focused, + }, this.props.className, ); - const leftClassName = classNames( + const leftClassName = cx( "dropdown__side-arrow", - { "dropdown__side-arrow--disabled": _.includes(disabledArrows, ArrowDirection.LEFT) }, + { + "dropdown__side-arrow--disabled": _.includes(disabledArrows, ArrowDirection.LEFT), + }, ); - const rightClassName = classNames( + const rightClassName = cx( "dropdown__side-arrow", - { "dropdown__side-arrow--disabled": _.includes(disabledArrows, ArrowDirection.RIGHT) }, + { + "dropdown__side-arrow--disabled": _.includes(disabledArrows, ArrowDirection.RIGHT), + }, ); return
@@ -109,12 +126,15 @@ export default class Dropdown extends React.Component { onClick={() => this.props.onArrowClick(ArrowDirection.LEFT)}> {this.props.title}{this.props.title && !isTimeRange ? ":" : ""} {content ? content :