From 60d69870060952a0f411d625de73a942b27d0eaf Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 12 Sep 2023 09:49:19 -0400 Subject: [PATCH 1/3] errorutil: moved kv-specific errors out of errorutil Errors for missing store and node descriptors have been moved from the errorutil package to the kvpb package, because the errorutil package is a low-level package to aid in working with errors in general. It should not contain facilities for creating specific errors as this muddles the package and can lead to import cycles. Release note: None --- pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel | 1 - .../kvfollowerreadsccl/followerreads_test.go | 5 +-- pkg/gossip/gossip.go | 2 +- pkg/kv/kvclient/kvcoord/BUILD.bazel | 1 - pkg/kv/kvclient/kvcoord/replica_slice_test.go | 6 +-- pkg/kv/kvclient/kvtenant/BUILD.bazel | 1 - pkg/kv/kvclient/kvtenant/connector.go | 5 +-- pkg/kv/kvclient/kvtenant/connector_test.go | 6 +-- pkg/kv/kvpb/BUILD.bazel | 1 + pkg/kv/kvpb/errors.go | 41 +++++++++++++++++++ pkg/kv/kvserver/BUILD.bazel | 1 - pkg/kv/kvserver/store_test.go | 3 +- pkg/sql/crdb_internal.go | 2 +- pkg/sql/crdb_internal_ranges_deprecated.go | 5 ++- pkg/util/errorutil/BUILD.bazel | 3 +- pkg/util/errorutil/descriptor.go | 38 ----------------- pkg/util/errorutil/sentinel.go | 15 +++++++ 17 files changed, 74 insertions(+), 62 deletions(-) delete mode 100644 pkg/util/errorutil/descriptor.go create mode 100644 pkg/util/errorutil/sentinel.go diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel b/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel index bcecb833c133..d655ac1ff45a 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/BUILD.bazel @@ -74,7 +74,6 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util", - "//pkg/util/errorutil", "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/log", diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index c5ed58701ec5..d8fae81c711c 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -40,7 +40,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -497,7 +496,7 @@ func (s mockNodeStore) GetNodeDescriptor(id roachpb.NodeID) (*roachpb.NodeDescri return desc, nil } } - return nil, errorutil.NewNodeNotFoundError(id) + return nil, kvpb.NewNodeDescNotFoundError(id) } func (s mockNodeStore) GetNodeDescriptorCount() int { @@ -505,7 +504,7 @@ func (s mockNodeStore) GetNodeDescriptorCount() int { } func (s mockNodeStore) GetStoreDescriptor(id roachpb.StoreID) (*roachpb.StoreDescriptor, error) { - return nil, errorutil.NewStoreNotFoundError(id) + return nil, kvpb.NewStoreDescNotFoundError(id) } // TestOracle tests the Oracle exposed by this package. diff --git a/pkg/gossip/gossip.go b/pkg/gossip/gossip.go index 830330cf240d..7f05ed97210e 100644 --- a/pkg/gossip/gossip.go +++ b/pkg/gossip/gossip.go @@ -877,7 +877,7 @@ func (g *Gossip) getNodeDescriptor( return nodeDescriptor, nil } - return nil, errorutil.NewNodeNotFoundError(nodeID) + return nil, kvpb.NewNodeDescNotFoundError(nodeID) } // getNodeIDAddress looks up the address of the node by ID. The method accepts a diff --git a/pkg/kv/kvclient/kvcoord/BUILD.bazel b/pkg/kv/kvclient/kvcoord/BUILD.bazel index d90de5491fb7..921b057d11a8 100644 --- a/pkg/kv/kvclient/kvcoord/BUILD.bazel +++ b/pkg/kv/kvclient/kvcoord/BUILD.bazel @@ -206,7 +206,6 @@ go_test( "//pkg/util/caller", "//pkg/util/ctxgroup", "//pkg/util/encoding", - "//pkg/util/errorutil", "//pkg/util/grpcutil", "//pkg/util/hlc", "//pkg/util/leaktest", diff --git a/pkg/kv/kvclient/kvcoord/replica_slice_test.go b/pkg/kv/kvclient/kvcoord/replica_slice_test.go index 7d693d41dbab..1c3329871ce9 100644 --- a/pkg/kv/kvclient/kvcoord/replica_slice_test.go +++ b/pkg/kv/kvclient/kvcoord/replica_slice_test.go @@ -17,9 +17,9 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/shuffle" @@ -39,7 +39,7 @@ func (ns *mockNodeStore) GetNodeDescriptor(nodeID roachpb.NodeID) (*roachpb.Node return &nd, nil } } - return nil, errorutil.NewNodeNotFoundError(nodeID) + return nil, kvpb.NewNodeDescNotFoundError(nodeID) } // GetNodeDescriptorCount is part of the NodeDescStore interface. @@ -51,7 +51,7 @@ func (ns *mockNodeStore) GetNodeDescriptorCount() int { func (ns *mockNodeStore) GetStoreDescriptor( storeID roachpb.StoreID, ) (*roachpb.StoreDescriptor, error) { - return nil, errorutil.NewStoreNotFoundError(storeID) + return nil, kvpb.NewStoreDescNotFoundError(storeID) } func TestNewReplicaSlice(t *testing.T) { diff --git a/pkg/kv/kvclient/kvtenant/BUILD.bazel b/pkg/kv/kvclient/kvtenant/BUILD.bazel index 7e6b7933f4df..63c0919f80cd 100644 --- a/pkg/kv/kvclient/kvtenant/BUILD.bazel +++ b/pkg/kv/kvclient/kvtenant/BUILD.bazel @@ -33,7 +33,6 @@ go_library( "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/ts/tspb", - "//pkg/util/errorutil", "//pkg/util/grpcutil", "//pkg/util/hlc", "//pkg/util/log", diff --git a/pkg/kv/kvclient/kvtenant/connector.go b/pkg/kv/kvclient/kvtenant/connector.go index 88efb4d82bd9..d432efdbe3eb 100644 --- a/pkg/kv/kvclient/kvtenant/connector.go +++ b/pkg/kv/kvclient/kvtenant/connector.go @@ -39,7 +39,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/ts/tspb" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -517,7 +516,7 @@ func (c *connector) GetNodeDescriptor(nodeID roachpb.NodeID) (*roachpb.NodeDescr defer c.mu.RUnlock() desc, ok := c.mu.nodeDescs[nodeID] if !ok { - return nil, errorutil.NewNodeNotFoundError(nodeID) + return nil, kvpb.NewNodeDescNotFoundError(nodeID) } return desc, nil } @@ -535,7 +534,7 @@ func (c *connector) GetStoreDescriptor(storeID roachpb.StoreID) (*roachpb.StoreD defer c.mu.RUnlock() desc, ok := c.mu.storeDescs[storeID] if !ok { - return nil, errorutil.NewStoreNotFoundError(storeID) + return nil, kvpb.NewStoreDescNotFoundError(storeID) } return desc, nil } diff --git a/pkg/kv/kvclient/kvtenant/connector_test.go b/pkg/kv/kvclient/kvtenant/connector_test.go index 5391b13c895a..54d159384ca8 100644 --- a/pkg/kv/kvclient/kvtenant/connector_test.go +++ b/pkg/kv/kvclient/kvtenant/connector_test.go @@ -321,7 +321,7 @@ func TestConnectorGossipSubscription(t *testing.T) { require.NoError(t, err) desc, err = c.GetNodeDescriptor(3) require.Nil(t, desc) - require.Regexp(t, "unable to look up descriptor for n3", err) + require.Regexp(t, "node descriptor with node ID 3 was not found", err) // Test GetStoreDescriptor. storeID1 := roachpb.StoreID(1) @@ -340,7 +340,7 @@ func TestConnectorGossipSubscription(t *testing.T) { require.Equal(t, store2, storeDesc) storeDesc, err = c.GetStoreDescriptor(3) require.Nil(t, storeDesc) - require.Regexp(t, "unable to look up descriptor for store ID 3", err) + require.Regexp(t, "store descriptor with store ID 3 was not found", err) // Return updated GossipSubscription response. node1Up := &roachpb.NodeDescriptor{NodeID: 1, Address: util.MakeUnresolvedAddr("tcp", "1.2.3.4")} @@ -567,7 +567,7 @@ func TestConnectorRetriesUnreachable(t *testing.T) { require.NoError(t, err) desc, err = c.GetNodeDescriptor(3) require.Nil(t, desc) - require.Regexp(t, "unable to look up descriptor for n3", err) + require.Regexp(t, "node descriptor with node ID 3 was not found", err) } // TestConnectorRetriesError tests that connector iterates over each of diff --git a/pkg/kv/kvpb/BUILD.bazel b/pkg/kv/kvpb/BUILD.bazel index 1c71ca5e63ec..38df21dee212 100644 --- a/pkg/kv/kvpb/BUILD.bazel +++ b/pkg/kv/kvpb/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "//pkg/util/admission/admissionpb", "//pkg/util/buildutil", "//pkg/util/caller", + "//pkg/util/errorutil", "//pkg/util/hlc", "//pkg/util/humanizeutil", "//pkg/util/log", diff --git a/pkg/kv/kvpb/errors.go b/pkg/kv/kvpb/errors.go index 798c5c9439fe..ea5dbd197c16 100644 --- a/pkg/kv/kvpb/errors.go +++ b/pkg/kv/kvpb/errors.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/caller" + "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -1627,10 +1628,50 @@ func (e *MissingRecordError) SafeFormatError(p errors.Printer) (next error) { return nil } +// DescNotFoundError is reported when a descriptor is missing. +type DescNotFoundError struct { + storeID roachpb.StoreID + nodeID roachpb.NodeID + isStore bool +} + +// NewStoreDescNotFoundError initializes a new DescNotFoundError for a missing +// store descriptor. +func NewStoreDescNotFoundError(storeID roachpb.StoreID) *DescNotFoundError { + return &DescNotFoundError{ + storeID: storeID, + isStore: true, + } +} + +// NewNodeDescNotFoundError initializes a new DescNotFoundError for a missing +// node descriptor. +func NewNodeDescNotFoundError(nodeID roachpb.NodeID) *DescNotFoundError { + return &DescNotFoundError{ + nodeID: nodeID, + isStore: false, + } +} + +func (e *DescNotFoundError) Error() string { + return redact.Sprint(e).StripMarkers() +} + +func (e *DescNotFoundError) SafeFormatError(p errors.Printer) (next error) { + if e.isStore { + p.Printf("store descriptor with store ID %d was not found", e.storeID) + } else { + p.Printf("node descriptor with node ID %d was not found", e.nodeID) + } + return nil +} + func init() { errors.RegisterLeafDecoder(errors.GetTypeKey((*MissingRecordError)(nil)), func(_ context.Context, _ string, _ []string, _ proto.Message) error { return &MissingRecordError{} }) + errorutilpath := reflect.TypeOf(errorutil.TempSentinel{}).PkgPath() + errors.RegisterTypeMigration(errorutilpath, "*errorutil.descriptorNotFound", &DescNotFoundError{}) } var _ errors.SafeFormatter = &MissingRecordError{} diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 5f1692e28ee0..463e28c55450 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -480,7 +480,6 @@ go_test( "//pkg/util/circuit", "//pkg/util/ctxgroup", "//pkg/util/encoding", - "//pkg/util/errorutil", "//pkg/util/future", "//pkg/util/grunning", "//pkg/util/hlc", diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 3783475b0e7c..cdf2b1d808d2 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -60,7 +60,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/gossiputil" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -155,7 +154,7 @@ func (m mockNodeStore) GetNodeDescriptorCount() int { } func (m mockNodeStore) GetStoreDescriptor(id roachpb.StoreID) (*roachpb.StoreDescriptor, error) { - return nil, errorutil.NewStoreNotFoundError(id) + return nil, kvpb.NewStoreDescNotFoundError(id) } type dummyFirstRangeProvider struct { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 553fc6adcc13..13eac6c4f8ad 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -4530,7 +4530,7 @@ CREATE TABLE crdb_internal.ranges_no_leases ( replicaLocalityDatum := tree.DNull nodeDesc, err := p.ExecCfg().NodeDescs.GetNodeDescriptor(replica.NodeID) if err != nil { - if !errorutil.IsDescriptorNotFoundError(err) { + if !errors.Is(err, &kvpb.DescNotFoundError{}) { return nil, err } } else { diff --git a/pkg/sql/crdb_internal_ranges_deprecated.go b/pkg/sql/crdb_internal_ranges_deprecated.go index 956a9c112e6d..dc16e4307f75 100644 --- a/pkg/sql/crdb_internal_ranges_deprecated.go +++ b/pkg/sql/crdb_internal_ranges_deprecated.go @@ -15,6 +15,7 @@ import ( "sort" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -24,8 +25,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/errors" ) // crdbInternalRangesViewDEPRECATED is the pre-v23.1 @@ -200,7 +201,7 @@ CREATE TABLE crdb_internal.ranges_no_leases ( replicaLocalityDatum := tree.DNull nodeDesc, err := p.ExecCfg().NodeDescs.GetNodeDescriptor(replica.NodeID) if err != nil { - if !errorutil.IsDescriptorNotFoundError(err) { + if !errors.Is(err, &kvpb.DescNotFoundError{}) { return nil, err } } else { diff --git a/pkg/util/errorutil/BUILD.bazel b/pkg/util/errorutil/BUILD.bazel index 3e68201a21b8..0039476c297f 100644 --- a/pkg/util/errorutil/BUILD.bazel +++ b/pkg/util/errorutil/BUILD.bazel @@ -4,15 +4,14 @@ go_library( name = "errorutil", srcs = [ "catch.go", - "descriptor.go", "error.go", + "sentinel.go", "tenant.go", "tenant_deprecated_wrapper.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/util/errorutil", visibility = ["//visibility:public"], deps = [ - "//pkg/roachpb", "//pkg/settings", "//pkg/util/errorutil/unimplemented", "//pkg/util/log/logcrash", diff --git a/pkg/util/errorutil/descriptor.go b/pkg/util/errorutil/descriptor.go deleted file mode 100644 index 734e13019d4e..000000000000 --- a/pkg/util/errorutil/descriptor.go +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright 2019 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 errorutil - -import ( - "fmt" - - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/errors" -) - -type descriptorNotFound struct { - msg string -} - -func (e *descriptorNotFound) Error() string { - return e.msg -} - -func IsDescriptorNotFoundError(err error) bool { - return errors.HasType(err, (*descriptorNotFound)(nil)) -} - -func NewNodeNotFoundError(nodeID roachpb.NodeID) error { - return &descriptorNotFound{fmt.Sprintf("unable to look up descriptor for n%d", nodeID)} -} - -func NewStoreNotFoundError(storeID roachpb.StoreID) error { - return &descriptorNotFound{fmt.Sprintf("unable to look up descriptor for store ID %d", storeID)} -} diff --git a/pkg/util/errorutil/sentinel.go b/pkg/util/errorutil/sentinel.go new file mode 100644 index 000000000000..4b46814c7268 --- /dev/null +++ b/pkg/util/errorutil/sentinel.go @@ -0,0 +1,15 @@ +// Copyright 2023 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 errorutil + +// TempSentinel is a sentinel type that allows other packages to retrieve the +// path to this package with reflect and PkgPath. +type TempSentinel struct{} From 9e751effab6d61bb04c1ff8d53bf210081a42993 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 12 Sep 2023 09:58:11 -0400 Subject: [PATCH 2/3] errorutil: move SendReport to new sentryutil package The `SendReport` function has been moved out of the errorutil package and into a new sentryutil package. This avoids muddling the errorutil package with a Sentry-specific function, and it breaks errorutil's dependence on `pkg/settings` and `pkg/util/log/logcrash`. Release note: None --- pkg/BUILD.bazel | 1 + pkg/sql/BUILD.bazel | 1 + pkg/sql/conn_executor.go | 3 ++- pkg/sql/opt/norm/BUILD.bazel | 1 + pkg/sql/opt/norm/factory.go | 3 ++- pkg/sql/stats/BUILD.bazel | 1 + pkg/sql/stats/forecast.go | 3 ++- pkg/util/errorutil/BUILD.bazel | 2 -- pkg/util/errorutil/error.go | 14 -------------- pkg/util/sentryutil/BUILD.bazel | 13 +++++++++++++ pkg/util/sentryutil/sentry.go | 30 ++++++++++++++++++++++++++++++ 11 files changed, 53 insertions(+), 19 deletions(-) create mode 100644 pkg/util/sentryutil/BUILD.bazel create mode 100644 pkg/util/sentryutil/sentry.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 058bdb1cd73a..52b19b4455e8 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -2419,6 +2419,7 @@ GO_TARGETS = [ "//pkg/util/sdnotify:sdnotify_test", "//pkg/util/search:search", "//pkg/util/search:search_test", + "//pkg/util/sentryutil:sentryutil", "//pkg/util/shuffle:shuffle", "//pkg/util/shuffle:shuffle_test", "//pkg/util/slidingwindow:slidingwindow", diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index de7cb6f6f44d..70e91c05f24e 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -564,6 +564,7 @@ go_library( "//pkg/util/rangedesc", "//pkg/util/retry", "//pkg/util/ring", + "//pkg/util/sentryutil", "//pkg/util/startup", "//pkg/util/stop", "//pkg/util/syncutil", diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 4a0372114db0..ddf30eef7af6 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -80,6 +80,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log/severity" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/cockroach/pkg/util/sentryutil" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -3792,7 +3793,7 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper( errors.Safe(advInfo.txnEvent.eventType.String()), res.Err()) log.Errorf(ex.Ctx(), "%v", err) - errorutil.SendReport(ex.Ctx(), &ex.server.cfg.Settings.SV, err) + sentryutil.SendReport(ex.Ctx(), &ex.server.cfg.Settings.SV, err) return advanceInfo{}, err } diff --git a/pkg/sql/opt/norm/BUILD.bazel b/pkg/sql/opt/norm/BUILD.bazel index f5cc59b47bad..5cf9b2c64ab4 100644 --- a/pkg/sql/opt/norm/BUILD.bazel +++ b/pkg/sql/opt/norm/BUILD.bazel @@ -57,6 +57,7 @@ go_library( "//pkg/util/errorutil", "//pkg/util/intsets", "//pkg/util/json", + "//pkg/util/sentryutil", "@com_github_cockroachdb_apd_v3//:apd", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", diff --git a/pkg/sql/opt/norm/factory.go b/pkg/sql/opt/norm/factory.go index fa636e130935..a92f82b998a4 100644 --- a/pkg/sql/opt/norm/factory.go +++ b/pkg/sql/opt/norm/factory.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/intsets" + "github.com/cockroachdb/cockroach/pkg/util/sentryutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -411,7 +412,7 @@ func (f *Factory) onMaxConstructorStackDepthExceeded() { if buildutil.CrdbTestBuild { panic(err) } - errorutil.SendReport(f.ctx, &f.evalCtx.Settings.SV, err) + sentryutil.SendReport(f.ctx, &f.evalCtx.Settings.SV, err) } // onConstructRelational is called as a final step by each factory method that diff --git a/pkg/sql/stats/BUILD.bazel b/pkg/sql/stats/BUILD.bazel index 8fa7dc7c0338..a269c542ea2a 100644 --- a/pkg/sql/stats/BUILD.bazel +++ b/pkg/sql/stats/BUILD.bazel @@ -55,6 +55,7 @@ go_library( "//pkg/util/log/logcrash", "//pkg/util/mon", "//pkg/util/protoutil", + "//pkg/util/sentryutil", "//pkg/util/stop", "//pkg/util/syncutil", "//pkg/util/timeutil", diff --git a/pkg/sql/stats/forecast.go b/pkg/sql/stats/forecast.go index 32e7f226cb33..7d3b15e2dbd9 100644 --- a/pkg/sql/stats/forecast.go +++ b/pkg/sql/stats/forecast.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" + "github.com/cockroachdb/cockroach/pkg/util/sentryutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -343,7 +344,7 @@ func forecastColumnStatistics( "forecasted histogram had first bucket with non-zero NumRange or DistinctRange: %s", debugging, ) - errorutil.SendReport(ctx, sv, err) + sentryutil.SendReport(ctx, sv, err) return nil, err } if bucket.UpperBound != tree.DNull { diff --git a/pkg/util/errorutil/BUILD.bazel b/pkg/util/errorutil/BUILD.bazel index 0039476c297f..9d4db9556cf1 100644 --- a/pkg/util/errorutil/BUILD.bazel +++ b/pkg/util/errorutil/BUILD.bazel @@ -12,9 +12,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/util/errorutil", visibility = ["//visibility:public"], deps = [ - "//pkg/settings", "//pkg/util/errorutil/unimplemented", - "//pkg/util/log/logcrash", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/util/errorutil/error.go b/pkg/util/errorutil/error.go index 48c8618bcb40..11f68f38c185 100644 --- a/pkg/util/errorutil/error.go +++ b/pkg/util/errorutil/error.go @@ -11,11 +11,8 @@ package errorutil import ( - "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/errors" ) @@ -37,14 +34,3 @@ func UnexpectedWithIssueErrorf(issue int, format string, args ...interface{}) er issue)) return err } - -// SendReport creates a Sentry report about the error, if the settings allow. -// The format string will be reproduced ad litteram in the report; the arguments -// will be sanitized. -func SendReport(ctx context.Context, sv *settings.Values, err error) { - if !logcrash.ShouldSendReport(sv) { - return - } - event, extraDetails := errors.BuildSentryReport(err) - logcrash.SendReport(ctx, logcrash.ReportTypeError, event, extraDetails) -} diff --git a/pkg/util/sentryutil/BUILD.bazel b/pkg/util/sentryutil/BUILD.bazel new file mode 100644 index 000000000000..730451ff50ad --- /dev/null +++ b/pkg/util/sentryutil/BUILD.bazel @@ -0,0 +1,13 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "sentryutil", + srcs = ["sentry.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/util/sentryutil", + visibility = ["//visibility:public"], + deps = [ + "//pkg/settings", + "//pkg/util/log/logcrash", + "@com_github_cockroachdb_errors//:errors", + ], +) diff --git a/pkg/util/sentryutil/sentry.go b/pkg/util/sentryutil/sentry.go new file mode 100644 index 000000000000..c7ceccc0327a --- /dev/null +++ b/pkg/util/sentryutil/sentry.go @@ -0,0 +1,30 @@ +// Copyright 2023 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 sentryutil + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" + "github.com/cockroachdb/errors" +) + +// SendReport creates a Sentry report about the error, if the settings allow. +// The format string will be reproduced ad litteram in the report; the arguments +// will be sanitized. +func SendReport(ctx context.Context, sv *settings.Values, err error) { + if !logcrash.ShouldSendReport(sv) { + return + } + event, extraDetails := errors.BuildSentryReport(err) + logcrash.SendReport(ctx, logcrash.ReportTypeError, event, extraDetails) +} From e8d3de261ac24daca00121b7a077488407498e13 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Mon, 11 Sep 2023 15:59:32 -0400 Subject: [PATCH 3/3] util/pretty: mitigate stack overflows of Pretty This commit reduces the chance of a stack overflow from recursive calls of `*beExec.be`. The `Pretty` function will now return an internal error if the recursive depth of `be` surpasses 10,000. Informs #91197 Release note: None --- pkg/cli/sqlfmt.go | 6 ++- pkg/cmd/reduce/reduce/reducesql/reducesql.go | 12 ++++- pkg/internal/sqlsmith/setup_test.go | 5 +- pkg/internal/sqlsmith/sqlsmith.go | 7 ++- pkg/kv/kvpb/errors.go | 13 +++--- pkg/sql/BUILD.bazel | 1 + pkg/sql/explain_bundle.go | 32 ++++++++++--- pkg/sql/logictest/logic.go | 6 ++- pkg/sql/opt/optgen/cmd/optfmt/main.go | 5 +- pkg/sql/sem/builtins/BUILD.bazel | 1 + pkg/sql/sem/builtins/builtins.go | 11 ++++- pkg/sql/sem/tree/pretty.go | 4 +- pkg/sql/sem/tree/pretty_test.go | 48 ++++++++++++++++++-- pkg/sql/show_create_clauses.go | 29 ++++++++---- pkg/testutils/sqlutils/pretty.go | 5 +- pkg/util/pretty/BUILD.bazel | 4 ++ pkg/util/pretty/pretty.go | 43 +++++++++++++++++- pkg/util/pretty/pretty_test.go | 10 +++- 18 files changed, 203 insertions(+), 39 deletions(-) diff --git a/pkg/cli/sqlfmt.go b/pkg/cli/sqlfmt.go index bf8e7e7f14df..b5aa31425f81 100644 --- a/pkg/cli/sqlfmt.go +++ b/pkg/cli/sqlfmt.go @@ -76,7 +76,11 @@ func runSQLFmt(cmd *cobra.Command, args []string) error { } for i := range sl { - fmt.Print(cfg.Pretty(sl[i].AST)) + p, err := cfg.Pretty(sl[i].AST) + if err != nil { + return err + } + fmt.Print(p) if len(sl) > 1 { fmt.Print(";") } diff --git a/pkg/cmd/reduce/reduce/reducesql/reducesql.go b/pkg/cmd/reduce/reduce/reducesql/reducesql.go index 690a9d713b79..257eba640e9c 100644 --- a/pkg/cmd/reduce/reduce/reducesql/reducesql.go +++ b/pkg/cmd/reduce/reduce/reducesql/reducesql.go @@ -427,6 +427,7 @@ func collectASTs(stmts statements.Statements) []tree.NodeFormatter { func joinASTs(stmts []tree.NodeFormatter) string { var sb strings.Builder + var fmtCtx *tree.FmtCtx for i, stmt := range stmts { if i > 0 { sb.WriteString("\n\n") @@ -438,7 +439,16 @@ func joinASTs(stmts []tree.NodeFormatter) string { UseTabs: false, Simplify: true, } - sb.WriteString(cfg.Pretty(stmt)) + p, err := cfg.Pretty(stmt) + if err != nil { + // Use simple printing if pretty-printing fails. + if fmtCtx == nil { + fmtCtx = tree.NewFmtCtx(tree.FmtParsable) + } + stmt.Format(fmtCtx) + p = fmtCtx.CloseAndGetString() + } + sb.WriteString(p) sb.WriteString(";") } return sb.String() diff --git a/pkg/internal/sqlsmith/setup_test.go b/pkg/internal/sqlsmith/setup_test.go index a440af48df0f..47bc35562193 100644 --- a/pkg/internal/sqlsmith/setup_test.go +++ b/pkg/internal/sqlsmith/setup_test.go @@ -115,7 +115,10 @@ func TestGenerateParse(t *testing.T) { if err != nil { t.Fatalf("%v: %v", stmt, err) } - stmt = sqlsmith.TestingPrettyCfg.Pretty(parsed.AST) + stmt, err = sqlsmith.TestingPrettyCfg.Pretty(parsed.AST) + if err != nil { + t.Fatal(err) + } fmt.Print("STMT: ", i, "\n", stmt, ";\n\n") if *flagExec { db.Exec(t, `SET statement_timeout = '9s'`) diff --git a/pkg/internal/sqlsmith/sqlsmith.go b/pkg/internal/sqlsmith/sqlsmith.go index ec12fed83a92..3dbd2c55256f 100644 --- a/pkg/internal/sqlsmith/sqlsmith.go +++ b/pkg/internal/sqlsmith/sqlsmith.go @@ -190,7 +190,12 @@ func (s *Smither) Generate() string { continue } i = 0 - return prettyCfg.Pretty(stmt) + p, err := prettyCfg.Pretty(stmt) + if err != nil { + // Use simple printing if pretty-printing fails. + p = tree.AsStringWithFlags(stmt, tree.FmtParsable) + } + return p } } diff --git a/pkg/kv/kvpb/errors.go b/pkg/kv/kvpb/errors.go index ea5dbd197c16..a141eaca21aa 100644 --- a/pkg/kv/kvpb/errors.go +++ b/pkg/kv/kvpb/errors.go @@ -1630,8 +1630,7 @@ func (e *MissingRecordError) SafeFormatError(p errors.Printer) (next error) { // DescNotFoundError is reported when a descriptor is missing. type DescNotFoundError struct { - storeID roachpb.StoreID - nodeID roachpb.NodeID + id int32 isStore bool } @@ -1639,7 +1638,7 @@ type DescNotFoundError struct { // store descriptor. func NewStoreDescNotFoundError(storeID roachpb.StoreID) *DescNotFoundError { return &DescNotFoundError{ - storeID: storeID, + id: int32(storeID), isStore: true, } } @@ -1648,7 +1647,7 @@ func NewStoreDescNotFoundError(storeID roachpb.StoreID) *DescNotFoundError { // node descriptor. func NewNodeDescNotFoundError(nodeID roachpb.NodeID) *DescNotFoundError { return &DescNotFoundError{ - nodeID: nodeID, + id: int32(nodeID), isStore: false, } } @@ -1658,11 +1657,11 @@ func (e *DescNotFoundError) Error() string { } func (e *DescNotFoundError) SafeFormatError(p errors.Printer) (next error) { + s := redact.SafeString("node") if e.isStore { - p.Printf("store descriptor with store ID %d was not found", e.storeID) - } else { - p.Printf("node descriptor with node ID %d was not found", e.nodeID) + s = "store" } + p.Printf("%s descriptor with %s ID %d was not found", s, s, e.id) return nil } diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 70e91c05f24e..b5194b30431a 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -558,6 +558,7 @@ go_library( "//pkg/util/metric", "//pkg/util/mon", "//pkg/util/optional", + "//pkg/util/pretty", "//pkg/util/protoutil", "//pkg/util/quotapool", "//pkg/util/randutil", diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index 301413552a7e..cc96e8a4a880 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/memzipper" + "github.com/cockroachdb/cockroach/pkg/util/pretty" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" "github.com/cockroachdb/errors" @@ -148,7 +149,10 @@ func buildStatementBundle( if plan == nil { return diagnosticsBundle{collectionErr: errors.AssertionFailedf("execution terminated early")} } - b := makeStmtBundleBuilder(explainFlags, db, ie, stmtRawSQL, plan, trace, placeholders, sv) + b, err := makeStmtBundleBuilder(explainFlags, db, ie, stmtRawSQL, plan, trace, placeholders, sv) + if err != nil { + return diagnosticsBundle{collectionErr: err} + } b.addStatement() b.addOptPlans(ctx) @@ -226,18 +230,21 @@ func makeStmtBundleBuilder( trace tracingpb.Recording, placeholders *tree.PlaceholderInfo, sv *settings.Values, -) stmtBundleBuilder { +) (stmtBundleBuilder, error) { b := stmtBundleBuilder{ flags: flags, db: db, ie: ie, plan: plan, trace: trace, placeholders: placeholders, sv: sv, } - b.buildPrettyStatement(stmtRawSQL) + err := b.buildPrettyStatement(stmtRawSQL) + if err != nil { + return stmtBundleBuilder{}, err + } b.z.Init() - return b + return b, nil } // buildPrettyStatement saves the pretty-printed statement (without any // placeholder arguments). -func (b *stmtBundleBuilder) buildPrettyStatement(stmtRawSQL string) { +func (b *stmtBundleBuilder) buildPrettyStatement(stmtRawSQL string) error { // If we hit an early error, stmt or stmt.AST might not be initialized yet. In // this case use the original raw SQL. if b.plan.stmt == nil || b.plan.stmt.AST == nil { @@ -255,7 +262,19 @@ func (b *stmtBundleBuilder) buildPrettyStatement(stmtRawSQL string) { cfg.Align = tree.PrettyNoAlign cfg.JSONFmt = true cfg.ValueRedaction = b.flags.RedactValues - b.stmt = cfg.Pretty(b.plan.stmt.AST) + var err error + b.stmt, err = cfg.Pretty(b.plan.stmt.AST) + if errors.Is(err, pretty.ErrPrettyMaxRecursionDepthExceeded) { + // Use the raw statement string if pretty-printing fails. + b.stmt = stmtRawSQL + // If we're collecting a redacted bundle, redact the raw SQL + // completely. + if b.flags.RedactValues && b.stmt != "" { + b.stmt = string(redact.RedactedMarker()) + } + } else if err != nil { + return err + } // If we had ValueRedaction set, Pretty surrounded all constants with // redaction markers. We must call Redact to fully redact them. @@ -266,6 +285,7 @@ func (b *stmtBundleBuilder) buildPrettyStatement(stmtRawSQL string) { if b.stmt == "" { b.stmt = "-- no statement" } + return nil } // addStatement adds the pretty-printed statement in b.stmt as file diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 77d841476dab..60a8e869e55c 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -728,7 +728,11 @@ func (ls *logicStatement) readSQL( if i > 0 { fmt.Fprintln(&newSyntax, ";") } - fmt.Fprint(&newSyntax, pcfg.Pretty(stmtList[i].AST)) + p, err := pcfg.Pretty(stmtList[i].AST) + if err != nil { + return "", errors.Wrapf(err, "error while pretty printing") + } + fmt.Fprint(&newSyntax, p) } return newSyntax.String(), nil }(ls.sql) diff --git a/pkg/sql/opt/optgen/cmd/optfmt/main.go b/pkg/sql/opt/optgen/cmd/optfmt/main.go index ad4bfbcdf7ea..bd52d2b7a7c6 100644 --- a/pkg/sql/opt/optgen/cmd/optfmt/main.go +++ b/pkg/sql/opt/optgen/cmd/optfmt/main.go @@ -173,7 +173,10 @@ func prettyify(r io.Reader, n int, exprgen bool) (string, error) { exprs = parser.Exprs() } d := p.toDoc(exprs) - s := pretty.Pretty(d, n, false, 4, nil) + s, err := pretty.Pretty(d, n, false, 4, nil) + if err != nil { + return "", err + } // Remove any whitespace at EOL. This can happen in define rules where // we always insert a blank line above comments which are nested with diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index ffe2655a7721..cc28e74db577 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -116,6 +116,7 @@ go_library( "//pkg/util/json", "//pkg/util/log", "//pkg/util/mon", + "//pkg/util/pretty", "//pkg/util/protoutil", "//pkg/util/randident", "//pkg/util/randident/randidentcfg", diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 2a2bfead9b51..e82e42ff6eba 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -86,6 +86,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/pretty" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeofday" @@ -11157,7 +11158,15 @@ func prettyStatement(p tree.PrettyCfg, stmt string) (string, error) { } var formattedStmt strings.Builder for idx := range stmts { - formattedStmt.WriteString(p.Pretty(stmts[idx].AST)) + p, err := p.Pretty(stmts[idx].AST) + if errors.Is(err, pretty.ErrPrettyMaxRecursionDepthExceeded) { + // If pretty-printing the statement fails, use the original + // statement. + p = stmt + } else if err != nil { + return "", err + } + formattedStmt.WriteString(p) if len(stmts) > 1 { formattedStmt.WriteString(";") } diff --git a/pkg/sql/sem/tree/pretty.go b/pkg/sql/sem/tree/pretty.go index 21e210d22e19..7d1586b8c1c7 100644 --- a/pkg/sql/sem/tree/pretty.go +++ b/pkg/sql/sem/tree/pretty.go @@ -157,13 +157,13 @@ func (p *PrettyCfg) bracketKeyword( } // Pretty pretty prints stmt with default options. -func Pretty(stmt NodeFormatter) string { +func Pretty(stmt NodeFormatter) (string, error) { cfg := DefaultPrettyCfg() return cfg.Pretty(stmt) } // Pretty pretty prints stmt with specified options. -func (p *PrettyCfg) Pretty(stmt NodeFormatter) string { +func (p *PrettyCfg) Pretty(stmt NodeFormatter) (string, error) { doc := p.Doc(stmt) return pretty.Pretty(doc, p.LineWidth, p.UseTabs, p.TabWidth, p.Case) } diff --git a/pkg/sql/sem/tree/pretty_test.go b/pkg/sql/sem/tree/pretty_test.go index e51c353d7c1f..4aa46493e2ea 100644 --- a/pkg/sql/sem/tree/pretty_test.go +++ b/pkg/sql/sem/tree/pretty_test.go @@ -18,6 +18,7 @@ import ( "os" "path/filepath" "runtime" + "strconv" "strings" "testing" @@ -30,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/pretty" + "github.com/stretchr/testify/assert" "golang.org/x/sync/errgroup" ) @@ -112,7 +114,10 @@ func runTestPrettyData( for p := range work { thisCfg := cfg thisCfg.LineWidth = p.numCols - res[p.idx] = thisCfg.Pretty(stmt.AST) + res[p.idx], err = thisCfg.Pretty(stmt.AST) + if err != nil { + t.Fatal(err) + } } return nil } @@ -178,7 +183,10 @@ func TestPrettyVerify(t *testing.T) { if err != nil { t.Fatal(err) } - got := tree.Pretty(stmt.AST) + got, err := tree.Pretty(stmt.AST) + if err != nil { + t.Fatal(err) + } if pretty != got { t.Fatalf("got: %s\nexpected: %s", got, pretty) } @@ -186,6 +194,32 @@ func TestPrettyVerify(t *testing.T) { } } +func TestPrettyBigStatement(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Create a SELECT statement with a 1 million item IN expression. Without + // mitigation, this can cause stack overflows - see #91197. + var sb strings.Builder + sb.WriteString("SELECT * FROM foo WHERE id IN (") + for i := 0; i < 1_000_000; i++ { + if i != 0 { + sb.WriteByte(',') + } + sb.WriteString(strconv.Itoa(i)) + } + sb.WriteString(");") + + stmt, err := parser.ParseOne(sb.String()) + if err != nil { + t.Fatal(err) + } + + cfg := tree.DefaultPrettyCfg() + _, err = cfg.Pretty(stmt.AST) + assert.Errorf(t, err, "max call stack depth of be exceeded") +} + func BenchmarkPrettyData(b *testing.B) { matches, err := filepath.Glob(datapathutils.TestDataPath(b, "pretty", "*.sql")) if err != nil { @@ -209,7 +243,10 @@ func BenchmarkPrettyData(b *testing.B) { for i := 0; i < b.N; i++ { for _, doc := range docs { for _, w := range []int{1, 30, 80} { - pretty.Pretty(doc, w, true /*useTabs*/, 4 /*tabWidth*/, nil /* keywordTransform */) + _, err := pretty.Pretty(doc, w, true /*useTabs*/, 4 /*tabWidth*/, nil /* keywordTransform */) + if err != nil { + b.Fatal(err) + } } } } @@ -226,7 +263,10 @@ func TestPrettyExprs(t *testing.T) { } for expr, pretty := range tests { - got := tree.Pretty(expr) + got, err := tree.Pretty(expr) + if err != nil { + t.Fatal(err) + } if pretty != got { t.Fatalf("got: %s\nexpected: %s", got, pretty) } diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index 9b742de89780..06f46e79e521 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/pretty" "github.com/cockroachdb/errors" ) @@ -141,7 +142,10 @@ func ShowCreateView( cfg.UseTabs = true cfg.LineWidth = 100 - cfg.TabWidth cfg.ValueRedaction = redactableValues - q := formatViewQueryForDisplay(ctx, semaCtx, sessionData, desc, cfg) + q, err := formatViewQueryForDisplay(ctx, semaCtx, sessionData, desc, cfg) + if err != nil { + return "", err + } for i, line := range strings.Split(q, "\n") { if i > 0 { f.WriteString("\n\t") @@ -161,22 +165,31 @@ func formatViewQueryForDisplay( sessionData *sessiondata.SessionData, desc catalog.TableDescriptor, cfg tree.PrettyCfg, -) (query string) { +) (query string, err error) { defer func() { - parsed, err := parser.ParseOne(query) - if err != nil { + parsed, parseErr := parser.ParseOne(query) + if parseErr != nil { log.Warningf(ctx, "error parsing query for view %s (%v): %+v", desc.GetName(), desc.GetID(), err) return } - query = cfg.Pretty(parsed.AST) + var prettyErr error + query, prettyErr = cfg.Pretty(parsed.AST) + if errors.Is(prettyErr, pretty.ErrPrettyMaxRecursionDepthExceeded) { + // Use simple printing if pretty-printing fails. + query = tree.AsStringWithFlags(parsed.AST, tree.FmtParsable) + return + } else if prettyErr != nil { + err = prettyErr + return + } }() typeReplacedViewQuery, err := formatViewQueryTypesForDisplay(ctx, semaCtx, sessionData, desc) if err != nil { log.Warningf(ctx, "error deserializing user defined types for view %s (%v): %+v", desc.GetName(), desc.GetID(), err) - return desc.GetViewQuery() + return desc.GetViewQuery(), nil } // Convert sequences referenced by ID in the view back to their names. @@ -184,10 +197,10 @@ func formatViewQueryForDisplay( if err != nil { log.Warningf(ctx, "error converting sequence IDs to names for view %s (%v): %+v", desc.GetName(), desc.GetID(), err) - return typeReplacedViewQuery + return typeReplacedViewQuery, nil } - return sequenceReplacedViewQuery + return sequenceReplacedViewQuery, nil } // formatQuerySequencesForDisplay walks the view query and diff --git a/pkg/testutils/sqlutils/pretty.go b/pkg/testutils/sqlutils/pretty.go index 38351fa3330a..c6e075c795f5 100644 --- a/pkg/testutils/sqlutils/pretty.go +++ b/pkg/testutils/sqlutils/pretty.go @@ -66,7 +66,10 @@ func VerifyStatementPrettyRoundtrip(t *testing.T, sql string) { // origStmt := stmts[i].AST // Be careful to not simplify otherwise the tests won't round trip. - prettyStmt := cfg.Pretty(origStmt) + prettyStmt, err := cfg.Pretty(origStmt) + if err != nil { + t.Fatalf("%s: %s", err, prettyStmt) + } parsedPretty, err := parser.ParseOne(prettyStmt) if err != nil { t.Fatalf("%s: %s", err, prettyStmt) diff --git a/pkg/util/pretty/BUILD.bazel b/pkg/util/pretty/BUILD.bazel index a37f969901e1..07b8b6b6160a 100644 --- a/pkg/util/pretty/BUILD.bazel +++ b/pkg/util/pretty/BUILD.bazel @@ -9,6 +9,10 @@ go_library( ], importpath = "github.com/cockroachdb/cockroach/pkg/util/pretty", visibility = ["//visibility:public"], + deps = [ + "//pkg/util/errorutil", + "@com_github_cockroachdb_errors//:errors", + ], ) go_test( diff --git a/pkg/util/pretty/pretty.go b/pkg/util/pretty/pretty.go index 210223e3592b..838839fc9ac4 100644 --- a/pkg/util/pretty/pretty.go +++ b/pkg/util/pretty/pretty.go @@ -13,6 +13,9 @@ package pretty import ( "fmt" "strings" + + "github.com/cockroachdb/cockroach/pkg/util/errorutil" + "github.com/cockroachdb/errors" ) // See the referenced paper in the package documentation for explanations @@ -45,7 +48,25 @@ const ( // if not nil. keywordTransform must not change the visible length of its // argument. It can, for example, add invisible characters like control codes // (colors, etc.). -func Pretty(d Doc, n int, useTabs bool, tabWidth int, keywordTransform func(string) string) string { +func Pretty( + d Doc, n int, useTabs bool, tabWidth int, keywordTransform func(string) string, +) (_ string, err error) { + defer func() { + if r := recover(); r != nil { + // This code allows us to propagate internal errors without having + // to add error checks everywhere throughout the code. This is only + // possible because the code does not update shared state and does + // not manipulate locks. + if ok, e := errorutil.ShouldCatch(r); ok { + err = e + } else { + // Other panic objects can't be considered "safe" and thus are + // propagated as panics. + panic(r) + } + } + }() + var sb strings.Builder b := beExec{ w: int16(n), @@ -56,7 +77,7 @@ func Pretty(d Doc, n int, useTabs bool, tabWidth int, keywordTransform func(stri } ldoc := b.best(d) b.layout(&sb, useTabs, ldoc) - return sb.String() + return sb.String(), nil } // w is the max line width. @@ -103,9 +124,27 @@ type beExec struct { // keywordTransform filters keywords if not nil. keywordTransform func(string) string + + // beDepth is the depth of recursive calls of be. It is used to detect deep + // call stacks before a stack overflow occurs. + beDepth int } +// maxBeDepth is the maximum allowed recursive call depth of be. If the depth +// exceeds this value, be will panic. +const maxBeDepth = 50_000 + +// ErrPrettyMaxRecursionDepthExceeded is returned from Pretty when the maximum +// recursion depth of function invoked by Pretty is exceeded. +var ErrPrettyMaxRecursionDepthExceeded = errors.AssertionFailedf("max recursion depth exceeded") + func (b *beExec) be(k docPos, xlist *iDoc) *docBest { + b.beDepth++ + defer func() { b.beDepth-- }() + if b.beDepth > maxBeDepth { + panic(ErrPrettyMaxRecursionDepthExceeded) + } + // Shortcut: be k [] = Nil if xlist == nil { return nil diff --git a/pkg/util/pretty/pretty_test.go b/pkg/util/pretty/pretty_test.go index a21ad1b6b13f..729aff29c5a0 100644 --- a/pkg/util/pretty/pretty_test.go +++ b/pkg/util/pretty/pretty_test.go @@ -60,7 +60,10 @@ func Example_align() { for _, n := range []int{1, 15, 30, 80} { fmt.Printf("%d:\n", n) for _, doc := range testData { - p := pretty.Pretty(doc, n, true /*useTabs*/, 4 /*tabWidth*/, nil /*keywordTransform*/) + p, err := pretty.Pretty(doc, n, true /*useTabs*/, 4 /*tabWidth*/, nil /*keywordTransform*/) + if err != nil { + panic(err) + } fmt.Printf("%s\n\n", p) } } @@ -216,7 +219,10 @@ func Example_tree() { )) } for _, n := range []int{1, 30, 80} { - p := pretty.Pretty(showTree(tree), n, false /*useTabs*/, 4 /*tabWidth*/, nil /*keywordTransform*/) + p, err := pretty.Pretty(showTree(tree), n, false /*useTabs*/, 4 /*tabWidth*/, nil /*keywordTransform*/) + if err != nil { + panic(err) + } fmt.Printf("%d:\n%s\n\n", n, p) } // Output: